From edafd1617b27eff2eef812e46976930d75fd09e8 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Sat, 6 Jun 2020 15:03:50 -0700 Subject: [PATCH 01/24] Fill in the core partition set size properly for batch ingestion with dynamic partitioning --- .../input/impl/prefetch/JsonIterator.java | 4 +- .../common/parsers/CloseableIterator.java | 6 +- .../partition/BuildingNumberedShardSpec.java | 117 ++++++++ .../partition/NumberedOverwriteShardSpec.java | 20 +- .../partition/NumberedPartialShardSpec.java | 7 + .../timeline/partition/NumberedShardSpec.java | 5 + .../timeline/partition/PartialShardSpec.java | 5 +- .../druid/timeline/partition/ShardSpec.java | 25 +- .../timeline/partition/ShardSpecLookup.java | 4 + docs/configuration/index.md | 8 +- docs/ingestion/native-batch.md | 57 +++- .../druid/indexing/kafka/KafkaIndexTask.java | 6 + .../indexing/kafka/KafkaIndexTaskTest.java | 1 + .../indexing/kinesis/KinesisIndexTask.java | 6 + .../kinesis/KinesisIndexTaskTest.java | 2 +- .../indexing/common/task/AbstractTask.java | 6 + .../AppenderatorDriverRealtimeIndexTask.java | 6 + .../indexing/common/task/CompactionTask.java | 47 ++-- .../common/task/LocalSegmentAllocator.java | 4 +- .../common/task/RealtimeIndexTask.java | 6 + .../druid/indexing/common/task/Task.java | 5 + .../parallel/ParallelIndexSupervisorTask.java | 3 +- .../indexing/overlord/ForkingTaskRunner.java | 7 + ...penderatorDriverRealtimeIndexTaskTest.java | 1 + .../common/task/CompactionTaskTest.java | 29 +- .../indexing/common/task/IndexTaskTest.java | 2 + .../common/task/RealtimeIndexTaskTest.java | 6 + .../SinglePhaseParallelIndexingTest.java | 40 ++- .../segment/vector/NilVectorSelector.java | 2 +- .../segment/filter/ValueMatchersTest.java | 152 ++++++++++ server/pom.xml | 4 - .../druid/client/CachingClusteredClient.java | 4 +- .../org/apache/druid/client/DruidServer.java | 14 +- .../discovery/DruidNodeDiscoveryProvider.java | 3 +- .../SQLMetadataStorageActionHandler.java | 4 +- .../metadata/input/InputSourceModule.java | 51 ++++ .../druid/metadata/input/SqlEntity.java | 211 ++++++++++++++ .../druid/metadata/input/SqlInputFormat.java | 54 ++++ .../druid/metadata/input/SqlInputSource.java | 151 ++++++++++ .../druid/metadata/input/SqlReader.java | 95 +++++++ .../segment/loading/SegmentLoaderConfig.java | 5 +- .../SegmentLoaderLocalCacheManager.java | 1 - ...egmentServerAnnouncerLifecycleHandler.java | 104 ------- .../appenderator/SegmentPublisherHelper.java | 107 +++++++ .../TransactionalSegmentPublisher.java | 50 +--- .../realtime/firehose/SqlFirehoseFactory.java | 89 +----- ...oordinatorBasedSegmentHandoffNotifier.java | 2 +- .../coordination/DruidServerMetadata.java | 12 +- .../coordination/LoadableDataSegment.java | 81 ++++++ .../SegmentChangeRequestLoad.java | 14 +- .../coordination/SegmentLoadDropHandler.java | 28 +- .../druid/server/coordination/ServerType.java | 8 + .../server/coordinator/BalancerStrategy.java | 9 +- .../CachingCostBalancerStrategyFactory.java | 6 +- .../coordinator/CostBalancerStrategy.java | 8 +- .../server/coordinator/DruidCluster.java | 45 ++- .../server/coordinator/DruidCoordinator.java | 2 +- .../DruidCoordinatorRuntimeParams.java | 23 +- .../coordinator/RandomBalancerStrategy.java | 5 +- .../coordinator/ReservoirSegmentSampler.java | 16 +- .../server/coordinator/ServerHolder.java | 5 + .../coordinator/duty/BalanceSegments.java | 5 +- .../server/coordinator/duty/RunRules.java | 14 +- .../rules/BroadcastDistributionRule.java | 52 ++-- .../ForeverBroadcastDistributionRule.java | 21 +- .../IntervalBroadcastDistributionRule.java | 28 +- .../PeriodBroadcastDistributionRule.java | 29 +- .../server/http/DataSourcesResource.java | 2 +- .../metadata/input/InputSourceModuleTest.java | 62 +++++ .../druid/metadata/input/SqlEntityTest.java | 134 +++++++++ .../metadata/input/SqlInputSourceTest.java | 262 ++++++++++++++++++ .../druid/metadata/input/SqlTestUtils.java | 118 ++++++++ .../firehose/SqlFirehoseFactoryTest.java | 78 ++---- ...inatorBasedSegmentHandoffNotifierTest.java | 2 +- .../SegmentLoadDropHandlerTest.java | 193 +++++++++---- .../coordination/ZkCoordinatorTest.java | 68 ++++- .../coordinator/BalanceSegmentsTest.java | 45 ++- .../coordinator/DruidClusterBuilder.java | 9 +- .../ReservoirSegmentSamplerTest.java | 7 +- .../BroadcastDistributionRuleSerdeTest.java | 19 +- .../rules/BroadcastDistributionRuleTest.java | 18 +- .../java/org/apache/druid/cli/CliBroker.java | 17 +- .../java/org/apache/druid/cli/CliIndexer.java | 22 +- .../apache/druid/cli/CliMiddleManager.java | 2 + .../org/apache/druid/cli/CliOverlord.java | 2 + .../java/org/apache/druid/cli/CliPeon.java | 34 ++- .../java/org/apache/druid/cli/MainTest.java | 4 +- .../druid/sql/calcite/schema/DruidSchema.java | 6 +- .../druid/sql/calcite/CalciteQueryTest.java | 2 +- website/.spelling | 4 +- 90 files changed, 2466 insertions(+), 593 deletions(-) create mode 100644 core/src/main/java/org/apache/druid/timeline/partition/BuildingNumberedShardSpec.java create mode 100644 processing/src/test/java/org/apache/druid/segment/filter/ValueMatchersTest.java create mode 100644 server/src/main/java/org/apache/druid/metadata/input/InputSourceModule.java create mode 100644 server/src/main/java/org/apache/druid/metadata/input/SqlEntity.java create mode 100644 server/src/main/java/org/apache/druid/metadata/input/SqlInputFormat.java create mode 100644 server/src/main/java/org/apache/druid/metadata/input/SqlInputSource.java create mode 100644 server/src/main/java/org/apache/druid/metadata/input/SqlReader.java delete mode 100644 server/src/main/java/org/apache/druid/segment/realtime/CliIndexerDataSegmentServerAnnouncerLifecycleHandler.java create mode 100644 server/src/main/java/org/apache/druid/segment/realtime/appenderator/SegmentPublisherHelper.java create mode 100644 server/src/main/java/org/apache/druid/server/coordination/LoadableDataSegment.java create mode 100644 server/src/test/java/org/apache/druid/metadata/input/InputSourceModuleTest.java create mode 100644 server/src/test/java/org/apache/druid/metadata/input/SqlEntityTest.java create mode 100644 server/src/test/java/org/apache/druid/metadata/input/SqlInputSourceTest.java create mode 100644 server/src/test/java/org/apache/druid/metadata/input/SqlTestUtils.java diff --git a/core/src/main/java/org/apache/druid/data/input/impl/prefetch/JsonIterator.java b/core/src/main/java/org/apache/druid/data/input/impl/prefetch/JsonIterator.java index 2a241f66de7c..c03e5f6d8e84 100644 --- a/core/src/main/java/org/apache/druid/data/input/impl/prefetch/JsonIterator.java +++ b/core/src/main/java/org/apache/druid/data/input/impl/prefetch/JsonIterator.java @@ -27,11 +27,11 @@ import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.guava.CloseQuietly; import org.apache.druid.java.util.common.io.Closer; +import org.apache.druid.java.util.common.parsers.CloseableIterator; import java.io.Closeable; import java.io.IOException; import java.io.InputStream; -import java.util.Iterator; import java.util.NoSuchElementException; /** @@ -39,7 +39,7 @@ * * @param the type of object returned by this iterator */ -public class JsonIterator implements Iterator, Closeable +public class JsonIterator implements CloseableIterator { private JsonParser jp; private ObjectCodec objectCodec; diff --git a/core/src/main/java/org/apache/druid/java/util/common/parsers/CloseableIterator.java b/core/src/main/java/org/apache/druid/java/util/common/parsers/CloseableIterator.java index d5915369eada..af1baafe4198 100644 --- a/core/src/main/java/org/apache/druid/java/util/common/parsers/CloseableIterator.java +++ b/core/src/main/java/org/apache/druid/java/util/common/parsers/CloseableIterator.java @@ -66,10 +66,10 @@ default CloseableIterator flatMap(Function> funct return new CloseableIterator() { - CloseableIterator iterator = findNextIeteratorIfNecessary(); + CloseableIterator iterator = findNextIteratorIfNecessary(); @Nullable - private CloseableIterator findNextIeteratorIfNecessary() + private CloseableIterator findNextIteratorIfNecessary() { while ((iterator == null || !iterator.hasNext()) && delegate.hasNext()) { if (iterator != null) { @@ -105,7 +105,7 @@ public R next() return iterator.next(); } finally { - findNextIeteratorIfNecessary(); + findNextIteratorIfNecessary(); } } diff --git a/core/src/main/java/org/apache/druid/timeline/partition/BuildingNumberedShardSpec.java b/core/src/main/java/org/apache/druid/timeline/partition/BuildingNumberedShardSpec.java new file mode 100644 index 000000000000..9c3b00edf1fd --- /dev/null +++ b/core/src/main/java/org/apache/druid/timeline/partition/BuildingNumberedShardSpec.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.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.List; +import java.util.Map; + +/** + * This is a special shardSpec which is temporarily used during batch ingestion. In Druid, there is a concept + * of core partition set which is a set of segments atomically becoming queryable together in Brokers. The core + * partition set is represented as a range of partitionIds. For {@link NumberedShardSpec}, the core partition set + * is [0, {@link NumberedShardSpec#partitions}). + * + * The NumberedShardSpec is used for dynamic partitioning which is based on the number of rows in each segment. + * In streaming ingestion, the core partition set size cannot be determined since it's impossible to know how many + * segments will be created per time chunk. However, in batch ingestion with time chunk locking, the core partition + * set is the set of segments created by an initial task or an overwriting task. Since the core partition set is + * determined when the task publishes segments at the end, the task postpones creating proper NumberedShardSpec + * until the end. + * + * This shardSpec is used for such use case. A non-appending batch task can use this shardSpec until it publishes + * segments at last. When it publishes segments, it should convert the shardSpec of those segments to NumberedShardSpec. + * See {@code SegmentPublisherHelper#annotateShardSpec} for converting to NumberedShardSpec. Note that, when + * the segment lock is used, the Overlord coordinates the segment allocation and this class is never used. See + * {@link PartialShardSpec} for that case. + * + * This class should be Jackson-serializable + * + * Finally, this shardSpec has only partitionId which is same as {@link LinearShardSpec}. The difference between + * them is this shardSpec should never be published and so never be used in other places such as Broker timeline. + * + * @see NumberedShardSpec + */ +public class BuildingNumberedShardSpec implements ShardSpec +{ + private final int partitionId; + + @JsonCreator + public BuildingNumberedShardSpec(int partitionId) + { + Preconditions.checkArgument(partitionId >= 0, "partitionId >= 0"); + this.partitionId = partitionId; + } + + public NumberedShardSpec toNumberedShardSpec(int numTotalPartitions) + { + return new NumberedShardSpec(partitionId, numTotalPartitions); + } + + @Override + public PartitionChunk createChunk(T obj) + { + return new LinearPartitionChunk<>(partitionId, obj); + } + + @JsonProperty("partitionId") + @Override + public int getPartitionNum() + { + return partitionId; + } + + @Override + public ShardSpecLookup getLookup(List shardSpecs) + { + return NumberedShardSpec.createLookup(shardSpecs); + } + + // The below methods are used on the query side, and so must not be called for this shardSpec. + + @Override + public boolean isInChunk(long timestamp, InputRow inputRow) + { + throw new UnsupportedOperationException(); + } + + @Override + public List getDomainDimensions() + { + throw new UnsupportedOperationException(); + } + + @Override + public boolean possibleInDomain(Map> domain) + { + throw new UnsupportedOperationException(); + } + + @Override + public boolean isCompatible(Class other) + { + throw new UnsupportedOperationException(); + } +} 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 d5b36576a654..d07ce9d4f060 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 @@ -24,6 +24,7 @@ import com.google.common.base.Preconditions; import com.google.common.collect.RangeSet; import org.apache.druid.data.input.InputRow; +import org.apache.druid.timeline.DataSegment; import java.util.Collections; import java.util.List; @@ -31,7 +32,24 @@ import java.util.Objects; /** - * ShardSpec for segments which overshadow others with their minorVersion. + * This shardSpec is used only for the segments created by overwriting tasks with segment lock enabled. + * When the segment lock is used, there is a concept of atomic update group which is a set of segments atomically + * becoming queryable together in Brokers. It is a similar concept to the core partition set (explained + * {@link NumberedShardSpec}), but different in a sense that there is only one core partition set per time chunk + * while there could be multiple atomic update groups in one time chunk. + * + * The atomic update group has the root partition range and the minor version to determine the visibility between + * atomic update groups; the group of the highest minor version in the same root partition range becomes queryable + * when they have the same major version ({@link DataSegment#getVersion()}). + * + * Note that this shardSpec is used only when you overwrite existing segments with segment lock enabled. + * If the task doesn't overwrite segments, it will use NumberedShardSpec instead even when segment lock is used. + * Similar to NumberedShardSpec, the size of the atomic update group is determined when the task publishes segments + * at the end of ingestion. As a result, {@link #atomicUpdateGroupSize} is set to + * {@link PartitionIds#UNKNOWN_ATOMIC_UPDATE_GROUP_SIZE} first, and updated when publishing segments + * in {@code SegmentPublisherHelper#annotateShardSpec}. + * + * @see AtomicUpdateGroup */ public class NumberedOverwriteShardSpec implements OverwriteShardSpec { diff --git a/core/src/main/java/org/apache/druid/timeline/partition/NumberedPartialShardSpec.java b/core/src/main/java/org/apache/druid/timeline/partition/NumberedPartialShardSpec.java index 0e258ee786b2..7c7b9753aaa8 100644 --- a/core/src/main/java/org/apache/druid/timeline/partition/NumberedPartialShardSpec.java +++ b/core/src/main/java/org/apache/druid/timeline/partition/NumberedPartialShardSpec.java @@ -40,6 +40,13 @@ private NumberedPartialShardSpec() public ShardSpec complete(ObjectMapper objectMapper, @Nullable ShardSpec specOfPreviousMaxPartitionId) { if (specOfPreviousMaxPartitionId == null) { + // The shardSpec is created by the Overlord. + // - For streaming ingestion tasks, the core partition set is always 0. + // - For batch tasks, this code is executed only with segment locking (forceTimeChunkLock = false). + // In this mode, you can have 2 or more tasks concurrently ingesting into the same time chunk of + // the same datasource. Since there is no restriction for those tasks in segment allocation, the + // allocated IDs for each task can interleave. As a result, the core partition set cannot be + // represented as a range. We always set 0 for the core partition set size. return new NumberedShardSpec(0, 0); } else { final NumberedShardSpec prevSpec = (NumberedShardSpec) specOfPreviousMaxPartitionId; 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 d6f98d71751d..6f8898e298f1 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 @@ -67,6 +67,11 @@ public int getPartitionNum() @Override public ShardSpecLookup getLookup(final List shardSpecs) + { + return createLookup(shardSpecs); + } + + static ShardSpecLookup createLookup(List shardSpecs) { return (long timestamp, InputRow row) -> shardSpecs.get(0); } diff --git a/core/src/main/java/org/apache/druid/timeline/partition/PartialShardSpec.java b/core/src/main/java/org/apache/druid/timeline/partition/PartialShardSpec.java index 9cbee2c7571c..9ba9ac579c0e 100644 --- a/core/src/main/java/org/apache/druid/timeline/partition/PartialShardSpec.java +++ b/core/src/main/java/org/apache/druid/timeline/partition/PartialShardSpec.java @@ -29,7 +29,10 @@ /** * Class to contain all information of a {@link ShardSpec} except for the partition ID. - * This class is mainly used by the indexing tasks to allocate new segments using the Overlord. + * This class is used when the segment allocation is coordinated by the Overlord; when appending segments to an + * existing datasource (either streaming ingestion or batch append) or when using segment locking. + * The ingestion tasks send all information required for allocating a new segment using this class and the Overlord + * determins the partition ID to create a new segment. */ @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type") @JsonSubTypes({ 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 43aaf701db36..6b96d6560161 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 @@ -46,23 +46,46 @@ public interface ShardSpec boolean isInChunk(long timestamp, InputRow inputRow); + /** + * Returns the partition ID of this segment. + */ int getPartitionNum(); + /** + * Returns the start root partition ID of the atomic update group which this segment belongs to. + * + * @see AtomicUpdateGroup + */ default int getStartRootPartitionId() { return getPartitionNum(); } + /** + * Returns the end root partition ID of the atomic update group which this segment belongs to. + * + * @see AtomicUpdateGroup + */ default int getEndRootPartitionId() { return getPartitionNum() + 1; } + /** + * Returns the minor version associated to the atomic update group which this segment belongs to. + * + * @see AtomicUpdateGroup + */ default short getMinorVersion() { return 0; } + /** + * Returns the atomic update group size which this segment belongs to. + * + * @see AtomicUpdateGroup + */ default short getAtomicUpdateGroupSize() { return 1; @@ -84,7 +107,7 @@ default short getAtomicUpdateGroupSize() boolean possibleInDomain(Map> domain); /** - * Returns true if two segments of this and other shardSpecs can exist in the same timeChunk. + * Returns true if two segments of this and other shardSpecs can exist in the same time chunk. */ boolean isCompatible(Class other); } diff --git a/core/src/main/java/org/apache/druid/timeline/partition/ShardSpecLookup.java b/core/src/main/java/org/apache/druid/timeline/partition/ShardSpecLookup.java index 25c785b542e8..610e92a0ea6b 100644 --- a/core/src/main/java/org/apache/druid/timeline/partition/ShardSpecLookup.java +++ b/core/src/main/java/org/apache/druid/timeline/partition/ShardSpecLookup.java @@ -23,5 +23,9 @@ public interface ShardSpecLookup { + /** + * Returns a {@link ShardSpec} for the given timestamp and the inputRow. + * The timestamp must be bucketed using {@code GranularitySpec#getQueryGranularity}. + */ ShardSpec getShardSpec(long timestamp, InputRow row); } diff --git a/docs/configuration/index.md b/docs/configuration/index.md index 0f2175b3aa1c..34b458a49b2b 100644 --- a/docs/configuration/index.md +++ b/docs/configuration/index.md @@ -709,7 +709,7 @@ These Coordinator static configurations can be defined in the `coordinator/runti |`druid.manager.config.pollDuration`|How often the manager polls the config table for updates.|PT1M| |`druid.manager.segments.pollDuration`|The duration between polls the Coordinator does for updates to the set of active segments. Generally defines the amount of lag time it can take for the Coordinator to notice new segments.|PT1M| |`druid.manager.rules.pollDuration`|The duration between polls the Coordinator does for updates to the set of active rules. Generally defines the amount of lag time it can take for the Coordinator to notice rules.|PT1M| -|`druid.manager.rules.defaultTier`|The default tier from which default rules will be loaded from.|_default| +|`druid.manager.rules.defaultRule`|The default rule for the cluster|_default| |`druid.manager.rules.alertThreshold`|The duration after a failed poll upon which an alert should be emitted.|PT10M| #### Dynamic Configuration @@ -1204,7 +1204,7 @@ You can optionally configure caching to be enabled on the peons by setting cachi |`druid.realtime.cache.useCache`|true, false|Enable the cache on the realtime.|false| |`druid.realtime.cache.populateCache`|true, false|Populate the cache on the realtime.|false| |`druid.realtime.cache.unCacheable`|All druid query types|All query types to not cache.|`[]`| -|`druid.realtime.cache.maxEntrySize`|Maximum cache entry size in bytes.|1_000_000| +|`druid.realtime.cache.maxEntrySize`|positive integer|Maximum cache entry size in bytes.|1_000_000| See [cache configuration](#cache-configuration) for how to configure cache settings. @@ -1344,7 +1344,7 @@ You can optionally configure caching to be enabled on the Indexer by setting cac |`druid.realtime.cache.useCache`|true, false|Enable the cache on the realtime.|false| |`druid.realtime.cache.populateCache`|true, false|Populate the cache on the realtime.|false| |`druid.realtime.cache.unCacheable`|All druid query types|All query types to not cache.|`[]`| -|`druid.realtime.cache.maxEntrySize`|Maximum cache entry size in bytes.|1_000_000| +|`druid.realtime.cache.maxEntrySize`|positive integer|Maximum cache entry size in bytes.|1_000_000| See [cache configuration](#cache-configuration) for how to configure cache settings. @@ -1650,7 +1650,7 @@ You can optionally only configure caching to be enabled on the Broker by setting |`druid.broker.cache.resultLevelCacheLimit`|positive integer|Maximum size of query response that can be cached.|`Integer.MAX_VALUE`| |`druid.broker.cache.unCacheable`|All druid query types|All query types to not cache.|`[]`| |`druid.broker.cache.cacheBulkMergeLimit`|positive integer or 0|Queries with more segments than this number will not attempt to fetch from cache at the broker level, leaving potential caching fetches (and cache result merging) to the Historicals|`Integer.MAX_VALUE`| -|`druid.broker.cache.maxEntrySize`|Maximum cache entry size in bytes.|1_000_000| +|`druid.broker.cache.maxEntrySize`|positive integer|Maximum cache entry size in bytes.|1_000_000| See [cache configuration](#cache-configuration) for how to configure cache settings. diff --git a/docs/ingestion/native-batch.md b/docs/ingestion/native-batch.md index 02c8a321f984..981f9f700938 100644 --- a/docs/ingestion/native-batch.md +++ b/docs/ingestion/native-batch.md @@ -53,7 +53,7 @@ The detailed behavior of the Parallel task is different depending on the [`parti See each `partitionsSpec` for more details. To use this task, the [`inputSource`](#input-sources) in the `ioConfig` should be _splittable_ and `maxNumConcurrentSubTasks` should be set to larger than 1 in the `tuningConfig`. -Otherwise, this task runs sequentially; the `index_paralllel` task reads each input file one by one and creates segments by itself. +Otherwise, this task runs sequentially; the `index_parallel` task reads each input file one by one and creates segments by itself. The supported splittable input formats for now are: - [`s3`](#s3-input-source) reads data from AWS S3 storage. @@ -63,6 +63,7 @@ The supported splittable input formats for now are: - [`http`](#http-input-source) reads data from HTTP servers. - [`local`](#local-input-source) reads data from local storage. - [`druid`](#druid-input-source) reads data from a Druid datasource. +- [`sql`](#sql-input-source) reads data from a RDBMS source. Some other cloud storage types are supported with the legacy [`firehose`](#firehoses-deprecated). The below `firehose` types are also splittable. Note that only text formats are supported @@ -1310,6 +1311,59 @@ A spec that applies a filter and reads a subset of the original datasource's col This spec above will only return the `page`, `user` dimensions and `added` metric. Only rows where `page` = `Druid` will be returned. +### SQL Input Source + +The SQL input source is used to read data directly from RDBMS. +The SQL input source is _splittable_ and can be used by the [Parallel task](#parallel-task), where each worker task will read from one SQL query from the list of queries. +Since this input source has a fixed input format for reading events, no `inputFormat` field needs to be specified in the ingestion spec when using this input source. +Please refer to the Recommended practices section below before using this input source. + +|property|description|required?| +|--------|-----------|---------| +|type|This should be "sql".|Yes| +|database|Specifies the database connection details. The database type corresponds to the extension that supplies the `connectorConfig` support and this extension must be loaded into Druid. For database types `mysql` and `postgresql`, the `connectorConfig` support is provided by [mysql-metadata-storage](../development/extensions-core/mysql.md) and [postgresql-metadata-storage](../development/extensions-core/postgresql.md) extensions respectively.|Yes| +|foldCase|Toggle case folding of database column names. This may be enabled in cases where the database returns case insensitive column names in query results.|No| +|sqls|List of SQL queries where each SQL query would retrieve the data to be indexed.|Yes| + +An example SqlInputSource spec is shown below: + +```json +... + "ioConfig": { + "type": "index_parallel", + "inputSource": { + "type": "sql", + "database": { + "type": "mysql", + "connectorConfig": { + "connectURI": "jdbc:mysql://host:port/schema", + "user": "user", + "password": "password" + } + }, + "sqls": ["SELECT * FROM table1 WHERE timestamp BETWEEN '2013-01-01 00:00:00' AND '2013-01-01 11:59:59'", "SELECT * FROM table2 WHERE timestamp BETWEEN '2013-01-01 00:00:00' AND '2013-01-01 11:59:59'"] + }, +... +``` + +The spec above will read all events from two separate SQLs for the interval `2013-01-01/2013-01-02`. +Each of the SQL queries will be run in its own sub-task and thus for the above example, there would be two sub-tasks. + +**Recommended practices** + +Compared to the other native batch InputSources, SQL InputSource behaves differently in terms of reading the input data and so it would be helpful to consider the following points before using this InputSource in a production environment: + +* During indexing, each sub-task would execute one of the SQL queries and the results are stored locally on disk. The sub-tasks then proceed to read the data from these local input files and generate segments. Presently, there isn’t any restriction on the size of the generated files and this would require the MiddleManagers or Indexers to have sufficient disk capacity based on the volume of data being indexed. + +* Filtering the SQL queries based on the intervals specified in the `granularitySpec` can avoid unwanted data being retrieved and stored locally by the indexing sub-tasks. For example, if the `intervals` specified in the `granularitySpec` is `["2013-01-01/2013-01-02"]` and the SQL query is `SELECT * FROM table1`, `SqlInputSource` will read all the data for `table1` based on the query, even though only data between the intervals specified will be indexed into Druid. + +* Pagination may be used on the SQL queries to ensure that each query pulls a similar amount of data, thereby improving the efficiency of the sub-tasks. + +* Similar to file-based input formats, any updates to existing data will replace the data in segments specific to the intervals specified in the `granularitySpec`. + + +### + ## Firehoses (Deprecated) Firehoses are deprecated in 0.17.0. It's highly recommended to use the [Input source](#input-sources) instead. @@ -1544,6 +1598,7 @@ This firehose will accept any type of parser, but will only utilize the list of This Firehose can be used to ingest events residing in an RDBMS. The database connection information is provided as part of the ingestion spec. For each query, the results are fetched locally and indexed. If there are multiple queries from which data needs to be indexed, queries are prefetched in the background, up to `maxFetchCapacityBytes` bytes. +This Firehose is _splittable_ and can be used by [native parallel index tasks](native-batch.md#parallel-task). This firehose will accept any type of parser, but will only utilize the list of dimensions and the timestamp specification. See the extension documentation for more detailed ingestion examples. Requires one of the following extensions: 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 9f7f9977efb2..6bf40866b5ea 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 @@ -191,4 +191,10 @@ public String getType() { return TYPE; } + + @Override + public boolean supportsQueries() + { + return true; + } } 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 46a6a0418ba9..eb41749c33ad 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 @@ -345,6 +345,7 @@ public void testRunAfterDataInserted() throws Exception INPUT_FORMAT ) ); + Assert.assertTrue(task.supportsQueries()); final ListenableFuture future = runTask(task); 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 bfd375830497..9eee3bf6274d 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 @@ -137,6 +137,12 @@ public String getType() return TYPE; } + @Override + public boolean supportsQueries() + { + return true; + } + @VisibleForTesting AWSCredentialsConfig getAwsCredentialsConfig() { 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 142f81f21016..48ca434e7fd8 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 @@ -347,8 +347,8 @@ public void testRunAfterDataInserted() throws Exception null, false ) - ); + Assert.assertTrue(task.supportsQueries()); final ListenableFuture future = runTask(task); 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 40745bfe32f5..728f3dedfd0d 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 @@ -145,6 +145,12 @@ public QueryRunner getQueryRunner(Query query) return null; } + @Override + public boolean supportsQueries() + { + return false; + } + @Override public String getClasspathPrefix() { 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 c96b7f5b9002..9a786fa36ea8 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 @@ -245,6 +245,12 @@ public QueryRunner getQueryRunner(Query query) return (queryPlus, responseContext) -> queryPlus.run(appenderator, responseContext); } + @Override + public boolean supportsQueries() + { + return true; + } + @Override public boolean isReady(TaskActionClient taskActionClient) { 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 dac3bd9a0952..59c9019e6397 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 @@ -61,6 +61,7 @@ import org.apache.druid.indexing.input.DruidInputSource; import org.apache.druid.indexing.overlord.Segments; 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.RE; @@ -98,11 +99,9 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; -import java.util.Comparator; import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.Map.Entry; import java.util.TreeMap; import java.util.stream.Collectors; import java.util.stream.IntStream; @@ -524,10 +523,30 @@ static List createIngestionSchema( .add(p) ); - final List specs = new ArrayList<>(intervalToSegments.size()); - for (Entry>> entry : intervalToSegments.entrySet()) { - final Interval interval = entry.getKey(); - final List> segmentsToCompact = entry.getValue(); + // unify overlapping intervals to ensure overlapping segments compacting in the same indexSpec + List>>> intervalToSegmentsUnified = new ArrayList<>(); + Interval union = null; + List> segments = new ArrayList<>(); + for (Map.Entry>> entry : intervalToSegments.entrySet()) { + Interval cur = entry.getKey(); + if (union == null) { + union = cur; + segments.addAll(entry.getValue()); + } else if (union.overlaps(cur)) { + union = Intervals.utc(union.getStartMillis(), Math.max(union.getEndMillis(), cur.getEndMillis())); + segments.addAll(entry.getValue()); + } else { + intervalToSegmentsUnified.add(Pair.of(union, segments)); + union = cur; + segments = new ArrayList<>(entry.getValue()); + } + } + intervalToSegmentsUnified.add(Pair.of(union, segments)); + + final List specs = new ArrayList<>(intervalToSegmentsUnified.size()); + for (Pair>> entry : intervalToSegmentsUnified) { + final Interval interval = entry.lhs; + final List> segmentsToCompact = entry.rhs; final DataSchema dataSchema = createDataSchema( segmentProvider.dataSource, segmentsToCompact, @@ -710,20 +729,8 @@ private static DimensionsSpec createDimensionsSpec(List intervalComparator = Comparators.intervalsByStartThenEnd(); - for (int i = 0; i < queryableIndices.size() - 1; i++) { - final Interval shouldBeSmaller = queryableIndices.get(i).lhs.getDataInterval(); - final Interval shouldBeLarger = queryableIndices.get(i + 1).lhs.getDataInterval(); - Preconditions.checkState( - intervalComparator.compare(shouldBeSmaller, shouldBeLarger) <= 0, - "QueryableIndexes are not sorted! Interval[%s] of segment[%s] is laster than interval[%s] of segment[%s]", - shouldBeSmaller, - queryableIndices.get(i).rhs.getId(), - shouldBeLarger, - queryableIndices.get(i + 1).rhs.getId() - ); - } + // sort timelineSegments in order of interval, see https://github.com/apache/druid/pull/9905 + queryableIndices.sort((o1, o2) -> Comparators.intervalsByStartThenEnd().compare(o1.rhs.getInterval(), o2.rhs.getInterval())); int index = 0; for (Pair pair : Lists.reverse(queryableIndices)) { 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 ead0f635599f..2cb4db551c4a 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 @@ -30,7 +30,7 @@ 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.apache.druid.timeline.partition.BuildingNumberedShardSpec; import org.joda.time.DateTime; import org.joda.time.Interval; @@ -77,7 +77,7 @@ class LocalSegmentAllocator implements SegmentAllocator dataSource, interval, version, - new NumberedShardSpec(partitionId, 0) + new BuildingNumberedShardSpec(partitionId) ); }; } 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 ed2ddd2c604e..055a3fea3be7 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 @@ -196,6 +196,12 @@ public QueryRunner getQueryRunner(Query query) return plumber.getQueryRunner(query); } + @Override + public boolean supportsQueries() + { + return true; + } + @Override public boolean isReady(TaskActionClient taskActionClient) { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/Task.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/Task.java index c069c73cec70..4f18c81bc7c7 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/Task.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/Task.java @@ -146,6 +146,11 @@ default int getPriority() */ QueryRunner getQueryRunner(Query query); + /** + * @return true if this Task type is queryable, such as streaming ingestion tasks + */ + boolean supportsQueries(); + /** * Returns an extra classpath that should be prepended to the default classpath when running this task. If no * extra classpath should be prepended, this should return null or the empty string. 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 2d2ec3a0f2b6..cb1bc3986510 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 @@ -78,6 +78,7 @@ import org.apache.druid.server.security.Action; import org.apache.druid.server.security.AuthorizerMapper; import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.partition.BuildingNumberedShardSpec; import org.apache.druid.timeline.partition.NumberedShardSpec; import org.apache.druid.timeline.partition.PartitionBoundaries; import org.apache.druid.utils.CollectionUtils; @@ -984,7 +985,7 @@ SegmentIdWithShardSpec allocateNewSegment(DateTime timestamp) throws IOException dataSource, interval, version, - new NumberedShardSpec(partitionNum, 0) + new BuildingNumberedShardSpec(partitionNum) ); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/ForkingTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/ForkingTaskRunner.java index 2f1abc1d7f53..10f5e6b5e9c2 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/ForkingTaskRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/ForkingTaskRunner.java @@ -327,6 +327,13 @@ public TaskStatus call() command.add(nodeType); } + // If the task type is queryable, we need to load broadcast segments on the peon, used for + // join queries + if (task.supportsQueries()) { + command.add("--loadBroadcastSegments"); + command.add("true"); + } + if (!taskFile.exists()) { jsonMapper.writeValue(taskFile, task); } 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 de3fa29a489b..513a590be9fd 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 @@ -337,6 +337,7 @@ public void testBasics() throws Exception { expectPublishedSegments(1); final AppenderatorDriverRealtimeIndexTask task = makeRealtimeTask(null); + Assert.assertTrue(task.supportsQueries()); final ListenableFuture statusFuture = runTask(task); // Wait for firehose to show up, it starts off null. 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 1d4bec4f99e1..66ee8595d25d 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 @@ -73,7 +73,6 @@ import org.apache.druid.jackson.DefaultObjectMapper; 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.granularity.PeriodGranularity; @@ -160,7 +159,11 @@ public class CompactionTaskTest Intervals.of("2017-03-01/2017-04-01"), Intervals.of("2017-04-01/2017-05-01"), Intervals.of("2017-05-01/2017-06-01"), - Intervals.of("2017-06-01/2017-07-01") + Intervals.of("2017-06-01/2017-07-01"), + // overlapping intervals + Intervals.of("2017-06-01/2017-06-02"), + Intervals.of("2017-06-15/2017-06-16"), + Intervals.of("2017-06-30/2017-07-01") ); private static final Map MIXED_TYPE_COLUMN_MAP = new HashMap<>(); private static final ParallelIndexTuningConfig TUNING_CONFIG = createTuningConfig(); @@ -191,12 +194,17 @@ public static void setupClass() MIXED_TYPE_COLUMN_MAP.put(Intervals.of("2017-05-01/2017-06-01"), new DoubleDimensionSchema(MIXED_TYPE_COLUMN)); MIXED_TYPE_COLUMN_MAP.put(Intervals.of("2017-06-01/2017-07-01"), new DoubleDimensionSchema(MIXED_TYPE_COLUMN)); + MIXED_TYPE_COLUMN_MAP.put(Intervals.of("2017-06-01/2017-06-02"), new DoubleDimensionSchema(MIXED_TYPE_COLUMN)); + MIXED_TYPE_COLUMN_MAP.put(Intervals.of("2017-06-15/2017-06-16"), new DoubleDimensionSchema(MIXED_TYPE_COLUMN)); + MIXED_TYPE_COLUMN_MAP.put(Intervals.of("2017-06-30/2017-07-01"), new DoubleDimensionSchema(MIXED_TYPE_COLUMN)); + DIMENSIONS = new HashMap<>(); AGGREGATORS = new ArrayList<>(); DIMENSIONS.put(ColumnHolder.TIME_COLUMN_NAME, new LongDimensionSchema(ColumnHolder.TIME_COLUMN_NAME)); DIMENSIONS.put(TIMESTAMP_COLUMN, new LongDimensionSchema(TIMESTAMP_COLUMN)); - for (int i = 0; i < SEGMENT_INTERVALS.size(); i++) { + int numUmbrellaIntervals = 6; + for (int i = 0; i < numUmbrellaIntervals; i++) { final StringDimensionSchema schema = new StringDimensionSchema( "string_dim_" + i, null, @@ -204,15 +212,15 @@ public static void setupClass() ); DIMENSIONS.put(schema.getName(), schema); } - for (int i = 0; i < SEGMENT_INTERVALS.size(); i++) { + for (int i = 0; i < numUmbrellaIntervals; i++) { final LongDimensionSchema schema = new LongDimensionSchema("long_dim_" + i); DIMENSIONS.put(schema.getName(), schema); } - for (int i = 0; i < SEGMENT_INTERVALS.size(); i++) { + for (int i = 0; i < numUmbrellaIntervals; i++) { final FloatDimensionSchema schema = new FloatDimensionSchema("float_dim_" + i); DIMENSIONS.put(schema.getName(), schema); } - for (int i = 0; i < SEGMENT_INTERVALS.size(); i++) { + for (int i = 0; i < numUmbrellaIntervals; i++) { final DoubleDimensionSchema schema = new DoubleDimensionSchema("double_dim_" + i); DIMENSIONS.put(schema.getName(), schema); } @@ -224,14 +232,13 @@ public static void setupClass() AGGREGATORS.add(new DoubleLastAggregatorFactory("agg_4", "double_dim_4")); for (int i = 0; i < SEGMENT_INTERVALS.size(); i++) { - final Interval segmentInterval = Intervals.of(StringUtils.format("2017-0%d-01/2017-0%d-01", (i + 1), (i + 2))); SEGMENT_MAP.put( new DataSegment( DATA_SOURCE, - segmentInterval, - "version", + SEGMENT_INTERVALS.get(i), + "version_" + i, ImmutableMap.of(), - findDimensions(i, segmentInterval), + findDimensions(i, SEGMENT_INTERVALS.get(i)), AGGREGATORS.stream().map(AggregatorFactory::getName).collect(Collectors.toList()), new NumberedShardSpec(0, 1), 0, @@ -285,7 +292,7 @@ private static List findDimensions(int startIndex, Interval segmentInter dimensions.add(TIMESTAMP_COLUMN); for (int i = 0; i < 6; i++) { int postfix = i + startIndex; - postfix = postfix >= 6 ? postfix - 6 : postfix; + postfix = postfix % 6; dimensions.add("string_dim_" + postfix); dimensions.add("long_dim_" + postfix); dimensions.add("float_dim_" + postfix); 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 8668a937aa18..6e3fcf54dcf2 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 @@ -229,6 +229,8 @@ public void testDeterminePartitions() throws Exception appenderatorsManager ); + Assert.assertFalse(indexTask.supportsQueries()); + final List segments = runTask(indexTask).rhs; Assert.assertEquals(2, segments.size()); 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 8cc21d02e19a..12ea2145d582 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 @@ -189,6 +189,12 @@ public void testDefaultResource() Assert.assertEquals(task.getId(), task.getTaskResource().getAvailabilityGroup()); } + @Test(timeout = 60_000L) + public void testSupportsQueries() + { + final RealtimeIndexTask task = makeRealtimeTask(null); + Assert.assertTrue(task.supportsQueries()); + } @Test(timeout = 60_000L, expected = ExecutionException.class) public void testHandoffTimeout() throws Exception diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexingTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexingTest.java index 7b1baa9bc1f0..9b8129850e15 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexingTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexingTest.java @@ -43,6 +43,7 @@ import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.Partitions; import org.apache.druid.timeline.VersionedIntervalTimeline; +import org.apache.druid.timeline.partition.NumberedShardSpec; import org.joda.time.Interval; import org.junit.After; import org.junit.Assert; @@ -57,10 +58,13 @@ import java.io.Writer; import java.nio.charset.StandardCharsets; import java.nio.file.Files; +import java.util.ArrayList; import java.util.Collection; import java.util.Collections; +import java.util.HashMap; import java.util.HashSet; import java.util.Iterator; +import java.util.List; import java.util.Map; import java.util.Set; @@ -155,6 +159,7 @@ private void runTestTask(@Nullable Interval interval, Granularity segmentGranula final ParallelIndexSupervisorTask task = newTask(interval, segmentGranularity, appendToExisting, true); task.addToContext(Tasks.FORCE_TIME_CHUNK_LOCK_KEY, lockGranularity == LockGranularity.TIME_CHUNK); Assert.assertEquals(TaskState.SUCCESS, getIndexingServiceClient().runAndWait(task).getStatusCode()); + assertShardSpec(interval); } private void runTestTask(@Nullable Interval interval, Granularity segmentGranularity) @@ -184,6 +189,33 @@ private void testRunAndOverwrite(@Nullable Interval inputInterval, Granularity s Assert.assertEquals(new HashSet<>(newSegments), visibles); } + private void assertShardSpec(@Nullable Interval interval) + { + final Interval nonNullInterval = interval == null ? Intervals.ETERNITY : interval; + final Collection segments = + getStorageCoordinator().retrieveUsedSegmentsForInterval("dataSource", nonNullInterval, Segments.ONLY_VISIBLE); + if (lockGranularity != LockGranularity.SEGMENT) { + // Check the core partition set in the shardSpec + final Map> intervalToSegments = new HashMap<>(); + segments.forEach( + segment -> intervalToSegments.computeIfAbsent(segment.getInterval(), k -> new ArrayList<>()).add(segment) + ); + for (List segmentsPerInterval : intervalToSegments.values()) { + for (DataSegment segment : segmentsPerInterval) { + Assert.assertSame(NumberedShardSpec.class, segment.getShardSpec().getClass()); + final NumberedShardSpec shardSpec = (NumberedShardSpec) segment.getShardSpec(); + Assert.assertEquals(segmentsPerInterval.size(), shardSpec.getPartitions()); + } + } + } else { + for (DataSegment segment : segments) { + Assert.assertSame(NumberedShardSpec.class, segment.getShardSpec().getClass()); + final NumberedShardSpec shardSpec = (NumberedShardSpec) segment.getShardSpec(); + Assert.assertEquals(0, shardSpec.getPartitions()); + } + } + } + @Test public void testWithoutInterval() { @@ -213,9 +245,11 @@ public void testRunInParallelWithDifferentSegmentGranularity() @Test public void testRunInSequential() { - final ParallelIndexSupervisorTask task = newTask(Intervals.of("2017-12/P1M"), false, false); + final Interval interval = Intervals.of("2017-12/P1M"); + final ParallelIndexSupervisorTask task = newTask(interval, false, false); task.addToContext(Tasks.FORCE_TIME_CHUNK_LOCK_KEY, lockGranularity == LockGranularity.TIME_CHUNK); Assert.assertEquals(TaskState.SUCCESS, getIndexingServiceClient().runAndWait(task).getStatusCode()); + assertShardSpec(interval); } @Test @@ -229,8 +263,9 @@ public void testPublishEmptySegments() @Test public void testWith1MaxNumConcurrentSubTasks() { + final Interval interval = Intervals.of("2017-12/P1M"); final ParallelIndexSupervisorTask task = newTask( - Intervals.of("2017-12/P1M"), + interval, Granularities.DAY, false, true, @@ -266,6 +301,7 @@ public void testWith1MaxNumConcurrentSubTasks() task.addToContext(Tasks.FORCE_TIME_CHUNK_LOCK_KEY, lockGranularity == LockGranularity.TIME_CHUNK); Assert.assertEquals(TaskState.SUCCESS, getIndexingServiceClient().runAndWait(task).getStatusCode()); Assert.assertNull("Runner must be null if the task was in the sequential mode", task.getCurrentRunner()); + assertShardSpec(interval); } @Test diff --git a/processing/src/main/java/org/apache/druid/segment/vector/NilVectorSelector.java b/processing/src/main/java/org/apache/druid/segment/vector/NilVectorSelector.java index 3aeb32c1f262..9cc6dea06760 100644 --- a/processing/src/main/java/org/apache/druid/segment/vector/NilVectorSelector.java +++ b/processing/src/main/java/org/apache/druid/segment/vector/NilVectorSelector.java @@ -142,7 +142,7 @@ public String lookupName(final int id) @Override public boolean nameLookupPossibleInAdvance() { - return false; + return true; } @Nullable diff --git a/processing/src/test/java/org/apache/druid/segment/filter/ValueMatchersTest.java b/processing/src/test/java/org/apache/druid/segment/filter/ValueMatchersTest.java new file mode 100644 index 000000000000..6698c3ab6ac1 --- /dev/null +++ b/processing/src/test/java/org/apache/druid/segment/filter/ValueMatchersTest.java @@ -0,0 +1,152 @@ +/* + * 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.filter; + +import com.google.common.collect.ImmutableList; +import org.apache.druid.segment.DimensionSelector; +import org.apache.druid.segment.SimpleAscendingOffset; +import org.apache.druid.segment.data.GenericIndexed; +import org.apache.druid.segment.data.VSizeColumnarInts; +import org.apache.druid.segment.data.VSizeColumnarMultiInts; +import org.apache.druid.segment.serde.DictionaryEncodedColumnSupplier; +import org.apache.druid.segment.vector.NilVectorSelector; +import org.apache.druid.segment.vector.NoFilterVectorOffset; +import org.apache.druid.testing.InitializedNullHandlingTest; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +public class ValueMatchersTest extends InitializedNullHandlingTest +{ + private DictionaryEncodedColumnSupplier supplierSingleConstant; + private DictionaryEncodedColumnSupplier supplierSingle; + private DictionaryEncodedColumnSupplier supplierMulti; + + @Before + public void setup() + { + supplierSingleConstant = new DictionaryEncodedColumnSupplier( + GenericIndexed.fromIterable(ImmutableList.of("value"), GenericIndexed.STRING_STRATEGY), + () -> VSizeColumnarInts.fromArray(new int[]{0}), + null, + 0 + ); + supplierSingle = new DictionaryEncodedColumnSupplier( + GenericIndexed.fromIterable(ImmutableList.of("value", "value2"), GenericIndexed.STRING_STRATEGY), + () -> VSizeColumnarInts.fromArray(new int[]{0, 0, 1, 0, 1}), + null, + 0 + ); + supplierMulti = new DictionaryEncodedColumnSupplier( + GenericIndexed.fromIterable(ImmutableList.of("value"), GenericIndexed.STRING_STRATEGY), + null, + () -> VSizeColumnarMultiInts.fromIterable( + ImmutableList.of( + VSizeColumnarInts.fromArray(new int[]{0, 0}), + VSizeColumnarInts.fromArray(new int[]{0}) + ) + ), + 0 + ); + } + @Test + public void testNullDimensionSelectorCanBeBoolean() + { + Boolean resultMatchNull = ValueMatchers.toBooleanIfPossible( + DimensionSelector.constant(null), + false, + string -> string == null + ); + Assert.assertNotNull(resultMatchNull); + Assert.assertTrue(resultMatchNull); + + Boolean resultMatchNotNull = ValueMatchers.toBooleanIfPossible( + DimensionSelector.constant(null), + false, + string -> string != null + ); + Assert.assertNotNull(resultMatchNotNull); + Assert.assertFalse(resultMatchNotNull); + + Boolean resultMatchNonNilConstant = ValueMatchers.toBooleanIfPossible( + supplierSingleConstant.get().makeDimensionSelector(new SimpleAscendingOffset(1), null), + false, + string -> string != null + ); + Assert.assertNotNull(resultMatchNonNilConstant); + Assert.assertTrue(resultMatchNonNilConstant); + + Boolean resultMatchNonNil = ValueMatchers.toBooleanIfPossible( + supplierSingle.get().makeDimensionSelector(new SimpleAscendingOffset(1), null), + false, + string -> string != null + ); + Assert.assertNull(resultMatchNonNil); + + Boolean resultMatchNonNilMulti = ValueMatchers.toBooleanIfPossible( + supplierMulti.get().makeDimensionSelector(new SimpleAscendingOffset(1), null), + true, + string -> string != null + ); + Assert.assertNull(resultMatchNonNilMulti); + } + + @Test + public void testNilVectorSelectorCanBeBoolean() + { + Boolean resultMatchNull = ValueMatchers.toBooleanIfPossible( + NilVectorSelector.create(new NoFilterVectorOffset(10, 0, 100)), + false, + string -> string == null + ); + Assert.assertNotNull(resultMatchNull); + Assert.assertTrue(resultMatchNull); + + Boolean resultMatchNotNull = ValueMatchers.toBooleanIfPossible( + NilVectorSelector.create(new NoFilterVectorOffset(10, 0, 100)), + false, + string -> string != null + ); + Assert.assertNotNull(resultMatchNotNull); + Assert.assertFalse(resultMatchNotNull); + + Boolean resultMatchNotNilConstant = ValueMatchers.toBooleanIfPossible( + supplierSingleConstant.get().makeSingleValueDimensionVectorSelector(new NoFilterVectorOffset(10, 0, 1)), + false, + string -> string != null + ); + Assert.assertNotNull(resultMatchNotNilConstant); + Assert.assertTrue(resultMatchNotNilConstant); + + Boolean resultMatchNotNil = ValueMatchers.toBooleanIfPossible( + supplierSingle.get().makeSingleValueDimensionVectorSelector(new NoFilterVectorOffset(10, 0, 1)), + false, + string -> string != null + ); + Assert.assertNull(resultMatchNotNil); + + Boolean resultMatchNotNilMulti = ValueMatchers.toBooleanIfPossible( + supplierMulti.get().makeSingleValueDimensionVectorSelector(new NoFilterVectorOffset(10, 0, 1)), + true, + string -> string != null + ); + Assert.assertNull(resultMatchNotNilMulti); + } +} diff --git a/server/pom.xml b/server/pom.xml index 5d68a91dfd89..ddce092c1cd4 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -291,10 +291,6 @@ javax.validation validation-api - - org.hibernate - hibernate-validator - com.google.errorprone error_prone_annotations diff --git a/server/src/main/java/org/apache/druid/client/CachingClusteredClient.java b/server/src/main/java/org/apache/druid/client/CachingClusteredClient.java index ae4dc1689ccf..c382be3493e0 100644 --- a/server/src/main/java/org/apache/druid/client/CachingClusteredClient.java +++ b/server/src/main/java/org/apache/druid/client/CachingClusteredClient.java @@ -452,7 +452,7 @@ private String computeCurrentEtag(final Set segments, @Nu Hasher hasher = Hashing.sha1().newHasher(); boolean hasOnlyHistoricalSegments = true; for (SegmentServerSelector p : segments) { - if (!p.getServer().pick().getServer().segmentReplicatable()) { + if (!p.getServer().pick().getServer().isSegmentReplicationTarget()) { hasOnlyHistoricalSegments = false; break; } @@ -633,7 +633,7 @@ private void addSequencesFromServer( if (isBySegment) { serverResults = getBySegmentServerResults(serverRunner, segmentsOfServer, maxQueuedBytesPerServer); - } else if (!server.segmentReplicatable() || !populateCache) { + } else if (!server.isSegmentReplicationTarget() || !populateCache) { serverResults = getSimpleServerResults(serverRunner, segmentsOfServer, maxQueuedBytesPerServer); } else { serverResults = getAndCacheServerResults(serverRunner, segmentsOfServer, maxQueuedBytesPerServer); diff --git a/server/src/main/java/org/apache/druid/client/DruidServer.java b/server/src/main/java/org/apache/druid/client/DruidServer.java index ddcba54f1c3c..6c52866d0586 100644 --- a/server/src/main/java/org/apache/druid/client/DruidServer.java +++ b/server/src/main/java/org/apache/druid/client/DruidServer.java @@ -137,9 +137,19 @@ public String getTier() return metadata.getTier(); } - public boolean segmentReplicatable() + public boolean isSegmentReplicationTarget() { - return metadata.segmentReplicatable(); + return metadata.isSegmentReplicationTarget(); + } + + public boolean isSegmentBroadcastTarget() + { + return metadata.isSegmentBroadcastTarget(); + } + + public boolean isSegmentReplicationOrBroadcastTarget() + { + return metadata.isSegmentReplicationTarget() || metadata.isSegmentBroadcastTarget(); } @JsonProperty diff --git a/server/src/main/java/org/apache/druid/discovery/DruidNodeDiscoveryProvider.java b/server/src/main/java/org/apache/druid/discovery/DruidNodeDiscoveryProvider.java index 733202981839..898ce7c75956 100644 --- a/server/src/main/java/org/apache/druid/discovery/DruidNodeDiscoveryProvider.java +++ b/server/src/main/java/org/apache/druid/discovery/DruidNodeDiscoveryProvider.java @@ -44,7 +44,8 @@ public abstract class DruidNodeDiscoveryProvider private static final Map> SERVICE_TO_NODE_TYPES = ImmutableMap.of( LookupNodeService.DISCOVERY_SERVICE_KEY, ImmutableSet.of(NodeRole.BROKER, NodeRole.HISTORICAL, NodeRole.PEON, NodeRole.INDEXER), - DataNodeService.DISCOVERY_SERVICE_KEY, ImmutableSet.of(NodeRole.HISTORICAL, NodeRole.PEON, NodeRole.INDEXER), + DataNodeService.DISCOVERY_SERVICE_KEY, + ImmutableSet.of(NodeRole.HISTORICAL, NodeRole.PEON, NodeRole.INDEXER, NodeRole.BROKER), WorkerNodeService.DISCOVERY_SERVICE_KEY, ImmutableSet.of(NodeRole.MIDDLE_MANAGER, NodeRole.INDEXER) ); diff --git a/server/src/main/java/org/apache/druid/metadata/SQLMetadataStorageActionHandler.java b/server/src/main/java/org/apache/druid/metadata/SQLMetadataStorageActionHandler.java index eb9e9916cac8..e6fdcfb45f1a 100644 --- a/server/src/main/java/org/apache/druid/metadata/SQLMetadataStorageActionHandler.java +++ b/server/src/main/java/org/apache/druid/metadata/SQLMetadataStorageActionHandler.java @@ -22,7 +22,6 @@ import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Optional; import com.google.common.base.Throwables; import com.google.common.collect.Maps; @@ -173,8 +172,7 @@ public void insert( } } - @VisibleForTesting - protected static boolean isStatementException(Throwable e) + public static boolean isStatementException(Throwable e) { return e instanceof StatementException || (e instanceof CallbackFailedException && e.getCause() instanceof StatementException); diff --git a/server/src/main/java/org/apache/druid/metadata/input/InputSourceModule.java b/server/src/main/java/org/apache/druid/metadata/input/InputSourceModule.java new file mode 100644 index 000000000000..0423af4f9a03 --- /dev/null +++ b/server/src/main/java/org/apache/druid/metadata/input/InputSourceModule.java @@ -0,0 +1,51 @@ +/* + * 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.metadata.input; + +import com.fasterxml.jackson.databind.Module; +import com.fasterxml.jackson.databind.jsontype.NamedType; +import com.fasterxml.jackson.databind.module.SimpleModule; +import com.google.common.collect.ImmutableList; +import com.google.inject.Binder; +import org.apache.druid.initialization.DruidModule; + +import java.util.List; + +/** + * Module that installs {@link org.apache.druid.data.input.InputSource} implementations + */ +public class InputSourceModule implements DruidModule +{ + @Override + public List getJacksonModules() + { + return ImmutableList.of( + new SimpleModule("InputSourceModule") + .registerSubtypes( + new NamedType(SqlInputSource.class, "sql") + ) + ); + } + + @Override + public void configure(Binder binder) + { + } +} diff --git a/server/src/main/java/org/apache/druid/metadata/input/SqlEntity.java b/server/src/main/java/org/apache/druid/metadata/input/SqlEntity.java new file mode 100644 index 000000000000..724077a4c0d2 --- /dev/null +++ b/server/src/main/java/org/apache/druid/metadata/input/SqlEntity.java @@ -0,0 +1,211 @@ +/* + * 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.metadata.input; + +import com.fasterxml.jackson.core.JsonGenerator; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Preconditions; +import org.apache.druid.data.input.InputEntity; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.metadata.SQLFirehoseDatabaseConnector; +import org.apache.druid.metadata.SQLMetadataStorageActionHandler; +import org.skife.jdbi.v2.ResultIterator; +import org.skife.jdbi.v2.exceptions.ResultSetException; + +import javax.annotation.Nullable; +import java.io.File; +import java.io.FileOutputStream; +import java.io.IOException; +import java.io.InputStream; +import java.net.URI; +import java.sql.ResultSetMetaData; +import java.sql.SQLException; +import java.util.HashMap; +import java.util.Map; + +/** + * Represents a rdbms based input resource and knows how to read query results from the resource using SQL queries. + */ +public class SqlEntity implements InputEntity +{ + private static final Logger LOG = new Logger(SqlEntity.class); + + private final String sql; + private final ObjectMapper objectMapper; + private final SQLFirehoseDatabaseConnector sqlFirehoseDatabaseConnector; + private final boolean foldCase; + + public SqlEntity( + String sql, + SQLFirehoseDatabaseConnector sqlFirehoseDatabaseConnector, + boolean foldCase, + ObjectMapper objectMapper + ) + { + this.sql = sql; + this.sqlFirehoseDatabaseConnector = Preconditions.checkNotNull( + sqlFirehoseDatabaseConnector, + "SQL Metadata Connector not configured!" + ); + this.foldCase = foldCase; + this.objectMapper = objectMapper; + } + + public String getSql() + { + return sql; + } + + @Nullable + @Override + public URI getUri() + { + return null; + } + + @Override + public InputStream open() + { + throw new UnsupportedOperationException("Please use fetch() instead"); + } + + @Override + public CleanableFile fetch(File temporaryDirectory, byte[] fetchBuffer) throws IOException + { + final File tempFile = File.createTempFile("druid-sql-entity", ".tmp", temporaryDirectory); + return openCleanableFile(sql, sqlFirehoseDatabaseConnector, objectMapper, foldCase, tempFile); + + } + + /** + * Executes a SQL query on the specified database and fetches the result into the given file. + * The result file is deleted if the query execution or the file write fails. + * + * @param sql The SQL query to be executed + * @param sqlFirehoseDatabaseConnector The database connector + * @param objectMapper An object mapper, used for deserialization + * @param foldCase A boolean flag used to enable or disabling case sensitivity while handling database column names + * + * @return A {@link InputEntity.CleanableFile} object that wraps the file containing the SQL results + */ + + public static CleanableFile openCleanableFile( + String sql, + SQLFirehoseDatabaseConnector sqlFirehoseDatabaseConnector, + ObjectMapper objectMapper, + boolean foldCase, + File tempFile + ) + throws IOException + { + try (FileOutputStream fos = new FileOutputStream(tempFile); + final JsonGenerator jg = objectMapper.getFactory().createGenerator(fos);) { + + // Execute the sql query and lazily retrieve the results into the file in json format. + // foldCase is useful to handle differences in case sensitivity behavior across databases. + sqlFirehoseDatabaseConnector.retryWithHandle( + (handle) -> { + ResultIterator> resultIterator = handle.createQuery( + sql + ).map( + (index, r, ctx) -> { + Map resultRow = foldCase ? new CaseFoldedMap() : new HashMap<>(); + ResultSetMetaData resultMetadata; + try { + resultMetadata = r.getMetaData(); + } + catch (SQLException e) { + throw new ResultSetException("Unable to obtain metadata from result set", e, ctx); + } + try { + for (int i = 1; i <= resultMetadata.getColumnCount(); i++) { + String key = resultMetadata.getColumnName(i); + String alias = resultMetadata.getColumnLabel(i); + Object value = r.getObject(i); + resultRow.put(alias != null ? alias : key, value); + } + } + catch (SQLException e) { + throw new ResultSetException("Unable to access specific metadata from " + + "result set metadata", e, ctx); + } + return resultRow; + } + ).iterator(); + jg.writeStartArray(); + while (resultIterator.hasNext()) { + jg.writeObject(resultIterator.next()); + } + jg.writeEndArray(); + jg.close(); + return null; + }, + (exception) -> sqlFirehoseDatabaseConnector.isTransientException(exception) + && !(SQLMetadataStorageActionHandler.isStatementException(exception)) + ); + return new CleanableFile() + { + @Override + public File file() + { + return tempFile; + } + + @Override + public void close() + { + if (!tempFile.delete()) { + LOG.warn("Failed to remove file[%s]", tempFile.getAbsolutePath()); + } + } + }; + } + catch (Exception e) { + if (!tempFile.delete()) { + LOG.warn("Failed to remove file[%s]", tempFile.getAbsolutePath()); + } + throw new IOException(e); + } + } + + private static class CaseFoldedMap extends HashMap + { + public static final long serialVersionUID = 1L; + + @Override + public Object get(Object obj) + { + return super.get(obj == null ? null : StringUtils.toLowerCase((String) obj)); + } + + @Override + public Object put(String key, Object value) + { + return super.put(key == null ? null : StringUtils.toLowerCase(key), value); + } + + @Override + public boolean containsKey(Object obj) + { + return super.containsKey(obj == null ? null : StringUtils.toLowerCase((String) obj)); + } + } +} diff --git a/server/src/main/java/org/apache/druid/metadata/input/SqlInputFormat.java b/server/src/main/java/org/apache/druid/metadata/input/SqlInputFormat.java new file mode 100644 index 000000000000..6d0aa59a20d2 --- /dev/null +++ b/server/src/main/java/org/apache/druid/metadata/input/SqlInputFormat.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.metadata.input; + +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.data.input.InputEntity; +import org.apache.druid.data.input.InputEntityReader; +import org.apache.druid.data.input.InputFormat; +import org.apache.druid.data.input.InputRowSchema; + +import java.io.File; + +public class SqlInputFormat implements InputFormat +{ + private final ObjectMapper objectMapper; + + public SqlInputFormat(ObjectMapper objectMapper) + { + this.objectMapper = objectMapper; + } + + @Override + public boolean isSplittable() + { + return true; + } + + @Override + public InputEntityReader createReader( + InputRowSchema inputRowSchema, + InputEntity source, + File temporaryDirectory + ) + { + return new SqlReader(inputRowSchema, source, temporaryDirectory, objectMapper); + } +} diff --git a/server/src/main/java/org/apache/druid/metadata/input/SqlInputSource.java b/server/src/main/java/org/apache/druid/metadata/input/SqlInputSource.java new file mode 100644 index 000000000000..c7dfbb7fa365 --- /dev/null +++ b/server/src/main/java/org/apache/druid/metadata/input/SqlInputSource.java @@ -0,0 +1,151 @@ +/* + * 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.metadata.input; + +import com.fasterxml.jackson.annotation.JacksonInject; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Preconditions; +import org.apache.druid.data.input.AbstractInputSource; +import org.apache.druid.data.input.InputFormat; +import org.apache.druid.data.input.InputRowSchema; +import org.apache.druid.data.input.InputSourceReader; +import org.apache.druid.data.input.InputSplit; +import org.apache.druid.data.input.SplitHintSpec; +import org.apache.druid.data.input.impl.InputEntityIteratingReader; +import org.apache.druid.data.input.impl.SplittableInputSource; +import org.apache.druid.guice.annotations.Smile; +import org.apache.druid.metadata.SQLFirehoseDatabaseConnector; + +import javax.annotation.Nullable; +import java.io.File; +import java.util.Collections; +import java.util.List; +import java.util.Objects; +import java.util.stream.Stream; + +public class SqlInputSource extends AbstractInputSource implements SplittableInputSource +{ + private final List sqls; + private final SQLFirehoseDatabaseConnector sqlFirehoseDatabaseConnector; + private final ObjectMapper objectMapper; + private final boolean foldCase; + + @JsonCreator + public SqlInputSource( + @JsonProperty("sqls") List sqls, + @JsonProperty("foldCase") boolean foldCase, + @JsonProperty("database") SQLFirehoseDatabaseConnector sqlFirehoseDatabaseConnector, + @JacksonInject @Smile ObjectMapper objectMapper + ) + { + Preconditions.checkArgument(sqls.size() > 0, "No SQL queries provided"); + + this.sqls = sqls; + this.foldCase = foldCase; + this.sqlFirehoseDatabaseConnector = Preconditions.checkNotNull( + sqlFirehoseDatabaseConnector, + "SQL Metadata Connector not configured!" + ); + this.objectMapper = objectMapper; + } + + @JsonProperty + public List getSqls() + { + return sqls; + } + + @JsonProperty + public boolean isFoldCase() + { + return foldCase; + } + + @JsonProperty("database") + public SQLFirehoseDatabaseConnector getSQLFirehoseDatabaseConnector() + { + return sqlFirehoseDatabaseConnector; + } + + @Override + public Stream> createSplits(InputFormat inputFormat, @Nullable SplitHintSpec splitHintSpec) + { + return sqls.stream().map(InputSplit::new); + } + + @Override + public int estimateNumSplits(InputFormat inputFormat, @Nullable SplitHintSpec splitHintSpec) + { + return sqls.size(); + } + + @Override + public SplittableInputSource withSplit(InputSplit split) + { + return new SqlInputSource( + Collections.singletonList(split.get()), + foldCase, + sqlFirehoseDatabaseConnector, + objectMapper + ); + } + + @Override + protected InputSourceReader fixedFormatReader(InputRowSchema inputRowSchema, @Nullable File temporaryDirectory) + { + final SqlInputFormat inputFormat = new SqlInputFormat(objectMapper); + return new InputEntityIteratingReader( + inputRowSchema, + inputFormat, + createSplits(inputFormat, null) + .map(split -> new SqlEntity(split.get(), sqlFirehoseDatabaseConnector, foldCase, objectMapper)).iterator(), + temporaryDirectory + ); + } + + @Override + public boolean needsFormat() + { + return false; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + SqlInputSource that = (SqlInputSource) o; + return foldCase == that.foldCase && + sqls.equals(that.sqls) && + sqlFirehoseDatabaseConnector.equals(that.sqlFirehoseDatabaseConnector); + } + + @Override + public int hashCode() + { + return Objects.hash(sqls, sqlFirehoseDatabaseConnector, foldCase); + } +} diff --git a/server/src/main/java/org/apache/druid/metadata/input/SqlReader.java b/server/src/main/java/org/apache/druid/metadata/input/SqlReader.java new file mode 100644 index 000000000000..4657158c0463 --- /dev/null +++ b/server/src/main/java/org/apache/druid/metadata/input/SqlReader.java @@ -0,0 +1,95 @@ +/* + * 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.metadata.input; + +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.data.input.InputEntity; +import org.apache.druid.data.input.InputRow; +import org.apache.druid.data.input.InputRowSchema; +import org.apache.druid.data.input.IntermediateRowParsingReader; +import org.apache.druid.data.input.impl.MapInputRowParser; +import org.apache.druid.data.input.impl.prefetch.JsonIterator; +import org.apache.druid.java.util.common.io.Closer; +import org.apache.druid.java.util.common.parsers.CloseableIterator; +import org.apache.druid.java.util.common.parsers.ParseException; + +import java.io.File; +import java.io.FileInputStream; +import java.io.IOException; +import java.util.Collections; +import java.util.List; +import java.util.Map; + +/** + * Reader exclusively for {@link SqlEntity} + */ +public class SqlReader extends IntermediateRowParsingReader> +{ + private final InputRowSchema inputRowSchema; + private final SqlEntity source; + private final File temporaryDirectory; + private final ObjectMapper objectMapper; + + + SqlReader( + InputRowSchema inputRowSchema, + InputEntity source, + File temporaryDirectory, + ObjectMapper objectMapper + ) + { + this.inputRowSchema = inputRowSchema; + this.source = (SqlEntity) source; + this.temporaryDirectory = temporaryDirectory; + this.objectMapper = objectMapper; + } + + @Override + protected CloseableIterator> intermediateRowIterator() throws IOException + { + final Closer closer = Closer.create(); + //The results are fetched into local storage as this avoids having to keep a persistent database connection for a long time + final InputEntity.CleanableFile resultFile = closer.register(source.fetch(temporaryDirectory, null)); + FileInputStream inputStream = new FileInputStream(resultFile.file()); + JsonIterator> jsonIterator = new JsonIterator<>(new TypeReference>() + { + }, inputStream, closer, objectMapper); + return jsonIterator; + } + + @Override + protected List parseInputRows(Map intermediateRow) throws ParseException + { + return Collections.singletonList( + MapInputRowParser.parse( + inputRowSchema.getTimestampSpec(), + inputRowSchema.getDimensionsSpec(), + intermediateRow + ) + ); + } + + @Override + protected Map toMap(Map intermediateRow) + { + return intermediateRow; + } +} diff --git a/server/src/main/java/org/apache/druid/segment/loading/SegmentLoaderConfig.java b/server/src/main/java/org/apache/druid/segment/loading/SegmentLoaderConfig.java index c6c57233738c..39b3bde3129d 100644 --- a/server/src/main/java/org/apache/druid/segment/loading/SegmentLoaderConfig.java +++ b/server/src/main/java/org/apache/druid/segment/loading/SegmentLoaderConfig.java @@ -23,9 +23,9 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.Lists; import org.apache.druid.utils.JvmUtils; -import org.hibernate.validator.constraints.NotEmpty; import java.io.File; +import java.util.Collections; import java.util.List; import java.util.concurrent.TimeUnit; @@ -34,8 +34,7 @@ public class SegmentLoaderConfig { @JsonProperty - @NotEmpty - private List locations = null; + private List locations = Collections.emptyList(); @JsonProperty("lazyLoadOnStart") private boolean lazyLoadOnStart = false; diff --git a/server/src/main/java/org/apache/druid/segment/loading/SegmentLoaderLocalCacheManager.java b/server/src/main/java/org/apache/druid/segment/loading/SegmentLoaderLocalCacheManager.java index 398ad679b484..b2ac7e8f35b0 100644 --- a/server/src/main/java/org/apache/druid/segment/loading/SegmentLoaderLocalCacheManager.java +++ b/server/src/main/java/org/apache/druid/segment/loading/SegmentLoaderLocalCacheManager.java @@ -89,7 +89,6 @@ public SegmentLoaderLocalCacheManager( this.indexIO = indexIO; this.config = config; this.jsonMapper = mapper; - this.locations = new ArrayList<>(); for (StorageLocationConfig locationConfig : config.getLocations()) { locations.add( diff --git a/server/src/main/java/org/apache/druid/segment/realtime/CliIndexerDataSegmentServerAnnouncerLifecycleHandler.java b/server/src/main/java/org/apache/druid/segment/realtime/CliIndexerDataSegmentServerAnnouncerLifecycleHandler.java deleted file mode 100644 index e874a30c86d9..000000000000 --- a/server/src/main/java/org/apache/druid/segment/realtime/CliIndexerDataSegmentServerAnnouncerLifecycleHandler.java +++ /dev/null @@ -1,104 +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.segment.realtime; - -import com.google.common.base.Throwables; -import com.google.inject.Inject; -import org.apache.druid.concurrent.LifecycleLock; -import org.apache.druid.guice.ManageLifecycle; -import org.apache.druid.java.util.common.lifecycle.LifecycleStart; -import org.apache.druid.java.util.common.lifecycle.LifecycleStop; -import org.apache.druid.java.util.emitter.EmittingLogger; -import org.apache.druid.server.coordination.DataSegmentServerAnnouncer; - -import java.io.IOException; - -/** - * Ties the {@link DataSegmentServerAnnouncer} announce/unannounce to the lifecycle start and stop. - * - * Analogous to {@link org.apache.druid.server.coordination.SegmentLoadDropHandler} on the Historicals, - * but without segment cache management. - */ -@ManageLifecycle -public class CliIndexerDataSegmentServerAnnouncerLifecycleHandler -{ - private static final EmittingLogger LOG = new EmittingLogger(CliIndexerDataSegmentServerAnnouncerLifecycleHandler.class); - - private final DataSegmentServerAnnouncer dataSegmentServerAnnouncer; - - private final LifecycleLock lifecycleLock = new LifecycleLock(); - - @Inject - public CliIndexerDataSegmentServerAnnouncerLifecycleHandler( - DataSegmentServerAnnouncer dataSegmentServerAnnouncer - ) - { - this.dataSegmentServerAnnouncer = dataSegmentServerAnnouncer; - } - - @LifecycleStart - public void start() throws IOException - { - if (!lifecycleLock.canStart()) { - throw new RuntimeException("Lifecycle lock could not start"); - } - - try { - if (lifecycleLock.isStarted()) { - return; - } - - LOG.info("Starting..."); - try { - dataSegmentServerAnnouncer.announce(); - } - catch (Exception e) { - Throwables.propagateIfPossible(e, IOException.class); - throw new RuntimeException(e); - } - LOG.info("Started."); - lifecycleLock.started(); - } - finally { - lifecycleLock.exitStart(); - } - } - - @LifecycleStop - public void stop() - { - if (!lifecycleLock.canStop()) { - throw new RuntimeException("Lifecycle lock could not stop"); - } - - if (!lifecycleLock.isStarted()) { - return; - } - - LOG.info("Stopping..."); - try { - dataSegmentServerAnnouncer.unannounce(); - } - catch (Exception e) { - throw new RuntimeException(e); - } - LOG.info("Stopped."); - } -} diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/SegmentPublisherHelper.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/SegmentPublisherHelper.java new file mode 100644 index 000000000000..76093beb8c8a --- /dev/null +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/SegmentPublisherHelper.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.segment.realtime.appenderator; + +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.partition.BuildingNumberedShardSpec; +import org.apache.druid.timeline.partition.OverwriteShardSpec; +import org.apache.druid.timeline.partition.ShardSpec; +import org.joda.time.Interval; + +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.function.Function; +import java.util.stream.Collectors; + +public final class SegmentPublisherHelper +{ + /** + * This method fills missing information in the shard spec if necessary when publishing segments. + * + * - When time chunk lock is used, the non-appending task should set the proper size of the core partitions for + * dynamically-partitioned segments. See {@link #annotateNumberedShardSpecFn}. + * - When segment lock is used, the overwriting task should set the proper size of the atomic update group. + * See {@link #annotateAtomicUpdateGroupFn}. + */ + static Set annotateShardSpec(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 ShardSpec firstShardSpec = segmentsPerInterval.get(0).getShardSpec(); + final boolean anyMismatch = segmentsPerInterval.stream().anyMatch( + segment -> segment.getShardSpec().getClass() != firstShardSpec.getClass() + ); + if (anyMismatch) { + throw new ISE( + "Mismatched shardSpecs in interval[%s] for segments[%s]", + interval, + segmentsPerInterval + ); + } + final Function annotateFn; + if (firstShardSpec instanceof OverwriteShardSpec) { + annotateFn = annotateAtomicUpdateGroupFn(segmentsPerInterval.size()); + } else if (firstShardSpec instanceof BuildingNumberedShardSpec) { + annotateFn = annotateNumberedShardSpecFn(segmentsPerInterval.size()); + } else { + annotateFn = null; + } + + if (annotateFn != null) { + intervalToSegments.put(interval, segmentsPerInterval.stream().map(annotateFn).collect(Collectors.toList())); + } + } + + return intervalToSegments.values().stream().flatMap(Collection::stream).collect(Collectors.toSet()); + } + + private static Function annotateAtomicUpdateGroupFn(int atomicUpdateGroupSize) + { + // The segments which are published together consist an atomicUpdateGroup. + return segment -> { + final OverwriteShardSpec shardSpec = (OverwriteShardSpec) segment.getShardSpec(); + return segment.withShardSpec(shardSpec.withAtomicUpdateGroupSize((short) atomicUpdateGroupSize)); + }; + } + + private static Function annotateNumberedShardSpecFn(int corePartitionSetSize) + { + return segment -> { + final BuildingNumberedShardSpec shardSpec = (BuildingNumberedShardSpec) segment.getShardSpec(); + return segment.withShardSpec(shardSpec.toNumberedShardSpec(corePartitionSetSize)); + }; + } + + private SegmentPublisherHelper() + { + } +} 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 cb9b9ffb4448..067e4f9b5cdd 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 @@ -63,7 +63,7 @@ default SegmentPublishResult publishSegments( { return publishAnnotatedSegments( segmentsToBeOverwritten, - annotateAtomicUpdateGroupSize(segmentsToPublish), + SegmentPublisherHelper.annotateShardSpec(segmentsToPublish), commitMetadata ); } @@ -76,52 +76,4 @@ default boolean supportsEmptyPublish() { return false; } - - 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 OverwriteShardSpec; - - final boolean anyMismatch = segmentsPerInterval.stream().anyMatch( - segment -> (segment.getShardSpec() instanceof OverwriteShardSpec) != 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 OverwriteShardSpec)) - .collect(Collectors.toList()), - segmentsPerInterval.stream() - .filter(segment -> segment.getShardSpec() instanceof OverwriteShardSpec) - .collect(Collectors.toList()) - ); - } - - if (isNonFirstGeneration) { - // The segments which are published together consist an atomicUpdateGroup. - - intervalToSegments.put( - interval, - segmentsPerInterval - .stream() - .map(segment -> { - final OverwriteShardSpec shardSpec = (OverwriteShardSpec) 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/firehose/SqlFirehoseFactory.java b/server/src/main/java/org/apache/druid/segment/realtime/firehose/SqlFirehoseFactory.java index 9daa231f9b7e..0b5863d671d6 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/firehose/SqlFirehoseFactory.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/firehose/SqlFirehoseFactory.java @@ -22,32 +22,23 @@ import com.fasterxml.jackson.annotation.JacksonInject; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import com.fasterxml.jackson.core.JsonGenerator; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Preconditions; import org.apache.druid.data.input.FiniteFirehoseFactory; import org.apache.druid.data.input.InputSplit; import org.apache.druid.data.input.impl.InputRowParser; import org.apache.druid.guice.annotations.Smile; -import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.metadata.MetadataStorageConnectorConfig; import org.apache.druid.metadata.SQLFirehoseDatabaseConnector; -import org.skife.jdbi.v2.ResultIterator; -import org.skife.jdbi.v2.exceptions.CallbackFailedException; -import org.skife.jdbi.v2.exceptions.ResultSetException; -import org.skife.jdbi.v2.exceptions.StatementException; +import org.apache.druid.metadata.input.SqlEntity; import javax.annotation.Nullable; import java.io.File; import java.io.FileInputStream; -import java.io.FileOutputStream; import java.io.IOException; import java.io.InputStream; -import java.sql.ResultSetMetaData; -import java.sql.SQLException; import java.util.Collection; import java.util.Collections; -import java.util.HashMap; import java.util.List; import java.util.Map; @@ -86,7 +77,10 @@ public SqlFirehoseFactory( this.sqls = sqls; this.objectMapper = objectMapper; - this.sqlFirehoseDatabaseConnector = sqlFirehoseDatabaseConnector; + this.sqlFirehoseDatabaseConnector = Preconditions.checkNotNull( + sqlFirehoseDatabaseConnector, + "SQL Metadata Connector not configured!" + ); this.foldCase = foldCase; this.connectorConfig = null; } @@ -94,79 +88,8 @@ public SqlFirehoseFactory( @Override protected InputStream openObjectStream(String sql, File fileName) throws IOException { - Preconditions.checkNotNull(sqlFirehoseDatabaseConnector, "SQL Metadata Connector not configured!"); - try (FileOutputStream fos = new FileOutputStream(fileName)) { - final JsonGenerator jg = objectMapper.getFactory().createGenerator(fos); - sqlFirehoseDatabaseConnector.retryWithHandle( - (handle) -> { - ResultIterator> resultIterator = handle.createQuery( - sql - ).map( - (index, r, ctx) -> { - Map resultRow = foldCase ? new CaseFoldedMap() : new HashMap<>(); - ResultSetMetaData resultMetadata; - try { - resultMetadata = r.getMetaData(); - } - catch (SQLException e) { - throw new ResultSetException("Unable to obtain metadata from result set", e, ctx); - } - try { - for (int i = 1; i <= resultMetadata.getColumnCount(); i++) { - String key = resultMetadata.getColumnName(i); - String alias = resultMetadata.getColumnLabel(i); - Object value = r.getObject(i); - resultRow.put(alias != null ? alias : key, value); - } - } - catch (SQLException e) { - throw new ResultSetException("Unable to access specific metadata from " + - "result set metadata", e, ctx); - } - return resultRow; - } - ).iterator(); - jg.writeStartArray(); - while (resultIterator.hasNext()) { - jg.writeObject(resultIterator.next()); - } - jg.writeEndArray(); - jg.close(); - return null; - }, - (exception) -> { - final boolean isStatementException = exception instanceof StatementException || - (exception instanceof CallbackFailedException - && exception.getCause() instanceof StatementException); - return sqlFirehoseDatabaseConnector.isTransientException(exception) && !(isStatementException); - } - ); - } + SqlEntity.openCleanableFile(sql, sqlFirehoseDatabaseConnector, objectMapper, foldCase, fileName); return new FileInputStream(fileName); - - } - - private static class CaseFoldedMap extends HashMap - { - public static final long serialVersionUID = 1L; - - @Override - public Object get(Object obj) - { - return super.get(StringUtils.toLowerCase((String) obj)); - } - - @Override - public Object put(String key, Object value) - { - return super.put(StringUtils.toLowerCase(key), value); - } - - @Override - public boolean containsKey(Object obj) - { - return super.containsKey(StringUtils.toLowerCase((String) obj)); - } } @Override 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..2e97258a52da 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 @@ -142,7 +142,7 @@ static boolean isHandOffComplete(List serverView, Segm && segmentLoadInfo.getSegment().getShardSpec().getPartitionNum() == descriptor.getPartitionNumber() && segmentLoadInfo.getSegment().getVersion().compareTo(descriptor.getVersion()) >= 0 - && segmentLoadInfo.getServers().stream().anyMatch(DruidServerMetadata::segmentReplicatable)) { + && segmentLoadInfo.getServers().stream().anyMatch(DruidServerMetadata::isSegmentReplicationOrBroadcastTarget)) { return true; } } diff --git a/server/src/main/java/org/apache/druid/server/coordination/DruidServerMetadata.java b/server/src/main/java/org/apache/druid/server/coordination/DruidServerMetadata.java index e3673bbc9cae..3fda41b08dab 100644 --- a/server/src/main/java/org/apache/druid/server/coordination/DruidServerMetadata.java +++ b/server/src/main/java/org/apache/druid/server/coordination/DruidServerMetadata.java @@ -107,11 +107,21 @@ public int getPriority() return priority; } - public boolean segmentReplicatable() + public boolean isSegmentReplicationTarget() { return type.isSegmentReplicationTarget(); } + public boolean isSegmentBroadcastTarget() + { + return type.isSegmentBroadcastTarget(); + } + + public boolean isSegmentReplicationOrBroadcastTarget() + { + return isSegmentReplicationTarget() || isSegmentBroadcastTarget(); + } + @Override public boolean equals(Object o) { diff --git a/server/src/main/java/org/apache/druid/server/coordination/LoadableDataSegment.java b/server/src/main/java/org/apache/druid/server/coordination/LoadableDataSegment.java new file mode 100644 index 000000000000..4f4f7a5b1d19 --- /dev/null +++ b/server/src/main/java/org/apache/druid/server/coordination/LoadableDataSegment.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.server.coordination; + +import com.fasterxml.jackson.annotation.JacksonInject; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.databind.annotation.JsonDeserialize; +import org.apache.druid.jackson.CommaListJoinDeserializer; +import org.apache.druid.timeline.CompactionState; +import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.partition.ShardSpec; +import org.joda.time.Interval; + +import javax.annotation.Nullable; +import java.util.List; +import java.util.Map; + +/** + * A deserialization aid used by {@link SegmentChangeRequestLoad}. The broker prunes the loadSpec from segments + * for efficiency reasons, but the broker does need the loadSpec when it loads broadcast segments. + * + * This class always uses the non-pruning default {@link PruneSpecsHolder}. + */ +public class LoadableDataSegment extends DataSegment +{ + @JsonCreator + public LoadableDataSegment( + @JsonProperty("dataSource") String dataSource, + @JsonProperty("interval") Interval interval, + @JsonProperty("version") String version, + // use `Map` *NOT* `LoadSpec` because we want to do lazy materialization to prevent dependency pollution + @JsonProperty("loadSpec") @Nullable Map loadSpec, + @JsonProperty("dimensions") + @JsonDeserialize(using = CommaListJoinDeserializer.class) + @Nullable + List dimensions, + @JsonProperty("metrics") + @JsonDeserialize(using = CommaListJoinDeserializer.class) + @Nullable + List metrics, + @JsonProperty("shardSpec") @Nullable ShardSpec shardSpec, + @JsonProperty("lastCompactionState") @Nullable CompactionState lastCompactionState, + @JsonProperty("binaryVersion") Integer binaryVersion, + @JsonProperty("size") long size, + @JacksonInject PruneSpecsHolder pruneSpecsHolder + ) + { + super( + dataSource, + interval, + version, + loadSpec, + dimensions, + metrics, + shardSpec, + lastCompactionState, + binaryVersion, + size, + PruneSpecsHolder.DEFAULT + ); + + } +} diff --git a/server/src/main/java/org/apache/druid/server/coordination/SegmentChangeRequestLoad.java b/server/src/main/java/org/apache/druid/server/coordination/SegmentChangeRequestLoad.java index 097e02523032..130c7b50d80c 100644 --- a/server/src/main/java/org/apache/druid/server/coordination/SegmentChangeRequestLoad.java +++ b/server/src/main/java/org/apache/druid/server/coordination/SegmentChangeRequestLoad.java @@ -35,14 +35,26 @@ public class SegmentChangeRequestLoad implements DataSegmentChangeRequest { private final DataSegment segment; + /** + * To avoid pruning of the loadSpec on the broker, needed when the broker is loading broadcast segments, + * we deserialize into an {@link LoadableDataSegment}, which never removes the loadSpec. + */ @JsonCreator public SegmentChangeRequestLoad( - @JsonUnwrapped DataSegment segment + @JsonUnwrapped LoadableDataSegment segment ) { this.segment = segment; } + public SegmentChangeRequestLoad( + DataSegment segment + ) + { + this.segment = segment; + } + + @Override public void go(DataSegmentChangeHandler handler, @Nullable DataSegmentChangeCallback callback) { diff --git a/server/src/main/java/org/apache/druid/server/coordination/SegmentLoadDropHandler.java b/server/src/main/java/org/apache/druid/server/coordination/SegmentLoadDropHandler.java index 51aefc1b8d67..87a19365e6f5 100644 --- a/server/src/main/java/org/apache/druid/server/coordination/SegmentLoadDropHandler.java +++ b/server/src/main/java/org/apache/druid/server/coordination/SegmentLoadDropHandler.java @@ -34,6 +34,8 @@ import com.google.common.util.concurrent.SettableFuture; import com.google.inject.Inject; import org.apache.druid.guice.ManageLifecycle; +import org.apache.druid.guice.ServerTypeConfig; +import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.common.lifecycle.LifecycleStart; @@ -104,7 +106,8 @@ public SegmentLoadDropHandler( SegmentLoaderConfig config, DataSegmentAnnouncer announcer, DataSegmentServerAnnouncer serverAnnouncer, - SegmentManager segmentManager + SegmentManager segmentManager, + ServerTypeConfig serverTypeConfig ) { this( @@ -116,7 +119,8 @@ public SegmentLoadDropHandler( Executors.newScheduledThreadPool( config.getNumLoadingThreads(), Execs.makeThreadFactory("SimpleDataSegmentChangeHandler-%s") - ) + ), + serverTypeConfig ); } @@ -127,7 +131,8 @@ public SegmentLoadDropHandler( DataSegmentAnnouncer announcer, DataSegmentServerAnnouncer serverAnnouncer, SegmentManager segmentManager, - ScheduledExecutorService exec + ScheduledExecutorService exec, + ServerTypeConfig serverTypeConfig ) { this.jsonMapper = jsonMapper; @@ -139,6 +144,13 @@ public SegmentLoadDropHandler( this.exec = exec; this.segmentsToDelete = new ConcurrentSkipListSet<>(); + if (config.getLocations().isEmpty()) { + if (ServerType.HISTORICAL.equals(serverTypeConfig.getServerType())) { + throw new IAE("Segment cache locations must be set on historicals."); + } else { + log.info("Not starting SegmentLoadDropHandler with empty segment cache locations."); + } + } requestStatuses = CacheBuilder.newBuilder().maximumSize(config.getStatusQueueMaxSize()).initialCapacity(8).build(); } @@ -152,8 +164,10 @@ public void start() throws IOException log.info("Starting..."); try { - loadLocalCache(); - serverAnnouncer.announce(); + if (!config.getLocations().isEmpty()) { + loadLocalCache(); + serverAnnouncer.announce(); + } } catch (Exception e) { Throwables.propagateIfPossible(e, IOException.class); @@ -174,7 +188,9 @@ public void stop() log.info("Stopping..."); try { - serverAnnouncer.unannounce(); + if (!config.getLocations().isEmpty()) { + serverAnnouncer.unannounce(); + } } catch (Exception e) { throw new RuntimeException(e); diff --git a/server/src/main/java/org/apache/druid/server/coordination/ServerType.java b/server/src/main/java/org/apache/druid/server/coordination/ServerType.java index 42fb65a3fdfb..0b860a1b0afe 100644 --- a/server/src/main/java/org/apache/druid/server/coordination/ServerType.java +++ b/server/src/main/java/org/apache/druid/server/coordination/ServerType.java @@ -63,6 +63,14 @@ public boolean isSegmentReplicationTarget() { return false; } + }, + + BROKER { + @Override + public boolean isSegmentReplicationTarget() + { + return false; + } }; /** diff --git a/server/src/main/java/org/apache/druid/server/coordinator/BalancerStrategy.java b/server/src/main/java/org/apache/druid/server/coordinator/BalancerStrategy.java index d9fea81e3a95..889c167c8ef5 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/BalancerStrategy.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/BalancerStrategy.java @@ -26,6 +26,7 @@ import java.util.Iterator; import java.util.List; import java.util.NavigableSet; +import java.util.Set; /** * This interface describes the coordinator balancing strategy, which is responsible for making decisions on where @@ -56,11 +57,17 @@ public interface BalancerStrategy /** * Pick the best segment to move from one of the supplied set of servers according to the balancing strategy. * @param serverHolders set of historicals to consider for moving segments + * @param broadcastDatasources Datasources that contain segments which were loaded via broadcast rules. + * Balancing strategies should avoid rebalancing segments for such datasources, since + * they should be loaded on all servers anyway. + * NOTE: this should really be handled on a per-segment basis, to properly support + * the interval or period-based broadcast rules. For simplicity of the initial + * implementation, only forever broadcast rules are supported. * @return {@link BalancerSegmentHolder} containing segment to move and server it currently resides on, or null if * there are no segments to pick from (i. e. all provided serverHolders are empty). */ @Nullable - BalancerSegmentHolder pickSegmentToMove(List serverHolders); + BalancerSegmentHolder pickSegmentToMove(List serverHolders, Set broadcastDatasources); /** * Returns an iterator for a set of servers to drop from, ordered by preference of which server to drop from first diff --git a/server/src/main/java/org/apache/druid/server/coordinator/CachingCostBalancerStrategyFactory.java b/server/src/main/java/org/apache/druid/server/coordinator/CachingCostBalancerStrategyFactory.java index 4a1989df24df..1741087e8c5c 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/CachingCostBalancerStrategyFactory.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/CachingCostBalancerStrategyFactory.java @@ -71,7 +71,7 @@ public CachingCostBalancerStrategyFactory( @Override public ServerView.CallbackAction segmentAdded(DruidServerMetadata server, DataSegment segment) { - if (server.segmentReplicatable()) { + if (server.isSegmentReplicationTarget()) { clusterCostCacheBuilder.addSegment(server.getName(), segment); } return ServerView.CallbackAction.CONTINUE; @@ -80,7 +80,7 @@ public ServerView.CallbackAction segmentAdded(DruidServerMetadata server, DataSe @Override public ServerView.CallbackAction segmentRemoved(DruidServerMetadata server, DataSegment segment) { - if (server.segmentReplicatable()) { + if (server.isSegmentReplicationTarget()) { clusterCostCacheBuilder.removeSegment(server.getName(), segment); } return ServerView.CallbackAction.CONTINUE; @@ -98,7 +98,7 @@ public ServerView.CallbackAction segmentViewInitialized() serverInventoryView.registerServerRemovedCallback( executor, server -> { - if (server.segmentReplicatable()) { + if (server.isSegmentReplicationTarget()) { clusterCostCacheBuilder.removeServer(server.getName()); } return ServerView.CallbackAction.CONTINUE; diff --git a/server/src/main/java/org/apache/druid/server/coordinator/CostBalancerStrategy.java b/server/src/main/java/org/apache/druid/server/coordinator/CostBalancerStrategy.java index 4fd4164f3001..5d656d643f99 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/CostBalancerStrategy.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/CostBalancerStrategy.java @@ -35,6 +35,7 @@ import java.util.Iterator; import java.util.List; import java.util.NavigableSet; +import java.util.Set; import java.util.concurrent.ThreadLocalRandom; import java.util.stream.Collectors; @@ -211,9 +212,12 @@ static double computeJointSegmentsCost(final DataSegment segment, final Iterable @Override - public BalancerSegmentHolder pickSegmentToMove(final List serverHolders) + public BalancerSegmentHolder pickSegmentToMove( + final List serverHolders, + Set broadcastDatasources + ) { - return ReservoirSegmentSampler.getRandomBalancerSegmentHolder(serverHolders); + return ReservoirSegmentSampler.getRandomBalancerSegmentHolder(serverHolders, broadcastDatasources); } @Override diff --git a/server/src/main/java/org/apache/druid/server/coordinator/DruidCluster.java b/server/src/main/java/org/apache/druid/server/coordinator/DruidCluster.java index 318f663609b0..8fb4ccb056d4 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/DruidCluster.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/DruidCluster.java @@ -47,24 +47,28 @@ public class DruidCluster @VisibleForTesting static DruidCluster createDruidClusterFromBuilderInTest( @Nullable Set realtimes, - Map> historicals + Map> historicals, + @Nullable Set brokers ) { - return new DruidCluster(realtimes, historicals); + return new DruidCluster(realtimes, historicals, brokers); } private final Set realtimes; private final Map> historicals; + private final Set brokers; public DruidCluster() { this.realtimes = new HashSet<>(); this.historicals = new HashMap<>(); + this.brokers = new HashSet<>(); } private DruidCluster( @Nullable Set realtimes, - Map> historicals + Map> historicals, + @Nullable Set brokers ) { this.realtimes = realtimes == null ? new HashSet<>() : new HashSet<>(realtimes); @@ -72,6 +76,7 @@ private DruidCluster( historicals, holders -> CollectionUtils.newTreeSet(Comparator.reverseOrder(), holders) ); + this.brokers = brokers == null ? new HashSet<>() : new HashSet<>(brokers); } public void add(ServerHolder serverHolder) @@ -87,7 +92,11 @@ public void add(ServerHolder serverHolder) addHistorical(serverHolder); break; case INDEXER_EXECUTOR: - throw new IAE("unsupported server type[%s]", serverHolder.getServer().getType()); + addRealtime(serverHolder); + break; + case BROKER: + addBroker(serverHolder); + break; default: throw new IAE("unknown server type[%s]", serverHolder.getServer().getType()); } @@ -108,6 +117,11 @@ private void addHistorical(ServerHolder serverHolder) tierServers.add(serverHolder); } + private void addBroker(ServerHolder serverHolder) + { + brokers.add(serverHolder); + } + public Set getRealtimes() { return realtimes; @@ -118,6 +132,12 @@ public Map> getHistoricals() return historicals; } + + public Set getBrokers() + { + return brokers; + } + public Iterable getTierNames() { return historicals.keySet(); @@ -135,6 +155,7 @@ public Collection getAllServers() final List allServers = new ArrayList<>(historicalSize + realtimeSize); historicals.values().forEach(allServers::addAll); + allServers.addAll(brokers); allServers.addAll(realtimes); return allServers; } @@ -146,7 +167,7 @@ public Iterable> getSortedHistoricalsByTier() public boolean isEmpty() { - return historicals.isEmpty() && realtimes.isEmpty(); + return historicals.isEmpty() && realtimes.isEmpty() && brokers.isEmpty(); } public boolean hasHistoricals() @@ -159,9 +180,19 @@ public boolean hasRealtimes() return !realtimes.isEmpty(); } + public boolean hasBrokers() + { + return !brokers.isEmpty(); + } + public boolean hasTier(String tier) { - NavigableSet servers = historicals.get(tier); - return (servers != null) && !servers.isEmpty(); + NavigableSet historicalServers = historicals.get(tier); + boolean historicalsHasTier = (historicalServers != null) && !historicalServers.isEmpty(); + if (historicalsHasTier) { + return true; + } + + return false; } } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java b/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java index f8c3f43c76f5..36a414e780d9 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java @@ -761,7 +761,7 @@ List prepareCurrentServers() List currentServers = serverInventoryView .getInventory() .stream() - .filter(DruidServer::segmentReplicatable) + .filter(DruidServer::isSegmentReplicationOrBroadcastTarget) .map(DruidServer::toImmutableDruidServer) .collect(Collectors.toList()); 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 7bae11e61db0..3337b8e7647a 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 @@ -34,7 +34,9 @@ import java.util.Arrays; import java.util.Collection; import java.util.HashMap; +import java.util.HashSet; import java.util.Map; +import java.util.Set; import java.util.TreeSet; import java.util.concurrent.TimeUnit; @@ -70,6 +72,7 @@ private static TreeSet createUsedSegmentsSet(Iterable private final CoordinatorStats stats; private final DateTime balancerReferenceTimestamp; private final BalancerStrategy balancerStrategy; + private final Set broadcastDatasources; private DruidCoordinatorRuntimeParams( long startTimeNanos, @@ -85,7 +88,8 @@ private DruidCoordinatorRuntimeParams( CoordinatorCompactionConfig coordinatorCompactionConfig, CoordinatorStats stats, DateTime balancerReferenceTimestamp, - BalancerStrategy balancerStrategy + BalancerStrategy balancerStrategy, + Set broadcastDatasources ) { this.startTimeNanos = startTimeNanos; @@ -102,6 +106,7 @@ private DruidCoordinatorRuntimeParams( this.stats = stats; this.balancerReferenceTimestamp = balancerReferenceTimestamp; this.balancerStrategy = balancerStrategy; + this.broadcastDatasources = broadcastDatasources; } public long getStartTimeNanos() @@ -180,6 +185,11 @@ public BalancerStrategy getBalancerStrategy() return balancerStrategy; } + public Set getBroadcastDatasources() + { + return broadcastDatasources; + } + public boolean coordinatorIsLeadingEnoughTimeToMarkAsUnusedOvershadowedSegements() { long nanosElapsedSinceCoordinatorStart = System.nanoTime() - getStartTimeNanos(); @@ -256,6 +266,7 @@ public static class Builder private CoordinatorStats stats; private DateTime balancerReferenceTimestamp; private BalancerStrategy balancerStrategy; + private Set broadcastDatasources; private Builder() { @@ -272,6 +283,7 @@ private Builder() this.coordinatorDynamicConfig = CoordinatorDynamicConfig.builder().build(); this.coordinatorCompactionConfig = CoordinatorCompactionConfig.empty(); this.balancerReferenceTimestamp = DateTimes.nowUtc(); + this.broadcastDatasources = new HashSet<>(); } Builder( @@ -324,7 +336,8 @@ public DruidCoordinatorRuntimeParams build() coordinatorCompactionConfig, stats, balancerReferenceTimestamp, - balancerStrategy + balancerStrategy, + broadcastDatasources ); } @@ -436,5 +449,11 @@ public Builder withBalancerStrategy(BalancerStrategy balancerStrategy) this.balancerStrategy = balancerStrategy; return this; } + + public Builder withBroadcastDatasources(Set broadcastDatasources) + { + this.broadcastDatasources = broadcastDatasources; + return this; + } } } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/RandomBalancerStrategy.java b/server/src/main/java/org/apache/druid/server/coordinator/RandomBalancerStrategy.java index 8b0b30698175..72fdedf6e453 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/RandomBalancerStrategy.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/RandomBalancerStrategy.java @@ -26,6 +26,7 @@ import java.util.Iterator; import java.util.List; import java.util.NavigableSet; +import java.util.Set; import java.util.concurrent.ThreadLocalRandom; public class RandomBalancerStrategy implements BalancerStrategy @@ -51,9 +52,9 @@ public ServerHolder findNewSegmentHomeBalancer(DataSegment proposalSegment, List } @Override - public BalancerSegmentHolder pickSegmentToMove(List serverHolders) + public BalancerSegmentHolder pickSegmentToMove(List serverHolders, Set broadcastDatasources) { - return ReservoirSegmentSampler.getRandomBalancerSegmentHolder(serverHolders); + return ReservoirSegmentSampler.getRandomBalancerSegmentHolder(serverHolders, broadcastDatasources); } @Override diff --git a/server/src/main/java/org/apache/druid/server/coordinator/ReservoirSegmentSampler.java b/server/src/main/java/org/apache/druid/server/coordinator/ReservoirSegmentSampler.java index c2c4a7ad10c4..7181d52e152a 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/ReservoirSegmentSampler.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/ReservoirSegmentSampler.java @@ -22,19 +22,33 @@ import org.apache.druid.timeline.DataSegment; import java.util.List; +import java.util.Set; import java.util.concurrent.ThreadLocalRandom; final class ReservoirSegmentSampler { - static BalancerSegmentHolder getRandomBalancerSegmentHolder(final List serverHolders) + static BalancerSegmentHolder getRandomBalancerSegmentHolder( + final List serverHolders, + Set broadcastDatasources + ) { ServerHolder fromServerHolder = null; DataSegment proposalSegment = null; int numSoFar = 0; for (ServerHolder server : serverHolders) { + if (!server.getServer().getType().isSegmentReplicationTarget()) { + // if the server only handles broadcast segments (which don't need to be rebalanced), we have nothing to do + continue; + } + for (DataSegment segment : server.getServer().iterateAllSegments()) { + if (broadcastDatasources.contains(segment.getDataSource())) { + // we don't need to rebalance segments that were assigned via broadcast rules + continue; + } + int randNum = ThreadLocalRandom.current().nextInt(numSoFar + 1); // w.p. 1 / (numSoFar+1), swap out the server and segment if (randNum == numSoFar) { diff --git a/server/src/main/java/org/apache/druid/server/coordinator/ServerHolder.java b/server/src/main/java/org/apache/druid/server/coordinator/ServerHolder.java index ba96566a4dfd..26fa9a54c7ff 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/ServerHolder.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/ServerHolder.java @@ -122,6 +122,11 @@ public boolean isLoadingSegment(DataSegment segment) return peon.getSegmentsToLoad().contains(segment); } + public boolean isDroppingSegment(DataSegment segment) + { + return peon.getSegmentsToDrop().contains(segment); + } + public int getNumberOfSegmentsInQueue() { return peon.getNumberOfSegmentsInQueue(); diff --git a/server/src/main/java/org/apache/druid/server/coordinator/duty/BalanceSegments.java b/server/src/main/java/org/apache/druid/server/coordinator/duty/BalanceSegments.java index d42ca635bd62..a1c5237ddd15 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/duty/BalanceSegments.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/duty/BalanceSegments.java @@ -187,7 +187,10 @@ private Pair balanceServers( //noinspection ForLoopThatDoesntUseLoopVariable for (int iter = 0; (moved + unmoved) < maxSegmentsToMove; ++iter) { - final BalancerSegmentHolder segmentToMoveHolder = strategy.pickSegmentToMove(toMoveFrom); + final BalancerSegmentHolder segmentToMoveHolder = strategy.pickSegmentToMove( + toMoveFrom, + params.getBroadcastDatasources() + ); if (segmentToMoveHolder == null) { log.info("All servers to move segments from are empty, ending run."); break; diff --git a/server/src/main/java/org/apache/druid/server/coordinator/duty/RunRules.java b/server/src/main/java/org/apache/druid/server/coordinator/duty/RunRules.java index 5288bb35858f..3dc7b4d2f918 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/duty/RunRules.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/duty/RunRules.java @@ -28,11 +28,13 @@ import org.apache.druid.server.coordinator.DruidCoordinator; import org.apache.druid.server.coordinator.DruidCoordinatorRuntimeParams; import org.apache.druid.server.coordinator.ReplicationThrottler; +import org.apache.druid.server.coordinator.rules.BroadcastDistributionRule; import org.apache.druid.server.coordinator.rules.Rule; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.SegmentId; import org.joda.time.DateTime; +import java.util.HashSet; import java.util.List; import java.util.Set; @@ -101,6 +103,7 @@ public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params) final List segmentsWithMissingRules = Lists.newArrayListWithCapacity(MAX_MISSING_RULES); int missingRules = 0; + final Set broadcastDatasources = new HashSet<>(); for (DataSegment segment : params.getUsedSegments()) { if (overshadowed.contains(segment.getId())) { // Skipping overshadowed segments @@ -112,6 +115,12 @@ public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params) if (rule.appliesTo(segment, now)) { stats.accumulate(rule.run(coordinator, paramsWithReplicationManager, segment)); foundMatchingRule = true; + + // The set of broadcast datasources is used by BalanceSegments, so it's important that RunRules + // executes before BalanceSegments + if (rule instanceof BroadcastDistributionRule) { + broadcastDatasources.add(segment.getDataSource()); + } break; } } @@ -131,6 +140,9 @@ public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params) .emit(); } - return params.buildFromExisting().withCoordinatorStats(stats).build(); + return params.buildFromExisting() + .withCoordinatorStats(stats) + .withBroadcastDatasources(broadcastDatasources) + .build(); } } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/rules/BroadcastDistributionRule.java b/server/src/main/java/org/apache/druid/server/coordinator/rules/BroadcastDistributionRule.java index 658171236adc..35ff39ea6505 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/rules/BroadcastDistributionRule.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/rules/BroadcastDistributionRule.java @@ -20,6 +20,7 @@ package org.apache.druid.server.coordinator.rules; import org.apache.druid.java.util.emitter.EmittingLogger; +import org.apache.druid.server.coordination.ServerType; import org.apache.druid.server.coordinator.CoordinatorStats; import org.apache.druid.server.coordinator.DruidCoordinator; import org.apache.druid.server.coordinator.DruidCoordinatorRuntimeParams; @@ -27,8 +28,8 @@ import org.apache.druid.timeline.DataSegment; import java.util.HashSet; -import java.util.List; import java.util.Set; +import java.util.stream.Collectors; public abstract class BroadcastDistributionRule implements Rule { @@ -37,30 +38,35 @@ public abstract class BroadcastDistributionRule implements Rule @Override public CoordinatorStats run(DruidCoordinator coordinator, DruidCoordinatorRuntimeParams params, DataSegment segment) { - // Find servers which holds the segments of co-located data source - final Set loadServerHolders = new HashSet<>(); final Set dropServerHolders = new HashSet<>(); - final List colocatedDataSources = getColocatedDataSources(); - if (colocatedDataSources == null || colocatedDataSources.isEmpty()) { - loadServerHolders.addAll(params.getDruidCluster().getAllServers()); - } else { - params.getDruidCluster().getAllServers().forEach( - eachHolder -> { - if (!eachHolder.isDecommissioning() - && colocatedDataSources.stream() - .anyMatch(source -> eachHolder.getServer().getDataSource(source) != null)) { - loadServerHolders.add(eachHolder); - } else if (eachHolder.isServingSegment(segment)) { - if (!eachHolder.getPeon().getSegmentsToDrop().contains(segment)) { - dropServerHolders.add(eachHolder); - } - } - } - ); - } - final CoordinatorStats stats = new CoordinatorStats(); + // Find servers where we need to load the broadcast segments + final Set loadServerHolders = + params.getDruidCluster().getAllServers() + .stream() + .filter( + (serverHolder) -> { + ServerType serverType = serverHolder.getServer().getType(); + if (!serverType.isSegmentBroadcastTarget()) { + return false; + } + + final boolean isServingSegment = + serverHolder.isServingSegment(segment); + + if (serverHolder.isDecommissioning()) { + if (isServingSegment && !serverHolder.isDroppingSegment(segment)) { + dropServerHolders.add(serverHolder); + } + return false; + } + return !isServingSegment && !serverHolder.isLoadingSegment(segment); + } + ) + .collect(Collectors.toSet()); + + final CoordinatorStats stats = new CoordinatorStats(); return stats.accumulate(assign(loadServerHolders, segment)) .accumulate(drop(dropServerHolders, segment)); } @@ -110,6 +116,4 @@ private CoordinatorStats drop( return stats; } - - public abstract List getColocatedDataSources(); } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/rules/ForeverBroadcastDistributionRule.java b/server/src/main/java/org/apache/druid/server/coordinator/rules/ForeverBroadcastDistributionRule.java index d095f1100aea..ef5094cbea4a 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/rules/ForeverBroadcastDistributionRule.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/rules/ForeverBroadcastDistributionRule.java @@ -25,21 +25,16 @@ import org.joda.time.DateTime; import org.joda.time.Interval; -import java.util.List; import java.util.Objects; public class ForeverBroadcastDistributionRule extends BroadcastDistributionRule { static final String TYPE = "broadcastForever"; - private final List colocatedDataSources; - @JsonCreator - public ForeverBroadcastDistributionRule( - @JsonProperty("colocatedDataSources") List colocatedDataSources - ) + public ForeverBroadcastDistributionRule() { - this.colocatedDataSources = colocatedDataSources; + } @Override @@ -49,13 +44,6 @@ public String getType() return TYPE; } - @Override - @JsonProperty - public List getColocatedDataSources() - { - return colocatedDataSources; - } - @Override public boolean appliesTo(DataSegment segment, DateTime referenceTimestamp) { @@ -79,13 +67,12 @@ public boolean equals(Object o) return false; } - ForeverBroadcastDistributionRule that = (ForeverBroadcastDistributionRule) o; - return Objects.equals(colocatedDataSources, that.colocatedDataSources); + return true; } @Override public int hashCode() { - return Objects.hash(getType(), colocatedDataSources); + return Objects.hash(getType()); } } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/rules/IntervalBroadcastDistributionRule.java b/server/src/main/java/org/apache/druid/server/coordinator/rules/IntervalBroadcastDistributionRule.java index c40dff7268aa..b1bf29eedd20 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/rules/IntervalBroadcastDistributionRule.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/rules/IntervalBroadcastDistributionRule.java @@ -25,23 +25,19 @@ import org.joda.time.DateTime; import org.joda.time.Interval; -import java.util.List; import java.util.Objects; public class IntervalBroadcastDistributionRule extends BroadcastDistributionRule { static final String TYPE = "broadcastByInterval"; private final Interval interval; - private final List colocatedDataSources; @JsonCreator public IntervalBroadcastDistributionRule( - @JsonProperty("interval") Interval interval, - @JsonProperty("colocatedDataSources") List colocatedDataSources + @JsonProperty("interval") Interval interval ) { this.interval = interval; - this.colocatedDataSources = colocatedDataSources; } @Override @@ -51,13 +47,6 @@ public String getType() return TYPE; } - @Override - @JsonProperty - public List getColocatedDataSources() - { - return colocatedDataSources; - } - @Override public boolean appliesTo(DataSegment segment, DateTime referenceTimestamp) { @@ -79,26 +68,19 @@ public Interval getInterval() @Override public boolean equals(Object o) { - if (o == this) { + if (this == o) { return true; } - - if (o == null || o.getClass() != getClass()) { + if (o == null || getClass() != o.getClass()) { return false; } - IntervalBroadcastDistributionRule that = (IntervalBroadcastDistributionRule) o; - - if (!Objects.equals(interval, that.interval)) { - return false; - } - - return Objects.equals(colocatedDataSources, that.colocatedDataSources); + return Objects.equals(getInterval(), that.getInterval()); } @Override public int hashCode() { - return Objects.hash(getType(), interval, colocatedDataSources); + return Objects.hash(getInterval()); } } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/rules/PeriodBroadcastDistributionRule.java b/server/src/main/java/org/apache/druid/server/coordinator/rules/PeriodBroadcastDistributionRule.java index 97c6e11cfba2..d48353d3e50a 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/rules/PeriodBroadcastDistributionRule.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/rules/PeriodBroadcastDistributionRule.java @@ -26,7 +26,6 @@ import org.joda.time.Interval; import org.joda.time.Period; -import java.util.List; import java.util.Objects; public class PeriodBroadcastDistributionRule extends BroadcastDistributionRule @@ -36,18 +35,15 @@ public class PeriodBroadcastDistributionRule extends BroadcastDistributionRule private final Period period; private final boolean includeFuture; - private final List colocatedDataSources; @JsonCreator public PeriodBroadcastDistributionRule( @JsonProperty("period") Period period, - @JsonProperty("includeFuture") Boolean includeFuture, - @JsonProperty("colocatedDataSources") List colocatedDataSources + @JsonProperty("includeFuture") Boolean includeFuture ) { this.period = period; this.includeFuture = includeFuture == null ? DEFAULT_INCLUDE_FUTURE : includeFuture; - this.colocatedDataSources = colocatedDataSources; } @Override @@ -57,13 +53,6 @@ public String getType() return TYPE; } - @Override - @JsonProperty - public List getColocatedDataSources() - { - return colocatedDataSources; - } - @Override public boolean appliesTo(DataSegment segment, DateTime referenceTimestamp) { @@ -94,25 +83,17 @@ public boolean equals(Object o) if (this == o) { return true; } - - if (o == null || o.getClass() != getClass()) { + if (o == null || getClass() != o.getClass()) { return false; } - PeriodBroadcastDistributionRule that = (PeriodBroadcastDistributionRule) o; - - if (!Objects.equals(period, that.period)) { - return false; - } - if (includeFuture != that.includeFuture) { - return false; - } - return Objects.equals(colocatedDataSources, that.colocatedDataSources); + return isIncludeFuture() == that.isIncludeFuture() && + Objects.equals(getPeriod(), that.getPeriod()); } @Override public int hashCode() { - return Objects.hash(getType(), period, colocatedDataSources); + return Objects.hash(getPeriod(), isIncludeFuture()); } } 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 040297521b80..b6d310f1ba73 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 @@ -769,7 +769,7 @@ static boolean isSegmentLoaded(Iterable servedSegments && segmentLoadInfo.getSegment().getShardSpec().getPartitionNum() == descriptor.getPartitionNumber() && segmentLoadInfo.getSegment().getVersion().compareTo(descriptor.getVersion()) >= 0 && Iterables.any( - segmentLoadInfo.getServers(), DruidServerMetadata::segmentReplicatable + segmentLoadInfo.getServers(), DruidServerMetadata::isSegmentReplicationTarget )) { return true; } diff --git a/server/src/test/java/org/apache/druid/metadata/input/InputSourceModuleTest.java b/server/src/test/java/org/apache/druid/metadata/input/InputSourceModuleTest.java new file mode 100644 index 000000000000..67126b0c7b2f --- /dev/null +++ b/server/src/test/java/org/apache/druid/metadata/input/InputSourceModuleTest.java @@ -0,0 +1,62 @@ +/* + * 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.metadata.input; + +import com.fasterxml.jackson.databind.Module; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.cfg.MapperConfig; +import com.fasterxml.jackson.databind.introspect.AnnotatedClass; +import com.fasterxml.jackson.databind.introspect.AnnotatedClassResolver; +import com.fasterxml.jackson.databind.jsontype.NamedType; +import com.google.common.collect.Iterables; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.util.List; +import java.util.stream.Collectors; + +public class InputSourceModuleTest +{ + private final ObjectMapper mapper = new ObjectMapper(); + private final String SQL_NAMED_TYPE = "sql"; + + @Before + public void setUp() + { + for (Module jacksonModule : new InputSourceModule().getJacksonModules()) { + mapper.registerModule(jacksonModule); + } + } + + @Test + public void testSubTypeRegistration() + { + MapperConfig config = mapper.getDeserializationConfig(); + AnnotatedClass annotatedClass = AnnotatedClassResolver.resolveWithoutSuperTypes(config, SqlInputSource.class); + List subtypes = mapper.getSubtypeResolver() + .collectAndResolveSubtypesByClass(config, annotatedClass) + .stream() + .map(NamedType::getName) + .collect(Collectors.toList()); + Assert.assertNotNull(subtypes); + Assert.assertEquals(SQL_NAMED_TYPE, Iterables.getOnlyElement(subtypes)); + } +} diff --git a/server/src/test/java/org/apache/druid/metadata/input/SqlEntityTest.java b/server/src/test/java/org/apache/druid/metadata/input/SqlEntityTest.java new file mode 100644 index 000000000000..46a171b50c92 --- /dev/null +++ b/server/src/test/java/org/apache/druid/metadata/input/SqlEntityTest.java @@ -0,0 +1,134 @@ +/* + * 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.metadata.input; + +import com.fasterxml.jackson.databind.Module; +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.commons.io.IOUtils; +import org.apache.druid.data.input.InputEntity; +import org.apache.druid.metadata.TestDerbyConnector; +import org.apache.druid.segment.TestHelper; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; + +import java.io.File; +import java.io.FileInputStream; +import java.io.IOException; +import java.io.InputStream; +import java.nio.charset.StandardCharsets; + +public class SqlEntityTest +{ + @Rule + public final TestDerbyConnector.DerbyConnectorRule derbyConnectorRule = new TestDerbyConnector.DerbyConnectorRule(); + + private final ObjectMapper mapper = TestHelper.makeSmileMapper(); + private TestDerbyConnector derbyConnector; + String TABLE_NAME_1 = "FOOS_TABLE"; + + String VALID_SQL = "SELECT timestamp,a,b FROM FOOS_TABLE"; + String INVALID_SQL = "DONT SELECT timestamp,a,b FROM FOOS_TABLE"; + String resultJson = "[{\"a\":\"0\"," + + "\"b\":\"0\"," + + "\"timestamp\":\"2011-01-12T00:00:00.000Z\"" + + "}]"; + + @Before + public void setUp() + { + for (Module jacksonModule : new InputSourceModule().getJacksonModules()) { + mapper.registerModule(jacksonModule); + } + } + + @Test + public void testExecuteQuery() throws IOException + { + derbyConnector = derbyConnectorRule.getConnector(); + SqlTestUtils testUtils = new SqlTestUtils(derbyConnector); + testUtils.createAndUpdateTable(TABLE_NAME_1, 1); + File tmpFile = File.createTempFile( + "testQueryResults", + "" + ); + InputEntity.CleanableFile queryResult = SqlEntity.openCleanableFile( + VALID_SQL, + testUtils.getDerbyFirehoseConnector(), + mapper, + true, + tmpFile + ); + InputStream queryInputStream = new FileInputStream(queryResult.file()); + String actualJson = IOUtils.toString(queryInputStream, StandardCharsets.UTF_8); + + Assert.assertEquals(actualJson, resultJson); + testUtils.dropTable(TABLE_NAME_1); + } + + @Test(expected = IOException.class) + public void testFailOnInvalidQuery() throws IOException + { + derbyConnector = derbyConnectorRule.getConnector(); + SqlTestUtils testUtils = new SqlTestUtils(derbyConnector); + testUtils.createAndUpdateTable(TABLE_NAME_1, 1); + File tmpFile = File.createTempFile( + "testQueryResults", + "" + ); + InputEntity.CleanableFile queryResult = SqlEntity.openCleanableFile( + INVALID_SQL, + testUtils.getDerbyFirehoseConnector(), + mapper, + true, + tmpFile + ); + + Assert.assertTrue(tmpFile.exists()); + } + + @Test + public void testFileDeleteOnInvalidQuery() throws IOException + { + //The test parameters here are same as those used for testFailOnInvalidQuery(). + //The only difference is that this test checks if the temporary file is deleted upon failure. + derbyConnector = derbyConnectorRule.getConnector(); + SqlTestUtils testUtils = new SqlTestUtils(derbyConnector); + testUtils.createAndUpdateTable(TABLE_NAME_1, 1); + File tmpFile = File.createTempFile( + "testQueryResults", + "" + ); + try { + SqlEntity.openCleanableFile( + INVALID_SQL, + testUtils.getDerbyFirehoseConnector(), + mapper, + true, + tmpFile + ); + } + // Lets catch the exception so as to test temporary file deletion. + catch (IOException e) { + Assert.assertFalse(tmpFile.exists()); + } + } +} diff --git a/server/src/test/java/org/apache/druid/metadata/input/SqlInputSourceTest.java b/server/src/test/java/org/apache/druid/metadata/input/SqlInputSourceTest.java new file mode 100644 index 000000000000..7afa88894f26 --- /dev/null +++ b/server/src/test/java/org/apache/druid/metadata/input/SqlInputSourceTest.java @@ -0,0 +1,262 @@ +/* + * 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.metadata.input; + +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeName; +import com.fasterxml.jackson.databind.Module; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableList; +import nl.jqno.equalsverifier.EqualsVerifier; +import org.apache.commons.dbcp2.BasicDataSource; +import org.apache.commons.io.FileUtils; +import org.apache.druid.data.input.InputFormat; +import org.apache.druid.data.input.InputRow; +import org.apache.druid.data.input.InputRowSchema; +import org.apache.druid.data.input.InputSourceReader; +import org.apache.druid.data.input.InputSplit; +import org.apache.druid.data.input.Row; +import org.apache.druid.data.input.impl.DimensionsSpec; +import org.apache.druid.data.input.impl.TimestampSpec; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.parsers.CloseableIterator; +import org.apache.druid.metadata.MetadataStorageConnectorConfig; +import org.apache.druid.metadata.SQLFirehoseDatabaseConnector; +import org.apache.druid.metadata.TestDerbyConnector; +import org.apache.druid.segment.TestHelper; +import org.easymock.EasyMock; +import org.junit.AfterClass; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.skife.jdbi.v2.DBI; + +import java.io.File; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.Comparator; +import java.util.List; +import java.util.Objects; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +public class SqlInputSourceTest +{ + private static final List FIREHOSE_TMP_DIRS = new ArrayList<>(); + private final String TABLE_NAME_1 = "FOOS_TABLE_1"; + private final String TABLE_NAME_2 = "FOOS_TABLE_2"; + + private final List SQLLIST1 = ImmutableList.of("SELECT timestamp,a,b FROM FOOS_TABLE_1"); + private final List SQLLIST2 = ImmutableList.of( + "SELECT timestamp,a,b FROM FOOS_TABLE_1", + "SELECT timestamp,a,b FROM FOOS_TABLE_2" + ); + + private static final InputRowSchema INPUT_ROW_SCHEMA = new InputRowSchema( + new TimestampSpec("timestamp", "auto", null), + new DimensionsSpec( + DimensionsSpec.getDefaultSchemas(Arrays.asList("timestamp", "a", "b")), + new ArrayList<>(), + new ArrayList<>() + ), + Collections.emptyList() + ); + @Rule + public final TestDerbyConnector.DerbyConnectorRule derbyConnectorRule = new TestDerbyConnector.DerbyConnectorRule(); + private final ObjectMapper mapper = TestHelper.makeSmileMapper(); + private TestDerbyConnector derbyConnector; + + @Before + public void setUp() + { + for (Module jacksonModule : new InputSourceModule().getJacksonModules()) { + mapper.registerModule(jacksonModule); + } + } + + @AfterClass + public static void teardown() throws IOException + { + for (File dir : FIREHOSE_TMP_DIRS) { + FileUtils.forceDelete(dir); + } + } + + private void assertResult(List rows, List sqls) + { + Assert.assertEquals(10 * sqls.size(), rows.size()); + rows.sort(Comparator.comparing(Row::getTimestamp) + .thenComparingInt(r -> Integer.valueOf(r.getDimension("a").get(0))) + .thenComparingInt(r -> Integer.valueOf(r.getDimension("b").get(0)))); + int rowCount = 0; + for (int i = 0; i < 10; i++) { + for (int j = 0; j < sqls.size(); j++) { + final Row row = rows.get(rowCount); + String timestampSt = StringUtils.format("2011-01-12T00:0%s:00.000Z", i); + Assert.assertEquals(timestampSt, row.getTimestamp().toString()); + Assert.assertEquals(i, Integer.valueOf(row.getDimension("a").get(0)).intValue()); + Assert.assertEquals(i, Integer.valueOf(row.getDimension("b").get(0)).intValue()); + rowCount++; + } + } + } + + private File createFirehoseTmpDir(String dirSuffix) throws IOException + { + final File firehoseTempDir = File.createTempFile( + SqlInputSourceTest.class.getSimpleName(), + dirSuffix + ); + FileUtils.forceDelete(firehoseTempDir); + FileUtils.forceMkdir(firehoseTempDir); + FIREHOSE_TMP_DIRS.add(firehoseTempDir); + return firehoseTempDir; + } + + @Test + public void testSerde() throws IOException + { + mapper.registerSubtypes(TestSerdeFirehoseConnector.class); + final SqlInputSourceTest.TestSerdeFirehoseConnector testSerdeFirehoseConnector = new SqlInputSourceTest.TestSerdeFirehoseConnector( + new MetadataStorageConnectorConfig()); + final SqlInputSource sqlInputSource = new SqlInputSource(SQLLIST1, true, testSerdeFirehoseConnector, mapper); + final String valueString = mapper.writeValueAsString(sqlInputSource); + final SqlInputSource inputSourceFromJson = mapper.readValue(valueString, SqlInputSource.class); + Assert.assertEquals(sqlInputSource, inputSourceFromJson); + } + + @Test + public void testSingleSplit() throws Exception + { + derbyConnector = derbyConnectorRule.getConnector(); + SqlTestUtils testUtils = new SqlTestUtils(derbyConnector); + testUtils.createAndUpdateTable(TABLE_NAME_1, 10); + final File tempDir = createFirehoseTmpDir("testSingleSplit"); + SqlInputSource sqlInputSource = new SqlInputSource(SQLLIST1, true, testUtils.getDerbyFirehoseConnector(), mapper); + InputSourceReader sqlReader = sqlInputSource.fixedFormatReader(INPUT_ROW_SCHEMA, tempDir); + CloseableIterator resultIterator = sqlReader.read(); + final List rows = new ArrayList<>(); + while (resultIterator.hasNext()) { + rows.add(resultIterator.next()); + } + assertResult(rows, SQLLIST1); + testUtils.dropTable(TABLE_NAME_1); + } + + + @Test + public void testMultipleSplits() throws Exception + { + derbyConnector = derbyConnectorRule.getConnector(); + SqlTestUtils testUtils = new SqlTestUtils(derbyConnector); + testUtils.createAndUpdateTable(TABLE_NAME_1, 10); + testUtils.createAndUpdateTable(TABLE_NAME_2, 10); + final File tempDir = createFirehoseTmpDir("testMultipleSplit"); + SqlInputSource sqlInputSource = new SqlInputSource(SQLLIST2, true, testUtils.getDerbyFirehoseConnector(), mapper); + InputSourceReader sqlReader = sqlInputSource.fixedFormatReader(INPUT_ROW_SCHEMA, tempDir); + CloseableIterator resultIterator = sqlReader.read(); + final List rows = new ArrayList<>(); + while (resultIterator.hasNext()) { + rows.add(resultIterator.next()); + } + assertResult(rows, SQLLIST2); + testUtils.dropTable(TABLE_NAME_1); + testUtils.dropTable(TABLE_NAME_2); + } + + @Test + public void testNumSplits() + { + derbyConnector = derbyConnectorRule.getConnector(); + SqlTestUtils testUtils = new SqlTestUtils(derbyConnector); + SqlInputSource sqlInputSource = new SqlInputSource(SQLLIST2, true, testUtils.getDerbyFirehoseConnector(), mapper); + InputFormat inputFormat = EasyMock.createMock(InputFormat.class); + Stream> sqlSplits = sqlInputSource.createSplits(inputFormat, null); + Assert.assertEquals(SQLLIST2, sqlSplits.map(InputSplit::get).collect(Collectors.toList())); + Assert.assertEquals(2, sqlInputSource.estimateNumSplits(inputFormat, null)); + } + + @Test + public void testEquals() + { + EqualsVerifier.forClass(SqlInputSource.class) + .withPrefabValues( + ObjectMapper.class, + new ObjectMapper(), + new ObjectMapper() + ) + .withIgnoredFields("objectMapper") + .withNonnullFields("sqls", "sqlFirehoseDatabaseConnector") + .usingGetClass() + .verify(); + } + + @JsonTypeName("test") + private static class TestSerdeFirehoseConnector extends SQLFirehoseDatabaseConnector + { + private final DBI dbi; + private final MetadataStorageConnectorConfig metadataStorageConnectorConfig; + + private TestSerdeFirehoseConnector( + @JsonProperty("connectorConfig") MetadataStorageConnectorConfig metadataStorageConnectorConfig + ) + { + final BasicDataSource datasource = getDatasource(metadataStorageConnectorConfig); + datasource.setDriverClassLoader(getClass().getClassLoader()); + datasource.setDriverClassName("org.apache.derby.jdbc.ClientDriver"); + this.dbi = new DBI(datasource); + this.metadataStorageConnectorConfig = metadataStorageConnectorConfig; + } + + @JsonProperty("connectorConfig") + public MetadataStorageConnectorConfig getConnectorConfig() + { + return metadataStorageConnectorConfig; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + TestSerdeFirehoseConnector that = (TestSerdeFirehoseConnector) o; + return metadataStorageConnectorConfig.equals(that.metadataStorageConnectorConfig); + } + + @Override + public int hashCode() + { + return Objects.hash(metadataStorageConnectorConfig); + } + + @Override + public DBI getDBI() + { + return dbi; + } + } +} diff --git a/server/src/test/java/org/apache/druid/metadata/input/SqlTestUtils.java b/server/src/test/java/org/apache/druid/metadata/input/SqlTestUtils.java new file mode 100644 index 000000000000..60e7c73e4397 --- /dev/null +++ b/server/src/test/java/org/apache/druid/metadata/input/SqlTestUtils.java @@ -0,0 +1,118 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.metadata.input; + +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.collect.ImmutableList; +import org.apache.commons.dbcp2.BasicDataSource; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.metadata.MetadataStorageConnectorConfig; +import org.apache.druid.metadata.SQLFirehoseDatabaseConnector; +import org.apache.druid.metadata.TestDerbyConnector; +import org.junit.Rule; +import org.skife.jdbi.v2.Batch; +import org.skife.jdbi.v2.DBI; +import org.skife.jdbi.v2.tweak.HandleCallback; + +public class SqlTestUtils +{ + @Rule + public final TestDerbyConnector.DerbyConnectorRule derbyConnectorRule = new TestDerbyConnector.DerbyConnectorRule(); + private final TestDerbyFirehoseConnector derbyFirehoseConnector; + private final TestDerbyConnector derbyConnector; + + public SqlTestUtils(TestDerbyConnector derbyConnector) + { + this.derbyConnector = derbyConnector; + this.derbyFirehoseConnector = new SqlTestUtils.TestDerbyFirehoseConnector( + new MetadataStorageConnectorConfig(), + derbyConnector.getDBI() + ); + } + + private static class TestDerbyFirehoseConnector extends SQLFirehoseDatabaseConnector + { + private final DBI dbi; + + private TestDerbyFirehoseConnector( + @JsonProperty("connectorConfig") MetadataStorageConnectorConfig metadataStorageConnectorConfig, DBI dbi + ) + { + final BasicDataSource datasource = getDatasource(metadataStorageConnectorConfig); + datasource.setDriverClassLoader(getClass().getClassLoader()); + datasource.setDriverClassName("org.apache.derby.jdbc.ClientDriver"); + this.dbi = dbi; + } + + @Override + public DBI getDBI() + { + return dbi; + } + } + + public void createAndUpdateTable(final String tableName, int numEntries) + { + derbyConnector.createTable( + tableName, + ImmutableList.of( + StringUtils.format( + "CREATE TABLE %1$s (\n" + + " timestamp varchar(255) NOT NULL,\n" + + " a VARCHAR(255) NOT NULL,\n" + + " b VARCHAR(255) NOT NULL\n" + + ")", + tableName + ) + ) + ); + + derbyConnector.getDBI().withHandle( + (handle) -> { + Batch batch = handle.createBatch(); + for (int i = 0; i < numEntries; i++) { + String timestampSt = StringUtils.format("2011-01-12T00:0%s:00.000Z", i); + batch.add(StringUtils.format("INSERT INTO %1$s (timestamp, a, b) VALUES ('%2$s', '%3$s', '%4$s')", + tableName, timestampSt, + i, i + )); + } + batch.execute(); + return null; + } + ); + } + + public void dropTable(final String tableName) + { + derbyConnector.getDBI().withHandle( + (HandleCallback) handle -> { + handle.createStatement(StringUtils.format("DROP TABLE %s", tableName)) + .execute(); + return null; + } + ); + } + + public TestDerbyFirehoseConnector getDerbyFirehoseConnector() + { + return derbyFirehoseConnector; + } +} diff --git a/server/src/test/java/org/apache/druid/segment/realtime/firehose/SqlFirehoseFactoryTest.java b/server/src/test/java/org/apache/druid/segment/realtime/firehose/SqlFirehoseFactoryTest.java index 34fa763613fe..189aa4984aac 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/firehose/SqlFirehoseFactoryTest.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/firehose/SqlFirehoseFactoryTest.java @@ -34,6 +34,7 @@ import org.apache.druid.metadata.MetadataStorageConnectorConfig; import org.apache.druid.metadata.SQLFirehoseDatabaseConnector; import org.apache.druid.metadata.TestDerbyConnector; +import org.apache.druid.metadata.input.SqlTestUtils; import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.transform.TransformSpec; import org.junit.AfterClass; @@ -41,9 +42,7 @@ import org.junit.BeforeClass; import org.junit.Rule; import org.junit.Test; -import org.skife.jdbi.v2.Batch; import org.skife.jdbi.v2.DBI; -import org.skife.jdbi.v2.tweak.HandleCallback; import java.io.File; import java.io.IOException; @@ -82,7 +81,6 @@ public class SqlFirehoseFactoryTest ) ); private TestDerbyConnector derbyConnector; - private TestDerbyFirehoseConnector derbyFirehoseConnector; @BeforeClass public static void setup() throws IOException @@ -139,56 +137,12 @@ private File createFirehoseTmpDir(String dirSuffix) throws IOException return firehoseTempDir; } - private void dropTable(final String tableName) - { - derbyConnector.getDBI().withHandle( - (HandleCallback) handle -> { - handle.createStatement(StringUtils.format("DROP TABLE %s", tableName)) - .execute(); - return null; - } - ); - } - - private void createAndUpdateTable(final String tableName) - { - derbyConnector = derbyConnectorRule.getConnector(); - derbyFirehoseConnector = new TestDerbyFirehoseConnector(new MetadataStorageConnectorConfig(), - derbyConnector.getDBI()); - derbyConnector.createTable( - tableName, - ImmutableList.of( - StringUtils.format( - "CREATE TABLE %1$s (\n" - + " timestamp varchar(255) NOT NULL,\n" - + " a VARCHAR(255) NOT NULL,\n" - + " b VARCHAR(255) NOT NULL\n" - + ")", - tableName - ) - ) - ); - - derbyConnector.getDBI().withHandle( - (handle) -> { - Batch batch = handle.createBatch(); - for (int i = 0; i < 10; i++) { - String timestampSt = StringUtils.format("2011-01-12T00:0%s:00.000Z", i); - batch.add(StringUtils.format("INSERT INTO %1$s (timestamp, a, b) VALUES ('%2$s', '%3$s', '%4$s')", - tableName, timestampSt, - i, i - )); - } - batch.execute(); - return null; - } - ); - } - @Test public void testWithoutCacheAndFetch() throws Exception { - createAndUpdateTable(TABLE_NAME_1); + derbyConnector = derbyConnectorRule.getConnector(); + SqlTestUtils testUtils = new SqlTestUtils(derbyConnector); + testUtils.createAndUpdateTable(TABLE_NAME_1, 10); final SqlFirehoseFactory factory = new SqlFirehoseFactory( SQLLIST1, @@ -197,7 +151,7 @@ public void testWithoutCacheAndFetch() throws Exception 0L, 0L, true, - derbyFirehoseConnector, + testUtils.getDerbyFirehoseConnector(), mapper ); @@ -211,14 +165,16 @@ public void testWithoutCacheAndFetch() throws Exception assertResult(rows, SQLLIST1); assertNumRemainingCacheFiles(firehoseTmpDir, 0); - dropTable(TABLE_NAME_1); + testUtils.dropTable(TABLE_NAME_1); } @Test public void testWithoutCache() throws IOException { - createAndUpdateTable(TABLE_NAME_1); + derbyConnector = derbyConnectorRule.getConnector(); + SqlTestUtils testUtils = new SqlTestUtils(derbyConnector); + testUtils.createAndUpdateTable(TABLE_NAME_1, 10); final SqlFirehoseFactory factory = new SqlFirehoseFactory( SQLLIST1, @@ -227,7 +183,7 @@ public void testWithoutCache() throws IOException null, null, true, - derbyFirehoseConnector, + testUtils.getDerbyFirehoseConnector(), mapper ); @@ -242,15 +198,17 @@ public void testWithoutCache() throws IOException assertResult(rows, SQLLIST1); assertNumRemainingCacheFiles(firehoseTmpDir, 0); - dropTable(TABLE_NAME_1); + testUtils.dropTable(TABLE_NAME_1); } @Test public void testWithCacheAndFetch() throws IOException { - createAndUpdateTable(TABLE_NAME_1); - createAndUpdateTable(TABLE_NAME_2); + derbyConnector = derbyConnectorRule.getConnector(); + SqlTestUtils testUtils = new SqlTestUtils(derbyConnector); + testUtils.createAndUpdateTable(TABLE_NAME_1, 10); + testUtils.createAndUpdateTable(TABLE_NAME_2, 10); final SqlFirehoseFactory factory = new SqlFirehoseFactory( @@ -260,7 +218,7 @@ public void testWithCacheAndFetch() throws IOException 0L, null, true, - derbyFirehoseConnector, + testUtils.getDerbyFirehoseConnector(), mapper ); @@ -274,8 +232,8 @@ public void testWithCacheAndFetch() throws IOException assertResult(rows, SQLLIST2); assertNumRemainingCacheFiles(firehoseTmpDir, 2); - dropTable(TABLE_NAME_1); - dropTable(TABLE_NAME_2); + testUtils.dropTable(TABLE_NAME_1); + testUtils.dropTable(TABLE_NAME_2); } private static class TestDerbyFirehoseConnector extends SQLFirehoseDatabaseConnector diff --git a/server/src/test/java/org/apache/druid/segment/realtime/plumber/CoordinatorBasedSegmentHandoffNotifierTest.java b/server/src/test/java/org/apache/druid/segment/realtime/plumber/CoordinatorBasedSegmentHandoffNotifierTest.java index 47157347a413..f5534bd46a3b 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/plumber/CoordinatorBasedSegmentHandoffNotifierTest.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/plumber/CoordinatorBasedSegmentHandoffNotifierTest.java @@ -174,7 +174,7 @@ public void testHandoffChecksForAssignableServer() ) ); - Assert.assertFalse( + Assert.assertTrue( CoordinatorBasedSegmentHandoffNotifier.isHandOffComplete( Collections.singletonList( new ImmutableSegmentLoadInfo( diff --git a/server/src/test/java/org/apache/druid/server/coordination/SegmentLoadDropHandlerTest.java b/server/src/test/java/org/apache/druid/server/coordination/SegmentLoadDropHandlerTest.java index f5455fbbfb29..6d8ef0a8cc4e 100644 --- a/server/src/test/java/org/apache/druid/server/coordination/SegmentLoadDropHandlerTest.java +++ b/server/src/test/java/org/apache/druid/server/coordination/SegmentLoadDropHandlerTest.java @@ -24,6 +24,8 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterables; import com.google.common.util.concurrent.ListenableFuture; +import org.apache.druid.guice.ServerTypeConfig; +import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.common.concurrent.ScheduledExecutorFactory; @@ -32,6 +34,7 @@ import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.loading.CacheTestSegmentLoader; import org.apache.druid.segment.loading.SegmentLoaderConfig; +import org.apache.druid.segment.loading.StorageLocationConfig; import org.apache.druid.server.SegmentManager; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.partition.NoneShardSpec; @@ -39,12 +42,16 @@ import org.joda.time.Interval; import org.junit.Assert; import org.junit.Before; +import org.junit.Rule; import org.junit.Test; +import org.junit.rules.ExpectedException; +import org.junit.rules.TemporaryFolder; import java.io.File; import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collections; import java.util.HashSet; import java.util.List; import java.util.Set; @@ -67,6 +74,7 @@ public class SegmentLoadDropHandlerTest private final ObjectMapper jsonMapper = TestHelper.makeJsonMapper(); private SegmentLoadDropHandler segmentLoadDropHandler; + private DataSegmentAnnouncer announcer; private File infoDir; private AtomicInteger announceCount; @@ -74,22 +82,36 @@ public class SegmentLoadDropHandlerTest private CacheTestSegmentLoader segmentLoader; private SegmentManager segmentManager; private List scheduledRunnable; + private SegmentLoaderConfig segmentLoaderConfig; + private SegmentLoaderConfig segmentLoaderConfigNoLocations; + private ScheduledExecutorFactory scheduledExecutorFactory; + private List locations; + + @Rule + public ExpectedException expectedException = ExpectedException.none(); + + @Rule + public TemporaryFolder temporaryFolder = new TemporaryFolder(); @Before public void setUp() { try { - infoDir = new File(File.createTempFile("blah", "blah2").getParent(), "ZkCoordinatorTest"); - infoDir.mkdirs(); - for (File file : infoDir.listFiles()) { - file.delete(); - } + infoDir = temporaryFolder.newFolder(); log.info("Creating tmp test files in [%s]", infoDir); } catch (IOException e) { throw new RuntimeException(e); } + locations = Collections.singletonList( + new StorageLocationConfig( + infoDir, + 100L, + 100d + ) + ); + scheduledRunnable = new ArrayList<>(); segmentLoader = new CacheTestSegmentLoader(); @@ -132,57 +154,91 @@ public void unannounceSegments(Iterable segments) } }; - segmentLoadDropHandler = new SegmentLoadDropHandler( - jsonMapper, - new SegmentLoaderConfig() - { - @Override - public File getInfoDir() - { - return infoDir; - } - @Override - public int getNumLoadingThreads() - { - return 5; - } + segmentLoaderConfig = new SegmentLoaderConfig() + { + @Override + public File getInfoDir() + { + return infoDir; + } - @Override - public int getAnnounceIntervalMillis() - { - return 50; - } + @Override + public int getNumLoadingThreads() + { + return 5; + } + + @Override + public int getAnnounceIntervalMillis() + { + return 50; + } + + @Override + public List getLocations() + { + return locations; + } + + @Override + public int getDropSegmentDelayMillis() + { + return 0; + } + }; + + segmentLoaderConfigNoLocations = new SegmentLoaderConfig() + { + @Override + public int getNumLoadingThreads() + { + return 5; + } + + @Override + public int getAnnounceIntervalMillis() + { + return 50; + } + + @Override + public int getDropSegmentDelayMillis() + { + return 0; + } + }; + + scheduledExecutorFactory = new ScheduledExecutorFactory() + { + @Override + public ScheduledExecutorService create(int corePoolSize, String nameFormat) + { + /* + Override normal behavoir by adding the runnable to a list so that you can make sure + all the shceduled runnables are executed by explicitly calling run() on each item in the list + */ + return new ScheduledThreadPoolExecutor(corePoolSize, Execs.makeThreadFactory(nameFormat)) + { @Override - public int getDropSegmentDelayMillis() + public ScheduledFuture schedule(Runnable command, long delay, TimeUnit unit) { - return 0; + scheduledRunnable.add(command); + return null; } - }, + }; + } + }; + + segmentLoadDropHandler = new SegmentLoadDropHandler( + jsonMapper, + segmentLoaderConfig, announcer, EasyMock.createNiceMock(DataSegmentServerAnnouncer.class), segmentManager, - new ScheduledExecutorFactory() - { - @Override - public ScheduledExecutorService create(int corePoolSize, String nameFormat) - { - /* - Override normal behavoir by adding the runnable to a list so that you can make sure - all the shceduled runnables are executed by explicitly calling run() on each item in the list - */ - return new ScheduledThreadPoolExecutor(corePoolSize, Execs.makeThreadFactory(nameFormat)) - { - @Override - public ScheduledFuture schedule(Runnable command, long delay, TimeUnit unit) - { - scheduledRunnable.add(command); - return null; - } - }; - } - }.create(5, "SegmentLoadDropHandlerTest-[%d]") + scheduledExecutorFactory.create(5, "SegmentLoadDropHandlerTest-[%d]"), + new ServerTypeConfig(ServerType.HISTORICAL) ); } @@ -220,6 +276,40 @@ Because another addSegment() call is executed, which removes the segment from se segmentLoadDropHandler.stop(); } + @Test + public void testSegmentLoading1BrokerWithNoLocations() throws Exception + { + SegmentLoadDropHandler segmentLoadDropHandlerBrokerWithNoLocations = new SegmentLoadDropHandler( + jsonMapper, + segmentLoaderConfigNoLocations, + announcer, + EasyMock.createNiceMock(DataSegmentServerAnnouncer.class), + segmentManager, + scheduledExecutorFactory.create(5, "SegmentLoadDropHandlerTest-brokerNoLocations-[%d]"), + new ServerTypeConfig(ServerType.BROKER) + ); + + segmentLoadDropHandlerBrokerWithNoLocations.start(); + segmentLoadDropHandler.stop(); + } + + @Test + public void testSegmentLoading1HistoricalWithNoLocations() + { + expectedException.expect(IAE.class); + expectedException.expectMessage("Segment cache locations must be set on historicals."); + + new SegmentLoadDropHandler( + jsonMapper, + segmentLoaderConfigNoLocations, + announcer, + EasyMock.createNiceMock(DataSegmentServerAnnouncer.class), + segmentManager, + scheduledExecutorFactory.create(5, "SegmentLoadDropHandlerTest-[%d]"), + new ServerTypeConfig(ServerType.HISTORICAL) + ); + } + /** * Steps: * 1. addSegment() succesfully loads the segment and annouces it @@ -382,13 +472,20 @@ public int getNumLoadingThreads() return 5; } + @Override + public List getLocations() + { + return locations; + } + @Override public int getAnnounceIntervalMillis() { return 50; } }, - announcer, EasyMock.createNiceMock(DataSegmentServerAnnouncer.class), segmentManager + announcer, EasyMock.createNiceMock(DataSegmentServerAnnouncer.class), segmentManager, + new ServerTypeConfig(ServerType.HISTORICAL) ); Set segments = new HashSet<>(); diff --git a/server/src/test/java/org/apache/druid/server/coordination/ZkCoordinatorTest.java b/server/src/test/java/org/apache/druid/server/coordination/ZkCoordinatorTest.java index 9bdd84ed70fb..c30a37cb01ad 100644 --- a/server/src/test/java/org/apache/druid/server/coordination/ZkCoordinatorTest.java +++ b/server/src/test/java/org/apache/druid/server/coordination/ZkCoordinatorTest.java @@ -23,10 +23,13 @@ import com.google.common.collect.ImmutableMap; import org.apache.curator.utils.ZKPaths; import org.apache.druid.curator.CuratorTestBase; +import org.apache.druid.guice.ServerTypeConfig; import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.segment.IndexIO; import org.apache.druid.segment.loading.SegmentLoaderConfig; +import org.apache.druid.segment.loading.StorageLocationConfig; import org.apache.druid.server.SegmentManager; import org.apache.druid.server.ServerTestHelper; import org.apache.druid.server.initialization.ZkPathsConfig; @@ -37,9 +40,15 @@ import org.easymock.EasyMock; import org.junit.After; import org.junit.Before; +import org.junit.Rule; import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import java.io.File; +import java.io.IOException; import java.util.Arrays; +import java.util.Collections; +import java.util.List; import java.util.concurrent.CountDownLatch; import java.util.concurrent.ScheduledExecutorService; @@ -47,6 +56,8 @@ */ public class ZkCoordinatorTest extends CuratorTestBase { + private static final Logger log = new Logger(ZkCoordinatorTest.class); + private final ObjectMapper jsonMapper = ServerTestHelper.MAPPER; private final DruidServerMetadata me = new DruidServerMetadata( "dummyServer", @@ -67,9 +78,31 @@ public String getBase() }; private ZkCoordinator zkCoordinator; + private File infoDir; + private List locations; + + @Rule + public TemporaryFolder temporaryFolder = new TemporaryFolder(); + @Before public void setUp() throws Exception { + try { + infoDir = temporaryFolder.newFolder(); + log.info("Creating tmp test files in [%s]", infoDir); + } + catch (IOException e) { + throw new RuntimeException(e); + } + + locations = Collections.singletonList( + new StorageLocationConfig( + infoDir, + 100L, + 100d + ) + ); + setupServerAndCurator(); curator.start(); curator.blockUntilConnected(); @@ -102,11 +135,42 @@ public void testLoadDrop() throws Exception SegmentLoadDropHandler segmentLoadDropHandler = new SegmentLoadDropHandler( ServerTestHelper.MAPPER, - new SegmentLoaderConfig(), + new SegmentLoaderConfig() { + @Override + public File getInfoDir() + { + return infoDir; + } + + @Override + public int getNumLoadingThreads() + { + return 5; + } + + @Override + public int getAnnounceIntervalMillis() + { + return 50; + } + + @Override + public List getLocations() + { + return locations; + } + + @Override + public int getDropSegmentDelayMillis() + { + return 0; + } + }, EasyMock.createNiceMock(DataSegmentAnnouncer.class), EasyMock.createNiceMock(DataSegmentServerAnnouncer.class), EasyMock.createNiceMock(SegmentManager.class), - EasyMock.createNiceMock(ScheduledExecutorService.class) + EasyMock.createNiceMock(ScheduledExecutorService.class), + new ServerTypeConfig(ServerType.HISTORICAL) ) { @Override diff --git a/server/src/test/java/org/apache/druid/server/coordinator/BalanceSegmentsTest.java b/server/src/test/java/org/apache/druid/server/coordinator/BalanceSegmentsTest.java index 084a119ebe76..f37c92cb1056 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/BalanceSegmentsTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/BalanceSegmentsTest.java @@ -43,6 +43,7 @@ import java.util.Collections; import java.util.HashMap; import java.util.List; +import java.util.Set; import java.util.concurrent.Executors; import java.util.concurrent.atomic.AtomicInteger; import java.util.stream.Collectors; @@ -66,9 +67,11 @@ public class BalanceSegmentsTest private DataSegment segment2; private DataSegment segment3; private DataSegment segment4; + private DataSegment segment5; private List segments; private ListeningExecutorService balancerStrategyExecutor; private BalancerStrategy balancerStrategy; + private Set broadcastDatasources; @Before public void setUp() @@ -82,6 +85,7 @@ public void setUp() segment2 = EasyMock.createMock(DataSegment.class); segment3 = EasyMock.createMock(DataSegment.class); segment4 = EasyMock.createMock(DataSegment.class); + segment5 = EasyMock.createMock(DataSegment.class); DateTime start1 = DateTimes.of("2012-01-01"); DateTime start2 = DateTimes.of("2012-02-01"); @@ -130,12 +134,24 @@ public void setUp() 0, 8L ); + segment5 = new DataSegment( + "datasourceBroadcast", + new Interval(start2, start2.plusHours(1)), + version.toString(), + new HashMap<>(), + new ArrayList<>(), + new ArrayList<>(), + NoneShardSpec.instance(), + 0, + 8L + ); segments = new ArrayList<>(); segments.add(segment1); segments.add(segment2); segments.add(segment3); segments.add(segment4); + segments.add(segment5); peon1 = new LoadQueuePeonTester(); peon2 = new LoadQueuePeonTester(); @@ -147,6 +163,8 @@ public void setUp() balancerStrategyExecutor = MoreExecutors.listeningDecorator(Executors.newFixedThreadPool(1)); balancerStrategy = new CostBalancerStrategyFactory().createBalancerStrategy(balancerStrategyExecutor); + + broadcastDatasources = Collections.singleton("datasourceBroadcast"); } @After @@ -187,10 +205,11 @@ public void testMoveToEmptyServerBalancer() ImmutableList.of(peon1, peon2) ) .withBalancerStrategy(predefinedPickOrderStrategy) + .withBroadcastDatasources(broadcastDatasources) .build(); params = new BalanceSegmentsTester(coordinator).run(params); - Assert.assertEquals(2, params.getCoordinatorStats().getTieredStat("movedCount", "normal")); + Assert.assertEquals(3, params.getCoordinatorStats().getTieredStat("movedCount", "normal")); } /** @@ -213,10 +232,10 @@ public void testMoveDecommissioningMaxPercentOfMaxSegmentsToMove() mockCoordinator(coordinator); BalancerStrategy strategy = EasyMock.createMock(BalancerStrategy.class); - EasyMock.expect(strategy.pickSegmentToMove(ImmutableList.of(new ServerHolder(druidServer2, peon2, false)))) + EasyMock.expect(strategy.pickSegmentToMove(ImmutableList.of(new ServerHolder(druidServer2, peon2, false)), broadcastDatasources)) .andReturn(new BalancerSegmentHolder(druidServer2, segment3)) .andReturn(new BalancerSegmentHolder(druidServer2, segment4)); - EasyMock.expect(strategy.pickSegmentToMove(EasyMock.anyObject())) + EasyMock.expect(strategy.pickSegmentToMove(EasyMock.anyObject(), EasyMock.anyObject())) .andReturn(new BalancerSegmentHolder(druidServer1, segment1)) .andReturn(new BalancerSegmentHolder(druidServer1, segment2)); @@ -237,6 +256,7 @@ public void testMoveDecommissioningMaxPercentOfMaxSegmentsToMove() .build() // ceil(3 * 0.6) = 2 segments from decommissioning servers ) .withBalancerStrategy(strategy) + .withBroadcastDatasources(broadcastDatasources) .build(); params = new BalanceSegmentsTester(coordinator).run(params); @@ -280,7 +300,7 @@ public void testMoveDecommissioningMaxPercentOfMaxSegmentsToMoveWithNoDecommissi mockCoordinator(coordinator); BalancerStrategy strategy = EasyMock.createMock(BalancerStrategy.class); - EasyMock.expect(strategy.pickSegmentToMove(EasyMock.anyObject())) + EasyMock.expect(strategy.pickSegmentToMove(EasyMock.anyObject(), EasyMock.anyObject())) .andReturn(new BalancerSegmentHolder(druidServer1, segment1)) .andReturn(new BalancerSegmentHolder(druidServer1, segment2)) .andReturn(new BalancerSegmentHolder(druidServer2, segment3)) @@ -303,6 +323,7 @@ public void testMoveDecommissioningMaxPercentOfMaxSegmentsToMoveWithNoDecommissi .build() ) .withBalancerStrategy(strategy) + .withBroadcastDatasources(broadcastDatasources) .build(); params = new BalanceSegmentsTester(coordinator).run(params); @@ -328,7 +349,7 @@ public void testMoveToDecommissioningServer() mockCoordinator(coordinator); BalancerStrategy strategy = EasyMock.createMock(BalancerStrategy.class); - EasyMock.expect(strategy.pickSegmentToMove(EasyMock.anyObject())) + EasyMock.expect(strategy.pickSegmentToMove(EasyMock.anyObject(), EasyMock.anyObject())) .andReturn(new BalancerSegmentHolder(druidServer1, segment1)) .anyTimes(); EasyMock.expect(strategy.findNewSegmentHomeBalancer(EasyMock.anyObject(), EasyMock.anyObject())).andAnswer(() -> { @@ -343,6 +364,7 @@ public void testMoveToDecommissioningServer() ImmutableList.of(false, true) ) .withBalancerStrategy(strategy) + .withBroadcastDatasources(broadcastDatasources) .build(); params = new BalanceSegmentsTester(coordinator).run(params); @@ -362,7 +384,7 @@ public void testMoveFromDecommissioningServer() ServerHolder holder2 = new ServerHolder(druidServer2, peon2, false); BalancerStrategy strategy = EasyMock.createMock(BalancerStrategy.class); - EasyMock.expect(strategy.pickSegmentToMove(EasyMock.anyObject())) + EasyMock.expect(strategy.pickSegmentToMove(EasyMock.anyObject(), EasyMock.anyObject())) .andReturn(new BalancerSegmentHolder(druidServer1, segment1)) .once(); EasyMock.expect(strategy.findNewSegmentHomeBalancer(EasyMock.anyObject(), EasyMock.anyObject())) @@ -377,6 +399,7 @@ public void testMoveFromDecommissioningServer() ) .withDynamicConfigs(CoordinatorDynamicConfig.builder().withMaxSegmentsToMove(1).build()) .withBalancerStrategy(strategy) + .withBroadcastDatasources(broadcastDatasources) .build(); params = new BalanceSegmentsTester(coordinator).run(params); @@ -412,6 +435,7 @@ public void testMoveMaxLoadQueueServerBalancer() ImmutableList.of(peon1, peon2) ) .withBalancerStrategy(predefinedPickOrderStrategy) + .withBroadcastDatasources(broadcastDatasources) .withDynamicConfigs( CoordinatorDynamicConfig .builder() @@ -451,6 +475,7 @@ public void testMoveSameSegmentTwice() ImmutableList.of(peon1, peon2) ) .withBalancerStrategy(predefinedPickOrderStrategy) + .withBroadcastDatasources(broadcastDatasources) .withDynamicConfigs( CoordinatorDynamicConfig.builder().withMaxSegmentsToMove( 2 @@ -542,6 +567,7 @@ private DruidCoordinatorRuntimeParams.Builder defaultRuntimeParamsBuilder( ) .withUsedSegmentsInTest(segments) .withDynamicConfigs(CoordinatorDynamicConfig.builder().withMaxSegmentsToMove(MAX_SEGMENTS_TO_MOVE).build()) + .withBroadcastDatasources(broadcastDatasources) .withBalancerStrategy(balancerStrategy); } @@ -611,7 +637,7 @@ public ServerHolder findNewSegmentHomeReplicator(DataSegment proposalSegment, Li } @Override - public BalancerSegmentHolder pickSegmentToMove(List serverHolders) + public BalancerSegmentHolder pickSegmentToMove(List serverHolders, Set broadcastDatasources) { return pickOrder.get(pickCounter.getAndIncrement() % pickOrder.size()); } @@ -635,9 +661,9 @@ private DruidCoordinatorRuntimeParams setupParamsForDecommissioningMaxPercentOfM // either decommissioning servers list or acitve ones (ie servers list is [2] or [1, 3]) BalancerStrategy strategy = EasyMock.createMock(BalancerStrategy.class); - EasyMock.expect(strategy.pickSegmentToMove(ImmutableList.of(new ServerHolder(druidServer2, peon2, true)))) + EasyMock.expect(strategy.pickSegmentToMove(ImmutableList.of(new ServerHolder(druidServer2, peon2, true)), broadcastDatasources)) .andReturn(new BalancerSegmentHolder(druidServer2, segment2)); - EasyMock.expect(strategy.pickSegmentToMove(EasyMock.anyObject())) + EasyMock.expect(strategy.pickSegmentToMove(EasyMock.anyObject(), EasyMock.anyObject())) .andReturn(new BalancerSegmentHolder(druidServer1, segment1)); EasyMock.expect(strategy.findNewSegmentHomeBalancer(EasyMock.anyObject(), EasyMock.anyObject())) .andReturn(new ServerHolder(druidServer3, peon3)) @@ -656,6 +682,7 @@ private DruidCoordinatorRuntimeParams setupParamsForDecommissioningMaxPercentOfM .build() ) .withBalancerStrategy(strategy) + .withBroadcastDatasources(broadcastDatasources) .build(); } } diff --git a/server/src/test/java/org/apache/druid/server/coordinator/DruidClusterBuilder.java b/server/src/test/java/org/apache/druid/server/coordinator/DruidClusterBuilder.java index 772b7aec1401..5fb100073eae 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/DruidClusterBuilder.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/DruidClusterBuilder.java @@ -35,6 +35,7 @@ public static DruidClusterBuilder newBuilder() private @Nullable Set realtimes = null; private final Map> historicals = new HashMap<>(); + private @Nullable Set brokers = null; private DruidClusterBuilder() { @@ -46,6 +47,12 @@ public DruidClusterBuilder withRealtimes(ServerHolder... realtimes) return this; } + public DruidClusterBuilder withBrokers(ServerHolder... brokers) + { + this.brokers = new HashSet<>(Arrays.asList(brokers)); + return this; + } + public DruidClusterBuilder addTier(String tierName, ServerHolder... historicals) { if (this.historicals.putIfAbsent(tierName, Arrays.asList(historicals)) != null) { @@ -56,6 +63,6 @@ public DruidClusterBuilder addTier(String tierName, ServerHolder... historicals) public DruidCluster build() { - return DruidCluster.createDruidClusterFromBuilderInTest(realtimes, historicals); + return DruidCluster.createDruidClusterFromBuilderInTest(realtimes, historicals, brokers); } } diff --git a/server/src/test/java/org/apache/druid/server/coordinator/ReservoirSegmentSamplerTest.java b/server/src/test/java/org/apache/druid/server/coordinator/ReservoirSegmentSamplerTest.java index 73e829ce0e0c..8aef2f2e10fb 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/ReservoirSegmentSamplerTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/ReservoirSegmentSamplerTest.java @@ -23,6 +23,7 @@ import org.apache.druid.client.ImmutableDruidServer; import org.apache.druid.client.ImmutableDruidServerTests; import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.server.coordination.ServerType; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.partition.NoneShardSpec; import org.easymock.EasyMock; @@ -136,6 +137,7 @@ public void setUp() @Test public void getRandomBalancerSegmentHolderTest() { + EasyMock.expect(druidServer1.getType()).andReturn(ServerType.HISTORICAL).atLeastOnce(); EasyMock.expect(druidServer1.getName()).andReturn("1").atLeastOnce(); EasyMock.expect(druidServer1.getCurrSize()).andReturn(30L).atLeastOnce(); EasyMock.expect(druidServer1.getMaxSize()).andReturn(100L).atLeastOnce(); @@ -143,6 +145,7 @@ public void getRandomBalancerSegmentHolderTest() EasyMock.expect(druidServer1.getSegment(EasyMock.anyObject())).andReturn(null).anyTimes(); EasyMock.replay(druidServer1); + EasyMock.expect(druidServer2.getType()).andReturn(ServerType.HISTORICAL).atLeastOnce(); EasyMock.expect(druidServer2.getName()).andReturn("2").atLeastOnce(); EasyMock.expect(druidServer2.getTier()).andReturn("normal").anyTimes(); EasyMock.expect(druidServer2.getCurrSize()).andReturn(30L).atLeastOnce(); @@ -151,6 +154,7 @@ public void getRandomBalancerSegmentHolderTest() EasyMock.expect(druidServer2.getSegment(EasyMock.anyObject())).andReturn(null).anyTimes(); EasyMock.replay(druidServer2); + EasyMock.expect(druidServer3.getType()).andReturn(ServerType.HISTORICAL).atLeastOnce(); EasyMock.expect(druidServer3.getName()).andReturn("3").atLeastOnce(); EasyMock.expect(druidServer3.getTier()).andReturn("normal").anyTimes(); EasyMock.expect(druidServer3.getCurrSize()).andReturn(30L).atLeastOnce(); @@ -159,6 +163,7 @@ public void getRandomBalancerSegmentHolderTest() EasyMock.expect(druidServer3.getSegment(EasyMock.anyObject())).andReturn(null).anyTimes(); EasyMock.replay(druidServer3); + EasyMock.expect(druidServer4.getType()).andReturn(ServerType.HISTORICAL).atLeastOnce(); EasyMock.expect(druidServer4.getName()).andReturn("4").atLeastOnce(); EasyMock.expect(druidServer4.getTier()).andReturn("normal").anyTimes(); EasyMock.expect(druidServer4.getCurrSize()).andReturn(30L).atLeastOnce(); @@ -186,7 +191,7 @@ public void getRandomBalancerSegmentHolderTest() Map segmentCountMap = new HashMap<>(); for (int i = 0; i < 5000; i++) { - segmentCountMap.put(ReservoirSegmentSampler.getRandomBalancerSegmentHolder(holderList).getSegment(), 1); + segmentCountMap.put(ReservoirSegmentSampler.getRandomBalancerSegmentHolder(holderList, Collections.emptySet()).getSegment(), 1); } for (DataSegment segment : segments) { diff --git a/server/src/test/java/org/apache/druid/server/coordinator/rules/BroadcastDistributionRuleSerdeTest.java b/server/src/test/java/org/apache/druid/server/coordinator/rules/BroadcastDistributionRuleSerdeTest.java index e3b51a51b02d..0dfe0eab0ac1 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/rules/BroadcastDistributionRuleSerdeTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/rules/BroadcastDistributionRuleSerdeTest.java @@ -21,7 +21,6 @@ import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.Intervals; @@ -44,15 +43,15 @@ public class BroadcastDistributionRuleSerdeTest public static List constructorFeeder() { return Lists.newArrayList( - new Object[]{new ForeverBroadcastDistributionRule(ImmutableList.of("large_source1", "large_source2"))}, - new Object[]{new ForeverBroadcastDistributionRule(ImmutableList.of())}, - new Object[]{new ForeverBroadcastDistributionRule(null)}, - new Object[]{new IntervalBroadcastDistributionRule(Intervals.of("0/1000"), ImmutableList.of("large_source"))}, - new Object[]{new IntervalBroadcastDistributionRule(Intervals.of("0/1000"), ImmutableList.of())}, - new Object[]{new IntervalBroadcastDistributionRule(Intervals.of("0/1000"), null)}, - new Object[]{new PeriodBroadcastDistributionRule(new Period(1000), null, ImmutableList.of("large_source"))}, - new Object[]{new PeriodBroadcastDistributionRule(new Period(1000), null, ImmutableList.of())}, - new Object[]{new PeriodBroadcastDistributionRule(new Period(1000), null, null)} + new Object[]{new ForeverBroadcastDistributionRule()}, + new Object[]{new ForeverBroadcastDistributionRule()}, + new Object[]{new ForeverBroadcastDistributionRule()}, + new Object[]{new IntervalBroadcastDistributionRule(Intervals.of("0/1000"))}, + new Object[]{new IntervalBroadcastDistributionRule(Intervals.of("0/1000"))}, + new Object[]{new IntervalBroadcastDistributionRule(Intervals.of("0/1000"))}, + new Object[]{new PeriodBroadcastDistributionRule(new Period(1000), null)}, + new Object[]{new PeriodBroadcastDistributionRule(new Period(1000), null)}, + new Object[]{new PeriodBroadcastDistributionRule(new Period(1000), null)} ); } diff --git a/server/src/test/java/org/apache/druid/server/coordinator/rules/BroadcastDistributionRuleTest.java b/server/src/test/java/org/apache/druid/server/coordinator/rules/BroadcastDistributionRuleTest.java index 70ec3ebec052..c2d4fd3d0004 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/rules/BroadcastDistributionRuleTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/rules/BroadcastDistributionRuleTest.java @@ -19,7 +19,6 @@ package org.apache.druid.server.coordinator.rules; -import com.google.common.collect.ImmutableList; import org.apache.druid.client.DruidServer; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.Intervals; @@ -269,7 +268,7 @@ public void setUp() public void testBroadcastToSingleDataSource() { final ForeverBroadcastDistributionRule rule = - new ForeverBroadcastDistributionRule(ImmutableList.of("large_source")); + new ForeverBroadcastDistributionRule(); CoordinatorStats stats = rule.run( null, @@ -285,7 +284,7 @@ public void testBroadcastToSingleDataSource() smallSegment ); - Assert.assertEquals(3L, stats.getGlobalStat(LoadRule.ASSIGNED_COUNT)); + Assert.assertEquals(5L, stats.getGlobalStat(LoadRule.ASSIGNED_COUNT)); Assert.assertFalse(stats.hasPerTierStats()); Assert.assertTrue( @@ -295,10 +294,10 @@ public void testBroadcastToSingleDataSource() Assert.assertTrue( holdersOfLargeSegments2.stream() - .noneMatch(holder -> holder.getPeon().getSegmentsToLoad().contains(smallSegment)) + .allMatch(holder -> holder.getPeon().getSegmentsToLoad().contains(smallSegment)) ); - Assert.assertFalse(holderOfSmallSegment.getPeon().getSegmentsToLoad().contains(smallSegment)); + Assert.assertTrue(holderOfSmallSegment.isServingSegment(smallSegment)); } private static DruidCoordinatorRuntimeParams makeCoordinartorRuntimeParams( @@ -331,7 +330,7 @@ private static DruidCoordinatorRuntimeParams makeCoordinartorRuntimeParams( public void testBroadcastDecommissioning() { final ForeverBroadcastDistributionRule rule = - new ForeverBroadcastDistributionRule(ImmutableList.of("large_source")); + new ForeverBroadcastDistributionRule(); CoordinatorStats stats = rule.run( null, @@ -356,7 +355,6 @@ public void testBroadcastDecommissioning() public void testBroadcastToMultipleDataSources() { final ForeverBroadcastDistributionRule rule = new ForeverBroadcastDistributionRule( - ImmutableList.of("large_source", "large_source2") ); CoordinatorStats stats = rule.run( @@ -392,7 +390,7 @@ public void testBroadcastToMultipleDataSources() @Test public void testBroadcastToAllServers() { - final ForeverBroadcastDistributionRule rule = new ForeverBroadcastDistributionRule(null); + final ForeverBroadcastDistributionRule rule = new ForeverBroadcastDistributionRule(); CoordinatorStats stats = rule.run( null, @@ -408,14 +406,14 @@ public void testBroadcastToAllServers() smallSegment ); - Assert.assertEquals(6L, stats.getGlobalStat(LoadRule.ASSIGNED_COUNT)); + Assert.assertEquals(5L, stats.getGlobalStat(LoadRule.ASSIGNED_COUNT)); Assert.assertFalse(stats.hasPerTierStats()); Assert.assertTrue( druidCluster .getAllServers() .stream() - .allMatch(holder -> holder.getPeon().getSegmentsToLoad().contains(smallSegment)) + .allMatch(holder -> holder.isLoadingSegment(smallSegment) || holder.isServingSegment(smallSegment)) ); } } diff --git a/services/src/main/java/org/apache/druid/cli/CliBroker.java b/services/src/main/java/org/apache/druid/cli/CliBroker.java index 5badcb0e0e3a..18d2813fa411 100644 --- a/services/src/main/java/org/apache/druid/cli/CliBroker.java +++ b/services/src/main/java/org/apache/druid/cli/CliBroker.java @@ -33,6 +33,7 @@ import org.apache.druid.client.selector.CustomTierSelectorStrategyConfig; import org.apache.druid.client.selector.ServerSelectorStrategy; import org.apache.druid.client.selector.TierSelectorStrategy; +import org.apache.druid.discovery.DataNodeService; import org.apache.druid.discovery.LookupNodeService; import org.apache.druid.discovery.NodeRole; import org.apache.druid.guice.CacheModule; @@ -42,9 +43,11 @@ import org.apache.druid.guice.JsonConfigProvider; import org.apache.druid.guice.LazySingleton; import org.apache.druid.guice.LifecycleModule; +import org.apache.druid.guice.ManageLifecycle; import org.apache.druid.guice.QueryRunnerFactoryModule; import org.apache.druid.guice.QueryableModule; import org.apache.druid.guice.SegmentWranglerModule; +import org.apache.druid.guice.ServerTypeConfig; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.query.QuerySegmentWalker; import org.apache.druid.query.RetryQueryRunnerConfig; @@ -52,7 +55,12 @@ import org.apache.druid.server.BrokerQueryResource; import org.apache.druid.server.ClientInfoResource; import org.apache.druid.server.ClientQuerySegmentWalker; +import org.apache.druid.server.SegmentManager; +import org.apache.druid.server.coordination.ServerType; +import org.apache.druid.server.coordination.ZkCoordinator; import org.apache.druid.server.http.BrokerResource; +import org.apache.druid.server.http.HistoricalResource; +import org.apache.druid.server.http.SegmentListerResource; import org.apache.druid.server.http.SelfDiscoveryResource; import org.apache.druid.server.initialization.jetty.JettyServerInitializer; import org.apache.druid.server.metrics.QueryCountStatsProvider; @@ -123,12 +131,19 @@ protected List getModules() Jerseys.addResource(binder, HttpServerInventoryViewResource.class); LifecycleModule.register(binder, Server.class); + binder.bind(SegmentManager.class).in(LazySingleton.class); + binder.bind(ZkCoordinator.class).in(ManageLifecycle.class); + binder.bind(ServerTypeConfig.class).toInstance(new ServerTypeConfig(ServerType.BROKER)); + Jerseys.addResource(binder, HistoricalResource.class); + Jerseys.addResource(binder, SegmentListerResource.class); + + LifecycleModule.register(binder, ZkCoordinator.class); bindNodeRoleAndAnnouncer( binder, DiscoverySideEffectsProvider .builder(NodeRole.BROKER) - .serviceClasses(ImmutableList.of(LookupNodeService.class)) + .serviceClasses(ImmutableList.of(DataNodeService.class, LookupNodeService.class)) .useLegacyAnnouncer(true) .build() ); diff --git a/services/src/main/java/org/apache/druid/cli/CliIndexer.java b/services/src/main/java/org/apache/druid/cli/CliIndexer.java index 7483fec93a93..7029d6f039bc 100644 --- a/services/src/main/java/org/apache/druid/cli/CliIndexer.java +++ b/services/src/main/java/org/apache/druid/cli/CliIndexer.java @@ -25,9 +25,9 @@ import com.google.inject.Module; import com.google.inject.Provides; import com.google.inject.name.Names; -import com.google.inject.util.Providers; import io.airlift.airline.Command; import org.apache.druid.client.DruidServer; +import org.apache.druid.client.DruidServerConfig; import org.apache.druid.discovery.DataNodeService; import org.apache.druid.discovery.LookupNodeService; import org.apache.druid.discovery.NodeRole; @@ -43,6 +43,7 @@ import org.apache.druid.guice.JsonConfigProvider; import org.apache.druid.guice.LazySingleton; import org.apache.druid.guice.LifecycleModule; +import org.apache.druid.guice.ManageLifecycle; import org.apache.druid.guice.QueryRunnerFactoryModule; import org.apache.druid.guice.QueryableModule; import org.apache.druid.guice.QueryablePeonModule; @@ -58,14 +59,16 @@ import org.apache.druid.indexing.worker.config.WorkerConfig; import org.apache.druid.indexing.worker.http.ShuffleResource; import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.metadata.input.InputSourceModule; import org.apache.druid.query.QuerySegmentWalker; import org.apache.druid.query.lookup.LookupModule; -import org.apache.druid.segment.realtime.CliIndexerDataSegmentServerAnnouncerLifecycleHandler; import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager; import org.apache.druid.segment.realtime.appenderator.UnifiedIndexerAppenderatorsManager; import org.apache.druid.server.DruidNode; -import org.apache.druid.server.coordination.SegmentLoadDropHandler; +import org.apache.druid.server.SegmentManager; import org.apache.druid.server.coordination.ServerType; +import org.apache.druid.server.coordination.ZkCoordinator; +import org.apache.druid.server.http.HistoricalResource; import org.apache.druid.server.http.SegmentListerResource; import org.apache.druid.server.initialization.jetty.CliIndexerServerModule; import org.apache.druid.server.initialization.jetty.JettyServerInitializer; @@ -138,14 +141,14 @@ public void configure(Binder binder) binder.bind(JettyServerInitializer.class).to(QueryJettyServerInitializer.class); Jerseys.addResource(binder, SegmentListerResource.class); - - LifecycleModule.register(binder, CliIndexerDataSegmentServerAnnouncerLifecycleHandler.class); - Jerseys.addResource(binder, ShuffleResource.class); LifecycleModule.register(binder, Server.class, RemoteChatHandler.class); - binder.bind(SegmentLoadDropHandler.class).toProvider(Providers.of(null)); + binder.bind(SegmentManager.class).in(LazySingleton.class); + binder.bind(ZkCoordinator.class).in(ManageLifecycle.class); + Jerseys.addResource(binder, HistoricalResource.class); + LifecycleModule.register(binder, ZkCoordinator.class); bindNodeRoleAndAnnouncer( binder, @@ -186,11 +189,11 @@ public WorkerNodeService getWorkerNodeService(WorkerConfig workerConfig) @Provides @LazySingleton - public DataNodeService getDataNodeService() + public DataNodeService getDataNodeService(DruidServerConfig serverConfig) { return new DataNodeService( DruidServer.DEFAULT_TIER, - 0L, + serverConfig.getMaxSize(), ServerType.INDEXER_EXECUTOR, DruidServer.DEFAULT_PRIORITY ); @@ -200,6 +203,7 @@ public DataNodeService getDataNodeService() new IndexingServiceInputSourceModule(), new IndexingServiceTaskLogsModule(), new IndexingServiceTuningConfigModule(), + new InputSourceModule(), new QueryablePeonModule(), new CliIndexerServerModule(properties), new LookupModule() diff --git a/services/src/main/java/org/apache/druid/cli/CliMiddleManager.java b/services/src/main/java/org/apache/druid/cli/CliMiddleManager.java index d273c16ad331..8b3d12f286eb 100644 --- a/services/src/main/java/org/apache/druid/cli/CliMiddleManager.java +++ b/services/src/main/java/org/apache/druid/cli/CliMiddleManager.java @@ -61,6 +61,7 @@ import org.apache.druid.indexing.worker.http.TaskManagementResource; import org.apache.druid.indexing.worker.http.WorkerResource; import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.metadata.input.InputSourceModule; import org.apache.druid.query.lookup.LookupSerdeModule; import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager; import org.apache.druid.segment.realtime.appenderator.DummyForInjectionAppenderatorsManager; @@ -186,6 +187,7 @@ public WorkerNodeService getWorkerNodeService(WorkerConfig workerConfig) new IndexingServiceInputSourceModule(), new IndexingServiceTaskLogsModule(), new IndexingServiceTuningConfigModule(), + new InputSourceModule(), new LookupSerdeModule() ); } diff --git a/services/src/main/java/org/apache/druid/cli/CliOverlord.java b/services/src/main/java/org/apache/druid/cli/CliOverlord.java index 21ffa9a8bff2..4baf155d229c 100644 --- a/services/src/main/java/org/apache/druid/cli/CliOverlord.java +++ b/services/src/main/java/org/apache/druid/cli/CliOverlord.java @@ -98,6 +98,7 @@ import org.apache.druid.indexing.overlord.supervisor.SupervisorResource; import org.apache.druid.indexing.worker.config.WorkerConfig; import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.metadata.input.InputSourceModule; import org.apache.druid.query.lookup.LookupSerdeModule; import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager; import org.apache.druid.segment.realtime.appenderator.DummyForInjectionAppenderatorsManager; @@ -347,6 +348,7 @@ private void configureOverlordHelpers(Binder binder) new IndexingServiceInputSourceModule(), new IndexingServiceTaskLogsModule(), new IndexingServiceTuningConfigModule(), + new InputSourceModule(), new SupervisorModule(), new LookupSerdeModule(), new SamplerModule() diff --git a/services/src/main/java/org/apache/druid/cli/CliPeon.java b/services/src/main/java/org/apache/druid/cli/CliPeon.java index 1160eb9240c1..2eb2c68521fc 100644 --- a/services/src/main/java/org/apache/druid/cli/CliPeon.java +++ b/services/src/main/java/org/apache/druid/cli/CliPeon.java @@ -61,7 +61,6 @@ import org.apache.druid.guice.annotations.Json; import org.apache.druid.guice.annotations.Parent; import org.apache.druid.guice.annotations.Self; -import org.apache.druid.guice.annotations.Smile; import org.apache.druid.indexing.common.RetryPolicyConfig; import org.apache.druid.indexing.common.RetryPolicyFactory; import org.apache.druid.indexing.common.SingleFileTaskReportFileWriter; @@ -92,6 +91,7 @@ import org.apache.druid.java.util.common.lifecycle.Lifecycle; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.metadata.IndexerSQLMetadataStorageCoordinator; +import org.apache.druid.metadata.input.InputSourceModule; import org.apache.druid.query.QuerySegmentWalker; import org.apache.druid.query.lookup.LookupModule; import org.apache.druid.segment.loading.DataSegmentArchiver; @@ -109,15 +109,16 @@ import org.apache.druid.segment.realtime.plumber.CoordinatorBasedSegmentHandoffNotifierFactory; import org.apache.druid.segment.realtime.plumber.SegmentHandoffNotifierFactory; import org.apache.druid.server.DruidNode; -import org.apache.druid.server.coordination.BatchDataSegmentAnnouncer; +import org.apache.druid.server.SegmentManager; import org.apache.druid.server.coordination.ServerType; +import org.apache.druid.server.coordination.ZkCoordinator; +import org.apache.druid.server.http.HistoricalResource; import org.apache.druid.server.http.SegmentListerResource; import org.apache.druid.server.initialization.jetty.ChatHandlerServerModule; import org.apache.druid.server.initialization.jetty.JettyServerInitializer; import org.apache.druid.server.metrics.DataSourceTaskIdHolder; import org.eclipse.jetty.server.Server; -import javax.annotation.Nullable; import java.io.File; import java.io.IOException; import java.util.List; @@ -154,6 +155,14 @@ public class CliPeon extends GuiceRunnable @Option(name = "--nodeType", title = "nodeType", description = "Set the node type to expose on ZK") public String serverType = "indexer-executor"; + + /** + * If set to "true", the peon will bind classes necessary for loading broadcast segments. This is used for + * queryable tasks, such as streaming ingestion tasks. + */ + @Option(name = "--loadBroadcastSegments", title = "loadBroadcastSegments", description = "Enable loading of broadcast segments") + public String loadBroadcastSegments = "false"; + private static final Logger log = new Logger(CliPeon.class); @Inject @@ -174,6 +183,7 @@ protected List getModules() new JoinableFactoryModule(), new Module() { + @SuppressForbidden(reason = "System#out, System#err") @Override public void configure(Binder binder) { @@ -218,6 +228,13 @@ public void configure(Binder binder) Jerseys.addResource(binder, SegmentListerResource.class); binder.bind(ServerTypeConfig.class).toInstance(new ServerTypeConfig(ServerType.fromString(serverType))); LifecycleModule.register(binder, Server.class); + + if ("true".equals(loadBroadcastSegments)) { + binder.bind(SegmentManager.class).in(LazySingleton.class); + binder.bind(ZkCoordinator.class).in(ManageLifecycle.class); + Jerseys.addResource(binder, HistoricalResource.class); + LifecycleModule.register(binder, ZkCoordinator.class); + } } @Provides @@ -247,21 +264,12 @@ public String getTaskIDFromTask(final Task task) { return task.getId(); } - - @Provides - public SegmentListerResource getSegmentListerResource( - @Json ObjectMapper jsonMapper, - @Smile ObjectMapper smileMapper, - @Nullable BatchDataSegmentAnnouncer announcer - ) - { - return new SegmentListerResource(jsonMapper, smileMapper, announcer, null); - } }, new QueryablePeonModule(), new IndexingServiceFirehoseModule(), new IndexingServiceInputSourceModule(), new IndexingServiceTuningConfigModule(), + new InputSourceModule(), new ChatHandlerServerModule(properties), new LookupModule() ); diff --git a/services/src/test/java/org/apache/druid/cli/MainTest.java b/services/src/test/java/org/apache/druid/cli/MainTest.java index 3e960f69e442..a1fbed581e5d 100644 --- a/services/src/test/java/org/apache/druid/cli/MainTest.java +++ b/services/src/test/java/org/apache/druid/cli/MainTest.java @@ -50,7 +50,9 @@ public static Iterable constructorFeeder() //new Object[]{new CliInternalHadoopIndexer()}, new Object[]{new CliMiddleManager()}, - new Object[]{new CliRouter()} + new Object[]{new CliRouter()}, + + new Object[]{new CliIndexer()} ); } diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/schema/DruidSchema.java b/sql/src/main/java/org/apache/druid/sql/calcite/schema/DruidSchema.java index 9f087eaf9ccd..54e7e5a8f961 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/schema/DruidSchema.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/schema/DruidSchema.java @@ -355,7 +355,7 @@ void addSegment(final DruidServerMetadata server, final DataSegment segment) AvailableSegmentMetadata segmentMetadata = knownSegments != null ? knownSegments.get(segment.getId()) : null; if (segmentMetadata == null) { // segmentReplicatable is used to determine if segments are served by historical or realtime servers - long isRealtime = server.segmentReplicatable() ? 0 : 1; + long isRealtime = server.isSegmentReplicationTarget() ? 0 : 1; segmentMetadata = AvailableSegmentMetadata.builder( segment, isRealtime, @@ -366,7 +366,7 @@ void addSegment(final DruidServerMetadata server, final DataSegment segment) // Unknown segment. setAvailableSegmentMetadata(segment.getId(), segmentMetadata); segmentsNeedingRefresh.add(segment.getId()); - if (!server.segmentReplicatable()) { + if (!server.isSegmentReplicationTarget()) { log.debug("Added new mutable segment[%s].", segment.getId()); mutableSegments.add(segment.getId()); } else { @@ -384,7 +384,7 @@ void addSegment(final DruidServerMetadata server, final DataSegment segment) .withRealtime(recomputeIsRealtime(servers)) .build(); knownSegments.put(segment.getId(), metadataWithNumReplicas); - if (server.segmentReplicatable()) { + if (server.isSegmentReplicationTarget()) { // If a segment shows up on a replicatable (historical) server at any point, then it must be immutable, // even if it's also available on non-replicatable (realtime) servers. mutableSegments.remove(segment.getId()); diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteQueryTest.java index e3579aad6a94..083285870044 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteQueryTest.java @@ -11887,7 +11887,7 @@ public void testTimeExtractWithTooFewArguments() throws Exception @Test @Parameters(source = QueryContextForJoinProvider.class) - public void testNestedGroupByOnInlineDataSourceWithFilterIsNotSupported(Map queryContext) throws Exception + public void testNestedGroupByOnInlineDataSourceWithFilter(Map queryContext) throws Exception { try { testQuery( diff --git a/website/.spelling b/website/.spelling index 7452d1403f92..faeceef243b1 100644 --- a/website/.spelling +++ b/website/.spelling @@ -98,6 +98,7 @@ IndexTask InfluxDB InputFormat InputSource +InputSources Integer.MAX_VALUE JBOD JDBC @@ -151,6 +152,7 @@ S3 SDK SIGAR SPNEGO +SqlInputSource SQLServer SSD SSDs @@ -1752,4 +1754,4 @@ UserGroupInformation CVE-2019-17571 CVE-2019-12399 CVE-2018-17196 -bin.tar.gz \ No newline at end of file +bin.tar.gz From f1aeddb63487c4e646acc7b541fd00a3079479df Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Tue, 9 Jun 2020 17:37:38 -0700 Subject: [PATCH 02/24] incomplete javadoc --- .../druid/timeline/partition/BuildingNumberedShardSpec.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/java/org/apache/druid/timeline/partition/BuildingNumberedShardSpec.java b/core/src/main/java/org/apache/druid/timeline/partition/BuildingNumberedShardSpec.java index 9c3b00edf1fd..478db2d28b98 100644 --- a/core/src/main/java/org/apache/druid/timeline/partition/BuildingNumberedShardSpec.java +++ b/core/src/main/java/org/apache/druid/timeline/partition/BuildingNumberedShardSpec.java @@ -47,7 +47,7 @@ * the segment lock is used, the Overlord coordinates the segment allocation and this class is never used. See * {@link PartialShardSpec} for that case. * - * This class should be Jackson-serializable + * This class should be Jackson-serializable as the subtasks can send it to the parallel task in parallel ingestion. * * Finally, this shardSpec has only partitionId which is same as {@link LinearShardSpec}. The difference between * them is this shardSpec should never be published and so never be used in other places such as Broker timeline. From 42e37e2e85f58af0b0da4c7fcf6cfad29af4dadb Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Wed, 10 Jun 2020 16:51:06 -0700 Subject: [PATCH 03/24] Address comments --- .../timeline/partition/BuildingNumberedShardSpec.java | 10 +++++++--- .../druid/timeline/partition/PartialShardSpec.java | 11 ++++++----- .../appenderator/TransactionalSegmentPublisher.java | 10 ---------- 3 files changed, 13 insertions(+), 18 deletions(-) diff --git a/core/src/main/java/org/apache/druid/timeline/partition/BuildingNumberedShardSpec.java b/core/src/main/java/org/apache/druid/timeline/partition/BuildingNumberedShardSpec.java index 478db2d28b98..43098aae6167 100644 --- a/core/src/main/java/org/apache/druid/timeline/partition/BuildingNumberedShardSpec.java +++ b/core/src/main/java/org/apache/druid/timeline/partition/BuildingNumberedShardSpec.java @@ -44,8 +44,10 @@ * This shardSpec is used for such use case. A non-appending batch task can use this shardSpec until it publishes * segments at last. When it publishes segments, it should convert the shardSpec of those segments to NumberedShardSpec. * See {@code SegmentPublisherHelper#annotateShardSpec} for converting to NumberedShardSpec. Note that, when - * the segment lock is used, the Overlord coordinates the segment allocation and this class is never used. See - * {@link PartialShardSpec} for that case. + * the segment lock is used, the Overlord coordinates the segment allocation and this class is never used. Instead, + * the task sends {@link PartialShardSpec} to the Overlord to allocate a new segment. The result segment could have + * either a {@link ShardSpec} (for root generation segments) or an {@link OverwriteShardSpec} (for non-root + * generation segments). * * This class should be Jackson-serializable as the subtasks can send it to the parallel task in parallel ingestion. * @@ -73,7 +75,9 @@ public NumberedShardSpec toNumberedShardSpec(int numTotalPartitions) @Override public PartitionChunk createChunk(T obj) { - return new LinearPartitionChunk<>(partitionId, obj); + // This method can be called in AppenderatorImpl to create a sinkTimeline. + // The sinkTimeline doesn't seem in use in batch ingestion, let's set 'chunks' to 0 for now. + return new NumberedPartitionChunk<>(partitionId, 0, obj); } @JsonProperty("partitionId") diff --git a/core/src/main/java/org/apache/druid/timeline/partition/PartialShardSpec.java b/core/src/main/java/org/apache/druid/timeline/partition/PartialShardSpec.java index 9ba9ac579c0e..6afaa939471e 100644 --- a/core/src/main/java/org/apache/druid/timeline/partition/PartialShardSpec.java +++ b/core/src/main/java/org/apache/druid/timeline/partition/PartialShardSpec.java @@ -28,11 +28,12 @@ import javax.annotation.Nullable; /** - * Class to contain all information of a {@link ShardSpec} except for the partition ID. - * This class is used when the segment allocation is coordinated by the Overlord; when appending segments to an - * existing datasource (either streaming ingestion or batch append) or when using segment locking. - * The ingestion tasks send all information required for allocating a new segment using this class and the Overlord - * determins the partition ID to create a new segment. + * This interface is used in the segment allocation protocol when it is coordinated by the Overlord; when appending + * segments to an existing datasource (either streaming ingestion or batch append) or any case when segment + * lock is used. The implementations of this interface contain all information of the corresponding {@link ShardSpec} + * except the partition ID. + * The ingestion tasks send all information required for allocating a new segment using this interface and the Overlord + * determines the partition ID to create a new segment. */ @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type") @JsonSubTypes({ 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 067e4f9b5cdd..26d8fac9dff5 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,21 +20,11 @@ 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.OverwriteShardSpec; -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 { From d5e197ea19b23d8cd68d61d9c5bb12da66a53332 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Wed, 10 Jun 2020 17:20:19 -0700 Subject: [PATCH 04/24] fix tests --- .../common/task/CompactionTaskRunTest.java | 14 ++-- .../SinglePhaseParallelIndexingTest.java | 64 ++++++++++++++++--- 2 files changed, 63 insertions(+), 15 deletions(-) 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 2a5b02e1db17..f6ffa78f5b20 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 @@ -252,7 +252,7 @@ public void testRun() throws Exception segments.get(i).getShardSpec() ); } else { - Assert.assertEquals(new NumberedShardSpec(0, 0), segments.get(i).getShardSpec()); + Assert.assertEquals(new NumberedShardSpec(0, 1), segments.get(i).getShardSpec()); } } @@ -301,7 +301,7 @@ public void testRunCompactionTwice() throws Exception segments.get(i).getShardSpec() ); } else { - Assert.assertEquals(new NumberedShardSpec(0, 0), segments.get(i).getShardSpec()); + Assert.assertEquals(new NumberedShardSpec(0, 1), segments.get(i).getShardSpec()); } } @@ -414,7 +414,7 @@ public void testRunIndexAndCompactAtTheSameTimeForDifferentInterval() throws Exc 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.assertEquals(new NumberedShardSpec(i % 2, 2), segments.get(i).getShardSpec()); } Assert.assertTrue(compactionFuture.get().lhs.isSuccess()); @@ -472,7 +472,7 @@ public void testWithSegmentGranularity() throws Exception Assert.assertEquals(1, segments.size()); Assert.assertEquals(Intervals.of("2014-01-01/2014-01-02"), segments.get(0).getInterval()); - Assert.assertEquals(new NumberedShardSpec(0, 0), segments.get(0).getShardSpec()); + Assert.assertEquals(new NumberedShardSpec(0, 1), segments.get(0).getShardSpec()); Assert.assertEquals(DEFAULT_COMPACTION_STATE, segments.get(0).getLastCompactionState()); // hour segmentGranularity @@ -594,7 +594,7 @@ public void testRunIndexAndCompactForSameSegmentAtTheSameTime() throws Exception segments.get(i).getShardSpec() ); } else { - Assert.assertEquals(new NumberedShardSpec(i % 2, 0), segments.get(i).getShardSpec()); + Assert.assertEquals(new NumberedShardSpec(i % 2, 2), segments.get(i).getShardSpec()); } } @@ -667,7 +667,7 @@ public void testRunIndexAndCompactForSameSegmentAtTheSameTime2() throws Exceptio segments.get(i).getShardSpec() ); } else { - Assert.assertEquals(new NumberedShardSpec(i % 2, 0), segments.get(i).getShardSpec()); + Assert.assertEquals(new NumberedShardSpec(i % 2, 2), segments.get(i).getShardSpec()); } } @@ -757,7 +757,7 @@ public void testRunRegularIndexTaskWithIngestSegmentFirehose() throws Exception segments.get(i).getShardSpec() ); } else { - Assert.assertEquals(new NumberedShardSpec(0, 0), segments.get(i).getShardSpec()); + Assert.assertEquals(new NumberedShardSpec(0, 1), segments.get(i).getShardSpec()); } } } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexingTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexingTest.java index 9b8129850e15..e1e21a56719d 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexingTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexingTest.java @@ -43,6 +43,7 @@ import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.Partitions; import org.apache.druid.timeline.VersionedIntervalTimeline; +import org.apache.druid.timeline.partition.NumberedOverwriteShardSpec; import org.apache.druid.timeline.partition.NumberedShardSpec; import org.joda.time.Interval; import org.junit.After; @@ -159,7 +160,19 @@ private void runTestTask(@Nullable Interval interval, Granularity segmentGranula final ParallelIndexSupervisorTask task = newTask(interval, segmentGranularity, appendToExisting, true); task.addToContext(Tasks.FORCE_TIME_CHUNK_LOCK_KEY, lockGranularity == LockGranularity.TIME_CHUNK); Assert.assertEquals(TaskState.SUCCESS, getIndexingServiceClient().runAndWait(task).getStatusCode()); - assertShardSpec(interval); + assertShardSpec(interval, appendToExisting); + } + + private void runOverwriteTask( + @Nullable Interval interval, + Granularity segmentGranularity, + LockGranularity actualLockGranularity + ) + { + final ParallelIndexSupervisorTask task = newTask(interval, segmentGranularity, false, true); + task.addToContext(Tasks.FORCE_TIME_CHUNK_LOCK_KEY, lockGranularity == LockGranularity.TIME_CHUNK); + Assert.assertEquals(TaskState.SUCCESS, getIndexingServiceClient().runAndWait(task).getStatusCode()); + assertShardSpecAfterOverwrite(interval, actualLockGranularity); } private void runTestTask(@Nullable Interval interval, Granularity segmentGranularity) @@ -178,7 +191,11 @@ private void testRunAndOverwrite(@Nullable Interval inputInterval, Granularity s ); // Reingest the same data. Each segment should get replaced by a segment with a newer version. - runTestTask(inputInterval, secondSegmentGranularity); + runOverwriteTask( + inputInterval, + secondSegmentGranularity, + secondSegmentGranularity.equals(Granularities.DAY) ? lockGranularity : LockGranularity.TIME_CHUNK + ); // Verify that the segment has been replaced. final Collection newSegments = @@ -189,12 +206,12 @@ private void testRunAndOverwrite(@Nullable Interval inputInterval, Granularity s Assert.assertEquals(new HashSet<>(newSegments), visibles); } - private void assertShardSpec(@Nullable Interval interval) + private void assertShardSpec(@Nullable Interval interval, boolean appendToExisting) { final Interval nonNullInterval = interval == null ? Intervals.ETERNITY : interval; final Collection segments = getStorageCoordinator().retrieveUsedSegmentsForInterval("dataSource", nonNullInterval, Segments.ONLY_VISIBLE); - if (lockGranularity != LockGranularity.SEGMENT) { + if (!appendToExisting && lockGranularity != LockGranularity.SEGMENT) { // Check the core partition set in the shardSpec final Map> intervalToSegments = new HashMap<>(); segments.forEach( @@ -216,6 +233,35 @@ private void assertShardSpec(@Nullable Interval interval) } } + private void assertShardSpecAfterOverwrite(@Nullable Interval interval, LockGranularity actualLockGranularity) + { + final Interval nonNullInterval = interval == null ? Intervals.ETERNITY : interval; + final Collection segments = + getStorageCoordinator().retrieveUsedSegmentsForInterval("dataSource", nonNullInterval, Segments.ONLY_VISIBLE); + final Map> intervalToSegments = new HashMap<>(); + segments.forEach( + segment -> intervalToSegments.computeIfAbsent(segment.getInterval(), k -> new ArrayList<>()).add(segment) + ); + if (actualLockGranularity != LockGranularity.SEGMENT) { + // Check the core partition set in the shardSpec + for (List segmentsPerInterval : intervalToSegments.values()) { + for (DataSegment segment : segmentsPerInterval) { + Assert.assertSame(NumberedShardSpec.class, segment.getShardSpec().getClass()); + final NumberedShardSpec shardSpec = (NumberedShardSpec) segment.getShardSpec(); + Assert.assertEquals(segmentsPerInterval.size(), shardSpec.getPartitions()); + } + } + } else { + for (List segmentsPerInterval : intervalToSegments.values()) { + for (DataSegment segment : segmentsPerInterval) { + Assert.assertSame(NumberedOverwriteShardSpec.class, segment.getShardSpec().getClass()); + final NumberedOverwriteShardSpec shardSpec = (NumberedOverwriteShardSpec) segment.getShardSpec(); + Assert.assertEquals(segmentsPerInterval.size(), shardSpec.getAtomicUpdateGroupSize()); + } + } + } + } + @Test public void testWithoutInterval() { @@ -246,10 +292,11 @@ public void testRunInParallelWithDifferentSegmentGranularity() public void testRunInSequential() { final Interval interval = Intervals.of("2017-12/P1M"); - final ParallelIndexSupervisorTask task = newTask(interval, false, false); + final boolean appendToExisting = false; + final ParallelIndexSupervisorTask task = newTask(interval, appendToExisting, false); task.addToContext(Tasks.FORCE_TIME_CHUNK_LOCK_KEY, lockGranularity == LockGranularity.TIME_CHUNK); Assert.assertEquals(TaskState.SUCCESS, getIndexingServiceClient().runAndWait(task).getStatusCode()); - assertShardSpec(interval); + assertShardSpec(interval, appendToExisting); } @Test @@ -264,10 +311,11 @@ public void testPublishEmptySegments() public void testWith1MaxNumConcurrentSubTasks() { final Interval interval = Intervals.of("2017-12/P1M"); + final boolean appendToExisting = false; final ParallelIndexSupervisorTask task = newTask( interval, Granularities.DAY, - false, + appendToExisting, true, new ParallelIndexTuningConfig( null, @@ -301,7 +349,7 @@ public void testWith1MaxNumConcurrentSubTasks() task.addToContext(Tasks.FORCE_TIME_CHUNK_LOCK_KEY, lockGranularity == LockGranularity.TIME_CHUNK); Assert.assertEquals(TaskState.SUCCESS, getIndexingServiceClient().runAndWait(task).getStatusCode()); Assert.assertNull("Runner must be null if the task was in the sequential mode", task.getCurrentRunner()); - assertShardSpec(interval); + assertShardSpec(interval, appendToExisting); } @Test From 99842f9d4793e185ab5c156ea1ff66b8ee187737 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Wed, 10 Jun 2020 18:13:07 -0700 Subject: [PATCH 05/24] fix json serde, add tests --- .../partition/BuildingNumberedShardSpec.java | 34 ++++++- .../partition/NumberedOverwriteShardSpec.java | 1 + .../druid/timeline/partition/ShardSpec.java | 11 ++- .../BuildingNumberedShardSpecTest.java | 93 +++++++++++++++++++ .../NumberedOverwriteShardSpecTest.java | 53 +++++++++++ .../common/task/CompactionTaskRunTest.java | 12 ++- 6 files changed, 198 insertions(+), 6 deletions(-) create mode 100644 core/src/test/java/org/apache/druid/timeline/partition/BuildingNumberedShardSpecTest.java create mode 100644 core/src/test/java/org/apache/druid/timeline/partition/NumberedOverwriteShardSpecTest.java diff --git a/core/src/main/java/org/apache/druid/timeline/partition/BuildingNumberedShardSpec.java b/core/src/main/java/org/apache/druid/timeline/partition/BuildingNumberedShardSpec.java index 43098aae6167..4604f23a982c 100644 --- a/core/src/main/java/org/apache/druid/timeline/partition/BuildingNumberedShardSpec.java +++ b/core/src/main/java/org/apache/druid/timeline/partition/BuildingNumberedShardSpec.java @@ -20,6 +20,7 @@ package org.apache.druid.timeline.partition; import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; import com.google.common.collect.RangeSet; @@ -27,6 +28,7 @@ import java.util.List; import java.util.Map; +import java.util.Objects; /** * This is a special shardSpec which is temporarily used during batch ingestion. In Druid, there is a concept @@ -58,10 +60,12 @@ */ public class BuildingNumberedShardSpec implements ShardSpec { + public static final String TYPE = "building_numbered"; + private final int partitionId; @JsonCreator - public BuildingNumberedShardSpec(int partitionId) + public BuildingNumberedShardSpec(@JsonProperty("partitionId") int partitionId) { Preconditions.checkArgument(partitionId >= 0, "partitionId >= 0"); this.partitionId = partitionId; @@ -101,6 +105,7 @@ public boolean isInChunk(long timestamp, InputRow inputRow) throw new UnsupportedOperationException(); } + @JsonIgnore @Override public List getDomainDimensions() { @@ -118,4 +123,31 @@ public boolean isCompatible(Class other) { throw new UnsupportedOperationException(); } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + BuildingNumberedShardSpec shardSpec = (BuildingNumberedShardSpec) o; + return partitionId == shardSpec.partitionId; + } + + @Override + public int hashCode() + { + return Objects.hash(partitionId); + } + + @Override + public String toString() + { + return "BuildingNumberedShardSpec{" + + "partitionId=" + partitionId + + '}'; + } } 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 d07ce9d4f060..dbbb8f66e936 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 @@ -53,6 +53,7 @@ */ public class NumberedOverwriteShardSpec implements OverwriteShardSpec { + public static final String TYPE = "numbered_overwrite"; private final int partitionId; private final short startRootPartitionId; 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 6b96d6560161..5717ef159f8d 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 @@ -19,6 +19,8 @@ package org.apache.druid.timeline.partition; +import com.fasterxml.jackson.annotation.JsonIgnore; +import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonSubTypes; import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.google.common.collect.RangeSet; @@ -38,12 +40,15 @@ @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 = NumberedOverwriteShardSpec.class) + @JsonSubTypes.Type(name = NumberedOverwriteShardSpec.TYPE, value = NumberedOverwriteShardSpec.class), + @JsonSubTypes.Type(name = BuildingNumberedShardSpec.TYPE, value = BuildingNumberedShardSpec.class) }) public interface ShardSpec { + @JsonIgnore PartitionChunk createChunk(T obj); + @JsonIgnore boolean isInChunk(long timestamp, InputRow inputRow); /** @@ -91,6 +96,7 @@ default short getAtomicUpdateGroupSize() return 1; } + @JsonIgnore ShardSpecLookup getLookup(List shardSpecs); /** @@ -98,16 +104,19 @@ default short getAtomicUpdateGroupSize() * * @return list of dimensions who has its possible range. Dimensions with unknown possible range are not listed */ + @JsonIgnore List getDomainDimensions(); /** * if given domain ranges are not possible in this shard, return false; otherwise return true; * @return possibility of in domain */ + @JsonIgnore boolean possibleInDomain(Map> domain); /** * Returns true if two segments of this and other shardSpecs can exist in the same time chunk. */ + @JsonIgnore boolean isCompatible(Class other); } diff --git a/core/src/test/java/org/apache/druid/timeline/partition/BuildingNumberedShardSpecTest.java b/core/src/test/java/org/apache/druid/timeline/partition/BuildingNumberedShardSpecTest.java new file mode 100644 index 000000000000..21c5a03644e5 --- /dev/null +++ b/core/src/test/java/org/apache/druid/timeline/partition/BuildingNumberedShardSpecTest.java @@ -0,0 +1,93 @@ +/* + * 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.core.JsonProcessingException; +import com.fasterxml.jackson.databind.DeserializationFeature; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.jsontype.NamedType; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import nl.jqno.equalsverifier.EqualsVerifier; +import org.apache.druid.data.input.MapBasedInputRow; +import org.apache.druid.java.util.common.DateTimes; +import org.junit.Assert; +import org.junit.Test; + +import java.util.List; + +public class BuildingNumberedShardSpecTest +{ + @Test + public void testToNumberedShardSpec() + { + Assert.assertEquals(new NumberedShardSpec(5, 10), new BuildingNumberedShardSpec(5).toNumberedShardSpec(10)); + } + + @Test + public void testCreateChunk() + { + Assert.assertEquals( + new NumberedPartitionChunk<>(5, 0, "test"), + new BuildingNumberedShardSpec(5).createChunk("test") + ); + } + + @Test + public void testShardSpecLookup() + { + final List shardSpecs = ImmutableList.of( + new BuildingNumberedShardSpec(1), + new BuildingNumberedShardSpec(2), + new BuildingNumberedShardSpec(3) + ); + final ShardSpecLookup lookup = shardSpecs.get(0).getLookup(shardSpecs); + // Timestamp doesn't matter. It always returns the first shardSpec. + final long currentTime = DateTimes.nowUtc().getMillis(); + Assert.assertEquals( + shardSpecs.get(0), + lookup.getShardSpec( + currentTime, + new MapBasedInputRow( + currentTime, + ImmutableList.of("dim"), ImmutableMap.of("dim", "val", "time", currentTime) + ) + ) + ); + } + + @Test + public void testSerde() throws JsonProcessingException + { + final ObjectMapper mapper = new ObjectMapper(); + mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false); + mapper.registerSubtypes(new NamedType(BuildingNumberedShardSpec.class, BuildingNumberedShardSpec.TYPE)); + final BuildingNumberedShardSpec original = new BuildingNumberedShardSpec(5); + final String json = mapper.writeValueAsString(original); + final BuildingNumberedShardSpec fromJson = (BuildingNumberedShardSpec) mapper.readValue(json, ShardSpec.class); + Assert.assertEquals(original, fromJson); + } + + @Test + public void testEquals() + { + EqualsVerifier.forClass(BuildingNumberedShardSpec.class).usingGetClass().verify(); + } +} diff --git a/core/src/test/java/org/apache/druid/timeline/partition/NumberedOverwriteShardSpecTest.java b/core/src/test/java/org/apache/druid/timeline/partition/NumberedOverwriteShardSpecTest.java new file mode 100644 index 000000000000..d605105c48d5 --- /dev/null +++ b/core/src/test/java/org/apache/druid/timeline/partition/NumberedOverwriteShardSpecTest.java @@ -0,0 +1,53 @@ +/* + * 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.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.jsontype.NamedType; +import nl.jqno.equalsverifier.EqualsVerifier; +import org.junit.Assert; +import org.junit.Test; + +public class NumberedOverwriteShardSpecTest +{ + @Test + public void testEquals() + { + EqualsVerifier.forClass(NumberedOverwriteShardSpec.class).usingGetClass().verify(); + } + + @Test + public void testSerde() throws JsonProcessingException + { + final ObjectMapper mapper = new ObjectMapper(); + mapper.registerSubtypes(new NamedType(NumberedOverwriteShardSpec.class, NumberedOverwriteShardSpec.TYPE)); + final NumberedOverwriteShardSpec original = new NumberedOverwriteShardSpec( + PartitionIds.NON_ROOT_GEN_START_PARTITION_ID + 2, + 0, + 10, + (short) 1, + (short) 3 + ); + final String json = mapper.writeValueAsString(original); + final NumberedOverwriteShardSpec fromJson = (NumberedOverwriteShardSpec) mapper.readValue(json, ShardSpec.class); + Assert.assertEquals(original, fromJson); + } +} 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 f6ffa78f5b20..e08361450fb9 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 @@ -334,7 +334,7 @@ public void testRunCompactionTwice() throws Exception segments.get(i).getShardSpec() ); } else { - Assert.assertEquals(new NumberedShardSpec(0, 0), segments.get(i).getShardSpec()); + Assert.assertEquals(new NumberedShardSpec(0, 1), segments.get(i).getShardSpec()); } } } @@ -414,7 +414,11 @@ public void testRunIndexAndCompactAtTheSameTimeForDifferentInterval() throws Exc 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, 2), segments.get(i).getShardSpec()); + if (lockGranularity == LockGranularity.SEGMENT) { + Assert.assertEquals(new NumberedShardSpec(i % 2, 0), segments.get(i).getShardSpec()); + } else { + Assert.assertEquals(new NumberedShardSpec(i % 2, 2), segments.get(i).getShardSpec()); + } } Assert.assertTrue(compactionFuture.get().lhs.isSuccess()); @@ -434,7 +438,7 @@ public void testRunIndexAndCompactAtTheSameTimeForDifferentInterval() throws Exc segments.get(i).getShardSpec() ); } else { - Assert.assertEquals(new NumberedShardSpec(0, 0), segments.get(i).getShardSpec()); + Assert.assertEquals(new NumberedShardSpec(0, 1), segments.get(i).getShardSpec()); } } } @@ -490,7 +494,7 @@ public void testWithSegmentGranularity() 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 NumberedShardSpec(0, 1), segments.get(i).getShardSpec()); Assert.assertEquals(DEFAULT_COMPACTION_STATE, segments.get(i).getLastCompactionState()); } } From b2222b9a6596a88a9f44aa966adc364ba087c10e Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Wed, 10 Jun 2020 20:18:48 -0700 Subject: [PATCH 06/24] checkstyle --- .../main/java/org/apache/druid/timeline/partition/ShardSpec.java | 1 - 1 file changed, 1 deletion(-) 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 5717ef159f8d..06ff8dd6720d 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 @@ -20,7 +20,6 @@ package org.apache.druid.timeline.partition; import com.fasterxml.jackson.annotation.JsonIgnore; -import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonSubTypes; import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.google.common.collect.RangeSet; From f8ac529290320e64c4dd3e44a0c5d86f218397cd Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Thu, 11 Jun 2020 10:24:19 -0700 Subject: [PATCH 07/24] Set core partition set size for hash-partitioned segments properly in batch ingestion --- .../BuildingHashBasedNumberedShardSpec.java | 131 +++++++++++ .../partition/BuildingNumberedShardSpec.java | 62 +----- .../timeline/partition/BuildingShardSpec.java | 81 +++++++ .../HashBasedNumberedPartialShardSpec.java | 33 ++- .../partition/HashBasedNumberedShardSpec.java | 85 ++++++-- .../timeline/partition/NumberedShardSpec.java | 16 +- .../druid/timeline/partition/ShardSpec.java | 3 +- ...uildingHashBasedNumberedShardSpecTest.java | 134 ++++++++++++ .../BuildingNumberedShardSpecTest.java | 4 +- .../MaterializedViewSupervisorTest.java | 16 +- .../indexer/DetermineHashedPartitionsJob.java | 1 + .../HadoopDruidDetermineConfigurationJob.java | 1 + .../indexer/BatchDeltaIngestionTest.java | 2 +- .../indexer/HadoopDruidIndexerConfigTest.java | 2 +- .../druid/indexer/IndexGeneratorJobTest.java | 1 + .../parallel/PartialHashSegmentMergeTask.java | 8 +- .../partition/HashPartitionAnalysis.java | 4 +- .../actions/SegmentAllocateActionTest.java | 4 +- .../indexing/common/task/ShardSpecsTest.java | 4 +- ...bstractMultiPhaseParallelIndexingTest.java | 47 ++-- ...rtitionAdjustingCorePartitionSizeTest.java | 205 ++++++++++++++++++ .../parallel/ParallelIndexTestingFactory.java | 1 + .../appenderator/SegmentPublisherHelper.java | 14 +- .../server/shard/NumberedShardSpecTest.java | 7 + .../HashBasedNumberedShardSpecTest.java | 61 +++--- 25 files changed, 764 insertions(+), 163 deletions(-) create mode 100644 core/src/main/java/org/apache/druid/timeline/partition/BuildingHashBasedNumberedShardSpec.java create mode 100644 core/src/main/java/org/apache/druid/timeline/partition/BuildingShardSpec.java create mode 100644 core/src/test/java/org/apache/druid/timeline/partition/BuildingHashBasedNumberedShardSpecTest.java create mode 100644 indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionAdjustingCorePartitionSizeTest.java diff --git a/core/src/main/java/org/apache/druid/timeline/partition/BuildingHashBasedNumberedShardSpec.java b/core/src/main/java/org/apache/druid/timeline/partition/BuildingHashBasedNumberedShardSpec.java new file mode 100644 index 000000000000..c9fee408a2f7 --- /dev/null +++ b/core/src/main/java/org/apache/druid/timeline/partition/BuildingHashBasedNumberedShardSpec.java @@ -0,0 +1,131 @@ +/* + * 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.JacksonInject; +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; + +/** + * See {@link BuildingShardSpec} for how this class is used. + * + * @see HashBasedNumberedShardSpec + */ +public class BuildingHashBasedNumberedShardSpec implements BuildingShardSpec +{ + public static final String TYPE = "building_hashed"; + + private final int partitionId; + private final int numBuckets; + private final List partitionDimensions; + private final ObjectMapper jsonMapper; + + @JsonCreator + public BuildingHashBasedNumberedShardSpec( + @JsonProperty("partitionId") int partitionId, + @JsonProperty("numBuckets") int numBuckets, + @JsonProperty("partitionDimensions") @Nullable List partitionDimensions, + @JacksonInject ObjectMapper jsonMapper + ) + { + this.partitionId = partitionId; + this.numBuckets = numBuckets; + this.partitionDimensions = partitionDimensions == null + ? HashBasedNumberedShardSpec.DEFAULT_PARTITION_DIMENSIONS + : partitionDimensions; + this.jsonMapper = jsonMapper; + } + + @JsonProperty("partitionId") + @Override + public int getPartitionNum() + { + return partitionId; + } + + @JsonProperty + public int getNumBuckets() + { + return numBuckets; + } + + @JsonProperty + public List getPartitionDimensions() + { + return partitionDimensions; + } + + @Override + public PartitionChunk createChunk(T obj) + { + // This method can be called in AppenderatorImpl to create a sinkTimeline. + // The sinkTimeline doesn't seem in use in batch ingestion, let's set 'chunks' to 0 for now. + // HashBasedNumberedShardSpec is using NumberedPartitionChunk, so we use it here too. + return new NumberedPartitionChunk<>(partitionId, 0, obj); + } + + @Override + public ShardSpecLookup getLookup(List shardSpecs) + { + return HashBasedNumberedShardSpec.createHashLookup(jsonMapper, partitionDimensions, shardSpecs, numBuckets); + } + + @Override + public HashBasedNumberedShardSpec convert(int numCorePartitions) + { + return new HashBasedNumberedShardSpec(partitionId, numCorePartitions, numBuckets, partitionDimensions, jsonMapper); + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + BuildingHashBasedNumberedShardSpec that = (BuildingHashBasedNumberedShardSpec) o; + return partitionId == that.partitionId && + numBuckets == that.numBuckets && + Objects.equals(partitionDimensions, that.partitionDimensions); + } + + @Override + public int hashCode() + { + return Objects.hash(partitionId, numBuckets, partitionDimensions); + } + + @Override + public String toString() + { + return "BuildingHashBasedNumberedShardSpec{" + + "partitionId=" + partitionId + + ", numBuckets=" + numBuckets + + ", partitionDimensions=" + partitionDimensions + + '}'; + } +} diff --git a/core/src/main/java/org/apache/druid/timeline/partition/BuildingNumberedShardSpec.java b/core/src/main/java/org/apache/druid/timeline/partition/BuildingNumberedShardSpec.java index 4604f23a982c..d17ea8f21fa1 100644 --- a/core/src/main/java/org/apache/druid/timeline/partition/BuildingNumberedShardSpec.java +++ b/core/src/main/java/org/apache/druid/timeline/partition/BuildingNumberedShardSpec.java @@ -20,45 +20,21 @@ package org.apache.druid.timeline.partition; import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonIgnore; 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.List; -import java.util.Map; import java.util.Objects; /** - * This is a special shardSpec which is temporarily used during batch ingestion. In Druid, there is a concept - * of core partition set which is a set of segments atomically becoming queryable together in Brokers. The core - * partition set is represented as a range of partitionIds. For {@link NumberedShardSpec}, the core partition set - * is [0, {@link NumberedShardSpec#partitions}). + * See {@link BuildingShardSpec} for how this class is used. * - * The NumberedShardSpec is used for dynamic partitioning which is based on the number of rows in each segment. - * In streaming ingestion, the core partition set size cannot be determined since it's impossible to know how many - * segments will be created per time chunk. However, in batch ingestion with time chunk locking, the core partition - * set is the set of segments created by an initial task or an overwriting task. Since the core partition set is - * determined when the task publishes segments at the end, the task postpones creating proper NumberedShardSpec - * until the end. - * - * This shardSpec is used for such use case. A non-appending batch task can use this shardSpec until it publishes - * segments at last. When it publishes segments, it should convert the shardSpec of those segments to NumberedShardSpec. - * See {@code SegmentPublisherHelper#annotateShardSpec} for converting to NumberedShardSpec. Note that, when - * the segment lock is used, the Overlord coordinates the segment allocation and this class is never used. Instead, - * the task sends {@link PartialShardSpec} to the Overlord to allocate a new segment. The result segment could have - * either a {@link ShardSpec} (for root generation segments) or an {@link OverwriteShardSpec} (for non-root - * generation segments). - * - * This class should be Jackson-serializable as the subtasks can send it to the parallel task in parallel ingestion. - * - * Finally, this shardSpec has only partitionId which is same as {@link LinearShardSpec}. The difference between + * This shardSpec has only partitionId which is same as {@link LinearShardSpec}. The difference between * them is this shardSpec should never be published and so never be used in other places such as Broker timeline. * * @see NumberedShardSpec */ -public class BuildingNumberedShardSpec implements ShardSpec +public class BuildingNumberedShardSpec implements BuildingShardSpec { public static final String TYPE = "building_numbered"; @@ -71,7 +47,8 @@ public BuildingNumberedShardSpec(@JsonProperty("partitionId") int partitionId) this.partitionId = partitionId; } - public NumberedShardSpec toNumberedShardSpec(int numTotalPartitions) + @Override + public NumberedShardSpec convert(int numTotalPartitions) { return new NumberedShardSpec(partitionId, numTotalPartitions); } @@ -94,34 +71,7 @@ public int getPartitionNum() @Override public ShardSpecLookup getLookup(List shardSpecs) { - return NumberedShardSpec.createLookup(shardSpecs); - } - - // The below methods are used on the query side, and so must not be called for this shardSpec. - - @Override - public boolean isInChunk(long timestamp, InputRow inputRow) - { - throw new UnsupportedOperationException(); - } - - @JsonIgnore - @Override - public List getDomainDimensions() - { - throw new UnsupportedOperationException(); - } - - @Override - public boolean possibleInDomain(Map> domain) - { - throw new UnsupportedOperationException(); - } - - @Override - public boolean isCompatible(Class other) - { - throw new UnsupportedOperationException(); + return NumberedShardSpec.createNumberedLookup(shardSpecs); } @Override diff --git a/core/src/main/java/org/apache/druid/timeline/partition/BuildingShardSpec.java b/core/src/main/java/org/apache/druid/timeline/partition/BuildingShardSpec.java new file mode 100644 index 000000000000..ed1136e80a29 --- /dev/null +++ b/core/src/main/java/org/apache/druid/timeline/partition/BuildingShardSpec.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.timeline.partition; + +import com.fasterxml.jackson.annotation.JsonIgnore; +import com.google.common.collect.RangeSet; +import org.apache.druid.data.input.InputRow; + +import java.util.List; +import java.util.Map; + +/** + * This is a special shardSpec which is temporarily used during batch ingestion. In Druid, there is a concept + * of core partition set which is a set of segments atomically becoming queryable together in Brokers. The core + * partition set is represented as a range of partitionIds. For {@link NumberedShardSpec} as an example, the core + * partition set is [0, {@link NumberedShardSpec#partitions}). + * + * In streaming ingestion, the core partition set size cannot be determined since it's impossible to know how many + * segments will be created per time chunk upfront. However, in batch ingestion with time chunk locking, the core + * partition set is the set of segments created by an initial task or an overwriting task. Since the core partition + * set is determined when the task publishes segments at the end, the task postpones creating proper {@link ShardSpec} + * until the end. + * + * This BuildingShardSpec is used for such use case. A non-appending batch task can use this shardSpec until it + * publishes segments at last. When it publishes segments, it should convert the buildingShardSpec of those segments + * to a proper shardSpec type {@link T}. See {@code SegmentPublisherHelper#annotateShardSpec} for converting shardSpec. + * Note that, when the segment lock is used, the Overlord coordinates the segment allocation and this class is never + * used. Instead, the task sends {@link PartialShardSpec} to the Overlord to allocate a new segment. The result segment + * could have either a {@link ShardSpec} (for root generation segments) or an {@link OverwriteShardSpec} (for non-root + * generation segments). + * + * This class should be Jackson-serializable as the subtasks can send it to the parallel task in parallel ingestion. + */ +public interface BuildingShardSpec extends ShardSpec +{ + T convert(int numCorePartitions); + + // The below methods are used on the query side, and so must not be called for this shardSpec. + + @Override + default boolean isInChunk(long timestamp, InputRow inputRow) + { + throw new UnsupportedOperationException(); + } + + @JsonIgnore + @Override + default List getDomainDimensions() + { + throw new UnsupportedOperationException(); + } + + @Override + default boolean possibleInDomain(Map> domain) + { + throw new UnsupportedOperationException(); + } + + @Override + default boolean isCompatible(Class other) + { + throw new UnsupportedOperationException(); + } +} diff --git a/core/src/main/java/org/apache/druid/timeline/partition/HashBasedNumberedPartialShardSpec.java b/core/src/main/java/org/apache/druid/timeline/partition/HashBasedNumberedPartialShardSpec.java index 495a85284bb3..895c1b7b1cc9 100644 --- a/core/src/main/java/org/apache/druid/timeline/partition/HashBasedNumberedPartialShardSpec.java +++ b/core/src/main/java/org/apache/druid/timeline/partition/HashBasedNumberedPartialShardSpec.java @@ -61,19 +61,36 @@ public int getNumBuckets() @Override public ShardSpec complete(ObjectMapper objectMapper, @Nullable ShardSpec specOfPreviousMaxPartitionId) { - final HashBasedNumberedShardSpec prevSpec = (HashBasedNumberedShardSpec) specOfPreviousMaxPartitionId; - return new HashBasedNumberedShardSpec( - prevSpec == null ? 0 : prevSpec.getPartitionNum() + 1, - numBuckets, - partitionDimensions, - objectMapper - ); + if (specOfPreviousMaxPartitionId == null) { + // The shardSpec is created by the Overlord. + // For batch tasks, this code can is executed only with segment locking (forceTimeChunkLock = false). + // In this mode, you can have 2 or more tasks concurrently ingesting into the same time chunk of + // the same datasource. Since there is no restriction for those tasks in segment allocation, the + // allocated IDs for each task can interleave. As a result, the core partition set cannot be + // represented as a range. We always set 0 for the core partition set size. + return new HashBasedNumberedShardSpec( + 0, + 0, + numBuckets, + partitionDimensions, + objectMapper + ); + } else { + final HashBasedNumberedShardSpec prevSpec = (HashBasedNumberedShardSpec) specOfPreviousMaxPartitionId; + return new HashBasedNumberedShardSpec( + prevSpec.getPartitionNum() + 1, + prevSpec.getPartitions(), + numBuckets, + partitionDimensions, + objectMapper + ); + } } @Override public ShardSpec complete(ObjectMapper objectMapper, int partitionId) { - return new HashBasedNumberedShardSpec(partitionId, numBuckets, partitionDimensions, objectMapper); + return new HashBasedNumberedShardSpec(partitionId, 0, numBuckets, partitionDimensions, objectMapper); } @Override 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 2d8f525d6fc9..85d798edb031 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 @@ -35,29 +35,45 @@ import javax.annotation.Nullable; import java.util.List; +import java.util.Objects; public class HashBasedNumberedShardSpec extends NumberedShardSpec { + static final List DEFAULT_PARTITION_DIMENSIONS = ImmutableList.of(); + private static final HashFunction HASH_FUNCTION = Hashing.murmur3_32(); - private static final List DEFAULT_PARTITION_DIMENSIONS = ImmutableList.of(); + /** + * Number of hash buckets + */ + private final int numBuckets; private final ObjectMapper jsonMapper; @JsonIgnore private final List partitionDimensions; @JsonCreator public HashBasedNumberedShardSpec( - @JsonProperty("partitionNum") int partitionNum, // partitionId - @JsonProperty("partitions") int partitions, // # of partitions + @JsonProperty("partitionNum") int partitionNum, // partitionId, hash bucketId + @JsonProperty("partitions") int partitions, // core partition set size + @JsonProperty("numBuckets") @Nullable Integer numBuckets, // nullable for backward compatibility @JsonProperty("partitionDimensions") @Nullable List partitionDimensions, @JacksonInject ObjectMapper jsonMapper ) { super(partitionNum, partitions); + // If numBuckets is missing, assume that any hash bucket is not empty. + // Use the core partition set size as the number of buckets. + this.numBuckets = numBuckets == null ? partitions : numBuckets; this.jsonMapper = jsonMapper; this.partitionDimensions = partitionDimensions == null ? DEFAULT_PARTITION_DIMENSIONS : partitionDimensions; } + @JsonProperty + public int getNumBuckets() + { + return numBuckets; + } + @JsonProperty("partitionDimensions") public List getPartitionDimensions() { @@ -73,7 +89,7 @@ public boolean isCompatible(Class other) @Override public boolean isInChunk(long timestamp, InputRow inputRow) { - return (((long) hash(timestamp, inputRow)) - getPartitionNum()) % getPartitions() == 0; + return (((long) hash(timestamp, inputRow)) - getPartitionNum()) % numBuckets == 0; } /** @@ -88,7 +104,12 @@ public boolean isInChunk(long timestamp, InputRow inputRow) */ protected int hash(long timestamp, InputRow inputRow) { - final List groupKey = getGroupKey(timestamp, inputRow); + return hash(jsonMapper, partitionDimensions, timestamp, inputRow); + } + + private static int hash(ObjectMapper jsonMapper, List partitionDimensions, long timestamp, InputRow inputRow) + { + final List groupKey = getGroupKey(partitionDimensions, timestamp, inputRow); try { return hash(jsonMapper, groupKey); } @@ -98,7 +119,7 @@ protected int hash(long timestamp, InputRow inputRow) } @VisibleForTesting - List getGroupKey(final long timestamp, final InputRow inputRow) + static List getGroupKey(final List partitionDimensions, final long timestamp, final InputRow inputRow) { if (partitionDimensions.isEmpty()) { return Rows.toGroupKey(timestamp, inputRow); @@ -114,21 +135,55 @@ public static int hash(ObjectMapper jsonMapper, List objects) throws Jso } @Override - public String toString() + public ShardSpecLookup getLookup(final List shardSpecs) { - return "HashBasedNumberedShardSpec{" + - "partitionNum=" + getPartitionNum() + - ", partitions=" + getPartitions() + - ", partitionDimensions=" + getPartitionDimensions() + - '}'; + return createHashLookup(jsonMapper, partitionDimensions, shardSpecs, numBuckets); } - @Override - public ShardSpecLookup getLookup(final List shardSpecs) + static ShardSpecLookup createHashLookup( + ObjectMapper jsonMapper, + List partitionDimensions, + List shardSpecs, + int numBuckets + ) { return (long timestamp, InputRow row) -> { - int index = Math.abs(hash(timestamp, row) % getPartitions()); + int index = Math.abs(hash(jsonMapper, partitionDimensions, timestamp, row) % numBuckets); return shardSpecs.get(index); }; } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + if (!super.equals(o)) { + return false; + } + HashBasedNumberedShardSpec that = (HashBasedNumberedShardSpec) o; + return numBuckets == that.numBuckets && + Objects.equals(partitionDimensions, that.partitionDimensions); + } + + @Override + public int hashCode() + { + return Objects.hash(super.hashCode(), numBuckets, partitionDimensions); + } + + @Override + public String toString() + { + return "HashBasedNumberedShardSpec{" + + "partitionNum=" + getPartitionNum() + + ", partitions=" + getPartitions() + + ", numBuckets=" + numBuckets + + ", partitionDimensions=" + partitionDimensions + + '}'; + } } 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 6f8898e298f1..59ebba4aa152 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 @@ -68,10 +68,10 @@ public int getPartitionNum() @Override public ShardSpecLookup getLookup(final List shardSpecs) { - return createLookup(shardSpecs); + return createNumberedLookup(shardSpecs); } - static ShardSpecLookup createLookup(List shardSpecs) + static ShardSpecLookup createNumberedLookup(List shardSpecs) { return (long timestamp, InputRow row) -> shardSpecs.get(0); } @@ -127,16 +127,12 @@ public boolean equals(Object o) if (this == o) { return true; } - - if (!(o instanceof NumberedShardSpec)) { - return false; - } - - final NumberedShardSpec that = (NumberedShardSpec) o; - if (partitionNum != that.partitionNum) { + if (o == null || getClass() != o.getClass()) { return false; } - return partitions == that.partitions; + NumberedShardSpec that = (NumberedShardSpec) o; + return partitionNum == that.partitionNum && + partitions == that.partitions; } @Override 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 06ff8dd6720d..c964a7065ebd 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 @@ -40,7 +40,8 @@ @JsonSubTypes.Type(name = "numbered", value = NumberedShardSpec.class), @JsonSubTypes.Type(name = "hashed", value = HashBasedNumberedShardSpec.class), @JsonSubTypes.Type(name = NumberedOverwriteShardSpec.TYPE, value = NumberedOverwriteShardSpec.class), - @JsonSubTypes.Type(name = BuildingNumberedShardSpec.TYPE, value = BuildingNumberedShardSpec.class) + @JsonSubTypes.Type(name = BuildingNumberedShardSpec.TYPE, value = BuildingNumberedShardSpec.class), + @JsonSubTypes.Type(name = BuildingHashBasedNumberedShardSpec.TYPE, value = BuildingHashBasedNumberedShardSpec.class) }) public interface ShardSpec { diff --git a/core/src/test/java/org/apache/druid/timeline/partition/BuildingHashBasedNumberedShardSpecTest.java b/core/src/test/java/org/apache/druid/timeline/partition/BuildingHashBasedNumberedShardSpecTest.java new file mode 100644 index 000000000000..46ecec42c87f --- /dev/null +++ b/core/src/test/java/org/apache/druid/timeline/partition/BuildingHashBasedNumberedShardSpecTest.java @@ -0,0 +1,134 @@ +/* + * 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.core.JsonProcessingException; +import com.fasterxml.jackson.databind.DeserializationFeature; +import com.fasterxml.jackson.databind.InjectableValues.Std; +import com.fasterxml.jackson.databind.MapperFeature; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.jsontype.NamedType; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import nl.jqno.equalsverifier.EqualsVerifier; +import org.apache.druid.data.input.MapBasedInputRow; +import org.apache.druid.java.util.common.DateTimes; +import org.junit.Assert; +import org.junit.Test; + +import java.util.List; + +public class BuildingHashBasedNumberedShardSpecTest +{ + @Test + public void testConvert() + { + Assert.assertEquals( + new HashBasedNumberedShardSpec(5, 10, 12, ImmutableList.of("dim"), new ObjectMapper()), + new BuildingHashBasedNumberedShardSpec(5, 12, ImmutableList.of("dim"), new ObjectMapper()).convert(10) + ); + } + + @Test + public void testCreateChunk() + { + Assert.assertEquals( + new NumberedPartitionChunk<>(5, 0, "test"), + new BuildingHashBasedNumberedShardSpec(5, 12, ImmutableList.of("dim"), new ObjectMapper()).createChunk("test") + ); + } + + @Test + public void testShardSpecLookup() + { + final List shardSpecs = ImmutableList.of( + new BuildingHashBasedNumberedShardSpec(0, 3, ImmutableList.of("dim"), new ObjectMapper()), + new BuildingHashBasedNumberedShardSpec(1, 3, ImmutableList.of("dim"), new ObjectMapper()), + new BuildingHashBasedNumberedShardSpec(2, 3, ImmutableList.of("dim"), new ObjectMapper()) + ); + final ShardSpecLookup lookup = shardSpecs.get(0).getLookup(shardSpecs); + final long currentTime = DateTimes.nowUtc().getMillis(); + Assert.assertEquals( + shardSpecs.get(1), + lookup.getShardSpec( + currentTime, + new MapBasedInputRow( + currentTime, + ImmutableList.of("dim"), ImmutableMap.of("dim", "1", "time", currentTime) + ) + ) + ); + Assert.assertEquals( + shardSpecs.get(2), + lookup.getShardSpec( + currentTime, + new MapBasedInputRow( + currentTime, + ImmutableList.of("dim"), ImmutableMap.of("dim", "2", "time", currentTime) + ) + ) + ); + Assert.assertEquals( + shardSpecs.get(0), + lookup.getShardSpec( + currentTime, + new MapBasedInputRow( + currentTime, + ImmutableList.of("dim"), ImmutableMap.of("dim", "3", "time", currentTime) + ) + ) + ); + } + + @Test + public void testSerde() throws JsonProcessingException + { + final ObjectMapper mapper = new ObjectMapper(); + mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false); + mapper.configure(MapperFeature.AUTO_DETECT_SETTERS, false); + mapper.registerSubtypes(new NamedType( + BuildingHashBasedNumberedShardSpec.class, + BuildingHashBasedNumberedShardSpec.TYPE + )); + mapper.setInjectableValues(new Std().addValue(ObjectMapper.class, mapper)); + final BuildingHashBasedNumberedShardSpec original = new BuildingHashBasedNumberedShardSpec( + 5, + 12, + ImmutableList.of("dim"), + new ObjectMapper() + ); + final String json = mapper.writeValueAsString(original); + final BuildingHashBasedNumberedShardSpec fromJson = (BuildingHashBasedNumberedShardSpec) mapper.readValue( + json, + ShardSpec.class + ); + Assert.assertEquals(original, fromJson); + } + + @Test + public void testEquals() + { + EqualsVerifier.forClass(BuildingHashBasedNumberedShardSpec.class) + .withIgnoredFields("jsonMapper") + .withPrefabValues(ObjectMapper.class, new ObjectMapper(), new ObjectMapper()) + .usingGetClass() + .verify(); + } +} diff --git a/core/src/test/java/org/apache/druid/timeline/partition/BuildingNumberedShardSpecTest.java b/core/src/test/java/org/apache/druid/timeline/partition/BuildingNumberedShardSpecTest.java index 21c5a03644e5..4cc8e67b7237 100644 --- a/core/src/test/java/org/apache/druid/timeline/partition/BuildingNumberedShardSpecTest.java +++ b/core/src/test/java/org/apache/druid/timeline/partition/BuildingNumberedShardSpecTest.java @@ -36,9 +36,9 @@ public class BuildingNumberedShardSpecTest { @Test - public void testToNumberedShardSpec() + public void testConvert() { - Assert.assertEquals(new NumberedShardSpec(5, 10), new BuildingNumberedShardSpec(5).toNumberedShardSpec(10)); + Assert.assertEquals(new NumberedShardSpec(5, 10), new BuildingNumberedShardSpec(5).convert(10)); } @Test diff --git a/extensions-contrib/materialized-view-maintenance/src/test/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisorTest.java b/extensions-contrib/materialized-view-maintenance/src/test/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisorTest.java index ec276795ddf7..a85a1fb752f3 100644 --- a/extensions-contrib/materialized-view-maintenance/src/test/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisorTest.java +++ b/extensions-contrib/materialized-view-maintenance/src/test/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisorTest.java @@ -140,7 +140,7 @@ public void testCheckSegments() throws IOException ImmutableMap.of(), ImmutableList.of("dim1", "dim2"), ImmutableList.of("m1"), - new HashBasedNumberedShardSpec(0, 1, null, null), + new HashBasedNumberedShardSpec(0, 1, 1, null, null), 9, 1024 ), @@ -151,7 +151,7 @@ public void testCheckSegments() throws IOException ImmutableMap.of(), ImmutableList.of("dim1", "dim2"), ImmutableList.of("m1"), - new HashBasedNumberedShardSpec(0, 1, null, null), + new HashBasedNumberedShardSpec(0, 1, 1, null, null), 9, 1024 ), @@ -162,7 +162,7 @@ public void testCheckSegments() throws IOException ImmutableMap.of(), ImmutableList.of("dim1", "dim2"), ImmutableList.of("m1"), - new HashBasedNumberedShardSpec(0, 1, null, null), + new HashBasedNumberedShardSpec(0, 1, 1, null, null), 9, 1024 ) @@ -175,7 +175,7 @@ public void testCheckSegments() throws IOException ImmutableMap.of(), ImmutableList.of("dim1", "dim2"), ImmutableList.of("m1"), - new HashBasedNumberedShardSpec(0, 1, null, null), + new HashBasedNumberedShardSpec(0, 1, 1, null, null), 9, 1024 ), @@ -186,7 +186,7 @@ public void testCheckSegments() throws IOException ImmutableMap.of(), ImmutableList.of("dim1", "dim2"), ImmutableList.of("m1"), - new HashBasedNumberedShardSpec(0, 1, null, null), + new HashBasedNumberedShardSpec(0, 1, 1, null, null), 9, 1024 ) @@ -209,7 +209,7 @@ public void testCheckSegments() throws IOException ImmutableMap.of(), ImmutableList.of("dim1", "dim2"), ImmutableList.of("m1"), - new HashBasedNumberedShardSpec(0, 1, null, null), + new HashBasedNumberedShardSpec(0, 1, 1, null, null), 9, 1024 ) @@ -225,7 +225,7 @@ public void testCheckSegments() throws IOException ImmutableMap.of(), ImmutableList.of("dim1", "dim2"), ImmutableList.of("m1"), - new HashBasedNumberedShardSpec(0, 1, null, null), + new HashBasedNumberedShardSpec(0, 1, 1, null, null), 9, 1024 ) @@ -246,7 +246,7 @@ public void testCheckSegmentsAndSubmitTasks() throws IOException ImmutableMap.of(), ImmutableList.of("dim1", "dim2"), ImmutableList.of("m1"), - new HashBasedNumberedShardSpec(0, 1, null, null), + new HashBasedNumberedShardSpec(0, 1, 1, null, null), 9, 1024 ) diff --git a/indexing-hadoop/src/main/java/org/apache/druid/indexer/DetermineHashedPartitionsJob.java b/indexing-hadoop/src/main/java/org/apache/druid/indexer/DetermineHashedPartitionsJob.java index 9f20f171fb7b..c7b0fd07597c 100644 --- a/indexing-hadoop/src/main/java/org/apache/druid/indexer/DetermineHashedPartitionsJob.java +++ b/indexing-hadoop/src/main/java/org/apache/druid/indexer/DetermineHashedPartitionsJob.java @@ -196,6 +196,7 @@ public boolean run() new HashBasedNumberedShardSpec( i, numberOfShards, + numberOfShards, null, HadoopDruidIndexerConfig.JSON_MAPPER ), diff --git a/indexing-hadoop/src/main/java/org/apache/druid/indexer/HadoopDruidDetermineConfigurationJob.java b/indexing-hadoop/src/main/java/org/apache/druid/indexer/HadoopDruidDetermineConfigurationJob.java index a25f274aae31..aed0901b4fa3 100644 --- a/indexing-hadoop/src/main/java/org/apache/druid/indexer/HadoopDruidDetermineConfigurationJob.java +++ b/indexing-hadoop/src/main/java/org/apache/druid/indexer/HadoopDruidDetermineConfigurationJob.java @@ -81,6 +81,7 @@ public boolean run() new HashBasedNumberedShardSpec( i, shardsPerInterval, + shardsPerInterval, config.getPartitionsSpec().getPartitionDimensions(), HadoopDruidIndexerConfig.JSON_MAPPER ), diff --git a/indexing-hadoop/src/test/java/org/apache/druid/indexer/BatchDeltaIngestionTest.java b/indexing-hadoop/src/test/java/org/apache/druid/indexer/BatchDeltaIngestionTest.java index 6ab20d210fd1..50629c7192db 100644 --- a/indexing-hadoop/src/test/java/org/apache/druid/indexer/BatchDeltaIngestionTest.java +++ b/indexing-hadoop/src/test/java/org/apache/druid/indexer/BatchDeltaIngestionTest.java @@ -489,7 +489,7 @@ private HadoopDruidIndexerConfig makeHadoopDruidIndexerConfig( INTERVAL_FULL.getStartMillis(), ImmutableList.of( new HadoopyShardSpec( - new HashBasedNumberedShardSpec(0, 1, null, HadoopDruidIndexerConfig.JSON_MAPPER), + new HashBasedNumberedShardSpec(0, 1, 1, null, HadoopDruidIndexerConfig.JSON_MAPPER), 0 ) ) diff --git a/indexing-hadoop/src/test/java/org/apache/druid/indexer/HadoopDruidIndexerConfigTest.java b/indexing-hadoop/src/test/java/org/apache/druid/indexer/HadoopDruidIndexerConfigTest.java index e3e357c3955f..ee3d3d69b739 100644 --- a/indexing-hadoop/src/test/java/org/apache/druid/indexer/HadoopDruidIndexerConfigTest.java +++ b/indexing-hadoop/src/test/java/org/apache/druid/indexer/HadoopDruidIndexerConfigTest.java @@ -63,7 +63,7 @@ public void testHashedBucketSelection() final int partitionCount = 10; for (int i = 0; i < partitionCount; i++) { shardSpecs.add(new HadoopyShardSpec( - new HashBasedNumberedShardSpec(i, partitionCount, null, new DefaultObjectMapper()), + new HashBasedNumberedShardSpec(i, partitionCount, partitionCount, null, new DefaultObjectMapper()), i )); } 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 a667c75dba28..a5964e33906e 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 @@ -562,6 +562,7 @@ private List constructShardSpecFromShardInfo(String partitionType, Ob specs.add(new HashBasedNumberedShardSpec( shardInfo[0], shardInfo[1], + shardInfo[1], null, HadoopDruidIndexerConfig.JSON_MAPPER )); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentMergeTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentMergeTask.java index cf5aaea2edc9..501b39709fa8 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentMergeTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentMergeTask.java @@ -29,7 +29,7 @@ import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.common.task.IndexTaskClientFactory; import org.apache.druid.indexing.common.task.TaskResource; -import org.apache.druid.timeline.partition.HashBasedNumberedShardSpec; +import org.apache.druid.timeline.partition.BuildingHashBasedNumberedShardSpec; import org.joda.time.Interval; import javax.annotation.Nullable; @@ -42,7 +42,7 @@ */ public class PartialHashSegmentMergeTask - extends PartialSegmentMergeTask + extends PartialSegmentMergeTask { public static final String TYPE = "partial_index_merge"; @@ -100,9 +100,9 @@ public String getType() } @Override - HashBasedNumberedShardSpec createShardSpec(TaskToolbox toolbox, Interval interval, int partitionId) + BuildingHashBasedNumberedShardSpec createShardSpec(TaskToolbox toolbox, Interval interval, int partitionId) { - return new HashBasedNumberedShardSpec( + return new BuildingHashBasedNumberedShardSpec( partitionId, Preconditions.checkNotNull(partitionsSpec.getNumShards(), "numShards"), partitionsSpec.getPartitionDimensions(), diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/partition/HashPartitionAnalysis.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/partition/HashPartitionAnalysis.java index a4b3a86a1b55..f1fb6a7808f1 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/partition/HashPartitionAnalysis.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/partition/HashPartitionAnalysis.java @@ -24,7 +24,7 @@ import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.java.util.common.IAE; import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; -import org.apache.druid.timeline.partition.HashBasedNumberedShardSpec; +import org.apache.druid.timeline.partition.BuildingHashBasedNumberedShardSpec; import org.joda.time.Interval; import java.util.Collections; @@ -106,7 +106,7 @@ public Map> convertToIntervalToSegmentIds interval, IntStream.range(0, numBuckets) .mapToObj(i -> { - final HashBasedNumberedShardSpec shardSpec = new HashBasedNumberedShardSpec( + final BuildingHashBasedNumberedShardSpec shardSpec = new BuildingHashBasedNumberedShardSpec( i, numBuckets, partitionsSpec.getPartitionDimensions(), 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 c5301bf66419..0ed6472fe30f 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 @@ -914,14 +914,14 @@ public void testWithPartialShardSpecAndOvershadowingSegments() throws IOExceptio .dataSource(DATA_SOURCE) .interval(Granularities.HOUR.bucket(PARTY_TIME)) .version(PARTY_TIME.toString()) - .shardSpec(new HashBasedNumberedShardSpec(0, 2, ImmutableList.of("dim1"), objectMapper)) + .shardSpec(new HashBasedNumberedShardSpec(0, 2, 2, ImmutableList.of("dim1"), objectMapper)) .size(0) .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)) + .shardSpec(new HashBasedNumberedShardSpec(1, 2, 2, ImmutableList.of("dim1"), objectMapper)) .size(0) .build() ) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ShardSpecsTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ShardSpecsTest.java index 64b0a0cad15f..30cfa0fa067d 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ShardSpecsTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ShardSpecsTest.java @@ -50,8 +50,8 @@ public ShardSpecsTest() @Test public void testShardSpecSelectionWithNullPartitionDimension() { - ShardSpec spec1 = new HashBasedNumberedShardSpec(0, 2, null, jsonMapper); - ShardSpec spec2 = new HashBasedNumberedShardSpec(1, 2, null, jsonMapper); + ShardSpec spec1 = new HashBasedNumberedShardSpec(0, 2, 2, null, jsonMapper); + ShardSpec spec2 = new HashBasedNumberedShardSpec(1, 2, 2, null, jsonMapper); Map> shardSpecMap = new HashMap<>(); shardSpecMap.put(Intervals.of("2014-01-01T00:00:00.000Z/2014-01-02T00:00:00.000Z"), Lists.newArrayList(spec1, spec2)); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractMultiPhaseParallelIndexingTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractMultiPhaseParallelIndexingTest.java index 7970e1dd9b7f..64949deefb34 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractMultiPhaseParallelIndexingTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractMultiPhaseParallelIndexingTest.java @@ -117,31 +117,23 @@ Set runTestTask( maxNumConcurrentSubTasks ); + return runTask(task, expectedTaskStatus); + } + + Set runTask(ParallelIndexSupervisorTask task, TaskState expectedTaskStatus) + { task.addToContext(Tasks.FORCE_TIME_CHUNK_LOCK_KEY, lockGranularity == LockGranularity.TIME_CHUNK); TaskStatus taskStatus = getIndexingServiceClient().runAndWait(task); Assert.assertEquals(expectedTaskStatus, taskStatus.getStatusCode()); return getIndexingServiceClient().getPublishedSegments(task); } - private ParallelIndexSupervisorTask newTask( - @Nullable TimestampSpec timestampSpec, - @Nullable DimensionsSpec dimensionsSpec, - @Nullable InputFormat inputFormat, - @Nullable ParseSpec parseSpec, - Interval interval, - File inputDir, - String filter, + ParallelIndexTuningConfig newTuningConfig( DimensionBasedPartitionsSpec partitionsSpec, int maxNumConcurrentSubTasks ) { - GranularitySpec granularitySpec = new UniformGranularitySpec( - Granularities.DAY, - Granularities.MINUTE, - interval == null ? null : Collections.singletonList(interval) - ); - - ParallelIndexTuningConfig tuningConfig = new ParallelIndexTuningConfig( + return new ParallelIndexTuningConfig( null, null, null, @@ -169,6 +161,27 @@ private ParallelIndexSupervisorTask newTask( null, null ); + } + + private ParallelIndexSupervisorTask newTask( + @Nullable TimestampSpec timestampSpec, + @Nullable DimensionsSpec dimensionsSpec, + @Nullable InputFormat inputFormat, + @Nullable ParseSpec parseSpec, + Interval interval, + File inputDir, + String filter, + DimensionBasedPartitionsSpec partitionsSpec, + int maxNumConcurrentSubTasks + ) + { + GranularitySpec granularitySpec = new UniformGranularitySpec( + Granularities.DAY, + Granularities.MINUTE, + interval == null ? null : Collections.singletonList(interval) + ); + + ParallelIndexTuningConfig tuningConfig = newTuningConfig(partitionsSpec, maxNumConcurrentSubTasks); final ParallelIndexIngestionSpec ingestionSpec; @@ -185,9 +198,7 @@ private ParallelIndexSupervisorTask newTask( "dataSource", timestampSpec, dimensionsSpec, - new AggregatorFactory[]{ - new LongSumAggregatorFactory("val", "val") - }, + new AggregatorFactory[]{new LongSumAggregatorFactory("val", "val")}, granularitySpec, null ), diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionAdjustingCorePartitionSizeTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionAdjustingCorePartitionSizeTest.java new file mode 100644 index 000000000000..04d82b073fd7 --- /dev/null +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionAdjustingCorePartitionSizeTest.java @@ -0,0 +1,205 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.common.task.batch.parallel; + +import com.google.common.collect.ImmutableList; +import org.apache.druid.data.input.InputFormat; +import org.apache.druid.data.input.impl.CsvInputFormat; +import org.apache.druid.data.input.impl.DimensionsSpec; +import org.apache.druid.data.input.impl.InlineInputSource; +import org.apache.druid.data.input.impl.TimestampSpec; +import org.apache.druid.indexer.TaskState; +import org.apache.druid.indexer.partitions.DimensionBasedPartitionsSpec; +import org.apache.druid.indexer.partitions.HashedPartitionsSpec; +import org.apache.druid.indexing.common.LockGranularity; +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.query.aggregation.AggregatorFactory; +import org.apache.druid.query.aggregation.LongSumAggregatorFactory; +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.timeline.DataSegment; +import org.apache.druid.timeline.partition.HashBasedNumberedShardSpec; +import org.joda.time.Interval; +import org.junit.Assert; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import java.io.ByteArrayOutputStream; +import java.io.DataOutputStream; +import java.io.IOException; +import java.util.Arrays; +import java.util.Collections; +import java.util.Set; + +@RunWith(Parameterized.class) +public class HashPartitionAdjustingCorePartitionSizeTest extends AbstractMultiPhaseParallelIndexingTest +{ + private static final TimestampSpec TIMESTAMP_SPEC = new TimestampSpec("ts", "auto", null); + private static final DimensionsSpec DIMENSIONS_SPEC = new DimensionsSpec( + DimensionsSpec.getDefaultSchemas(Arrays.asList("ts", "dim1", "dim2")) + ); + private static final AggregatorFactory[] METRICS = new AggregatorFactory[]{ + new LongSumAggregatorFactory("val", "val") + }; + private static final InputFormat INPUT_FORMAT = new CsvInputFormat( + Arrays.asList("ts", "dim1", "dim2", "val"), + null, + false, + false, + 0 + ); + private static final Interval INTERVAL_TO_INDEX = Intervals.of("2020-01-01/P1M"); + private static final GranularitySpec GRANULARITY_SPEC = new UniformGranularitySpec( + Granularities.DAY, + Granularities.NONE, + Collections.singletonList(INTERVAL_TO_INDEX) + ); + private static final int MAX_NUM_CONCURRENT_SUB_TASKS = 2; + + @Parameterized.Parameters(name = "{0}") + public static Iterable constructorFeeder() + { + return ImmutableList.of( + new Object[]{LockGranularity.TIME_CHUNK}, + new Object[]{LockGranularity.SEGMENT} + ); + } + + public HashPartitionAdjustingCorePartitionSizeTest(LockGranularity lockGranularity) + { + super(lockGranularity, true); + } + + @Test + public void testLessPartitionsThanBuckets() throws IOException + { + final String data; + try (final ByteArrayOutputStream baos = new ByteArrayOutputStream(1024); + final DataOutputStream out = new DataOutputStream(baos)) { + for (int i = 0; i < 3; i++) { + out.writeUTF(StringUtils.format("2020-01-01T00:00:00,%s,b1,%d\n", "a" + (i + 1), 10 * (i + 1))); + } + data = baos.toString(StringUtils.UTF8_STRING); + } + final ParallelIndexIOConfig ioConfig = new ParallelIndexIOConfig( + null, + new InlineInputSource(data), + INPUT_FORMAT, + false + ); + final DimensionBasedPartitionsSpec partitionsSpec = new HashedPartitionsSpec( + null, + 10, + ImmutableList.of("dim1") + ); + final ParallelIndexSupervisorTask task = new ParallelIndexSupervisorTask( + null, + null, + null, + new ParallelIndexIngestionSpec( + new DataSchema( + "testDatasource", + TIMESTAMP_SPEC, + DIMENSIONS_SPEC, + METRICS, + GRANULARITY_SPEC, + null + ), + ioConfig, + newTuningConfig(partitionsSpec, MAX_NUM_CONCURRENT_SUB_TASKS) + ), + null, + null, + null, + null, + null, + null + ); + final Set segments = runTask(task, TaskState.SUCCESS); + Assert.assertEquals(3, segments.size()); + segments.forEach(segment -> { + Assert.assertSame(HashBasedNumberedShardSpec.class, segment.getShardSpec().getClass()); + final HashBasedNumberedShardSpec shardSpec = (HashBasedNumberedShardSpec) segment.getShardSpec(); + Assert.assertEquals(3, shardSpec.getPartitions()); + Assert.assertEquals(10, shardSpec.getNumBuckets()); + Assert.assertEquals(ImmutableList.of("dim1"), shardSpec.getPartitionDimensions()); + }); + } + + @Test + public void testEqualNumberOfPartitionsToBuckets() throws IOException + { + final String data; + try (final ByteArrayOutputStream baos = new ByteArrayOutputStream(1024); + final DataOutputStream out = new DataOutputStream(baos)) { + for (int i = 0; i < 10; i++) { + out.writeUTF(StringUtils.format("2020-01-01T00:00:00,%s,b1,%d\n", "aa" + (i + 10), 10 * (i + 1))); + } + data = baos.toString(StringUtils.UTF8_STRING); + } + final ParallelIndexIOConfig ioConfig = new ParallelIndexIOConfig( + null, + new InlineInputSource(data), + INPUT_FORMAT, + false + ); + final DimensionBasedPartitionsSpec partitionsSpec = new HashedPartitionsSpec( + null, + 5, + ImmutableList.of("dim1") + ); + final ParallelIndexSupervisorTask task = new ParallelIndexSupervisorTask( + null, + null, + null, + new ParallelIndexIngestionSpec( + new DataSchema( + "testDatasource", + TIMESTAMP_SPEC, + DIMENSIONS_SPEC, + METRICS, + GRANULARITY_SPEC, + null + ), + ioConfig, + newTuningConfig(partitionsSpec, MAX_NUM_CONCURRENT_SUB_TASKS) + ), + null, + null, + null, + null, + null, + null + ); + final Set segments = runTask(task, TaskState.SUCCESS); + Assert.assertEquals(5, segments.size()); + segments.forEach(segment -> { + Assert.assertSame(HashBasedNumberedShardSpec.class, segment.getShardSpec().getClass()); + final HashBasedNumberedShardSpec shardSpec = (HashBasedNumberedShardSpec) segment.getShardSpec(); + Assert.assertEquals(5, shardSpec.getPartitions()); + Assert.assertEquals(5, shardSpec.getNumBuckets()); + Assert.assertEquals(ImmutableList.of("dim1"), shardSpec.getPartitionDimensions()); + }); + } +} diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexTestingFactory.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexTestingFactory.java index 08dd92c5f67a..e64235e48869 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexTestingFactory.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexTestingFactory.java @@ -103,6 +103,7 @@ public > File fetchSegmentFile( static final HashBasedNumberedShardSpec HASH_BASED_NUMBERED_SHARD_SPEC = new HashBasedNumberedShardSpec( PARTITION_ID, PARTITION_ID + 1, + PARTITION_ID + 1, Collections.singletonList("dim"), ParallelIndexTestingFactory.NESTED_OBJECT_MAPPER ); diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/SegmentPublisherHelper.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/SegmentPublisherHelper.java index 76093beb8c8a..8c311e8f7e9f 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/SegmentPublisherHelper.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/SegmentPublisherHelper.java @@ -21,7 +21,7 @@ import org.apache.druid.java.util.common.ISE; import org.apache.druid.timeline.DataSegment; -import org.apache.druid.timeline.partition.BuildingNumberedShardSpec; +import org.apache.druid.timeline.partition.BuildingShardSpec; import org.apache.druid.timeline.partition.OverwriteShardSpec; import org.apache.druid.timeline.partition.ShardSpec; import org.joda.time.Interval; @@ -42,7 +42,7 @@ public final class SegmentPublisherHelper * This method fills missing information in the shard spec if necessary when publishing segments. * * - When time chunk lock is used, the non-appending task should set the proper size of the core partitions for - * dynamically-partitioned segments. See {@link #annotateNumberedShardSpecFn}. + * dynamically-partitioned segments. See {@link #annotateCorePartitionSetSizeFn}. * - When segment lock is used, the overwriting task should set the proper size of the atomic update group. * See {@link #annotateAtomicUpdateGroupFn}. */ @@ -70,8 +70,8 @@ static Set annotateShardSpec(Set segments) final Function annotateFn; if (firstShardSpec instanceof OverwriteShardSpec) { annotateFn = annotateAtomicUpdateGroupFn(segmentsPerInterval.size()); - } else if (firstShardSpec instanceof BuildingNumberedShardSpec) { - annotateFn = annotateNumberedShardSpecFn(segmentsPerInterval.size()); + } else if (firstShardSpec instanceof BuildingShardSpec) { + annotateFn = annotateCorePartitionSetSizeFn(segmentsPerInterval.size()); } else { annotateFn = null; } @@ -93,11 +93,11 @@ private static Function annotateAtomicUpdateGroupFn(in }; } - private static Function annotateNumberedShardSpecFn(int corePartitionSetSize) + private static Function annotateCorePartitionSetSizeFn(int corePartitionSetSize) { return segment -> { - final BuildingNumberedShardSpec shardSpec = (BuildingNumberedShardSpec) segment.getShardSpec(); - return segment.withShardSpec(shardSpec.toNumberedShardSpec(corePartitionSetSize)); + final BuildingShardSpec shardSpec = (BuildingShardSpec) segment.getShardSpec(); + return segment.withShardSpec(shardSpec.convert(corePartitionSetSize)); }; } 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 bfd72b4b8851..99f8fa4f4ff8 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 @@ -24,6 +24,7 @@ import com.google.common.collect.ImmutableSet; import com.google.common.collect.Lists; import com.google.common.collect.Ordering; +import nl.jqno.equalsverifier.EqualsVerifier; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.server.ServerTestHelper; import org.apache.druid.timeline.Overshadowable; @@ -45,6 +46,12 @@ public class NumberedShardSpecTest { + @Test + public void testEquals() + { + EqualsVerifier.forClass(NumberedShardSpec.class).usingGetClass().verify(); + } + @Test public void testSerdeRoundTrip() throws Exception { 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 9ac4d27823d8..50cf0f2a2fbd 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 @@ -19,10 +19,12 @@ package org.apache.druid.timeline.partition; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Function; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; +import nl.jqno.equalsverifier.EqualsVerifier; import org.apache.druid.data.input.InputRow; import org.apache.druid.data.input.MapBasedInputRow; import org.apache.druid.data.input.Row; @@ -41,6 +43,16 @@ public class HashBasedNumberedShardSpecTest { + @Test + public void testEquals() + { + EqualsVerifier.forClass(HashBasedNumberedShardSpec.class) + .withIgnoredFields("jsonMapper") + .withPrefabValues(ObjectMapper.class, new ObjectMapper(), new ObjectMapper()) + .usingGetClass() + .verify(); + } + @Test public void testSerdeRoundTrip() throws Exception { @@ -50,6 +62,7 @@ public void testSerdeRoundTrip() throws Exception new HashBasedNumberedShardSpec( 1, 2, + 3, ImmutableList.of("visitor_id"), ServerTestHelper.MAPPER ) @@ -58,6 +71,7 @@ public void testSerdeRoundTrip() throws Exception ); Assert.assertEquals(1, spec.getPartitionNum()); Assert.assertEquals(2, ((HashBasedNumberedShardSpec) spec).getPartitions()); + Assert.assertEquals(3, ((HashBasedNumberedShardSpec) spec).getNumBuckets()); Assert.assertEquals(ImmutableList.of("visitor_id"), ((HashBasedNumberedShardSpec) spec).getPartitionDimensions()); } @@ -77,16 +91,20 @@ public void testSerdeBackwardsCompat() throws Exception ); Assert.assertEquals(1, specWithPartitionDimensions.getPartitionNum()); Assert.assertEquals(2, ((HashBasedNumberedShardSpec) specWithPartitionDimensions).getPartitions()); - Assert.assertEquals(ImmutableList.of("visitor_id"), ((HashBasedNumberedShardSpec) specWithPartitionDimensions).getPartitionDimensions()); + Assert.assertEquals(2, ((HashBasedNumberedShardSpec) specWithPartitionDimensions).getNumBuckets()); + Assert.assertEquals( + ImmutableList.of("visitor_id"), + ((HashBasedNumberedShardSpec) specWithPartitionDimensions).getPartitionDimensions() + ); } @Test public void testPartitionChunks() { final List specs = ImmutableList.of( - new HashBasedNumberedShardSpec(0, 3, null, ServerTestHelper.MAPPER), - new HashBasedNumberedShardSpec(1, 3, null, ServerTestHelper.MAPPER), - new HashBasedNumberedShardSpec(2, 3, null, ServerTestHelper.MAPPER) + new HashBasedNumberedShardSpec(0, 3, 3, null, ServerTestHelper.MAPPER), + new HashBasedNumberedShardSpec(1, 3, 3, null, ServerTestHelper.MAPPER), + new HashBasedNumberedShardSpec(2, 3, 3, null, ServerTestHelper.MAPPER) ); final List> chunks = Lists.transform( @@ -157,35 +175,26 @@ public void testIsInChunkWithMorePartitionsBeyondNumBucketsReturningTrue() @Test public void testGetGroupKey() { - final HashBasedNumberedShardSpec shardSpec1 = new HashBasedNumberedShardSpec( - 1, - 2, - ImmutableList.of("visitor_id"), - ServerTestHelper.MAPPER - ); + final List partitionDimensions1 = ImmutableList.of("visitor_id"); final DateTime time = DateTimes.nowUtc(); final InputRow inputRow = new MapBasedInputRow( time, ImmutableList.of("visitor_id", "cnt"), ImmutableMap.of("visitor_id", "v1", "cnt", 10) ); - Assert.assertEquals(ImmutableList.of(Collections.singletonList("v1")), shardSpec1.getGroupKey(time.getMillis(), inputRow)); - - final HashBasedNumberedShardSpec shardSpec2 = new HashBasedNumberedShardSpec( - 1, - 2, - null, - ServerTestHelper.MAPPER + Assert.assertEquals( + ImmutableList.of(Collections.singletonList("v1")), + HashBasedNumberedShardSpec.getGroupKey(partitionDimensions1, time.getMillis(), inputRow) ); - Assert.assertEquals(ImmutableList.of( + + Assert.assertEquals( + ImmutableList.of( time.getMillis(), - ImmutableMap.of( - "cnt", - Collections.singletonList(10), - "visitor_id", - Collections.singletonList("v1") - ) - ).toString(), shardSpec2.getGroupKey(time.getMillis(), inputRow).toString()); + ImmutableMap.of("cnt", Collections.singletonList(10), "visitor_id", Collections.singletonList("v1"))) + .toString(), + // empty list when partitionDimensions is null + HashBasedNumberedShardSpec.getGroupKey(ImmutableList.of(), time.getMillis(), inputRow).toString() + ); } public boolean assertExistsInOneSpec(List specs, InputRow row) @@ -202,7 +211,7 @@ public static class HashOverridenShardSpec extends HashBasedNumberedShardSpec { public HashOverridenShardSpec(int partitionNum, int partitions) { - super(partitionNum, partitions, null, ServerTestHelper.MAPPER); + super(partitionNum, partitions, partitions, null, ServerTestHelper.MAPPER); } @Override From 4600bd689c3ab3e1580d7e531ff3751cf13b855a Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Thu, 11 Jun 2020 15:23:52 -0700 Subject: [PATCH 08/24] test for both parallel and single-threaded task --- ...rtitionAdjustingCorePartitionSizeTest.java | 116 +++++++----------- 1 file changed, 43 insertions(+), 73 deletions(-) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionAdjustingCorePartitionSizeTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionAdjustingCorePartitionSizeTest.java index 04d82b073fd7..5dec9a87ded8 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionAdjustingCorePartitionSizeTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionAdjustingCorePartitionSizeTest.java @@ -24,6 +24,7 @@ import org.apache.druid.data.input.impl.CsvInputFormat; import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.InlineInputSource; +import org.apache.druid.data.input.impl.LocalInputSource; import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.indexer.TaskState; import org.apache.druid.indexer.partitions.DimensionBasedPartitionsSpec; @@ -47,7 +48,11 @@ import java.io.ByteArrayOutputStream; import java.io.DataOutputStream; +import java.io.File; import java.io.IOException; +import java.io.Writer; +import java.nio.charset.StandardCharsets; +import java.nio.file.Files; import java.util.Arrays; import java.util.Collections; import java.util.Set; @@ -75,68 +80,52 @@ public class HashPartitionAdjustingCorePartitionSizeTest extends AbstractMultiPh Granularities.NONE, Collections.singletonList(INTERVAL_TO_INDEX) ); - private static final int MAX_NUM_CONCURRENT_SUB_TASKS = 2; - @Parameterized.Parameters(name = "{0}") + @Parameterized.Parameters(name = "{0}, maxNumConcurrentSubTasks={1}") public static Iterable constructorFeeder() { return ImmutableList.of( - new Object[]{LockGranularity.TIME_CHUNK}, - new Object[]{LockGranularity.SEGMENT} + new Object[]{LockGranularity.TIME_CHUNK, 2}, + new Object[]{LockGranularity.TIME_CHUNK, 1}, + new Object[]{LockGranularity.SEGMENT, 2} ); } - public HashPartitionAdjustingCorePartitionSizeTest(LockGranularity lockGranularity) + private final int maxNumConcurrentSubTasks; + + public HashPartitionAdjustingCorePartitionSizeTest(LockGranularity lockGranularity, int maxNumConcurrentSubTasks) { super(lockGranularity, true); + this.maxNumConcurrentSubTasks = maxNumConcurrentSubTasks; } @Test public void testLessPartitionsThanBuckets() throws IOException { - final String data; - try (final ByteArrayOutputStream baos = new ByteArrayOutputStream(1024); - final DataOutputStream out = new DataOutputStream(baos)) { - for (int i = 0; i < 3; i++) { - out.writeUTF(StringUtils.format("2020-01-01T00:00:00,%s,b1,%d\n", "a" + (i + 1), 10 * (i + 1))); + final File inputDir = temporaryFolder.newFolder(); + for (int i = 0; i < 3; i++) { + try (final Writer writer = + Files.newBufferedWriter(new File(inputDir, "test_" + i).toPath(), StandardCharsets.UTF_8)) { + writer.write(StringUtils.format("2020-01-01T00:00:00,%s,b1,%d\n", "a" + (i + 1), 10 * (i + 1))); } - data = baos.toString(StringUtils.UTF8_STRING); } - final ParallelIndexIOConfig ioConfig = new ParallelIndexIOConfig( - null, - new InlineInputSource(data), - INPUT_FORMAT, - false - ); final DimensionBasedPartitionsSpec partitionsSpec = new HashedPartitionsSpec( null, 10, ImmutableList.of("dim1") ); - final ParallelIndexSupervisorTask task = new ParallelIndexSupervisorTask( - null, - null, - null, - new ParallelIndexIngestionSpec( - new DataSchema( - "testDatasource", - TIMESTAMP_SPEC, - DIMENSIONS_SPEC, - METRICS, - GRANULARITY_SPEC, - null - ), - ioConfig, - newTuningConfig(partitionsSpec, MAX_NUM_CONCURRENT_SUB_TASKS) - ), - null, - null, - null, - null, + final Set segments = runTestTask( + TIMESTAMP_SPEC, + DIMENSIONS_SPEC, + INPUT_FORMAT, null, - null + INTERVAL_TO_INDEX, + inputDir, + "test_*", + partitionsSpec, + maxNumConcurrentSubTasks, + TaskState.SUCCESS ); - final Set segments = runTask(task, TaskState.SUCCESS); Assert.assertEquals(3, segments.size()); segments.forEach(segment -> { Assert.assertSame(HashBasedNumberedShardSpec.class, segment.getShardSpec().getClass()); @@ -150,49 +139,30 @@ public void testLessPartitionsThanBuckets() throws IOException @Test public void testEqualNumberOfPartitionsToBuckets() throws IOException { - final String data; - try (final ByteArrayOutputStream baos = new ByteArrayOutputStream(1024); - final DataOutputStream out = new DataOutputStream(baos)) { - for (int i = 0; i < 10; i++) { - out.writeUTF(StringUtils.format("2020-01-01T00:00:00,%s,b1,%d\n", "aa" + (i + 10), 10 * (i + 1))); + final File inputDir = temporaryFolder.newFolder(); + for (int i = 0; i < 10; i++) { + try (final Writer writer = + Files.newBufferedWriter(new File(inputDir, "test_" + i).toPath(), StandardCharsets.UTF_8)) { + writer.write(StringUtils.format("2020-01-01T00:00:00,%s,b1,%d\n", "aa" + (i + 10), 10 * (i + 1))); } - data = baos.toString(StringUtils.UTF8_STRING); } - final ParallelIndexIOConfig ioConfig = new ParallelIndexIOConfig( - null, - new InlineInputSource(data), - INPUT_FORMAT, - false - ); final DimensionBasedPartitionsSpec partitionsSpec = new HashedPartitionsSpec( null, 5, ImmutableList.of("dim1") ); - final ParallelIndexSupervisorTask task = new ParallelIndexSupervisorTask( - null, - null, - null, - new ParallelIndexIngestionSpec( - new DataSchema( - "testDatasource", - TIMESTAMP_SPEC, - DIMENSIONS_SPEC, - METRICS, - GRANULARITY_SPEC, - null - ), - ioConfig, - newTuningConfig(partitionsSpec, MAX_NUM_CONCURRENT_SUB_TASKS) - ), - null, - null, - null, - null, + final Set segments = runTestTask( + TIMESTAMP_SPEC, + DIMENSIONS_SPEC, + INPUT_FORMAT, null, - null + INTERVAL_TO_INDEX, + inputDir, + "test_*", + partitionsSpec, + maxNumConcurrentSubTasks, + TaskState.SUCCESS ); - final Set segments = runTask(task, TaskState.SUCCESS); Assert.assertEquals(5, segments.size()); segments.forEach(segment -> { Assert.assertSame(HashBasedNumberedShardSpec.class, segment.getShardSpec().getClass()); From 7448773ca598453cc27308da38eca0929a2de337 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Thu, 11 Jun 2020 15:56:03 -0700 Subject: [PATCH 09/24] unused variables --- ...rtitionAdjustingCorePartitionSizeTest.java | 19 ------------------- 1 file changed, 19 deletions(-) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionAdjustingCorePartitionSizeTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionAdjustingCorePartitionSizeTest.java index 5dec9a87ded8..8f1fb67449ea 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionAdjustingCorePartitionSizeTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionAdjustingCorePartitionSizeTest.java @@ -23,8 +23,6 @@ import org.apache.druid.data.input.InputFormat; import org.apache.druid.data.input.impl.CsvInputFormat; import org.apache.druid.data.input.impl.DimensionsSpec; -import org.apache.druid.data.input.impl.InlineInputSource; -import org.apache.druid.data.input.impl.LocalInputSource; import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.indexer.TaskState; import org.apache.druid.indexer.partitions.DimensionBasedPartitionsSpec; @@ -32,12 +30,6 @@ import org.apache.druid.indexing.common.LockGranularity; 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.query.aggregation.AggregatorFactory; -import org.apache.druid.query.aggregation.LongSumAggregatorFactory; -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.timeline.DataSegment; import org.apache.druid.timeline.partition.HashBasedNumberedShardSpec; import org.joda.time.Interval; @@ -46,15 +38,12 @@ import org.junit.runner.RunWith; import org.junit.runners.Parameterized; -import java.io.ByteArrayOutputStream; -import java.io.DataOutputStream; import java.io.File; import java.io.IOException; import java.io.Writer; import java.nio.charset.StandardCharsets; import java.nio.file.Files; import java.util.Arrays; -import java.util.Collections; import java.util.Set; @RunWith(Parameterized.class) @@ -64,9 +53,6 @@ public class HashPartitionAdjustingCorePartitionSizeTest extends AbstractMultiPh private static final DimensionsSpec DIMENSIONS_SPEC = new DimensionsSpec( DimensionsSpec.getDefaultSchemas(Arrays.asList("ts", "dim1", "dim2")) ); - private static final AggregatorFactory[] METRICS = new AggregatorFactory[]{ - new LongSumAggregatorFactory("val", "val") - }; private static final InputFormat INPUT_FORMAT = new CsvInputFormat( Arrays.asList("ts", "dim1", "dim2", "val"), null, @@ -75,11 +61,6 @@ public class HashPartitionAdjustingCorePartitionSizeTest extends AbstractMultiPh 0 ); private static final Interval INTERVAL_TO_INDEX = Intervals.of("2020-01-01/P1M"); - private static final GranularitySpec GRANULARITY_SPEC = new UniformGranularitySpec( - Granularities.DAY, - Granularities.NONE, - Collections.singletonList(INTERVAL_TO_INDEX) - ); @Parameterized.Parameters(name = "{0}, maxNumConcurrentSubTasks={1}") public static Iterable constructorFeeder() From 08e2635fe76a93377e219cec5963338ec748a0d5 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Thu, 11 Jun 2020 17:39:13 -0700 Subject: [PATCH 10/24] fix test --- .../HashPartitionCachingLocalSegmentAllocatorTest.java | 5 +++-- .../IndexerSQLMetadataStorageCoordinatorTest.java | 9 ++++++--- 2 files changed, 9 insertions(+), 5 deletions(-) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionCachingLocalSegmentAllocatorTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionCachingLocalSegmentAllocatorTest.java index 16c20b342acc..cd34565a4997 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionCachingLocalSegmentAllocatorTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionCachingLocalSegmentAllocatorTest.java @@ -40,6 +40,7 @@ import org.apache.druid.java.util.common.granularity.NoneGranularity; import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; import org.apache.druid.timeline.SegmentId; +import org.apache.druid.timeline.partition.BuildingHashBasedNumberedShardSpec; import org.apache.druid.timeline.partition.HashBasedNumberedShardSpec; import org.easymock.EasyMock; import org.joda.time.Interval; @@ -102,9 +103,9 @@ public void allocatesCorrectShardSpec() throws IOException SegmentId.of(DATASOURCE, INTERVAL, VERSION, PARTITION_NUM), segmentIdWithShardSpec.asSegmentId() ); - HashBasedNumberedShardSpec shardSpec = (HashBasedNumberedShardSpec) segmentIdWithShardSpec.getShardSpec(); + BuildingHashBasedNumberedShardSpec shardSpec = (BuildingHashBasedNumberedShardSpec) segmentIdWithShardSpec.getShardSpec(); Assert.assertEquals(PARTITION_DIMENSIONS, shardSpec.getPartitionDimensions()); - Assert.assertEquals(NUM_PARTITONS, shardSpec.getPartitions()); + Assert.assertEquals(NUM_PARTITONS, shardSpec.getNumBuckets()); Assert.assertEquals(PARTITION_NUM, shardSpec.getPartitionNum()); } 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 5eb34ffbe28d..af6eae551982 100644 --- a/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java +++ b/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java @@ -1048,7 +1048,8 @@ public void testAllocatePendingSegmentsForHashBasedNumberedShardSpec() throws IO HashBasedNumberedShardSpec shardSpec = (HashBasedNumberedShardSpec) id.getShardSpec(); Assert.assertEquals(0, shardSpec.getPartitionNum()); - Assert.assertEquals(5, shardSpec.getPartitions()); + Assert.assertEquals(0, shardSpec.getPartitions()); + Assert.assertEquals(5, shardSpec.getNumBuckets()); coordinator.announceHistoricalSegments( Collections.singleton( @@ -1078,7 +1079,8 @@ public void testAllocatePendingSegmentsForHashBasedNumberedShardSpec() throws IO shardSpec = (HashBasedNumberedShardSpec) id.getShardSpec(); Assert.assertEquals(1, shardSpec.getPartitionNum()); - Assert.assertEquals(5, shardSpec.getPartitions()); + Assert.assertEquals(0, shardSpec.getPartitions()); + Assert.assertEquals(5, shardSpec.getNumBuckets()); coordinator.announceHistoricalSegments( Collections.singleton( @@ -1108,6 +1110,7 @@ public void testAllocatePendingSegmentsForHashBasedNumberedShardSpec() throws IO shardSpec = (HashBasedNumberedShardSpec) id.getShardSpec(); Assert.assertEquals(2, shardSpec.getPartitionNum()); - Assert.assertEquals(3, shardSpec.getPartitions()); + Assert.assertEquals(0, shardSpec.getPartitions()); + Assert.assertEquals(3, shardSpec.getNumBuckets()); } } From 30cae0e323316f123c96b3db7023c0d68c0df4f6 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Thu, 11 Jun 2020 19:34:51 -0700 Subject: [PATCH 11/24] unused imports --- .../parallel/HashPartitionCachingLocalSegmentAllocatorTest.java | 1 - 1 file changed, 1 deletion(-) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionCachingLocalSegmentAllocatorTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionCachingLocalSegmentAllocatorTest.java index cd34565a4997..cea8589c51d6 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionCachingLocalSegmentAllocatorTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionCachingLocalSegmentAllocatorTest.java @@ -41,7 +41,6 @@ import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; import org.apache.druid.timeline.SegmentId; import org.apache.druid.timeline.partition.BuildingHashBasedNumberedShardSpec; -import org.apache.druid.timeline.partition.HashBasedNumberedShardSpec; import org.easymock.EasyMock; import org.joda.time.Interval; import org.junit.Assert; From 5664a6c4d3324f343971f9f8f655cabe46cbfd1e Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Sat, 13 Jun 2020 00:48:09 -0700 Subject: [PATCH 12/24] add hash/range buckets --- .../BuildingHashBasedNumberedShardSpec.java | 28 ++- .../timeline/partition/BuildingShardSpec.java | 12 ++ .../BuildingSingleDimensionShardSpec.java | 88 +++++++++ .../HashBasedNumberedPartialShardSpec.java | 54 +++--- .../partition/HashBasedNumberedShardSpec.java | 22 ++- .../timeline/partition/LinearShardSpec.java | 6 + .../timeline/partition/NoneShardSpec.java | 6 + .../partition/NumberedPartialShardSpec.java | 2 +- .../timeline/partition/NumberedShardSpec.java | 3 +- .../partition/OverwriteShardSpec.java | 9 + .../druid/timeline/partition/ShardSpec.java | 2 + .../SingleDimensionPartialShardSpec.java | 25 ++- .../partition/SingleDimensionShardSpec.java | 51 +++-- .../druid/timeline/DataSegmentTest.java | 6 + ...uildingHashBasedNumberedShardSpecTest.java | 14 +- ...HashBasedNumberedPartialShardSpecTest.java | 5 +- .../MaterializedViewSupervisorTest.java | 16 +- .../indexer/DetermineHashedPartitionsJob.java | 1 + .../druid/indexer/DeterminePartitionsJob.java | 9 +- .../HadoopDruidDetermineConfigurationJob.java | 1 + .../indexer/BatchDeltaIngestionTest.java | 2 +- .../indexer/HadoopDruidIndexerConfigTest.java | 2 +- .../druid/indexer/IndexGeneratorJobTest.java | 8 +- .../task/CachingLocalSegmentAllocator.java | 115 ++++++------ .../common/task/CachingSegmentAllocator.java | 38 ---- .../druid/indexing/common/task/IndexTask.java | 10 +- ...nearlyPartitionedSequenceNameFunction.java | 16 +- .../common/task/SegmentAllocators.java | 10 +- .../indexing/common/task/ShardSpecs.java | 124 ++++++------- .../PartialHashSegmentGenerateTask.java | 6 +- .../parallel/PartialHashSegmentMergeTask.java | 1 + .../PartialRangeSegmentGenerateTask.java | 6 +- .../parallel/PartialSegmentGenerateTask.java | 12 +- .../partition/CompletePartitionAnalysis.java | 13 +- .../task/batch/partition/HashBucket.java | 80 ++++++++ .../batch/partition/HashBucketLookup.java | 77 ++++++++ .../partition/HashPartitionAnalysis.java | 89 ++++++--- .../task/batch/partition/PartitionBucket.java | 34 ++++ .../partition/PartitionBucketLookup.java | 37 ++++ .../batch/partition/PartitionBuckets.java | 58 ++++++ .../task/batch/partition/RangeBucket.java | 109 +++++++++++ .../batch/partition/RangeBucketLookup.java | 53 ++++++ .../partition/RangePartitionAnalysis.java | 138 ++++++++------ .../actions/SegmentAllocateActionTest.java | 17 +- .../indexing/common/task/IndexTaskTest.java | 4 +- ...itionCachingLocalSegmentAllocatorTest.java | 11 +- .../indexing/common/task/ShardSpecsTest.java | 174 +++++++++--------- ...rtitionAdjustingCorePartitionSizeTest.java | 4 +- ...itionCachingLocalSegmentAllocatorTest.java | 13 +- .../parallel/ParallelIndexTestingFactory.java | 1 + .../SinglePhaseParallelIndexingTest.java | 6 +- .../indexing/overlord/TaskLockboxTest.java | 4 +- .../client/CachingClusteredClientTest.java | 6 +- ...exerSQLMetadataStorageCoordinatorTest.java | 10 +- .../SegmentIdWithShardSpecTest.java | 2 +- .../server/shard/NumberedShardSpecTest.java | 4 +- .../shard/SingleDimensionShardSpecTest.java | 2 +- .../HashBasedNumberedShardSpecTest.java | 16 +- 58 files changed, 1157 insertions(+), 515 deletions(-) create mode 100644 core/src/main/java/org/apache/druid/timeline/partition/BuildingSingleDimensionShardSpec.java 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/batch/partition/HashBucket.java create mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/partition/HashBucketLookup.java create mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/partition/PartitionBucket.java create mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/partition/PartitionBucketLookup.java create mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/partition/PartitionBuckets.java create mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/partition/RangeBucket.java create mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/partition/RangeBucketLookup.java diff --git a/core/src/main/java/org/apache/druid/timeline/partition/BuildingHashBasedNumberedShardSpec.java b/core/src/main/java/org/apache/druid/timeline/partition/BuildingHashBasedNumberedShardSpec.java index c9fee408a2f7..f6a2f9e9f3bd 100644 --- a/core/src/main/java/org/apache/druid/timeline/partition/BuildingHashBasedNumberedShardSpec.java +++ b/core/src/main/java/org/apache/druid/timeline/partition/BuildingHashBasedNumberedShardSpec.java @@ -38,6 +38,7 @@ public class BuildingHashBasedNumberedShardSpec implements BuildingShardSpec partitionDimensions; private final ObjectMapper jsonMapper; @@ -45,12 +46,14 @@ public class BuildingHashBasedNumberedShardSpec implements BuildingShardSpec partitionDimensions, @JacksonInject ObjectMapper jsonMapper ) { this.partitionId = partitionId; + this.bucketId = bucketId; this.numBuckets = numBuckets; this.partitionDimensions = partitionDimensions == null ? HashBasedNumberedShardSpec.DEFAULT_PARTITION_DIMENSIONS @@ -65,6 +68,12 @@ public int getPartitionNum() return partitionId; } + @JsonProperty + public int getBucketId() + { + return bucketId; + } + @JsonProperty public int getNumBuckets() { @@ -86,16 +95,17 @@ public PartitionChunk createChunk(T obj) return new NumberedPartitionChunk<>(partitionId, 0, obj); } - @Override - public ShardSpecLookup getLookup(List shardSpecs) - { - return HashBasedNumberedShardSpec.createHashLookup(jsonMapper, partitionDimensions, shardSpecs, numBuckets); - } - @Override public HashBasedNumberedShardSpec convert(int numCorePartitions) { - return new HashBasedNumberedShardSpec(partitionId, numCorePartitions, numBuckets, partitionDimensions, jsonMapper); + return new HashBasedNumberedShardSpec( + partitionId, + numCorePartitions, + bucketId, + numBuckets, + partitionDimensions, + jsonMapper + ); } @Override @@ -109,6 +119,7 @@ public boolean equals(Object o) } BuildingHashBasedNumberedShardSpec that = (BuildingHashBasedNumberedShardSpec) o; return partitionId == that.partitionId && + bucketId == that.bucketId && numBuckets == that.numBuckets && Objects.equals(partitionDimensions, that.partitionDimensions); } @@ -116,7 +127,7 @@ public boolean equals(Object o) @Override public int hashCode() { - return Objects.hash(partitionId, numBuckets, partitionDimensions); + return Objects.hash(partitionId, bucketId, numBuckets, partitionDimensions); } @Override @@ -124,6 +135,7 @@ public String toString() { return "BuildingHashBasedNumberedShardSpec{" + "partitionId=" + partitionId + + ", bucketId=" + bucketId + ", numBuckets=" + numBuckets + ", partitionDimensions=" + partitionDimensions + '}'; diff --git a/core/src/main/java/org/apache/druid/timeline/partition/BuildingShardSpec.java b/core/src/main/java/org/apache/druid/timeline/partition/BuildingShardSpec.java index ed1136e80a29..b6e74dd5a0a2 100644 --- a/core/src/main/java/org/apache/druid/timeline/partition/BuildingShardSpec.java +++ b/core/src/main/java/org/apache/druid/timeline/partition/BuildingShardSpec.java @@ -52,6 +52,18 @@ public interface BuildingShardSpec extends ShardSpec { T convert(int numCorePartitions); + @Override + default int getNumCorePartitions() + { + throw new UnsupportedOperationException(); + } + + @Override + default ShardSpecLookup getLookup(List shardSpecs) + { + throw new UnsupportedOperationException(); + } + // The below methods are used on the query side, and so must not be called for this shardSpec. @Override diff --git a/core/src/main/java/org/apache/druid/timeline/partition/BuildingSingleDimensionShardSpec.java b/core/src/main/java/org/apache/druid/timeline/partition/BuildingSingleDimensionShardSpec.java new file mode 100644 index 000000000000..6f9560a4a1d1 --- /dev/null +++ b/core/src/main/java/org/apache/druid/timeline/partition/BuildingSingleDimensionShardSpec.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 javax.annotation.Nullable; + +public class BuildingSingleDimensionShardSpec implements BuildingShardSpec +{ + private final String dimension; + @Nullable + private final String start; + @Nullable + private final String end; + private final int partitionNum; + + @JsonCreator + public BuildingSingleDimensionShardSpec( + @JsonProperty("dimension") String dimension, + @JsonProperty("start") @Nullable String start, + @JsonProperty("end") @Nullable String end, + @JsonProperty("partitionNum") int partitionNum + ) + { + this.dimension = dimension; + this.start = start; + this.end = end; + this.partitionNum = partitionNum; + } + + @JsonProperty("dimension") + public String getDimension() + { + return dimension; + } + + @Nullable + @JsonProperty("start") + public String getStart() + { + return start; + } + + @Nullable + @JsonProperty("end") + public String getEnd() + { + return end; + } + + @Override + @JsonProperty("partitionNum") + public int getPartitionNum() + { + return partitionNum; + } + + @Override + public SingleDimensionShardSpec convert(int numCorePartitions) + { + return new SingleDimensionShardSpec(dimension, start, end, partitionNum, numCorePartitions); + } + + @Override + public PartitionChunk createChunk(T obj) + { + return new NumberedPartitionChunk<>(partitionNum, 0, obj); + } +} diff --git a/core/src/main/java/org/apache/druid/timeline/partition/HashBasedNumberedPartialShardSpec.java b/core/src/main/java/org/apache/druid/timeline/partition/HashBasedNumberedPartialShardSpec.java index 895c1b7b1cc9..0e32ee04bcf1 100644 --- a/core/src/main/java/org/apache/druid/timeline/partition/HashBasedNumberedPartialShardSpec.java +++ b/core/src/main/java/org/apache/druid/timeline/partition/HashBasedNumberedPartialShardSpec.java @@ -33,15 +33,18 @@ public class HashBasedNumberedPartialShardSpec implements PartialShardSpec @Nullable private final List partitionDimensions; + private final int bucketId; private final int numBuckets; @JsonCreator public HashBasedNumberedPartialShardSpec( @JsonProperty("partitionDimensions") @Nullable List partitionDimensions, + @JsonProperty("bucketId") int bucketId, @JsonProperty("numPartitions") int numBuckets ) { this.partitionDimensions = partitionDimensions; + this.bucketId = bucketId; this.numBuckets = numBuckets; } @@ -52,6 +55,12 @@ public List getPartitionDimensions() return partitionDimensions; } + @JsonProperty + public int getBucketId() + { + return bucketId; + } + @JsonProperty("numPartitions") public int getNumBuckets() { @@ -61,36 +70,26 @@ public int getNumBuckets() @Override public ShardSpec complete(ObjectMapper objectMapper, @Nullable ShardSpec specOfPreviousMaxPartitionId) { - if (specOfPreviousMaxPartitionId == null) { - // The shardSpec is created by the Overlord. - // For batch tasks, this code can is executed only with segment locking (forceTimeChunkLock = false). - // In this mode, you can have 2 or more tasks concurrently ingesting into the same time chunk of - // the same datasource. Since there is no restriction for those tasks in segment allocation, the - // allocated IDs for each task can interleave. As a result, the core partition set cannot be - // represented as a range. We always set 0 for the core partition set size. - return new HashBasedNumberedShardSpec( - 0, - 0, - numBuckets, - partitionDimensions, - objectMapper - ); - } else { - final HashBasedNumberedShardSpec prevSpec = (HashBasedNumberedShardSpec) specOfPreviousMaxPartitionId; - return new HashBasedNumberedShardSpec( - prevSpec.getPartitionNum() + 1, - prevSpec.getPartitions(), - numBuckets, - partitionDimensions, - objectMapper - ); - } + // The shardSpec is created by the Overlord. + // For batch tasks, this code can be executed only with segment locking (forceTimeChunkLock = false). + // In this mode, you can have 2 or more tasks concurrently ingesting into the same time chunk of + // the same datasource. Since there is no restriction for those tasks in segment allocation, the + // allocated IDs for each task can interleave. As a result, the core partition set cannot be + // represented as a range. We always set 0 for the core partition set size if this is an initial segment. + return new HashBasedNumberedShardSpec( + specOfPreviousMaxPartitionId == null ? 0 : specOfPreviousMaxPartitionId.getPartitionNum() + 1, + specOfPreviousMaxPartitionId == null ? 0 : specOfPreviousMaxPartitionId.getNumCorePartitions(), + bucketId, + numBuckets, + partitionDimensions, + objectMapper + ); } @Override public ShardSpec complete(ObjectMapper objectMapper, int partitionId) { - return new HashBasedNumberedShardSpec(partitionId, 0, numBuckets, partitionDimensions, objectMapper); + return new HashBasedNumberedShardSpec(partitionId, 0, bucketId, numBuckets, partitionDimensions, objectMapper); } @Override @@ -109,13 +108,14 @@ public boolean equals(Object o) return false; } HashBasedNumberedPartialShardSpec that = (HashBasedNumberedPartialShardSpec) o; - return numBuckets == that.numBuckets && + return bucketId == that.bucketId && + numBuckets == that.numBuckets && Objects.equals(partitionDimensions, that.partitionDimensions); } @Override public int hashCode() { - return Objects.hash(partitionDimensions, numBuckets); + return Objects.hash(partitionDimensions, bucketId, numBuckets); } } 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 85d798edb031..215794a6c0a6 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 @@ -43,6 +43,7 @@ public class HashBasedNumberedShardSpec extends NumberedShardSpec private static final HashFunction HASH_FUNCTION = Hashing.murmur3_32(); + private final int bucketId; /** * Number of hash buckets */ @@ -55,12 +56,15 @@ public class HashBasedNumberedShardSpec extends NumberedShardSpec public HashBasedNumberedShardSpec( @JsonProperty("partitionNum") int partitionNum, // partitionId, hash bucketId @JsonProperty("partitions") int partitions, // core partition set size + @JsonProperty("bucketId") @Nullable Integer bucketId, // nullable for backward compatibility @JsonProperty("numBuckets") @Nullable Integer numBuckets, // nullable for backward compatibility @JsonProperty("partitionDimensions") @Nullable List partitionDimensions, @JacksonInject ObjectMapper jsonMapper ) { super(partitionNum, partitions); + // Use partitionId as bucketId if it's missing. + this.bucketId = bucketId == null ? partitionNum : bucketId; // If numBuckets is missing, assume that any hash bucket is not empty. // Use the core partition set size as the number of buckets. this.numBuckets = numBuckets == null ? partitions : numBuckets; @@ -68,6 +72,12 @@ public HashBasedNumberedShardSpec( this.partitionDimensions = partitionDimensions == null ? DEFAULT_PARTITION_DIMENSIONS : partitionDimensions; } + @JsonProperty + public int getBucketId() + { + return bucketId; + } + @JsonProperty public int getNumBuckets() { @@ -89,7 +99,7 @@ public boolean isCompatible(Class other) @Override public boolean isInChunk(long timestamp, InputRow inputRow) { - return (((long) hash(timestamp, inputRow)) - getPartitionNum()) % numBuckets == 0; + return (((long) hash(timestamp, inputRow)) - bucketId) % numBuckets == 0; } /** @@ -107,7 +117,7 @@ protected int hash(long timestamp, InputRow inputRow) return hash(jsonMapper, partitionDimensions, timestamp, inputRow); } - private static int hash(ObjectMapper jsonMapper, List partitionDimensions, long timestamp, InputRow inputRow) + public static int hash(ObjectMapper jsonMapper, List partitionDimensions, long timestamp, InputRow inputRow) { final List groupKey = getGroupKey(partitionDimensions, timestamp, inputRow); try { @@ -166,14 +176,15 @@ public boolean equals(Object o) return false; } HashBasedNumberedShardSpec that = (HashBasedNumberedShardSpec) o; - return numBuckets == that.numBuckets && + return bucketId == that.bucketId && + numBuckets == that.numBuckets && Objects.equals(partitionDimensions, that.partitionDimensions); } @Override public int hashCode() { - return Objects.hash(super.hashCode(), numBuckets, partitionDimensions); + return Objects.hash(super.hashCode(), bucketId, numBuckets, partitionDimensions); } @Override @@ -181,7 +192,8 @@ public String toString() { return "HashBasedNumberedShardSpec{" + "partitionNum=" + getPartitionNum() + - ", partitions=" + getPartitions() + + ", partitions=" + getNumCorePartitions() + + ", bucketId=" + bucketId + ", numBuckets=" + numBuckets + ", partitionDimensions=" + partitionDimensions + '}'; 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 1ebb24e16038..584b5e5a1178 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 @@ -49,6 +49,12 @@ public int getPartitionNum() return partitionNum; } + @Override + public int getNumCorePartitions() + { + return 0; + } + @Override public ShardSpecLookup getLookup(final List shardSpecs) { 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 dde92167362a..25c9f752bec0 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 @@ -65,6 +65,12 @@ public int getPartitionNum() return 0; } + @Override + public int getNumCorePartitions() + { + return 0; + } + @Override public ShardSpecLookup getLookup(final List shardSpecs) { diff --git a/core/src/main/java/org/apache/druid/timeline/partition/NumberedPartialShardSpec.java b/core/src/main/java/org/apache/druid/timeline/partition/NumberedPartialShardSpec.java index 7c7b9753aaa8..730502845f30 100644 --- a/core/src/main/java/org/apache/druid/timeline/partition/NumberedPartialShardSpec.java +++ b/core/src/main/java/org/apache/druid/timeline/partition/NumberedPartialShardSpec.java @@ -50,7 +50,7 @@ public ShardSpec complete(ObjectMapper objectMapper, @Nullable ShardSpec specOfP return new NumberedShardSpec(0, 0); } else { final NumberedShardSpec prevSpec = (NumberedShardSpec) specOfPreviousMaxPartitionId; - return new NumberedShardSpec(prevSpec.getPartitionNum() + 1, prevSpec.getPartitions()); + return new NumberedShardSpec(prevSpec.getPartitionNum() + 1, prevSpec.getNumCorePartitions()); } } 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 59ebba4aa152..a952e0436768 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 @@ -94,8 +94,9 @@ public boolean isCompatible(Class other) return other == NumberedShardSpec.class || other == NumberedOverwriteShardSpec.class; } + @Override @JsonProperty("partitions") - public int getPartitions() + public int getNumCorePartitions() { return partitions; } diff --git a/core/src/main/java/org/apache/druid/timeline/partition/OverwriteShardSpec.java b/core/src/main/java/org/apache/druid/timeline/partition/OverwriteShardSpec.java index 0fea5646e311..193b573acbf5 100644 --- a/core/src/main/java/org/apache/druid/timeline/partition/OverwriteShardSpec.java +++ b/core/src/main/java/org/apache/druid/timeline/partition/OverwriteShardSpec.java @@ -28,6 +28,15 @@ */ public interface OverwriteShardSpec extends ShardSpec { + /** + * TODO + */ + @Override + default int getNumCorePartitions() + { + return 0; + } + default OverwriteShardSpec withAtomicUpdateGroupSize(int atomicUpdateGroupSize) { return 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 c964a7065ebd..af78e3dcdd2f 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 @@ -56,6 +56,8 @@ public interface ShardSpec */ int getPartitionNum(); + int getNumCorePartitions(); + /** * Returns the start root partition ID of the atomic update group which this segment belongs to. * diff --git a/core/src/main/java/org/apache/druid/timeline/partition/SingleDimensionPartialShardSpec.java b/core/src/main/java/org/apache/druid/timeline/partition/SingleDimensionPartialShardSpec.java index 22ca97d3fe19..e2ebd295c356 100644 --- a/core/src/main/java/org/apache/druid/timeline/partition/SingleDimensionPartialShardSpec.java +++ b/core/src/main/java/org/apache/druid/timeline/partition/SingleDimensionPartialShardSpec.java @@ -87,15 +87,19 @@ public int getNumBuckets() @Override public ShardSpec complete(ObjectMapper objectMapper, @Nullable ShardSpec specOfPreviousMaxPartitionId) { - final int partitionId; - if (specOfPreviousMaxPartitionId != null) { - assert specOfPreviousMaxPartitionId instanceof SingleDimensionShardSpec; - final SingleDimensionShardSpec prevSpec = (SingleDimensionShardSpec) specOfPreviousMaxPartitionId; - partitionId = prevSpec.getPartitionNum() + 1; - } else { - partitionId = 0; - } - return complete(objectMapper, partitionId); + // The shardSpec is created by the Overlord. + // For batch tasks, this code can be executed only with segment locking (forceTimeChunkLock = false). + // In this mode, you can have 2 or more tasks concurrently ingesting into the same time chunk of + // the same datasource. Since there is no restriction for those tasks in segment allocation, the + // allocated IDs for each task can interleave. As a result, the core partition set cannot be + // represented as a range. We always set 0 for the core partition set size if this is an initial segment. + return new SingleDimensionShardSpec( + partitionDimension, + start, + end, + specOfPreviousMaxPartitionId == null ? 0 : specOfPreviousMaxPartitionId.getPartitionNum() + 1, + specOfPreviousMaxPartitionId == null ? 0 : specOfPreviousMaxPartitionId.getNumCorePartitions() + ); } @Override @@ -106,7 +110,8 @@ public ShardSpec complete(ObjectMapper objectMapper, int partitionId) partitionDimension, start, end, - partitionId + partitionId, + 0 ); } 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 9db390c462fe..05bad5f3ae7e 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 @@ -38,12 +38,15 @@ */ public class SingleDimensionShardSpec implements ShardSpec { + public static final int UNKNOWN_NUM_CORE_PARTITIONS = -1; + private final String dimension; @Nullable private final String start; @Nullable private final String end; private final int partitionNum; + private final int numCorePartitions; /** * @param dimension partition dimension @@ -56,7 +59,8 @@ public SingleDimensionShardSpec( @JsonProperty("dimension") String dimension, @JsonProperty("start") @Nullable String start, @JsonProperty("end") @Nullable String end, - @JsonProperty("partitionNum") int partitionNum + @JsonProperty("partitionNum") int partitionNum, + @JsonProperty("numCorePartitions") @Nullable Integer numCorePartitions // nullable for backward compatibility ) { Preconditions.checkArgument(partitionNum >= 0, "partitionNum >= 0"); @@ -64,6 +68,7 @@ public SingleDimensionShardSpec( this.start = start; this.end = end; this.partitionNum = partitionNum; + this.numCorePartitions = numCorePartitions == null ? UNKNOWN_NUM_CORE_PARTITIONS : numCorePartitions; } @JsonProperty("dimension") @@ -93,6 +98,13 @@ public int getPartitionNum() return partitionNum; } + @Override + @JsonProperty + public int getNumCorePartitions() + { + return numCorePartitions; + } + @Override public ShardSpecLookup getLookup(final List shardSpecs) { @@ -146,6 +158,7 @@ public boolean isCompatible(Class other) @Override public PartitionChunk createChunk(T obj) { + // TODO: use numberedChunk return new StringPartitionChunk(start, end, partitionNum, obj); } @@ -175,17 +188,6 @@ private boolean checkValue(String value) (end == null || value.compareTo(end) < 0); } - @Override - public String toString() - { - return "SingleDimensionShardSpec{" + - "dimension='" + dimension + '\'' + - ", start='" + start + '\'' + - ", end='" + end + '\'' + - ", partitionNum=" + partitionNum + - '}'; - } - @Override public boolean equals(Object o) { @@ -195,16 +197,29 @@ public boolean equals(Object o) if (o == null || getClass() != o.getClass()) { return false; } - SingleDimensionShardSpec that = (SingleDimensionShardSpec) o; - return partitionNum == that.partitionNum && - Objects.equals(dimension, that.dimension) && - Objects.equals(start, that.start) && - Objects.equals(end, that.end); + SingleDimensionShardSpec shardSpec = (SingleDimensionShardSpec) o; + return partitionNum == shardSpec.partitionNum && + numCorePartitions == shardSpec.numCorePartitions && + Objects.equals(dimension, shardSpec.dimension) && + Objects.equals(start, shardSpec.start) && + Objects.equals(end, shardSpec.end); } @Override public int hashCode() { - return Objects.hash(dimension, start, end, partitionNum); + return Objects.hash(dimension, start, end, partitionNum, numCorePartitions); + } + + @Override + public String toString() + { + return "SingleDimensionShardSpec{" + + "dimension='" + dimension + '\'' + + ", start='" + start + '\'' + + ", end='" + end + '\'' + + ", partitionNum=" + partitionNum + + ", numCorePartitions=" + numCorePartitions + + '}'; } } 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 6d0af3f2892e..62913646e92b 100644 --- a/core/src/test/java/org/apache/druid/timeline/DataSegmentTest.java +++ b/core/src/test/java/org/apache/druid/timeline/DataSegmentTest.java @@ -74,6 +74,12 @@ public int getPartitionNum() return partitionNum; } + @Override + public int getNumCorePartitions() + { + return 0; + } + @Override public ShardSpecLookup getLookup(List shardSpecs) { diff --git a/core/src/test/java/org/apache/druid/timeline/partition/BuildingHashBasedNumberedShardSpecTest.java b/core/src/test/java/org/apache/druid/timeline/partition/BuildingHashBasedNumberedShardSpecTest.java index 46ecec42c87f..8b678a8852d1 100644 --- a/core/src/test/java/org/apache/druid/timeline/partition/BuildingHashBasedNumberedShardSpecTest.java +++ b/core/src/test/java/org/apache/druid/timeline/partition/BuildingHashBasedNumberedShardSpecTest.java @@ -41,8 +41,8 @@ public class BuildingHashBasedNumberedShardSpecTest public void testConvert() { Assert.assertEquals( - new HashBasedNumberedShardSpec(5, 10, 12, ImmutableList.of("dim"), new ObjectMapper()), - new BuildingHashBasedNumberedShardSpec(5, 12, ImmutableList.of("dim"), new ObjectMapper()).convert(10) + new HashBasedNumberedShardSpec(5, 10, 5, 12, ImmutableList.of("dim"), new ObjectMapper()), + new BuildingHashBasedNumberedShardSpec(5, 5, 12, ImmutableList.of("dim"), new ObjectMapper()).convert(10) ); } @@ -51,7 +51,8 @@ public void testCreateChunk() { Assert.assertEquals( new NumberedPartitionChunk<>(5, 0, "test"), - new BuildingHashBasedNumberedShardSpec(5, 12, ImmutableList.of("dim"), new ObjectMapper()).createChunk("test") + new BuildingHashBasedNumberedShardSpec(5, 5, 12, ImmutableList.of("dim"), new ObjectMapper()) + .createChunk("test") ); } @@ -59,9 +60,9 @@ public void testCreateChunk() public void testShardSpecLookup() { final List shardSpecs = ImmutableList.of( - new BuildingHashBasedNumberedShardSpec(0, 3, ImmutableList.of("dim"), new ObjectMapper()), - new BuildingHashBasedNumberedShardSpec(1, 3, ImmutableList.of("dim"), new ObjectMapper()), - new BuildingHashBasedNumberedShardSpec(2, 3, ImmutableList.of("dim"), new ObjectMapper()) + new BuildingHashBasedNumberedShardSpec(0, 0, 3, ImmutableList.of("dim"), new ObjectMapper()), + new BuildingHashBasedNumberedShardSpec(1, 1, 3, ImmutableList.of("dim"), new ObjectMapper()), + new BuildingHashBasedNumberedShardSpec(2, 2, 3, ImmutableList.of("dim"), new ObjectMapper()) ); final ShardSpecLookup lookup = shardSpecs.get(0).getLookup(shardSpecs); final long currentTime = DateTimes.nowUtc().getMillis(); @@ -109,6 +110,7 @@ public void testSerde() throws JsonProcessingException )); mapper.setInjectableValues(new Std().addValue(ObjectMapper.class, mapper)); final BuildingHashBasedNumberedShardSpec original = new BuildingHashBasedNumberedShardSpec( + 3, 5, 12, ImmutableList.of("dim"), diff --git a/core/src/test/java/org/apache/druid/timeline/partition/HashBasedNumberedPartialShardSpecTest.java b/core/src/test/java/org/apache/druid/timeline/partition/HashBasedNumberedPartialShardSpecTest.java index 9b2c664679f3..551992bdb2d6 100644 --- a/core/src/test/java/org/apache/druid/timeline/partition/HashBasedNumberedPartialShardSpecTest.java +++ b/core/src/test/java/org/apache/druid/timeline/partition/HashBasedNumberedPartialShardSpecTest.java @@ -46,6 +46,7 @@ public void testSerde() throws IOException { final HashBasedNumberedPartialShardSpec expected = new HashBasedNumberedPartialShardSpec( ImmutableList.of("dim1", "dim2"), + 1, 3 ); final byte[] json = MAPPER.writeValueAsBytes(expected); @@ -61,14 +62,16 @@ public void testJsonPropertyNames() throws IOException { final HashBasedNumberedPartialShardSpec expected = new HashBasedNumberedPartialShardSpec( ImmutableList.of("dim1", "dim2"), + 1, 3 ); final byte[] json = MAPPER.writeValueAsBytes(expected); //noinspection unchecked final Map map = MAPPER.readValue(json, Map.class); - Assert.assertEquals(3, map.size()); + Assert.assertEquals(4, map.size()); Assert.assertEquals(HashBasedNumberedPartialShardSpec.TYPE, map.get("type")); Assert.assertEquals(expected.getPartitionDimensions(), map.get("partitionDimensions")); + Assert.assertEquals(expected.getBucketId(), map.get("bucketId")); Assert.assertEquals(expected.getNumBuckets(), map.get("numPartitions")); } } diff --git a/extensions-contrib/materialized-view-maintenance/src/test/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisorTest.java b/extensions-contrib/materialized-view-maintenance/src/test/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisorTest.java index a85a1fb752f3..766f5109269e 100644 --- a/extensions-contrib/materialized-view-maintenance/src/test/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisorTest.java +++ b/extensions-contrib/materialized-view-maintenance/src/test/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisorTest.java @@ -140,7 +140,7 @@ public void testCheckSegments() throws IOException ImmutableMap.of(), ImmutableList.of("dim1", "dim2"), ImmutableList.of("m1"), - new HashBasedNumberedShardSpec(0, 1, 1, null, null), + new HashBasedNumberedShardSpec(0, 1, 0, 1, null, null), 9, 1024 ), @@ -151,7 +151,7 @@ public void testCheckSegments() throws IOException ImmutableMap.of(), ImmutableList.of("dim1", "dim2"), ImmutableList.of("m1"), - new HashBasedNumberedShardSpec(0, 1, 1, null, null), + new HashBasedNumberedShardSpec(0, 1, 0, 1, null, null), 9, 1024 ), @@ -162,7 +162,7 @@ public void testCheckSegments() throws IOException ImmutableMap.of(), ImmutableList.of("dim1", "dim2"), ImmutableList.of("m1"), - new HashBasedNumberedShardSpec(0, 1, 1, null, null), + new HashBasedNumberedShardSpec(0, 1, 0, 1, null, null), 9, 1024 ) @@ -175,7 +175,7 @@ public void testCheckSegments() throws IOException ImmutableMap.of(), ImmutableList.of("dim1", "dim2"), ImmutableList.of("m1"), - new HashBasedNumberedShardSpec(0, 1, 1, null, null), + new HashBasedNumberedShardSpec(0, 1, 0, 1, null, null), 9, 1024 ), @@ -186,7 +186,7 @@ public void testCheckSegments() throws IOException ImmutableMap.of(), ImmutableList.of("dim1", "dim2"), ImmutableList.of("m1"), - new HashBasedNumberedShardSpec(0, 1, 1, null, null), + new HashBasedNumberedShardSpec(0, 1, 0, 1, null, null), 9, 1024 ) @@ -209,7 +209,7 @@ public void testCheckSegments() throws IOException ImmutableMap.of(), ImmutableList.of("dim1", "dim2"), ImmutableList.of("m1"), - new HashBasedNumberedShardSpec(0, 1, 1, null, null), + new HashBasedNumberedShardSpec(0, 1, 0, 1, null, null), 9, 1024 ) @@ -225,7 +225,7 @@ public void testCheckSegments() throws IOException ImmutableMap.of(), ImmutableList.of("dim1", "dim2"), ImmutableList.of("m1"), - new HashBasedNumberedShardSpec(0, 1, 1, null, null), + new HashBasedNumberedShardSpec(0, 1, 0, 1, null, null), 9, 1024 ) @@ -246,7 +246,7 @@ public void testCheckSegmentsAndSubmitTasks() throws IOException ImmutableMap.of(), ImmutableList.of("dim1", "dim2"), ImmutableList.of("m1"), - new HashBasedNumberedShardSpec(0, 1, 1, null, null), + new HashBasedNumberedShardSpec(0, 1, 0, 1, null, null), 9, 1024 ) diff --git a/indexing-hadoop/src/main/java/org/apache/druid/indexer/DetermineHashedPartitionsJob.java b/indexing-hadoop/src/main/java/org/apache/druid/indexer/DetermineHashedPartitionsJob.java index c7b0fd07597c..62a13d3f9fc4 100644 --- a/indexing-hadoop/src/main/java/org/apache/druid/indexer/DetermineHashedPartitionsJob.java +++ b/indexing-hadoop/src/main/java/org/apache/druid/indexer/DetermineHashedPartitionsJob.java @@ -196,6 +196,7 @@ public boolean run() new HashBasedNumberedShardSpec( i, numberOfShards, + i, numberOfShards, null, HadoopDruidIndexerConfig.JSON_MAPPER diff --git a/indexing-hadoop/src/main/java/org/apache/druid/indexer/DeterminePartitionsJob.java b/indexing-hadoop/src/main/java/org/apache/druid/indexer/DeterminePartitionsJob.java index d9e6a14ce56a..a0bebc6239a5 100644 --- a/indexing-hadoop/src/main/java/org/apache/druid/indexer/DeterminePartitionsJob.java +++ b/indexing-hadoop/src/main/java/org/apache/druid/indexer/DeterminePartitionsJob.java @@ -666,7 +666,8 @@ protected void innerReduce(Context context, SortableBytes keyBytes, Iterable constructShardSpecFromShardInfo(String partitionType, Ob specs.add(new HashBasedNumberedShardSpec( shardInfo[0], shardInfo[1], + shardInfo[0], shardInfo[1], null, HadoopDruidIndexerConfig.JSON_MAPPER @@ -574,7 +575,8 @@ private List constructShardSpecFromShardInfo(String partitionType, Ob "host", shardInfo[0], shardInfo[1], - partitionNum++ + partitionNum++, + shardInfoForEachShard.length )); } } else { @@ -694,12 +696,12 @@ private void verifyJob(IndexGeneratorJob job) throws IOException if (forceExtendableShardSpecs) { NumberedShardSpec spec = (NumberedShardSpec) dataSegment.getShardSpec(); Assert.assertEquals(i, spec.getPartitionNum()); - Assert.assertEquals(shardInfo.length, spec.getPartitions()); + Assert.assertEquals(shardInfo.length, spec.getNumCorePartitions()); } else if ("hashed".equals(partitionType)) { Integer[] hashShardInfo = (Integer[]) shardInfo[i]; HashBasedNumberedShardSpec spec = (HashBasedNumberedShardSpec) dataSegment.getShardSpec(); Assert.assertEquals((int) hashShardInfo[0], spec.getPartitionNum()); - Assert.assertEquals((int) hashShardInfo[1], spec.getPartitions()); + Assert.assertEquals((int) hashShardInfo[1], spec.getNumCorePartitions()); } else if ("single".equals(partitionType)) { String[] singleDimensionShardInfo = (String[]) shardInfo[i]; SingleDimensionShardSpec spec = (SingleDimensionShardSpec) dataSegment.getShardSpec(); 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 9040c9acd3f9..01ecc0855be3 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,6 +20,8 @@ package org.apache.druid.indexing.common.task; import com.google.common.base.Preconditions; +import it.unimi.dsi.fastutil.objects.Object2IntMap; +import it.unimi.dsi.fastutil.objects.Object2IntOpenHashMap; import org.apache.druid.data.input.InputRow; import org.apache.druid.indexing.common.TaskLock; import org.apache.druid.indexing.common.TaskToolbox; @@ -27,15 +29,18 @@ import org.apache.druid.indexing.common.actions.SurrogateAction; import org.apache.druid.indexing.common.actions.TaskAction; import org.apache.druid.indexing.common.task.batch.parallel.SupervisorTaskAccess; +import org.apache.druid.indexing.common.task.batch.partition.CompletePartitionAnalysis; +import org.apache.druid.indexing.common.task.batch.partition.PartitionBucket; +import org.apache.druid.indexing.common.task.batch.partition.PartitionBucketLookup; +import org.apache.druid.indexing.common.task.batch.partition.PartitionBuckets; import org.apache.druid.java.util.common.ISE; -import org.apache.druid.java.util.common.granularity.Granularity; +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.ShardSpec; 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; @@ -46,38 +51,27 @@ /** * Allocates all necessary segments locally at the beginning and reuses them. */ -public class CachingLocalSegmentAllocator implements CachingSegmentAllocator +public class CachingLocalSegmentAllocator implements SegmentAllocator { - private final String taskId; - private final Map sequenceNameToSegmentId; - private final ShardSpecs shardSpecs; + private final String dataSource; + private final Map sequenceNameToBucket; + private final Function versionFinder; + private final NonLinearlyPartitionedSequenceNameFunction sequenceNameFunction; - @FunctionalInterface - interface IntervalToSegmentIdsCreator - { - /** - * @param versionFinder Returns the version for the specified interval - * - * @return Information for segment preallocation - */ - Map> create( - TaskToolbox toolbox, - String dataSource, - Function versionFinder - ); - } + private final Map sequenceNameToSegmentId = new HashMap<>(); + private final Object2IntMap intervalToNextPartitionId; CachingLocalSegmentAllocator( TaskToolbox toolbox, String dataSource, String taskId, - Granularity queryGranularity, + GranularitySpec granularitySpec, @Nullable SupervisorTaskAccess supervisorTaskAccess, - IntervalToSegmentIdsCreator intervalToSegmentIdsCreator + CompletePartitionAnalysis partitionAnalysis ) throws IOException { - this.taskId = taskId; - this.sequenceNameToSegmentId = new HashMap<>(); + this.dataSource = dataSource; + this.sequenceNameToBucket = new HashMap<>(); final TaskAction> action; if (supervisorTaskAccess == null) { @@ -90,30 +84,26 @@ Map> create( toolbox.getTaskActionClient() .submit(action) .stream() - .collect(Collectors.toMap( - TaskLock::getInterval, - TaskLock::getVersion - )); - Function versionFinder = interval -> findVersion(intervalToVersion, interval); + .collect(Collectors.toMap(TaskLock::getInterval, TaskLock::getVersion)); + this.versionFinder = interval -> findVersion(intervalToVersion, interval); + final Map intervalToBucketLookup = partitionAnalysis.createBuckets(toolbox); + intervalToNextPartitionId = new Object2IntOpenHashMap<>(intervalToBucketLookup.size()); + intervalToNextPartitionId.defaultReturnValue(0); - final Map> intervalToIds = intervalToSegmentIdsCreator.create( - toolbox, - dataSource, - versionFinder + sequenceNameFunction = new NonLinearlyPartitionedSequenceNameFunction( + taskId, + new PartitionBuckets(granularitySpec, intervalToBucketLookup) ); - final Map> shardSpecMap = new HashMap<>(); - for (Entry> entry : intervalToIds.entrySet()) { + for (Entry entry : intervalToBucketLookup.entrySet()) { final Interval interval = entry.getKey(); - final List idsPerInterval = intervalToIds.get(interval); + final PartitionBucketLookup bucketLookup = entry.getValue(); - for (SegmentIdWithShardSpec segmentIdentifier : idsPerInterval) { - shardSpecMap.computeIfAbsent(interval, k -> new ArrayList<>()).add(segmentIdentifier.getShardSpec()); + bucketLookup.iterator().forEachRemaining(bucket -> { // The shardSpecs for partitioning and publishing can be different if isExtendableShardSpecs = true. - sequenceNameToSegmentId.put(getSequenceName(interval, segmentIdentifier.getShardSpec()), segmentIdentifier); - } + sequenceNameToBucket.put(sequenceNameFunction.getSequenceName(interval, bucket), bucket); + }); } - shardSpecs = new ShardSpecs(shardSpecMap, queryGranularity); } private static String findVersion(Map intervalToVersion, Interval interval) @@ -133,28 +123,37 @@ public SegmentIdWithShardSpec allocate( boolean skipSegmentLineageCheck ) { - return Preconditions.checkNotNull( - sequenceNameToSegmentId.get(sequenceName), - "Missing segmentId for the sequence[%s]", - sequenceName + return sequenceNameToSegmentId.computeIfAbsent( + sequenceName, + k -> { + final PartitionBucket bucket = Preconditions.checkNotNull( + sequenceNameToBucket.get(sequenceName), + "Missing bucket for sequence[%s]", + sequenceName + ); + final Interval interval = bucket.getInterval(); + final String version = versionFinder.apply(interval); + return new SegmentIdWithShardSpec( + dataSource, + interval, + version, + bucket.toShardSpec(getPartitionIdAndIncrement(interval)) + ); + } ); } - /** - * 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) + private int getPartitionIdAndIncrement(Interval interval) { - // Note: We do not use String format here since this can be called in a tight loop - // and it's faster to add strings together than it is to use String#format - return taskId + "_" + interval + "_" + shardSpec.getPartitionNum(); + return intervalToNextPartitionId.computeInt( + interval, + (i, nextPartitionId) -> nextPartitionId == null ? 0 : nextPartitionId + 1 + ); } - @Override - public ShardSpecs getShardSpecs() + // TODO: i don't like it + public SequenceNameFunction getSequenceNameFunction() { - return shardSpecs; + return sequenceNameFunction; } } 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 176d45e2d2c4..000000000000 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CachingSegmentAllocator.java +++ /dev/null @@ -1,38 +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.segment.realtime.appenderator.SegmentAllocator; - -/** - * SegmentAllocator that allocates all necessary segments upfront. This allocator should be used for the hash or range - * secondary partitioning. - * - * In the hash or range secondary partitioning, the information about all partition buckets should be known before - * the task starts to allocate segments. For example, for the hash partitioning, the task should know how many hash - * buckets it will create, what is the hash value allocated for each bucket, etc. Similar for the range partitioning. - */ -public interface CachingSegmentAllocator extends SegmentAllocator -{ - /** - * Returns the {@link org.apache.druid.timeline.partition.ShardSpec}s of all segments allocated upfront. - */ - ShardSpecs getShardSpecs(); -} 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 14d806b54c88..fa8d1661fd52 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 @@ -882,18 +882,16 @@ private TaskStatus generateAndPublishSegments( switch (partitionsSpec.getType()) { case HASH: case RANGE: - final CachingSegmentAllocator localSegmentAllocator = SegmentAllocators.forNonLinearPartitioning( + final SegmentAllocator localSegmentAllocator = SegmentAllocators.forNonLinearPartitioning( toolbox, getDataSource(), getId(), - dataSchema.getGranularitySpec().getQueryGranularity(), + dataSchema.getGranularitySpec(), null, (CompletePartitionAnalysis) partitionAnalysis ); - sequenceNameFunction = new NonLinearlyPartitionedSequenceNameFunction( - getId(), - localSegmentAllocator.getShardSpecs() - ); + // TODO: + sequenceNameFunction = ((CachingLocalSegmentAllocator) localSegmentAllocator).getSequenceNameFunction(); segmentAllocator = localSegmentAllocator; break; case LINEAR: diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/NonLinearlyPartitionedSequenceNameFunction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/NonLinearlyPartitionedSequenceNameFunction.java index c34aa2995983..82fdebe7cef3 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/NonLinearlyPartitionedSequenceNameFunction.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/NonLinearlyPartitionedSequenceNameFunction.java @@ -20,7 +20,8 @@ package org.apache.druid.indexing.common.task; import org.apache.druid.data.input.InputRow; -import org.apache.druid.timeline.partition.ShardSpec; +import org.apache.druid.indexing.common.task.batch.partition.PartitionBucket; +import org.apache.druid.indexing.common.task.batch.partition.PartitionBuckets; import org.joda.time.Interval; /** @@ -30,24 +31,23 @@ * Note that all segment IDs should be allocated upfront to use this function. * * @see org.apache.druid.indexer.partitions.SecondaryPartitionType - * @see CachingSegmentAllocator */ public class NonLinearlyPartitionedSequenceNameFunction implements SequenceNameFunction { private final String taskId; - private final ShardSpecs shardSpecs; + private final PartitionBuckets partitionBuckets; - public NonLinearlyPartitionedSequenceNameFunction(String taskId, ShardSpecs shardSpecs) + public NonLinearlyPartitionedSequenceNameFunction(String taskId, PartitionBuckets partitionBuckets) { this.taskId = taskId; - this.shardSpecs = shardSpecs; + this.partitionBuckets = partitionBuckets; } @Override public String getSequenceName(Interval interval, InputRow inputRow) { // Sequence name is based solely on the shardSpec, and there will only be one segment per sequence. - return getSequenceName(interval, shardSpecs.getShardSpec(interval, inputRow)); + return getSequenceName(interval, partitionBuckets.lookupBucket(inputRow)); } /** @@ -55,10 +55,10 @@ public String getSequenceName(Interval interval, InputRow inputRow) * * See {@link org.apache.druid.timeline.partition.HashBasedNumberedShardSpec} as an example of partitioning. */ - private String getSequenceName(Interval interval, ShardSpec shardSpec) + public String getSequenceName(Interval interval, PartitionBucket bucket) { // Note: We do not use String format here since this can be called in a tight loop // and it's faster to add strings together than it is to use String#format - return taskId + "_" + interval + "_" + shardSpec.getPartitionNum(); + return taskId + "_" + interval + "_" + bucket.getBucketId(); } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/SegmentAllocators.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/SegmentAllocators.java index de88c68f65a6..5a7c7ab34dad 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/SegmentAllocators.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/SegmentAllocators.java @@ -23,8 +23,8 @@ import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.common.task.batch.parallel.SupervisorTaskAccess; import org.apache.druid.indexing.common.task.batch.partition.CompletePartitionAnalysis; -import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.segment.indexing.DataSchema; +import org.apache.druid.segment.indexing.granularity.GranularitySpec; import org.apache.druid.segment.realtime.appenderator.SegmentAllocator; import javax.annotation.Nullable; @@ -74,11 +74,11 @@ public static SegmentAllocator forLinearPartitioning( * Creates a new {@link SegmentAllocator} for the hash and range partitioning. * supervisorTaskAccess can be null if this method is called by the {@link IndexTask}. */ - public static CachingSegmentAllocator forNonLinearPartitioning( + public static SegmentAllocator forNonLinearPartitioning( final TaskToolbox toolbox, final String dataSource, final String taskId, - final Granularity queryGranularity, + final GranularitySpec granularitySpec, final @Nullable SupervisorTaskAccess supervisorTaskAccess, final CompletePartitionAnalysis partitionAnalysis ) throws IOException @@ -87,9 +87,9 @@ public static CachingSegmentAllocator forNonLinearPartitioning( toolbox, dataSource, taskId, - queryGranularity, + granularitySpec, supervisorTaskAccess, - partitionAnalysis::convertToIntervalToSegmentIds + partitionAnalysis ); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/ShardSpecs.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/ShardSpecs.java index 42f7ce1cd531..372d87671c95 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/ShardSpecs.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/ShardSpecs.java @@ -1,62 +1,62 @@ -/* - * 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.java.util.common.ISE; -import org.apache.druid.java.util.common.granularity.Granularity; -import org.apache.druid.timeline.partition.ShardSpec; -import org.joda.time.Interval; - -import java.util.List; -import java.util.Map; - -/** - * This class represents a map of (Interval, ShardSpec) and is used for easy shardSpec generation. - */ -public class ShardSpecs -{ - private final Map> map; - private Granularity queryGranularity; - - ShardSpecs(final Map> map, Granularity queryGranularity) - { - this.map = map; - this.queryGranularity = queryGranularity; - } - - /** - * Return a shardSpec for the given interval and input row. - * - * @param interval interval for shardSpec - * @param row input row - * - * @return a shardSpec - */ - ShardSpec getShardSpec(Interval interval, InputRow row) - { - final List shardSpecs = map.get(interval); - if (shardSpecs == null || shardSpecs.isEmpty()) { - throw new ISE("Failed to get shardSpec for interval[%s]", interval); - } - final long truncatedTimestamp = queryGranularity.bucketStart(row.getTimestamp()).getMillis(); - return shardSpecs.get(0).getLookup(shardSpecs).getShardSpec(truncatedTimestamp, row); - } -} +///* +// * 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.java.util.common.ISE; +//import org.apache.druid.java.util.common.granularity.Granularity; +//import org.apache.druid.timeline.partition.ShardSpec; +//import org.joda.time.Interval; +// +//import java.util.List; +//import java.util.Map; +// +///** +// * This class represents a map of (Interval, ShardSpec) and is used for easy shardSpec generation. +// */ +//public class ShardSpecs +//{ +// private final Map> map; +// private Granularity queryGranularity; +// +// ShardSpecs(final Map> map, Granularity queryGranularity) +// { +// this.map = map; +// this.queryGranularity = queryGranularity; +// } +// +// /** +// * Return a shardSpec for the given interval and input row. +// * +// * @param interval interval for shardSpec +// * @param row input row +// * +// * @return a shardSpec +// */ +// ShardSpec getShardSpec(Interval interval, InputRow row) +// { +// final List shardSpecs = map.get(interval); +// if (shardSpecs == null || shardSpecs.isEmpty()) { +// throw new ISE("Failed to get shardSpec for interval[%s]", interval); +// } +// final long truncatedTimestamp = queryGranularity.bucketStart(row.getTimestamp()).getMillis(); +// return shardSpecs.get(0).getLookup(shardSpecs).getShardSpec(truncatedTimestamp, row); +// } +//} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentGenerateTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentGenerateTask.java index bf33faef292e..599f2470df09 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentGenerateTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentGenerateTask.java @@ -26,7 +26,6 @@ import org.apache.druid.indexer.partitions.HashedPartitionsSpec; import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.common.actions.TaskActionClient; -import org.apache.druid.indexing.common.task.CachingSegmentAllocator; import org.apache.druid.indexing.common.task.IndexTaskClientFactory; import org.apache.druid.indexing.common.task.SegmentAllocators; import org.apache.druid.indexing.common.task.TaskResource; @@ -34,6 +33,7 @@ import org.apache.druid.indexing.common.task.batch.partition.HashPartitionAnalysis; import org.apache.druid.segment.indexing.granularity.GranularitySpec; import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager; +import org.apache.druid.segment.realtime.appenderator.SegmentAllocator; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.partition.PartialShardSpec; import org.joda.time.Interval; @@ -127,7 +127,7 @@ public boolean isReady(TaskActionClient taskActionClient) throws Exception } @Override - CachingSegmentAllocator createSegmentAllocator(TaskToolbox toolbox, ParallelIndexSupervisorTaskClient taskClient) + SegmentAllocator createSegmentAllocator(TaskToolbox toolbox, ParallelIndexSupervisorTaskClient taskClient) throws IOException { final GranularitySpec granularitySpec = ingestionSchema.getDataSchema().getGranularitySpec(); @@ -137,7 +137,7 @@ CachingSegmentAllocator createSegmentAllocator(TaskToolbox toolbox, ParallelInde toolbox, getDataSource(), getId(), - granularitySpec.getQueryGranularity(), + granularitySpec, new SupervisorTaskAccess(supervisorTaskId, taskClient), createHashPartitionAnalysisFromPartitionsSpec(granularitySpec, partitionsSpec) ); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentMergeTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentMergeTask.java index 501b39709fa8..be869a2c7937 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentMergeTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentMergeTask.java @@ -104,6 +104,7 @@ BuildingHashBasedNumberedShardSpec createShardSpec(TaskToolbox toolbox, Interval { return new BuildingHashBasedNumberedShardSpec( partitionId, + partitionId, // TODO: should use shardSpec instead Preconditions.checkNotNull(partitionsSpec.getNumShards(), "numShards"), partitionsSpec.getPartitionDimensions(), toolbox.getJsonMapper() diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialRangeSegmentGenerateTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialRangeSegmentGenerateTask.java index 60bbb7aac9c8..33b073d44f15 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialRangeSegmentGenerateTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialRangeSegmentGenerateTask.java @@ -28,7 +28,6 @@ import org.apache.druid.indexer.partitions.SingleDimensionPartitionsSpec; import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.common.actions.TaskActionClient; -import org.apache.druid.indexing.common.task.CachingSegmentAllocator; import org.apache.druid.indexing.common.task.IndexTaskClientFactory; import org.apache.druid.indexing.common.task.SegmentAllocators; import org.apache.druid.indexing.common.task.TaskResource; @@ -36,6 +35,7 @@ import org.apache.druid.indexing.common.task.batch.partition.RangePartitionAnalysis; import org.apache.druid.indexing.worker.ShuffleDataSegmentPusher; import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager; +import org.apache.druid.segment.realtime.appenderator.SegmentAllocator; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.partition.PartitionBoundaries; import org.joda.time.Interval; @@ -150,7 +150,7 @@ public boolean isReady(TaskActionClient taskActionClient) } @Override - CachingSegmentAllocator createSegmentAllocator(TaskToolbox toolbox, ParallelIndexSupervisorTaskClient taskClient) + SegmentAllocator createSegmentAllocator(TaskToolbox toolbox, ParallelIndexSupervisorTaskClient taskClient) throws IOException { final RangePartitionAnalysis partitionAnalysis = new RangePartitionAnalysis( @@ -161,7 +161,7 @@ CachingSegmentAllocator createSegmentAllocator(TaskToolbox toolbox, ParallelInde toolbox, getDataSource(), getId(), - ingestionSchema.getDataSchema().getGranularitySpec().getQueryGranularity(), + ingestionSchema.getDataSchema().getGranularitySpec(), new SupervisorTaskAccess(supervisorTaskId, taskClient), partitionAnalysis ); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentGenerateTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentGenerateTask.java index 65e570a9d84d..1c6018ac360f 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentGenerateTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentGenerateTask.java @@ -28,11 +28,10 @@ import org.apache.druid.indexing.common.stats.DropwizardRowIngestionMeters; import org.apache.druid.indexing.common.stats.RowIngestionMeters; import org.apache.druid.indexing.common.task.BatchAppenderators; -import org.apache.druid.indexing.common.task.CachingSegmentAllocator; +import org.apache.druid.indexing.common.task.CachingLocalSegmentAllocator; import org.apache.druid.indexing.common.task.ClientBasedTaskInfoProvider; import org.apache.druid.indexing.common.task.IndexTaskClientFactory; import org.apache.druid.indexing.common.task.InputSourceProcessor; -import org.apache.druid.indexing.common.task.NonLinearlyPartitionedSequenceNameFunction; import org.apache.druid.indexing.common.task.SequenceNameFunction; import org.apache.druid.indexing.common.task.TaskResource; import org.apache.druid.indexing.common.task.Tasks; @@ -129,7 +128,7 @@ public final TaskStatus runTask(TaskToolbox toolbox) throws Exception /** * @return {@link SegmentAllocator} suitable for the desired segment partitioning strategy. */ - abstract CachingSegmentAllocator createSegmentAllocator( + abstract SegmentAllocator createSegmentAllocator( TaskToolbox toolbox, ParallelIndexSupervisorTaskClient taskClient ) throws IOException; @@ -171,11 +170,8 @@ private List generateSegments( final PartitionsSpec partitionsSpec = tuningConfig.getGivenOrDefaultPartitionsSpec(); final long pushTimeout = tuningConfig.getPushTimeout(); - final CachingSegmentAllocator segmentAllocator = createSegmentAllocator(toolbox, taskClient); - final SequenceNameFunction sequenceNameFunction = new NonLinearlyPartitionedSequenceNameFunction( - getId(), - segmentAllocator.getShardSpecs() - ); + final SegmentAllocator segmentAllocator = createSegmentAllocator(toolbox, taskClient); + final SequenceNameFunction sequenceNameFunction = ((CachingLocalSegmentAllocator) segmentAllocator).getSequenceNameFunction(); // TODO final Appenderator appenderator = BatchAppenderators.newAppenderator( getId(), diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/partition/CompletePartitionAnalysis.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/partition/CompletePartitionAnalysis.java index 2e4328043f43..761bd522226f 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/partition/CompletePartitionAnalysis.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/partition/CompletePartitionAnalysis.java @@ -35,9 +35,12 @@ */ public interface CompletePartitionAnalysis extends PartitionAnalysis { - Map> convertToIntervalToSegmentIds( - TaskToolbox toolbox, - String dataSource, - Function versionFinder - ); +// Map> convertToIntervalToSegmentIds( +// TaskToolbox toolbox, +// String dataSource, +// Function versionFinder +// ); + + // TODO: strongly typed + Map createBuckets(TaskToolbox toolbox); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/partition/HashBucket.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/partition/HashBucket.java new file mode 100644 index 000000000000..81bf5c0c6283 --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/partition/HashBucket.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.indexing.common.task.batch.partition; + +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.timeline.partition.BuildingHashBasedNumberedShardSpec; +import org.apache.druid.timeline.partition.ShardSpec; +import org.joda.time.Interval; + +import java.util.List; + +public class HashBucket implements PartitionBucket +{ + private final Interval interval; + private final int bucketId; + private final int numBuckets; + private final List partitionDimensions; + private final ObjectMapper jsonMapper; + + public HashBucket( + Interval interval, + int bucketId, + int numBuckets, + List partitionDimensions, + ObjectMapper jsonMapper + ) + { + this.interval = interval; + this.bucketId = bucketId; + this.numBuckets = numBuckets; + this.partitionDimensions = partitionDimensions; + this.jsonMapper = jsonMapper; + } + + @Override + public Interval getInterval() + { + return interval; + } + + @Override + public int getBucketId() + { + return bucketId; + } + + @Override + public ShardSpec toShardSpec(int partitionId) + { + return new BuildingHashBasedNumberedShardSpec(partitionId, bucketId, numBuckets, partitionDimensions, jsonMapper); + } + + @Override + public String toString() + { + return "HashBucket{" + + "interval=" + interval + + ", bucketId=" + bucketId + + ", numBuckets=" + numBuckets + + ", partitionDimensions=" + partitionDimensions + + '}'; + } +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/partition/HashBucketLookup.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/partition/HashBucketLookup.java new file mode 100644 index 000000000000..5789ef9e6531 --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/partition/HashBucketLookup.java @@ -0,0 +1,77 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.common.task.batch.partition; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Preconditions; +import it.unimi.dsi.fastutil.ints.Int2ObjectMap; +import it.unimi.dsi.fastutil.ints.Int2ObjectOpenHashMap; +import org.apache.druid.data.input.InputRow; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.timeline.partition.HashBasedNumberedShardSpec; + +import java.util.Collection; +import java.util.Iterator; +import java.util.List; + +public class HashBucketLookup implements PartitionBucketLookup +{ + private final ObjectMapper jsonMapper; + private final List partitionDimensions; + private final int numBuckets; + private final Int2ObjectMap bucketIdToBucket; + + public HashBucketLookup( + ObjectMapper jsonMapper, + List partitionDimensions, + Collection buckets + ) + { + this.jsonMapper = jsonMapper; + this.partitionDimensions = partitionDimensions; + this.numBuckets = buckets.size(); + bucketIdToBucket = new Int2ObjectOpenHashMap<>(buckets.size()); + buckets.forEach(bucket -> { + final HashBucket old = bucketIdToBucket.put(bucket.getBucketId(), bucket); + if (old != null) { + throw new ISE("Duplicate bucketId in bucket[%s] and bucket[%s]", bucket, old); + } + }); + } + + @Override + public HashBucket find(long timestamp, InputRow row) + { + final int hash = + Math.abs(HashBasedNumberedShardSpec.hash(jsonMapper, partitionDimensions, timestamp, row) % numBuckets); + return Preconditions.checkNotNull( + bucketIdToBucket.get(hash), + "Cannot find bucket for timestamp[%s] and row[%s]", + timestamp, + row + ); + } + + @Override + public Iterator iterator() + { + return bucketIdToBucket.values().iterator(); + } +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/partition/HashPartitionAnalysis.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/partition/HashPartitionAnalysis.java index f1fb6a7808f1..5b86db121d13 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/partition/HashPartitionAnalysis.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/partition/HashPartitionAnalysis.java @@ -23,8 +23,7 @@ import org.apache.druid.indexer.partitions.HashedPartitionsSpec; import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.java.util.common.IAE; -import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; -import org.apache.druid.timeline.partition.BuildingHashBasedNumberedShardSpec; +import org.apache.druid.utils.CollectionUtils; import org.joda.time.Interval; import java.util.Collections; @@ -33,7 +32,6 @@ import java.util.Map; import java.util.Set; import java.util.function.BiConsumer; -import java.util.function.Function; import java.util.stream.Collectors; import java.util.stream.IntStream; @@ -92,37 +90,68 @@ public void forEach(BiConsumer consumer) } @Override - public Map> convertToIntervalToSegmentIds( - TaskToolbox toolbox, - String dataSource, - Function versionFinder - ) + public Map createBuckets(TaskToolbox toolbox) { - final Map> intervalToSegmentIds = - Maps.newHashMapWithExpectedSize(getNumTimePartitions()); - + final Map intervalToLookup = Maps.newHashMapWithExpectedSize( + intervalToNumBuckets.size() + ); forEach((interval, numBuckets) -> { - intervalToSegmentIds.put( + final List buckets = IntStream + .range(0, numBuckets) + .mapToObj(i -> new HashBucket( + interval, + i, + numBuckets, + partitionsSpec.getPartitionDimensions(), + toolbox.getJsonMapper() + )) + .collect(Collectors.toList()); + intervalToLookup.put( interval, - IntStream.range(0, numBuckets) - .mapToObj(i -> { - final BuildingHashBasedNumberedShardSpec shardSpec = new BuildingHashBasedNumberedShardSpec( - i, - numBuckets, - partitionsSpec.getPartitionDimensions(), - toolbox.getJsonMapper() - ); - return new SegmentIdWithShardSpec( - dataSource, - interval, - versionFinder.apply(interval), - shardSpec - ); - }) - .collect(Collectors.toList()) + new HashBucketLookup( + toolbox.getJsonMapper(), + partitionsSpec.getPartitionDimensions(), + buckets + ) ); }); - - return intervalToSegmentIds; + return intervalToLookup; } + +// @Override +// public Map> convertToIntervalToSegmentIds( +// TaskToolbox toolbox, +// String dataSource, +// Function versionFinder +// ) +// { +// final Map> intervalToSegmentIds = +// Maps.newHashMapWithExpectedSize(getNumTimePartitions()); +// +// forEach((interval, numBuckets) -> { +// intervalToSegmentIds.put( +// interval, +// IntStream.range(0, numBuckets) +// .mapToObj(i -> { +// final HashBucket bucket = new HashBucket( +// i, +// numBuckets, +// partitionsSpec.getPartitionDimensions(), +// toolbox.getJsonMapper() +// ); +// return new SegmentIdWithShardSpec( +// dataSource, +// interval, +// versionFinder.apply(interval), +// shardSpec +// ); +// }) +// .collect(Collectors.toList()) +// ); +// }); +// +// return intervalToSegmentIds; +// } + + } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/partition/PartitionBucket.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/partition/PartitionBucket.java new file mode 100644 index 000000000000..640781f253f7 --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/partition/PartitionBucket.java @@ -0,0 +1,34 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.common.task.batch.partition; + +import org.apache.druid.timeline.partition.ShardSpec; +import org.joda.time.Interval; + +public interface PartitionBucket +{ + Interval getInterval(); + + int getBucketId(); + + ShardSpec toShardSpec(int partitionId); + + // TODO: lookup +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/partition/PartitionBucketLookup.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/partition/PartitionBucketLookup.java new file mode 100644 index 000000000000..1217257f1dad --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/partition/PartitionBucketLookup.java @@ -0,0 +1,37 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.common.task.batch.partition; + +import org.apache.druid.data.input.InputRow; + +import java.util.Iterator; + +public interface PartitionBucketLookup +{ + /** + * Returns a {@link PartitionBucket} for the given timestamp and the inputRow. + * + * @param timestamp timestamp of the row bucketed using {@code GranularitySpec#getQueryGranularity} + * @param row input row + */ + T find(long timestamp, InputRow row); + + Iterator iterator(); +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/partition/PartitionBuckets.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/partition/PartitionBuckets.java new file mode 100644 index 000000000000..c5eb6230e3aa --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/partition/PartitionBuckets.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.indexing.common.task.batch.partition; + +import com.google.common.base.Optional; +import org.apache.druid.data.input.InputRow; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.granularity.Granularity; +import org.apache.druid.segment.indexing.granularity.GranularitySpec; +import org.joda.time.Interval; + +import java.util.Map; + +public class PartitionBuckets +{ + private final GranularitySpec granularitySpec; + private final Granularity queryGranularity; + private final Map intervalToBuckets; + + public PartitionBuckets(GranularitySpec granularitySpec, Map intervalToBuckets) + { + this.granularitySpec = granularitySpec; + this.queryGranularity = granularitySpec.getQueryGranularity(); + this.intervalToBuckets = intervalToBuckets; + } + + public PartitionBucket lookupBucket(InputRow row) + { + final Optional optInterval = granularitySpec.bucketInterval(row.getTimestamp()); + if (!optInterval.isPresent()) { + throw new ISE("Cannot compute a bucketed interval for row[%s]", row); + } + final Interval interval = optInterval.get(); + final PartitionBucketLookup bucketLookup = intervalToBuckets.get(interval); + if (bucketLookup == null) { + throw new ISE("Cannot find partitionBucketLookup for row[%s], bucketed interval[%s]", row, interval); + } + final long bucketedTimestamp = queryGranularity.bucketStart(row.getTimestamp()).getMillis(); + return bucketLookup.find(bucketedTimestamp, row); + } +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/partition/RangeBucket.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/partition/RangeBucket.java new file mode 100644 index 000000000000..c856c58ad222 --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/partition/RangeBucket.java @@ -0,0 +1,109 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.common.task.batch.partition; + +import org.apache.druid.data.input.InputRow; +import org.apache.druid.timeline.partition.BuildingSingleDimensionShardSpec; +import org.apache.druid.timeline.partition.ShardSpec; +import org.joda.time.Interval; + +import javax.annotation.Nullable; +import java.util.List; + +public class RangeBucket implements PartitionBucket +{ + private final Interval interval; + private final int bucketId; + private final String dimension; + @Nullable + private final String start; + @Nullable + private final String end; + + public RangeBucket( + Interval interval, + int bucketId, + String dimension, + @Nullable String start, + @Nullable String end + ) + { + this.interval = interval; + this.bucketId = bucketId; + this.dimension = dimension; + this.start = start; + this.end = end; + } + + @Override + public Interval getInterval() + { + return interval; + } + + @Override + public int getBucketId() + { + return bucketId; + } + + @Override + public ShardSpec toShardSpec(int partitionId) + { + return new BuildingSingleDimensionShardSpec(dimension, start, end, partitionId); + } + + public boolean isInBucket(long timestamp, InputRow row) + { + final List values = row.getDimension(dimension); + + if (values == null || values.size() != 1) { + return checkValue(null); + } else { + return checkValue(values.get(0)); + } + } + + private boolean checkValue(String value) + { + if (value == null) { + return start == null; + } + + if (start == null) { + return end == null || value.compareTo(end) < 0; + } + + return value.compareTo(start) >= 0 && + (end == null || value.compareTo(end) < 0); + } + + @Override + public String toString() + { + return "RangeBucket{" + + "interval=" + interval + + ", bucketId=" + bucketId + + ", dimension='" + dimension + '\'' + + ", start='" + start + '\'' + + ", end='" + end + '\'' + + '}'; + } +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/partition/RangeBucketLookup.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/partition/RangeBucketLookup.java new file mode 100644 index 000000000000..2081a7bc465a --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/partition/RangeBucketLookup.java @@ -0,0 +1,53 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.common.task.batch.partition; + +import org.apache.druid.data.input.InputRow; +import org.apache.druid.java.util.common.ISE; + +import java.util.Iterator; +import java.util.List; + +public class RangeBucketLookup implements PartitionBucketLookup +{ + private final List buckets; + + public RangeBucketLookup(List buckets) + { + this.buckets = buckets; + } + + @Override + public RangeBucket find(long timestamp, InputRow row) + { + for (RangeBucket bucket : buckets) { + if (bucket.isInBucket(timestamp, row)) { + return bucket; + } + } + throw new ISE("row[%s] doesn't fit in any bucket[%s]", row, buckets); + } + + @Override + public Iterator iterator() + { + return buckets.iterator(); + } +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/partition/RangePartitionAnalysis.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/partition/RangePartitionAnalysis.java index b2753931ba1b..6f6efe3503c5 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/partition/RangePartitionAnalysis.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/partition/RangePartitionAnalysis.java @@ -23,19 +23,15 @@ import org.apache.druid.indexer.partitions.SingleDimensionPartitionsSpec; import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.java.util.common.IAE; -import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; import org.apache.druid.timeline.partition.PartitionBoundaries; -import org.apache.druid.timeline.partition.SingleDimensionShardSpec; import org.joda.time.Interval; -import javax.annotation.Nullable; import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Set; import java.util.function.BiConsumer; -import java.util.function.Function; import java.util.stream.Collectors; import java.util.stream.IntStream; @@ -90,44 +86,42 @@ public int getNumTimePartitions() return intervalToPartitionBoundaries.size(); } - @Override - public Map> convertToIntervalToSegmentIds( - TaskToolbox toolbox, - String dataSource, - Function versionFinder - ) - { - final String partitionDimension = partitionsSpec.getPartitionDimension(); - final Map> intervalToSegmentIds = Maps.newHashMapWithExpectedSize( - getNumTimePartitions() - ); - - forEach((interval, partitionBoundaries) -> - intervalToSegmentIds.put( - interval, - translatePartitionBoundaries( - dataSource, - interval, - partitionDimension, - partitionBoundaries, - versionFinder - ) - ) - ); - - return intervalToSegmentIds; - } +// @Override +// public Map> convertToIntervalToSegmentIds( +// TaskToolbox toolbox, +// String dataSource, +// Function versionFinder +// ) +// { +// final String partitionDimension = partitionsSpec.getPartitionDimension(); +// final Map> intervalToSegmentIds = Maps.newHashMapWithExpectedSize( +// getNumTimePartitions() +// ); +// +// forEach((interval, partitionBoundaries) -> +// intervalToSegmentIds.put( +// interval, +// translatePartitionBoundaries( +// dataSource, +// interval, +// partitionDimension, +// partitionBoundaries, +// versionFinder +// ) +// ) +// ); +// +// return intervalToSegmentIds; +// } /** * Translate {@link PartitionBoundaries} into the corresponding * {@link SingleDimensionPartitionsSpec} with segment id. */ - private static List translatePartitionBoundaries( - String dataSource, + private static List translatePartitionBoundaries( Interval interval, String partitionDimension, - PartitionBoundaries partitionBoundaries, - Function versionFinder + PartitionBoundaries partitionBoundaries ) { if (partitionBoundaries.isEmpty()) { @@ -135,40 +129,62 @@ private static List translatePartitionBoundaries( } return IntStream.range(0, partitionBoundaries.size() - 1) - .mapToObj(i -> createSegmentIdWithShardSpec( - dataSource, + .mapToObj(i -> new RangeBucket( interval, - versionFinder.apply(interval), + i, partitionDimension, partitionBoundaries.get(i), - partitionBoundaries.get(i + 1), - i + partitionBoundaries.get(i + 1) )) .collect(Collectors.toList()); } - private static SegmentIdWithShardSpec createSegmentIdWithShardSpec( - String dataSource, - Interval interval, - String version, - String partitionDimension, - String partitionStart, - @Nullable String partitionEnd, - int partitionNum - ) + @Override + public Map createBuckets(TaskToolbox toolbox) { - // The shardSpec created here will be reused in PartialGenericSegmentMergeTask. This is ok because - // all PartialSegmentGenerateTasks create the same set of segmentIds (and thus shardSpecs). - return new SegmentIdWithShardSpec( - dataSource, - interval, - version, - new SingleDimensionShardSpec( - partitionDimension, - partitionStart, - partitionEnd, - partitionNum - ) + final String partitionDimension = partitionsSpec.getPartitionDimension(); + final Map intervalToSegmentIds = Maps.newHashMapWithExpectedSize( + getNumTimePartitions() ); + + forEach((interval, partitionBoundaries) -> + intervalToSegmentIds.put( + interval, + new RangeBucketLookup( + translatePartitionBoundaries( + interval, + partitionDimension, + partitionBoundaries + ) + ) + ) + ); + + return intervalToSegmentIds; } + +// private static SegmentIdWithShardSpec createSegmentIdWithShardSpec( +// String dataSource, +// Interval interval, +// String version, +// String partitionDimension, +// String partitionStart, +// @Nullable String partitionEnd, +// int partitionNum +// ) +// { +// // The shardSpec created here will be reused in PartialGenericSegmentMergeTask. This is ok because +// // all PartialSegmentGenerateTasks create the same set of segmentIds (and thus shardSpecs). +// return new SegmentIdWithShardSpec( +// dataSource, +// interval, +// version, +// new SingleDimensionShardSpec( +// partitionDimension, +// partitionStart, +// partitionEnd, +// partitionNum +// ) +// ); +// } } 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 0ed6472fe30f..ce1e1fc1fccd 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 @@ -880,14 +880,14 @@ public void testCannotAddToExistingSingleDimensionShardSpecs() throws Exception .dataSource(DATA_SOURCE) .interval(Granularities.HOUR.bucket(PARTY_TIME)) .version(PARTY_TIME.toString()) - .shardSpec(new SingleDimensionShardSpec("foo", null, "bar", 0)) + .shardSpec(new SingleDimensionShardSpec("foo", null, "bar", 0, 2)) .size(0) .build(), DataSegment.builder() .dataSource(DATA_SOURCE) .interval(Granularities.HOUR.bucket(PARTY_TIME)) .version(PARTY_TIME.toString()) - .shardSpec(new SingleDimensionShardSpec("foo", "bar", null, 1)) + .shardSpec(new SingleDimensionShardSpec("foo", "bar", null, 1, 2)) .size(0) .build() ) @@ -914,14 +914,14 @@ public void testWithPartialShardSpecAndOvershadowingSegments() throws IOExceptio .dataSource(DATA_SOURCE) .interval(Granularities.HOUR.bucket(PARTY_TIME)) .version(PARTY_TIME.toString()) - .shardSpec(new HashBasedNumberedShardSpec(0, 2, 2, ImmutableList.of("dim1"), objectMapper)) + .shardSpec(new HashBasedNumberedShardSpec(0, 2, 0, 2, ImmutableList.of("dim1"), objectMapper)) .size(0) .build(), DataSegment.builder() .dataSource(DATA_SOURCE) .interval(Granularities.HOUR.bucket(PARTY_TIME)) .version(PARTY_TIME.toString()) - .shardSpec(new HashBasedNumberedShardSpec(1, 2, 2, ImmutableList.of("dim1"), objectMapper)) + .shardSpec(new HashBasedNumberedShardSpec(1, 2, 1, 2, ImmutableList.of("dim1"), objectMapper)) .size(0) .build() ) @@ -935,7 +935,7 @@ public void testWithPartialShardSpecAndOvershadowingSegments() throws IOExceptio "seq", null, true, - new HashBasedNumberedPartialShardSpec(ImmutableList.of("dim1"), 2), + new HashBasedNumberedPartialShardSpec(ImmutableList.of("dim1"), 1, 2), lockGranularity ); final SegmentIdWithShardSpec segmentIdentifier = action.perform(task, taskActionTestKit.getTaskActionToolbox()); @@ -946,7 +946,7 @@ public void testWithPartialShardSpecAndOvershadowingSegments() throws IOExceptio Assert.assertTrue(shardSpec instanceof HashBasedNumberedShardSpec); final HashBasedNumberedShardSpec hashBasedNumberedShardSpec = (HashBasedNumberedShardSpec) shardSpec; - Assert.assertEquals(2, hashBasedNumberedShardSpec.getPartitions()); + Assert.assertEquals(2, hashBasedNumberedShardSpec.getNumCorePartitions()); Assert.assertEquals(ImmutableList.of("dim1"), hashBasedNumberedShardSpec.getPartitionDimensions()); } @@ -1029,10 +1029,7 @@ private void assertSameIdentifier(final SegmentIdWithShardSpec expected, final S if (expected.getShardSpec().getClass() == NumberedShardSpec.class && actual.getShardSpec().getClass() == NumberedShardSpec.class) { - Assert.assertEquals( - ((NumberedShardSpec) expected.getShardSpec()).getPartitions(), - ((NumberedShardSpec) actual.getShardSpec()).getPartitions() - ); + Assert.assertEquals(expected.getShardSpec().getNumCorePartitions(), actual.getShardSpec().getNumCorePartitions()); } else if (expected.getShardSpec().getClass() == LinearShardSpec.class && actual.getShardSpec().getClass() == LinearShardSpec.class) { // do nothing 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 6e3fcf54dcf2..513c13c56e44 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 @@ -239,13 +239,13 @@ public void testDeterminePartitions() throws Exception Assert.assertEquals(Intervals.of("2014/P1D"), segments.get(0).getInterval()); Assert.assertEquals(HashBasedNumberedShardSpec.class, segments.get(0).getShardSpec().getClass()); Assert.assertEquals(0, segments.get(0).getShardSpec().getPartitionNum()); - Assert.assertEquals(2, ((NumberedShardSpec) segments.get(0).getShardSpec()).getPartitions()); + Assert.assertEquals(2, ((NumberedShardSpec) segments.get(0).getShardSpec()).getNumCorePartitions()); Assert.assertEquals("test", segments.get(1).getDataSource()); Assert.assertEquals(Intervals.of("2014/P1D"), segments.get(1).getInterval()); Assert.assertEquals(HashBasedNumberedShardSpec.class, segments.get(1).getShardSpec().getClass()); Assert.assertEquals(1, segments.get(1).getShardSpec().getPartitionNum()); - Assert.assertEquals(2, ((NumberedShardSpec) segments.get(1).getShardSpec()).getPartitions()); + Assert.assertEquals(2, ((NumberedShardSpec) segments.get(1).getShardSpec()).getNumCorePartitions()); } @Test diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/RangePartitionCachingLocalSegmentAllocatorTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/RangePartitionCachingLocalSegmentAllocatorTest.java index 12a16152fcb1..6cda8addccc0 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/RangePartitionCachingLocalSegmentAllocatorTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/RangePartitionCachingLocalSegmentAllocatorTest.java @@ -19,6 +19,7 @@ package org.apache.druid.indexing.common.task; +import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import org.apache.druid.data.input.InputRow; import org.apache.druid.indexer.partitions.SingleDimensionPartitionsSpec; @@ -30,7 +31,9 @@ import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.StringUtils; -import org.apache.druid.java.util.common.granularity.NoneGranularity; +import org.apache.druid.java.util.common.granularity.Granularities; +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.timeline.SegmentId; import org.apache.druid.timeline.partition.PartitionBoundaries; @@ -82,7 +85,7 @@ public class RangePartitionCachingLocalSegmentAllocatorTest INTERVAL_NORMAL, NORMAL_PARTITIONS ); - private CachingSegmentAllocator target; + private SegmentAllocator target; private SequenceNameFunction sequenceNameFunction; @Rule @@ -105,11 +108,11 @@ public void setup() throws IOException toolbox, DATASOURCE, TASKID, - new NoneGranularity(), + new UniformGranularitySpec(Granularities.HOUR, Granularities.NONE, ImmutableList.of()), new SupervisorTaskAccessWithNullClient(SUPERVISOR_TASKID), partitionAnalysis ); - sequenceNameFunction = new NonLinearlyPartitionedSequenceNameFunction(TASKID, target.getShardSpecs()); + sequenceNameFunction = ((CachingLocalSegmentAllocator) target).getSequenceNameFunction(); } @Test diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ShardSpecsTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ShardSpecsTest.java index 30cfa0fa067d..6755c8abaad5 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ShardSpecsTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ShardSpecsTest.java @@ -1,87 +1,87 @@ -/* - * 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.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.collect.ImmutableMap; -import com.google.common.collect.Lists; -import org.apache.druid.data.input.InputRow; -import org.apache.druid.data.input.MapBasedInputRow; -import org.apache.druid.indexing.common.TestUtils; -import org.apache.druid.java.util.common.DateTimes; -import org.apache.druid.java.util.common.Intervals; -import org.apache.druid.java.util.common.granularity.Granularities; -import org.apache.druid.timeline.partition.HashBasedNumberedShardSpec; -import org.apache.druid.timeline.partition.ShardSpec; -import org.joda.time.Interval; -import org.junit.Assert; -import org.junit.Test; - -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -public class ShardSpecsTest extends IngestionTestBase -{ - private final TestUtils testUtils = new TestUtils(); - private final ObjectMapper jsonMapper = testUtils.getTestObjectMapper(); - - public ShardSpecsTest() - { - } - - @Test - public void testShardSpecSelectionWithNullPartitionDimension() - { - ShardSpec spec1 = new HashBasedNumberedShardSpec(0, 2, 2, null, jsonMapper); - ShardSpec spec2 = new HashBasedNumberedShardSpec(1, 2, 2, null, jsonMapper); - - Map> shardSpecMap = new HashMap<>(); - shardSpecMap.put(Intervals.of("2014-01-01T00:00:00.000Z/2014-01-02T00:00:00.000Z"), Lists.newArrayList(spec1, spec2)); - - ShardSpecs shardSpecs = new ShardSpecs(shardSpecMap, Granularities.HOUR); - String visitorId = "visitorId"; - String clientType = "clientType"; - long timestamp1 = DateTimes.of("2014-01-01T00:00:00.000Z").getMillis(); - InputRow row1 = new MapBasedInputRow(timestamp1, - Lists.newArrayList(visitorId, clientType), - ImmutableMap.of(visitorId, "0", clientType, "iphone") - ); - - long timestamp2 = DateTimes.of("2014-01-01T00:30:20.456Z").getMillis(); - InputRow row2 = new MapBasedInputRow(timestamp2, - Lists.newArrayList(visitorId, clientType), - ImmutableMap.of(visitorId, "0", clientType, "iphone") - ); - - long timestamp3 = DateTimes.of("2014-01-01T10:10:20.456Z").getMillis(); - InputRow row3 = new MapBasedInputRow(timestamp3, - Lists.newArrayList(visitorId, clientType), - ImmutableMap.of(visitorId, "0", clientType, "iphone") - ); - - ShardSpec spec3 = shardSpecs.getShardSpec(Intervals.of("2014-01-01T00:00:00.000Z/2014-01-02T00:00:00.000Z"), row1); - ShardSpec spec4 = shardSpecs.getShardSpec(Intervals.of("2014-01-01T00:00:00.000Z/2014-01-02T00:00:00.000Z"), row2); - ShardSpec spec5 = shardSpecs.getShardSpec(Intervals.of("2014-01-01T00:00:00.000Z/2014-01-02T00:00:00.000Z"), row3); - - Assert.assertSame(true, spec3 == spec4); - Assert.assertSame(false, spec3 == spec5); - } -} +///* +// * 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.fasterxml.jackson.databind.ObjectMapper; +//import com.google.common.collect.ImmutableMap; +//import com.google.common.collect.Lists; +//import org.apache.druid.data.input.InputRow; +//import org.apache.druid.data.input.MapBasedInputRow; +//import org.apache.druid.indexing.common.TestUtils; +//import org.apache.druid.java.util.common.DateTimes; +//import org.apache.druid.java.util.common.Intervals; +//import org.apache.druid.java.util.common.granularity.Granularities; +//import org.apache.druid.timeline.partition.HashBasedNumberedShardSpec; +//import org.apache.druid.timeline.partition.ShardSpec; +//import org.joda.time.Interval; +//import org.junit.Assert; +//import org.junit.Test; +// +//import java.util.HashMap; +//import java.util.List; +//import java.util.Map; +// +//public class ShardSpecsTest extends IngestionTestBase +//{ +// private final TestUtils testUtils = new TestUtils(); +// private final ObjectMapper jsonMapper = testUtils.getTestObjectMapper(); +// +// public ShardSpecsTest() +// { +// } +// +// @Test +// public void testShardSpecSelectionWithNullPartitionDimension() +// { +// ShardSpec spec1 = new HashBasedNumberedShardSpec(0, 2, 2, null, jsonMapper); +// ShardSpec spec2 = new HashBasedNumberedShardSpec(1, 2, 2, null, jsonMapper); +// +// Map> shardSpecMap = new HashMap<>(); +// shardSpecMap.put(Intervals.of("2014-01-01T00:00:00.000Z/2014-01-02T00:00:00.000Z"), Lists.newArrayList(spec1, spec2)); +// +// ShardSpecs shardSpecs = new ShardSpecs(shardSpecMap, Granularities.HOUR); +// String visitorId = "visitorId"; +// String clientType = "clientType"; +// long timestamp1 = DateTimes.of("2014-01-01T00:00:00.000Z").getMillis(); +// InputRow row1 = new MapBasedInputRow(timestamp1, +// Lists.newArrayList(visitorId, clientType), +// ImmutableMap.of(visitorId, "0", clientType, "iphone") +// ); +// +// long timestamp2 = DateTimes.of("2014-01-01T00:30:20.456Z").getMillis(); +// InputRow row2 = new MapBasedInputRow(timestamp2, +// Lists.newArrayList(visitorId, clientType), +// ImmutableMap.of(visitorId, "0", clientType, "iphone") +// ); +// +// long timestamp3 = DateTimes.of("2014-01-01T10:10:20.456Z").getMillis(); +// InputRow row3 = new MapBasedInputRow(timestamp3, +// Lists.newArrayList(visitorId, clientType), +// ImmutableMap.of(visitorId, "0", clientType, "iphone") +// ); +// +// ShardSpec spec3 = shardSpecs.getShardSpec(Intervals.of("2014-01-01T00:00:00.000Z/2014-01-02T00:00:00.000Z"), row1); +// ShardSpec spec4 = shardSpecs.getShardSpec(Intervals.of("2014-01-01T00:00:00.000Z/2014-01-02T00:00:00.000Z"), row2); +// ShardSpec spec5 = shardSpecs.getShardSpec(Intervals.of("2014-01-01T00:00:00.000Z/2014-01-02T00:00:00.000Z"), row3); +// +// Assert.assertSame(true, spec3 == spec4); +// Assert.assertSame(false, spec3 == spec5); +// } +//} diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionAdjustingCorePartitionSizeTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionAdjustingCorePartitionSizeTest.java index 8f1fb67449ea..ed9ffa22260d 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionAdjustingCorePartitionSizeTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionAdjustingCorePartitionSizeTest.java @@ -111,7 +111,7 @@ public void testLessPartitionsThanBuckets() throws IOException segments.forEach(segment -> { Assert.assertSame(HashBasedNumberedShardSpec.class, segment.getShardSpec().getClass()); final HashBasedNumberedShardSpec shardSpec = (HashBasedNumberedShardSpec) segment.getShardSpec(); - Assert.assertEquals(3, shardSpec.getPartitions()); + Assert.assertEquals(3, shardSpec.getNumCorePartitions()); Assert.assertEquals(10, shardSpec.getNumBuckets()); Assert.assertEquals(ImmutableList.of("dim1"), shardSpec.getPartitionDimensions()); }); @@ -148,7 +148,7 @@ public void testEqualNumberOfPartitionsToBuckets() throws IOException segments.forEach(segment -> { Assert.assertSame(HashBasedNumberedShardSpec.class, segment.getShardSpec().getClass()); final HashBasedNumberedShardSpec shardSpec = (HashBasedNumberedShardSpec) segment.getShardSpec(); - Assert.assertEquals(5, shardSpec.getPartitions()); + Assert.assertEquals(5, shardSpec.getNumCorePartitions()); Assert.assertEquals(5, shardSpec.getNumBuckets()); Assert.assertEquals(ImmutableList.of("dim1"), shardSpec.getPartitionDimensions()); }); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionCachingLocalSegmentAllocatorTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionCachingLocalSegmentAllocatorTest.java index cea8589c51d6..0e3fe2e7843c 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionCachingLocalSegmentAllocatorTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionCachingLocalSegmentAllocatorTest.java @@ -29,15 +29,16 @@ import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.common.actions.LockListAction; import org.apache.druid.indexing.common.actions.TaskActionClient; -import org.apache.druid.indexing.common.task.CachingSegmentAllocator; -import org.apache.druid.indexing.common.task.NonLinearlyPartitionedSequenceNameFunction; +import org.apache.druid.indexing.common.task.CachingLocalSegmentAllocator; import org.apache.druid.indexing.common.task.SegmentAllocators; import org.apache.druid.indexing.common.task.SequenceNameFunction; import org.apache.druid.indexing.common.task.SupervisorTaskAccessWithNullClient; import org.apache.druid.indexing.common.task.batch.partition.HashPartitionAnalysis; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.StringUtils; -import org.apache.druid.java.util.common.granularity.NoneGranularity; +import org.apache.druid.java.util.common.granularity.Granularities; +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.timeline.SegmentId; import org.apache.druid.timeline.partition.BuildingHashBasedNumberedShardSpec; @@ -70,7 +71,7 @@ public class HashPartitionCachingLocalSegmentAllocatorTest Collections.singletonList(DIMENSION) ); - private CachingSegmentAllocator target; + private SegmentAllocator target; private SequenceNameFunction sequenceNameFunction; @Before @@ -83,11 +84,11 @@ public void setup() throws IOException toolbox, DATASOURCE, TASKID, - new NoneGranularity(), + new UniformGranularitySpec(Granularities.HOUR, Granularities.NONE, ImmutableList.of()), new SupervisorTaskAccessWithNullClient(SUPERVISOR_TASKID), partitionAnalysis ); - sequenceNameFunction = new NonLinearlyPartitionedSequenceNameFunction(TASKID, target.getShardSpecs()); + sequenceNameFunction = ((CachingLocalSegmentAllocator) target).getSequenceNameFunction(); } @Test diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexTestingFactory.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexTestingFactory.java index e64235e48869..39136ccc1057 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexTestingFactory.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexTestingFactory.java @@ -103,6 +103,7 @@ public > File fetchSegmentFile( static final HashBasedNumberedShardSpec HASH_BASED_NUMBERED_SHARD_SPEC = new HashBasedNumberedShardSpec( PARTITION_ID, PARTITION_ID + 1, + PARTITION_ID, PARTITION_ID + 1, Collections.singletonList("dim"), ParallelIndexTestingFactory.NESTED_OBJECT_MAPPER diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexingTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexingTest.java index e1e21a56719d..4abb539f333c 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexingTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexingTest.java @@ -221,14 +221,14 @@ private void assertShardSpec(@Nullable Interval interval, boolean appendToExisti for (DataSegment segment : segmentsPerInterval) { Assert.assertSame(NumberedShardSpec.class, segment.getShardSpec().getClass()); final NumberedShardSpec shardSpec = (NumberedShardSpec) segment.getShardSpec(); - Assert.assertEquals(segmentsPerInterval.size(), shardSpec.getPartitions()); + Assert.assertEquals(segmentsPerInterval.size(), shardSpec.getNumCorePartitions()); } } } else { for (DataSegment segment : segments) { Assert.assertSame(NumberedShardSpec.class, segment.getShardSpec().getClass()); final NumberedShardSpec shardSpec = (NumberedShardSpec) segment.getShardSpec(); - Assert.assertEquals(0, shardSpec.getPartitions()); + Assert.assertEquals(0, shardSpec.getNumCorePartitions()); } } } @@ -248,7 +248,7 @@ private void assertShardSpecAfterOverwrite(@Nullable Interval interval, LockGran for (DataSegment segment : segmentsPerInterval) { Assert.assertSame(NumberedShardSpec.class, segment.getShardSpec().getClass()); final NumberedShardSpec shardSpec = (NumberedShardSpec) segment.getShardSpec(); - Assert.assertEquals(segmentsPerInterval.size(), shardSpec.getPartitions()); + Assert.assertEquals(segmentsPerInterval.size(), shardSpec.getNumCorePartitions()); } } } else { 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 00f76a48952a..ae8fc3b64fe1 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 @@ -971,8 +971,8 @@ public void testRequestForNewSegmentWithHashPartition() final Task task = NoopTask.create(); lockbox.add(task); - allocateSegmentsAndAssert(task, "seq", 3, new HashBasedNumberedPartialShardSpec(null, 3)); - allocateSegmentsAndAssert(task, "seq2", 5, new HashBasedNumberedPartialShardSpec(null, 5)); + allocateSegmentsAndAssert(task, "seq", 3, new HashBasedNumberedPartialShardSpec(null, 1, 3)); + allocateSegmentsAndAssert(task, "seq2", 5, new HashBasedNumberedPartialShardSpec(null, 3, 5)); } private void allocateSegmentsAndAssert( 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 ed6ececbeb86..fd479c1adaa5 100644 --- a/server/src/test/java/org/apache/druid/client/CachingClusteredClientTest.java +++ b/server/src/test/java/org/apache/druid/client/CachingClusteredClientTest.java @@ -1538,7 +1538,7 @@ private ServerSelector makeMockSingleDimensionSelector( null, null, null, - new SingleDimensionShardSpec(dimension, start, end, partitionNum), + new SingleDimensionShardSpec(dimension, start, end, partitionNum, SingleDimensionShardSpec.UNKNOWN_NUM_CORE_PARTITIONS), // TODO null, 9, 0L @@ -1966,7 +1966,7 @@ private List> populateTimeline( final ShardSpec shardSpec; if (numChunks == 1) { - shardSpec = new SingleDimensionShardSpec("dimAll", null, null, 0); + shardSpec = new SingleDimensionShardSpec("dimAll", null, null, 0, 1); } else { String start = null; String end = null; @@ -1976,7 +1976,7 @@ private List> populateTimeline( if (j + 1 < numChunks) { end = String.valueOf(j + 1); } - shardSpec = new SingleDimensionShardSpec("dim" + k, start, end, j); + shardSpec = new SingleDimensionShardSpec("dim" + k, start, end, j, numChunks); } DataSegment mockSegment = makeMock(mocks, DataSegment.class); ServerExpectation expectation = new ServerExpectation<>( 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 af6eae551982..bd8e5ef3a8b0 100644 --- a/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java +++ b/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java @@ -1032,7 +1032,7 @@ public void testAllocatePendingSegmentsWithOvershadowingSegments() throws IOExce @Test public void testAllocatePendingSegmentsForHashBasedNumberedShardSpec() throws IOException { - final PartialShardSpec partialShardSpec = new HashBasedNumberedPartialShardSpec(null, 5); + final PartialShardSpec partialShardSpec = new HashBasedNumberedPartialShardSpec(null, 2, 5); final String dataSource = "ds"; final Interval interval = Intervals.of("2017-01-01/2017-02-01"); @@ -1048,7 +1048,7 @@ public void testAllocatePendingSegmentsForHashBasedNumberedShardSpec() throws IO HashBasedNumberedShardSpec shardSpec = (HashBasedNumberedShardSpec) id.getShardSpec(); Assert.assertEquals(0, shardSpec.getPartitionNum()); - Assert.assertEquals(0, shardSpec.getPartitions()); + Assert.assertEquals(0, shardSpec.getNumCorePartitions()); Assert.assertEquals(5, shardSpec.getNumBuckets()); coordinator.announceHistoricalSegments( @@ -1079,7 +1079,7 @@ public void testAllocatePendingSegmentsForHashBasedNumberedShardSpec() throws IO shardSpec = (HashBasedNumberedShardSpec) id.getShardSpec(); Assert.assertEquals(1, shardSpec.getPartitionNum()); - Assert.assertEquals(0, shardSpec.getPartitions()); + Assert.assertEquals(0, shardSpec.getNumCorePartitions()); Assert.assertEquals(5, shardSpec.getNumBuckets()); coordinator.announceHistoricalSegments( @@ -1103,14 +1103,14 @@ public void testAllocatePendingSegmentsForHashBasedNumberedShardSpec() throws IO "seq3", null, interval, - new HashBasedNumberedPartialShardSpec(null, 3), + new HashBasedNumberedPartialShardSpec(null, 2, 3), "version", true ); shardSpec = (HashBasedNumberedShardSpec) id.getShardSpec(); Assert.assertEquals(2, shardSpec.getPartitionNum()); - Assert.assertEquals(0, shardSpec.getPartitions()); + Assert.assertEquals(0, shardSpec.getNumCorePartitions()); Assert.assertEquals(3, shardSpec.getNumBuckets()); } } diff --git a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/SegmentIdWithShardSpecTest.java b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/SegmentIdWithShardSpecTest.java index 55ff5892fd76..558736980a85 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/SegmentIdWithShardSpecTest.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/SegmentIdWithShardSpecTest.java @@ -53,7 +53,7 @@ public void testSerde() throws Exception Assert.assertEquals(INTERVAL, id2.getInterval()); Assert.assertEquals(VERSION, id2.getVersion()); Assert.assertEquals(SHARD_SPEC_1.getPartitionNum(), id2.getShardSpec().getPartitionNum()); - Assert.assertEquals(SHARD_SPEC_1.getPartitions(), ((NumberedShardSpec) id2.getShardSpec()).getPartitions()); + Assert.assertEquals(SHARD_SPEC_1.getNumCorePartitions(), ((NumberedShardSpec) id2.getShardSpec()).getNumCorePartitions()); } @Test 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 99f8fa4f4ff8..b00228d4f4ae 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 @@ -60,7 +60,7 @@ public void testSerdeRoundTrip() throws Exception ShardSpec.class ); Assert.assertEquals(1, spec.getPartitionNum()); - Assert.assertEquals(2, ((NumberedShardSpec) spec).getPartitions()); + Assert.assertEquals(2, ((NumberedShardSpec) spec).getNumCorePartitions()); } @Test @@ -71,7 +71,7 @@ public void testSerdeBackwardsCompat() throws Exception ShardSpec.class ); Assert.assertEquals(1, spec.getPartitionNum()); - Assert.assertEquals(2, ((NumberedShardSpec) spec).getPartitions()); + Assert.assertEquals(2, ((NumberedShardSpec) spec).getNumCorePartitions()); } @Test diff --git a/server/src/test/java/org/apache/druid/server/shard/SingleDimensionShardSpecTest.java b/server/src/test/java/org/apache/druid/server/shard/SingleDimensionShardSpecTest.java index 2722179973e7..0787cf2c119c 100644 --- a/server/src/test/java/org/apache/druid/server/shard/SingleDimensionShardSpecTest.java +++ b/server/src/test/java/org/apache/druid/server/shard/SingleDimensionShardSpecTest.java @@ -158,7 +158,7 @@ private SingleDimensionShardSpec makeSpec(String start, String end) private SingleDimensionShardSpec makeSpec(String dimension, String start, String end) { - return new SingleDimensionShardSpec(dimension, start, end, 0); + return new SingleDimensionShardSpec(dimension, start, end, 0, SingleDimensionShardSpec.UNKNOWN_NUM_CORE_PARTITIONS); } private Map makeMap(String value) 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 50cf0f2a2fbd..ecb110237498 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 @@ -62,6 +62,7 @@ public void testSerdeRoundTrip() throws Exception new HashBasedNumberedShardSpec( 1, 2, + 1, 3, ImmutableList.of("visitor_id"), ServerTestHelper.MAPPER @@ -70,7 +71,8 @@ public void testSerdeRoundTrip() throws Exception ShardSpec.class ); Assert.assertEquals(1, spec.getPartitionNum()); - Assert.assertEquals(2, ((HashBasedNumberedShardSpec) spec).getPartitions()); + Assert.assertEquals(2, spec.getNumCorePartitions()); + Assert.assertEquals(1, ((HashBasedNumberedShardSpec) spec).getBucketId()); Assert.assertEquals(3, ((HashBasedNumberedShardSpec) spec).getNumBuckets()); Assert.assertEquals(ImmutableList.of("visitor_id"), ((HashBasedNumberedShardSpec) spec).getPartitionDimensions()); } @@ -83,14 +85,14 @@ public void testSerdeBackwardsCompat() throws Exception ShardSpec.class ); Assert.assertEquals(1, spec.getPartitionNum()); - Assert.assertEquals(2, ((HashBasedNumberedShardSpec) spec).getPartitions()); + Assert.assertEquals(2, ((HashBasedNumberedShardSpec) spec).getNumCorePartitions()); final ShardSpec specWithPartitionDimensions = ServerTestHelper.MAPPER.readValue( "{\"type\": \"hashed\", \"partitions\": 2, \"partitionNum\": 1, \"partitionDimensions\":[\"visitor_id\"]}", ShardSpec.class ); Assert.assertEquals(1, specWithPartitionDimensions.getPartitionNum()); - Assert.assertEquals(2, ((HashBasedNumberedShardSpec) specWithPartitionDimensions).getPartitions()); + Assert.assertEquals(2, ((HashBasedNumberedShardSpec) specWithPartitionDimensions).getNumCorePartitions()); Assert.assertEquals(2, ((HashBasedNumberedShardSpec) specWithPartitionDimensions).getNumBuckets()); Assert.assertEquals( ImmutableList.of("visitor_id"), @@ -102,9 +104,9 @@ public void testSerdeBackwardsCompat() throws Exception public void testPartitionChunks() { final List specs = ImmutableList.of( - new HashBasedNumberedShardSpec(0, 3, 3, null, ServerTestHelper.MAPPER), - new HashBasedNumberedShardSpec(1, 3, 3, null, ServerTestHelper.MAPPER), - new HashBasedNumberedShardSpec(2, 3, 3, null, ServerTestHelper.MAPPER) + new HashBasedNumberedShardSpec(0, 3, 0, 3, null, ServerTestHelper.MAPPER), + new HashBasedNumberedShardSpec(1, 3, 1, 3, null, ServerTestHelper.MAPPER), + new HashBasedNumberedShardSpec(2, 3, 2, 3, null, ServerTestHelper.MAPPER) ); final List> chunks = Lists.transform( @@ -211,7 +213,7 @@ public static class HashOverridenShardSpec extends HashBasedNumberedShardSpec { public HashOverridenShardSpec(int partitionNum, int partitions) { - super(partitionNum, partitions, partitions, null, ServerTestHelper.MAPPER); + super(partitionNum, partitions, partitionNum, partitions, null, ServerTestHelper.MAPPER); } @Override From f2c9bf3631eb39eb755a300385d12e2afdc2eef0 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Sat, 13 Jun 2020 08:47:24 -0700 Subject: [PATCH 13/24] some test adjustment and missing json serde --- .../BuildingSingleDimensionShardSpec.java | 36 +++++++++++++++++++ .../druid/timeline/partition/ShardSpec.java | 3 +- .../task/CachingLocalSegmentAllocator.java | 5 +-- .../partition/RangePartitionAnalysis.java | 28 --------------- ...rtitionMultiPhaseParallelIndexingTest.java | 21 +++++++---- 5 files changed, 55 insertions(+), 38 deletions(-) diff --git a/core/src/main/java/org/apache/druid/timeline/partition/BuildingSingleDimensionShardSpec.java b/core/src/main/java/org/apache/druid/timeline/partition/BuildingSingleDimensionShardSpec.java index 6f9560a4a1d1..dfcde031c549 100644 --- a/core/src/main/java/org/apache/druid/timeline/partition/BuildingSingleDimensionShardSpec.java +++ b/core/src/main/java/org/apache/druid/timeline/partition/BuildingSingleDimensionShardSpec.java @@ -23,9 +23,12 @@ import com.fasterxml.jackson.annotation.JsonProperty; import javax.annotation.Nullable; +import java.util.Objects; public class BuildingSingleDimensionShardSpec implements BuildingShardSpec { + public static final String TYPE = "building_single_dim"; + private final String dimension; @Nullable private final String start; @@ -85,4 +88,37 @@ public PartitionChunk createChunk(T obj) { return new NumberedPartitionChunk<>(partitionNum, 0, obj); } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + BuildingSingleDimensionShardSpec that = (BuildingSingleDimensionShardSpec) o; + return partitionNum == that.partitionNum && + Objects.equals(dimension, that.dimension) && + Objects.equals(start, that.start) && + Objects.equals(end, that.end); + } + + @Override + public int hashCode() + { + return Objects.hash(dimension, start, end, partitionNum); + } + + @Override + public String toString() + { + return "BuildingSingleDimensionShardSpec{" + + "dimension='" + dimension + '\'' + + ", start='" + start + '\'' + + ", end='" + end + '\'' + + ", partitionNum=" + partitionNum + + '}'; + } } 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 af78e3dcdd2f..63f49b710a4d 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 @@ -41,7 +41,8 @@ @JsonSubTypes.Type(name = "hashed", value = HashBasedNumberedShardSpec.class), @JsonSubTypes.Type(name = NumberedOverwriteShardSpec.TYPE, value = NumberedOverwriteShardSpec.class), @JsonSubTypes.Type(name = BuildingNumberedShardSpec.TYPE, value = BuildingNumberedShardSpec.class), - @JsonSubTypes.Type(name = BuildingHashBasedNumberedShardSpec.TYPE, value = BuildingHashBasedNumberedShardSpec.class) + @JsonSubTypes.Type(name = BuildingHashBasedNumberedShardSpec.TYPE, value = BuildingHashBasedNumberedShardSpec.class), + @JsonSubTypes.Type(name = BuildingSingleDimensionShardSpec.TYPE, value = BuildingSingleDimensionShardSpec.class) }) public interface ShardSpec { 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 01ecc0855be3..0e01a36f88d6 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 @@ -88,7 +88,6 @@ public class CachingLocalSegmentAllocator implements SegmentAllocator this.versionFinder = interval -> findVersion(intervalToVersion, interval); final Map intervalToBucketLookup = partitionAnalysis.createBuckets(toolbox); intervalToNextPartitionId = new Object2IntOpenHashMap<>(intervalToBucketLookup.size()); - intervalToNextPartitionId.defaultReturnValue(0); sequenceNameFunction = new NonLinearlyPartitionedSequenceNameFunction( taskId, @@ -103,6 +102,7 @@ public class CachingLocalSegmentAllocator implements SegmentAllocator // The shardSpecs for partitioning and publishing can be different if isExtendableShardSpecs = true. sequenceNameToBucket.put(sequenceNameFunction.getSequenceName(interval, bucket), bucket); }); + intervalToNextPartitionId.put(interval, 0); } } @@ -147,7 +147,8 @@ private int getPartitionIdAndIncrement(Interval interval) { return intervalToNextPartitionId.computeInt( interval, - (i, nextPartitionId) -> nextPartitionId == null ? 0 : nextPartitionId + 1 + (i, nextPartitionId) -> + Preconditions.checkNotNull(nextPartitionId, "nextPartitionId for interval[%s]", interval) + 1 ); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/partition/RangePartitionAnalysis.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/partition/RangePartitionAnalysis.java index 6f6efe3503c5..031d15565684 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/partition/RangePartitionAnalysis.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/partition/RangePartitionAnalysis.java @@ -86,34 +86,6 @@ public int getNumTimePartitions() return intervalToPartitionBoundaries.size(); } -// @Override -// public Map> convertToIntervalToSegmentIds( -// TaskToolbox toolbox, -// String dataSource, -// Function versionFinder -// ) -// { -// final String partitionDimension = partitionsSpec.getPartitionDimension(); -// final Map> intervalToSegmentIds = Maps.newHashMapWithExpectedSize( -// getNumTimePartitions() -// ); -// -// forEach((interval, partitionBoundaries) -> -// intervalToSegmentIds.put( -// interval, -// translatePartitionBoundaries( -// dataSource, -// interval, -// partitionDimension, -// partitionBoundaries, -// versionFinder -// ) -// ) -// ); -// -// return intervalToSegmentIds; -// } - /** * Translate {@link PartitionBoundaries} into the corresponding * {@link SingleDimensionPartitionsSpec} with segment id. diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionMultiPhaseParallelIndexingTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionMultiPhaseParallelIndexingTest.java index bcd3cbe85842..a7e892d09310 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionMultiPhaseParallelIndexingTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionMultiPhaseParallelIndexingTest.java @@ -75,24 +75,31 @@ public class HashPartitionMultiPhaseParallelIndexingTest extends AbstractMultiPh false, 0 ); - private static final int MAX_NUM_CONCURRENT_SUB_TASKS = 2; private static final Interval INTERVAL_TO_INDEX = Intervals.of("2017-12/P1M"); @Parameterized.Parameters(name = "{0}, useInputFormatApi={1}") public static Iterable constructorFeeder() { return ImmutableList.of( - new Object[]{LockGranularity.TIME_CHUNK, false}, - new Object[]{LockGranularity.TIME_CHUNK, true}, - new Object[]{LockGranularity.SEGMENT, true} +// new Object[]{LockGranularity.TIME_CHUNK, false, 2}, + new Object[]{LockGranularity.TIME_CHUNK, true, 2} +// new Object[]{LockGranularity.TIME_CHUNK, true, 1}, +// new Object[]{LockGranularity.SEGMENT, true, 2} ); } + private final int maxNumConcurrentSubTasks; + private File inputDir; - public HashPartitionMultiPhaseParallelIndexingTest(LockGranularity lockGranularity, boolean useInputFormatApi) + public HashPartitionMultiPhaseParallelIndexingTest( + LockGranularity lockGranularity, + boolean useInputFormatApi, + int maxNumConcurrentSubTasks + ) { super(lockGranularity, useInputFormatApi); + this.maxNumConcurrentSubTasks = maxNumConcurrentSubTasks; } @Before @@ -132,7 +139,7 @@ public void testRun() throws Exception inputDir, "test_*", new HashedPartitionsSpec(null, 2, ImmutableList.of("dim1", "dim2")), - MAX_NUM_CONCURRENT_SUB_TASKS, + maxNumConcurrentSubTasks, TaskState.SUCCESS ); } else { @@ -145,7 +152,7 @@ public void testRun() throws Exception inputDir, "test_*", new HashedPartitionsSpec(null, 2, ImmutableList.of("dim1", "dim2")), - MAX_NUM_CONCURRENT_SUB_TASKS, + maxNumConcurrentSubTasks, TaskState.SUCCESS ); } From af2b6958752674affa4a7e05019263eafe07cb0e Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Sat, 13 Jun 2020 13:37:06 -0700 Subject: [PATCH 14/24] centralized partition id allocation in parallel and simple tasks --- .../segment/loading/DataSegmentPusher.java | 9 + .../partition/BucketNumberedShardSpec.java | 44 +++-- .../BuildingHashBasedNumberedShardSpec.java | 1 + .../partition/BuildingNumberedShardSpec.java | 14 +- .../timeline/partition/BuildingShardSpec.java | 10 +- .../BuildingSingleDimensionShardSpec.java | 33 +++- .../partition/HashBasedNumberedShardSpec.java | 4 +- .../partition/HashBucketShardSpec.java | 139 ++++++++++++++ .../timeline/partition/LinearShardSpec.java | 2 +- .../timeline/partition/NoneShardSpec.java | 2 +- .../partition/NumberedOverwriteShardSpec.java | 2 +- .../timeline/partition/NumberedShardSpec.java | 4 +- .../partition/RangeBucketShardSpec.java | 149 +++++++++++++++ .../druid/timeline/partition/ShardSpec.java | 8 +- .../partition/SingleDimensionShardSpec.java | 33 +++- .../druid/timeline/DataSegmentTest.java | 2 +- ...uildingHashBasedNumberedShardSpecTest.java | 67 +------ .../BuildingSingleDimensionShardSpecTest.java | 72 +++++++ .../partition/HashBucketShardSpecTest.java | 121 ++++++++++++ .../partition/RangeBucketShardSpecTest.java | 115 ++++++++++++ .../partition/ShardSpecTestUtils.java | 49 +++++ .../task/CachingLocalSegmentAllocator.java | 78 ++++---- ...nearlyPartitionedSequenceNameFunction.java | 13 +- .../indexing/common/task/ShardSpecs.java | 124 ++++++------ .../parallel/GenericPartitionLocation.java | 12 +- .../batch/parallel/GenericPartitionStat.java | 13 +- .../batch/parallel/HashPartitionLocation.java | 2 +- .../batch/parallel/HashPartitionStat.java | 2 +- .../parallel/ParallelIndexSupervisorTask.java | 85 ++++----- .../PartialGenericSegmentMergeTask.java | 36 ++-- ...egmentGenerateParallelIndexTaskRunner.java | 2 +- .../PartialHashSegmentGenerateTask.java | 19 +- .../PartialRangeSegmentGenerateTask.java | 3 +- .../parallel/PartialSegmentMergeTask.java | 32 ++-- .../batch/parallel/PartitionLocation.java | 8 +- .../task/batch/parallel/PartitionStat.java | 2 +- .../partition/CompletePartitionAnalysis.java | 11 +- .../task/batch/partition/HashBucket.java | 80 -------- .../batch/partition/HashBucketLookup.java | 77 -------- .../partition/HashPartitionAnalysis.java | 58 +----- .../task/batch/partition/PartitionBucket.java | 34 ---- .../partition/PartitionBucketLookup.java | 37 ---- .../batch/partition/PartitionBuckets.java | 58 ------ .../task/batch/partition/RangeBucket.java | 109 ----------- .../partition/RangePartitionAnalysis.java | 45 +---- .../worker/IntermediaryDataManager.java | 14 +- .../indexing/worker/http/ShuffleResource.java | 8 +- .../indexing/common/task/ShardSpecsTest.java | 176 +++++++++--------- ...stractParallelIndexSupervisorTaskTest.java | 2 +- .../GenericPartitionLocationTest.java | 2 +- .../parallel/GenericPartitionStatTest.java | 12 +- ...rtitionAdjustingCorePartitionSizeTest.java | 35 ++-- ...rtitionMultiPhaseParallelIndexingTest.java | 8 +- .../batch/parallel/HashPartitionStatTest.java | 2 +- .../batch/parallel/HttpShuffleClientTest.java | 2 +- .../ParallelIndexSupervisorTaskTest.java | 2 +- .../parallel/ParallelIndexTestingFactory.java | 5 +- ...rtitionAdjustingCorePartitionSizeTest.java | 167 +++++++++++++++++ 58 files changed, 1309 insertions(+), 946 deletions(-) rename indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/partition/RangeBucketLookup.java => core/src/main/java/org/apache/druid/timeline/partition/BucketNumberedShardSpec.java (50%) create mode 100644 core/src/main/java/org/apache/druid/timeline/partition/HashBucketShardSpec.java create mode 100644 core/src/main/java/org/apache/druid/timeline/partition/RangeBucketShardSpec.java create mode 100644 core/src/test/java/org/apache/druid/timeline/partition/BuildingSingleDimensionShardSpecTest.java create mode 100644 core/src/test/java/org/apache/druid/timeline/partition/HashBucketShardSpecTest.java create mode 100644 core/src/test/java/org/apache/druid/timeline/partition/RangeBucketShardSpecTest.java create mode 100644 core/src/test/java/org/apache/druid/timeline/partition/ShardSpecTestUtils.java delete mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/partition/HashBucket.java delete mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/partition/HashBucketLookup.java delete mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/partition/PartitionBucket.java delete mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/partition/PartitionBucketLookup.java delete mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/partition/PartitionBuckets.java delete mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/partition/RangeBucket.java create mode 100644 indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/RangePartitionAdjustingCorePartitionSizeTest.java 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 7e9eb9234b76..98c75505c05f 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 @@ -20,9 +20,11 @@ package org.apache.druid.segment.loading; import com.google.common.base.Joiner; +import com.google.common.base.Preconditions; import org.apache.druid.guice.annotations.ExtensionPoint; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.partition.BucketNumberedShardSpec; import java.io.File; import java.io.IOException; @@ -105,6 +107,13 @@ default List getAllowedPropertyPrefixesForHadoop() // on segment deletion if segment being deleted was the only segment static String getDefaultStorageDir(DataSegment segment, boolean useUniquePath) { + // Sanity check for shardSpec type. + // BucketNumberedShardSpec should never be used in segment push. + Preconditions.checkArgument( + !(segment.getShardSpec() instanceof BucketNumberedShardSpec), + "Illegal shardSpec type[%s]", + segment.getShardSpec() + ); return JOINER.join( segment.getDataSource(), StringUtils.format("%s_%s", segment.getInterval().getStart(), segment.getInterval().getEnd()), diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/partition/RangeBucketLookup.java b/core/src/main/java/org/apache/druid/timeline/partition/BucketNumberedShardSpec.java similarity index 50% rename from indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/partition/RangeBucketLookup.java rename to core/src/main/java/org/apache/druid/timeline/partition/BucketNumberedShardSpec.java index 2081a7bc465a..c9cc37ae9505 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/partition/RangeBucketLookup.java +++ b/core/src/main/java/org/apache/druid/timeline/partition/BucketNumberedShardSpec.java @@ -17,37 +17,47 @@ * under the License. */ -package org.apache.druid.indexing.common.task.batch.partition; +package org.apache.druid.timeline.partition; -import org.apache.druid.data.input.InputRow; -import org.apache.druid.java.util.common.ISE; +import com.fasterxml.jackson.annotation.JsonIgnore; +import com.google.common.collect.RangeSet; -import java.util.Iterator; import java.util.List; +import java.util.Map; -public class RangeBucketLookup implements PartitionBucketLookup +/** + * TODO + */ +public interface BucketNumberedShardSpec extends ShardSpec { - private final List buckets; + int getBucketId(); + + T convert(int partitionId); + + @Override + default int getNumCorePartitions() + { + throw new UnsupportedOperationException(); + } - public RangeBucketLookup(List buckets) + // The below methods are used on the query side, and so must not be called for this shardSpec. + + @JsonIgnore + @Override + default List getDomainDimensions() { - this.buckets = buckets; + throw new UnsupportedOperationException(); } @Override - public RangeBucket find(long timestamp, InputRow row) + default boolean possibleInDomain(Map> domain) { - for (RangeBucket bucket : buckets) { - if (bucket.isInBucket(timestamp, row)) { - return bucket; - } - } - throw new ISE("row[%s] doesn't fit in any bucket[%s]", row, buckets); + throw new UnsupportedOperationException(); } @Override - public Iterator iterator() + default boolean isCompatible(Class other) { - return buckets.iterator(); + throw new UnsupportedOperationException(); } } diff --git a/core/src/main/java/org/apache/druid/timeline/partition/BuildingHashBasedNumberedShardSpec.java b/core/src/main/java/org/apache/druid/timeline/partition/BuildingHashBasedNumberedShardSpec.java index f6a2f9e9f3bd..fb896fc2ac84 100644 --- a/core/src/main/java/org/apache/druid/timeline/partition/BuildingHashBasedNumberedShardSpec.java +++ b/core/src/main/java/org/apache/druid/timeline/partition/BuildingHashBasedNumberedShardSpec.java @@ -68,6 +68,7 @@ public int getPartitionNum() return partitionId; } + @Override @JsonProperty public int getBucketId() { diff --git a/core/src/main/java/org/apache/druid/timeline/partition/BuildingNumberedShardSpec.java b/core/src/main/java/org/apache/druid/timeline/partition/BuildingNumberedShardSpec.java index d17ea8f21fa1..7b2739385753 100644 --- a/core/src/main/java/org/apache/druid/timeline/partition/BuildingNumberedShardSpec.java +++ b/core/src/main/java/org/apache/druid/timeline/partition/BuildingNumberedShardSpec.java @@ -23,7 +23,6 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; -import java.util.List; import java.util.Objects; /** @@ -47,6 +46,13 @@ public BuildingNumberedShardSpec(@JsonProperty("partitionId") int partitionId) this.partitionId = partitionId; } + @Override + public int getBucketId() + { + // TODO: explain... + return 0; + } + @Override public NumberedShardSpec convert(int numTotalPartitions) { @@ -68,12 +74,6 @@ public int getPartitionNum() return partitionId; } - @Override - public ShardSpecLookup getLookup(List shardSpecs) - { - return NumberedShardSpec.createNumberedLookup(shardSpecs); - } - @Override public boolean equals(Object o) { diff --git a/core/src/main/java/org/apache/druid/timeline/partition/BuildingShardSpec.java b/core/src/main/java/org/apache/druid/timeline/partition/BuildingShardSpec.java index b6e74dd5a0a2..c6066af7d126 100644 --- a/core/src/main/java/org/apache/druid/timeline/partition/BuildingShardSpec.java +++ b/core/src/main/java/org/apache/druid/timeline/partition/BuildingShardSpec.java @@ -50,16 +50,24 @@ */ public interface BuildingShardSpec extends ShardSpec { + int getBucketId(); + T convert(int numCorePartitions); + /** + * TODO + */ @Override default int getNumCorePartitions() { throw new UnsupportedOperationException(); } + /** + * TODO + */ @Override - default ShardSpecLookup getLookup(List shardSpecs) + default ShardSpecLookup getLookup(List shardSpecs) { throw new UnsupportedOperationException(); } diff --git a/core/src/main/java/org/apache/druid/timeline/partition/BuildingSingleDimensionShardSpec.java b/core/src/main/java/org/apache/druid/timeline/partition/BuildingSingleDimensionShardSpec.java index dfcde031c549..8b97fb1250ac 100644 --- a/core/src/main/java/org/apache/druid/timeline/partition/BuildingSingleDimensionShardSpec.java +++ b/core/src/main/java/org/apache/druid/timeline/partition/BuildingSingleDimensionShardSpec.java @@ -25,29 +25,35 @@ import javax.annotation.Nullable; import java.util.Objects; +/** + * TODO + */ public class BuildingSingleDimensionShardSpec implements BuildingShardSpec { public static final String TYPE = "building_single_dim"; + private final int bucketId; private final String dimension; @Nullable private final String start; @Nullable private final String end; - private final int partitionNum; + private final int partitionId; @JsonCreator public BuildingSingleDimensionShardSpec( + @JsonProperty("bucketId") int bucketId, @JsonProperty("dimension") String dimension, @JsonProperty("start") @Nullable String start, @JsonProperty("end") @Nullable String end, @JsonProperty("partitionNum") int partitionNum ) { + this.bucketId = bucketId; this.dimension = dimension; this.start = start; this.end = end; - this.partitionNum = partitionNum; + this.partitionId = partitionNum; } @JsonProperty("dimension") @@ -74,19 +80,26 @@ public String getEnd() @JsonProperty("partitionNum") public int getPartitionNum() { - return partitionNum; + return partitionId; + } + + @Override + @JsonProperty("bucketId") + public int getBucketId() + { + return bucketId; } @Override public SingleDimensionShardSpec convert(int numCorePartitions) { - return new SingleDimensionShardSpec(dimension, start, end, partitionNum, numCorePartitions); + return new SingleDimensionShardSpec(dimension, start, end, partitionId, numCorePartitions); } @Override public PartitionChunk createChunk(T obj) { - return new NumberedPartitionChunk<>(partitionNum, 0, obj); + return new NumberedPartitionChunk<>(partitionId, 0, obj); } @Override @@ -99,7 +112,8 @@ public boolean equals(Object o) return false; } BuildingSingleDimensionShardSpec that = (BuildingSingleDimensionShardSpec) o; - return partitionNum == that.partitionNum && + return bucketId == that.bucketId && + partitionId == that.partitionId && Objects.equals(dimension, that.dimension) && Objects.equals(start, that.start) && Objects.equals(end, that.end); @@ -108,17 +122,18 @@ public boolean equals(Object o) @Override public int hashCode() { - return Objects.hash(dimension, start, end, partitionNum); + return Objects.hash(bucketId, dimension, start, end, partitionId); } @Override public String toString() { return "BuildingSingleDimensionShardSpec{" + - "dimension='" + dimension + '\'' + + "bucketId=" + bucketId + + ", dimension='" + dimension + '\'' + ", start='" + start + '\'' + ", end='" + end + '\'' + - ", partitionNum=" + partitionNum + + ", partitionNum=" + partitionId + '}'; } } 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 215794a6c0a6..23cdb4ef3868 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 @@ -145,7 +145,7 @@ public static int hash(ObjectMapper jsonMapper, List objects) throws Jso } @Override - public ShardSpecLookup getLookup(final List shardSpecs) + public ShardSpecLookup getLookup(final List shardSpecs) { return createHashLookup(jsonMapper, partitionDimensions, shardSpecs, numBuckets); } @@ -153,7 +153,7 @@ public ShardSpecLookup getLookup(final List shardSpecs) static ShardSpecLookup createHashLookup( ObjectMapper jsonMapper, List partitionDimensions, - List shardSpecs, + List shardSpecs, int numBuckets ) { diff --git a/core/src/main/java/org/apache/druid/timeline/partition/HashBucketShardSpec.java b/core/src/main/java/org/apache/druid/timeline/partition/HashBucketShardSpec.java new file mode 100644 index 000000000000..90af43536c06 --- /dev/null +++ b/core/src/main/java/org/apache/druid/timeline/partition/HashBucketShardSpec.java @@ -0,0 +1,139 @@ +/* + * 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.JacksonInject; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.data.input.InputRow; + +import java.util.List; +import java.util.Objects; + +/** + * TODO + */ +public class HashBucketShardSpec implements BucketNumberedShardSpec +{ + public static final String TYPE = "bucket_hash"; + + private final int bucketId; + private final int numBuckets; + private final List partitionDimensions; + private final ObjectMapper jsonMapper; + + @JsonCreator + public HashBucketShardSpec( + @JsonProperty("bucketId") int bucketId, + @JsonProperty("numBuckets") int numBuckets, + @JsonProperty("partitionDimensions") List partitionDimensions, + @JacksonInject ObjectMapper jsonMapper + ) + { + this.bucketId = bucketId; + this.numBuckets = numBuckets; + this.partitionDimensions = partitionDimensions; + this.jsonMapper = jsonMapper; + } + + @Override + @JsonProperty + public int getBucketId() + { + return bucketId; + } + + @JsonProperty + public int getNumBuckets() + { + return numBuckets; + } + + @JsonProperty + public List getPartitionDimensions() + { + return partitionDimensions; + } + + @Override + public BuildingHashBasedNumberedShardSpec convert(int partitionId) + { + return new BuildingHashBasedNumberedShardSpec(partitionId, bucketId, numBuckets, partitionDimensions, jsonMapper); + } + + @Override + public PartitionChunk createChunk(T obj) + { + // TODO: explain.. + return new NumberedPartitionChunk<>(bucketId, 0, obj); + } + + @Override + public boolean isInChunk(long timestamp, InputRow inputRow) + { + // not in use + throw new UnsupportedOperationException(); + } + + @Override + public int getPartitionNum() + { + // TODO: explain.. + return bucketId; + } + + @Override + public ShardSpecLookup getLookup(List shardSpecs) + { + return HashBasedNumberedShardSpec.createHashLookup(jsonMapper, partitionDimensions, shardSpecs, numBuckets); + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + HashBucketShardSpec that = (HashBucketShardSpec) o; + return bucketId == that.bucketId && + numBuckets == that.numBuckets && + Objects.equals(partitionDimensions, that.partitionDimensions); + } + + @Override + public int hashCode() + { + return Objects.hash(bucketId, numBuckets, partitionDimensions); + } + + @Override + public String toString() + { + return "HashBucket{" + + ", bucketId=" + bucketId + + ", numBuckets=" + numBuckets + + ", partitionDimensions=" + partitionDimensions + + '}'; + } +} 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 584b5e5a1178..95b0bd832b9b 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 @@ -56,7 +56,7 @@ public int getNumCorePartitions() } @Override - public ShardSpecLookup getLookup(final List shardSpecs) + public ShardSpecLookup getLookup(final List shardSpecs) { return (long timestamp, InputRow row) -> shardSpecs.get(0); } 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 25c9f752bec0..d53011819413 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 @@ -72,7 +72,7 @@ public int getNumCorePartitions() } @Override - public ShardSpecLookup getLookup(final List shardSpecs) + public ShardSpecLookup getLookup(final List shardSpecs) { return (long timestamp, InputRow row) -> shardSpecs.get(0); } 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 dbbb8f66e936..adb0d288d6a6 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 @@ -188,7 +188,7 @@ public short getAtomicUpdateGroupSize() } @Override - public ShardSpecLookup getLookup(List shardSpecs) + public ShardSpecLookup getLookup(List shardSpecs) { return (long timestamp, InputRow row) -> shardSpecs.get(0); } 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 a952e0436768..72490d1d9be1 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 @@ -66,12 +66,12 @@ public int getPartitionNum() } @Override - public ShardSpecLookup getLookup(final List shardSpecs) + public ShardSpecLookup getLookup(final List shardSpecs) { return createNumberedLookup(shardSpecs); } - static ShardSpecLookup createNumberedLookup(List shardSpecs) + static ShardSpecLookup createNumberedLookup(List shardSpecs) { return (long timestamp, InputRow row) -> shardSpecs.get(0); } diff --git a/core/src/main/java/org/apache/druid/timeline/partition/RangeBucketShardSpec.java b/core/src/main/java/org/apache/druid/timeline/partition/RangeBucketShardSpec.java new file mode 100644 index 000000000000..919ad6aadb6e --- /dev/null +++ b/core/src/main/java/org/apache/druid/timeline/partition/RangeBucketShardSpec.java @@ -0,0 +1,149 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.timeline.partition; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.druid.data.input.InputRow; + +import javax.annotation.Nullable; +import java.util.List; +import java.util.Objects; + +/** + * + */ +public class RangeBucketShardSpec implements BucketNumberedShardSpec +{ + public static final String TYPE = "bucket_single_dim"; + + private final int bucketId; + private final String dimension; + @Nullable + private final String start; + @Nullable + private final String end; + + @JsonCreator + public RangeBucketShardSpec( + @JsonProperty("bucketId") int bucketId, + @JsonProperty("dimension") String dimension, + @JsonProperty("start") @Nullable String start, + @JsonProperty("end") @Nullable String end + ) + { + this.bucketId = bucketId; + this.dimension = dimension; + this.start = start; + this.end = end; + } + + @Override + @JsonProperty + public int getBucketId() + { + return bucketId; + } + + @JsonProperty + public String getDimension() + { + return dimension; + } + + @Nullable + @JsonProperty + public String getStart() + { + return start; + } + + @Nullable + @JsonProperty + public String getEnd() + { + return end; + } + + @Override + public BuildingSingleDimensionShardSpec convert(int partitionId) + { + return new BuildingSingleDimensionShardSpec(bucketId, dimension, start, end, partitionId); + } + + @Override + public boolean isInChunk(long timestamp, InputRow inputRow) + { + return SingleDimensionShardSpec.isInChunk(dimension, start, end, inputRow); + } + + @Override + public PartitionChunk createChunk(T obj) + { + // TODO: explain.. + return new NumberedPartitionChunk<>(bucketId, 0, obj); + } + + @Override + public int getPartitionNum() + { + // TODO: explain.. + return bucketId; + } + + @Override + public ShardSpecLookup getLookup(List shardSpecs) + { + return SingleDimensionShardSpec.createLookup(shardSpecs); + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + RangeBucketShardSpec bucket = (RangeBucketShardSpec) o; + return bucketId == bucket.bucketId && + Objects.equals(dimension, bucket.dimension) && + Objects.equals(start, bucket.start) && + Objects.equals(end, bucket.end); + } + + @Override + public int hashCode() + { + return Objects.hash(bucketId, dimension, start, end); + } + + @Override + public String toString() + { + return "RangeBucket{" + + ", bucketId=" + bucketId + + ", dimension='" + dimension + '\'' + + ", start='" + start + '\'' + + ", end='" + end + '\'' + + '}'; + } +} 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 63f49b710a4d..1f99b16058bf 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 @@ -40,9 +40,13 @@ @JsonSubTypes.Type(name = "numbered", value = NumberedShardSpec.class), @JsonSubTypes.Type(name = "hashed", value = HashBasedNumberedShardSpec.class), @JsonSubTypes.Type(name = NumberedOverwriteShardSpec.TYPE, value = NumberedOverwriteShardSpec.class), + // TODO: shouldn't be published @JsonSubTypes.Type(name = BuildingNumberedShardSpec.TYPE, value = BuildingNumberedShardSpec.class), @JsonSubTypes.Type(name = BuildingHashBasedNumberedShardSpec.TYPE, value = BuildingHashBasedNumberedShardSpec.class), - @JsonSubTypes.Type(name = BuildingSingleDimensionShardSpec.TYPE, value = BuildingSingleDimensionShardSpec.class) + @JsonSubTypes.Type(name = BuildingSingleDimensionShardSpec.TYPE, value = BuildingSingleDimensionShardSpec.class), + // TODO: shouldn't be used in push + @JsonSubTypes.Type(name = HashBucketShardSpec.TYPE, value = HashBucketShardSpec.class), + @JsonSubTypes.Type(name = RangeBucketShardSpec.TYPE, value = RangeBucketShardSpec.class) }) public interface ShardSpec { @@ -100,7 +104,7 @@ default short getAtomicUpdateGroupSize() } @JsonIgnore - ShardSpecLookup getLookup(List shardSpecs); + ShardSpecLookup getLookup(List shardSpecs); /** * Get dimensions who have possible range for the rows this shard contains. 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 05bad5f3ae7e..20972508c154 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 @@ -106,7 +106,12 @@ public int getNumCorePartitions() } @Override - public ShardSpecLookup getLookup(final List shardSpecs) + public ShardSpecLookup getLookup(final List shardSpecs) + { + return createLookup(shardSpecs); + } + + static ShardSpecLookup createLookup(List shardSpecs) { return (long timestamp, InputRow row) -> { for (ShardSpec spec : shardSpecs) { @@ -165,16 +170,10 @@ public PartitionChunk createChunk(T obj) @Override public boolean isInChunk(long timestamp, InputRow inputRow) { - final List values = inputRow.getDimension(dimension); - - if (values == null || values.size() != 1) { - return checkValue(null); - } else { - return checkValue(values.get(0)); - } + return isInChunk(dimension, start, end, inputRow); } - private boolean checkValue(String value) + private static boolean checkValue(@Nullable String start, @Nullable String end, String value) { if (value == null) { return start == null; @@ -188,6 +187,22 @@ private boolean checkValue(String value) (end == null || value.compareTo(end) < 0); } + public static boolean isInChunk( + String dimension, + @Nullable String start, + @Nullable String end, + InputRow inputRow + ) + { + final List values = inputRow.getDimension(dimension); + + if (values == null || values.size() != 1) { + return checkValue(start, end, null); + } else { + return checkValue(start, end, values.get(0)); + } + } + @Override public boolean equals(Object o) { 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 62913646e92b..c2b0b76e8881 100644 --- a/core/src/test/java/org/apache/druid/timeline/DataSegmentTest.java +++ b/core/src/test/java/org/apache/druid/timeline/DataSegmentTest.java @@ -81,7 +81,7 @@ public int getNumCorePartitions() } @Override - public ShardSpecLookup getLookup(List shardSpecs) + public ShardSpecLookup getLookup(List shardSpecs) { return null; } diff --git a/core/src/test/java/org/apache/druid/timeline/partition/BuildingHashBasedNumberedShardSpecTest.java b/core/src/test/java/org/apache/druid/timeline/partition/BuildingHashBasedNumberedShardSpecTest.java index 8b678a8852d1..48c5a874865f 100644 --- a/core/src/test/java/org/apache/druid/timeline/partition/BuildingHashBasedNumberedShardSpecTest.java +++ b/core/src/test/java/org/apache/druid/timeline/partition/BuildingHashBasedNumberedShardSpecTest.java @@ -26,23 +26,20 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.jsontype.NamedType; import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; import nl.jqno.equalsverifier.EqualsVerifier; -import org.apache.druid.data.input.MapBasedInputRow; -import org.apache.druid.java.util.common.DateTimes; import org.junit.Assert; import org.junit.Test; -import java.util.List; - public class BuildingHashBasedNumberedShardSpecTest { + private final ObjectMapper mapper = ShardSpecTestUtils.initObjectMapper(); + @Test public void testConvert() { Assert.assertEquals( - new HashBasedNumberedShardSpec(5, 10, 5, 12, ImmutableList.of("dim"), new ObjectMapper()), - new BuildingHashBasedNumberedShardSpec(5, 5, 12, ImmutableList.of("dim"), new ObjectMapper()).convert(10) + new HashBasedNumberedShardSpec(5, 10, 5, 12, ImmutableList.of("dim"), mapper), + new BuildingHashBasedNumberedShardSpec(5, 5, 12, ImmutableList.of("dim"), mapper).convert(10) ); } @@ -51,70 +48,24 @@ public void testCreateChunk() { Assert.assertEquals( new NumberedPartitionChunk<>(5, 0, "test"), - new BuildingHashBasedNumberedShardSpec(5, 5, 12, ImmutableList.of("dim"), new ObjectMapper()) + new BuildingHashBasedNumberedShardSpec(5, 5, 12, ImmutableList.of("dim"), mapper) .createChunk("test") ); } - @Test - public void testShardSpecLookup() - { - final List shardSpecs = ImmutableList.of( - new BuildingHashBasedNumberedShardSpec(0, 0, 3, ImmutableList.of("dim"), new ObjectMapper()), - new BuildingHashBasedNumberedShardSpec(1, 1, 3, ImmutableList.of("dim"), new ObjectMapper()), - new BuildingHashBasedNumberedShardSpec(2, 2, 3, ImmutableList.of("dim"), new ObjectMapper()) - ); - final ShardSpecLookup lookup = shardSpecs.get(0).getLookup(shardSpecs); - final long currentTime = DateTimes.nowUtc().getMillis(); - Assert.assertEquals( - shardSpecs.get(1), - lookup.getShardSpec( - currentTime, - new MapBasedInputRow( - currentTime, - ImmutableList.of("dim"), ImmutableMap.of("dim", "1", "time", currentTime) - ) - ) - ); - Assert.assertEquals( - shardSpecs.get(2), - lookup.getShardSpec( - currentTime, - new MapBasedInputRow( - currentTime, - ImmutableList.of("dim"), ImmutableMap.of("dim", "2", "time", currentTime) - ) - ) - ); - Assert.assertEquals( - shardSpecs.get(0), - lookup.getShardSpec( - currentTime, - new MapBasedInputRow( - currentTime, - ImmutableList.of("dim"), ImmutableMap.of("dim", "3", "time", currentTime) - ) - ) - ); - } - @Test public void testSerde() throws JsonProcessingException { - final ObjectMapper mapper = new ObjectMapper(); - mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false); - mapper.configure(MapperFeature.AUTO_DETECT_SETTERS, false); - mapper.registerSubtypes(new NamedType( - BuildingHashBasedNumberedShardSpec.class, - BuildingHashBasedNumberedShardSpec.TYPE - )); + mapper.registerSubtypes( + new NamedType(BuildingHashBasedNumberedShardSpec.class, BuildingHashBasedNumberedShardSpec.TYPE) + ); mapper.setInjectableValues(new Std().addValue(ObjectMapper.class, mapper)); final BuildingHashBasedNumberedShardSpec original = new BuildingHashBasedNumberedShardSpec( 3, 5, 12, ImmutableList.of("dim"), - new ObjectMapper() + mapper ); final String json = mapper.writeValueAsString(original); final BuildingHashBasedNumberedShardSpec fromJson = (BuildingHashBasedNumberedShardSpec) mapper.readValue( diff --git a/core/src/test/java/org/apache/druid/timeline/partition/BuildingSingleDimensionShardSpecTest.java b/core/src/test/java/org/apache/druid/timeline/partition/BuildingSingleDimensionShardSpecTest.java new file mode 100644 index 000000000000..d70a42ff5bae --- /dev/null +++ b/core/src/test/java/org/apache/druid/timeline/partition/BuildingSingleDimensionShardSpecTest.java @@ -0,0 +1,72 @@ +/* + * 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.core.JsonProcessingException; +import com.fasterxml.jackson.databind.InjectableValues.Std; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.jsontype.NamedType; +import nl.jqno.equalsverifier.EqualsVerifier; +import org.junit.Assert; +import org.junit.Test; + +public class BuildingSingleDimensionShardSpecTest +{ + @Test + public void testConvert() + { + Assert.assertEquals( + new SingleDimensionShardSpec("dim", "start", "end", 5, 10), + new BuildingSingleDimensionShardSpec(1, "dim", "start", "end", 5).convert(10) + ); + } + + @Test + public void testCreateChunk() + { + Assert.assertEquals( + new NumberedPartitionChunk<>(5, 0, "test"), + new BuildingSingleDimensionShardSpec(1, "dim", "start", "end", 5).createChunk("test") + ); + } + + @Test + public void testSerde() throws JsonProcessingException + { + final ObjectMapper mapper = ShardSpecTestUtils.initObjectMapper(); + mapper.registerSubtypes( + new NamedType(BuildingSingleDimensionShardSpec.class, BuildingSingleDimensionShardSpec.TYPE) + ); + mapper.setInjectableValues(new Std().addValue(ObjectMapper.class, mapper)); + final BuildingSingleDimensionShardSpec original = new BuildingSingleDimensionShardSpec(1, "dim", "start", "end", 5); + final String json = mapper.writeValueAsString(original); + final BuildingSingleDimensionShardSpec fromJson = (BuildingSingleDimensionShardSpec) mapper.readValue( + json, + ShardSpec.class + ); + Assert.assertEquals(original, fromJson); + } + + @Test + public void testEquals() + { + EqualsVerifier.forClass(BuildingSingleDimensionShardSpec.class).usingGetClass().verify(); + } +} diff --git a/core/src/test/java/org/apache/druid/timeline/partition/HashBucketShardSpecTest.java b/core/src/test/java/org/apache/druid/timeline/partition/HashBucketShardSpecTest.java new file mode 100644 index 000000000000..df2207b798f4 --- /dev/null +++ b/core/src/test/java/org/apache/druid/timeline/partition/HashBucketShardSpecTest.java @@ -0,0 +1,121 @@ +/* + * 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.core.JsonProcessingException; +import com.fasterxml.jackson.databind.InjectableValues.Std; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.jsontype.NamedType; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import nl.jqno.equalsverifier.EqualsVerifier; +import org.apache.druid.data.input.MapBasedInputRow; +import org.apache.druid.java.util.common.DateTimes; +import org.junit.Assert; +import org.junit.Test; + +import java.util.List; + +public class HashBucketShardSpecTest +{ + private final ObjectMapper mapper = ShardSpecTestUtils.initObjectMapper(); + + @Test + public void testConvert() + { + Assert.assertEquals( + new BuildingHashBasedNumberedShardSpec(3, 5, 12, ImmutableList.of("dim"), mapper), + new HashBucketShardSpec(5, 12, ImmutableList.of("dim"), mapper).convert(3) + ); + } + + @Test + public void testCreateChunk() + { + Assert.assertEquals( + new NumberedPartitionChunk<>(5, 0, "test"), + new HashBucketShardSpec(5, 12, ImmutableList.of("dim"), mapper).createChunk("test") + ); + } + + @Test + public void testShardSpecLookup() + { + final List shardSpecs = ImmutableList.of( + new HashBucketShardSpec(0, 3, ImmutableList.of("dim"), mapper), + new HashBucketShardSpec(1, 3, ImmutableList.of("dim"), mapper), + new HashBucketShardSpec(2, 3, ImmutableList.of("dim"), mapper) + ); + final ShardSpecLookup lookup = shardSpecs.get(0).getLookup(shardSpecs); + final long currentTime = DateTimes.nowUtc().getMillis(); + Assert.assertEquals( + shardSpecs.get(1), + lookup.getShardSpec( + currentTime, + new MapBasedInputRow( + currentTime, + ImmutableList.of("dim"), ImmutableMap.of("dim", "1", "time", currentTime) + ) + ) + ); + Assert.assertEquals( + shardSpecs.get(2), + lookup.getShardSpec( + currentTime, + new MapBasedInputRow( + currentTime, + ImmutableList.of("dim"), ImmutableMap.of("dim", "2", "time", currentTime) + ) + ) + ); + Assert.assertEquals( + shardSpecs.get(0), + lookup.getShardSpec( + currentTime, + new MapBasedInputRow( + currentTime, + ImmutableList.of("dim"), ImmutableMap.of("dim", "3", "time", currentTime) + ) + ) + ); + } + + @Test + public void testSerde() throws JsonProcessingException + { + mapper.registerSubtypes(new NamedType(HashBucketShardSpec.class, HashBucketShardSpec.TYPE)); + mapper.setInjectableValues(new Std().addValue(ObjectMapper.class, mapper)); + + final HashBucketShardSpec original = new HashBucketShardSpec(5, 12, ImmutableList.of("dim"), mapper); + final String json = mapper.writeValueAsString(original); + final HashBucketShardSpec fromJson = (HashBucketShardSpec) mapper.readValue(json, ShardSpec.class); + Assert.assertEquals(original, fromJson); + } + + @Test + public void testEquals() + { + EqualsVerifier.forClass(HashBucketShardSpec.class) + .withIgnoredFields("jsonMapper") + .withPrefabValues(ObjectMapper.class, new ObjectMapper(), new ObjectMapper()) + .usingGetClass() + .verify(); + } +} diff --git a/core/src/test/java/org/apache/druid/timeline/partition/RangeBucketShardSpecTest.java b/core/src/test/java/org/apache/druid/timeline/partition/RangeBucketShardSpecTest.java new file mode 100644 index 000000000000..d2c06e05f3f1 --- /dev/null +++ b/core/src/test/java/org/apache/druid/timeline/partition/RangeBucketShardSpecTest.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.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.InjectableValues.Std; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.jsontype.NamedType; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import nl.jqno.equalsverifier.EqualsVerifier; +import org.apache.druid.data.input.MapBasedInputRow; +import org.apache.druid.java.util.common.DateTimes; +import org.junit.Assert; +import org.junit.Test; + +import java.util.List; + +public class RangeBucketShardSpecTest +{ + @Test + public void testConvert() + { + Assert.assertEquals( + new BuildingSingleDimensionShardSpec(1, "dim", "start", "end", 5), + new RangeBucketShardSpec(1, "dim", "start", "end").convert(5) + ); + } + + @Test + public void testCreateChunk() + { + Assert.assertEquals( + new NumberedPartitionChunk<>(1, 0, "test"), + new RangeBucketShardSpec(1, "dim", "start", "end").createChunk("test") + ); + } + + @Test + public void testShardSpecLookup() + { + final List shardSpecs = ImmutableList.of( + new RangeBucketShardSpec(0, "dim", null, "c"), + new RangeBucketShardSpec(1, "dim", "f", "i"), + new RangeBucketShardSpec(2, "dim", "i", null) + ); + final ShardSpecLookup lookup = shardSpecs.get(0).getLookup(shardSpecs); + final long currentTime = DateTimes.nowUtc().getMillis(); + Assert.assertEquals( + shardSpecs.get(0), + lookup.getShardSpec( + currentTime, + new MapBasedInputRow( + currentTime, + ImmutableList.of("dim"), ImmutableMap.of("dim", "a", "time", currentTime) + ) + ) + ); + Assert.assertEquals( + shardSpecs.get(1), + lookup.getShardSpec( + currentTime, + new MapBasedInputRow( + currentTime, + ImmutableList.of("dim"), ImmutableMap.of("dim", "g", "time", currentTime) + ) + ) + ); + Assert.assertEquals( + shardSpecs.get(2), + lookup.getShardSpec( + currentTime, + new MapBasedInputRow( + currentTime, + ImmutableList.of("dim"), ImmutableMap.of("dim", "k", "time", currentTime) + ) + ) + ); + } + + @Test + public void testSerde() throws JsonProcessingException + { + final ObjectMapper mapper = ShardSpecTestUtils.initObjectMapper(); + mapper.registerSubtypes(new NamedType(RangeBucketShardSpec.class, RangeBucketShardSpec.TYPE)); + mapper.setInjectableValues(new Std().addValue(ObjectMapper.class, mapper)); + final RangeBucketShardSpec original = new RangeBucketShardSpec(1, "dim", "start", "end"); + final String json = mapper.writeValueAsString(original); + final RangeBucketShardSpec fromJson = (RangeBucketShardSpec) mapper.readValue(json, ShardSpec.class); + Assert.assertEquals(original, fromJson); + } + + @Test + public void testEquals() + { + EqualsVerifier.forClass(RangeBucketShardSpec.class).usingGetClass().verify(); + } +} diff --git a/core/src/test/java/org/apache/druid/timeline/partition/ShardSpecTestUtils.java b/core/src/test/java/org/apache/druid/timeline/partition/ShardSpecTestUtils.java new file mode 100644 index 000000000000..2f153651513b --- /dev/null +++ b/core/src/test/java/org/apache/druid/timeline/partition/ShardSpecTestUtils.java @@ -0,0 +1,49 @@ +/* + * 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.DeserializationFeature; +import com.fasterxml.jackson.databind.MapperFeature; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.SerializationFeature; + +public class ShardSpecTestUtils +{ + public static ObjectMapper initObjectMapper() + { + // Copied configurations from org.apache.druid.jackson.DefaultObjectMapper + final ObjectMapper mapper = new ObjectMapper(); + mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false); + mapper.configure(MapperFeature.AUTO_DETECT_GETTERS, false); + // See https://github.com/FasterXML/jackson-databind/issues/170 + // configure(MapperFeature.AUTO_DETECT_CREATORS, false); + mapper.configure(MapperFeature.AUTO_DETECT_FIELDS, false); + mapper.configure(MapperFeature.AUTO_DETECT_IS_GETTERS, false); + mapper.configure(MapperFeature.AUTO_DETECT_SETTERS, false); + mapper.configure(MapperFeature.ALLOW_FINAL_FIELDS_AS_MUTATORS, false); + mapper.configure(SerializationFeature.INDENT_OUTPUT, false); + mapper.configure(SerializationFeature.FLUSH_AFTER_WRITE_VALUE, false); + return mapper; + } + + private ShardSpecTestUtils() + { + } +} 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 0e01a36f88d6..ce2aa6af41ed 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 @@ -30,13 +30,13 @@ import org.apache.druid.indexing.common.actions.TaskAction; import org.apache.druid.indexing.common.task.batch.parallel.SupervisorTaskAccess; import org.apache.druid.indexing.common.task.batch.partition.CompletePartitionAnalysis; -import org.apache.druid.indexing.common.task.batch.partition.PartitionBucket; -import org.apache.druid.indexing.common.task.batch.partition.PartitionBucketLookup; -import org.apache.druid.indexing.common.task.batch.partition.PartitionBuckets; import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.Pair; 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.BucketNumberedShardSpec; +import org.apache.druid.timeline.partition.ShardSpec; import org.joda.time.Interval; import javax.annotation.Nullable; @@ -54,12 +54,13 @@ public class CachingLocalSegmentAllocator implements SegmentAllocator { private final String dataSource; - private final Map sequenceNameToBucket; + private final Map> sequenceNameToBucket; private final Function versionFinder; private final NonLinearlyPartitionedSequenceNameFunction sequenceNameFunction; + private final boolean isParallel; private final Map sequenceNameToSegmentId = new HashMap<>(); - private final Object2IntMap intervalToNextPartitionId; + private final Object2IntMap intervalToNextPartitionId = new Object2IntOpenHashMap<>(); CachingLocalSegmentAllocator( TaskToolbox toolbox, @@ -76,36 +77,44 @@ public class CachingLocalSegmentAllocator implements SegmentAllocator final TaskAction> action; if (supervisorTaskAccess == null) { action = new LockListAction(); + isParallel = false; } else { action = new SurrogateAction<>(supervisorTaskAccess.getSupervisorTaskId(), new LockListAction()); + isParallel = true; } - final Map intervalToVersion = - toolbox.getTaskActionClient() - .submit(action) - .stream() - .collect(Collectors.toMap(TaskLock::getInterval, TaskLock::getVersion)); - this.versionFinder = interval -> findVersion(intervalToVersion, interval); - final Map intervalToBucketLookup = partitionAnalysis.createBuckets(toolbox); - intervalToNextPartitionId = new Object2IntOpenHashMap<>(intervalToBucketLookup.size()); + this.versionFinder = createVersionFinder(toolbox, action); + final Map> intervalToShardSpecs = partitionAnalysis.createBuckets(toolbox); sequenceNameFunction = new NonLinearlyPartitionedSequenceNameFunction( taskId, - new PartitionBuckets(granularitySpec, intervalToBucketLookup) + new ShardSpecs(intervalToShardSpecs, granularitySpec.getQueryGranularity()) ); - for (Entry entry : intervalToBucketLookup.entrySet()) { + for (Entry> entry : intervalToShardSpecs.entrySet()) { final Interval interval = entry.getKey(); - final PartitionBucketLookup bucketLookup = entry.getValue(); + final List buckets = entry.getValue(); - bucketLookup.iterator().forEachRemaining(bucket -> { - // The shardSpecs for partitioning and publishing can be different if isExtendableShardSpecs = true. - sequenceNameToBucket.put(sequenceNameFunction.getSequenceName(interval, bucket), bucket); + buckets.forEach(bucket -> { + sequenceNameToBucket.put(sequenceNameFunction.getSequenceName(interval, bucket), Pair.of(interval, bucket)); }); - intervalToNextPartitionId.put(interval, 0); } } + static Function createVersionFinder( + TaskToolbox toolbox, + TaskAction> lockListAction + ) throws IOException + { + final Map intervalToVersion = + toolbox.getTaskActionClient() + .submit(lockListAction) + .stream() + .collect(Collectors.toMap(TaskLock::getInterval, TaskLock::getVersion)); + + return interval -> findVersion(intervalToVersion, interval); + } + private static String findVersion(Map intervalToVersion, Interval interval) { return intervalToVersion.entrySet().stream() @@ -126,32 +135,27 @@ public SegmentIdWithShardSpec allocate( return sequenceNameToSegmentId.computeIfAbsent( sequenceName, k -> { - final PartitionBucket bucket = Preconditions.checkNotNull( + final Pair pair = Preconditions.checkNotNull( sequenceNameToBucket.get(sequenceName), "Missing bucket for sequence[%s]", sequenceName ); - final Interval interval = bucket.getInterval(); + final Interval interval = pair.lhs; + // TODO: fuck..... i hate this code + final ShardSpec shardSpec = isParallel + ? pair.rhs + : pair.rhs.convert( + intervalToNextPartitionId.computeInt( + interval, + (i, nextPartitionId) -> nextPartitionId == null ? 0 : nextPartitionId + 1 + ) + ); final String version = versionFinder.apply(interval); - return new SegmentIdWithShardSpec( - dataSource, - interval, - version, - bucket.toShardSpec(getPartitionIdAndIncrement(interval)) - ); + return new SegmentIdWithShardSpec(dataSource, interval, version, shardSpec); } ); } - private int getPartitionIdAndIncrement(Interval interval) - { - return intervalToNextPartitionId.computeInt( - interval, - (i, nextPartitionId) -> - Preconditions.checkNotNull(nextPartitionId, "nextPartitionId for interval[%s]", interval) + 1 - ); - } - // TODO: i don't like it public SequenceNameFunction getSequenceNameFunction() { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/NonLinearlyPartitionedSequenceNameFunction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/NonLinearlyPartitionedSequenceNameFunction.java index 82fdebe7cef3..44fd520f913b 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/NonLinearlyPartitionedSequenceNameFunction.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/NonLinearlyPartitionedSequenceNameFunction.java @@ -20,8 +20,7 @@ package org.apache.druid.indexing.common.task; import org.apache.druid.data.input.InputRow; -import org.apache.druid.indexing.common.task.batch.partition.PartitionBucket; -import org.apache.druid.indexing.common.task.batch.partition.PartitionBuckets; +import org.apache.druid.timeline.partition.BucketNumberedShardSpec; import org.joda.time.Interval; /** @@ -35,19 +34,19 @@ public class NonLinearlyPartitionedSequenceNameFunction implements SequenceNameFunction { private final String taskId; - private final PartitionBuckets partitionBuckets; + private final ShardSpecs shardSpecs; - public NonLinearlyPartitionedSequenceNameFunction(String taskId, PartitionBuckets partitionBuckets) + public NonLinearlyPartitionedSequenceNameFunction(String taskId, ShardSpecs shardSpecs) { this.taskId = taskId; - this.partitionBuckets = partitionBuckets; + this.shardSpecs = shardSpecs; } @Override public String getSequenceName(Interval interval, InputRow inputRow) { // Sequence name is based solely on the shardSpec, and there will only be one segment per sequence. - return getSequenceName(interval, partitionBuckets.lookupBucket(inputRow)); + return getSequenceName(interval, shardSpecs.getShardSpec(interval, inputRow)); } /** @@ -55,7 +54,7 @@ public String getSequenceName(Interval interval, InputRow inputRow) * * See {@link org.apache.druid.timeline.partition.HashBasedNumberedShardSpec} as an example of partitioning. */ - public String getSequenceName(Interval interval, PartitionBucket bucket) + public String getSequenceName(Interval interval, BucketNumberedShardSpec bucket) { // Note: We do not use String format here since this can be called in a tight loop // and it's faster to add strings together than it is to use String#format diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/ShardSpecs.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/ShardSpecs.java index 372d87671c95..0d0994634e7f 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/ShardSpecs.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/ShardSpecs.java @@ -1,62 +1,62 @@ -///* -// * 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.java.util.common.ISE; -//import org.apache.druid.java.util.common.granularity.Granularity; -//import org.apache.druid.timeline.partition.ShardSpec; -//import org.joda.time.Interval; -// -//import java.util.List; -//import java.util.Map; -// -///** -// * This class represents a map of (Interval, ShardSpec) and is used for easy shardSpec generation. -// */ -//public class ShardSpecs -//{ -// private final Map> map; -// private Granularity queryGranularity; -// -// ShardSpecs(final Map> map, Granularity queryGranularity) -// { -// this.map = map; -// this.queryGranularity = queryGranularity; -// } -// -// /** -// * Return a shardSpec for the given interval and input row. -// * -// * @param interval interval for shardSpec -// * @param row input row -// * -// * @return a shardSpec -// */ -// ShardSpec getShardSpec(Interval interval, InputRow row) -// { -// final List shardSpecs = map.get(interval); -// if (shardSpecs == null || shardSpecs.isEmpty()) { -// throw new ISE("Failed to get shardSpec for interval[%s]", interval); -// } -// final long truncatedTimestamp = queryGranularity.bucketStart(row.getTimestamp()).getMillis(); -// return shardSpecs.get(0).getLookup(shardSpecs).getShardSpec(truncatedTimestamp, row); -// } -//} +/* + * 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.java.util.common.ISE; +import org.apache.druid.java.util.common.granularity.Granularity; +import org.apache.druid.timeline.partition.BucketNumberedShardSpec; +import org.joda.time.Interval; + +import java.util.List; +import java.util.Map; + +/** + * This class represents a map of (Interval, ShardSpec) and is used for easy shardSpec generation. + */ +public class ShardSpecs +{ + private final Map> map; + private Granularity queryGranularity; + + ShardSpecs(final Map> map, Granularity queryGranularity) + { + this.map = map; + this.queryGranularity = queryGranularity; + } + + /** + * Return a shardSpec for the given interval and input row. + * + * @param interval interval for shardSpec + * @param row input row + * + * @return a shardSpec + */ + BucketNumberedShardSpec getShardSpec(Interval interval, InputRow row) + { + final List shardSpecs = map.get(interval); + if (shardSpecs == null || shardSpecs.isEmpty()) { + throw new ISE("Failed to get shardSpec for interval[%s]", interval); + } + final long truncatedTimestamp = queryGranularity.bucketStart(row.getTimestamp()).getMillis(); + return (BucketNumberedShardSpec) shardSpecs.get(0).getLookup(shardSpecs).getShardSpec(truncatedTimestamp, row); + } +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/GenericPartitionLocation.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/GenericPartitionLocation.java index bbfd1e2ddfa2..74c4c1738b0d 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/GenericPartitionLocation.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/GenericPartitionLocation.java @@ -22,14 +22,14 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonProperty; -import org.apache.druid.timeline.partition.ShardSpec; +import org.apache.druid.timeline.partition.BuildingShardSpec; import org.joda.time.Interval; /** * This class represents the intermediary data server where the partition of {@code interval} and {@code shardSpec} * is stored. */ -public class GenericPartitionLocation extends PartitionLocation +public class GenericPartitionLocation extends PartitionLocation { @JsonCreator public GenericPartitionLocation( @@ -38,7 +38,7 @@ public GenericPartitionLocation( @JsonProperty("useHttps") boolean useHttps, @JsonProperty("subTaskId") String subTaskId, @JsonProperty("interval") Interval interval, - @JsonProperty("shardSpec") ShardSpec shardSpec + @JsonProperty("shardSpec") BuildingShardSpec shardSpec ) { super(host, port, useHttps, subTaskId, interval, shardSpec); @@ -46,13 +46,13 @@ public GenericPartitionLocation( @JsonIgnore @Override - public int getPartitionId() + public int getBucketId() { - return getSecondaryPartition().getPartitionNum(); + return getSecondaryPartition().getBucketId(); } @JsonProperty - ShardSpec getShardSpec() + BuildingShardSpec getShardSpec() { return getSecondaryPartition(); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/GenericPartitionStat.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/GenericPartitionStat.java index 5f4d16db2b19..a4ac80bdec04 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/GenericPartitionStat.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/GenericPartitionStat.java @@ -21,6 +21,7 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.druid.timeline.partition.BucketNumberedShardSpec; import org.apache.druid.timeline.partition.ShardSpec; import org.joda.time.Interval; @@ -33,12 +34,12 @@ * partition key). The {@link ShardSpec} is later used by {@link PartialGenericSegmentMergeTask} to merge the partial * segments. */ -public class GenericPartitionStat extends PartitionStat +public class GenericPartitionStat extends PartitionStat { private static final String PROP_SHARD_SPEC = "shardSpec"; // Secondary partition key - private final ShardSpec shardSpec; + private final BucketNumberedShardSpec shardSpec; @JsonCreator public GenericPartitionStat( @@ -46,7 +47,7 @@ public GenericPartitionStat( @JsonProperty("taskExecutorPort") int taskExecutorPort, @JsonProperty("useHttps") boolean useHttps, @JsonProperty("interval") Interval interval, - @JsonProperty(PROP_SHARD_SPEC) ShardSpec shardSpec, + @JsonProperty(PROP_SHARD_SPEC) BucketNumberedShardSpec shardSpec, @JsonProperty("numRows") @Nullable Integer numRows, @JsonProperty("sizeBytes") @Nullable Long sizeBytes ) @@ -56,14 +57,14 @@ public GenericPartitionStat( } @Override - public int getPartitionId() + public int getBucketId() { - return shardSpec.getPartitionNum(); + return shardSpec.getBucketId(); } @JsonProperty(PROP_SHARD_SPEC) @Override - ShardSpec getSecondaryPartition() + BucketNumberedShardSpec getSecondaryPartition() { return shardSpec; } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionLocation.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionLocation.java index 604eb7a6f504..9ac092fe8ce4 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionLocation.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionLocation.java @@ -44,7 +44,7 @@ public HashPartitionLocation( @JsonProperty @Override - public int getPartitionId() + public int getBucketId() { return getSecondaryPartition(); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionStat.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionStat.java index 21019abe0fe7..03c53db43b2c 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionStat.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionStat.java @@ -54,7 +54,7 @@ public HashPartitionStat( @JsonProperty @Override - public int getPartitionId() + public int getBucketId() { return partitionId; } 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 cb1bc3986510..26625b65afd9 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 @@ -29,6 +29,8 @@ import com.google.common.base.Throwables; import com.google.common.collect.ArrayListMultimap; import com.google.common.collect.Multimap; +import it.unimi.dsi.fastutil.objects.Object2IntMap; +import it.unimi.dsi.fastutil.objects.Object2IntOpenHashMap; import org.apache.druid.client.indexing.IndexingServiceClient; import org.apache.druid.data.input.FiniteFirehoseFactory; import org.apache.druid.data.input.InputFormat; @@ -79,6 +81,7 @@ import org.apache.druid.server.security.AuthorizerMapper; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.partition.BuildingNumberedShardSpec; +import org.apache.druid.timeline.partition.BuildingShardSpec; import org.apache.druid.timeline.partition.NumberedShardSpec; import org.apache.druid.timeline.partition.PartitionBoundaries; import org.apache.druid.utils.CollectionUtils; @@ -544,10 +547,8 @@ private TaskStatus runMultiPhaseParallel(TaskToolbox toolbox) throws Exception private TaskStatus runHashPartitionMultiPhaseParallel(TaskToolbox toolbox) throws Exception { // 1. Partial segment generation phase - ParallelIndexTaskRunner indexingRunner = createRunner( - toolbox, - this::createPartialHashSegmentGenerateRunner - ); + ParallelIndexTaskRunner> indexingRunner + = createRunner(toolbox, this::createPartialHashSegmentGenerateRunner); TaskState state = runNextPhase(indexingRunner); if (state.isFailure()) { @@ -557,16 +558,16 @@ private TaskStatus runHashPartitionMultiPhaseParallel(TaskToolbox toolbox) throw // 2. Partial segment merge phase // partition (interval, partitionId) -> partition locations - Map, List> partitionToLocations = - groupHashPartitionLocationsPerPartition(indexingRunner.getReports()); - final List ioConfigs = createHashMergeIOConfigs( + Map, List> partitionToLocations = + groupGenericPartitionLocationsPerPartition(indexingRunner.getReports()); + final List ioConfigs = createGenericMergeIOConfigs( ingestionSchema.getTuningConfig().getTotalNumMergeTasks(), partitionToLocations ); - final ParallelIndexTaskRunner mergeRunner = createRunner( + final ParallelIndexTaskRunner mergeRunner = createRunner( toolbox, - tb -> createPartialHashSegmentMergeRunner(tb, ioConfigs) + tb -> createPartialGenericSegmentMergeRunner(tb, ioConfigs) ); state = runNextPhase(mergeRunner); if (state.isSuccess()) { @@ -659,38 +660,34 @@ private PartitionBoundaries determineRangePartition(Collection, List> groupHashPartitionLocationsPerPartition( - Map subTaskIdToReport - ) - { - BiFunction createPartitionLocationFunction = - (subtaskId, partitionStat) -> - new HashPartitionLocation( - partitionStat.getTaskExecutorHost(), - partitionStat.getTaskExecutorPort(), - partitionStat.isUseHttps(), - subtaskId, - partitionStat.getInterval(), - partitionStat.getSecondaryPartition() - ); - - return groupPartitionLocationsPerPartition(subTaskIdToReport, createPartitionLocationFunction); - } - private static Map, List> groupGenericPartitionLocationsPerPartition( Map> subTaskIdToReport ) { - BiFunction createPartitionLocationFunction = - (subtaskId, partitionStat) -> - new GenericPartitionLocation( - partitionStat.getTaskExecutorHost(), - partitionStat.getTaskExecutorPort(), - partitionStat.isUseHttps(), - subtaskId, - partitionStat.getInterval(), - partitionStat.getSecondaryPartition() - ); + final Map, BuildingShardSpec> intervalAndIntegerToShardSpec = new HashMap<>(); + final Object2IntMap intervalToNextPartitionId = new Object2IntOpenHashMap<>(); + final BiFunction createPartitionLocationFunction = + (subtaskId, partitionStat) -> { + final BuildingShardSpec shardSpec = intervalAndIntegerToShardSpec.computeIfAbsent( + Pair.of(partitionStat.getInterval(), partitionStat.getBucketId()), + key -> { + // TODO: explain why we determine partition id here.. + final int partitionId = intervalToNextPartitionId.computeInt( + partitionStat.getInterval(), + ((interval, nextPartitionId) -> nextPartitionId == null ? 0 : nextPartitionId + 1) + ); + return partitionStat.getSecondaryPartition().convert(partitionId); + } + ); + return new GenericPartitionLocation( + partitionStat.getTaskExecutorHost(), + partitionStat.getTaskExecutorPort(), + partitionStat.isUseHttps(), + subtaskId, + partitionStat.getInterval(), + shardSpec + ); + }; return groupPartitionLocationsPerPartition(subTaskIdToReport, createPartitionLocationFunction); } @@ -708,7 +705,7 @@ Map, List> groupPartitionLocationsPerPartition( final GeneratedPartitionsReport report = entry.getValue(); for (S partitionStat : report.getPartitionStats()) { final List locationsOfSamePartition = partitionToLocations.computeIfAbsent( - Pair.of(partitionStat.getInterval(), partitionStat.getPartitionId()), + Pair.of(partitionStat.getInterval(), partitionStat.getBucketId()), k -> new ArrayList<>() ); locationsOfSamePartition.add(createPartitionLocationFunction.apply(subTaskId, partitionStat)); @@ -718,18 +715,6 @@ Map, List> groupPartitionLocationsPerPartition( return partitionToLocations; } - private static List createHashMergeIOConfigs( - int totalNumMergeTasks, - Map, List> partitionToLocations - ) - { - return createMergeIOConfigs( - totalNumMergeTasks, - partitionToLocations, - PartialHashSegmentMergeIOConfig::new - ); - } - private static List createGenericMergeIOConfigs( int totalNumMergeTasks, Map, List> partitionToLocations diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialGenericSegmentMergeTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialGenericSegmentMergeTask.java index 858eff4ae63d..fed80d9252f8 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialGenericSegmentMergeTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialGenericSegmentMergeTask.java @@ -29,6 +29,8 @@ import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.common.task.IndexTaskClientFactory; import org.apache.druid.indexing.common.task.TaskResource; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.timeline.partition.BuildingShardSpec; import org.apache.druid.timeline.partition.ShardSpec; import org.joda.time.Interval; @@ -39,12 +41,12 @@ /** * {@link ParallelIndexTaskRunner} for the phase to merge generic partitioned segments in multi-phase parallel indexing. */ -public class PartialGenericSegmentMergeTask extends PartialSegmentMergeTask +public class PartialGenericSegmentMergeTask extends PartialSegmentMergeTask { public static final String TYPE = "partial_index_generic_merge"; private final PartialGenericSegmentMergeIngestionSpec ingestionSchema; - private final Table intervalAndIntegerToShardSpec; + private final Table> intervalAndIntegerToShardSpec; @JsonCreator public PartialGenericSegmentMergeTask( @@ -82,24 +84,28 @@ public PartialGenericSegmentMergeTask( ); } - private static Table createIntervalAndIntegerToShardSpec( + private static Table> createIntervalAndIntegerToShardSpec( List partitionLocations ) { - Table intervalAndIntegerToShardSpec = HashBasedTable.create(); + final Table> intervalAndIntegerToShardSpec = HashBasedTable.create(); partitionLocations.forEach( p -> { - ShardSpec currShardSpec = intervalAndIntegerToShardSpec.get(p.getInterval(), p.getPartitionId()); - Preconditions.checkArgument( - currShardSpec == null || p.getShardSpec().equals(currShardSpec), - "interval %s, partitionId %s mismatched shard specs: %s", - p.getInterval(), - p.getPartitionId(), - partitionLocations - ); - - intervalAndIntegerToShardSpec.put(p.getInterval(), p.getPartitionId(), p.getShardSpec()); + final ShardSpec currShardSpec = intervalAndIntegerToShardSpec.get(p.getInterval(), p.getBucketId()); + if (currShardSpec == null) { + intervalAndIntegerToShardSpec.put(p.getInterval(), p.getBucketId(), p.getShardSpec()); + } else { + if (!p.getShardSpec().equals(currShardSpec)) { + throw new ISE( + "interval %s, bucketId %s mismatched shard specs: %s and %s", + p.getInterval(), + p.getBucketId(), + currShardSpec, + p.getShardSpec() + ); + } + } } ); @@ -119,7 +125,7 @@ public String getType() } @Override - ShardSpec createShardSpec(TaskToolbox toolbox, Interval interval, int partitionId) + BuildingShardSpec createShardSpec(TaskToolbox toolbox, Interval interval, int partitionId) { return Preconditions.checkNotNull( intervalAndIntegerToShardSpec.get(interval, partitionId), diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentGenerateParallelIndexTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentGenerateParallelIndexTaskRunner.java index ef8869c42727..29f742a74dd8 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentGenerateParallelIndexTaskRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentGenerateParallelIndexTaskRunner.java @@ -31,7 +31,7 @@ * @see PartialHashSegmentMergeParallelIndexTaskRunner */ class PartialHashSegmentGenerateParallelIndexTaskRunner - extends InputSourceSplitParallelIndexTaskRunner + extends InputSourceSplitParallelIndexTaskRunner> { private static final String PHASE_NAME = "partial segment generation"; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentGenerateTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentGenerateTask.java index 599f2470df09..61d62db32655 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentGenerateTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentGenerateTask.java @@ -35,6 +35,7 @@ import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager; import org.apache.druid.segment.realtime.appenderator.SegmentAllocator; import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.partition.BucketNumberedShardSpec; import org.apache.druid.timeline.partition.PartialShardSpec; import org.joda.time.Interval; @@ -51,7 +52,7 @@ * hashing the segment granularity and partition dimensions in {@link HashedPartitionsSpec}. Partitioned segments are * stored in local storage using {@link org.apache.druid.indexing.worker.ShuffleDataSegmentPusher}. */ -public class PartialHashSegmentGenerateTask extends PartialSegmentGenerateTask +public class PartialHashSegmentGenerateTask extends PartialSegmentGenerateTask { public static final String TYPE = "partial_index_generate"; private static final String PROP_SPEC = "spec"; @@ -144,22 +145,22 @@ SegmentAllocator createSegmentAllocator(TaskToolbox toolbox, ParallelIndexSuperv } @Override - GeneratedHashPartitionsReport createGeneratedPartitionsReport(TaskToolbox toolbox, List segments) + GeneratedPartitionsMetadataReport createGeneratedPartitionsReport(TaskToolbox toolbox, List segments) { - List partitionStats = segments.stream() - .map(segment -> createPartitionStat(toolbox, segment)) - .collect(Collectors.toList()); - return new GeneratedHashPartitionsReport(getId(), partitionStats); + List partitionStats = segments.stream() + .map(segment -> createPartitionStat(toolbox, segment)) + .collect(Collectors.toList()); + return new GeneratedPartitionsMetadataReport(getId(), partitionStats); } - private HashPartitionStat createPartitionStat(TaskToolbox toolbox, DataSegment segment) + private GenericPartitionStat createPartitionStat(TaskToolbox toolbox, DataSegment segment) { - return new HashPartitionStat( + return new GenericPartitionStat( toolbox.getTaskExecutorNode().getHost(), toolbox.getTaskExecutorNode().getPortToUse(), toolbox.getTaskExecutorNode().isEnableTlsPort(), segment.getInterval(), - segment.getShardSpec().getPartitionNum(), + (BucketNumberedShardSpec) segment.getShardSpec(), null, // numRows is not supported yet null // sizeBytes is not supported yet ); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialRangeSegmentGenerateTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialRangeSegmentGenerateTask.java index 33b073d44f15..c58bc5bc5eca 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialRangeSegmentGenerateTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialRangeSegmentGenerateTask.java @@ -37,6 +37,7 @@ import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager; import org.apache.druid.segment.realtime.appenderator.SegmentAllocator; import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.partition.BucketNumberedShardSpec; import org.apache.druid.timeline.partition.PartitionBoundaries; import org.joda.time.Interval; @@ -183,7 +184,7 @@ private GenericPartitionStat createPartitionStat(TaskToolbox toolbox, DataSegmen toolbox.getTaskExecutorNode().getPortToUse(), toolbox.getTaskExecutorNode().isEnableTlsPort(), segment.getInterval(), - segment.getShardSpec(), + (BucketNumberedShardSpec) segment.getShardSpec(), null, // numRows is not supported yet null // sizeBytes is not supported yet ); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentMergeTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentMergeTask.java index 96596be4dc73..b105d5e8da09 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentMergeTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentMergeTask.java @@ -139,10 +139,10 @@ public boolean isReady(TaskActionClient taskActionClient) public TaskStatus runTask(TaskToolbox toolbox) throws Exception { // Group partitionLocations by interval and partitionId - final Map>> intervalToPartitions = new HashMap<>(); + final Map>> intervalToBuckets = new HashMap<>(); for (P location : ioConfig.getPartitionLocations()) { - intervalToPartitions.computeIfAbsent(location.getInterval(), k -> new Int2ObjectOpenHashMap<>()) - .computeIfAbsent(location.getPartitionId(), k -> new ArrayList<>()) + intervalToBuckets.computeIfAbsent(location.getInterval(), k -> new Int2ObjectOpenHashMap<>()) + .computeIfAbsent(location.getBucketId(), k -> new ArrayList<>()) .add(location); } @@ -168,7 +168,7 @@ public TaskStatus runTask(TaskToolbox toolbox) throws Exception final Stopwatch fetchStopwatch = Stopwatch.createStarted(); final Map>> intervalToUnzippedFiles = fetchSegmentFiles( toolbox, - intervalToPartitions + intervalToBuckets ); final long fetchTime = fetchStopwatch.elapsed(TimeUnit.SECONDS); fetchStopwatch.stop(); @@ -202,7 +202,7 @@ public TaskStatus runTask(TaskToolbox toolbox) throws Exception private Map>> fetchSegmentFiles( TaskToolbox toolbox, - Map>> intervalToPartitions + Map>> intervalToBuckets ) throws IOException { final File tempDir = toolbox.getIndexingTmpDir(); @@ -211,26 +211,26 @@ private Map>> fetchSegmentFiles( final Map>> intervalToUnzippedFiles = new HashMap<>(); // Fetch partition files - for (Entry>> entryPerInterval : intervalToPartitions.entrySet()) { + for (Entry>> entryPerInterval : intervalToBuckets.entrySet()) { final Interval interval = entryPerInterval.getKey(); - for (Int2ObjectMap.Entry> entryPerPartitionId : + for (Int2ObjectMap.Entry> entryPerBucketId : entryPerInterval.getValue().int2ObjectEntrySet()) { - final int partitionId = entryPerPartitionId.getIntKey(); + final int bucketId = entryPerBucketId.getIntKey(); final File partitionDir = FileUtils.getFile( tempDir, interval.getStart().toString(), interval.getEnd().toString(), - Integer.toString(partitionId) + Integer.toString(bucketId) ); FileUtils.forceMkdir(partitionDir); - for (P location : entryPerPartitionId.getValue()) { + for (P location : entryPerBucketId.getValue()) { final File zippedFile = shuffleClient.fetchSegmentFile(partitionDir, supervisorTaskId, location); try { final File unzippedDir = new File(partitionDir, StringUtils.format("unzipped_%s", location.getSubTaskId())); FileUtils.forceMkdir(unzippedDir); CompressionUtils.unzip(zippedFile, unzippedDir); intervalToUnzippedFiles.computeIfAbsent(interval, k -> new Int2ObjectOpenHashMap<>()) - .computeIfAbsent(partitionId, k -> new ArrayList<>()) + .computeIfAbsent(bucketId, k -> new ArrayList<>()) .add(unzippedDir); } finally { @@ -247,7 +247,7 @@ private Map>> fetchSegmentFiles( /** * Create a {@link ShardSpec} suitable for the desired secondary partitioning strategy. */ - abstract S createShardSpec(TaskToolbox toolbox, Interval interval, int partitionId); + abstract S createShardSpec(TaskToolbox toolbox, Interval interval, int bucketId); private Set mergeAndPushSegments( TaskToolbox toolbox, @@ -262,9 +262,9 @@ private Set mergeAndPushSegments( final Set pushedSegments = new HashSet<>(); for (Entry>> entryPerInterval : intervalToUnzippedFiles.entrySet()) { final Interval interval = entryPerInterval.getKey(); - for (Int2ObjectMap.Entry> entryPerPartitionId : entryPerInterval.getValue().int2ObjectEntrySet()) { - final int partitionId = entryPerPartitionId.getIntKey(); - final List segmentFilesToMerge = entryPerPartitionId.getValue(); + for (Int2ObjectMap.Entry> entryPerBucketId : entryPerInterval.getValue().int2ObjectEntrySet()) { + final int bucketId = entryPerBucketId.getIntKey(); + final List segmentFilesToMerge = entryPerBucketId.getValue(); final Pair> mergedFileAndDimensionNames = mergeSegmentsInSamePartition( dataSchema, tuningConfig, @@ -290,7 +290,7 @@ private Set mergeAndPushSegments( null, // will be filled in the segmentPusher mergedFileAndDimensionNames.rhs, metricNames, - createShardSpec(toolbox, interval, partitionId), + createShardSpec(toolbox, interval, bucketId), null, // will be filled in the segmentPusher 0 // will be filled in the segmentPusher ), diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartitionLocation.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartitionLocation.java index e6578c50924f..da382cec0604 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartitionLocation.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartitionLocation.java @@ -29,7 +29,7 @@ /** * This class represents the intermediary data server where the partition of {@link #interval} and - * {@link #getPartitionId()} is stored. + * {@link #getBucketId()} is stored. */ abstract class PartitionLocation { @@ -93,13 +93,13 @@ public T getSecondaryPartition() return secondaryPartition; } - abstract int getPartitionId(); + abstract int getBucketId(); final URI toIntermediaryDataServerURI(String supervisorTaskId) { return URI.create( StringUtils.format( - "%s://%s:%d/druid/worker/v1/shuffle/task/%s/%s/partition?startTime=%s&endTime=%s&partitionId=%d", + "%s://%s:%d/druid/worker/v1/shuffle/task/%s/%s/partition?startTime=%s&endTime=%s&bucketId=%d", useHttps ? "https" : "http", host, port, @@ -107,7 +107,7 @@ final URI toIntermediaryDataServerURI(String supervisorTaskId) StringUtils.urlEncode(subTaskId), interval.getStart(), interval.getEnd(), - getPartitionId() + getBucketId() ) ); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartitionStat.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartitionStat.java index 66974c297eea..c7f1a55c2a2e 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartitionStat.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartitionStat.java @@ -104,7 +104,7 @@ public final Long getSizeBytes() /** * @return Uniquely identifying index from 0..N-1 of the N partitions */ - abstract int getPartitionId(); + abstract int getBucketId(); /** * @return Definition of secondary partition. For example, for range partitioning, this should include the start/end. diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/partition/CompletePartitionAnalysis.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/partition/CompletePartitionAnalysis.java index 761bd522226f..1bc6e9b60840 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/partition/CompletePartitionAnalysis.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/partition/CompletePartitionAnalysis.java @@ -21,12 +21,11 @@ import org.apache.druid.indexer.partitions.PartitionsSpec; import org.apache.druid.indexing.common.TaskToolbox; -import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; +import org.apache.druid.timeline.partition.BucketNumberedShardSpec; import org.joda.time.Interval; import java.util.List; import java.util.Map; -import java.util.function.Function; /** * This interface represents the PartitionAnalysis that has the complete picture of secondary partitions to create. @@ -35,12 +34,6 @@ */ public interface CompletePartitionAnalysis extends PartitionAnalysis { -// Map> convertToIntervalToSegmentIds( -// TaskToolbox toolbox, -// String dataSource, -// Function versionFinder -// ); - // TODO: strongly typed - Map createBuckets(TaskToolbox toolbox); + Map> createBuckets(TaskToolbox toolbox); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/partition/HashBucket.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/partition/HashBucket.java deleted file mode 100644 index 81bf5c0c6283..000000000000 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/partition/HashBucket.java +++ /dev/null @@ -1,80 +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.batch.partition; - -import com.fasterxml.jackson.databind.ObjectMapper; -import org.apache.druid.timeline.partition.BuildingHashBasedNumberedShardSpec; -import org.apache.druid.timeline.partition.ShardSpec; -import org.joda.time.Interval; - -import java.util.List; - -public class HashBucket implements PartitionBucket -{ - private final Interval interval; - private final int bucketId; - private final int numBuckets; - private final List partitionDimensions; - private final ObjectMapper jsonMapper; - - public HashBucket( - Interval interval, - int bucketId, - int numBuckets, - List partitionDimensions, - ObjectMapper jsonMapper - ) - { - this.interval = interval; - this.bucketId = bucketId; - this.numBuckets = numBuckets; - this.partitionDimensions = partitionDimensions; - this.jsonMapper = jsonMapper; - } - - @Override - public Interval getInterval() - { - return interval; - } - - @Override - public int getBucketId() - { - return bucketId; - } - - @Override - public ShardSpec toShardSpec(int partitionId) - { - return new BuildingHashBasedNumberedShardSpec(partitionId, bucketId, numBuckets, partitionDimensions, jsonMapper); - } - - @Override - public String toString() - { - return "HashBucket{" + - "interval=" + interval + - ", bucketId=" + bucketId + - ", numBuckets=" + numBuckets + - ", partitionDimensions=" + partitionDimensions + - '}'; - } -} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/partition/HashBucketLookup.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/partition/HashBucketLookup.java deleted file mode 100644 index 5789ef9e6531..000000000000 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/partition/HashBucketLookup.java +++ /dev/null @@ -1,77 +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.batch.partition; - -import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.base.Preconditions; -import it.unimi.dsi.fastutil.ints.Int2ObjectMap; -import it.unimi.dsi.fastutil.ints.Int2ObjectOpenHashMap; -import org.apache.druid.data.input.InputRow; -import org.apache.druid.java.util.common.ISE; -import org.apache.druid.timeline.partition.HashBasedNumberedShardSpec; - -import java.util.Collection; -import java.util.Iterator; -import java.util.List; - -public class HashBucketLookup implements PartitionBucketLookup -{ - private final ObjectMapper jsonMapper; - private final List partitionDimensions; - private final int numBuckets; - private final Int2ObjectMap bucketIdToBucket; - - public HashBucketLookup( - ObjectMapper jsonMapper, - List partitionDimensions, - Collection buckets - ) - { - this.jsonMapper = jsonMapper; - this.partitionDimensions = partitionDimensions; - this.numBuckets = buckets.size(); - bucketIdToBucket = new Int2ObjectOpenHashMap<>(buckets.size()); - buckets.forEach(bucket -> { - final HashBucket old = bucketIdToBucket.put(bucket.getBucketId(), bucket); - if (old != null) { - throw new ISE("Duplicate bucketId in bucket[%s] and bucket[%s]", bucket, old); - } - }); - } - - @Override - public HashBucket find(long timestamp, InputRow row) - { - final int hash = - Math.abs(HashBasedNumberedShardSpec.hash(jsonMapper, partitionDimensions, timestamp, row) % numBuckets); - return Preconditions.checkNotNull( - bucketIdToBucket.get(hash), - "Cannot find bucket for timestamp[%s] and row[%s]", - timestamp, - row - ); - } - - @Override - public Iterator iterator() - { - return bucketIdToBucket.values().iterator(); - } -} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/partition/HashPartitionAnalysis.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/partition/HashPartitionAnalysis.java index 5b86db121d13..ab37276c7f7c 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/partition/HashPartitionAnalysis.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/partition/HashPartitionAnalysis.java @@ -23,7 +23,8 @@ import org.apache.druid.indexer.partitions.HashedPartitionsSpec; import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.java.util.common.IAE; -import org.apache.druid.utils.CollectionUtils; +import org.apache.druid.timeline.partition.BucketNumberedShardSpec; +import org.apache.druid.timeline.partition.HashBucketShardSpec; import org.joda.time.Interval; import java.util.Collections; @@ -90,68 +91,23 @@ public void forEach(BiConsumer consumer) } @Override - public Map createBuckets(TaskToolbox toolbox) + public Map> createBuckets(TaskToolbox toolbox) { - final Map intervalToLookup = Maps.newHashMapWithExpectedSize( + final Map> intervalToLookup = Maps.newHashMapWithExpectedSize( intervalToNumBuckets.size() ); forEach((interval, numBuckets) -> { - final List buckets = IntStream + final List buckets = IntStream .range(0, numBuckets) - .mapToObj(i -> new HashBucket( - interval, + .mapToObj(i -> new HashBucketShardSpec( i, numBuckets, partitionsSpec.getPartitionDimensions(), toolbox.getJsonMapper() )) .collect(Collectors.toList()); - intervalToLookup.put( - interval, - new HashBucketLookup( - toolbox.getJsonMapper(), - partitionsSpec.getPartitionDimensions(), - buckets - ) - ); + intervalToLookup.put(interval, buckets); }); return intervalToLookup; } - -// @Override -// public Map> convertToIntervalToSegmentIds( -// TaskToolbox toolbox, -// String dataSource, -// Function versionFinder -// ) -// { -// final Map> intervalToSegmentIds = -// Maps.newHashMapWithExpectedSize(getNumTimePartitions()); -// -// forEach((interval, numBuckets) -> { -// intervalToSegmentIds.put( -// interval, -// IntStream.range(0, numBuckets) -// .mapToObj(i -> { -// final HashBucket bucket = new HashBucket( -// i, -// numBuckets, -// partitionsSpec.getPartitionDimensions(), -// toolbox.getJsonMapper() -// ); -// return new SegmentIdWithShardSpec( -// dataSource, -// interval, -// versionFinder.apply(interval), -// shardSpec -// ); -// }) -// .collect(Collectors.toList()) -// ); -// }); -// -// return intervalToSegmentIds; -// } - - } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/partition/PartitionBucket.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/partition/PartitionBucket.java deleted file mode 100644 index 640781f253f7..000000000000 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/partition/PartitionBucket.java +++ /dev/null @@ -1,34 +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.batch.partition; - -import org.apache.druid.timeline.partition.ShardSpec; -import org.joda.time.Interval; - -public interface PartitionBucket -{ - Interval getInterval(); - - int getBucketId(); - - ShardSpec toShardSpec(int partitionId); - - // TODO: lookup -} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/partition/PartitionBucketLookup.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/partition/PartitionBucketLookup.java deleted file mode 100644 index 1217257f1dad..000000000000 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/partition/PartitionBucketLookup.java +++ /dev/null @@ -1,37 +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.batch.partition; - -import org.apache.druid.data.input.InputRow; - -import java.util.Iterator; - -public interface PartitionBucketLookup -{ - /** - * Returns a {@link PartitionBucket} for the given timestamp and the inputRow. - * - * @param timestamp timestamp of the row bucketed using {@code GranularitySpec#getQueryGranularity} - * @param row input row - */ - T find(long timestamp, InputRow row); - - Iterator iterator(); -} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/partition/PartitionBuckets.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/partition/PartitionBuckets.java deleted file mode 100644 index c5eb6230e3aa..000000000000 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/partition/PartitionBuckets.java +++ /dev/null @@ -1,58 +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.batch.partition; - -import com.google.common.base.Optional; -import org.apache.druid.data.input.InputRow; -import org.apache.druid.java.util.common.ISE; -import org.apache.druid.java.util.common.granularity.Granularity; -import org.apache.druid.segment.indexing.granularity.GranularitySpec; -import org.joda.time.Interval; - -import java.util.Map; - -public class PartitionBuckets -{ - private final GranularitySpec granularitySpec; - private final Granularity queryGranularity; - private final Map intervalToBuckets; - - public PartitionBuckets(GranularitySpec granularitySpec, Map intervalToBuckets) - { - this.granularitySpec = granularitySpec; - this.queryGranularity = granularitySpec.getQueryGranularity(); - this.intervalToBuckets = intervalToBuckets; - } - - public PartitionBucket lookupBucket(InputRow row) - { - final Optional optInterval = granularitySpec.bucketInterval(row.getTimestamp()); - if (!optInterval.isPresent()) { - throw new ISE("Cannot compute a bucketed interval for row[%s]", row); - } - final Interval interval = optInterval.get(); - final PartitionBucketLookup bucketLookup = intervalToBuckets.get(interval); - if (bucketLookup == null) { - throw new ISE("Cannot find partitionBucketLookup for row[%s], bucketed interval[%s]", row, interval); - } - final long bucketedTimestamp = queryGranularity.bucketStart(row.getTimestamp()).getMillis(); - return bucketLookup.find(bucketedTimestamp, row); - } -} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/partition/RangeBucket.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/partition/RangeBucket.java deleted file mode 100644 index c856c58ad222..000000000000 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/partition/RangeBucket.java +++ /dev/null @@ -1,109 +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.batch.partition; - -import org.apache.druid.data.input.InputRow; -import org.apache.druid.timeline.partition.BuildingSingleDimensionShardSpec; -import org.apache.druid.timeline.partition.ShardSpec; -import org.joda.time.Interval; - -import javax.annotation.Nullable; -import java.util.List; - -public class RangeBucket implements PartitionBucket -{ - private final Interval interval; - private final int bucketId; - private final String dimension; - @Nullable - private final String start; - @Nullable - private final String end; - - public RangeBucket( - Interval interval, - int bucketId, - String dimension, - @Nullable String start, - @Nullable String end - ) - { - this.interval = interval; - this.bucketId = bucketId; - this.dimension = dimension; - this.start = start; - this.end = end; - } - - @Override - public Interval getInterval() - { - return interval; - } - - @Override - public int getBucketId() - { - return bucketId; - } - - @Override - public ShardSpec toShardSpec(int partitionId) - { - return new BuildingSingleDimensionShardSpec(dimension, start, end, partitionId); - } - - public boolean isInBucket(long timestamp, InputRow row) - { - final List values = row.getDimension(dimension); - - if (values == null || values.size() != 1) { - return checkValue(null); - } else { - return checkValue(values.get(0)); - } - } - - private boolean checkValue(String value) - { - if (value == null) { - return start == null; - } - - if (start == null) { - return end == null || value.compareTo(end) < 0; - } - - return value.compareTo(start) >= 0 && - (end == null || value.compareTo(end) < 0); - } - - @Override - public String toString() - { - return "RangeBucket{" + - "interval=" + interval + - ", bucketId=" + bucketId + - ", dimension='" + dimension + '\'' + - ", start='" + start + '\'' + - ", end='" + end + '\'' + - '}'; - } -} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/partition/RangePartitionAnalysis.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/partition/RangePartitionAnalysis.java index 031d15565684..3d545b4c9ec7 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/partition/RangePartitionAnalysis.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/partition/RangePartitionAnalysis.java @@ -23,7 +23,9 @@ import org.apache.druid.indexer.partitions.SingleDimensionPartitionsSpec; import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.java.util.common.IAE; +import org.apache.druid.timeline.partition.BucketNumberedShardSpec; import org.apache.druid.timeline.partition.PartitionBoundaries; +import org.apache.druid.timeline.partition.RangeBucketShardSpec; import org.joda.time.Interval; import java.util.Collections; @@ -90,8 +92,7 @@ public int getNumTimePartitions() * Translate {@link PartitionBoundaries} into the corresponding * {@link SingleDimensionPartitionsSpec} with segment id. */ - private static List translatePartitionBoundaries( - Interval interval, + private static List translatePartitionBoundaries( String partitionDimension, PartitionBoundaries partitionBoundaries ) @@ -101,8 +102,7 @@ private static List translatePartitionBoundaries( } return IntStream.range(0, partitionBoundaries.size() - 1) - .mapToObj(i -> new RangeBucket( - interval, + .mapToObj(i -> new RangeBucketShardSpec( i, partitionDimension, partitionBoundaries.get(i), @@ -112,51 +112,20 @@ private static List translatePartitionBoundaries( } @Override - public Map createBuckets(TaskToolbox toolbox) + public Map> createBuckets(TaskToolbox toolbox) { final String partitionDimension = partitionsSpec.getPartitionDimension(); - final Map intervalToSegmentIds = Maps.newHashMapWithExpectedSize( + final Map> intervalToSegmentIds = Maps.newHashMapWithExpectedSize( getNumTimePartitions() ); forEach((interval, partitionBoundaries) -> intervalToSegmentIds.put( interval, - new RangeBucketLookup( - translatePartitionBoundaries( - interval, - partitionDimension, - partitionBoundaries - ) - ) + translatePartitionBoundaries(partitionDimension, partitionBoundaries) ) ); return intervalToSegmentIds; } - -// private static SegmentIdWithShardSpec createSegmentIdWithShardSpec( -// String dataSource, -// Interval interval, -// String version, -// String partitionDimension, -// String partitionStart, -// @Nullable String partitionEnd, -// int partitionNum -// ) -// { -// // The shardSpec created here will be reused in PartialGenericSegmentMergeTask. This is ok because -// // all PartialSegmentGenerateTasks create the same set of segmentIds (and thus shardSpecs). -// return new SegmentIdWithShardSpec( -// dataSource, -// interval, -// version, -// new SingleDimensionShardSpec( -// partitionDimension, -// partitionStart, -// partitionEnd, -// partitionNum -// ) -// ); -// } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/worker/IntermediaryDataManager.java b/indexing-service/src/main/java/org/apache/druid/indexing/worker/IntermediaryDataManager.java index 78090ca5181f..6df598c24031 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/worker/IntermediaryDataManager.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/worker/IntermediaryDataManager.java @@ -70,7 +70,7 @@ * and phase 2 tasks read those files via HTTP. * * The directory where segment files are placed is structured as - * {@link StorageLocation#path}/supervisorTaskId/startTimeOfSegment/endTimeOfSegment/partitionIdOfSegment. + * {@link StorageLocation#path}/supervisorTaskId/startTimeOfSegment/endTimeOfSegment/bucketIdOfSegment. * * This class provides interfaces to store, find, and remove segment files. * It also has a self-cleanup mechanism to clean up stale segment files. It periodically checks the last access time @@ -335,11 +335,11 @@ long addSegment(String supervisorTaskId, String subTaskId, DataSegment segment, } @Nullable - public File findPartitionFile(String supervisorTaskId, String subTaskId, Interval interval, int partitionId) + public File findPartitionFile(String supervisorTaskId, String subTaskId, Interval interval, int bucketId) { TaskIdUtils.validateId("supervisorTaskId", supervisorTaskId); for (StorageLocation location : shuffleDataLocations) { - final File partitionDir = new File(location.getPath(), getPartitionDir(supervisorTaskId, interval, partitionId)); + final File partitionDir = new File(location.getPath(), getPartitionDir(supervisorTaskId, interval, bucketId)); if (partitionDir.exists()) { supervisorTaskCheckTimes.put(supervisorTaskId, getExpiryTimeFromNow()); final File[] segmentFiles = partitionDir.listFiles(); @@ -384,23 +384,23 @@ private static String getPartitionFilePath( String supervisorTaskId, String subTaskId, Interval interval, - int partitionId + int bucketId ) { - return Paths.get(getPartitionDir(supervisorTaskId, interval, partitionId), subTaskId).toString(); + return Paths.get(getPartitionDir(supervisorTaskId, interval, bucketId), subTaskId).toString(); } private static String getPartitionDir( String supervisorTaskId, Interval interval, - int partitionId + int bucketId ) { return Paths.get( supervisorTaskId, interval.getStart().toString(), interval.getEnd().toString(), - String.valueOf(partitionId) + String.valueOf(bucketId) ).toString(); } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/worker/http/ShuffleResource.java b/indexing-service/src/main/java/org/apache/druid/indexing/worker/http/ShuffleResource.java index d1adcb9f2d22..0e0e9364e211 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/worker/http/ShuffleResource.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/worker/http/ShuffleResource.java @@ -75,7 +75,7 @@ public Response getPartition( @PathParam("subTaskId") String subTaskId, @QueryParam("startTime") String startTime, @QueryParam("endTime") String endTime, - @QueryParam("partitionId") int partitionId + @QueryParam("bucketId") int bucketId ) { final Interval interval = new Interval(DateTimes.of(startTime), DateTimes.of(endTime)); @@ -83,16 +83,16 @@ public Response getPartition( supervisorTaskId, subTaskId, interval, - partitionId + bucketId ); if (partitionFile == null) { final String errorMessage = StringUtils.format( - "Can't find the partition for supervisorTask[%s], subTask[%s], interval[%s], and partitionId[%s]", + "Can't find the partition for supervisorTask[%s], subTask[%s], interval[%s], and bucketId[%s]", supervisorTaskId, subTaskId, interval, - partitionId + bucketId ); return Response.status(Status.NOT_FOUND).entity(errorMessage).build(); } else { diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ShardSpecsTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ShardSpecsTest.java index 6755c8abaad5..ae681c17d6bc 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ShardSpecsTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ShardSpecsTest.java @@ -1,87 +1,89 @@ -///* -// * 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.fasterxml.jackson.databind.ObjectMapper; -//import com.google.common.collect.ImmutableMap; -//import com.google.common.collect.Lists; -//import org.apache.druid.data.input.InputRow; -//import org.apache.druid.data.input.MapBasedInputRow; -//import org.apache.druid.indexing.common.TestUtils; -//import org.apache.druid.java.util.common.DateTimes; -//import org.apache.druid.java.util.common.Intervals; -//import org.apache.druid.java.util.common.granularity.Granularities; -//import org.apache.druid.timeline.partition.HashBasedNumberedShardSpec; -//import org.apache.druid.timeline.partition.ShardSpec; -//import org.joda.time.Interval; -//import org.junit.Assert; -//import org.junit.Test; -// -//import java.util.HashMap; -//import java.util.List; -//import java.util.Map; -// -//public class ShardSpecsTest extends IngestionTestBase -//{ -// private final TestUtils testUtils = new TestUtils(); -// private final ObjectMapper jsonMapper = testUtils.getTestObjectMapper(); -// -// public ShardSpecsTest() -// { -// } -// -// @Test -// public void testShardSpecSelectionWithNullPartitionDimension() -// { -// ShardSpec spec1 = new HashBasedNumberedShardSpec(0, 2, 2, null, jsonMapper); -// ShardSpec spec2 = new HashBasedNumberedShardSpec(1, 2, 2, null, jsonMapper); -// -// Map> shardSpecMap = new HashMap<>(); -// shardSpecMap.put(Intervals.of("2014-01-01T00:00:00.000Z/2014-01-02T00:00:00.000Z"), Lists.newArrayList(spec1, spec2)); -// -// ShardSpecs shardSpecs = new ShardSpecs(shardSpecMap, Granularities.HOUR); -// String visitorId = "visitorId"; -// String clientType = "clientType"; -// long timestamp1 = DateTimes.of("2014-01-01T00:00:00.000Z").getMillis(); -// InputRow row1 = new MapBasedInputRow(timestamp1, -// Lists.newArrayList(visitorId, clientType), -// ImmutableMap.of(visitorId, "0", clientType, "iphone") -// ); -// -// long timestamp2 = DateTimes.of("2014-01-01T00:30:20.456Z").getMillis(); -// InputRow row2 = new MapBasedInputRow(timestamp2, -// Lists.newArrayList(visitorId, clientType), -// ImmutableMap.of(visitorId, "0", clientType, "iphone") -// ); -// -// long timestamp3 = DateTimes.of("2014-01-01T10:10:20.456Z").getMillis(); -// InputRow row3 = new MapBasedInputRow(timestamp3, -// Lists.newArrayList(visitorId, clientType), -// ImmutableMap.of(visitorId, "0", clientType, "iphone") -// ); -// -// ShardSpec spec3 = shardSpecs.getShardSpec(Intervals.of("2014-01-01T00:00:00.000Z/2014-01-02T00:00:00.000Z"), row1); -// ShardSpec spec4 = shardSpecs.getShardSpec(Intervals.of("2014-01-01T00:00:00.000Z/2014-01-02T00:00:00.000Z"), row2); -// ShardSpec spec5 = shardSpecs.getShardSpec(Intervals.of("2014-01-01T00:00:00.000Z/2014-01-02T00:00:00.000Z"), row3); -// -// Assert.assertSame(true, spec3 == spec4); -// Assert.assertSame(false, spec3 == spec5); -// } -//} +/* + * 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.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Lists; +import org.apache.druid.data.input.InputRow; +import org.apache.druid.data.input.MapBasedInputRow; +import org.apache.druid.indexing.common.TestUtils; +import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.granularity.Granularities; +import org.apache.druid.timeline.partition.BucketNumberedShardSpec; +import org.apache.druid.timeline.partition.HashBucketShardSpec; +import org.apache.druid.timeline.partition.ShardSpec; +import org.joda.time.Interval; +import org.junit.Assert; +import org.junit.Test; + +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +public class ShardSpecsTest extends IngestionTestBase +{ + private final TestUtils testUtils = new TestUtils(); + private final ObjectMapper jsonMapper = testUtils.getTestObjectMapper(); + + public ShardSpecsTest() + { + } + + @Test + public void testShardSpecSelectionWithNullPartitionDimension() + { + BucketNumberedShardSpec spec1 = new HashBucketShardSpec(0, 2, null, jsonMapper); + BucketNumberedShardSpec spec2 = new HashBucketShardSpec(1, 2, null, jsonMapper); + + Map> shardSpecMap = new HashMap<>(); + shardSpecMap.put(Intervals.of("2014-01-01T00:00:00.000Z/2014-01-02T00:00:00.000Z"), ImmutableList.of(spec1, spec2)); + + ShardSpecs shardSpecs = new ShardSpecs(shardSpecMap, Granularities.HOUR); + String visitorId = "visitorId"; + String clientType = "clientType"; + long timestamp1 = DateTimes.of("2014-01-01T00:00:00.000Z").getMillis(); + InputRow row1 = new MapBasedInputRow(timestamp1, + Lists.newArrayList(visitorId, clientType), + ImmutableMap.of(visitorId, "0", clientType, "iphone") + ); + + long timestamp2 = DateTimes.of("2014-01-01T00:30:20.456Z").getMillis(); + InputRow row2 = new MapBasedInputRow(timestamp2, + Lists.newArrayList(visitorId, clientType), + ImmutableMap.of(visitorId, "0", clientType, "iphone") + ); + + long timestamp3 = DateTimes.of("2014-01-01T10:10:20.456Z").getMillis(); + InputRow row3 = new MapBasedInputRow(timestamp3, + Lists.newArrayList(visitorId, clientType), + ImmutableMap.of(visitorId, "0", clientType, "iphone") + ); + + ShardSpec spec3 = shardSpecs.getShardSpec(Intervals.of("2014-01-01T00:00:00.000Z/2014-01-02T00:00:00.000Z"), row1); + ShardSpec spec4 = shardSpecs.getShardSpec(Intervals.of("2014-01-01T00:00:00.000Z/2014-01-02T00:00:00.000Z"), row2); + ShardSpec spec5 = shardSpecs.getShardSpec(Intervals.of("2014-01-01T00:00:00.000Z/2014-01-02T00:00:00.000Z"), row3); + + Assert.assertSame(true, spec3 == spec4); + Assert.assertSame(false, spec3 == spec5); + } +} 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 ca33815f28eb..73fa1779bdae 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 @@ -646,7 +646,7 @@ public > File fetchSegmentFile( supervisorTaskId, location.getSubTaskId(), location.getInterval(), - location.getPartitionId() + location.getBucketId() ); if (zippedFile == null) { throw new ISE("Can't find segment file for location[%s] at path[%s]", location); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/GenericPartitionLocationTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/GenericPartitionLocationTest.java index 956dbc8fd150..4e46e388f6c6 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/GenericPartitionLocationTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/GenericPartitionLocationTest.java @@ -53,6 +53,6 @@ public void serializesDeserializes() @Test public void hasPartitionIdThatMatchesShardSpec() { - Assert.assertEquals(ParallelIndexTestingFactory.PARTITION_ID, target.getPartitionId()); + Assert.assertEquals(ParallelIndexTestingFactory.PARTITION_ID, target.getBucketId()); } } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/GenericPartitionStatTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/GenericPartitionStatTest.java index 2bcac8edfd47..ffeab43f60d9 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/GenericPartitionStatTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/GenericPartitionStatTest.java @@ -21,10 +21,13 @@ import com.fasterxml.jackson.databind.ObjectMapper; import org.apache.druid.segment.TestHelper; +import org.apache.druid.timeline.partition.HashBucketShardSpec; import org.junit.Assert; import org.junit.Before; import org.junit.Test; +import java.util.Collections; + public class GenericPartitionStatTest { private static final ObjectMapper OBJECT_MAPPER = ParallelIndexTestingFactory.createObjectMapper(); @@ -39,7 +42,12 @@ public void setup() ParallelIndexTestingFactory.TASK_EXECUTOR_PORT, ParallelIndexTestingFactory.USE_HTTPS, ParallelIndexTestingFactory.INTERVAL, - ParallelIndexTestingFactory.HASH_BASED_NUMBERED_SHARD_SPEC, + new HashBucketShardSpec( + ParallelIndexTestingFactory.PARTITION_ID, + ParallelIndexTestingFactory.PARTITION_ID + 1, + Collections.singletonList("dim"), + new ObjectMapper() + ), ParallelIndexTestingFactory.NUM_ROWS, ParallelIndexTestingFactory.SIZE_BYTES ); @@ -54,6 +62,6 @@ public void serializesDeserializes() @Test public void hasPartitionIdThatMatchesSecondaryPartition() { - Assert.assertEquals(target.getSecondaryPartition().getPartitionNum(), target.getPartitionId()); + Assert.assertEquals(target.getSecondaryPartition().getBucketId(), target.getBucketId()); } } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionAdjustingCorePartitionSizeTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionAdjustingCorePartitionSizeTest.java index ed9ffa22260d..e19b208e3691 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionAdjustingCorePartitionSizeTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionAdjustingCorePartitionSizeTest.java @@ -43,7 +43,10 @@ import java.io.Writer; import java.nio.charset.StandardCharsets; import java.nio.file.Files; +import java.util.ArrayList; import java.util.Arrays; +import java.util.Comparator; +import java.util.List; import java.util.Set; @RunWith(Parameterized.class) @@ -95,26 +98,32 @@ public void testLessPartitionsThanBuckets() throws IOException 10, ImmutableList.of("dim1") ); - final Set segments = runTestTask( - TIMESTAMP_SPEC, - DIMENSIONS_SPEC, - INPUT_FORMAT, - null, - INTERVAL_TO_INDEX, - inputDir, - "test_*", - partitionsSpec, - maxNumConcurrentSubTasks, - TaskState.SUCCESS + final List segments = new ArrayList<>( + runTestTask( + TIMESTAMP_SPEC, + DIMENSIONS_SPEC, + INPUT_FORMAT, + null, + INTERVAL_TO_INDEX, + inputDir, + "test_*", + partitionsSpec, + maxNumConcurrentSubTasks, + TaskState.SUCCESS + ) ); Assert.assertEquals(3, segments.size()); - segments.forEach(segment -> { + segments.sort(Comparator.comparing(segment -> segment.getShardSpec().getPartitionNum())); + int prevPartitionId = -1; + for (DataSegment segment : segments) { Assert.assertSame(HashBasedNumberedShardSpec.class, segment.getShardSpec().getClass()); final HashBasedNumberedShardSpec shardSpec = (HashBasedNumberedShardSpec) segment.getShardSpec(); Assert.assertEquals(3, shardSpec.getNumCorePartitions()); Assert.assertEquals(10, shardSpec.getNumBuckets()); Assert.assertEquals(ImmutableList.of("dim1"), shardSpec.getPartitionDimensions()); - }); + Assert.assertEquals(prevPartitionId + 1, shardSpec.getPartitionNum()); + prevPartitionId = shardSpec.getPartitionNum(); + } } @Test diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionMultiPhaseParallelIndexingTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionMultiPhaseParallelIndexingTest.java index a7e892d09310..7fcabd0911ba 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionMultiPhaseParallelIndexingTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionMultiPhaseParallelIndexingTest.java @@ -81,10 +81,10 @@ public class HashPartitionMultiPhaseParallelIndexingTest extends AbstractMultiPh public static Iterable constructorFeeder() { return ImmutableList.of( -// new Object[]{LockGranularity.TIME_CHUNK, false, 2}, - new Object[]{LockGranularity.TIME_CHUNK, true, 2} -// new Object[]{LockGranularity.TIME_CHUNK, true, 1}, -// new Object[]{LockGranularity.SEGMENT, true, 2} + new Object[]{LockGranularity.TIME_CHUNK, false, 2}, + new Object[]{LockGranularity.TIME_CHUNK, true, 2}, + new Object[]{LockGranularity.TIME_CHUNK, true, 1}, + new Object[]{LockGranularity.SEGMENT, true, 2} ); } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionStatTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionStatTest.java index 1eb6f867cd8e..6d86470b53ab 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionStatTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionStatTest.java @@ -54,6 +54,6 @@ public void serializesDeserializes() @Test public void hasPartitionIdThatMatchesSecondaryPartition() { - Assert.assertEquals(target.getSecondaryPartition().intValue(), target.getPartitionId()); + Assert.assertEquals(target.getSecondaryPartition().intValue(), target.getBucketId()); } } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HttpShuffleClientTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HttpShuffleClientTest.java index 45df76d58da2..3ddb63b65fbe 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HttpShuffleClientTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HttpShuffleClientTest.java @@ -203,7 +203,7 @@ private TestPartitionLocation() } @Override - int getPartitionId() + int getBucketId() { return getSecondaryPartition(); } 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 b882aac207ab..211d391b2f72 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 @@ -136,7 +136,7 @@ private static void assertNoMissingPartitions( .flatMap( i -> i.getPartitionLocations() .stream() - .map(HashPartitionLocation::getPartitionId) + .map(HashPartitionLocation::getBucketId) ) .sorted() .collect(Collectors.toList()); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexTestingFactory.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexTestingFactory.java index 39136ccc1057..de7ee8b6cc40 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexTestingFactory.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexTestingFactory.java @@ -44,7 +44,7 @@ import org.apache.druid.segment.indexing.granularity.GranularitySpec; import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager; import org.apache.druid.segment.transform.TransformSpec; -import org.apache.druid.timeline.partition.HashBasedNumberedShardSpec; +import org.apache.druid.timeline.partition.BuildingHashBasedNumberedShardSpec; import org.easymock.EasyMock; import org.joda.time.Duration; import org.joda.time.Interval; @@ -100,9 +100,8 @@ public > File fetchSegmentFile( private static final String SCHEMA_DIMENSION = "dim"; private static final String DATASOURCE = "datasource"; - static final HashBasedNumberedShardSpec HASH_BASED_NUMBERED_SHARD_SPEC = new HashBasedNumberedShardSpec( + static final BuildingHashBasedNumberedShardSpec HASH_BASED_NUMBERED_SHARD_SPEC = new BuildingHashBasedNumberedShardSpec( PARTITION_ID, - PARTITION_ID + 1, PARTITION_ID, PARTITION_ID + 1, Collections.singletonList("dim"), diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/RangePartitionAdjustingCorePartitionSizeTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/RangePartitionAdjustingCorePartitionSizeTest.java new file mode 100644 index 000000000000..2e4aa44dc5f7 --- /dev/null +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/RangePartitionAdjustingCorePartitionSizeTest.java @@ -0,0 +1,167 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.common.task.batch.parallel; + +import com.google.common.collect.ImmutableList; +import org.apache.druid.data.input.InputFormat; +import org.apache.druid.data.input.impl.CsvInputFormat; +import org.apache.druid.data.input.impl.DimensionsSpec; +import org.apache.druid.data.input.impl.TimestampSpec; +import org.apache.druid.indexer.TaskState; +import org.apache.druid.indexer.partitions.DimensionBasedPartitionsSpec; +import org.apache.druid.indexer.partitions.SingleDimensionPartitionsSpec; +import org.apache.druid.indexing.common.LockGranularity; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.partition.SingleDimensionShardSpec; +import org.joda.time.Interval; +import org.junit.Assert; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import java.io.File; +import java.io.IOException; +import java.io.Writer; +import java.nio.charset.StandardCharsets; +import java.nio.file.Files; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Set; + +@RunWith(Parameterized.class) +public class RangePartitionAdjustingCorePartitionSizeTest extends AbstractMultiPhaseParallelIndexingTest +{ + private static final TimestampSpec TIMESTAMP_SPEC = new TimestampSpec("ts", "auto", null); + private static final DimensionsSpec DIMENSIONS_SPEC = new DimensionsSpec( + DimensionsSpec.getDefaultSchemas(Arrays.asList("ts", "dim1", "dim2")) + ); + private static final InputFormat INPUT_FORMAT = new CsvInputFormat( + Arrays.asList("ts", "dim1", "dim2", "val"), + null, + false, + false, + 0 + ); + private static final Interval INTERVAL_TO_INDEX = Intervals.of("2020-01-01/P1M"); + + @Parameterized.Parameters(name = "{0}, maxNumConcurrentSubTasks={1}") + public static Iterable constructorFeeder() + { + return ImmutableList.of( + new Object[]{LockGranularity.TIME_CHUNK, 2}, + new Object[]{LockGranularity.TIME_CHUNK, 1}, + new Object[]{LockGranularity.SEGMENT, 2} + ); + } + + private final int maxNumConcurrentSubTasks; + + public RangePartitionAdjustingCorePartitionSizeTest(LockGranularity lockGranularity, int maxNumConcurrentSubTasks) + { + super(lockGranularity, true); + this.maxNumConcurrentSubTasks = maxNumConcurrentSubTasks; + } + + @Test + public void testLessPartitionsThanBuckets() throws IOException + { + final File inputDir = temporaryFolder.newFolder(); + for (int i = 0; i < 2; i++) { + try (final Writer writer = + Files.newBufferedWriter(new File(inputDir, "test_" + i).toPath(), StandardCharsets.UTF_8)) { + writer.write(StringUtils.format("2020-01-01T00:00:00,aaa,b1,10\n")); + } + } + for (int i = 0; i < 3; i++) { + try (final Writer writer = + Files.newBufferedWriter(new File(inputDir, "test_" + (i + 2)).toPath(), StandardCharsets.UTF_8)) { + writer.write(StringUtils.format("2020-01-01T00:00:00,zzz,b1,10\n")); + } + } + final DimensionBasedPartitionsSpec partitionsSpec = new SingleDimensionPartitionsSpec( + 2, + null, + "dim1", + false + ); + final List segments = new ArrayList<>( + runTestTask( + TIMESTAMP_SPEC, + DIMENSIONS_SPEC, + INPUT_FORMAT, + null, + INTERVAL_TO_INDEX, + inputDir, + "test_*", + partitionsSpec, + maxNumConcurrentSubTasks, + TaskState.SUCCESS + ) + ); + Assert.assertEquals(1, segments.size()); + final DataSegment segment = segments.get(0); + Assert.assertSame(SingleDimensionShardSpec.class, segment.getShardSpec().getClass()); + final SingleDimensionShardSpec shardSpec = (SingleDimensionShardSpec) segment.getShardSpec(); + Assert.assertEquals(1, shardSpec.getNumCorePartitions()); + Assert.assertEquals(0, shardSpec.getPartitionNum()); + Assert.assertEquals("dim1", shardSpec.getDimension()); + } + + @Test + public void testEqualNumberOfPartitionsToBuckets() throws IOException + { + final File inputDir = temporaryFolder.newFolder(); + for (int i = 0; i < 10; i++) { + try (final Writer writer = + Files.newBufferedWriter(new File(inputDir, "test_" + i).toPath(), StandardCharsets.UTF_8)) { + writer.write(StringUtils.format("2020-01-01T00:00:00,%s,b1,%d\n", "aa" + (i + 10), 10 * (i + 1))); + } + } + final DimensionBasedPartitionsSpec partitionsSpec = new SingleDimensionPartitionsSpec( + 2, + null, + "dim1", + false + ); + final Set segments = runTestTask( + TIMESTAMP_SPEC, + DIMENSIONS_SPEC, + INPUT_FORMAT, + null, + INTERVAL_TO_INDEX, + inputDir, + "test_*", + partitionsSpec, + maxNumConcurrentSubTasks, + TaskState.SUCCESS + ); + Assert.assertEquals(5, segments.size()); + segments.forEach(segment -> { + Assert.assertSame(SingleDimensionShardSpec.class, segment.getShardSpec().getClass()); + final SingleDimensionShardSpec shardSpec = (SingleDimensionShardSpec) segment.getShardSpec(); + Assert.assertEquals(5, shardSpec.getNumCorePartitions()); + Assert.assertTrue(shardSpec.getPartitionNum() < shardSpec.getNumCorePartitions()); + Assert.assertEquals("dim1", shardSpec.getDimension()); + }); + } +} From ae87c94af7f3e95fd4c3e060ac5c794decb24e73 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Sat, 13 Jun 2020 14:19:21 -0700 Subject: [PATCH 15/24] remove string partition chunk --- .../partition/SingleDimensionShardSpec.java | 3 +- .../partition/StringPartitionChunk.java | 130 ------------------ .../PartitionHolderCompletenessTest.java | 87 ++++++++++++ .../partition/StringPartitionChunkTest.java | 110 --------------- .../client/CachingClusteredClientTest.java | 36 ++--- 5 files changed, 106 insertions(+), 260 deletions(-) delete mode 100644 core/src/main/java/org/apache/druid/timeline/partition/StringPartitionChunk.java create mode 100644 core/src/test/java/org/apache/druid/timeline/partition/PartitionHolderCompletenessTest.java delete mode 100644 core/src/test/java/org/apache/druid/timeline/partition/StringPartitionChunkTest.java 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 20972508c154..b1019d03e1d1 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 @@ -163,8 +163,7 @@ public boolean isCompatible(Class other) @Override public PartitionChunk createChunk(T obj) { - // TODO: use numberedChunk - return new StringPartitionChunk(start, end, partitionNum, obj); + return new NumberedPartitionChunk<>(partitionNum, numCorePartitions, obj); } @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 deleted file mode 100644 index 28d7505e70a6..000000000000 --- a/core/src/main/java/org/apache/druid/timeline/partition/StringPartitionChunk.java +++ /dev/null @@ -1,130 +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; - -/** - */ -public class StringPartitionChunk implements PartitionChunk -{ - private final String start; - private final String end; - private final int chunkNumber; - private final T object; - - public static StringPartitionChunk make(String start, String end, int chunkNumber, T obj) - { - return new StringPartitionChunk(start, end, chunkNumber, obj); - } - - public StringPartitionChunk( - String start, - String end, - int chunkNumber, - T object - ) - { - this.start = start; - this.end = end; - this.chunkNumber = chunkNumber; - this.object = object; - } - - @Override - public T getObject() - { - return object; - } - - @Override - public boolean abuts(PartitionChunk chunk) - { - if (chunk instanceof StringPartitionChunk) { - StringPartitionChunk stringChunk = (StringPartitionChunk) chunk; - - return !stringChunk.isStart() && stringChunk.start.equals(end); - } - - return false; - } - - @Override - public boolean isStart() - { - return start == null; - } - - @Override - - public boolean isEnd() - { - return end == null; - } - - @Override - public int getChunkNumber() - { - return chunkNumber; - } - - @Override - public int compareTo(PartitionChunk chunk) - { - if (chunk instanceof StringPartitionChunk) { - StringPartitionChunk stringChunk = (StringPartitionChunk) chunk; - - return Integer.compare(chunkNumber, stringChunk.chunkNumber); - } - throw new IllegalArgumentException("Cannot compare against something that is not a StringPartitionChunk."); - } - - @Override - @SuppressWarnings("unchecked") - public boolean equals(Object o) - { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - - return compareTo((StringPartitionChunk) o) == 0; - } - - @Override - public int hashCode() - { - int result = start != null ? start.hashCode() : 0; - result = 31 * result + (end != null ? end.hashCode() : 0); - 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/partition/PartitionHolderCompletenessTest.java b/core/src/test/java/org/apache/druid/timeline/partition/PartitionHolderCompletenessTest.java new file mode 100644 index 000000000000..920b02ade00e --- /dev/null +++ b/core/src/test/java/org/apache/druid/timeline/partition/PartitionHolderCompletenessTest.java @@ -0,0 +1,87 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.timeline.partition; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableList; +import org.junit.Assert; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import java.util.List; + +@RunWith(Parameterized.class) +public class PartitionHolderCompletenessTest +{ + @Parameterized.Parameters(name = "{1}") + public static Iterable constructorFeeder() + { + return ImmutableList.of( + new Object[]{ + ImmutableList.of( + new NumberedShardSpec(0, 3), + new NumberedShardSpec(1, 3), + new NumberedShardSpec(2, 3) + ), + NumberedShardSpec.class.getSimpleName() + }, + new Object[]{ + // Simulate empty hash buckets + ImmutableList.of( + new HashBasedNumberedShardSpec(0, 3, 0, 5, null, new ObjectMapper()), + new HashBasedNumberedShardSpec(1, 3, 2, 5, null, new ObjectMapper()), + new HashBasedNumberedShardSpec(2, 3, 3, 5, null, new ObjectMapper()) + ), + HashBasedNumberedShardSpec.class.getSimpleName() + }, + new Object[]{ + // Simulate empty range buckets + ImmutableList.of( + new SingleDimensionShardSpec("dim", null, "aaa", 0, 3), + new SingleDimensionShardSpec("dim", "bbb", "fff", 1, 3), + new SingleDimensionShardSpec("dim", "ttt", "zzz", 2, 3) + ), + SingleDimensionShardSpec.class.getSimpleName() + } + ); + } + + private final List shardSpecs; + + public PartitionHolderCompletenessTest(List shardSpecs, String paramName) + { + this.shardSpecs = shardSpecs; + } + + @Test + public void testIsComplete() + { + final PartitionHolder holder = new PartitionHolder<>( + shardSpecs.get(0).createChunk(new OvershadowableInteger("version", shardSpecs.get(0).getPartitionNum(), 0)) + ); + for (int i = 0; i < shardSpecs.size() - 1; i++) { + Assert.assertFalse(holder.isComplete()); + final ShardSpec shardSpec = shardSpecs.get(i + 1); + holder.add(shardSpec.createChunk(new OvershadowableInteger("version", shardSpec.getPartitionNum(), 0))); + } + Assert.assertTrue(holder.isComplete()); + } +} diff --git a/core/src/test/java/org/apache/druid/timeline/partition/StringPartitionChunkTest.java b/core/src/test/java/org/apache/druid/timeline/partition/StringPartitionChunkTest.java deleted file mode 100644 index e3f98462903d..000000000000 --- a/core/src/test/java/org/apache/druid/timeline/partition/StringPartitionChunkTest.java +++ /dev/null @@ -1,110 +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 org.junit.Assert; -import org.junit.Test; - -public class StringPartitionChunkTest -{ - @Test - public void testAbuts() - { - StringPartitionChunk lhs = StringPartitionChunk.make(null, "10", 0, 1); - - Assert.assertTrue(lhs.abuts(StringPartitionChunk.make("10", null, 1, 2))); - Assert.assertFalse(lhs.abuts(StringPartitionChunk.make("11", null, 2, 3))); - Assert.assertFalse(lhs.abuts(StringPartitionChunk.make(null, null, 3, 4))); - - Assert.assertFalse(StringPartitionChunk.make(null, null, 0, 1).abuts(StringPartitionChunk.make(null, null, 1, 2))); - } - - @Test - public void testIsStart() - { - Assert.assertTrue(StringPartitionChunk.make(null, "10", 0, 1).isStart()); - Assert.assertFalse(StringPartitionChunk.make("10", null, 0, 1).isStart()); - Assert.assertFalse(StringPartitionChunk.make("10", "11", 0, 1).isStart()); - Assert.assertTrue(StringPartitionChunk.make(null, null, 0, 1).isStart()); - } - - @Test - public void testIsEnd() - { - Assert.assertFalse(StringPartitionChunk.make(null, "10", 0, 1).isEnd()); - Assert.assertTrue(StringPartitionChunk.make("10", null, 0, 1).isEnd()); - Assert.assertFalse(StringPartitionChunk.make("10", "11", 0, 1).isEnd()); - Assert.assertTrue(StringPartitionChunk.make(null, null, 0, 1).isEnd()); - } - - @Test - public void testCompareTo() - { - Assert.assertEquals( - 0, - StringPartitionChunk.make(null, null, 0, 1) - .compareTo(StringPartitionChunk.make(null, null, 0, 2)) - ); - Assert.assertEquals( - 0, - StringPartitionChunk.make("10", null, 0, 1) - .compareTo(StringPartitionChunk.make("10", null, 0, 2)) - ); - Assert.assertEquals( - 0, - StringPartitionChunk.make(null, "10", 1, 1) - .compareTo(StringPartitionChunk.make(null, "10", 1, 2)) - ); - Assert.assertEquals( - 0, - StringPartitionChunk.make("10", "11", 1, 1) - .compareTo(StringPartitionChunk.make("10", "11", 1, 2)) - ); - Assert.assertEquals( - -1, - StringPartitionChunk.make(null, "10", 0, 1) - .compareTo(StringPartitionChunk.make("10", null, 1, 2)) - ); - Assert.assertEquals( - -1, - StringPartitionChunk.make("11", "20", 0, 1) - .compareTo(StringPartitionChunk.make("20", "33", 1, 1)) - ); - Assert.assertEquals( - 1, - StringPartitionChunk.make("20", "33", 1, 1) - .compareTo(StringPartitionChunk.make("11", "20", 0, 1)) - ); - Assert.assertEquals( - 1, - StringPartitionChunk.make("10", null, 1, 1) - .compareTo(StringPartitionChunk.make(null, "10", 0, 1)) - ); - } - - @Test - public void testEquals() - { - Assert.assertEquals(StringPartitionChunk.make(null, null, 0, 1), StringPartitionChunk.make(null, null, 0, 1)); - Assert.assertEquals(StringPartitionChunk.make(null, "10", 0, 1), StringPartitionChunk.make(null, "10", 0, 1)); - Assert.assertEquals(StringPartitionChunk.make("10", null, 0, 1), StringPartitionChunk.make("10", null, 0, 1)); - Assert.assertEquals(StringPartitionChunk.make("10", "11", 0, 1), StringPartitionChunk.make("10", "11", 0, 1)); - } -} 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 fd479c1adaa5..0e5b297f9602 100644 --- a/server/src/test/java/org/apache/druid/client/CachingClusteredClientTest.java +++ b/server/src/test/java/org/apache/druid/client/CachingClusteredClientTest.java @@ -125,10 +125,10 @@ import org.apache.druid.timeline.SegmentId; import org.apache.druid.timeline.VersionedIntervalTimeline; import org.apache.druid.timeline.partition.NoneShardSpec; +import org.apache.druid.timeline.partition.NumberedPartitionChunk; import org.apache.druid.timeline.partition.ShardSpec; import org.apache.druid.timeline.partition.SingleDimensionShardSpec; import org.apache.druid.timeline.partition.SingleElementPartitionChunk; -import org.apache.druid.timeline.partition.StringPartitionChunk; import org.easymock.Capture; import org.easymock.EasyMock; import org.easymock.IAnswer; @@ -1486,19 +1486,19 @@ public void testSingleDimensionPruning() QueryRunner runner = new FinalizeResultsQueryRunner(getDefaultQueryRunner(), new TimeseriesQueryQueryToolChest()); final DruidServer lastServer = servers[random.nextInt(servers.length)]; - ServerSelector selector1 = makeMockSingleDimensionSelector(lastServer, "dim1", null, "b", 1); - ServerSelector selector2 = makeMockSingleDimensionSelector(lastServer, "dim1", "e", "f", 2); - ServerSelector selector3 = makeMockSingleDimensionSelector(lastServer, "dim1", "hi", "zzz", 3); - ServerSelector selector4 = makeMockSingleDimensionSelector(lastServer, "dim2", "a", "e", 4); - ServerSelector selector5 = makeMockSingleDimensionSelector(lastServer, "dim2", null, null, 5); - ServerSelector selector6 = makeMockSingleDimensionSelector(lastServer, "other", "b", null, 6); - - timeline.add(interval1, "v", new StringPartitionChunk<>(null, "a", 1, selector1)); - timeline.add(interval1, "v", new StringPartitionChunk<>("a", "b", 2, selector2)); - timeline.add(interval1, "v", new StringPartitionChunk<>("b", null, 3, selector3)); - timeline.add(interval2, "v", new StringPartitionChunk<>(null, "d", 4, selector4)); - timeline.add(interval2, "v", new StringPartitionChunk<>("d", null, 5, selector5)); - timeline.add(interval3, "v", new StringPartitionChunk<>(null, null, 6, selector6)); + ServerSelector selector1 = makeMockSingleDimensionSelector(lastServer, "dim1", null, "b", 0); + ServerSelector selector2 = makeMockSingleDimensionSelector(lastServer, "dim1", "e", "f", 1); + ServerSelector selector3 = makeMockSingleDimensionSelector(lastServer, "dim1", "hi", "zzz", 2); + ServerSelector selector4 = makeMockSingleDimensionSelector(lastServer, "dim2", "a", "e", 0); + ServerSelector selector5 = makeMockSingleDimensionSelector(lastServer, "dim2", null, null, 1); + ServerSelector selector6 = makeMockSingleDimensionSelector(lastServer, "other", "b", null, 0); + + timeline.add(interval1, "v", new NumberedPartitionChunk<>(0, 3, selector1)); + timeline.add(interval1, "v", new NumberedPartitionChunk<>(1, 3, selector2)); + timeline.add(interval1, "v", new NumberedPartitionChunk<>(2, 3, selector3)); + timeline.add(interval2, "v", new NumberedPartitionChunk<>(0, 2, selector4)); + timeline.add(interval2, "v", new NumberedPartitionChunk<>(1, 2, selector5)); + timeline.add(interval3, "v", new NumberedPartitionChunk<>(0, 1, selector6)); final Capture capture = Capture.newInstance(); final Capture contextCap = Capture.newInstance(); @@ -1514,10 +1514,10 @@ public void testSingleDimensionPruning() EasyMock.replay(mockRunner); List descriptors = new ArrayList<>(); - descriptors.add(new SegmentDescriptor(interval1, "v", 1)); - descriptors.add(new SegmentDescriptor(interval1, "v", 3)); - descriptors.add(new SegmentDescriptor(interval2, "v", 5)); - descriptors.add(new SegmentDescriptor(interval3, "v", 6)); + descriptors.add(new SegmentDescriptor(interval1, "v", 0)); + descriptors.add(new SegmentDescriptor(interval1, "v", 2)); + descriptors.add(new SegmentDescriptor(interval2, "v", 1)); + descriptors.add(new SegmentDescriptor(interval3, "v", 0)); MultipleSpecificSegmentSpec expected = new MultipleSpecificSegmentSpec(descriptors); runner.run(QueryPlus.wrap(query)).toList(); From dbe705a9cfac40042b280530dace57ce8f12cd6e Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Sat, 13 Jun 2020 14:33:13 -0700 Subject: [PATCH 16/24] revive string partition chunk --- .../partition/SingleDimensionShardSpec.java | 6 +- .../partition/StringPartitionChunk.java | 130 ++++++++++++++++++ .../PartitionHolderCompletenessTest.java | 18 ++- .../partition/StringPartitionChunkTest.java | 110 +++++++++++++++ 4 files changed, 262 insertions(+), 2 deletions(-) create mode 100644 core/src/main/java/org/apache/druid/timeline/partition/StringPartitionChunk.java create mode 100644 core/src/test/java/org/apache/druid/timeline/partition/StringPartitionChunkTest.java 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 b1019d03e1d1..0c89aaf73700 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 @@ -163,7 +163,11 @@ public boolean isCompatible(Class other) @Override public PartitionChunk createChunk(T obj) { - return new NumberedPartitionChunk<>(partitionNum, numCorePartitions, obj); + if (numCorePartitions == UNKNOWN_NUM_CORE_PARTITIONS) { + return new StringPartitionChunk<>(start, end, partitionNum, obj); + } else { + return new NumberedPartitionChunk<>(partitionNum, numCorePartitions, obj); + } } @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 new file mode 100644 index 000000000000..28d7505e70a6 --- /dev/null +++ b/core/src/main/java/org/apache/druid/timeline/partition/StringPartitionChunk.java @@ -0,0 +1,130 @@ +/* + * 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; + +/** + */ +public class StringPartitionChunk implements PartitionChunk +{ + private final String start; + private final String end; + private final int chunkNumber; + private final T object; + + public static StringPartitionChunk make(String start, String end, int chunkNumber, T obj) + { + return new StringPartitionChunk(start, end, chunkNumber, obj); + } + + public StringPartitionChunk( + String start, + String end, + int chunkNumber, + T object + ) + { + this.start = start; + this.end = end; + this.chunkNumber = chunkNumber; + this.object = object; + } + + @Override + public T getObject() + { + return object; + } + + @Override + public boolean abuts(PartitionChunk chunk) + { + if (chunk instanceof StringPartitionChunk) { + StringPartitionChunk stringChunk = (StringPartitionChunk) chunk; + + return !stringChunk.isStart() && stringChunk.start.equals(end); + } + + return false; + } + + @Override + public boolean isStart() + { + return start == null; + } + + @Override + + public boolean isEnd() + { + return end == null; + } + + @Override + public int getChunkNumber() + { + return chunkNumber; + } + + @Override + public int compareTo(PartitionChunk chunk) + { + if (chunk instanceof StringPartitionChunk) { + StringPartitionChunk stringChunk = (StringPartitionChunk) chunk; + + return Integer.compare(chunkNumber, stringChunk.chunkNumber); + } + throw new IllegalArgumentException("Cannot compare against something that is not a StringPartitionChunk."); + } + + @Override + @SuppressWarnings("unchecked") + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + + return compareTo((StringPartitionChunk) o) == 0; + } + + @Override + public int hashCode() + { + int result = start != null ? start.hashCode() : 0; + result = 31 * result + (end != null ? end.hashCode() : 0); + 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/partition/PartitionHolderCompletenessTest.java b/core/src/test/java/org/apache/druid/timeline/partition/PartitionHolderCompletenessTest.java index 920b02ade00e..38b9a47854f2 100644 --- a/core/src/test/java/org/apache/druid/timeline/partition/PartitionHolderCompletenessTest.java +++ b/core/src/test/java/org/apache/druid/timeline/partition/PartitionHolderCompletenessTest.java @@ -21,6 +21,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableList; +import org.apache.druid.java.util.common.StringUtils; import org.junit.Assert; import org.junit.Test; import org.junit.runner.RunWith; @@ -59,7 +60,22 @@ public static Iterable constructorFeeder() new SingleDimensionShardSpec("dim", "bbb", "fff", 1, 3), new SingleDimensionShardSpec("dim", "ttt", "zzz", 2, 3) ), - SingleDimensionShardSpec.class.getSimpleName() + StringUtils.format( + "%s with empty buckets", + SingleDimensionShardSpec.class.getSimpleName() + ) + }, + new Object[]{ + // Simulate old format segments with missing numCorePartitions + ImmutableList.of( + new SingleDimensionShardSpec("dim", null, "bbb", 0, null), + new SingleDimensionShardSpec("dim", "bbb", "fff", 1, null), + new SingleDimensionShardSpec("dim", "fff", null, 2, null) + ), + StringUtils.format( + "%s with missing numCorePartitions", + SingleDimensionShardSpec.class.getSimpleName() + ) } ); } diff --git a/core/src/test/java/org/apache/druid/timeline/partition/StringPartitionChunkTest.java b/core/src/test/java/org/apache/druid/timeline/partition/StringPartitionChunkTest.java new file mode 100644 index 000000000000..e3f98462903d --- /dev/null +++ b/core/src/test/java/org/apache/druid/timeline/partition/StringPartitionChunkTest.java @@ -0,0 +1,110 @@ +/* + * 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 org.junit.Assert; +import org.junit.Test; + +public class StringPartitionChunkTest +{ + @Test + public void testAbuts() + { + StringPartitionChunk lhs = StringPartitionChunk.make(null, "10", 0, 1); + + Assert.assertTrue(lhs.abuts(StringPartitionChunk.make("10", null, 1, 2))); + Assert.assertFalse(lhs.abuts(StringPartitionChunk.make("11", null, 2, 3))); + Assert.assertFalse(lhs.abuts(StringPartitionChunk.make(null, null, 3, 4))); + + Assert.assertFalse(StringPartitionChunk.make(null, null, 0, 1).abuts(StringPartitionChunk.make(null, null, 1, 2))); + } + + @Test + public void testIsStart() + { + Assert.assertTrue(StringPartitionChunk.make(null, "10", 0, 1).isStart()); + Assert.assertFalse(StringPartitionChunk.make("10", null, 0, 1).isStart()); + Assert.assertFalse(StringPartitionChunk.make("10", "11", 0, 1).isStart()); + Assert.assertTrue(StringPartitionChunk.make(null, null, 0, 1).isStart()); + } + + @Test + public void testIsEnd() + { + Assert.assertFalse(StringPartitionChunk.make(null, "10", 0, 1).isEnd()); + Assert.assertTrue(StringPartitionChunk.make("10", null, 0, 1).isEnd()); + Assert.assertFalse(StringPartitionChunk.make("10", "11", 0, 1).isEnd()); + Assert.assertTrue(StringPartitionChunk.make(null, null, 0, 1).isEnd()); + } + + @Test + public void testCompareTo() + { + Assert.assertEquals( + 0, + StringPartitionChunk.make(null, null, 0, 1) + .compareTo(StringPartitionChunk.make(null, null, 0, 2)) + ); + Assert.assertEquals( + 0, + StringPartitionChunk.make("10", null, 0, 1) + .compareTo(StringPartitionChunk.make("10", null, 0, 2)) + ); + Assert.assertEquals( + 0, + StringPartitionChunk.make(null, "10", 1, 1) + .compareTo(StringPartitionChunk.make(null, "10", 1, 2)) + ); + Assert.assertEquals( + 0, + StringPartitionChunk.make("10", "11", 1, 1) + .compareTo(StringPartitionChunk.make("10", "11", 1, 2)) + ); + Assert.assertEquals( + -1, + StringPartitionChunk.make(null, "10", 0, 1) + .compareTo(StringPartitionChunk.make("10", null, 1, 2)) + ); + Assert.assertEquals( + -1, + StringPartitionChunk.make("11", "20", 0, 1) + .compareTo(StringPartitionChunk.make("20", "33", 1, 1)) + ); + Assert.assertEquals( + 1, + StringPartitionChunk.make("20", "33", 1, 1) + .compareTo(StringPartitionChunk.make("11", "20", 0, 1)) + ); + Assert.assertEquals( + 1, + StringPartitionChunk.make("10", null, 1, 1) + .compareTo(StringPartitionChunk.make(null, "10", 0, 1)) + ); + } + + @Test + public void testEquals() + { + Assert.assertEquals(StringPartitionChunk.make(null, null, 0, 1), StringPartitionChunk.make(null, null, 0, 1)); + Assert.assertEquals(StringPartitionChunk.make(null, "10", 0, 1), StringPartitionChunk.make(null, "10", 0, 1)); + Assert.assertEquals(StringPartitionChunk.make("10", null, 0, 1), StringPartitionChunk.make("10", null, 0, 1)); + Assert.assertEquals(StringPartitionChunk.make("10", "11", 0, 1), StringPartitionChunk.make("10", "11", 0, 1)); + } +} From 71e23248749c98d209cdc6dfc8cd7c589db1bc18 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Sat, 13 Jun 2020 14:40:09 -0700 Subject: [PATCH 17/24] fill numCorePartitions for hadoop --- .../timeline/partition/SingleDimensionShardSpec.java | 11 +++++++++++ .../apache/druid/indexer/DeterminePartitionsJob.java | 10 ++++++++-- .../druid/indexer/DeterminePartitionsJobTest.java | 1 + 3 files changed, 20 insertions(+), 2 deletions(-) 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 0c89aaf73700..1a00534d6ce8 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 @@ -71,6 +71,17 @@ public SingleDimensionShardSpec( this.numCorePartitions = numCorePartitions == null ? UNKNOWN_NUM_CORE_PARTITIONS : numCorePartitions; } + public SingleDimensionShardSpec withNumCorePartitions(int numCorePartitions) + { + return new SingleDimensionShardSpec( + dimension, + start, + end, + partitionNum, + numCorePartitions + ); + } + @JsonProperty("dimension") public String getDimension() { diff --git a/indexing-hadoop/src/main/java/org/apache/druid/indexer/DeterminePartitionsJob.java b/indexing-hadoop/src/main/java/org/apache/druid/indexer/DeterminePartitionsJob.java index a0bebc6239a5..569290d3e469 100644 --- a/indexing-hadoop/src/main/java/org/apache/druid/indexer/DeterminePartitionsJob.java +++ b/indexing-hadoop/src/main/java/org/apache/druid/indexer/DeterminePartitionsJob.java @@ -76,6 +76,7 @@ import java.util.List; import java.util.Map; import java.util.TreeMap; +import java.util.stream.Collectors; /** * Determines appropriate ShardSpecs for a job by determining whether or not partitioning is necessary, and if so, @@ -819,8 +820,13 @@ protected void innerReduce(Context context, SortableBytes keyBytes, Iterable dimPartition.shardSpec ); + final List annotatedShardSpecs = chosenShardSpecs + .stream() + .map(shardSpec -> ((SingleDimensionShardSpec) shardSpec).withNumCorePartitions(chosenShardSpecs.size())) + .collect(Collectors.toList()); + log.info("Chosen partitions:"); - for (ShardSpec shardSpec : chosenShardSpecs) { + for (ShardSpec shardSpec : annotatedShardSpecs) { log.info(" %s", HadoopDruidIndexerConfig.JSON_MAPPER.writeValueAsString(shardSpec)); } @@ -831,7 +837,7 @@ protected void innerReduce(Context context, SortableBytes keyBytes, Iterable Date: Sat, 13 Jun 2020 15:15:07 -0700 Subject: [PATCH 18/24] clean up hash stuffs --- .../druid/indexer/DeterminePartitionsJob.java | 22 ++-- .../indexer/DeterminePartitionsJobTest.java | 1 - .../task/CachingLocalSegmentAllocator.java | 5 +- .../druid/indexing/common/task/IndexTask.java | 11 +- .../common/task/LocalSegmentAllocator.java | 12 +- .../OverlordCoordinatingSegmentAllocator.java | 12 +- ...fig.java => SegmentAllocatorForBatch.java} | 28 ++--- .../common/task/SegmentAllocators.java | 8 +- ...visorTaskCoordinatingSegmentAllocator.java | 12 +- .../druid/indexing/common/task/Task.java | 2 - .../GeneratedHashPartitionsReport.java | 44 ------- .../batch/parallel/HashPartitionLocation.java | 51 -------- .../batch/parallel/HashPartitionStat.java | 90 -------------- .../parallel/ParallelIndexSupervisorTask.java | 18 --- .../parallel/ParallelIndexTaskRunner.java | 5 +- .../parallel/ParallelIndexTuningConfig.java | 4 +- ...egmentGenerateParallelIndexTaskRunner.java | 2 - .../PartialHashSegmentGenerateTask.java | 4 +- .../PartialHashSegmentMergeIngestionSpec.java | 37 ------ ...shSegmentMergeParallelIndexTaskRunner.java | 115 ------------------ .../parallel/PartialHashSegmentMergeTask.java | 113 ----------------- ...egmentGenerateParallelIndexTaskRunner.java | 2 - .../PartialRangeSegmentGenerateTask.java | 4 +- .../parallel/PartialSegmentGenerateTask.java | 8 +- .../batch/parallel/SinglePhaseSubTask.java | 1 + .../task/batch/parallel/SubTaskReport.java | 1 - ...stractParallelIndexSupervisorTaskTest.java | 1 - .../GeneratedHashPartitionsReportTest.java | 59 --------- .../batch/parallel/HashPartitionStatTest.java | 59 --------- ...rallelIndexSupervisorTaskResourceTest.java | 1 + .../ParallelIndexSupervisorTaskTest.java | 24 ++-- .../PartialHashSegmentMergeIOConfigTest.java | 54 -------- ...tialHashSegmentMergeIngestionSpecTest.java | 68 ----------- .../PartialHashSegmentMergeTaskTest.java | 88 -------------- .../indexer/AbstractITBatchIndexTest.java | 2 - 35 files changed, 93 insertions(+), 875 deletions(-) rename indexing-service/src/main/java/org/apache/druid/indexing/common/task/{batch/parallel/PartialHashSegmentMergeIOConfig.java => SegmentAllocatorForBatch.java} (55%) delete mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/GeneratedHashPartitionsReport.java delete mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionLocation.java delete mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionStat.java delete mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentMergeIngestionSpec.java delete mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentMergeParallelIndexTaskRunner.java delete mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentMergeTask.java delete mode 100644 indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/GeneratedHashPartitionsReportTest.java delete mode 100644 indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionStatTest.java delete mode 100644 indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentMergeIOConfigTest.java delete mode 100644 indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentMergeIngestionSpecTest.java delete mode 100644 indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentMergeTaskTest.java diff --git a/indexing-hadoop/src/main/java/org/apache/druid/indexer/DeterminePartitionsJob.java b/indexing-hadoop/src/main/java/org/apache/druid/indexer/DeterminePartitionsJob.java index 569290d3e469..0b246be384b6 100644 --- a/indexing-hadoop/src/main/java/org/apache/druid/indexer/DeterminePartitionsJob.java +++ b/indexing-hadoop/src/main/java/org/apache/druid/indexer/DeterminePartitionsJob.java @@ -76,7 +76,6 @@ import java.util.List; import java.util.Map; import java.util.TreeMap; -import java.util.stream.Collectors; /** * Determines appropriate ShardSpecs for a job by determining whether or not partitioning is necessary, and if so, @@ -668,7 +667,9 @@ protected void innerReduce(Context context, SortableBytes keyBytes, Iterable dimPartition.shardSpec ); - final List annotatedShardSpecs = chosenShardSpecs - .stream() - .map(shardSpec -> ((SingleDimensionShardSpec) shardSpec).withNumCorePartitions(chosenShardSpecs.size())) - .collect(Collectors.toList()); - log.info("Chosen partitions:"); - for (ShardSpec shardSpec : annotatedShardSpecs) { + for (ShardSpec shardSpec : chosenShardSpecs) { log.info(" %s", HadoopDruidIndexerConfig.JSON_MAPPER.writeValueAsString(shardSpec)); } @@ -837,7 +837,7 @@ protected void innerReduce(Context context, SortableBytes keyBytes, Iterable> sequenceNameToBucket; @@ -156,7 +155,7 @@ public SegmentIdWithShardSpec allocate( ); } - // TODO: i don't like it + @Override public SequenceNameFunction getSequenceNameFunction() { return sequenceNameFunction; 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 fa8d1661fd52..89d69604be26 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 @@ -97,7 +97,6 @@ import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager; import org.apache.druid.segment.realtime.appenderator.BaseAppenderatorDriver; import org.apache.druid.segment.realtime.appenderator.BatchAppenderatorDriver; -import org.apache.druid.segment.realtime.appenderator.SegmentAllocator; import org.apache.druid.segment.realtime.appenderator.SegmentsAndCommitMetadata; import org.apache.druid.segment.realtime.appenderator.TransactionalSegmentPublisher; import org.apache.druid.segment.realtime.firehose.ChatHandler; @@ -877,12 +876,12 @@ private TaskStatus generateAndPublishSegments( final IndexTuningConfig tuningConfig = ingestionSchema.getTuningConfig(); final long pushTimeout = tuningConfig.getPushTimeout(); - final SegmentAllocator segmentAllocator; + final SegmentAllocatorForBatch segmentAllocator; final SequenceNameFunction sequenceNameFunction; switch (partitionsSpec.getType()) { case HASH: case RANGE: - final SegmentAllocator localSegmentAllocator = SegmentAllocators.forNonLinearPartitioning( + final SegmentAllocatorForBatch localSegmentAllocator = SegmentAllocators.forNonLinearPartitioning( toolbox, getDataSource(), getId(), @@ -890,20 +889,20 @@ private TaskStatus generateAndPublishSegments( null, (CompletePartitionAnalysis) partitionAnalysis ); - // TODO: - sequenceNameFunction = ((CachingLocalSegmentAllocator) localSegmentAllocator).getSequenceNameFunction(); + sequenceNameFunction = localSegmentAllocator.getSequenceNameFunction(); segmentAllocator = localSegmentAllocator; break; case LINEAR: segmentAllocator = SegmentAllocators.forLinearPartitioning( toolbox, + getId(), null, dataSchema, getTaskLockHelper(), ingestionSchema.getIOConfig().isAppendToExisting(), partitionAnalysis.getPartitionsSpec() ); - sequenceNameFunction = new LinearlyPartitionedSequenceNameFunction(getId()); + sequenceNameFunction = segmentAllocator.getSequenceNameFunction(); break; default: throw new UOE("[%s] secondary partition type is not supported", partitionsSpec.getType()); 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 2cb4db551c4a..c2488e6244ba 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 @@ -43,11 +43,12 @@ /** * Segment allocator which allocates new segments locally per request. */ -class LocalSegmentAllocator implements SegmentAllocator +class LocalSegmentAllocator implements SegmentAllocatorForBatch { private final SegmentAllocator internalAllocator; + private final SequenceNameFunction sequenceNameFunction; - LocalSegmentAllocator(TaskToolbox toolbox, String dataSource, GranularitySpec granularitySpec) throws IOException + LocalSegmentAllocator(TaskToolbox toolbox, String taskId, String dataSource, GranularitySpec granularitySpec) throws IOException { final Map intervalToVersion = toolbox .getTaskActionClient() @@ -80,6 +81,7 @@ class LocalSegmentAllocator implements SegmentAllocator new BuildingNumberedShardSpec(partitionId) ); }; + sequenceNameFunction = new LinearlyPartitionedSequenceNameFunction(taskId); } @Nullable @@ -93,4 +95,10 @@ public SegmentIdWithShardSpec allocate( { return internalAllocator.allocate(row, sequenceName, previousSegmentId, skipSegmentLineageCheck); } + + @Override + public SequenceNameFunction getSequenceNameFunction() + { + return sequenceNameFunction; + } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/OverlordCoordinatingSegmentAllocator.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/OverlordCoordinatingSegmentAllocator.java index 1598dee0c301..87daaa865506 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/OverlordCoordinatingSegmentAllocator.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/OverlordCoordinatingSegmentAllocator.java @@ -31,7 +31,6 @@ import org.apache.druid.java.util.common.ISE; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.granularity.GranularitySpec; -import org.apache.druid.segment.realtime.appenderator.SegmentAllocator; import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; import org.apache.druid.timeline.partition.NumberedOverwritePartialShardSpec; import org.apache.druid.timeline.partition.NumberedPartialShardSpec; @@ -44,12 +43,14 @@ /** * Segment allocator which allocates new segments using the overlord per request. */ -public class OverlordCoordinatingSegmentAllocator implements SegmentAllocator +public class OverlordCoordinatingSegmentAllocator implements SegmentAllocatorForBatch { private final ActionBasedSegmentAllocator internalAllocator; + private final LinearlyPartitionedSequenceNameFunction sequenceNameFunction; OverlordCoordinatingSegmentAllocator( final TaskToolbox toolbox, + final String taskId, final @Nullable SupervisorTaskAccess supervisorTaskAccess, final DataSchema dataSchema, final TaskLockHelper taskLockHelper, @@ -101,6 +102,7 @@ public class OverlordCoordinatingSegmentAllocator implements SegmentAllocator } } ); + this.sequenceNameFunction = new LinearlyPartitionedSequenceNameFunction(taskId); } @Nullable @@ -146,4 +148,10 @@ private static PartialShardSpec createPartialShardSpec( ); } } + + @Override + public SequenceNameFunction getSequenceNameFunction() + { + return sequenceNameFunction; + } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentMergeIOConfig.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/SegmentAllocatorForBatch.java similarity index 55% rename from indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentMergeIOConfig.java rename to indexing-service/src/main/java/org/apache/druid/indexing/common/task/SegmentAllocatorForBatch.java index 2bc00ce3d9b6..f2bf5030a87f 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentMergeIOConfig.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/SegmentAllocatorForBatch.java @@ -17,24 +17,18 @@ * under the License. */ -package org.apache.druid.indexing.common.task.batch.parallel; +package org.apache.druid.indexing.common.task; -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; -import com.fasterxml.jackson.annotation.JsonTypeName; -import org.apache.druid.segment.indexing.IOConfig; +import org.apache.druid.segment.realtime.appenderator.SegmentAllocator; -import java.util.List; - -@JsonTypeName(PartialHashSegmentMergeTask.TYPE) -class PartialHashSegmentMergeIOConfig extends PartialSegmentMergeIOConfig - implements IOConfig +/** + * + */ +public interface SegmentAllocatorForBatch extends SegmentAllocator { - @JsonCreator - PartialHashSegmentMergeIOConfig( - @JsonProperty("partitionLocations") List partitionLocations - ) - { - super(partitionLocations); - } + /** + * + * @return + */ + SequenceNameFunction getSequenceNameFunction(); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/SegmentAllocators.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/SegmentAllocators.java index 5a7c7ab34dad..47df3f235b7b 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/SegmentAllocators.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/SegmentAllocators.java @@ -36,8 +36,9 @@ public final class SegmentAllocators * Creates a new {@link SegmentAllocator} for the linear partitioning. * supervisorTaskAccess can be null if this method is called by the {@link IndexTask}. */ - public static SegmentAllocator forLinearPartitioning( + public static SegmentAllocatorForBatch forLinearPartitioning( final TaskToolbox toolbox, + final String taskId, final @Nullable SupervisorTaskAccess supervisorTaskAccess, final DataSchema dataSchema, final TaskLockHelper taskLockHelper, @@ -48,6 +49,7 @@ public static SegmentAllocator forLinearPartitioning( if (appendToExisting || taskLockHelper.isUseSegmentLock()) { return new OverlordCoordinatingSegmentAllocator( toolbox, + taskId, supervisorTaskAccess, dataSchema, taskLockHelper, @@ -58,12 +60,14 @@ public static SegmentAllocator forLinearPartitioning( if (supervisorTaskAccess == null) { return new LocalSegmentAllocator( toolbox, + taskId, dataSchema.getDataSource(), dataSchema.getGranularitySpec() ); } else { return new SupervisorTaskCoordinatingSegmentAllocator( supervisorTaskAccess.getSupervisorTaskId(), + taskId, supervisorTaskAccess.getTaskClient() ); } @@ -74,7 +78,7 @@ public static SegmentAllocator forLinearPartitioning( * Creates a new {@link SegmentAllocator} for the hash and range partitioning. * supervisorTaskAccess can be null if this method is called by the {@link IndexTask}. */ - public static SegmentAllocator forNonLinearPartitioning( + public static SegmentAllocatorForBatch forNonLinearPartitioning( final TaskToolbox toolbox, final String dataSource, final String taskId, diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/SupervisorTaskCoordinatingSegmentAllocator.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/SupervisorTaskCoordinatingSegmentAllocator.java index 7fde4b879d00..d0956aadbc8f 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/SupervisorTaskCoordinatingSegmentAllocator.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/SupervisorTaskCoordinatingSegmentAllocator.java @@ -21,7 +21,6 @@ import org.apache.druid.data.input.InputRow; import org.apache.druid.indexing.common.task.batch.parallel.ParallelIndexSupervisorTaskClient; -import org.apache.druid.segment.realtime.appenderator.SegmentAllocator; import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; import java.io.IOException; @@ -29,18 +28,21 @@ /** * Segment allocator that allocates new segments using the supervisor task per request. */ -public class SupervisorTaskCoordinatingSegmentAllocator implements SegmentAllocator +public class SupervisorTaskCoordinatingSegmentAllocator implements SegmentAllocatorForBatch { private final String supervisorTaskId; private final ParallelIndexSupervisorTaskClient taskClient; + private final SequenceNameFunction sequenceNameFunction; SupervisorTaskCoordinatingSegmentAllocator( String supervisorTaskId, + String taskId, ParallelIndexSupervisorTaskClient taskClient ) { this.supervisorTaskId = supervisorTaskId; this.taskClient = taskClient; + this.sequenceNameFunction = new LinearlyPartitionedSequenceNameFunction(taskId); } @Override @@ -53,4 +55,10 @@ public SegmentIdWithShardSpec allocate( { return taskClient.allocateSegment(supervisorTaskId, row.getTimestamp()); } + + @Override + public SequenceNameFunction getSequenceNameFunction() + { + return sequenceNameFunction; + } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/Task.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/Task.java index 4f18c81bc7c7..20a7da6e8be1 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/Task.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/Task.java @@ -31,7 +31,6 @@ import org.apache.druid.indexing.common.task.batch.parallel.PartialDimensionDistributionTask; import org.apache.druid.indexing.common.task.batch.parallel.PartialGenericSegmentMergeTask; import org.apache.druid.indexing.common.task.batch.parallel.PartialHashSegmentGenerateTask; -import org.apache.druid.indexing.common.task.batch.parallel.PartialHashSegmentMergeTask; import org.apache.druid.indexing.common.task.batch.parallel.PartialRangeSegmentGenerateTask; import org.apache.druid.indexing.common.task.batch.parallel.SinglePhaseSubTask; import org.apache.druid.query.Query; @@ -63,7 +62,6 @@ // for backward compatibility @Type(name = SinglePhaseSubTask.OLD_TYPE_NAME, value = LegacySinglePhaseSubTask.class), @Type(name = PartialHashSegmentGenerateTask.TYPE, value = PartialHashSegmentGenerateTask.class), - @Type(name = PartialHashSegmentMergeTask.TYPE, value = PartialHashSegmentMergeTask.class), @Type(name = PartialRangeSegmentGenerateTask.TYPE, value = PartialRangeSegmentGenerateTask.class), @Type(name = PartialDimensionDistributionTask.TYPE, value = PartialDimensionDistributionTask.class), @Type(name = PartialGenericSegmentMergeTask.TYPE, value = PartialGenericSegmentMergeTask.class), diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/GeneratedHashPartitionsReport.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/GeneratedHashPartitionsReport.java deleted file mode 100644 index 85574b7b254b..000000000000 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/GeneratedHashPartitionsReport.java +++ /dev/null @@ -1,44 +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.batch.parallel; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; - -import java.util.List; - -/** - * Report containing the {@link HashPartitionStat}s created by a {@link PartialHashSegmentGenerateTask}. - * This report is collected by {@link ParallelIndexSupervisorTask} and - * used to generate {@link PartialHashSegmentMergeIOConfig}. - */ -class GeneratedHashPartitionsReport extends GeneratedPartitionsReport implements SubTaskReport -{ - public static final String TYPE = "generated_partitions"; - - @JsonCreator - GeneratedHashPartitionsReport( - @JsonProperty("taskId") String taskId, - @JsonProperty("partitionStats") List partitionStats - ) - { - super(taskId, partitionStats); - } -} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionLocation.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionLocation.java deleted file mode 100644 index 9ac092fe8ce4..000000000000 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionLocation.java +++ /dev/null @@ -1,51 +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.batch.parallel; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; -import org.joda.time.Interval; - -/** - * This class represents the intermediary data server where the partition of {@code interval} and {@code partitionId} - * is stored. - */ -public class HashPartitionLocation extends PartitionLocation -{ - @JsonCreator - public HashPartitionLocation( - @JsonProperty("host") String host, - @JsonProperty("port") int port, - @JsonProperty("useHttps") boolean useHttps, - @JsonProperty("subTaskId") String subTaskId, - @JsonProperty("interval") Interval interval, - @JsonProperty("partitionId") int partitionId - ) - { - super(host, port, useHttps, subTaskId, interval, partitionId); - } - - @JsonProperty - @Override - public int getBucketId() - { - return getSecondaryPartition(); - } -} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionStat.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionStat.java deleted file mode 100644 index 03c53db43b2c..000000000000 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionStat.java +++ /dev/null @@ -1,90 +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.batch.parallel; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonIgnore; -import com.fasterxml.jackson.annotation.JsonProperty; -import org.joda.time.Interval; - -import javax.annotation.Nullable; -import java.util.Objects; - -/** - * Statistics about a partition created by {@link PartialHashSegmentGenerateTask}. Each partition is a set of data - * of the same time chunk (primary partition key) and the same partitionId (secondary partition key). This class - * holds the statistics of a single partition created by a task. - */ -public class HashPartitionStat extends PartitionStat -{ - // Secondary partition key - private final int partitionId; - - @JsonCreator - public HashPartitionStat( - @JsonProperty("taskExecutorHost") String taskExecutorHost, - @JsonProperty("taskExecutorPort") int taskExecutorPort, - @JsonProperty("useHttps") boolean useHttps, - @JsonProperty("interval") Interval interval, - @JsonProperty("partitionId") int partitionId, - @JsonProperty("numRows") @Nullable Integer numRows, - @JsonProperty("sizeBytes") @Nullable Long sizeBytes - ) - { - super(taskExecutorHost, taskExecutorPort, useHttps, interval, numRows, sizeBytes); - this.partitionId = partitionId; - } - - @JsonProperty - @Override - public int getBucketId() - { - return partitionId; - } - - @JsonIgnore - @Override - Integer getSecondaryPartition() - { - return partitionId; - } - - @Override - public boolean equals(Object o) - { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - if (!super.equals(o)) { - return false; - } - HashPartitionStat that = (HashPartitionStat) o; - return partitionId == that.partitionId; - } - - @Override - public int hashCode() - { - return Objects.hash(super.hashCode(), partitionId); - } -} 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 26625b65afd9..705ae09d894e 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 @@ -339,24 +339,6 @@ PartialRangeSegmentGenerateParallelIndexTaskRunner createPartialRangeSegmentGene ); } - @VisibleForTesting - PartialHashSegmentMergeParallelIndexTaskRunner createPartialHashSegmentMergeRunner( - TaskToolbox toolbox, - List ioConfigs - ) - { - return new PartialHashSegmentMergeParallelIndexTaskRunner( - toolbox, - getId(), - getGroupId(), - getIngestionSchema().getDataSchema(), - ioConfigs, - getIngestionSchema().getTuningConfig(), - getContext(), - indexingServiceClient - ); - } - @VisibleForTesting PartialGenericSegmentMergeParallelIndexTaskRunner createPartialGenericSegmentMergeRunner( TaskToolbox toolbox, diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexTaskRunner.java index 49b2b48ecd61..05103e85c6ab 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexTaskRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexTaskRunner.java @@ -38,8 +38,9 @@ * uses {@link SinglePhaseParallelIndexTaskRunner} for it. * * For perfect rollup, parallel indexing is executed in multiple phases. The supervisor task currently uses - * {@link PartialHashSegmentGenerateParallelIndexTaskRunner} and {@link PartialHashSegmentMergeParallelIndexTaskRunner}, - * and can use more runners in the future. + * {@link PartialHashSegmentGenerateParallelIndexTaskRunner}, {@link PartialRangeSegmentGenerateParallelIndexTaskRunner}, + * and {@link PartialGenericSegmentMergeParallelIndexTaskRunner}. + * More runners can be added in the future. */ public interface ParallelIndexTaskRunner { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexTuningConfig.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexTuningConfig.java index 73e2ac7c7414..935eeb48c21e 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexTuningConfig.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexTuningConfig.java @@ -57,14 +57,14 @@ public class ParallelIndexTuningConfig extends IndexTuningConfig /** * Max number of segments to merge at the same time. - * Used only by {@link PartialHashSegmentMergeTask}. + * Used only by {@link PartialGenericSegmentMergeTask}. * This configuration was temporarily added to avoid using too much memory while merging segments, * and will be removed once {@link org.apache.druid.segment.IndexMerger} is improved to not use much memory. */ private final int maxNumSegmentsToMerge; /** - * Total number of tasks for partial segment merge (that is, number of {@link PartialHashSegmentMergeTask}s). + * Total number of tasks for partial segment merge (that is, number of {@link PartialGenericSegmentMergeTask}s). * Used only when this task runs with shuffle. */ private final int totalNumMergeTasks; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentGenerateParallelIndexTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentGenerateParallelIndexTaskRunner.java index 29f742a74dd8..e067eb909196 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentGenerateParallelIndexTaskRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentGenerateParallelIndexTaskRunner.java @@ -27,8 +27,6 @@ /** * {@link ParallelIndexTaskRunner} for the phase to create hash partitioned segments in multi-phase parallel indexing. - * - * @see PartialHashSegmentMergeParallelIndexTaskRunner */ class PartialHashSegmentGenerateParallelIndexTaskRunner extends InputSourceSplitParallelIndexTaskRunner> diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentGenerateTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentGenerateTask.java index 61d62db32655..1bfda30522af 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentGenerateTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentGenerateTask.java @@ -27,13 +27,13 @@ import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.common.actions.TaskActionClient; import org.apache.druid.indexing.common.task.IndexTaskClientFactory; +import org.apache.druid.indexing.common.task.SegmentAllocatorForBatch; import org.apache.druid.indexing.common.task.SegmentAllocators; import org.apache.druid.indexing.common.task.TaskResource; import org.apache.druid.indexing.common.task.batch.parallel.iterator.DefaultIndexTaskInputRowIteratorBuilder; import org.apache.druid.indexing.common.task.batch.partition.HashPartitionAnalysis; import org.apache.druid.segment.indexing.granularity.GranularitySpec; import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager; -import org.apache.druid.segment.realtime.appenderator.SegmentAllocator; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.partition.BucketNumberedShardSpec; import org.apache.druid.timeline.partition.PartialShardSpec; @@ -128,7 +128,7 @@ public boolean isReady(TaskActionClient taskActionClient) throws Exception } @Override - SegmentAllocator createSegmentAllocator(TaskToolbox toolbox, ParallelIndexSupervisorTaskClient taskClient) + SegmentAllocatorForBatch createSegmentAllocator(TaskToolbox toolbox, ParallelIndexSupervisorTaskClient taskClient) throws IOException { final GranularitySpec granularitySpec = ingestionSchema.getDataSchema().getGranularitySpec(); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentMergeIngestionSpec.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentMergeIngestionSpec.java deleted file mode 100644 index abfef7608809..000000000000 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentMergeIngestionSpec.java +++ /dev/null @@ -1,37 +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.batch.parallel; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; -import org.apache.druid.segment.indexing.DataSchema; - -class PartialHashSegmentMergeIngestionSpec extends PartialSegmentMergeIngestionSpec -{ - @JsonCreator - PartialHashSegmentMergeIngestionSpec( - @JsonProperty("dataSchema") DataSchema dataSchema, - @JsonProperty("ioConfig") PartialHashSegmentMergeIOConfig ioConfig, - @JsonProperty("tuningConfig") ParallelIndexTuningConfig tuningConfig - ) - { - super(dataSchema, ioConfig, tuningConfig); - } -} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentMergeParallelIndexTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentMergeParallelIndexTaskRunner.java deleted file mode 100644 index c693513c6527..000000000000 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentMergeParallelIndexTaskRunner.java +++ /dev/null @@ -1,115 +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.batch.parallel; - -import com.google.common.annotations.VisibleForTesting; -import org.apache.druid.client.indexing.IndexingServiceClient; -import org.apache.druid.data.input.InputSplit; -import org.apache.druid.indexing.common.TaskToolbox; -import org.apache.druid.segment.indexing.DataSchema; - -import java.util.Iterator; -import java.util.List; -import java.util.Map; - -/** - * {@link ParallelIndexTaskRunner} for the phase to merge hash partitioned segments in multi-phase parallel indexing. - * - * @see PartialHashSegmentGenerateParallelIndexTaskRunner - */ -class PartialHashSegmentMergeParallelIndexTaskRunner - extends ParallelIndexPhaseRunner -{ - private static final String PHASE_NAME = "partial segment merge"; - - private final DataSchema dataSchema; - private final List mergeIOConfigs; - - PartialHashSegmentMergeParallelIndexTaskRunner( - TaskToolbox toolbox, - String taskId, - String groupId, - DataSchema dataSchema, - List mergeIOConfigs, - ParallelIndexTuningConfig tuningConfig, - Map context, - IndexingServiceClient indexingServiceClient - ) - { - super(toolbox, taskId, groupId, tuningConfig, context, indexingServiceClient); - - this.dataSchema = dataSchema; - this.mergeIOConfigs = mergeIOConfigs; - } - - @Override - public String getName() - { - return PHASE_NAME; - } - - @Override - Iterator> subTaskSpecIterator() - { - return mergeIOConfigs.stream().map(this::newTaskSpec).iterator(); - } - - @Override - int estimateTotalNumSubTasks() - { - return mergeIOConfigs.size(); - } - - @VisibleForTesting - SubTaskSpec newTaskSpec(PartialHashSegmentMergeIOConfig ioConfig) - { - final PartialHashSegmentMergeIngestionSpec ingestionSpec = - new PartialHashSegmentMergeIngestionSpec( - dataSchema, - ioConfig, - getTuningConfig() - ); - return new SubTaskSpec( - getTaskId() + "_" + getAndIncrementNextSpecId(), - getGroupId(), - getTaskId(), - getContext(), - new InputSplit<>(ioConfig.getPartitionLocations()) - ) - { - @Override - public PartialHashSegmentMergeTask newSubTask(int numAttempts) - { - return new PartialHashSegmentMergeTask( - null, - getGroupId(), - null, - getSupervisorTaskId(), - numAttempts, - ingestionSpec, - getContext(), - null, - null, - null - ); - } - }; - } -} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentMergeTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentMergeTask.java deleted file mode 100644 index be869a2c7937..000000000000 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentMergeTask.java +++ /dev/null @@ -1,113 +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.batch.parallel; - -import com.fasterxml.jackson.annotation.JacksonInject; -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; -import com.google.common.base.Preconditions; -import org.apache.druid.client.indexing.IndexingServiceClient; -import org.apache.druid.indexer.partitions.HashedPartitionsSpec; -import org.apache.druid.indexer.partitions.PartitionsSpec; -import org.apache.druid.indexing.common.TaskToolbox; -import org.apache.druid.indexing.common.task.IndexTaskClientFactory; -import org.apache.druid.indexing.common.task.TaskResource; -import org.apache.druid.timeline.partition.BuildingHashBasedNumberedShardSpec; -import org.joda.time.Interval; - -import javax.annotation.Nullable; -import java.util.Map; - -/** - * The worker task of {@link PartialHashSegmentMergeParallelIndexTaskRunner}. This task reads partitioned segments - * created by {@link PartialHashSegmentGenerateTask}s, merges them, and pushes to deep storage. The pushed segments are - * reported to {@link PartialHashSegmentMergeParallelIndexTaskRunner}. - */ - -public class PartialHashSegmentMergeTask - extends PartialSegmentMergeTask -{ - public static final String TYPE = "partial_index_merge"; - - private final HashedPartitionsSpec partitionsSpec; - private final PartialHashSegmentMergeIngestionSpec ingestionSchema; - - @JsonCreator - public PartialHashSegmentMergeTask( - // id shouldn't be null except when this task is created by ParallelIndexSupervisorTask - @JsonProperty("id") @Nullable String id, - @JsonProperty("groupId") final String groupId, - @JsonProperty("resource") final TaskResource taskResource, - @JsonProperty("supervisorTaskId") final String supervisorTaskId, - @JsonProperty("numAttempts") final int numAttempts, // zero-based counting - @JsonProperty("spec") final PartialHashSegmentMergeIngestionSpec ingestionSchema, - @JsonProperty("context") final Map context, - @JacksonInject IndexingServiceClient indexingServiceClient, - @JacksonInject IndexTaskClientFactory taskClientFactory, - @JacksonInject ShuffleClient shuffleClient - ) - { - super( - getOrMakeId(id, TYPE, ingestionSchema.getDataSchema().getDataSource()), - groupId, - taskResource, - supervisorTaskId, - ingestionSchema.getDataSchema(), - ingestionSchema.getIOConfig(), - ingestionSchema.getTuningConfig(), - numAttempts, - context, - indexingServiceClient, - taskClientFactory, - shuffleClient - ); - - this.ingestionSchema = ingestionSchema; - - PartitionsSpec inputPartitionsSpec = ingestionSchema.getTuningConfig().getGivenOrDefaultPartitionsSpec(); - Preconditions.checkArgument(inputPartitionsSpec instanceof HashedPartitionsSpec, "hashed partitionsSpec required"); - partitionsSpec = (HashedPartitionsSpec) inputPartitionsSpec; - Preconditions.checkNotNull(partitionsSpec.getNumShards(), "hashed partitionsSpec numShards required"); - } - - @JsonProperty("spec") - private PartialHashSegmentMergeIngestionSpec getIngestionSchema() - { - return ingestionSchema; - } - - @Override - public String getType() - { - return TYPE; - } - - @Override - BuildingHashBasedNumberedShardSpec createShardSpec(TaskToolbox toolbox, Interval interval, int partitionId) - { - return new BuildingHashBasedNumberedShardSpec( - partitionId, - partitionId, // TODO: should use shardSpec instead - Preconditions.checkNotNull(partitionsSpec.getNumShards(), "numShards"), - partitionsSpec.getPartitionDimensions(), - toolbox.getJsonMapper() - ); - } -} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialRangeSegmentGenerateParallelIndexTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialRangeSegmentGenerateParallelIndexTaskRunner.java index e0f9461d166b..39a7e6576c90 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialRangeSegmentGenerateParallelIndexTaskRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialRangeSegmentGenerateParallelIndexTaskRunner.java @@ -29,8 +29,6 @@ /** * {@link ParallelIndexTaskRunner} for the phase to create range partitioned segments in multi-phase parallel indexing. - * - * @see PartialHashSegmentMergeParallelIndexTaskRunner */ class PartialRangeSegmentGenerateParallelIndexTaskRunner extends InputSourceSplitParallelIndexTaskRunner> diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialRangeSegmentGenerateTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialRangeSegmentGenerateTask.java index c58bc5bc5eca..949c3748ac4a 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialRangeSegmentGenerateTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialRangeSegmentGenerateTask.java @@ -29,13 +29,13 @@ import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.common.actions.TaskActionClient; import org.apache.druid.indexing.common.task.IndexTaskClientFactory; +import org.apache.druid.indexing.common.task.SegmentAllocatorForBatch; import org.apache.druid.indexing.common.task.SegmentAllocators; import org.apache.druid.indexing.common.task.TaskResource; import org.apache.druid.indexing.common.task.batch.parallel.iterator.RangePartitionIndexTaskInputRowIteratorBuilder; import org.apache.druid.indexing.common.task.batch.partition.RangePartitionAnalysis; import org.apache.druid.indexing.worker.ShuffleDataSegmentPusher; import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager; -import org.apache.druid.segment.realtime.appenderator.SegmentAllocator; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.partition.BucketNumberedShardSpec; import org.apache.druid.timeline.partition.PartitionBoundaries; @@ -151,7 +151,7 @@ public boolean isReady(TaskActionClient taskActionClient) } @Override - SegmentAllocator createSegmentAllocator(TaskToolbox toolbox, ParallelIndexSupervisorTaskClient taskClient) + SegmentAllocatorForBatch createSegmentAllocator(TaskToolbox toolbox, ParallelIndexSupervisorTaskClient taskClient) throws IOException { final RangePartitionAnalysis partitionAnalysis = new RangePartitionAnalysis( diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentGenerateTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentGenerateTask.java index 1c6018ac360f..cbde9285cded 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentGenerateTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentGenerateTask.java @@ -28,10 +28,10 @@ import org.apache.druid.indexing.common.stats.DropwizardRowIngestionMeters; import org.apache.druid.indexing.common.stats.RowIngestionMeters; import org.apache.druid.indexing.common.task.BatchAppenderators; -import org.apache.druid.indexing.common.task.CachingLocalSegmentAllocator; import org.apache.druid.indexing.common.task.ClientBasedTaskInfoProvider; import org.apache.druid.indexing.common.task.IndexTaskClientFactory; import org.apache.druid.indexing.common.task.InputSourceProcessor; +import org.apache.druid.indexing.common.task.SegmentAllocatorForBatch; import org.apache.druid.indexing.common.task.SequenceNameFunction; import org.apache.druid.indexing.common.task.TaskResource; import org.apache.druid.indexing.common.task.Tasks; @@ -128,7 +128,7 @@ public final TaskStatus runTask(TaskToolbox toolbox) throws Exception /** * @return {@link SegmentAllocator} suitable for the desired segment partitioning strategy. */ - abstract SegmentAllocator createSegmentAllocator( + abstract SegmentAllocatorForBatch createSegmentAllocator( TaskToolbox toolbox, ParallelIndexSupervisorTaskClient taskClient ) throws IOException; @@ -170,8 +170,8 @@ private List generateSegments( final PartitionsSpec partitionsSpec = tuningConfig.getGivenOrDefaultPartitionsSpec(); final long pushTimeout = tuningConfig.getPushTimeout(); - final SegmentAllocator segmentAllocator = createSegmentAllocator(toolbox, taskClient); - final SequenceNameFunction sequenceNameFunction = ((CachingLocalSegmentAllocator) segmentAllocator).getSequenceNameFunction(); // TODO + final SegmentAllocatorForBatch segmentAllocator = createSegmentAllocator(toolbox, taskClient); + final SequenceNameFunction sequenceNameFunction = segmentAllocator.getSequenceNameFunction(); final Appenderator appenderator = BatchAppenderators.newAppenderator( getId(), diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseSubTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseSubTask.java index 9f1dc52c1ff4..61908a8f255b 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseSubTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseSubTask.java @@ -316,6 +316,7 @@ private Set generateAndPushSegments( final boolean explicitIntervals = granularitySpec.bucketIntervals().isPresent(); final SegmentAllocator segmentAllocator = SegmentAllocators.forLinearPartitioning( toolbox, + getId(), new SupervisorTaskAccess(getSupervisorTaskId(), taskClient), getIngestionSchema().getDataSchema(), getTaskLockHelper(), diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SubTaskReport.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SubTaskReport.java index 564b3af8ab6f..26f20f6f8688 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SubTaskReport.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SubTaskReport.java @@ -30,7 +30,6 @@ @JsonTypeInfo(use = Id.NAME, property = "type", defaultImpl = PushedSegmentsReport.class) @JsonSubTypes(value = { @Type(name = PushedSegmentsReport.TYPE, value = PushedSegmentsReport.class), - @Type(name = GeneratedHashPartitionsReport.TYPE, value = GeneratedHashPartitionsReport.class), @Type(name = DimensionDistributionReport.TYPE, value = DimensionDistributionReport.class), @Type(name = GeneratedPartitionsMetadataReport.TYPE, value = GeneratedPartitionsMetadataReport.class) }) 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 73fa1779bdae..470a00372b48 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 @@ -521,7 +521,6 @@ public static void prepareObjectMapper( new NamedType(ParallelIndexSupervisorTask.class, ParallelIndexSupervisorTask.TYPE), new NamedType(SinglePhaseSubTask.class, SinglePhaseSubTask.TYPE), new NamedType(PartialHashSegmentGenerateTask.class, PartialHashSegmentGenerateTask.TYPE), - new NamedType(PartialHashSegmentMergeTask.class, PartialHashSegmentMergeTask.TYPE), new NamedType(PartialRangeSegmentGenerateTask.class, PartialRangeSegmentGenerateTask.TYPE), new NamedType(PartialGenericSegmentMergeTask.class, PartialGenericSegmentMergeTask.TYPE), new NamedType(PartialDimensionDistributionTask.class, PartialDimensionDistributionTask.TYPE) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/GeneratedHashPartitionsReportTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/GeneratedHashPartitionsReportTest.java deleted file mode 100644 index 1343b9425833..000000000000 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/GeneratedHashPartitionsReportTest.java +++ /dev/null @@ -1,59 +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.batch.parallel; - -import com.fasterxml.jackson.databind.ObjectMapper; -import org.apache.druid.segment.TestHelper; -import org.junit.Before; -import org.junit.Test; - -import java.util.Collections; - -public class GeneratedHashPartitionsReportTest -{ - private static final ObjectMapper OBJECT_MAPPER = ParallelIndexTestingFactory.createObjectMapper(); - - private GeneratedHashPartitionsReport target; - - @Before - public void setup() - { - target = new GeneratedHashPartitionsReport( - "task-id", - Collections.singletonList( - new HashPartitionStat( - ParallelIndexTestingFactory.TASK_EXECUTOR_HOST, - ParallelIndexTestingFactory.TASK_EXECUTOR_PORT, - ParallelIndexTestingFactory.USE_HTTPS, - ParallelIndexTestingFactory.INTERVAL, - ParallelIndexTestingFactory.PARTITION_ID, - ParallelIndexTestingFactory.NUM_ROWS, - ParallelIndexTestingFactory.SIZE_BYTES - ) - ) - ); - } - - @Test - public void serializesDeserializes() - { - TestHelper.testSerializesDeserializes(OBJECT_MAPPER, target); - } -} diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionStatTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionStatTest.java deleted file mode 100644 index 6d86470b53ab..000000000000 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionStatTest.java +++ /dev/null @@ -1,59 +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.batch.parallel; - -import com.fasterxml.jackson.databind.ObjectMapper; -import org.apache.druid.segment.TestHelper; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; - -public class HashPartitionStatTest -{ - private static final ObjectMapper OBJECT_MAPPER = ParallelIndexTestingFactory.createObjectMapper(); - - private HashPartitionStat target; - - @Before - public void setup() - { - target = new HashPartitionStat( - ParallelIndexTestingFactory.TASK_EXECUTOR_HOST, - ParallelIndexTestingFactory.TASK_EXECUTOR_PORT, - ParallelIndexTestingFactory.USE_HTTPS, - ParallelIndexTestingFactory.INTERVAL, - ParallelIndexTestingFactory.PARTITION_ID, - ParallelIndexTestingFactory.NUM_ROWS, - ParallelIndexTestingFactory.SIZE_BYTES - ); - } - - @Test - public void serializesDeserializes() - { - TestHelper.testSerializesDeserializes(OBJECT_MAPPER, target); - } - - @Test - public void hasPartitionIdThatMatchesSecondaryPartition() - { - Assert.assertEquals(target.getSecondaryPartition().intValue(), target.getBucketId()); - } -} 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 88dac06b89ed..9f8a07dfaebd 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,6 +677,7 @@ public TaskStatus run(final TaskToolbox toolbox) throws Exception .getGivenOrDefaultPartitionsSpec(); final SegmentAllocator segmentAllocator = SegmentAllocators.forLinearPartitioning( toolbox, + getId(), new SupervisorTaskAccess(getSupervisorTaskId(), taskClient), getIngestionSchema().getDataSchema(), getTaskLockHelper(), 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 211d391b2f72..3ae79a3b6966 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,9 +19,11 @@ package org.apache.druid.indexing.common.task.batch.parallel; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.Ordering; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.Pair; +import org.apache.druid.timeline.partition.BuildingHashBasedNumberedShardSpec; import org.hamcrest.Matchers; import org.joda.time.Interval; import org.junit.Assert; @@ -45,8 +47,8 @@ public class ParallelIndexSupervisorTaskTest public static class CreateMergeIoConfigsTest { private static final int TOTAL_NUM_MERGE_TASKS = 10; - private static final Function, PartialHashSegmentMergeIOConfig> - CREATE_PARTIAL_SEGMENT_MERGE_IO_CONFIG = PartialHashSegmentMergeIOConfig::new; + private static final Function, PartialGenericSegmentMergeIOConfig> + CREATE_PARTIAL_SEGMENT_MERGE_IO_CONFIG = PartialGenericSegmentMergeIOConfig::new; @Parameterized.Parameters(name = "count = {0}") public static Iterable data() @@ -66,14 +68,14 @@ public static Iterable data() @Test public void handlesLastPartitionCorrectly() { - List assignedPartitionLocation = createMergeIOConfigs(); + List assignedPartitionLocation = createMergeIOConfigs(); assertNoMissingPartitions(count, assignedPartitionLocation); } @Test public void sizesPartitionsEvenly() { - List assignedPartitionLocation = createMergeIOConfigs(); + List assignedPartitionLocation = createMergeIOConfigs(); List actualPartitionSizes = assignedPartitionLocation.stream() .map(i -> i.getPartitionLocations().size()) .collect(Collectors.toList()); @@ -89,7 +91,7 @@ public void sizesPartitionsEvenly() ); } - private List createMergeIOConfigs() + private List createMergeIOConfigs() { return ParallelIndexSupervisorTask.createMergeIOConfigs( TOTAL_NUM_MERGE_TASKS, @@ -98,7 +100,7 @@ private List createMergeIOConfigs() ); } - private static Map, List> createPartitionToLocations(int count) + private static Map, List> createPartitionToLocations(int count) { return IntStream.range(0, count).boxed().collect( Collectors.toMap( @@ -108,15 +110,15 @@ private static Map, List> createP ); } - private static HashPartitionLocation createPartitionLocation(int id) + private static GenericPartitionLocation createPartitionLocation(int id) { - return new HashPartitionLocation( + return new GenericPartitionLocation( "host", 0, false, "subTaskId", createInterval(id), - id + new BuildingHashBasedNumberedShardSpec(id, id, id + 1, null, new ObjectMapper()) ); } @@ -127,7 +129,7 @@ private static Interval createInterval(int id) private static void assertNoMissingPartitions( int count, - List assignedPartitionLocation + List assignedPartitionLocation ) { List expectedIds = IntStream.range(0, count).boxed().collect(Collectors.toList()); @@ -136,7 +138,7 @@ private static void assertNoMissingPartitions( .flatMap( i -> i.getPartitionLocations() .stream() - .map(HashPartitionLocation::getBucketId) + .map(GenericPartitionLocation::getBucketId) ) .sorted() .collect(Collectors.toList()); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentMergeIOConfigTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentMergeIOConfigTest.java deleted file mode 100644 index 413c34d9d668..000000000000 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentMergeIOConfigTest.java +++ /dev/null @@ -1,54 +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.batch.parallel; - -import com.fasterxml.jackson.databind.ObjectMapper; -import org.apache.druid.segment.TestHelper; -import org.junit.Before; -import org.junit.Test; - -import java.util.Collections; - -public class PartialHashSegmentMergeIOConfigTest -{ - private static final ObjectMapper OBJECT_MAPPER = ParallelIndexTestingFactory.createObjectMapper(); - private static final HashPartitionLocation HASH_PARTITION_LOCATION = new HashPartitionLocation( - ParallelIndexTestingFactory.HOST, - ParallelIndexTestingFactory.PORT, - ParallelIndexTestingFactory.USE_HTTPS, - ParallelIndexTestingFactory.SUBTASK_ID, - ParallelIndexTestingFactory.INTERVAL, - ParallelIndexTestingFactory.PARTITION_ID - ); - - private PartialHashSegmentMergeIOConfig target; - - @Before - public void setup() - { - target = new PartialHashSegmentMergeIOConfig(Collections.singletonList(HASH_PARTITION_LOCATION)); - } - - @Test - public void serializesDeserializes() - { - TestHelper.testSerializesDeserializes(OBJECT_MAPPER, target); - } -} diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentMergeIngestionSpecTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentMergeIngestionSpecTest.java deleted file mode 100644 index d734739a0146..000000000000 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentMergeIngestionSpecTest.java +++ /dev/null @@ -1,68 +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.batch.parallel; - -import com.fasterxml.jackson.databind.ObjectMapper; -import org.apache.druid.indexer.partitions.HashedPartitionsSpec; -import org.apache.druid.segment.TestHelper; -import org.junit.Before; -import org.junit.Test; - -import java.util.Collections; - -public class PartialHashSegmentMergeIngestionSpecTest -{ - private static final ObjectMapper OBJECT_MAPPER = ParallelIndexTestingFactory.createObjectMapper(); - private static final HashPartitionLocation HASH_PARTITION_LOCATION = new HashPartitionLocation( - ParallelIndexTestingFactory.HOST, - ParallelIndexTestingFactory.PORT, - ParallelIndexTestingFactory.USE_HTTPS, - ParallelIndexTestingFactory.SUBTASK_ID, - ParallelIndexTestingFactory.INTERVAL, - ParallelIndexTestingFactory.PARTITION_ID - ); - private static final PartialHashSegmentMergeIOConfig IO_CONFIG = - new PartialHashSegmentMergeIOConfig(Collections.singletonList(HASH_PARTITION_LOCATION)); - private static final HashedPartitionsSpec PARTITIONS_SPEC = new HashedPartitionsSpec( - null, - 1, - Collections.emptyList() - ); - - private PartialHashSegmentMergeIngestionSpec target; - - @Before - public void setup() - { - target = new PartialHashSegmentMergeIngestionSpec( - ParallelIndexTestingFactory.createDataSchema(ParallelIndexTestingFactory.INPUT_INTERVALS), - IO_CONFIG, - new ParallelIndexTestingFactory.TuningConfigBuilder() - .partitionsSpec(PARTITIONS_SPEC) - .build() - ); - } - - @Test - public void serializesDeserializes() - { - TestHelper.testSerializesDeserializes(OBJECT_MAPPER, target); - } -} diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentMergeTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentMergeTaskTest.java deleted file mode 100644 index d6fe0bb864c4..000000000000 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentMergeTaskTest.java +++ /dev/null @@ -1,88 +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.batch.parallel; - -import org.apache.druid.indexer.partitions.HashedPartitionsSpec; -import org.apache.druid.segment.TestHelper; -import org.hamcrest.Matchers; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; - -import java.util.Collections; - -public class PartialHashSegmentMergeTaskTest extends AbstractParallelIndexSupervisorTaskTest -{ - private static final HashPartitionLocation HASH_PARTITION_LOCATION = new HashPartitionLocation( - ParallelIndexTestingFactory.HOST, - ParallelIndexTestingFactory.PORT, - ParallelIndexTestingFactory.USE_HTTPS, - ParallelIndexTestingFactory.SUBTASK_ID, - ParallelIndexTestingFactory.INTERVAL, - ParallelIndexTestingFactory.PARTITION_ID - ); - private static final PartialHashSegmentMergeIOConfig IO_CONFIG = - new PartialHashSegmentMergeIOConfig(Collections.singletonList(HASH_PARTITION_LOCATION)); - private static final HashedPartitionsSpec PARTITIONS_SPEC = new HashedPartitionsSpec( - null, - 1, - Collections.emptyList() - ); - private static final PartialHashSegmentMergeIngestionSpec INGESTION_SPEC = - new PartialHashSegmentMergeIngestionSpec( - ParallelIndexTestingFactory.createDataSchema(ParallelIndexTestingFactory.INPUT_INTERVALS), - IO_CONFIG, - new ParallelIndexTestingFactory.TuningConfigBuilder() - .partitionsSpec(PARTITIONS_SPEC) - .build() - ); - - private PartialHashSegmentMergeTask target; - - @Before - public void setup() - { - target = new PartialHashSegmentMergeTask( - ParallelIndexTestingFactory.AUTOMATIC_ID, - ParallelIndexTestingFactory.GROUP_ID, - ParallelIndexTestingFactory.TASK_RESOURCE, - ParallelIndexTestingFactory.SUPERVISOR_TASK_ID, - ParallelIndexTestingFactory.NUM_ATTEMPTS, - INGESTION_SPEC, - ParallelIndexTestingFactory.CONTEXT, - ParallelIndexTestingFactory.INDEXING_SERVICE_CLIENT, - ParallelIndexTestingFactory.TASK_CLIENT_FACTORY, - ParallelIndexTestingFactory.SHUFFLE_CLIENT - ); - } - - @Test - public void serializesDeserializes() - { - TestHelper.testSerializesDeserializes(getObjectMapper(), target); - } - - @Test - public void hasCorrectPrefixForAutomaticId() - { - String id = target.getId(); - Assert.assertThat(id, Matchers.startsWith(PartialHashSegmentMergeTask.TYPE)); - } -} 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 7309e7c8641f..5c64dcd626b0 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 @@ -25,7 +25,6 @@ import org.apache.druid.indexing.common.task.batch.parallel.PartialDimensionDistributionTask; import org.apache.druid.indexing.common.task.batch.parallel.PartialGenericSegmentMergeTask; import org.apache.druid.indexing.common.task.batch.parallel.PartialHashSegmentGenerateTask; -import org.apache.druid.indexing.common.task.batch.parallel.PartialHashSegmentMergeTask; import org.apache.druid.indexing.common.task.batch.parallel.PartialRangeSegmentGenerateTask; import org.apache.druid.indexing.common.task.batch.parallel.SinglePhaseSubTask; import org.apache.druid.java.util.common.ISE; @@ -312,7 +311,6 @@ private long countCompleteSubTasks(final String dataSource, final boolean perfec return t.getType().equals(SinglePhaseSubTask.TYPE); } else { return t.getType().equalsIgnoreCase(PartialHashSegmentGenerateTask.TYPE) - || t.getType().equalsIgnoreCase(PartialHashSegmentMergeTask.TYPE) || t.getType().equalsIgnoreCase(PartialDimensionDistributionTask.TYPE) || t.getType().equalsIgnoreCase(PartialRangeSegmentGenerateTask.TYPE) || t.getType().equalsIgnoreCase(PartialGenericSegmentMergeTask.TYPE); From 7d0d3b90918793f36681e6201985156df2a1ca7e Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Sat, 13 Jun 2020 15:21:40 -0700 Subject: [PATCH 19/24] resolved todos --- .../common/task/CachingLocalSegmentAllocator.java | 5 ++++- .../apache/druid/client/CachingClusteredClientTest.java | 8 +++++++- 2 files changed, 11 insertions(+), 2 deletions(-) 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 36c183612d1d..26c9a7de107e 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 @@ -140,7 +140,10 @@ public SegmentIdWithShardSpec allocate( sequenceName ); final Interval interval = pair.lhs; - // TODO: fuck..... i hate this code + // Determines the partitionId if this segment allocator is used by the single-threaded task. + // In parallel ingestion, the partitionId is determined in the supervisor task. + // See ParallelIndexSupervisorTask.groupGenericPartitionLocationsPerPartition(). + // This code... isn't pretty, but should be simple enough to understand. final ShardSpec shardSpec = isParallel ? pair.rhs : pair.rhs.convert( 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 0e5b297f9602..30fb7659c11e 100644 --- a/server/src/test/java/org/apache/druid/client/CachingClusteredClientTest.java +++ b/server/src/test/java/org/apache/druid/client/CachingClusteredClientTest.java @@ -1538,7 +1538,13 @@ private ServerSelector makeMockSingleDimensionSelector( null, null, null, - new SingleDimensionShardSpec(dimension, start, end, partitionNum, SingleDimensionShardSpec.UNKNOWN_NUM_CORE_PARTITIONS), // TODO + new SingleDimensionShardSpec( + dimension, + start, + end, + partitionNum, + SingleDimensionShardSpec.UNKNOWN_NUM_CORE_PARTITIONS + ), null, 9, 0L From 6cff5b86a07367b73d0e3f147366ffd3ecb01334 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Sat, 13 Jun 2020 16:40:56 -0700 Subject: [PATCH 20/24] javadocs --- .../partition/BucketNumberedShardSpec.java | 45 ++++++++++++++++++- .../partition/BuildingNumberedShardSpec.java | 4 +- .../timeline/partition/BuildingShardSpec.java | 18 ++++---- .../BuildingSingleDimensionShardSpec.java | 4 +- .../partition/HashBucketShardSpec.java | 18 ++------ .../partition/OverwriteShardSpec.java | 4 +- .../partition/RangeBucketShardSpec.java | 16 +------ .../druid/timeline/partition/ShardSpec.java | 7 ++- .../task/CachingLocalSegmentAllocator.java | 8 ++-- .../indexing/common/task/ShardSpecs.java | 8 ++-- .../parallel/ParallelIndexSupervisorTask.java | 3 +- .../partition/CompletePartitionAnalysis.java | 3 +- .../partition/HashPartitionAnalysis.java | 6 +-- .../partition/RangePartitionAnalysis.java | 6 +-- .../indexing/common/task/ShardSpecsTest.java | 6 +-- 15 files changed, 93 insertions(+), 63 deletions(-) diff --git a/core/src/main/java/org/apache/druid/timeline/partition/BucketNumberedShardSpec.java b/core/src/main/java/org/apache/druid/timeline/partition/BucketNumberedShardSpec.java index c9cc37ae9505..d692dad113da 100644 --- a/core/src/main/java/org/apache/druid/timeline/partition/BucketNumberedShardSpec.java +++ b/core/src/main/java/org/apache/druid/timeline/partition/BucketNumberedShardSpec.java @@ -26,7 +26,36 @@ import java.util.Map; /** - * TODO + * This is one of the special shardSpecs which are temporarily used during batch ingestion. In Druid, there is a + * concept of core partition set which is a set of segments atomically becoming queryable together in Brokers. The core + * partition set is represented as a range of partitionIds, i.e., [0, {@link ShardSpec#getNumCorePartitions()}). + * + * When you run a batch ingestion task with a non-linear partitioning scheme, the task populates all possible buckets + * upfront at the beginning (see {@code CachingLocalSegmentAllocator}) and uses them to partition input rows. However, + * some of the buckets can be empty even after the task consumes all inputs if the data is highly skewed. Since Druid + * doesn't create empty segments, the partitionId should be dynamically allocated when a bucket is actually in use, + * so that we can always create the packed core partition set without missing partitionIds. + * + * This BucketNumberedShardSpec is used for such use case. The task with a non-linear partitioning scheme uses it + * to postpone the partitionId allocation until all empty buckets are identified. See + * {@code ParallelIndexSupervisorTask.groupGenericPartitionLocationsPerPartition} and + * {@code CachingLocalSegmentAllocator} for parallel and sequential ingestion, respectively. + * + * Note that {@link org.apache.druid.timeline.SegmentId} requires the partitionId. Since the segmentId is used + * everwhere during ingestion, this class should implement {@link #getPartitionNum()} which returns the bucketId. + * This should be fine because the segmentId is only used to identify each segment until pushing them to deep storage. + * The bucketId should be enough to uniquely identify each segment. However, when pushing segments to deep storage, + * the partitionId is used to create the path to store the segment on deep storage + * ({@link org.apache.druid.segment.loading.DataSegmentPusher#getDefaultStorageDir} which should be correct. + * As a result, this shardSpec should not be used in pushing segments. + * + * This class should be Jackson-serializable as the subtasks can send it to the parallel task in parallel ingestion. + * + * This interface doesn't really have to extend {@link ShardSpec}. The only reason is the ShardSpec is used in many + * places such as {@link org.apache.druid.timeline.DataSegment}, and we have to modify those places to allow other + * types than ShardSpec which seems pretty invasive. Maybe we could clean up this mess someday in the future. + * + * @see BuildingShardSpec */ public interface BucketNumberedShardSpec extends ShardSpec { @@ -34,6 +63,20 @@ public interface BucketNumberedShardSpec extends Sh T convert(int partitionId); + @Override + default PartitionChunk createChunk(O obj) + { + // The partitionId (or partitionNum, chunkNumber) is not determined yet. Use bucketId for now. + return new NumberedPartitionChunk<>(getBucketId(), 0, obj); + } + + @Override + default int getPartitionNum() + { + // See the class-level Javadoc for returning bucketId here. + return getBucketId(); + } + @Override default int getNumCorePartitions() { diff --git a/core/src/main/java/org/apache/druid/timeline/partition/BuildingNumberedShardSpec.java b/core/src/main/java/org/apache/druid/timeline/partition/BuildingNumberedShardSpec.java index 7b2739385753..a179d3ca7bac 100644 --- a/core/src/main/java/org/apache/druid/timeline/partition/BuildingNumberedShardSpec.java +++ b/core/src/main/java/org/apache/druid/timeline/partition/BuildingNumberedShardSpec.java @@ -49,8 +49,8 @@ public BuildingNumberedShardSpec(@JsonProperty("partitionId") int partitionId) @Override public int getBucketId() { - // TODO: explain... - return 0; + // This method is currently not called when the shardSpec type is this class. + throw new UnsupportedOperationException(); } @Override diff --git a/core/src/main/java/org/apache/druid/timeline/partition/BuildingShardSpec.java b/core/src/main/java/org/apache/druid/timeline/partition/BuildingShardSpec.java index c6066af7d126..973fdf4d2a7d 100644 --- a/core/src/main/java/org/apache/druid/timeline/partition/BuildingShardSpec.java +++ b/core/src/main/java/org/apache/druid/timeline/partition/BuildingShardSpec.java @@ -27,10 +27,9 @@ import java.util.Map; /** - * This is a special shardSpec which is temporarily used during batch ingestion. In Druid, there is a concept - * of core partition set which is a set of segments atomically becoming queryable together in Brokers. The core - * partition set is represented as a range of partitionIds. For {@link NumberedShardSpec} as an example, the core - * partition set is [0, {@link NumberedShardSpec#partitions}). + * This is one of the special shardSpecs which are temporarily used during batch ingestion. In Druid, there is a + * concept of core partition set which is a set of segments atomically becoming queryable together in Brokers. The core + * partition set is represented as a range of partitionIds, i.e., [0, {@link ShardSpec#getNumCorePartitions()}). * * In streaming ingestion, the core partition set size cannot be determined since it's impossible to know how many * segments will be created per time chunk upfront. However, in batch ingestion with time chunk locking, the core @@ -47,6 +46,12 @@ * generation segments). * * This class should be Jackson-serializable as the subtasks can send it to the parallel task in parallel ingestion. + * + * This interface doesn't really have to extend {@link ShardSpec}. The only reason is the ShardSpec is used in many + * places such as {@link org.apache.druid.timeline.DataSegment}, and we have to modify those places to allow other + * types than ShardSpec which seems pretty invasive. Maybe we could clean up this mess someday in the future. + * + * @see BucketNumberedShardSpec */ public interface BuildingShardSpec extends ShardSpec { @@ -54,9 +59,6 @@ public interface BuildingShardSpec extends ShardSpec T convert(int numCorePartitions); - /** - * TODO - */ @Override default int getNumCorePartitions() { @@ -64,7 +66,7 @@ default int getNumCorePartitions() } /** - * TODO + * {@link BucketNumberedShardSpec} should be used for shard spec lookup. */ @Override default ShardSpecLookup getLookup(List shardSpecs) diff --git a/core/src/main/java/org/apache/druid/timeline/partition/BuildingSingleDimensionShardSpec.java b/core/src/main/java/org/apache/druid/timeline/partition/BuildingSingleDimensionShardSpec.java index 8b97fb1250ac..6dd099205447 100644 --- a/core/src/main/java/org/apache/druid/timeline/partition/BuildingSingleDimensionShardSpec.java +++ b/core/src/main/java/org/apache/druid/timeline/partition/BuildingSingleDimensionShardSpec.java @@ -26,7 +26,9 @@ import java.util.Objects; /** - * TODO + * See {@link BuildingShardSpec} for how this class is used. + * + * @see SingleDimensionShardSpec */ public class BuildingSingleDimensionShardSpec implements BuildingShardSpec { diff --git a/core/src/main/java/org/apache/druid/timeline/partition/HashBucketShardSpec.java b/core/src/main/java/org/apache/druid/timeline/partition/HashBucketShardSpec.java index 90af43536c06..395ab0fded2e 100644 --- a/core/src/main/java/org/apache/druid/timeline/partition/HashBucketShardSpec.java +++ b/core/src/main/java/org/apache/druid/timeline/partition/HashBucketShardSpec.java @@ -29,7 +29,9 @@ import java.util.Objects; /** - * TODO + * See {@link BucketNumberedShardSpec} for how this class is used. + * + * @see BuildingHashBasedNumberedShardSpec */ public class HashBucketShardSpec implements BucketNumberedShardSpec { @@ -79,13 +81,6 @@ public BuildingHashBasedNumberedShardSpec convert(int partitionId) return new BuildingHashBasedNumberedShardSpec(partitionId, bucketId, numBuckets, partitionDimensions, jsonMapper); } - @Override - public PartitionChunk createChunk(T obj) - { - // TODO: explain.. - return new NumberedPartitionChunk<>(bucketId, 0, obj); - } - @Override public boolean isInChunk(long timestamp, InputRow inputRow) { @@ -93,13 +88,6 @@ public boolean isInChunk(long timestamp, InputRow inputRow) throw new UnsupportedOperationException(); } - @Override - public int getPartitionNum() - { - // TODO: explain.. - return bucketId; - } - @Override public ShardSpecLookup getLookup(List shardSpecs) { diff --git a/core/src/main/java/org/apache/druid/timeline/partition/OverwriteShardSpec.java b/core/src/main/java/org/apache/druid/timeline/partition/OverwriteShardSpec.java index 193b573acbf5..6a77ea55286b 100644 --- a/core/src/main/java/org/apache/druid/timeline/partition/OverwriteShardSpec.java +++ b/core/src/main/java/org/apache/druid/timeline/partition/OverwriteShardSpec.java @@ -29,7 +29,9 @@ public interface OverwriteShardSpec extends ShardSpec { /** - * TODO + * The core partition concept is not used with segment locking. Instead, the {@link AtomicUpdateGroup} is used + * to atomically overshadow segments. Here, we always returns 0 so that the {@link PartitionHolder} skips checking + * the completeness of the core partitions. */ @Override default int getNumCorePartitions() diff --git a/core/src/main/java/org/apache/druid/timeline/partition/RangeBucketShardSpec.java b/core/src/main/java/org/apache/druid/timeline/partition/RangeBucketShardSpec.java index 919ad6aadb6e..a329131e7487 100644 --- a/core/src/main/java/org/apache/druid/timeline/partition/RangeBucketShardSpec.java +++ b/core/src/main/java/org/apache/druid/timeline/partition/RangeBucketShardSpec.java @@ -28,7 +28,9 @@ import java.util.Objects; /** + * See {@link BucketNumberedShardSpec} for how this class is used. * + * @see BuildingSingleDimensionShardSpec */ public class RangeBucketShardSpec implements BucketNumberedShardSpec { @@ -94,20 +96,6 @@ public boolean isInChunk(long timestamp, InputRow inputRow) return SingleDimensionShardSpec.isInChunk(dimension, start, end, inputRow); } - @Override - public PartitionChunk createChunk(T obj) - { - // TODO: explain.. - return new NumberedPartitionChunk<>(bucketId, 0, obj); - } - - @Override - public int getPartitionNum() - { - // TODO: explain.. - return bucketId; - } - @Override public ShardSpecLookup getLookup(List shardSpecs) { 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 1f99b16058bf..8f59d3959e72 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 @@ -40,11 +40,14 @@ @JsonSubTypes.Type(name = "numbered", value = NumberedShardSpec.class), @JsonSubTypes.Type(name = "hashed", value = HashBasedNumberedShardSpec.class), @JsonSubTypes.Type(name = NumberedOverwriteShardSpec.TYPE, value = NumberedOverwriteShardSpec.class), - // TODO: shouldn't be published + // BuildingShardSpecs are the shardSpec with missing numCorePartitions, and thus must not be published. + // See BuildingShardSpec for more details. @JsonSubTypes.Type(name = BuildingNumberedShardSpec.TYPE, value = BuildingNumberedShardSpec.class), @JsonSubTypes.Type(name = BuildingHashBasedNumberedShardSpec.TYPE, value = BuildingHashBasedNumberedShardSpec.class), @JsonSubTypes.Type(name = BuildingSingleDimensionShardSpec.TYPE, value = BuildingSingleDimensionShardSpec.class), - // TODO: shouldn't be used in push + // BucketShardSpecs are the shardSpec with missing partitionId and numCorePartitions. + // These shardSpecs must not be used in segment push. + // See BucketShardSpec for more details. @JsonSubTypes.Type(name = HashBucketShardSpec.TYPE, value = HashBucketShardSpec.class), @JsonSubTypes.Type(name = RangeBucketShardSpec.TYPE, value = RangeBucketShardSpec.class) }) 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 26c9a7de107e..0ad2e99435a5 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 @@ -83,16 +83,18 @@ public class CachingLocalSegmentAllocator implements SegmentAllocatorForBatch } this.versionFinder = createVersionFinder(toolbox, action); - final Map> intervalToShardSpecs = partitionAnalysis.createBuckets(toolbox); + final Map>> intervalToShardSpecs = partitionAnalysis.createBuckets( + toolbox + ); sequenceNameFunction = new NonLinearlyPartitionedSequenceNameFunction( taskId, new ShardSpecs(intervalToShardSpecs, granularitySpec.getQueryGranularity()) ); - for (Entry> entry : intervalToShardSpecs.entrySet()) { + for (Entry>> entry : intervalToShardSpecs.entrySet()) { final Interval interval = entry.getKey(); - final List buckets = entry.getValue(); + final List> buckets = entry.getValue(); buckets.forEach(bucket -> { sequenceNameToBucket.put(sequenceNameFunction.getSequenceName(interval, bucket), Pair.of(interval, bucket)); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/ShardSpecs.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/ShardSpecs.java index 0d0994634e7f..3db4beba2735 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/ShardSpecs.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/ShardSpecs.java @@ -33,10 +33,10 @@ */ public class ShardSpecs { - private final Map> map; - private Granularity queryGranularity; + private final Map>> map; + private final Granularity queryGranularity; - ShardSpecs(final Map> map, Granularity queryGranularity) + ShardSpecs(final Map>> map, Granularity queryGranularity) { this.map = map; this.queryGranularity = queryGranularity; @@ -52,7 +52,7 @@ public class ShardSpecs */ BucketNumberedShardSpec getShardSpec(Interval interval, InputRow row) { - final List shardSpecs = map.get(interval); + final List> shardSpecs = map.get(interval); if (shardSpecs == null || shardSpecs.isEmpty()) { throw new ISE("Failed to get shardSpec 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 705ae09d894e..bed85dea6926 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 @@ -653,7 +653,8 @@ private static Map, List> grou final BuildingShardSpec shardSpec = intervalAndIntegerToShardSpec.computeIfAbsent( Pair.of(partitionStat.getInterval(), partitionStat.getBucketId()), key -> { - // TODO: explain why we determine partition id here.. + // Lazily determine the partitionId to create packed partitionIds for the core partitions. + // See the Javadoc of BucketNumberedShardSpec for details. final int partitionId = intervalToNextPartitionId.computeInt( partitionStat.getInterval(), ((interval, nextPartitionId) -> nextPartitionId == null ? 0 : nextPartitionId + 1) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/partition/CompletePartitionAnalysis.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/partition/CompletePartitionAnalysis.java index 1bc6e9b60840..efecdda5e096 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/partition/CompletePartitionAnalysis.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/partition/CompletePartitionAnalysis.java @@ -34,6 +34,5 @@ */ public interface CompletePartitionAnalysis extends PartitionAnalysis { - // TODO: strongly typed - Map> createBuckets(TaskToolbox toolbox); + Map>> createBuckets(TaskToolbox toolbox); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/partition/HashPartitionAnalysis.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/partition/HashPartitionAnalysis.java index ab37276c7f7c..5773f095c10f 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/partition/HashPartitionAnalysis.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/partition/HashPartitionAnalysis.java @@ -91,13 +91,13 @@ public void forEach(BiConsumer consumer) } @Override - public Map> createBuckets(TaskToolbox toolbox) + public Map>> createBuckets(TaskToolbox toolbox) { - final Map> intervalToLookup = Maps.newHashMapWithExpectedSize( + final Map>> intervalToLookup = Maps.newHashMapWithExpectedSize( intervalToNumBuckets.size() ); forEach((interval, numBuckets) -> { - final List buckets = IntStream + final List> buckets = IntStream .range(0, numBuckets) .mapToObj(i -> new HashBucketShardSpec( i, diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/partition/RangePartitionAnalysis.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/partition/RangePartitionAnalysis.java index 3d545b4c9ec7..c8a2b8804d07 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/partition/RangePartitionAnalysis.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/partition/RangePartitionAnalysis.java @@ -92,7 +92,7 @@ public int getNumTimePartitions() * Translate {@link PartitionBoundaries} into the corresponding * {@link SingleDimensionPartitionsSpec} with segment id. */ - private static List translatePartitionBoundaries( + private static List> translatePartitionBoundaries( String partitionDimension, PartitionBoundaries partitionBoundaries ) @@ -112,10 +112,10 @@ private static List translatePartitionBoundaries( } @Override - public Map> createBuckets(TaskToolbox toolbox) + public Map>> createBuckets(TaskToolbox toolbox) { final String partitionDimension = partitionsSpec.getPartitionDimension(); - final Map> intervalToSegmentIds = Maps.newHashMapWithExpectedSize( + final Map>> intervalToSegmentIds = Maps.newHashMapWithExpectedSize( getNumTimePartitions() ); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ShardSpecsTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ShardSpecsTest.java index ae681c17d6bc..0818605fc8ea 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ShardSpecsTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ShardSpecsTest.java @@ -52,10 +52,10 @@ public ShardSpecsTest() @Test public void testShardSpecSelectionWithNullPartitionDimension() { - BucketNumberedShardSpec spec1 = new HashBucketShardSpec(0, 2, null, jsonMapper); - BucketNumberedShardSpec spec2 = new HashBucketShardSpec(1, 2, null, jsonMapper); + HashBucketShardSpec spec1 = new HashBucketShardSpec(0, 2, null, jsonMapper); + HashBucketShardSpec spec2 = new HashBucketShardSpec(1, 2, null, jsonMapper); - Map> shardSpecMap = new HashMap<>(); + Map>> shardSpecMap = new HashMap<>(); shardSpecMap.put(Intervals.of("2014-01-01T00:00:00.000Z/2014-01-02T00:00:00.000Z"), ImmutableList.of(spec1, spec2)); ShardSpecs shardSpecs = new ShardSpecs(shardSpecMap, Granularities.HOUR); From 07a9e1b984abb976e3105104400e22032a73748f Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Sat, 13 Jun 2020 16:48:11 -0700 Subject: [PATCH 21/24] Fix tests --- .../partition/HashBucketShardSpec.java | 4 ++- ...uildingHashBasedNumberedShardSpecTest.java | 2 -- .../BuildingNumberedShardSpecTest.java | 31 +---------------- .../NumberedOverwriteShardSpecTest.java | 2 +- ...itionCachingLocalSegmentAllocatorTest.java | 33 ++++++++++--------- ...itionCachingLocalSegmentAllocatorTest.java | 5 +-- 6 files changed, 25 insertions(+), 52 deletions(-) diff --git a/core/src/main/java/org/apache/druid/timeline/partition/HashBucketShardSpec.java b/core/src/main/java/org/apache/druid/timeline/partition/HashBucketShardSpec.java index 395ab0fded2e..324c02044985 100644 --- a/core/src/main/java/org/apache/druid/timeline/partition/HashBucketShardSpec.java +++ b/core/src/main/java/org/apache/druid/timeline/partition/HashBucketShardSpec.java @@ -52,7 +52,9 @@ public HashBucketShardSpec( { this.bucketId = bucketId; this.numBuckets = numBuckets; - this.partitionDimensions = partitionDimensions; + this.partitionDimensions = partitionDimensions == null + ? HashBasedNumberedShardSpec.DEFAULT_PARTITION_DIMENSIONS + : partitionDimensions; this.jsonMapper = jsonMapper; } diff --git a/core/src/test/java/org/apache/druid/timeline/partition/BuildingHashBasedNumberedShardSpecTest.java b/core/src/test/java/org/apache/druid/timeline/partition/BuildingHashBasedNumberedShardSpecTest.java index 48c5a874865f..2c052d51c228 100644 --- a/core/src/test/java/org/apache/druid/timeline/partition/BuildingHashBasedNumberedShardSpecTest.java +++ b/core/src/test/java/org/apache/druid/timeline/partition/BuildingHashBasedNumberedShardSpecTest.java @@ -20,9 +20,7 @@ package org.apache.druid.timeline.partition; import com.fasterxml.jackson.core.JsonProcessingException; -import com.fasterxml.jackson.databind.DeserializationFeature; import com.fasterxml.jackson.databind.InjectableValues.Std; -import com.fasterxml.jackson.databind.MapperFeature; import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.jsontype.NamedType; import com.google.common.collect.ImmutableList; diff --git a/core/src/test/java/org/apache/druid/timeline/partition/BuildingNumberedShardSpecTest.java b/core/src/test/java/org/apache/druid/timeline/partition/BuildingNumberedShardSpecTest.java index 4cc8e67b7237..b608d4cda730 100644 --- a/core/src/test/java/org/apache/druid/timeline/partition/BuildingNumberedShardSpecTest.java +++ b/core/src/test/java/org/apache/druid/timeline/partition/BuildingNumberedShardSpecTest.java @@ -23,16 +23,10 @@ import com.fasterxml.jackson.databind.DeserializationFeature; import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.jsontype.NamedType; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; import nl.jqno.equalsverifier.EqualsVerifier; -import org.apache.druid.data.input.MapBasedInputRow; -import org.apache.druid.java.util.common.DateTimes; import org.junit.Assert; import org.junit.Test; -import java.util.List; - public class BuildingNumberedShardSpecTest { @Test @@ -50,33 +44,10 @@ public void testCreateChunk() ); } - @Test - public void testShardSpecLookup() - { - final List shardSpecs = ImmutableList.of( - new BuildingNumberedShardSpec(1), - new BuildingNumberedShardSpec(2), - new BuildingNumberedShardSpec(3) - ); - final ShardSpecLookup lookup = shardSpecs.get(0).getLookup(shardSpecs); - // Timestamp doesn't matter. It always returns the first shardSpec. - final long currentTime = DateTimes.nowUtc().getMillis(); - Assert.assertEquals( - shardSpecs.get(0), - lookup.getShardSpec( - currentTime, - new MapBasedInputRow( - currentTime, - ImmutableList.of("dim"), ImmutableMap.of("dim", "val", "time", currentTime) - ) - ) - ); - } - @Test public void testSerde() throws JsonProcessingException { - final ObjectMapper mapper = new ObjectMapper(); + final ObjectMapper mapper = ShardSpecTestUtils.initObjectMapper(); mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false); mapper.registerSubtypes(new NamedType(BuildingNumberedShardSpec.class, BuildingNumberedShardSpec.TYPE)); final BuildingNumberedShardSpec original = new BuildingNumberedShardSpec(5); diff --git a/core/src/test/java/org/apache/druid/timeline/partition/NumberedOverwriteShardSpecTest.java b/core/src/test/java/org/apache/druid/timeline/partition/NumberedOverwriteShardSpecTest.java index d605105c48d5..c6d7935b5699 100644 --- a/core/src/test/java/org/apache/druid/timeline/partition/NumberedOverwriteShardSpecTest.java +++ b/core/src/test/java/org/apache/druid/timeline/partition/NumberedOverwriteShardSpecTest.java @@ -37,7 +37,7 @@ public void testEquals() @Test public void testSerde() throws JsonProcessingException { - final ObjectMapper mapper = new ObjectMapper(); + final ObjectMapper mapper = ShardSpecTestUtils.initObjectMapper(); mapper.registerSubtypes(new NamedType(NumberedOverwriteShardSpec.class, NumberedOverwriteShardSpec.TYPE)); final NumberedOverwriteShardSpec original = new NumberedOverwriteShardSpec( PartitionIds.NON_ROOT_GEN_START_PARTITION_ID + 2, diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/RangePartitionCachingLocalSegmentAllocatorTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/RangePartitionCachingLocalSegmentAllocatorTest.java index 6cda8addccc0..affb04319520 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/RangePartitionCachingLocalSegmentAllocatorTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/RangePartitionCachingLocalSegmentAllocatorTest.java @@ -37,6 +37,7 @@ import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; import org.apache.druid.timeline.SegmentId; import org.apache.druid.timeline.partition.PartitionBoundaries; +import org.apache.druid.timeline.partition.RangeBucketShardSpec; import org.apache.druid.timeline.partition.SingleDimensionShardSpec; import org.easymock.EasyMock; import org.joda.time.Interval; @@ -166,37 +167,37 @@ public void getSequenceName() } @SuppressWarnings("SameParameterValue") - private void testAllocate(InputRow row, Interval interval, int partitionNum) + private void testAllocate(InputRow row, Interval interval, int bucketId) { - String partitionEnd = getPartitionEnd(interval, partitionNum); - testAllocate(row, interval, partitionNum, partitionEnd); + String partitionEnd = getPartitionEnd(interval, bucketId); + testAllocate(row, interval, bucketId, partitionEnd); } @Nullable - private static String getPartitionEnd(Interval interval, int partitionNum) + private static String getPartitionEnd(Interval interval, int bucketId) { PartitionBoundaries partitions = INTERVAL_TO_PARTITONS.get(interval); - boolean isLastPartition = (partitionNum + 1) == partitions.size(); - return isLastPartition ? null : partitions.get(partitionNum + 1); + boolean isLastPartition = (bucketId + 1) == partitions.size(); + return isLastPartition ? null : partitions.get(bucketId + 1); } - private void testAllocate(InputRow row, Interval interval, int partitionNum, @Nullable String partitionEnd) + private void testAllocate(InputRow row, Interval interval, int bucketId, @Nullable String partitionEnd) { - String partitionStart = getPartitionStart(interval, partitionNum); - testAllocate(row, interval, partitionNum, partitionStart, partitionEnd); + String partitionStart = getPartitionStart(interval, bucketId); + testAllocate(row, interval, bucketId, partitionStart, partitionEnd); } @Nullable - private static String getPartitionStart(Interval interval, int partitionNum) + private static String getPartitionStart(Interval interval, int bucketId) { - boolean isFirstPartition = partitionNum == 0; - return isFirstPartition ? null : INTERVAL_TO_PARTITONS.get(interval).get(partitionNum); + boolean isFirstPartition = bucketId == 0; + return isFirstPartition ? null : INTERVAL_TO_PARTITONS.get(interval).get(bucketId); } private void testAllocate( InputRow row, Interval interval, - int partitionNum, + int bucketId, @Nullable String partitionStart, @Nullable String partitionEnd ) @@ -205,12 +206,12 @@ private void testAllocate( SegmentIdWithShardSpec segmentIdWithShardSpec = allocate(row, sequenceName); Assert.assertEquals( - SegmentId.of(DATASOURCE, interval, INTERVAL_TO_VERSION.get(interval), partitionNum), + SegmentId.of(DATASOURCE, interval, INTERVAL_TO_VERSION.get(interval), bucketId), segmentIdWithShardSpec.asSegmentId() ); - SingleDimensionShardSpec shardSpec = (SingleDimensionShardSpec) segmentIdWithShardSpec.getShardSpec(); + RangeBucketShardSpec shardSpec = (RangeBucketShardSpec) segmentIdWithShardSpec.getShardSpec(); Assert.assertEquals(PARTITION_DIMENSION, shardSpec.getDimension()); - Assert.assertEquals(partitionNum, shardSpec.getPartitionNum()); + Assert.assertEquals(bucketId, shardSpec.getBucketId()); Assert.assertEquals(partitionStart, shardSpec.getStart()); Assert.assertEquals(partitionEnd, shardSpec.getEnd()); } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionCachingLocalSegmentAllocatorTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionCachingLocalSegmentAllocatorTest.java index 0e3fe2e7843c..6912cfca4c86 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionCachingLocalSegmentAllocatorTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionCachingLocalSegmentAllocatorTest.java @@ -42,6 +42,7 @@ import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; import org.apache.druid.timeline.SegmentId; import org.apache.druid.timeline.partition.BuildingHashBasedNumberedShardSpec; +import org.apache.druid.timeline.partition.HashBucketShardSpec; import org.easymock.EasyMock; import org.joda.time.Interval; import org.junit.Assert; @@ -103,10 +104,10 @@ public void allocatesCorrectShardSpec() throws IOException SegmentId.of(DATASOURCE, INTERVAL, VERSION, PARTITION_NUM), segmentIdWithShardSpec.asSegmentId() ); - BuildingHashBasedNumberedShardSpec shardSpec = (BuildingHashBasedNumberedShardSpec) segmentIdWithShardSpec.getShardSpec(); + HashBucketShardSpec shardSpec = (HashBucketShardSpec) segmentIdWithShardSpec.getShardSpec(); Assert.assertEquals(PARTITION_DIMENSIONS, shardSpec.getPartitionDimensions()); Assert.assertEquals(NUM_PARTITONS, shardSpec.getNumBuckets()); - Assert.assertEquals(PARTITION_NUM, shardSpec.getPartitionNum()); + Assert.assertEquals(PARTITION_NUM, shardSpec.getBucketId()); } @Test From 6e0ac6ef7f396c42501dbb3b6786ae45486e0d45 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Sat, 13 Jun 2020 17:02:56 -0700 Subject: [PATCH 22/24] add more tests --- .../appenderator/SegmentPublisherHelper.java | 3 + .../SegmentPublisherHelperTest.java | 173 ++++++++++++++++++ 2 files changed, 176 insertions(+) create mode 100644 server/src/test/java/org/apache/druid/segment/realtime/appenderator/SegmentPublisherHelperTest.java diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/SegmentPublisherHelper.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/SegmentPublisherHelper.java index 8c311e8f7e9f..ec4a65af3a4d 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/SegmentPublisherHelper.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/SegmentPublisherHelper.java @@ -21,6 +21,7 @@ import org.apache.druid.java.util.common.ISE; import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.partition.BucketNumberedShardSpec; import org.apache.druid.timeline.partition.BuildingShardSpec; import org.apache.druid.timeline.partition.OverwriteShardSpec; import org.apache.druid.timeline.partition.ShardSpec; @@ -72,6 +73,8 @@ static Set annotateShardSpec(Set segments) annotateFn = annotateAtomicUpdateGroupFn(segmentsPerInterval.size()); } else if (firstShardSpec instanceof BuildingShardSpec) { annotateFn = annotateCorePartitionSetSizeFn(segmentsPerInterval.size()); + } else if (firstShardSpec instanceof BucketNumberedShardSpec) { + throw new ISE("Cannot publish segments with shardSpec[%s]", firstShardSpec); } else { annotateFn = null; } diff --git a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/SegmentPublisherHelperTest.java b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/SegmentPublisherHelperTest.java new file mode 100644 index 000000000000..1f2af7298809 --- /dev/null +++ b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/SegmentPublisherHelperTest.java @@ -0,0 +1,173 @@ +/* + * 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; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.partition.BuildingHashBasedNumberedShardSpec; +import org.apache.druid.timeline.partition.BuildingNumberedShardSpec; +import org.apache.druid.timeline.partition.BuildingSingleDimensionShardSpec; +import org.apache.druid.timeline.partition.HashBasedNumberedShardSpec; +import org.apache.druid.timeline.partition.HashBucketShardSpec; +import org.apache.druid.timeline.partition.NumberedOverwriteShardSpec; +import org.apache.druid.timeline.partition.NumberedShardSpec; +import org.apache.druid.timeline.partition.PartitionIds; +import org.apache.druid.timeline.partition.ShardSpec; +import org.apache.druid.timeline.partition.SingleDimensionShardSpec; +import org.junit.Assert; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; + +import java.util.Set; + +public class SegmentPublisherHelperTest +{ + @Rule + public ExpectedException expectedException = ExpectedException.none(); + + @Test + public void testAnnotateAtomicUpdateGroupSize() + { + final Set segments = ImmutableSet.of( + newSegment( + new NumberedOverwriteShardSpec( + PartitionIds.NON_ROOT_GEN_START_PARTITION_ID, + 0, + 3, + (short) 1 + ) + ), + newSegment( + new NumberedOverwriteShardSpec( + PartitionIds.NON_ROOT_GEN_START_PARTITION_ID + 1, + 0, + 3, + (short) 1 + ) + ), + newSegment( + new NumberedOverwriteShardSpec( + PartitionIds.NON_ROOT_GEN_START_PARTITION_ID + 2, + 0, + 3, + (short) 1 + ) + ) + ); + final Set annotated = SegmentPublisherHelper.annotateShardSpec(segments); + for (DataSegment segment : annotated) { + Assert.assertSame(NumberedOverwriteShardSpec.class, segment.getShardSpec().getClass()); + final NumberedOverwriteShardSpec shardSpec = (NumberedOverwriteShardSpec) segment.getShardSpec(); + Assert.assertEquals(3, shardSpec.getAtomicUpdateGroupSize()); + } + } + + @Test + public void testAnnotateCorePartitionSetSizeForNumberedShardSpec() + { + final Set segments = ImmutableSet.of( + newSegment(new BuildingNumberedShardSpec(0)), + newSegment(new BuildingNumberedShardSpec(1)), + newSegment(new BuildingNumberedShardSpec(2)) + ); + final Set annotated = SegmentPublisherHelper.annotateShardSpec(segments); + for (DataSegment segment : annotated) { + Assert.assertSame(NumberedShardSpec.class, segment.getShardSpec().getClass()); + final NumberedShardSpec shardSpec = (NumberedShardSpec) segment.getShardSpec(); + Assert.assertEquals(3, shardSpec.getNumCorePartitions()); + } + } + + @Test + public void testAnnotateCorePartitionSetSizeForHashNumberedShardSpec() + { + final Set segments = ImmutableSet.of( + newSegment(new BuildingHashBasedNumberedShardSpec(0, 0, 3, null, new ObjectMapper())), + newSegment(new BuildingHashBasedNumberedShardSpec(1, 1, 3, null, new ObjectMapper())), + newSegment(new BuildingHashBasedNumberedShardSpec(2, 2, 3, null, new ObjectMapper())) + ); + final Set annotated = SegmentPublisherHelper.annotateShardSpec(segments); + for (DataSegment segment : annotated) { + Assert.assertSame(HashBasedNumberedShardSpec.class, segment.getShardSpec().getClass()); + final HashBasedNumberedShardSpec shardSpec = (HashBasedNumberedShardSpec) segment.getShardSpec(); + Assert.assertEquals(3, shardSpec.getNumCorePartitions()); + } + } + + @Test + public void testAnnotateCorePartitionSetSizeForSingleDimensionShardSpec() + { + final Set segments = ImmutableSet.of( + newSegment(new BuildingSingleDimensionShardSpec(0, "dim", null, "ccc", 0)), + newSegment(new BuildingSingleDimensionShardSpec(1, "dim", null, "ccc", 1)), + newSegment(new BuildingSingleDimensionShardSpec(2, "dim", null, "ccc", 2)) + ); + final Set annotated = SegmentPublisherHelper.annotateShardSpec(segments); + for (DataSegment segment : annotated) { + Assert.assertSame(SingleDimensionShardSpec.class, segment.getShardSpec().getClass()); + final SingleDimensionShardSpec shardSpec = (SingleDimensionShardSpec) segment.getShardSpec(); + Assert.assertEquals(3, shardSpec.getNumCorePartitions()); + } + } + + @Test + public void testAnnotateShardSpecDoNothing() + { + final Set segments = ImmutableSet.of( + newSegment(new NumberedShardSpec(0, 0)), + newSegment(new NumberedShardSpec(1, 0)), + newSegment(new NumberedShardSpec(2, 0)) + ); + final Set annotated = SegmentPublisherHelper.annotateShardSpec(segments); + Assert.assertEquals(segments, annotated); + } + + @Test + public void testAnnotateShardSpecThrowingExceptionForBucketNumberedShardSpec() + { + final Set segments = ImmutableSet.of( + newSegment(new HashBucketShardSpec(0, 3, null, new ObjectMapper())), + newSegment(new HashBucketShardSpec(1, 3, null, new ObjectMapper())), + newSegment(new HashBucketShardSpec(2, 3, null, new ObjectMapper())) + ); + expectedException.expect(IllegalStateException.class); + expectedException.expectMessage("Cannot publish segments with shardSpec"); + SegmentPublisherHelper.annotateShardSpec(segments); + } + + private static DataSegment newSegment(ShardSpec shardSpec) + { + return new DataSegment( + "datasource", + Intervals.of("2020-01-01/P1d"), + "version", + null, + ImmutableList.of("dim"), + ImmutableList.of("met"), + shardSpec, + 9, + 10L + ); + } +} From 4a27771ffe6a0c6cf784d2516ece13a91d6f671c Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Sat, 13 Jun 2020 17:05:06 -0700 Subject: [PATCH 23/24] doc --- docs/ingestion/native-batch.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ingestion/native-batch.md b/docs/ingestion/native-batch.md index 981f9f700938..799b32e0fbad 100644 --- a/docs/ingestion/native-batch.md +++ b/docs/ingestion/native-batch.md @@ -302,7 +302,7 @@ and then by the hash value of `partitionDimensions` (secondary partition key) in The partitioned data is stored in local storage of the [middleManager](../design/middlemanager.md) or the [indexer](../design/indexer.md). - The `partial segment merge` phase is similar to the Reduce phase in MapReduce. -The Parallel task spawns a new set of worker tasks (type `partial_index_merge`) to merge the partitioned data +The Parallel task spawns a new set of worker tasks (type `partial_index_generic_merge`) to merge the partitioned data created in the previous phase. Here, the partitioned data is shuffled based on the time chunk and the hash value of `partitionDimensions` to be merged; each worker task reads the data falling in the same time chunk and the same hash value from multiple MiddleManager/Indexer processes and merges From 7eaed9bdd649bfc571e5159dff4e758ba6935168 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Sat, 13 Jun 2020 18:14:10 -0700 Subject: [PATCH 24/24] unused imports --- .../task/RangePartitionCachingLocalSegmentAllocatorTest.java | 1 - .../parallel/HashPartitionCachingLocalSegmentAllocatorTest.java | 1 - 2 files changed, 2 deletions(-) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/RangePartitionCachingLocalSegmentAllocatorTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/RangePartitionCachingLocalSegmentAllocatorTest.java index affb04319520..e841fba0f370 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/RangePartitionCachingLocalSegmentAllocatorTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/RangePartitionCachingLocalSegmentAllocatorTest.java @@ -38,7 +38,6 @@ import org.apache.druid.timeline.SegmentId; import org.apache.druid.timeline.partition.PartitionBoundaries; import org.apache.druid.timeline.partition.RangeBucketShardSpec; -import org.apache.druid.timeline.partition.SingleDimensionShardSpec; import org.easymock.EasyMock; import org.joda.time.Interval; import org.junit.Assert; diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionCachingLocalSegmentAllocatorTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionCachingLocalSegmentAllocatorTest.java index 6912cfca4c86..ef8f09562311 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionCachingLocalSegmentAllocatorTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionCachingLocalSegmentAllocatorTest.java @@ -41,7 +41,6 @@ import org.apache.druid.segment.realtime.appenderator.SegmentAllocator; import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; import org.apache.druid.timeline.SegmentId; -import org.apache.druid.timeline.partition.BuildingHashBasedNumberedShardSpec; import org.apache.druid.timeline.partition.HashBucketShardSpec; import org.easymock.EasyMock; import org.joda.time.Interval;