From edafd1617b27eff2eef812e46976930d75fd09e8 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Sat, 6 Jun 2020 15:03:50 -0700 Subject: [PATCH 1/6] 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 2/6] 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 3/6] 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 4/6] 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 5/6] 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 6/6] 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;