From 656667ee8973147277e46368d6132fd8acbf3a08 Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Sun, 14 Jul 2024 21:43:06 -0700 Subject: [PATCH 1/7] Tests: Add utility class TuningConfigBuilder to make IndexTask tests more readable and concise (#16732) Changes: - No functional change - Add class `TuningConfigBuilder` to build `IndexTuningConfig`, `CompactionTuningConfig` - Remove old class `ParallelIndexTestingFactory.TuningConfigBuilder` - Remove some unused fields and methods --- .../msq/indexing/MSQCompactionRunnerTest.java | 45 +- .../druid/indexing/common/TestIndexTask.java | 36 +- .../ClientCompactionTaskQuerySerdeTest.java | 55 +- .../common/task/CompactionTaskRunTest.java | 39 +- .../common/task/CompactionTaskTest.java | 534 ++++++------------ .../task/CompactionTuningConfigTest.java | 152 +---- .../common/task/IndexTaskSerdeTest.java | 188 +----- .../indexing/common/task/IndexTaskTest.java | 149 ++--- .../indexing/common/task/TaskSerdeTest.java | 69 +-- .../common/task/TuningConfigBuilder.java | 415 ++++++++++++++ ...stractParallelIndexSupervisorTaskTest.java | 86 +-- .../ParallelIndexSupervisorTaskKillTest.java | 36 +- ...rallelIndexSupervisorTaskResourceTest.java | 36 +- .../ParallelIndexSupervisorTaskSerdeTest.java | 60 +- .../ParallelIndexSupervisorTaskTest.java | 195 +++---- .../parallel/ParallelIndexTestingFactory.java | 147 ----- .../ParallelIndexTuningConfigTest.java | 351 ++---------- .../PartialDimensionCardinalityTaskTest.java | 71 ++- .../PartialDimensionDistributionTaskTest.java | 92 +-- .../PartialGenericSegmentMergeTaskTest.java | 21 +- .../PartialHashSegmentGenerateTaskTest.java | 11 +- .../PartialRangeSegmentGenerateTaskTest.java | 32 +- .../PartialSegmentMergeIOConfigTest.java | 9 - .../PartialSegmentMergeIngestionSpecTest.java | 22 +- .../SinglePhaseParallelIndexingTest.java | 123 +--- .../indexing/overlord/TaskLifecycleTest.java | 136 +---- 26 files changed, 1082 insertions(+), 2028 deletions(-) create mode 100644 indexing-service/src/test/java/org/apache/druid/indexing/common/task/TuningConfigBuilder.java diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQCompactionRunnerTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQCompactionRunnerTest.java index 6f1a4396ada9..b95243f7783f 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQCompactionRunnerTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQCompactionRunnerTest.java @@ -39,6 +39,7 @@ import org.apache.druid.indexer.partitions.PartitionsSpec; import org.apache.druid.indexing.common.task.CompactionIntervalSpec; import org.apache.druid.indexing.common.task.CompactionTask; +import org.apache.druid.indexing.common.task.TuningConfigBuilder; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.granularity.Granularities; @@ -397,39 +398,17 @@ private static CompactionTask.CompactionTuningConfig createTuningConfig( PartitionsSpec partitionsSpec ) { - return new CompactionTask.CompactionTuningConfig( - null, - null, // null to compute maxRowsPerSegment automatically - null, - 500000, - 1000000L, - null, - null, - null, - null, - partitionsSpec, - indexSpec, - null, - null, - !(partitionsSpec instanceof DynamicPartitionsSpec), - false, - 5000L, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null - ); + return TuningConfigBuilder + .forCompactionTask() + .withMaxRowsInMemory(500000) + .withMaxBytesInMemory(1000000L) + .withMaxTotalRows(Long.MAX_VALUE) + .withPartitionsSpec(partitionsSpec) + .withIndexSpec(indexSpec) + .withForceGuaranteedRollup(!(partitionsSpec instanceof DynamicPartitionsSpec)) + .withReportParseExceptions(false) + .withPushTimeout(5000L) + .build(); } private static IndexSpec createIndexSpec() diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/TestIndexTask.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/TestIndexTask.java index b4166b7bc292..c69ac562821b 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/TestIndexTask.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/TestIndexTask.java @@ -31,6 +31,7 @@ import org.apache.druid.indexing.common.actions.TaskAction; import org.apache.druid.indexing.common.task.IndexTask; import org.apache.druid.indexing.common.task.TaskResource; +import org.apache.druid.indexing.common.task.TuningConfigBuilder; import org.apache.druid.indexing.overlord.SegmentPublishResult; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.segment.IndexSpec; @@ -69,34 +70,13 @@ public TestIndexTask( false, false ), - - new IndexTask.IndexTuningConfig( - null, - null, - null, - 10, - null, - null, - null, - null, - null, - null, - new DynamicPartitionsSpec(10000, null), - IndexSpec.DEFAULT, - null, - 3, - false, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null - ) + TuningConfigBuilder.forIndexTask() + .withMaxRowsInMemory(10) + .withIndexSpec(IndexSpec.DEFAULT) + .withPartitionsSpec(new DynamicPartitionsSpec(10000, null)) + .withForceGuaranteedRollup(false) + .withMaxPendingPersists(3) + .build() ), null ); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ClientCompactionTaskQuerySerdeTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ClientCompactionTaskQuerySerdeTest.java index fcded3ab9eeb..cd942a9a7668 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ClientCompactionTaskQuerySerdeTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ClientCompactionTaskQuerySerdeTest.java @@ -342,40 +342,27 @@ private CompactionTask createCompactionTask(ClientCompactionTaskTransformSpec tr ) .inputSpec(new CompactionIntervalSpec(Intervals.of("2019/2020"), "testSha256OfSortedSegmentIds"), true) .tuningConfig( - new ParallelIndexTuningConfig( - null, - null, - new OnheapIncrementalIndex.Spec(true), - 40000, - 2000L, - null, - null, - null, - SEGMENTS_SPLIT_HINT_SPEC, - DYNAMIC_PARTITIONS_SPEC, - INDEX_SPEC, - INDEX_SPEC_FOR_INTERMEDIATE_PERSISTS, - 2, - null, - null, - 1000L, - TmpFileSegmentWriteOutMediumFactory.instance(), - null, - 100, - 5, - 1000L, - new Duration(3000L), - 7, - 1000, - 100, - null, - null, - null, - 2, - null, - null, - null - ) + TuningConfigBuilder + .forParallelIndexTask() + .withAppendableIndexSpec(new OnheapIncrementalIndex.Spec(true)) + .withMaxRowsInMemory(40000) + .withMaxBytesInMemory(2000L) + .withSplitHintSpec(SEGMENTS_SPLIT_HINT_SPEC) + .withPartitionsSpec(DYNAMIC_PARTITIONS_SPEC) + .withIndexSpec(INDEX_SPEC) + .withIndexSpecForIntermediatePersists(INDEX_SPEC_FOR_INTERMEDIATE_PERSISTS) + .withMaxPendingPersists(2) + .withPushTimeout(1000L) + .withSegmentWriteOutMediumFactory(TmpFileSegmentWriteOutMediumFactory.instance()) + .withMaxNumConcurrentSubTasks(100) + .withMaxRetry(5) + .withTaskStatusCheckPeriodMs(1000L) + .withChatHandlerTimeout(new Duration(3000L)) + .withChatHandlerNumRetries(7) + .withMaxNumSegmentsToMerge(1000) + .withTotalNumMergeTasks(100) + .withMaxColumnsToMerge(2) + .build() ) .granularitySpec(CLIENT_COMPACTION_TASK_GRANULARITY_SPEC) .dimensionsSpec( 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 54902d5f7c64..0c274a019668 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 @@ -51,7 +51,6 @@ import org.apache.druid.indexing.common.config.TaskConfig; import org.apache.druid.indexing.common.config.TaskConfigBuilder; import org.apache.druid.indexing.common.task.CompactionTask.Builder; -import org.apache.druid.indexing.common.task.batch.parallel.ParallelIndexTuningConfig; import org.apache.druid.indexing.overlord.Segments; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.ISE; @@ -354,40 +353,10 @@ public void testRunWithHashPartitioning() throws Exception final CompactionTask compactionTask = builder .interval(Intervals.of("2014-01-01/2014-01-02")) .tuningConfig( - new ParallelIndexTuningConfig( - null, - null, - null, - null, - null, - null, - null, - null, - null, - new HashedPartitionsSpec(null, 3, null), - null, - null, - null, - true, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null - ) + TuningConfigBuilder.forParallelIndexTask() + .withForceGuaranteedRollup(true) + .withPartitionsSpec(new HashedPartitionsSpec(null, 3, null)) + .build() ) .build(); 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 b138a25469f1..25dc84d98b79 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 @@ -324,44 +324,21 @@ private static List findDimensions(int startIndex, Interval segmentInter private static CompactionTask.CompactionTuningConfig createTuningConfig() { - return new CompactionTask.CompactionTuningConfig( - null, - null, // null to compute maxRowsPerSegment automatically - null, - 500000, - 1000000L, - null, - null, - null, - null, - null, - IndexSpec.builder() - .withBitmapSerdeFactory(RoaringBitmapSerdeFactory.getInstance()) - .withDimensionCompression(CompressionStrategy.LZ4) - .withMetricCompression(CompressionStrategy.LZF) - .withLongEncoding(LongEncodingStrategy.LONGS) - .build(), - null, - null, - true, - false, - 5000L, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null - ); + return TuningConfigBuilder.forCompactionTask() + .withMaxRowsInMemory(500_000) + .withMaxBytesInMemory(1_000_000L) + .withIndexSpec( + IndexSpec.builder() + .withBitmapSerdeFactory(RoaringBitmapSerdeFactory.getInstance()) + .withDimensionCompression(CompressionStrategy.LZ4) + .withMetricCompression(CompressionStrategy.LZF) + .withLongEncoding(LongEncodingStrategy.LONGS) + .build() + ) + .withForceGuaranteedRollup(true) + .withReportParseExceptions(false) + .withPushTimeout(5000L) + .build(); } @Rule @@ -596,38 +573,21 @@ public void testSerdeWithOldTuningConfigSuccessfullyDeserializeToNewOne() throws null, null, null, - new IndexTuningConfig( - null, - null, // null to compute maxRowsPerSegment automatically - null, - 500000, - 1000000L, - null, - null, - null, - null, - null, - null, - IndexSpec.builder() - .withBitmapSerdeFactory(RoaringBitmapSerdeFactory.getInstance()) - .withDimensionCompression(CompressionStrategy.LZ4) - .withMetricCompression(CompressionStrategy.LZF) - .withLongEncoding(LongEncodingStrategy.LONGS) - .build(), - null, - null, - true, - false, - 5000L, - null, - null, - null, - null, - null, - null, - null, - null - ), + TuningConfigBuilder + .forIndexTask() + .withMaxRowsInMemory(500000) + .withIndexSpec( + IndexSpec.builder() + .withBitmapSerdeFactory(RoaringBitmapSerdeFactory.getInstance()) + .withDimensionCompression(CompressionStrategy.LZ4) + .withMetricCompression(CompressionStrategy.LZF) + .withLongEncoding(LongEncodingStrategy.LONGS) + .build() + ) + .withForceGuaranteedRollup(true) + .withReportParseExceptions(false) + .withPublishTimeout(5000L) + .build(), null, toolbox.getJsonMapper(), AuthTestUtils.TEST_AUTHORIZER_MAPPER, @@ -676,77 +636,37 @@ public void testInputSourceResources() @Test public void testGetTuningConfigWithIndexTuningConfig() { - IndexTuningConfig indexTuningConfig = new IndexTuningConfig( - null, - null, // null to compute maxRowsPerSegment automatically - null, - 500000, - 1000000L, - null, - null, - null, - null, - null, - null, - IndexSpec.builder() - .withBitmapSerdeFactory(RoaringBitmapSerdeFactory.getInstance()) - .withDimensionCompression(CompressionStrategy.LZ4) - .withMetricCompression(CompressionStrategy.LZF) - .withLongEncoding(LongEncodingStrategy.LONGS) - .build(), - null, - null, - true, - false, - null, - null, - null, - null, - null, - null, - null, - null, - null - ); + IndexTuningConfig indexTuningConfig = TuningConfigBuilder + .forIndexTask() + .withMaxRowsInMemory(500000) + .withMaxBytesInMemory(1000000L) + .withIndexSpec( + IndexSpec.builder() + .withBitmapSerdeFactory(RoaringBitmapSerdeFactory.getInstance()) + .withDimensionCompression(CompressionStrategy.LZ4) + .withMetricCompression(CompressionStrategy.LZF) + .withLongEncoding(LongEncodingStrategy.LONGS) + .build() + ) + .withForceGuaranteedRollup(true) + .withReportParseExceptions(false) + .build(); - CompactionTask.CompactionTuningConfig compactionTuningConfig = new CompactionTask.CompactionTuningConfig( - null, - null, - null, - 500000, - 1000000L, - null, - null, - null, - null, - null, - IndexSpec.builder() - .withBitmapSerdeFactory(RoaringBitmapSerdeFactory.getInstance()) - .withDimensionCompression(CompressionStrategy.LZ4) - .withMetricCompression(CompressionStrategy.LZF) - .withLongEncoding(LongEncodingStrategy.LONGS) - .build(), - null, - null, - true, - false, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null - ); + CompactionTask.CompactionTuningConfig compactionTuningConfig = TuningConfigBuilder + .forCompactionTask() + .withMaxRowsInMemory(500000) + .withMaxBytesInMemory(1000000L) + .withIndexSpec( + IndexSpec.builder() + .withBitmapSerdeFactory(RoaringBitmapSerdeFactory.getInstance()) + .withDimensionCompression(CompressionStrategy.LZ4) + .withMetricCompression(CompressionStrategy.LZF) + .withLongEncoding(LongEncodingStrategy.LONGS) + .build() + ) + .withForceGuaranteedRollup(true) + .withReportParseExceptions(false) + .build(); Assert.assertEquals(compactionTuningConfig, CompactionTask.getTuningConfig(indexTuningConfig)); @@ -755,84 +675,39 @@ public void testGetTuningConfigWithIndexTuningConfig() @Test public void testGetTuningConfigWithParallelIndexTuningConfig() { - ParallelIndexTuningConfig parallelIndexTuningConfig = new ParallelIndexTuningConfig( - null, - null, // null to compute maxRowsPerSegment automatically - null, - 500000, - 1000000L, - null, - null, - null, - null, - null, - IndexSpec.builder() - .withBitmapSerdeFactory(RoaringBitmapSerdeFactory.getInstance()) - .withDimensionCompression(CompressionStrategy.LZ4) - .withMetricCompression(CompressionStrategy.LZF) - .withLongEncoding(LongEncodingStrategy.LONGS) - .build(), - null, - null, - true, - false, - 5000L, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null - ); + ParallelIndexTuningConfig parallelIndexTuningConfig = TuningConfigBuilder + .forParallelIndexTask() + .withMaxRowsInMemory(500000) + .withMaxBytesInMemory(1000000L) + .withIndexSpec( + IndexSpec.builder() + .withBitmapSerdeFactory(RoaringBitmapSerdeFactory.getInstance()) + .withDimensionCompression(CompressionStrategy.LZ4) + .withMetricCompression(CompressionStrategy.LZF) + .withLongEncoding(LongEncodingStrategy.LONGS) + .build() + ) + .withForceGuaranteedRollup(true) + .withReportParseExceptions(false) + .withPushTimeout(5000L) + .build(); - CompactionTask.CompactionTuningConfig compactionTuningConfig = new CompactionTask.CompactionTuningConfig( - null, - null, // null to compute maxRowsPerSegment automatically - null, - 500000, - 1000000L, - null, - null, - null, - null, - null, - IndexSpec.builder() - .withBitmapSerdeFactory(RoaringBitmapSerdeFactory.getInstance()) - .withDimensionCompression(CompressionStrategy.LZ4) - .withMetricCompression(CompressionStrategy.LZF) - .withLongEncoding(LongEncodingStrategy.LONGS) - .build(), - null, - null, - true, - false, - 5000L, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null - ); + CompactionTask.CompactionTuningConfig compactionTuningConfig = TuningConfigBuilder + .forCompactionTask() + .withMaxRowsInMemory(500000) + .withMaxBytesInMemory(1000000L) + .withIndexSpec( + IndexSpec.builder() + .withBitmapSerdeFactory(RoaringBitmapSerdeFactory.getInstance()) + .withDimensionCompression(CompressionStrategy.LZ4) + .withMetricCompression(CompressionStrategy.LZF) + .withLongEncoding(LongEncodingStrategy.LONGS) + .build() + ) + .withForceGuaranteedRollup(true) + .withReportParseExceptions(false) + .withPushTimeout(5000L) + .build(); Assert.assertEquals(compactionTuningConfig, CompactionTask.getTuningConfig(parallelIndexTuningConfig)); } @@ -909,44 +784,24 @@ public void testCreateIngestionSchema() throws IOException @Test public void testCreateIngestionSchemaWithTargetPartitionSize() throws IOException { - final CompactionTask.CompactionTuningConfig tuningConfig = new CompactionTask.CompactionTuningConfig( - 100000, - null, - null, - 500000, - 1000000L, - null, - null, - null, - null, - null, - IndexSpec.builder() - .withBitmapSerdeFactory(RoaringBitmapSerdeFactory.getInstance()) - .withDimensionCompression(CompressionStrategy.LZ4) - .withMetricCompression(CompressionStrategy.LZF) - .withLongEncoding(LongEncodingStrategy.LONGS) - .build(), - null, - null, - true, - false, - null, - null, - null, - 10, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null - ); + final CompactionTask.CompactionTuningConfig tuningConfig = TuningConfigBuilder + .forCompactionTask() + .withTargetPartitionSize(100000) + .withMaxRowsInMemory(500000) + .withMaxBytesInMemory(1000000L) + .withIndexSpec( + IndexSpec.builder() + .withBitmapSerdeFactory(RoaringBitmapSerdeFactory.getInstance()) + .withDimensionCompression(CompressionStrategy.LZ4) + .withMetricCompression(CompressionStrategy.LZF) + .withLongEncoding(LongEncodingStrategy.LONGS) + .build() + ) + .withForceGuaranteedRollup(true) + .withReportParseExceptions(false) + .withMaxNumConcurrentSubTasks(10) + .build(); + final Map dataSchemasForIntervals = CompactionTask.createDataSchemasForIntervals( toolbox, LockGranularity.TIME_CHUNK, @@ -991,44 +846,24 @@ public void testCreateIngestionSchemaWithTargetPartitionSize() throws IOExceptio @Test public void testCreateIngestionSchemaWithMaxTotalRows() throws IOException { - final CompactionTask.CompactionTuningConfig tuningConfig = new CompactionTask.CompactionTuningConfig( - null, - null, - null, - 500000, - 1000000L, - null, - 1000000L, - null, - null, - null, - IndexSpec.builder() - .withBitmapSerdeFactory(RoaringBitmapSerdeFactory.getInstance()) - .withDimensionCompression(CompressionStrategy.LZ4) - .withMetricCompression(CompressionStrategy.LZF) - .withLongEncoding(LongEncodingStrategy.LONGS) - .build(), - null, - null, - false, - false, - 5000L, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null - ); + final CompactionTask.CompactionTuningConfig tuningConfig = TuningConfigBuilder + .forCompactionTask() + .withMaxRowsInMemory(500000) + .withMaxBytesInMemory(1000000L) + .withMaxTotalRows(1000000L) + .withIndexSpec( + IndexSpec.builder() + .withBitmapSerdeFactory(RoaringBitmapSerdeFactory.getInstance()) + .withDimensionCompression(CompressionStrategy.LZ4) + .withMetricCompression(CompressionStrategy.LZF) + .withLongEncoding(LongEncodingStrategy.LONGS) + .build() + ) + .withForceGuaranteedRollup(false) + .withReportParseExceptions(false) + .withPushTimeout(5000L) + .build(); + final Map dataSchemasForIntervals = CompactionTask.createDataSchemasForIntervals( toolbox, LockGranularity.TIME_CHUNK, @@ -1073,44 +908,25 @@ public void testCreateIngestionSchemaWithMaxTotalRows() throws IOException @Test public void testCreateIngestionSchemaWithNumShards() throws IOException { - final CompactionTask.CompactionTuningConfig tuningConfig = new CompactionTask.CompactionTuningConfig( - null, - null, - null, - 500000, - 1000000L, - null, - null, - null, - null, - new HashedPartitionsSpec(null, 3, null), - IndexSpec.builder() - .withBitmapSerdeFactory(RoaringBitmapSerdeFactory.getInstance()) - .withDimensionCompression(CompressionStrategy.LZ4) - .withMetricCompression(CompressionStrategy.LZF) - .withLongEncoding(LongEncodingStrategy.LONGS) - .build(), - null, - null, - true, - false, - 5000L, - null, - null, - 10, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null - ); + final CompactionTask.CompactionTuningConfig tuningConfig = TuningConfigBuilder + .forCompactionTask() + .withMaxRowsInMemory(500000) + .withMaxBytesInMemory(1000000L) + .withPartitionsSpec(new HashedPartitionsSpec(null, 3, null)) + .withIndexSpec( + IndexSpec.builder() + .withBitmapSerdeFactory(RoaringBitmapSerdeFactory.getInstance()) + .withDimensionCompression(CompressionStrategy.LZ4) + .withMetricCompression(CompressionStrategy.LZF) + .withLongEncoding(LongEncodingStrategy.LONGS) + .build() + ) + .withForceGuaranteedRollup(true) + .withReportParseExceptions(false) + .withPushTimeout(5000L) + .withMaxNumConcurrentSubTasks(10) + .build(); + final Map dataSchemasForIntervals = CompactionTask.createDataSchemasForIntervals( toolbox, LockGranularity.TIME_CHUNK, @@ -1821,44 +1637,24 @@ private void assertIngestionSchema( expectedDimensionsSpecs, expectedMetricsSpec, expectedSegmentIntervals, - new CompactionTask.CompactionTuningConfig( - null, - null, - null, - 500000, - 1000000L, - null, - Long.MAX_VALUE, - null, - null, - new HashedPartitionsSpec(5000000, null, null), // automatically computed targetPartitionSize - IndexSpec.builder() - .withBitmapSerdeFactory(RoaringBitmapSerdeFactory.getInstance()) - .withDimensionCompression(CompressionStrategy.LZ4) - .withMetricCompression(CompressionStrategy.LZF) - .withLongEncoding(LongEncodingStrategy.LONGS) - .build(), - null, - null, - true, - false, - 5000L, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null - ), + TuningConfigBuilder + .forCompactionTask() + .withMaxRowsInMemory(500000) + .withMaxBytesInMemory(1000000L) + .withMaxTotalRows(Long.MAX_VALUE) + .withPartitionsSpec(new HashedPartitionsSpec(5000000, null, null)) + .withIndexSpec( + IndexSpec.builder() + .withBitmapSerdeFactory(RoaringBitmapSerdeFactory.getInstance()) + .withDimensionCompression(CompressionStrategy.LZ4) + .withMetricCompression(CompressionStrategy.LZF) + .withLongEncoding(LongEncodingStrategy.LONGS) + .build() + ) + .withForceGuaranteedRollup(true) + .withReportParseExceptions(false) + .withPushTimeout(5000L) + .build(), expectedSegmentGranularity, expectedQueryGranularity, expectedDropExisting diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTuningConfigTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTuningConfigTest.java index 6831ea4adb60..a342dab82300 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTuningConfigTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTuningConfigTest.java @@ -22,21 +22,14 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.jsontype.NamedType; import nl.jqno.equalsverifier.EqualsVerifier; -import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; import org.apache.druid.indexing.common.task.batch.parallel.ParallelIndexTuningConfig; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.segment.IndexSpec; -import org.apache.druid.segment.data.CompressionFactory.LongEncodingStrategy; import org.apache.druid.segment.data.CompressionStrategy; -import org.apache.druid.segment.data.RoaringBitmapSerdeFactory; import org.apache.druid.segment.indexing.TuningConfig; -import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; -import org.joda.time.Duration; import org.junit.Assert; import org.junit.Before; -import org.junit.Rule; import org.junit.Test; -import org.junit.rules.ExpectedException; import java.io.IOException; @@ -44,9 +37,6 @@ public class CompactionTuningConfigTest { private final ObjectMapper mapper = new DefaultObjectMapper(); - @Rule - public ExpectedException expectedException = ExpectedException.none(); - @Before public void setup() { @@ -56,142 +46,44 @@ public void setup() @Test public void testSerdeDefault() throws IOException { - final CompactionTask.CompactionTuningConfig tuningConfig = CompactionTask.CompactionTuningConfig.defaultConfig(); + final CompactionTask.CompactionTuningConfig tuningConfig = + CompactionTask.CompactionTuningConfig.defaultConfig(); final byte[] json = mapper.writeValueAsBytes(tuningConfig); - final ParallelIndexTuningConfig fromJson = (CompactionTask.CompactionTuningConfig) mapper.readValue(json, TuningConfig.class); + final ParallelIndexTuningConfig fromJson = + (CompactionTask.CompactionTuningConfig) mapper.readValue(json, TuningConfig.class); Assert.assertEquals(fromJson, tuningConfig); } @Test - public void testSerdeWithNonZeroAwaitSegmentAvailabilityTimeoutMillis() + public void testConfigWithNonZeroAwaitSegmentAvailabilityTimeoutThrowsException() { - expectedException.expect(IllegalArgumentException.class); - expectedException.expectMessage("awaitSegmentAvailabilityTimeoutMillis is not supported for Compcation Task"); - final CompactionTask.CompactionTuningConfig tuningConfig = new CompactionTask.CompactionTuningConfig( - null, - null, - null, - 10, - 1000L, - null, - null, - null, - null, - new DynamicPartitionsSpec(100, 100L), - IndexSpec.builder() - .withBitmapSerdeFactory(RoaringBitmapSerdeFactory.getInstance()) - .withDimensionCompression(CompressionStrategy.UNCOMPRESSED) - .withMetricCompression(CompressionStrategy.LZF) - .withLongEncoding(LongEncodingStrategy.LONGS) - .build(), - IndexSpec.DEFAULT, - 1, - false, - true, - 10000L, - OffHeapMemorySegmentWriteOutMediumFactory.instance(), - null, - 250, - 100, - 20L, - new Duration(3600), - 128, - null, - null, - false, - null, - null, - null, - 5L, - null + final Exception e = Assert.assertThrows( + IllegalArgumentException.class, + () -> TuningConfigBuilder.forCompactionTask() + .withAwaitSegmentAvailabilityTimeoutMillis(5L) + .build() + ); + Assert.assertEquals( + "awaitSegmentAvailabilityTimeoutMillis is not supported for Compcation Task", + e.getMessage() ); } @Test - public void testSerdeWithZeroAwaitSegmentAvailabilityTimeoutMillis() + public void testConfigWithZeroAwaitSegmentAvailabilityTimeoutMillis() { - final CompactionTask.CompactionTuningConfig tuningConfig = new CompactionTask.CompactionTuningConfig( - null, - null, - null, - 10, - 1000L, - null, - null, - null, - null, - new DynamicPartitionsSpec(100, 100L), - IndexSpec.builder() - .withBitmapSerdeFactory(RoaringBitmapSerdeFactory.getInstance()) - .withDimensionCompression(CompressionStrategy.UNCOMPRESSED) - .withMetricCompression(CompressionStrategy.LZF) - .withLongEncoding(LongEncodingStrategy.LONGS) - .build(), - IndexSpec.DEFAULT, - 1, - false, - true, - 10000L, - OffHeapMemorySegmentWriteOutMediumFactory.instance(), - null, - 250, - 100, - 20L, - new Duration(3600), - 128, - null, - null, - false, - null, - null, - null, - 0L, - null - ); + final CompactionTask.CompactionTuningConfig tuningConfig = TuningConfigBuilder + .forCompactionTask() + .withAwaitSegmentAvailabilityTimeoutMillis(0L) + .build(); Assert.assertEquals(0L, tuningConfig.getAwaitSegmentAvailabilityTimeoutMillis()); } @Test - public void testSerdeWithNullAwaitSegmentAvailabilityTimeoutMillis() + public void testDefaultAwaitSegmentAvailabilityTimeoutMillis() { - final CompactionTask.CompactionTuningConfig tuningConfig = new CompactionTask.CompactionTuningConfig( - null, - null, - null, - 10, - 1000L, - null, - null, - null, - null, - new DynamicPartitionsSpec(100, 100L), - IndexSpec.builder() - .withBitmapSerdeFactory(RoaringBitmapSerdeFactory.getInstance()) - .withDimensionCompression(CompressionStrategy.UNCOMPRESSED) - .withMetricCompression(CompressionStrategy.LZF) - .withLongEncoding(LongEncodingStrategy.LONGS) - .build(), - IndexSpec.DEFAULT, - 1, - false, - true, - 10000L, - OffHeapMemorySegmentWriteOutMediumFactory.instance(), - null, - 250, - 100, - 20L, - new Duration(3600), - 128, - null, - null, - false, - null, - null, - null, - null, - null - ); + final CompactionTask.CompactionTuningConfig tuningConfig = + TuningConfigBuilder.forCompactionTask().build(); Assert.assertEquals(0L, tuningConfig.getAwaitSegmentAvailabilityTimeoutMillis()); } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskSerdeTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskSerdeTest.java index 40d9cb4919ff..493f6b477565 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskSerdeTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskSerdeTest.java @@ -56,114 +56,38 @@ public static void setup() @Test public void testSerdeTuningConfigWithDynamicPartitionsSpec() throws IOException { - final IndexTuningConfig tuningConfig = new IndexTuningConfig( - null, - null, - null, - 100, - 2000L, - null, - null, - null, - null, - null, - new DynamicPartitionsSpec(1000, 2000L), - IndexSpec.builder() - .withBitmapSerdeFactory(RoaringBitmapSerdeFactory.getInstance()) - .withDimensionCompression(CompressionStrategy.LZ4) - .withMetricCompression(CompressionStrategy.LZF) - .withLongEncoding(LongEncodingStrategy.LONGS) - .build(), - null, - null, - false, - null, - null, - 100L, - OffHeapMemorySegmentWriteOutMediumFactory.instance(), - true, - 10, - 100, - 1234, - 0L, - null - ); + final IndexTuningConfig tuningConfig = TuningConfigBuilder + .forIndexTask() + .withPartitionsSpec(new DynamicPartitionsSpec(1000, 2000L)) + .withForceGuaranteedRollup(false) + .build(); assertSerdeTuningConfig(tuningConfig); } @Test public void testSerdeTuningConfigWithHashedPartitionsSpec() throws IOException { - final IndexTuningConfig tuningConfig = new IndexTuningConfig( - null, - null, - null, - 100, - 2000L, - null, - null, - null, - null, - null, - new HashedPartitionsSpec(null, 10, ImmutableList.of("dim1", "dim2")), - IndexSpec.builder() - .withBitmapSerdeFactory(RoaringBitmapSerdeFactory.getInstance()) - .withDimensionCompression(CompressionStrategy.LZ4) - .withMetricCompression(CompressionStrategy.LZF) - .withLongEncoding(LongEncodingStrategy.LONGS) - .build(), - null, - null, - true, - null, - null, - 100L, - OffHeapMemorySegmentWriteOutMediumFactory.instance(), - true, - 10, - 100, - null, - -1L, - null - ); + final IndexTuningConfig tuningConfig = TuningConfigBuilder + .forIndexTask() + .withPartitionsSpec(new HashedPartitionsSpec(null, 10, null)) + .withForceGuaranteedRollup(true) + .build(); assertSerdeTuningConfig(tuningConfig); } @Test public void testSerdeTuningConfigWithDeprecatedDynamicPartitionsSpec() throws IOException { - final IndexTuningConfig tuningConfig = new IndexTuningConfig( - null, - 1000, - null, - 100, - 2000L, - null, - 3000L, - null, - null, - null, - null, - IndexSpec.builder() - .withBitmapSerdeFactory(RoaringBitmapSerdeFactory.getInstance()) - .withDimensionCompression(CompressionStrategy.LZ4) - .withMetricCompression(CompressionStrategy.LZF) - .withLongEncoding(LongEncodingStrategy.LONGS) - .build(), - null, - null, - false, - null, - null, - 100L, - OffHeapMemorySegmentWriteOutMediumFactory.instance(), - true, - 10, - 100, - null, - 1L, - null - ); + final IndexTuningConfig tuningConfig = TuningConfigBuilder + .forIndexTask() + .withMaxRowsPerSegment(1000) + .withMaxRowsInMemory(100) + .withMaxBytesInMemory(2000L) + .withMaxTotalRows(3000L) + .withForceGuaranteedRollup(false) + .withPushTimeout(100L) + .withAwaitSegmentAvailabilityTimeoutMillis(1L) + .build(); assertSerdeTuningConfig(tuningConfig); } @@ -210,38 +134,10 @@ public void testForceGuaranteedRollupWithDynamicPartitionsSpec() { expectedException.expect(IllegalArgumentException.class); expectedException.expectMessage("DynamicPartitionsSpec cannot be used for perfect rollup"); - final IndexTuningConfig tuningConfig = new IndexTuningConfig( - null, - null, - null, - 100, - 2000L, - null, - null, - null, - null, - null, - new DynamicPartitionsSpec(1000, 2000L), - IndexSpec.builder() - .withBitmapSerdeFactory(RoaringBitmapSerdeFactory.getInstance()) - .withDimensionCompression(CompressionStrategy.LZ4) - .withMetricCompression(CompressionStrategy.LZF) - .withLongEncoding(LongEncodingStrategy.LONGS) - .build(), - null, - null, - true, - null, - null, - 100L, - OffHeapMemorySegmentWriteOutMediumFactory.instance(), - true, - 10, - 100, - null, - null, - null - ); + TuningConfigBuilder.forIndexTask() + .withForceGuaranteedRollup(true) + .withPartitionsSpec(new DynamicPartitionsSpec(1000, 2000L)) + .build(); } @Test @@ -249,38 +145,10 @@ public void testBestEffortRollupWithHashedPartitionsSpec() { expectedException.expect(IllegalArgumentException.class); expectedException.expectMessage("DynamicPartitionsSpec must be used for best-effort rollup"); - final IndexTuningConfig tuningConfig = new IndexTuningConfig( - null, - null, - null, - 100, - 2000L, - null, - null, - null, - null, - null, - new HashedPartitionsSpec(null, 10, ImmutableList.of("dim1", "dim2")), - IndexSpec.builder() - .withBitmapSerdeFactory(RoaringBitmapSerdeFactory.getInstance()) - .withDimensionCompression(CompressionStrategy.LZ4) - .withMetricCompression(CompressionStrategy.LZF) - .withLongEncoding(LongEncodingStrategy.LONGS) - .build(), - null, - null, - false, - null, - null, - 100L, - OffHeapMemorySegmentWriteOutMediumFactory.instance(), - true, - 10, - 100, - null, - null, - null - ); + TuningConfigBuilder.forIndexTask() + .withForceGuaranteedRollup(false) + .withPartitionsSpec(new HashedPartitionsSpec(null, 10, null)) + .build(); } private static void assertSerdeTuningConfig(IndexTuningConfig tuningConfig) throws IOException 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 1a5cf19a21a6..40be38445857 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 @@ -1499,33 +1499,16 @@ public void testMultipleParseExceptionsSuccess() throws Exception writer.write("this is not JSON\n"); // invalid JSON } - final IndexTuningConfig tuningConfig = new IndexTuningConfig( - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - new HashedPartitionsSpec(2, null, null), - INDEX_SPEC, - null, - null, - true, - false, - null, - null, - null, - true, - 7, - 7, - null, - null, - null - ); + final IndexTuningConfig tuningConfig = TuningConfigBuilder + .forIndexTask() + .withPartitionsSpec(new HashedPartitionsSpec(2, null, null)) + .withIndexSpec(INDEX_SPEC) + .withForceGuaranteedRollup(true) + .withReportParseExceptions(false) + .withLogParseExceptions(true) + .withMaxParseExceptions(7) + .withMaxSavedParseExceptions(7) + .build(); final TimestampSpec timestampSpec = new TimestampSpec("time", "auto", null); final DimensionsSpec dimensionsSpec = new DimensionsSpec( @@ -1668,33 +1651,16 @@ public void testMultipleParseExceptionsFailure() throws Exception } // Allow up to 3 parse exceptions, and save up to 2 parse exceptions - final IndexTuningConfig tuningConfig = new IndexTuningConfig( - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - new DynamicPartitionsSpec(2, null), - INDEX_SPEC, - null, - null, - false, - false, - null, - null, - null, - true, - 2, - 5, - null, - null, - null - ); + final IndexTuningConfig tuningConfig = TuningConfigBuilder + .forIndexTask() + .withPartitionsSpec(new DynamicPartitionsSpec(2, null)) + .withIndexSpec(INDEX_SPEC) + .withForceGuaranteedRollup(false) + .withReportParseExceptions(false) + .withLogParseExceptions(true) + .withMaxParseExceptions(2) + .withMaxSavedParseExceptions(5) + .build(); final TimestampSpec timestampSpec = new TimestampSpec("time", "auto", null); final DimensionsSpec dimensionsSpec = new DimensionsSpec( @@ -1804,33 +1770,16 @@ public void testMultipleParseExceptionsFailureAtDeterminePartitions() throws Exc } // Allow up to 3 parse exceptions, and save up to 2 parse exceptions - final IndexTuningConfig tuningConfig = new IndexTuningConfig( - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - new HashedPartitionsSpec(2, null, null), - INDEX_SPEC, - null, - null, - true, - false, - null, - null, - null, - true, - 2, - 5, - null, - null, - null - ); + final IndexTuningConfig tuningConfig = TuningConfigBuilder + .forIndexTask() + .withPartitionsSpec(new HashedPartitionsSpec(2, null, null)) + .withIndexSpec(INDEX_SPEC) + .withForceGuaranteedRollup(true) + .withReportParseExceptions(false) + .withLogParseExceptions(true) + .withMaxParseExceptions(2) + .withMaxSavedParseExceptions(5) + .build(); final TimestampSpec timestampSpec = new TimestampSpec("time", "auto", null); final DimensionsSpec dimensionsSpec = new DimensionsSpec( @@ -2611,33 +2560,17 @@ static IndexTuningConfig createTuningConfig( boolean reportParseException ) { - return new IndexTuningConfig( - null, - maxRowsPerSegment, - null, - maxRowsInMemory, - null, - null, - maxTotalRows, - null, - null, - null, - partitionsSpec, - INDEX_SPEC, - null, - null, - forceGuaranteedRollup, - reportParseException, - null, - null, - null, - null, - null, - 1, - null, - null, - null - ); + return TuningConfigBuilder + .forIndexTask() + .withMaxRowsPerSegment(maxRowsPerSegment) + .withMaxRowsInMemory(maxRowsInMemory) + .withMaxTotalRows(maxTotalRows) + .withPartitionsSpec(partitionsSpec) + .withIndexSpec(INDEX_SPEC) + .withForceGuaranteedRollup(forceGuaranteedRollup) + .withReportParseExceptions(reportParseException) + .withMaxSavedParseExceptions(1) + .build(); } @SuppressWarnings("unchecked") diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TaskSerdeTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TaskSerdeTest.java index b217f8d5bd83..9cf6c0102fbe 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TaskSerdeTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TaskSerdeTest.java @@ -243,33 +243,14 @@ public void testIndexTaskSerde() throws Exception null ), new IndexIOConfig(null, new LocalInputSource(new File("lol"), "rofl"), new NoopInputFormat(), true, false), - new IndexTuningConfig( - null, - null, - null, - 10, - null, - null, - null, - 9999, - null, - null, - new DynamicPartitionsSpec(10000, null), - indexSpec, - null, - 3, - false, - null, - null, - null, - null, - null, - null, - null, - null, - 1L, - null - ) + TuningConfigBuilder.forIndexTask() + .withMaxRowsInMemory(10) + .withPartitionsSpec(new DynamicPartitionsSpec(10000, null)) + .withIndexSpec(indexSpec) + .withMaxPendingPersists(3) + .withForceGuaranteedRollup(false) + .withAwaitSegmentAvailabilityTimeoutMillis(1L) + .build() ), null ); @@ -330,33 +311,13 @@ public void testIndexTaskwithResourceSerde() throws Exception null ), new IndexIOConfig(null, new LocalInputSource(new File("lol"), "rofl"), new NoopInputFormat(), true, false), - new IndexTuningConfig( - null, - null, - null, - 10, - null, - null, - null, - null, - null, - null, - new DynamicPartitionsSpec(10000, null), - indexSpec, - null, - 3, - false, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null - ) + TuningConfigBuilder.forIndexTask() + .withMaxRowsInMemory(10) + .withForceGuaranteedRollup(false) + .withPartitionsSpec(new DynamicPartitionsSpec(10000, null)) + .withIndexSpec(indexSpec) + .withMaxPendingPersists(3) + .build() ), null ); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TuningConfigBuilder.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TuningConfigBuilder.java new file mode 100644 index 000000000000..c46c0becc76a --- /dev/null +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TuningConfigBuilder.java @@ -0,0 +1,415 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.common.task; + +import org.apache.druid.data.input.SplitHintSpec; +import org.apache.druid.indexer.partitions.PartitionsSpec; +import org.apache.druid.indexing.common.task.batch.parallel.ParallelIndexTuningConfig; +import org.apache.druid.segment.IndexSpec; +import org.apache.druid.segment.incremental.AppendableIndexSpec; +import org.apache.druid.segment.writeout.SegmentWriteOutMediumFactory; +import org.joda.time.Duration; + +/** + * Builder utility for various task tuning configs. + * + * @param Type of config that is being built + * @see TuningConfigBuilder#forIndexTask() + * @see TuningConfigBuilder#forParallelIndexTask() + * @see TuningConfigBuilder#forCompactionTask() + */ +public abstract class TuningConfigBuilder +{ + protected Integer targetPartitionSize; + protected Integer maxRowsPerSegment; + protected AppendableIndexSpec appendableIndexSpec; + protected Integer maxRowsInMemory; + protected Long maxBytesInMemory; + protected Boolean skipBytesInMemoryOverheadCheck; + protected Long maxTotalRows; + protected Integer numShards; + protected SplitHintSpec splitHintSpec; + protected PartitionsSpec partitionsSpec; + protected IndexSpec indexSpec; + protected IndexSpec indexSpecForIntermediatePersists; + protected Integer maxPendingPersists; + protected Boolean forceGuaranteedRollup; + protected Boolean reportParseExceptions; + protected Long publishTimeout; + protected Long pushTimeout; + protected SegmentWriteOutMediumFactory segmentWriteOutMediumFactory; + protected Integer maxNumSubTasks; + protected Integer maxNumConcurrentSubTasks; + protected Integer maxRetry; + protected Long taskStatusCheckPeriodMs; + protected Duration chatHandlerTimeout; + protected Integer chatHandlerNumRetries; + protected Integer maxNumSegmentsToMerge; + protected Integer totalNumMergeTasks; + protected Boolean logParseExceptions; + protected Integer maxParseExceptions; + protected Integer maxSavedParseExceptions; + protected Integer maxColumnsToMerge; + protected Long awaitSegmentAvailabilityTimeoutMillis; + protected Integer maxAllowedLockCount; + protected Integer numPersistThreads; + + public TuningConfigBuilder withTargetPartitionSize(Integer targetPartitionSize) + { + this.targetPartitionSize = targetPartitionSize; + return this; + } + + public TuningConfigBuilder withMaxRowsPerSegment(Integer maxRowsPerSegment) + { + this.maxRowsPerSegment = maxRowsPerSegment; + return this; + } + + public TuningConfigBuilder withAppendableIndexSpec(AppendableIndexSpec appendableIndexSpec) + { + this.appendableIndexSpec = appendableIndexSpec; + return this; + } + + public TuningConfigBuilder withMaxRowsInMemory(Integer maxRowsInMemory) + { + this.maxRowsInMemory = maxRowsInMemory; + return this; + } + + public TuningConfigBuilder withMaxBytesInMemory(Long maxBytesInMemory) + { + this.maxBytesInMemory = maxBytesInMemory; + return this; + } + + public TuningConfigBuilder withSkipBytesInMemoryOverheadCheck(Boolean skipBytesInMemoryOverheadCheck) + { + this.skipBytesInMemoryOverheadCheck = skipBytesInMemoryOverheadCheck; + return this; + } + + public TuningConfigBuilder withMaxTotalRows(Long maxTotalRows) + { + this.maxTotalRows = maxTotalRows; + return this; + } + + public TuningConfigBuilder withNumShards(Integer numShards) + { + this.numShards = numShards; + return this; + } + + public TuningConfigBuilder withSplitHintSpec(SplitHintSpec splitHintSpec) + { + this.splitHintSpec = splitHintSpec; + return this; + } + + public TuningConfigBuilder withPartitionsSpec(PartitionsSpec partitionsSpec) + { + this.partitionsSpec = partitionsSpec; + return this; + } + + public TuningConfigBuilder withIndexSpec(IndexSpec indexSpec) + { + this.indexSpec = indexSpec; + return this; + } + + public TuningConfigBuilder withIndexSpecForIntermediatePersists(IndexSpec indexSpecForIntermediatePersists) + { + this.indexSpecForIntermediatePersists = indexSpecForIntermediatePersists; + return this; + } + + public TuningConfigBuilder withMaxPendingPersists(Integer maxPendingPersists) + { + this.maxPendingPersists = maxPendingPersists; + return this; + } + + public TuningConfigBuilder withForceGuaranteedRollup(Boolean forceGuaranteedRollup) + { + this.forceGuaranteedRollup = forceGuaranteedRollup; + return this; + } + + public TuningConfigBuilder withReportParseExceptions(Boolean reportParseExceptions) + { + this.reportParseExceptions = reportParseExceptions; + return this; + } + + public TuningConfigBuilder withPushTimeout(Long pushTimeout) + { + this.pushTimeout = pushTimeout; + return this; + } + + public TuningConfigBuilder withPublishTimeout(Long publishTimeout) + { + this.publishTimeout = publishTimeout; + return this; + } + + public TuningConfigBuilder withSegmentWriteOutMediumFactory(SegmentWriteOutMediumFactory segmentWriteOutMediumFactory) + { + this.segmentWriteOutMediumFactory = segmentWriteOutMediumFactory; + return this; + } + + public TuningConfigBuilder withMaxNumSubTasks(Integer maxNumSubTasks) + { + this.maxNumSubTasks = maxNumSubTasks; + return this; + } + + public TuningConfigBuilder withMaxNumConcurrentSubTasks(Integer maxNumConcurrentSubTasks) + { + this.maxNumConcurrentSubTasks = maxNumConcurrentSubTasks; + return this; + } + + public TuningConfigBuilder withMaxRetry(Integer maxRetry) + { + this.maxRetry = maxRetry; + return this; + } + + public TuningConfigBuilder withTaskStatusCheckPeriodMs(Long taskStatusCheckPeriodMs) + { + this.taskStatusCheckPeriodMs = taskStatusCheckPeriodMs; + return this; + } + + public TuningConfigBuilder withChatHandlerTimeout(Duration chatHandlerTimeout) + { + this.chatHandlerTimeout = chatHandlerTimeout; + return this; + } + + public TuningConfigBuilder withChatHandlerNumRetries(Integer chatHandlerNumRetries) + { + this.chatHandlerNumRetries = chatHandlerNumRetries; + return this; + } + + public TuningConfigBuilder withMaxNumSegmentsToMerge(Integer maxNumSegmentsToMerge) + { + this.maxNumSegmentsToMerge = maxNumSegmentsToMerge; + return this; + } + + public TuningConfigBuilder withTotalNumMergeTasks(Integer totalNumMergeTasks) + { + this.totalNumMergeTasks = totalNumMergeTasks; + return this; + } + + public TuningConfigBuilder withLogParseExceptions(Boolean logParseExceptions) + { + this.logParseExceptions = logParseExceptions; + return this; + } + + public TuningConfigBuilder withMaxParseExceptions(Integer maxParseExceptions) + { + this.maxParseExceptions = maxParseExceptions; + return this; + } + + public TuningConfigBuilder withMaxSavedParseExceptions(Integer maxSavedParseExceptions) + { + this.maxSavedParseExceptions = maxSavedParseExceptions; + return this; + } + + public TuningConfigBuilder withMaxColumnsToMerge(Integer maxColumnsToMerge) + { + this.maxColumnsToMerge = maxColumnsToMerge; + return this; + } + + public TuningConfigBuilder withAwaitSegmentAvailabilityTimeoutMillis(Long awaitSegmentAvailabilityTimeoutMillis) + { + this.awaitSegmentAvailabilityTimeoutMillis = awaitSegmentAvailabilityTimeoutMillis; + return this; + } + + public TuningConfigBuilder withNumPersistThreads(Integer numPersistThreads) + { + this.numPersistThreads = numPersistThreads; + return this; + } + + public TuningConfigBuilder withMaxAllowedLockCount(Integer maxAllowedLockCount) + { + this.maxAllowedLockCount = maxAllowedLockCount; + return this; + } + + public abstract C build(); + + /** + * Creates a new builder for {@link CompactionTask.CompactionTuningConfig}. + */ + public static Compact forCompactionTask() + { + return new Compact(); + } + + /** + * Creates a new builder for {@link ParallelIndexTuningConfig}. + */ + public static ParallelIndex forParallelIndexTask() + { + return new ParallelIndex(); + } + + /** + * Creates a new builder for {@link IndexTask.IndexTuningConfig}. + */ + public static Index forIndexTask() + { + return new Index(); + } + + public static class Index extends TuningConfigBuilder + { + @Override + public IndexTask.IndexTuningConfig build() + { + return new IndexTask.IndexTuningConfig( + targetPartitionSize, + maxRowsPerSegment, + appendableIndexSpec, + maxRowsInMemory, + maxBytesInMemory, + skipBytesInMemoryOverheadCheck, + maxTotalRows, + null, + numShards, + null, + partitionsSpec, + indexSpec, + indexSpecForIntermediatePersists, + maxPendingPersists, + forceGuaranteedRollup, + reportParseExceptions, + publishTimeout, + pushTimeout, + segmentWriteOutMediumFactory, + logParseExceptions, + maxParseExceptions, + maxSavedParseExceptions, + maxColumnsToMerge, + awaitSegmentAvailabilityTimeoutMillis, + numPersistThreads + ); + } + } + + public static class ParallelIndex extends TuningConfigBuilder + { + @Override + public ParallelIndexTuningConfig build() + { + return new ParallelIndexTuningConfig( + targetPartitionSize, + maxRowsPerSegment, + appendableIndexSpec, + maxRowsInMemory, + maxBytesInMemory, + skipBytesInMemoryOverheadCheck, + maxTotalRows, + numShards, + splitHintSpec, + partitionsSpec, + indexSpec, + indexSpecForIntermediatePersists, + maxPendingPersists, + forceGuaranteedRollup, + reportParseExceptions, + pushTimeout, + segmentWriteOutMediumFactory, + maxNumSubTasks, + maxNumConcurrentSubTasks, + maxRetry, + taskStatusCheckPeriodMs, + chatHandlerTimeout, + chatHandlerNumRetries, + maxNumSegmentsToMerge, + totalNumMergeTasks, + logParseExceptions, + maxParseExceptions, + maxSavedParseExceptions, + maxColumnsToMerge, + awaitSegmentAvailabilityTimeoutMillis, + maxAllowedLockCount, + numPersistThreads + ); + } + } + + public static class Compact extends TuningConfigBuilder + { + @Override + public CompactionTask.CompactionTuningConfig build() + { + return new CompactionTask.CompactionTuningConfig( + targetPartitionSize, + maxRowsPerSegment, + appendableIndexSpec, + maxRowsInMemory, + maxBytesInMemory, + skipBytesInMemoryOverheadCheck, + maxTotalRows, + numShards, + splitHintSpec, + partitionsSpec, + indexSpec, + indexSpecForIntermediatePersists, + maxPendingPersists, + forceGuaranteedRollup, + reportParseExceptions, + pushTimeout, + segmentWriteOutMediumFactory, + maxNumSubTasks, + maxNumConcurrentSubTasks, + maxRetry, + taskStatusCheckPeriodMs, + chatHandlerTimeout, + chatHandlerNumRetries, + maxNumSegmentsToMerge, + totalNumMergeTasks, + logParseExceptions, + maxParseExceptions, + maxSavedParseExceptions, + maxColumnsToMerge, + awaitSegmentAvailabilityTimeoutMillis, + numPersistThreads + ); + } + } +} diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractParallelIndexSupervisorTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractParallelIndexSupervisorTaskTest.java index 8f68846cd645..ea6f9d963fef 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractParallelIndexSupervisorTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractParallelIndexSupervisorTaskTest.java @@ -66,6 +66,7 @@ import org.apache.druid.indexing.common.task.Task; import org.apache.druid.indexing.common.task.Tasks; import org.apache.druid.indexing.common.task.TestAppenderatorsManager; +import org.apache.druid.indexing.common.task.TuningConfigBuilder; import org.apache.druid.indexing.overlord.Segments; import org.apache.druid.indexing.worker.config.WorkerConfig; import org.apache.druid.indexing.worker.shuffle.IntermediaryDataManager; @@ -165,40 +166,10 @@ public class AbstractParallelIndexSupervisorTaskTest extends IngestionTestBase 0 ); public static final ParallelIndexTuningConfig DEFAULT_TUNING_CONFIG_FOR_PARALLEL_INDEXING = - new ParallelIndexTuningConfig( - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - 2, - null, - null, - null, - null, - null, - null, - null, - null, - 5, - null, - null, - null, - null - ); + TuningConfigBuilder.forParallelIndexTask() + .withMaxNumConcurrentSubTasks(2) + .withMaxSavedParseExceptions(5) + .build(); protected static final double DEFAULT_TRANSIENT_TASK_FAILURE_RATE = 0.2; protected static final double DEFAULT_TRANSIENT_API_FAILURE_RATE = 0.2; @@ -276,40 +247,13 @@ protected ParallelIndexTuningConfig newTuningConfig( boolean forceGuaranteedRollup ) { - return new ParallelIndexTuningConfig( - null, - null, - null, - null, - null, - null, - null, - null, - new MaxSizeSplitHintSpec(null, 1), - partitionsSpec, - null, - null, - null, - forceGuaranteedRollup, - null, - null, - null, - null, - maxNumConcurrentSubTasks, - null, - null, - null, - null, - null, - null, - null, - null, - 5, - null, - null, - null, - null - ); + return TuningConfigBuilder.forParallelIndexTask() + .withSplitHintSpec(new MaxSizeSplitHintSpec(null, 1)) + .withPartitionsSpec(partitionsSpec) + .withForceGuaranteedRollup(forceGuaranteedRollup) + .withMaxNumConcurrentSubTasks(maxNumConcurrentSubTasks) + .withMaxParseExceptions(5) + .build(); } protected LocalOverlordClient getIndexingServiceClient() @@ -329,12 +273,10 @@ protected static class TaskContainer private volatile Future statusFuture; @MonotonicNonNull private volatile TestLocalTaskActionClient actionClient; - private final CountDownLatch taskFinishLatch; - private TaskContainer(Task task, CountDownLatch taskFinishLatch) + private TaskContainer(Task task) { this.task = task; - this.taskFinishLatch = taskFinishLatch; } public Task getTask() @@ -461,7 +403,7 @@ private Future runTask(Task task) final CountDownLatch taskFinishLatch = useTaskFinishLatches ? new CountDownLatch(1) : new CountDownLatch(0); allTaskLatches.add(taskFinishLatch); - final TaskContainer taskContainer = new TaskContainer(task, taskFinishLatch); + final TaskContainer taskContainer = new TaskContainer(task); if (tasks.put(task.getId(), taskContainer) != null) { throw new ISE("Duplicate task ID[%s]", task.getId()); } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskKillTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskKillTest.java index 14e1bab9540b..e85df4fe5dd4 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskKillTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskKillTest.java @@ -32,6 +32,7 @@ import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.common.actions.TaskActionClient; import org.apache.druid.indexing.common.task.TaskResource; +import org.apache.druid.indexing.common.task.TuningConfigBuilder; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.granularity.Granularities; @@ -163,40 +164,7 @@ private ParallelIndexSupervisorTask newTask( null ), ioConfig, - new ParallelIndexTuningConfig( - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - numTotalSubTasks, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null - ) + TuningConfigBuilder.forParallelIndexTask().withMaxNumConcurrentSubTasks(numTotalSubTasks).build() ); // set up test tools diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskResourceTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskResourceTest.java index 01d502de85cb..50939a99d8a1 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskResourceTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskResourceTest.java @@ -39,6 +39,7 @@ import org.apache.druid.indexing.common.task.AbstractTask; import org.apache.druid.indexing.common.task.SegmentAllocators; import org.apache.druid.indexing.common.task.TaskResource; +import org.apache.druid.indexing.common.task.TuningConfigBuilder; import org.apache.druid.indexing.common.task.batch.parallel.ParallelIndexTaskRunner.SubTaskSpecStatus; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.ISE; @@ -415,40 +416,7 @@ private TestSupervisorTask newTask( null ), ioConfig, - new ParallelIndexTuningConfig( - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - NUM_SUB_TASKS, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null - ) + TuningConfigBuilder.forParallelIndexTask().withMaxNumConcurrentSubTasks(NUM_SUB_TASKS).build() ); return new TestSupervisorTask( diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskSerdeTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskSerdeTest.java index 7b92ba44bd9d..cd588d0b6eea 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskSerdeTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskSerdeTest.java @@ -31,13 +31,13 @@ import org.apache.druid.indexing.common.TestUtils; import org.apache.druid.indexing.common.task.Task; import org.apache.druid.indexing.common.task.TaskResource; +import org.apache.druid.indexing.common.task.TuningConfigBuilder; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.LongSumAggregatorFactory; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; -import org.hamcrest.CoreMatchers; import org.joda.time.Interval; import org.junit.Assert; import org.junit.Rule; @@ -58,16 +58,9 @@ public class ParallelIndexSupervisorTaskSerdeTest NullHandling.initializeForTests(); } - private static final ObjectMapper OBJECT_MAPPER = createObjectMapper(); + private static final ObjectMapper OBJECT_MAPPER = new TestUtils().getTestObjectMapper(); private static final List INTERVALS = Collections.singletonList(Intervals.of("2018/2019")); - private static ObjectMapper createObjectMapper() - { - TestUtils testUtils = new TestUtils(); - ObjectMapper objectMapper = testUtils.getTestObjectMapper(); - return objectMapper; - } - @Rule public ExpectedException expectedException = ExpectedException.none(); @@ -89,19 +82,18 @@ public void serde() throws IOException @Test public void forceGuaranteedRollupWithHashPartitionsMissingNumShards() { - Integer numShards = null; ParallelIndexSupervisorTask task = new ParallelIndexSupervisorTaskBuilder() .ingestionSpec( new ParallelIndexIngestionSpecBuilder() .forceGuaranteedRollup(true) - .partitionsSpec(new HashedPartitionsSpec(null, numShards, null)) + .partitionsSpec(new HashedPartitionsSpec(null, null, null)) .inputIntervals(INTERVALS) .build() ) .build(); PartitionsSpec partitionsSpec = task.getIngestionSchema().getTuningConfig().getPartitionsSpec(); - Assert.assertThat(partitionsSpec, CoreMatchers.instanceOf(HashedPartitionsSpec.class)); + Assert.assertTrue(partitionsSpec instanceof HashedPartitionsSpec); } @Test @@ -119,7 +111,7 @@ public void forceGuaranteedRollupWithHashPartitionsValid() .build(); PartitionsSpec partitionsSpec = task.getIngestionSchema().getTuningConfig().getPartitionsSpec(); - Assert.assertThat(partitionsSpec, CoreMatchers.instanceOf(HashedPartitionsSpec.class)); + Assert.assertTrue(partitionsSpec instanceof HashedPartitionsSpec); } @Test @@ -153,7 +145,7 @@ public void forceGuaranteedRollupWithSingleDimPartitionsValid() .build(); PartitionsSpec partitionsSpec = task.getIngestionSchema().getTuningConfig().getPartitionsSpec(); - Assert.assertThat(partitionsSpec, CoreMatchers.instanceOf(SingleDimensionPartitionsSpec.class)); + Assert.assertTrue(partitionsSpec instanceof SingleDimensionPartitionsSpec); } private static class ParallelIndexSupervisorTaskBuilder @@ -240,41 +232,11 @@ ParallelIndexIngestionSpec build() null ); - ParallelIndexTuningConfig tuningConfig = new ParallelIndexTuningConfig( - null, - null, - null, - null, - null, - null, - null, - null, - null, - partitionsSpec, - null, - null, - null, - forceGuaranteedRollup, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null - ); - + ParallelIndexTuningConfig tuningConfig = TuningConfigBuilder + .forParallelIndexTask() + .withPartitionsSpec(partitionsSpec) + .withForceGuaranteedRollup(forceGuaranteedRollup) + .build(); return new ParallelIndexIngestionSpec(dataSchema, ioConfig, tuningConfig); } } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskTest.java index d763337978fb..699476dbc16d 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskTest.java @@ -37,6 +37,7 @@ import org.apache.druid.indexer.report.KillTaskReport; import org.apache.druid.indexer.report.TaskReport; import org.apache.druid.indexing.common.TaskToolbox; +import org.apache.druid.indexing.common.task.TuningConfigBuilder; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.Intervals; @@ -236,45 +237,32 @@ public void testFailToConstructWhenBothAppendToExistingAndForceGuaranteedRollupA appendToExisting, null ); - final ParallelIndexTuningConfig tuningConfig = new ParallelIndexTuningConfig( - null, - null, - null, - 10, - 1000L, - null, - null, - null, - null, - new HashedPartitionsSpec(null, 10, null), - IndexSpec.builder() - .withBitmapSerdeFactory(RoaringBitmapSerdeFactory.getInstance()) - .withDimensionCompression(CompressionStrategy.UNCOMPRESSED) - .withMetricCompression(CompressionStrategy.LZF) - .withLongEncoding(LongEncodingStrategy.LONGS) - .build(), - IndexSpec.DEFAULT, - 1, - forceGuaranteedRollup, - true, - 10000L, - OffHeapMemorySegmentWriteOutMediumFactory.instance(), - null, - 10, - 100, - 20L, - new Duration(3600), - 128, - null, - null, - false, - null, - null, - null, - null, - null, - null - ); + final ParallelIndexTuningConfig tuningConfig = TuningConfigBuilder + .forParallelIndexTask() + .withMaxRowsInMemory(10) + .withMaxBytesInMemory(1000L) + .withPartitionsSpec(new HashedPartitionsSpec(null, 10, null)) + .withIndexSpec( + IndexSpec.builder() + .withBitmapSerdeFactory(RoaringBitmapSerdeFactory.getInstance()) + .withDimensionCompression(CompressionStrategy.UNCOMPRESSED) + .withMetricCompression(CompressionStrategy.LZF) + .withLongEncoding(LongEncodingStrategy.LONGS) + .build() + ) + .withIndexSpecForIntermediatePersists(IndexSpec.DEFAULT) + .withMaxPendingPersists(1) + .withForceGuaranteedRollup(forceGuaranteedRollup) + .withReportParseExceptions(true) + .withPushTimeout(10000L) + .withSegmentWriteOutMediumFactory(OffHeapMemorySegmentWriteOutMediumFactory.instance()) + .withMaxNumConcurrentSubTasks(10) + .withMaxRetry(100) + .withTaskStatusCheckPeriodMs(20L) + .withChatHandlerTimeout(new Duration(3600)) + .withChatHandlerNumRetries(128) + .withLogParseExceptions(false) + .build(); final ParallelIndexIngestionSpec indexIngestionSpec = new ParallelIndexIngestionSpec( new DataSchema( "datasource", @@ -309,45 +297,32 @@ public void testFailToConstructWhenBothInputSourceAndParserAreSet() false, null ); - final ParallelIndexTuningConfig tuningConfig = new ParallelIndexTuningConfig( - null, - null, - null, - 10, - 1000L, - null, - null, - null, - null, - new HashedPartitionsSpec(null, 10, null), - IndexSpec.builder() - .withBitmapSerdeFactory(RoaringBitmapSerdeFactory.getInstance()) - .withDimensionCompression(CompressionStrategy.UNCOMPRESSED) - .withMetricCompression(CompressionStrategy.LZF) - .withLongEncoding(LongEncodingStrategy.LONGS) - .build(), - IndexSpec.DEFAULT, - 1, - true, - true, - 10000L, - OffHeapMemorySegmentWriteOutMediumFactory.instance(), - null, - 10, - 100, - 20L, - new Duration(3600), - 128, - null, - null, - false, - null, - null, - null, - null, - null, - null - ); + final ParallelIndexTuningConfig tuningConfig = TuningConfigBuilder + .forParallelIndexTask() + .withMaxRowsInMemory(10) + .withMaxBytesInMemory(1000L) + .withPartitionsSpec(new HashedPartitionsSpec(null, 10, null)) + .withIndexSpec( + IndexSpec.builder() + .withBitmapSerdeFactory(RoaringBitmapSerdeFactory.getInstance()) + .withDimensionCompression(CompressionStrategy.UNCOMPRESSED) + .withMetricCompression(CompressionStrategy.LZF) + .withLongEncoding(LongEncodingStrategy.LONGS) + .build() + ) + .withIndexSpecForIntermediatePersists(IndexSpec.DEFAULT) + .withMaxPendingPersists(1) + .withForceGuaranteedRollup(true) + .withReportParseExceptions(true) + .withPushTimeout(10000L) + .withSegmentWriteOutMediumFactory(OffHeapMemorySegmentWriteOutMediumFactory.instance()) + .withMaxNumConcurrentSubTasks(10) + .withMaxRetry(100) + .withTaskStatusCheckPeriodMs(20L) + .withChatHandlerTimeout(new Duration(3600)) + .withChatHandlerNumRetries(128) + .withLogParseExceptions(false) + .build(); expectedException.expect(IAE.class); expectedException.expectMessage("Cannot use parser and inputSource together. Try using inputFormat instead of parser."); @@ -559,45 +534,33 @@ public void testCompactionTaskDoesntCleanup() throws Exception appendToExisting, null ); - final ParallelIndexTuningConfig tuningConfig = new ParallelIndexTuningConfig( - null, - null, - null, - 10, - 1000L, - null, - null, - null, - null, - new HashedPartitionsSpec(null, 10, null), + final ParallelIndexTuningConfig tuningConfig = TuningConfigBuilder + .forParallelIndexTask() + .withMaxRowsInMemory(10) + .withMaxBytesInMemory(1000L) + .withPartitionsSpec(new HashedPartitionsSpec(null, 10, null)) + .withIndexSpec( IndexSpec.builder() - .withBitmapSerdeFactory(RoaringBitmapSerdeFactory.getInstance()) - .withDimensionCompression(CompressionStrategy.UNCOMPRESSED) - .withMetricCompression(CompressionStrategy.LZF) - .withLongEncoding(LongEncodingStrategy.LONGS) - .build(), - IndexSpec.DEFAULT, - 1, - forceGuaranteedRollup, - true, - 10000L, - OffHeapMemorySegmentWriteOutMediumFactory.instance(), - null, - 10, - 100, - 20L, - new Duration(3600), - 128, - null, - null, - false, - null, - null, - null, - null, - null, - null - ); + .withBitmapSerdeFactory(RoaringBitmapSerdeFactory.getInstance()) + .withDimensionCompression(CompressionStrategy.UNCOMPRESSED) + .withMetricCompression(CompressionStrategy.LZF) + .withLongEncoding(LongEncodingStrategy.LONGS) + .build() + ) + .withIndexSpecForIntermediatePersists(IndexSpec.DEFAULT) + .withMaxPendingPersists(1) + .withForceGuaranteedRollup(forceGuaranteedRollup) + .withReportParseExceptions(true) + .withPushTimeout(10000L) + .withSegmentWriteOutMediumFactory(OffHeapMemorySegmentWriteOutMediumFactory.instance()) + .withMaxNumConcurrentSubTasks(10) + .withMaxRetry(100) + .withTaskStatusCheckPeriodMs(20L) + .withChatHandlerTimeout(new Duration(3600)) + .withChatHandlerNumRetries(128) + .withLogParseExceptions(false) + .build(); + final ParallelIndexIngestionSpec indexIngestionSpec = new ParallelIndexIngestionSpec( new DataSchema( "datasource", diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexTestingFactory.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexTestingFactory.java index 0a0d4d4692f0..16976e827d23 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexTestingFactory.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexTestingFactory.java @@ -26,30 +26,20 @@ import org.apache.druid.data.input.InputFormat; import org.apache.druid.data.input.InputSource; import org.apache.druid.data.input.impl.DimensionsSpec; -import org.apache.druid.data.input.impl.JsonInputFormat; import org.apache.druid.data.input.impl.TimestampSpec; -import org.apache.druid.indexer.partitions.HashedPartitionsSpec; -import org.apache.druid.indexer.partitions.PartitionsSpec; -import org.apache.druid.indexer.partitions.SingleDimensionPartitionsSpec; import org.apache.druid.indexing.common.TestUtils; import org.apache.druid.indexing.common.task.TaskResource; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.query.aggregation.AggregatorFactory; -import org.apache.druid.segment.IndexIO; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.granularity.ArbitraryGranularitySpec; import org.apache.druid.segment.indexing.granularity.GranularitySpec; -import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager; import org.apache.druid.segment.transform.TransformSpec; import org.apache.druid.timeline.partition.BuildingHashBasedNumberedShardSpec; import org.apache.druid.timeline.partition.HashPartitionFunction; -import org.easymock.EasyMock; -import org.joda.time.Duration; import org.joda.time.Interval; -import javax.annotation.Nullable; -import java.io.File; import java.util.Collections; import java.util.HashMap; import java.util.List; @@ -68,20 +58,6 @@ class ParallelIndexTestingFactory static final String SUBTASK_SPEC_ID = "subtask-spec-id"; static final int NUM_ATTEMPTS = 1; static final Map CONTEXT = Collections.emptyMap(); - static final ParallelIndexSupervisorTaskClientProvider TASK_CLIENT_PROVIDER = TestUtils.TASK_CLIENT_PROVIDER; - static final AppenderatorsManager APPENDERATORS_MANAGER = TestUtils.APPENDERATORS_MANAGER; - static final ShuffleClient SHUFFLE_CLIENT = new ShuffleClient() - { - @Override - public File fetchSegmentFile( - File partitionDir, - String supervisorTaskId, - PartitionLocation location - ) - { - return null; - } - }; static final List INPUT_INTERVALS = Collections.singletonList(Intervals.ETERNITY); static final String TASK_EXECUTOR_HOST = "task-executor-host"; static final int TASK_EXECUTOR_PORT = 1; @@ -113,83 +89,6 @@ static ObjectMapper createObjectMapper() return TEST_UTILS.getTestObjectMapper(); } - static IndexIO getIndexIO() - { - return TEST_UTILS.getTestIndexIO(); - } - - @SuppressWarnings("SameParameterValue") - static class TuningConfigBuilder - { - private PartitionsSpec partitionsSpec = - new HashedPartitionsSpec(null, 2, null); - private boolean forceGuaranteedRollup = true; - private boolean logParseExceptions = false; - private int maxParseExceptions = Integer.MAX_VALUE; - - TuningConfigBuilder partitionsSpec(PartitionsSpec partitionsSpec) - { - this.partitionsSpec = partitionsSpec; - return this; - } - - TuningConfigBuilder forceGuaranteedRollup(boolean forceGuaranteedRollup) - { - this.forceGuaranteedRollup = forceGuaranteedRollup; - return this; - } - - TuningConfigBuilder logParseExceptions(boolean logParseExceptions) - { - this.logParseExceptions = logParseExceptions; - return this; - } - - TuningConfigBuilder maxParseExceptions(int maxParseExceptions) - { - this.maxParseExceptions = maxParseExceptions; - return this; - } - - ParallelIndexTuningConfig build() - { - return new ParallelIndexTuningConfig( - 1, - null, - null, - 3, - 4L, - null, - 5L, - 6, - null, - partitionsSpec, - null, - null, - 10, - forceGuaranteedRollup, - false, - 14L, - null, - null, - 16, - 17, - 18L, - Duration.ZERO, - 20, - 21, - 22, - logParseExceptions, - maxParseExceptions, - 25, - null, - null, - null, - 2 - ); - } - } - static DataSchema createDataSchema(List granularitySpecInputIntervals) { GranularitySpec granularitySpec = new ArbitraryGranularitySpec(Granularities.DAY, granularitySpecInputIntervals); @@ -222,47 +121,6 @@ static ParallelIndexIngestionSpec createIngestionSpec( return new ParallelIndexIngestionSpec(dataSchema, ioConfig, tuningConfig); } - static class SingleDimensionPartitionsSpecBuilder - { - @Nullable - private String partitionDimension = SCHEMA_DIMENSION; - private boolean assumeGrouped = false; - - SingleDimensionPartitionsSpecBuilder partitionDimension(@Nullable String partitionDimension) - { - this.partitionDimension = partitionDimension; - return this; - } - - SingleDimensionPartitionsSpecBuilder assumeGrouped(boolean assumeGrouped) - { - this.assumeGrouped = assumeGrouped; - return this; - } - - SingleDimensionPartitionsSpec build() - { - return new SingleDimensionPartitionsSpec( - 1, - null, - partitionDimension, - assumeGrouped - ); - } - } - - static ParallelIndexSupervisorTaskClientProvider createTaskClientFactory() - { - return (supervisorTaskId, httpTimeout, numRetries) -> createTaskClient(); - } - - private static ParallelIndexSupervisorTaskClient createTaskClient() - { - ParallelIndexSupervisorTaskClient taskClient = EasyMock.niceMock(ParallelIndexSupervisorTaskClient.class); - EasyMock.replay(taskClient); - return taskClient; - } - static String createRow(long timestamp, Object dimensionValue) { try { @@ -287,9 +145,4 @@ static String createRowFromMap(long timestamp, Map fields) throw new RuntimeException(e); } } - - static InputFormat getInputFormat() - { - return new JsonInputFormat(null, null, null, null, null); - } } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexTuningConfigTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexTuningConfigTest.java index a3f775509e2d..33d6a7aaa76f 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexTuningConfigTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexTuningConfigTest.java @@ -22,17 +22,14 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.jsontype.NamedType; import nl.jqno.equalsverifier.EqualsVerifier; +import org.apache.druid.indexer.partitions.DimensionRangePartitionsSpec; import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; import org.apache.druid.indexer.partitions.HashedPartitionsSpec; -import org.apache.druid.indexer.partitions.SingleDimensionPartitionsSpec; +import org.apache.druid.indexing.common.task.TuningConfigBuilder; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.segment.IndexSpec; -import org.apache.druid.segment.data.CompressionFactory.LongEncodingStrategy; import org.apache.druid.segment.data.CompressionStrategy; -import org.apache.druid.segment.data.RoaringBitmapSerdeFactory; import org.apache.druid.segment.indexing.TuningConfig; -import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; -import org.joda.time.Duration; import org.junit.Assert; import org.junit.Before; import org.junit.Rule; @@ -40,6 +37,7 @@ import org.junit.rules.ExpectedException; import java.io.IOException; +import java.util.Collections; public class ParallelIndexTuningConfigTest { @@ -57,201 +55,53 @@ public void setup() @Test public void testSerdeDefault() throws IOException { - final ParallelIndexTuningConfig tuningConfig = ParallelIndexTuningConfig.defaultConfig(); - final byte[] json = mapper.writeValueAsBytes(tuningConfig); - final ParallelIndexTuningConfig fromJson = (ParallelIndexTuningConfig) mapper.readValue(json, TuningConfig.class); - Assert.assertEquals(fromJson, tuningConfig); + verifyConfigSerde(ParallelIndexTuningConfig.defaultConfig()); } @Test - public void testSerdeWithMaxRowsPerSegment() - throws IOException + public void testSerdeWithNullMaxRowsPerSegment() throws IOException { - final ParallelIndexTuningConfig tuningConfig = new ParallelIndexTuningConfig( - null, - null, - null, - 10, - 1000L, - null, - null, - null, - null, - new DynamicPartitionsSpec(100, 100L), - IndexSpec.builder() - .withBitmapSerdeFactory(RoaringBitmapSerdeFactory.getInstance()) - .withDimensionCompression(CompressionStrategy.UNCOMPRESSED) - .withMetricCompression(CompressionStrategy.LZF) - .withLongEncoding(LongEncodingStrategy.LONGS) - .build(), - IndexSpec.DEFAULT, - 1, - false, - true, - 10000L, - OffHeapMemorySegmentWriteOutMediumFactory.instance(), - null, - 250, - 100, - 20L, - new Duration(3600), - 128, - null, - null, - false, - null, - null, - null, - null, - null, - 2 - ); - final byte[] json = mapper.writeValueAsBytes(tuningConfig); - final ParallelIndexTuningConfig fromJson = (ParallelIndexTuningConfig) mapper.readValue(json, TuningConfig.class); - Assert.assertEquals(fromJson, tuningConfig); + final ParallelIndexTuningConfig tuningConfig = TuningConfigBuilder + .forParallelIndexTask() + .withMaxRowsInMemory(10) + .withMaxBytesInMemory(1000L) + .withPartitionsSpec(new DynamicPartitionsSpec(100, 1000L)) + .withForceGuaranteedRollup(false) + .build(); + verifyConfigSerde(tuningConfig); } @Test public void testSerdeWithMaxNumConcurrentSubTasks() throws IOException { - final int maxNumConcurrentSubTasks = 250; - final ParallelIndexTuningConfig tuningConfig = new ParallelIndexTuningConfig( - null, - null, - null, - 10, - 1000L, - null, - null, - null, - null, - new DynamicPartitionsSpec(100, 100L), - IndexSpec.builder() - .withBitmapSerdeFactory(RoaringBitmapSerdeFactory.getInstance()) - .withDimensionCompression(CompressionStrategy.UNCOMPRESSED) - .withMetricCompression(CompressionStrategy.LZF) - .withLongEncoding(LongEncodingStrategy.LONGS) - .build(), - IndexSpec.DEFAULT, - 1, - false, - true, - 10000L, - OffHeapMemorySegmentWriteOutMediumFactory.instance(), - null, - maxNumConcurrentSubTasks, - 100, - 20L, - new Duration(3600), - 128, - null, - null, - false, - null, - null, - null, - null, - null, - 2 - ); - final byte[] json = mapper.writeValueAsBytes(tuningConfig); - final ParallelIndexTuningConfig fromJson = (ParallelIndexTuningConfig) mapper.readValue(json, TuningConfig.class); - Assert.assertEquals(fromJson, tuningConfig); + final ParallelIndexTuningConfig tuningConfig = TuningConfigBuilder + .forParallelIndexTask() + .withMaxNumConcurrentSubTasks(250) + .build(); + verifyConfigSerde(tuningConfig); } @Test public void testSerdeWithMaxNumSubTasks() throws IOException { - final int maxNumSubTasks = 250; - final ParallelIndexTuningConfig tuningConfig = new ParallelIndexTuningConfig( - null, - null, - null, - 10, - 1000L, - null, - null, - null, - null, - new DynamicPartitionsSpec(100, 100L), - IndexSpec.builder() - .withBitmapSerdeFactory(RoaringBitmapSerdeFactory.getInstance()) - .withDimensionCompression(CompressionStrategy.UNCOMPRESSED) - .withMetricCompression(CompressionStrategy.LZF) - .withLongEncoding(LongEncodingStrategy.LONGS) - .build(), - IndexSpec.DEFAULT, - 1, - false, - true, - 10000L, - OffHeapMemorySegmentWriteOutMediumFactory.instance(), - maxNumSubTasks, - null, - 100, - 20L, - new Duration(3600), - 128, - null, - null, - false, - null, - null, - null, - null, - null, - 2 - ); - final byte[] json = mapper.writeValueAsBytes(tuningConfig); - final ParallelIndexTuningConfig fromJson = (ParallelIndexTuningConfig) mapper.readValue(json, TuningConfig.class); - Assert.assertEquals(fromJson, tuningConfig); + final ParallelIndexTuningConfig tuningConfig = TuningConfigBuilder + .forParallelIndexTask() + .withMaxNumSubTasks(250) + .build(); + verifyConfigSerde(tuningConfig); } @Test - public void testSerdeWithMaxNumSubTasksAndMaxNumConcurrentSubTasks() + public void testConfigWithBothMaxNumSubTasksAndMaxNumConcurrentSubTasksIsInvalid() { expectedException.expect(IllegalArgumentException.class); expectedException.expectMessage("Can't use both maxNumSubTasks and maxNumConcurrentSubTasks"); final int maxNumSubTasks = 250; - final ParallelIndexTuningConfig tuningConfig = new ParallelIndexTuningConfig( - null, - null, - null, - 10, - 1000L, - null, - null, - null, - null, - new DynamicPartitionsSpec(100, 100L), - IndexSpec.builder() - .withBitmapSerdeFactory(RoaringBitmapSerdeFactory.getInstance()) - .withDimensionCompression(CompressionStrategy.UNCOMPRESSED) - .withMetricCompression(CompressionStrategy.LZF) - .withLongEncoding(LongEncodingStrategy.LONGS) - .build(), - IndexSpec.DEFAULT, - 1, - false, - true, - 10000L, - OffHeapMemorySegmentWriteOutMediumFactory.instance(), - maxNumSubTasks, - maxNumSubTasks, - 100, - 20L, - new Duration(3600), - 128, - null, - null, - false, - null, - null, - null, - null, - null, - null - ); + TuningConfigBuilder + .forParallelIndexTask() + .withMaxNumSubTasks(maxNumSubTasks) + .withMaxNumConcurrentSubTasks(maxNumSubTasks) + .build(); } @Test @@ -259,46 +109,11 @@ public void testConstructorWithHashedPartitionsSpecAndNonForceGuaranteedRollupFa { expectedException.expect(IllegalArgumentException.class); expectedException.expectMessage("DynamicPartitionsSpec must be used for best-effort rollup"); - final boolean forceGuaranteedRollup = false; - new ParallelIndexTuningConfig( - null, - null, - null, - 10, - 1000L, - null, - null, - null, - null, - new HashedPartitionsSpec(null, 10, null), - IndexSpec.builder() - .withBitmapSerdeFactory(RoaringBitmapSerdeFactory.getInstance()) - .withDimensionCompression(CompressionStrategy.UNCOMPRESSED) - .withMetricCompression(CompressionStrategy.LZF) - .withLongEncoding(LongEncodingStrategy.LONGS) - .build(), - IndexSpec.DEFAULT, - 1, - forceGuaranteedRollup, - true, - 10000L, - OffHeapMemorySegmentWriteOutMediumFactory.instance(), - null, - 10, - 100, - 20L, - new Duration(3600), - 128, - null, - null, - false, - null, - null, - null, - null, - null, - null - ); + TuningConfigBuilder + .forParallelIndexTask() + .withPartitionsSpec(new HashedPartitionsSpec(null, 10, null)) + .withForceGuaranteedRollup(false) + .build(); } @Test @@ -306,46 +121,11 @@ public void testConstructorWithSingleDimensionPartitionsSpecAndNonForceGuarantee { expectedException.expect(IllegalArgumentException.class); expectedException.expectMessage("DynamicPartitionsSpec must be used for best-effort rollup"); - final boolean forceGuaranteedRollup = false; - new ParallelIndexTuningConfig( - null, - null, - null, - 10, - 1000L, - null, - null, - null, - null, - new SingleDimensionPartitionsSpec(100, null, "dim", false), - IndexSpec.builder() - .withBitmapSerdeFactory(RoaringBitmapSerdeFactory.getInstance()) - .withDimensionCompression(CompressionStrategy.UNCOMPRESSED) - .withMetricCompression(CompressionStrategy.LZF) - .withLongEncoding(LongEncodingStrategy.LONGS) - .build(), - IndexSpec.DEFAULT, - 1, - forceGuaranteedRollup, - true, - 10000L, - OffHeapMemorySegmentWriteOutMediumFactory.instance(), - null, - 10, - 100, - 20L, - new Duration(3600), - 128, - null, - null, - false, - null, - null, - null, - null, - null, - null - ); + TuningConfigBuilder + .forParallelIndexTask() + .withPartitionsSpec(new DimensionRangePartitionsSpec(null, 100, Collections.singletonList("dim1"), false)) + .withForceGuaranteedRollup(false) + .build(); } @Test @@ -353,46 +133,19 @@ public void testConstructorWithDynamicPartitionsSpecAndForceGuaranteedRollupFail { expectedException.expect(IllegalArgumentException.class); expectedException.expectMessage("cannot be used for perfect rollup"); - final boolean forceGuaranteedRollup = true; - new ParallelIndexTuningConfig( - null, - null, - null, - 10, - 1000L, - null, - null, - null, - null, - new DynamicPartitionsSpec(100, null), - IndexSpec.builder() - .withBitmapSerdeFactory(RoaringBitmapSerdeFactory.getInstance()) - .withDimensionCompression(CompressionStrategy.UNCOMPRESSED) - .withMetricCompression(CompressionStrategy.LZF) - .withLongEncoding(LongEncodingStrategy.LONGS) - .build(), - IndexSpec.DEFAULT, - 1, - forceGuaranteedRollup, - true, - 10000L, - OffHeapMemorySegmentWriteOutMediumFactory.instance(), - null, - 10, - 100, - 20L, - new Duration(3600), - 128, - null, - null, - false, - null, - null, - null, - null, - null, - null - ); + TuningConfigBuilder + .forParallelIndexTask() + .withPartitionsSpec(new DynamicPartitionsSpec(100, null)) + .withForceGuaranteedRollup(true) + .build(); + } + + private void verifyConfigSerde(ParallelIndexTuningConfig tuningConfig) throws IOException + { + final byte[] json = mapper.writeValueAsBytes(tuningConfig); + final ParallelIndexTuningConfig fromJson = + (ParallelIndexTuningConfig) mapper.readValue(json, TuningConfig.class); + Assert.assertEquals(fromJson, tuningConfig); } @Test diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialDimensionCardinalityTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialDimensionCardinalityTaskTest.java index c4ae9392450a..0c33f002a613 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialDimensionCardinalityTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialDimensionCardinalityTaskTest.java @@ -29,14 +29,17 @@ import org.apache.druid.data.input.InputSource; import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.InlineInputSource; +import org.apache.druid.data.input.impl.JsonInputFormat; import org.apache.druid.indexer.TaskState; import org.apache.druid.indexer.TaskStatus; +import org.apache.druid.indexer.partitions.DimensionRangePartitionsSpec; import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; import org.apache.druid.indexer.partitions.HashedPartitionsSpec; import org.apache.druid.indexer.partitions.PartitionsSpec; import org.apache.druid.indexer.partitions.SingleDimensionPartitionsSpec; import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.common.stats.DropwizardRowIngestionMetersFactory; +import org.apache.druid.indexing.common.task.TuningConfigBuilder; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.granularity.Granularities; @@ -52,7 +55,6 @@ import org.apache.logging.log4j.core.LogEvent; import org.easymock.Capture; import org.easymock.EasyMock; -import org.hamcrest.Matchers; import org.joda.time.Interval; import org.junit.Assert; import org.junit.Before; @@ -82,9 +84,10 @@ public void requiresForceGuaranteedRollup() exception.expect(IllegalArgumentException.class); exception.expectMessage("forceGuaranteedRollup must be set"); - ParallelIndexTuningConfig tuningConfig = new ParallelIndexTestingFactory.TuningConfigBuilder() - .forceGuaranteedRollup(false) - .partitionsSpec(new DynamicPartitionsSpec(null, null)) + ParallelIndexTuningConfig tuningConfig = TuningConfigBuilder + .forParallelIndexTask() + .withPartitionsSpec(new DynamicPartitionsSpec(null, null)) + .withForceGuaranteedRollup(false) .build(); new PartialDimensionCardinalityTaskBuilder() @@ -100,7 +103,10 @@ public void requiresHashedPartitions() PartitionsSpec partitionsSpec = new SingleDimensionPartitionsSpec(null, 1, "a", false); ParallelIndexTuningConfig tuningConfig = - new ParallelIndexTestingFactory.TuningConfigBuilder().partitionsSpec(partitionsSpec).build(); + TuningConfigBuilder.forParallelIndexTask() + .withForceGuaranteedRollup(true) + .withPartitionsSpec(partitionsSpec) + .build(); new PartialDimensionCardinalityTaskBuilder() .tuningConfig(tuningConfig) @@ -136,7 +142,7 @@ public void hasCorrectPrefixForAutomaticId() PartialDimensionCardinalityTask task = new PartialDimensionCardinalityTaskBuilder() .id(ParallelIndexTestingFactory.AUTOMATIC_ID) .build(); - Assert.assertThat(task.getId(), Matchers.startsWith(PartialDimensionCardinalityTask.TYPE)); + Assert.assertTrue(task.getId().startsWith(PartialDimensionCardinalityTask.TYPE)); } } @@ -176,9 +182,11 @@ public void requiresPartitionDimension() throws Exception exception.expect(IllegalArgumentException.class); exception.expectMessage("partitionDimensions must be specified"); - ParallelIndexTuningConfig tuningConfig = new ParallelIndexTestingFactory.TuningConfigBuilder() - .partitionsSpec( - new ParallelIndexTestingFactory.SingleDimensionPartitionsSpecBuilder().partitionDimension(null).build() + ParallelIndexTuningConfig tuningConfig = TuningConfigBuilder + .forParallelIndexTask() + .withForceGuaranteedRollup(true) + .withPartitionsSpec( + new DimensionRangePartitionsSpec(null, null, null, false) ) .build(); PartialDimensionCardinalityTask task = new PartialDimensionCardinalityTaskBuilder() @@ -195,9 +203,11 @@ public void logsParseExceptionsIfEnabled() throws Exception InputSource inlineInputSource = new InlineInputSource( ParallelIndexTestingFactory.createRow(invalidTimestamp, "a") ); - ParallelIndexTuningConfig tuningConfig = new ParallelIndexTestingFactory.TuningConfigBuilder() - .partitionsSpec(HASHED_PARTITIONS_SPEC) - .logParseExceptions(true) + ParallelIndexTuningConfig tuningConfig = TuningConfigBuilder + .forParallelIndexTask() + .withPartitionsSpec(HASHED_PARTITIONS_SPEC) + .withForceGuaranteedRollup(true) + .withLogParseExceptions(true) .build(); PartialDimensionCardinalityTask task = new PartialDimensionCardinalityTaskBuilder() .inputSource(inlineInputSource) @@ -209,15 +219,17 @@ public void logsParseExceptionsIfEnabled() throws Exception List logEvents = logger.getLogEvents(); Assert.assertEquals(1, logEvents.size()); String logMessage = logEvents.get(0).getMessage().getFormattedMessage(); - Assert.assertThat(logMessage, Matchers.containsString("Encountered parse exception")); + Assert.assertTrue(logMessage.contains("Encountered parse exception")); } @Test public void doesNotLogParseExceptionsIfDisabled() throws Exception { - ParallelIndexTuningConfig tuningConfig = new ParallelIndexTestingFactory.TuningConfigBuilder() - .partitionsSpec(HASHED_PARTITIONS_SPEC) - .logParseExceptions(false) + ParallelIndexTuningConfig tuningConfig = TuningConfigBuilder + .forParallelIndexTask() + .withPartitionsSpec(HASHED_PARTITIONS_SPEC) + .withForceGuaranteedRollup(true) + .withLogParseExceptions(false) .build(); PartialDimensionCardinalityTask task = new PartialDimensionCardinalityTaskBuilder() .tuningConfig(tuningConfig) @@ -231,9 +243,11 @@ public void doesNotLogParseExceptionsIfDisabled() throws Exception @Test public void failsWhenTooManyParseExceptions() throws Exception { - ParallelIndexTuningConfig tuningConfig = new ParallelIndexTestingFactory.TuningConfigBuilder() - .partitionsSpec(HASHED_PARTITIONS_SPEC) - .maxParseExceptions(0) + ParallelIndexTuningConfig tuningConfig = TuningConfigBuilder + .forParallelIndexTask() + .withPartitionsSpec(HASHED_PARTITIONS_SPEC) + .withForceGuaranteedRollup(true) + .withMaxParseExceptions(0) .build(); PartialDimensionCardinalityTask task = new PartialDimensionCardinalityTaskBuilder() .tuningConfig(tuningConfig) @@ -273,11 +287,12 @@ public void sendsCorrectReportWhenNonEmptyPartitionDimension() ParallelIndexTestingFactory.createRowFromMap(0, ImmutableMap.of("dim1", "b", "dim2", "3")) + "\n" + ParallelIndexTestingFactory.createRowFromMap(0, ImmutableMap.of("dim1", "b", "dim2", "4")) ); - HashedPartitionsSpec partitionsSpec = new HashedPartitionsSpec(null, null, - Collections.singletonList("dim1") - ); - ParallelIndexTuningConfig tuningConfig = new ParallelIndexTestingFactory.TuningConfigBuilder() - .partitionsSpec(partitionsSpec) + HashedPartitionsSpec partitionsSpec = + new HashedPartitionsSpec(null, null, Collections.singletonList("dim1")); + ParallelIndexTuningConfig tuningConfig = TuningConfigBuilder + .forParallelIndexTask() + .withPartitionsSpec(partitionsSpec) + .withForceGuaranteedRollup(true) .build(); PartialDimensionCardinalityTaskBuilder taskBuilder = new PartialDimensionCardinalityTaskBuilder() @@ -357,12 +372,14 @@ private DimensionCardinalityReport runTask(PartialDimensionCardinalityTaskBuilde private static class PartialDimensionCardinalityTaskBuilder { - private static final InputFormat INPUT_FORMAT = ParallelIndexTestingFactory.getInputFormat(); + private static final InputFormat INPUT_FORMAT = new JsonInputFormat(null, null, null, null, null); private String id = ParallelIndexTestingFactory.ID; private InputSource inputSource = new InlineInputSource("row-with-invalid-timestamp"); - private ParallelIndexTuningConfig tuningConfig = new ParallelIndexTestingFactory.TuningConfigBuilder() - .partitionsSpec(HASHED_PARTITIONS_SPEC) + private ParallelIndexTuningConfig tuningConfig = TuningConfigBuilder + .forParallelIndexTask() + .withPartitionsSpec(HASHED_PARTITIONS_SPEC) + .withForceGuaranteedRollup(true) .build(); private DataSchema dataSchema = ParallelIndexTestingFactory diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialDimensionDistributionTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialDimensionDistributionTaskTest.java index 1346b29d889d..2f5c477e3f5f 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialDimensionDistributionTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialDimensionDistributionTaskTest.java @@ -19,21 +19,21 @@ package org.apache.druid.indexing.common.task.batch.parallel; -import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Joiner; import com.google.common.collect.Iterables; import org.apache.druid.data.input.InputFormat; import org.apache.druid.data.input.InputSource; import org.apache.druid.data.input.StringTuple; import org.apache.druid.data.input.impl.InlineInputSource; +import org.apache.druid.data.input.impl.JsonInputFormat; import org.apache.druid.indexer.TaskState; import org.apache.druid.indexer.TaskStatus; -import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; +import org.apache.druid.indexer.partitions.DimensionRangePartitionsSpec; import org.apache.druid.indexer.partitions.HashedPartitionsSpec; -import org.apache.druid.indexer.partitions.PartitionsSpec; import org.apache.druid.indexer.partitions.SingleDimensionPartitionsSpec; import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.common.stats.DropwizardRowIngestionMetersFactory; +import org.apache.druid.indexing.common.task.TuningConfigBuilder; import org.apache.druid.indexing.common.task.batch.parallel.distribution.StringDistribution; import org.apache.druid.indexing.common.task.batch.parallel.distribution.StringSketch; import org.apache.druid.java.util.common.StringUtils; @@ -48,7 +48,6 @@ import org.apache.logging.log4j.core.LogEvent; import org.easymock.Capture; import org.easymock.EasyMock; -import org.hamcrest.Matchers; import org.joda.time.Interval; import org.junit.Assert; import org.junit.Before; @@ -68,9 +67,8 @@ public class PartialDimensionDistributionTaskTest { - private static final ObjectMapper OBJECT_MAPPER = ParallelIndexTestingFactory.createObjectMapper(); private static final SingleDimensionPartitionsSpec SINGLE_DIM_PARTITIONS_SPEC = - new ParallelIndexTestingFactory.SingleDimensionPartitionsSpecBuilder().build(); + new SingleDimensionPartitionsSpec(null, 1000, "dim", false); public static class ConstructorTest { @@ -83,9 +81,9 @@ public void requiresForceGuaranteedRollup() exception.expect(IllegalArgumentException.class); exception.expectMessage("forceGuaranteedRollup must be set"); - ParallelIndexTuningConfig tuningConfig = new ParallelIndexTestingFactory.TuningConfigBuilder() - .forceGuaranteedRollup(false) - .partitionsSpec(new DynamicPartitionsSpec(null, null)) + ParallelIndexTuningConfig tuningConfig = TuningConfigBuilder + .forParallelIndexTask() + .withForceGuaranteedRollup(false) .build(); new PartialDimensionDistributionTaskBuilder() @@ -99,9 +97,11 @@ public void requiresMultiDimensionPartitions() exception.expect(IllegalArgumentException.class); exception.expectMessage("range partitionsSpec required"); - PartitionsSpec partitionsSpec = new HashedPartitionsSpec(null, 1, null); ParallelIndexTuningConfig tuningConfig = - new ParallelIndexTestingFactory.TuningConfigBuilder().partitionsSpec(partitionsSpec).build(); + TuningConfigBuilder.forParallelIndexTask() + .withForceGuaranteedRollup(true) + .withPartitionsSpec(new HashedPartitionsSpec(null, 1, null)) + .build(); new PartialDimensionDistributionTaskBuilder() .tuningConfig(tuningConfig) @@ -114,7 +114,7 @@ public void hasCorrectPrefixForAutomaticId() PartialDimensionDistributionTask task = new PartialDimensionDistributionTaskBuilder() .id(ParallelIndexTestingFactory.AUTOMATIC_ID) .build(); - Assert.assertThat(task.getId(), Matchers.startsWith(PartialDimensionDistributionTask.TYPE)); + Assert.assertTrue(task.getId().startsWith(PartialDimensionDistributionTask.TYPE)); } } @@ -154,10 +154,10 @@ public void requiresPartitionDimensions() throws Exception exception.expect(IllegalArgumentException.class); exception.expectMessage("partitionDimensions must be specified"); - ParallelIndexTuningConfig tuningConfig = new ParallelIndexTestingFactory.TuningConfigBuilder() - .partitionsSpec( - new ParallelIndexTestingFactory.SingleDimensionPartitionsSpecBuilder().partitionDimension(null).build() - ) + ParallelIndexTuningConfig tuningConfig = TuningConfigBuilder + .forParallelIndexTask() + .withForceGuaranteedRollup(true) + .withPartitionsSpec(new DimensionRangePartitionsSpec(null, null, null, false)) .build(); PartialDimensionDistributionTask task = new PartialDimensionDistributionTaskBuilder() .tuningConfig(tuningConfig) @@ -173,9 +173,11 @@ public void logsParseExceptionsIfEnabled() throws Exception InputSource inlineInputSource = new InlineInputSource( ParallelIndexTestingFactory.createRow(invalidTimestamp, "a") ); - ParallelIndexTuningConfig tuningConfig = new ParallelIndexTestingFactory.TuningConfigBuilder() - .partitionsSpec(SINGLE_DIM_PARTITIONS_SPEC) - .logParseExceptions(true) + ParallelIndexTuningConfig tuningConfig = TuningConfigBuilder + .forParallelIndexTask() + .withForceGuaranteedRollup(true) + .withPartitionsSpec(SINGLE_DIM_PARTITIONS_SPEC) + .withLogParseExceptions(true) .build(); PartialDimensionDistributionTask task = new PartialDimensionDistributionTaskBuilder() .inputSource(inlineInputSource) @@ -187,15 +189,17 @@ public void logsParseExceptionsIfEnabled() throws Exception List logEvents = logger.getLogEvents(); Assert.assertEquals(1, logEvents.size()); String logMessage = logEvents.get(0).getMessage().getFormattedMessage(); - Assert.assertThat(logMessage, Matchers.containsString("Encountered parse exception")); + Assert.assertTrue(logMessage.contains("Encountered parse exception")); } @Test public void doesNotLogParseExceptionsIfDisabled() throws Exception { - ParallelIndexTuningConfig tuningConfig = new ParallelIndexTestingFactory.TuningConfigBuilder() - .partitionsSpec(SINGLE_DIM_PARTITIONS_SPEC) - .logParseExceptions(false) + ParallelIndexTuningConfig tuningConfig = TuningConfigBuilder + .forParallelIndexTask() + .withForceGuaranteedRollup(true) + .withPartitionsSpec(SINGLE_DIM_PARTITIONS_SPEC) + .withLogParseExceptions(false) .build(); PartialDimensionDistributionTask task = new PartialDimensionDistributionTaskBuilder() .tuningConfig(tuningConfig) @@ -209,9 +213,11 @@ public void doesNotLogParseExceptionsIfDisabled() throws Exception @Test public void failsWhenTooManyParseExceptions() throws Exception { - ParallelIndexTuningConfig tuningConfig = new ParallelIndexTestingFactory.TuningConfigBuilder() - .partitionsSpec(SINGLE_DIM_PARTITIONS_SPEC) - .maxParseExceptions(0) + ParallelIndexTuningConfig tuningConfig = TuningConfigBuilder + .forParallelIndexTask() + .withForceGuaranteedRollup(true) + .withPartitionsSpec(SINGLE_DIM_PARTITIONS_SPEC) + .withMaxParseExceptions(0) .build(); PartialDimensionDistributionTask task = new PartialDimensionDistributionTaskBuilder() .tuningConfig(tuningConfig) @@ -247,9 +253,11 @@ public void sendsCorrectReportWhenAssumeGroupedTrue() ParallelIndexTestingFactory.createRow(timestamp, dimensionValue) + "\n" + ParallelIndexTestingFactory.createRow(timestamp + 1, dimensionValue) ); - ParallelIndexTuningConfig tuningConfig = new ParallelIndexTestingFactory.TuningConfigBuilder() - .partitionsSpec( - new ParallelIndexTestingFactory.SingleDimensionPartitionsSpecBuilder().assumeGrouped(true).build() + ParallelIndexTuningConfig tuningConfig = TuningConfigBuilder + .forParallelIndexTask() + .withForceGuaranteedRollup(true) + .withPartitionsSpec( + new DimensionRangePartitionsSpec(null, 1000, Collections.singletonList("dim1"), true) ) .build(); PartialDimensionDistributionTaskBuilder taskBuilder = new PartialDimensionDistributionTaskBuilder() @@ -277,9 +285,11 @@ public void groupsRowsWhenAssumeGroupedFalse() ParallelIndexTestingFactory.createRow(timestamp, dimensionValue) + "\n" + ParallelIndexTestingFactory.createRow(timestamp + 1, dimensionValue) ); - ParallelIndexTuningConfig tuningConfig = new ParallelIndexTestingFactory.TuningConfigBuilder() - .partitionsSpec( - new ParallelIndexTestingFactory.SingleDimensionPartitionsSpecBuilder().assumeGrouped(false).build() + ParallelIndexTuningConfig tuningConfig = TuningConfigBuilder + .forParallelIndexTask() + .withForceGuaranteedRollup(true) + .withPartitionsSpec( + new DimensionRangePartitionsSpec(null, 1000, Collections.singletonList("dim1"), false) ) .build(); PartialDimensionDistributionTaskBuilder taskBuilder = new PartialDimensionDistributionTaskBuilder() @@ -321,9 +331,11 @@ public void preservesMinAndMaxWhenAssumeGroupedFalse() rows.get(rows.size() - 1) ) ); - ParallelIndexTuningConfig tuningConfig = new ParallelIndexTestingFactory.TuningConfigBuilder() - .partitionsSpec( - new ParallelIndexTestingFactory.SingleDimensionPartitionsSpecBuilder().assumeGrouped(false).build() + ParallelIndexTuningConfig tuningConfig = TuningConfigBuilder + .forParallelIndexTask() + .withForceGuaranteedRollup(true) + .withPartitionsSpec( + new DimensionRangePartitionsSpec(null, 1000, Collections.singletonList("dim"), false) ) .build(); DataSchema dataSchema = ParallelIndexTestingFactory.createDataSchema(ParallelIndexTestingFactory.INPUT_INTERVALS); @@ -401,12 +413,16 @@ private DimensionDistributionReport runTask(PartialDimensionDistributionTaskBuil private static class PartialDimensionDistributionTaskBuilder { - private static final InputFormat INPUT_FORMAT = ParallelIndexTestingFactory.getInputFormat(); + private static final InputFormat INPUT_FORMAT = new JsonInputFormat(null, null, null, null, null); private String id = ParallelIndexTestingFactory.ID; private InputSource inputSource = new InlineInputSource("row-with-invalid-timestamp"); - private ParallelIndexTuningConfig tuningConfig = new ParallelIndexTestingFactory.TuningConfigBuilder() - .partitionsSpec(new ParallelIndexTestingFactory.SingleDimensionPartitionsSpecBuilder().build()) + private ParallelIndexTuningConfig tuningConfig = TuningConfigBuilder + .forParallelIndexTask() + .withForceGuaranteedRollup(true) + .withPartitionsSpec( + new DimensionRangePartitionsSpec(null, 1000, Collections.singletonList("dim"), false) + ) .build(); private DataSchema dataSchema = ParallelIndexTestingFactory.createDataSchema(ParallelIndexTestingFactory.INPUT_INTERVALS); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialGenericSegmentMergeTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialGenericSegmentMergeTaskTest.java index a51856f7353a..c56dae82849a 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialGenericSegmentMergeTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialGenericSegmentMergeTaskTest.java @@ -21,8 +21,8 @@ import com.google.common.collect.ImmutableMap; import org.apache.druid.indexer.partitions.HashedPartitionsSpec; +import org.apache.druid.indexing.common.task.TuningConfigBuilder; import org.apache.druid.segment.TestHelper; -import org.hamcrest.Matchers; import org.junit.Assert; import org.junit.Before; import org.junit.Rule; @@ -38,7 +38,7 @@ public class PartialGenericSegmentMergeTaskTest extends AbstractParallelIndexSupervisorTaskTest { @Parameterized.Parameters(name = "partitionLocation = {0}") - public static Iterable data() + public static Iterable data() { return Arrays.asList( GENERIC_PARTITION_LOCATION, @@ -71,7 +71,6 @@ public static Iterable data() private PartialGenericSegmentMergeTask target; private PartialSegmentMergeIOConfig ioConfig; private HashedPartitionsSpec partitionsSpec; - private PartialSegmentMergeIngestionSpec ingestionSpec; public PartialGenericSegmentMergeTaskTest() { @@ -88,12 +87,13 @@ public void setup() 1, Collections.emptyList() ); - ingestionSpec = new PartialSegmentMergeIngestionSpec( + PartialSegmentMergeIngestionSpec ingestionSpec = new PartialSegmentMergeIngestionSpec( ParallelIndexTestingFactory.createDataSchema(ParallelIndexTestingFactory.INPUT_INTERVALS), ioConfig, - new ParallelIndexTestingFactory.TuningConfigBuilder() - .partitionsSpec(partitionsSpec) - .build() + TuningConfigBuilder.forParallelIndexTask() + .withForceGuaranteedRollup(true) + .withPartitionsSpec(partitionsSpec) + .build() ); target = new PartialGenericSegmentMergeTask( ParallelIndexTestingFactory.AUTOMATIC_ID, @@ -117,7 +117,7 @@ public void serializesDeserializes() public void hasCorrectPrefixForAutomaticId() { String id = target.getId(); - Assert.assertThat(id, Matchers.startsWith(PartialGenericSegmentMergeTask.TYPE)); + Assert.assertTrue(id.startsWith(PartialGenericSegmentMergeTask.TYPE)); } @Test @@ -136,8 +136,9 @@ public void requiresGranularitySpecInputIntervals() new PartialSegmentMergeIngestionSpec( ParallelIndexTestingFactory.createDataSchema(null), ioConfig, - new ParallelIndexTestingFactory.TuningConfigBuilder() - .partitionsSpec(partitionsSpec) + TuningConfigBuilder.forParallelIndexTask() + .withForceGuaranteedRollup(true) + .withPartitionsSpec(partitionsSpec) .build() ), ParallelIndexTestingFactory.CONTEXT diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentGenerateTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentGenerateTaskTest.java index c2760209c010..a0b59d8c0757 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentGenerateTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentGenerateTaskTest.java @@ -25,6 +25,7 @@ import org.apache.druid.data.input.impl.JsonInputFormat; import org.apache.druid.data.input.impl.LocalInputSource; import org.apache.druid.indexer.partitions.HashedPartitionsSpec; +import org.apache.druid.indexing.common.task.TuningConfigBuilder; import org.apache.druid.indexing.common.task.batch.partition.HashPartitionAnalysis; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.granularity.Granularities; @@ -53,7 +54,10 @@ public class PartialHashSegmentGenerateTaskTest private static final ParallelIndexIngestionSpec INGESTION_SPEC = ParallelIndexTestingFactory.createIngestionSpec( new LocalInputSource(new File("baseDir"), "filer"), new JsonInputFormat(null, null, null, null, null), - new ParallelIndexTestingFactory.TuningConfigBuilder().build(), + TuningConfigBuilder.forParallelIndexTask() + .withForceGuaranteedRollup(true) + .withPartitionsSpec(new HashedPartitionsSpec(null, 2, null)) + .build(), ParallelIndexTestingFactory.createDataSchema(ParallelIndexTestingFactory.INPUT_INTERVALS) ); @@ -180,7 +184,10 @@ public void requiresGranularitySpecInputIntervals() ParallelIndexTestingFactory.createIngestionSpec( new LocalInputSource(new File("baseDir"), "filer"), new JsonInputFormat(null, null, null, null, null), - new ParallelIndexTestingFactory.TuningConfigBuilder().build(), + TuningConfigBuilder.forParallelIndexTask() + .withForceGuaranteedRollup(true) + .withPartitionsSpec(new HashedPartitionsSpec(null, 2, null)) + .build(), ParallelIndexTestingFactory.createDataSchema(null) ), ParallelIndexTestingFactory.CONTEXT, diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialRangeSegmentGenerateTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialRangeSegmentGenerateTaskTest.java index cf18be0f7487..cdeda412c6c7 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialRangeSegmentGenerateTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialRangeSegmentGenerateTaskTest.java @@ -24,9 +24,12 @@ import org.apache.druid.data.input.InputSource; import org.apache.druid.data.input.StringTuple; import org.apache.druid.data.input.impl.InlineInputSource; +import org.apache.druid.data.input.impl.JsonInputFormat; +import org.apache.druid.indexer.partitions.DimensionRangePartitionsSpec; import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; import org.apache.druid.indexer.partitions.HashedPartitionsSpec; import org.apache.druid.indexer.partitions.PartitionsSpec; +import org.apache.druid.indexing.common.task.TuningConfigBuilder; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.indexing.DataSchema; @@ -35,7 +38,6 @@ import org.apache.druid.server.security.ResourceAction; import org.apache.druid.server.security.ResourceType; import org.apache.druid.timeline.partition.PartitionBoundaries; -import org.hamcrest.Matchers; import org.junit.Assert; import org.junit.Rule; import org.junit.Test; @@ -60,11 +62,10 @@ public void requiresForceGuaranteedRollup() exception.expect(IllegalArgumentException.class); exception.expectMessage("range or single_dim partitionsSpec required"); - ParallelIndexTuningConfig tuningConfig = new ParallelIndexTestingFactory.TuningConfigBuilder() - .forceGuaranteedRollup(false) - .partitionsSpec(new DynamicPartitionsSpec(null, null)) + ParallelIndexTuningConfig tuningConfig = TuningConfigBuilder + .forParallelIndexTask() + .withPartitionsSpec(new DynamicPartitionsSpec(null, null)) .build(); - new PartialRangeSegmentGenerateTaskBuilder() .tuningConfig(tuningConfig) .build(); @@ -77,8 +78,11 @@ public void requiresMultiDimensionPartitions() exception.expectMessage("range or single_dim partitionsSpec required"); PartitionsSpec partitionsSpec = new HashedPartitionsSpec(null, 1, null); - ParallelIndexTuningConfig tuningConfig = - new ParallelIndexTestingFactory.TuningConfigBuilder().partitionsSpec(partitionsSpec).build(); + ParallelIndexTuningConfig tuningConfig = TuningConfigBuilder + .forParallelIndexTask() + .withForceGuaranteedRollup(true) + .withPartitionsSpec(partitionsSpec) + .build(); new PartialRangeSegmentGenerateTaskBuilder() .tuningConfig(tuningConfig) @@ -123,16 +127,20 @@ public void hasCorrectInputSourceResources() public void hasCorrectPrefixForAutomaticId() { PartialRangeSegmentGenerateTask task = new PartialRangeSegmentGenerateTaskBuilder().build(); - Assert.assertThat(task.getId(), Matchers.startsWith(PartialRangeSegmentGenerateTask.TYPE)); + Assert.assertTrue(task.getId().startsWith(PartialRangeSegmentGenerateTask.TYPE)); } private static class PartialRangeSegmentGenerateTaskBuilder { private static final InputSource INPUT_SOURCE = new InlineInputSource("data"); - private static final InputFormat INPUT_FORMAT = ParallelIndexTestingFactory.getInputFormat(); - - private ParallelIndexTuningConfig tuningConfig = new ParallelIndexTestingFactory.TuningConfigBuilder() - .partitionsSpec(new ParallelIndexTestingFactory.SingleDimensionPartitionsSpecBuilder().build()) + private static final InputFormat INPUT_FORMAT = new JsonInputFormat(null, null, null, null, null); + + private ParallelIndexTuningConfig tuningConfig = TuningConfigBuilder + .forParallelIndexTask() + .withForceGuaranteedRollup(true) + .withPartitionsSpec( + new DimensionRangePartitionsSpec(null, 1000, Collections.singletonList("dim"), false) + ) .build(); private DataSchema dataSchema = ParallelIndexTestingFactory.createDataSchema(ParallelIndexTestingFactory.INPUT_INTERVALS); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentMergeIOConfigTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentMergeIOConfigTest.java index a7901ab31646..e1a64af2731c 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentMergeIOConfigTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentMergeIOConfigTest.java @@ -64,15 +64,6 @@ public static Collection data() } private static final ObjectMapper OBJECT_MAPPER = ParallelIndexTestingFactory.createObjectMapper(); - private static final GenericPartitionLocation GENERIC_PARTITION_LOCATION = new GenericPartitionLocation( - ParallelIndexTestingFactory.HOST, - ParallelIndexTestingFactory.PORT, - ParallelIndexTestingFactory.USE_HTTPS, - ParallelIndexTestingFactory.SUBTASK_ID, - ParallelIndexTestingFactory.INTERVAL, - ParallelIndexTestingFactory.HASH_BASED_NUMBERED_SHARD_SPEC - ); - private PartialSegmentMergeIOConfig target; @Before diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentMergeIngestionSpecTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentMergeIngestionSpecTest.java index 04c28190b865..47549f2fabea 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentMergeIngestionSpecTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentMergeIngestionSpecTest.java @@ -22,6 +22,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableMap; import org.apache.druid.indexer.partitions.HashedPartitionsSpec; +import org.apache.druid.indexing.common.task.TuningConfigBuilder; import org.apache.druid.segment.TestHelper; import org.junit.Before; import org.junit.Test; @@ -37,7 +38,7 @@ public class PartialSegmentMergeIngestionSpecTest private static final ObjectMapper OBJECT_MAPPER = ParallelIndexTestingFactory.createObjectMapper(); @Parameterized.Parameters(name = "partitionLocation = {0}") - public static Iterable data() + public static Iterable data() { return Arrays.asList( GENERIC_PARTITION_LOCATION, @@ -65,24 +66,21 @@ public static Iterable data() ); private PartialSegmentMergeIngestionSpec target; - private PartialSegmentMergeIOConfig ioConfig; - private HashedPartitionsSpec partitionsSpec; @Before public void setup() { - ioConfig = new PartialSegmentMergeIOConfig(Collections.singletonList(partitionLocation)); - partitionsSpec = new HashedPartitionsSpec( - null, - 1, - Collections.emptyList() - ); + PartialSegmentMergeIOConfig ioConfig = + new PartialSegmentMergeIOConfig(Collections.singletonList(partitionLocation)); + HashedPartitionsSpec partitionsSpec = + new HashedPartitionsSpec(null, 1, Collections.emptyList()); target = new PartialSegmentMergeIngestionSpec( ParallelIndexTestingFactory.createDataSchema(ParallelIndexTestingFactory.INPUT_INTERVALS), ioConfig, - new ParallelIndexTestingFactory.TuningConfigBuilder() - .partitionsSpec(partitionsSpec) - .build() + TuningConfigBuilder.forParallelIndexTask() + .withForceGuaranteedRollup(true) + .withPartitionsSpec(partitionsSpec) + .build() ); } 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 5ad774386b65..de060e521a9f 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 @@ -36,6 +36,7 @@ import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.common.actions.TaskActionClient; import org.apache.druid.indexing.common.task.Tasks; +import org.apache.druid.indexing.common.task.TuningConfigBuilder; import org.apache.druid.indexing.overlord.Segments; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.StringUtils; @@ -157,7 +158,7 @@ public void teardown() @Test public void testIsReady() throws Exception { - final ParallelIndexSupervisorTask task = newTask(INTERVAL_TO_INDEX, false, true); + final ParallelIndexSupervisorTask task = newTask(INTERVAL_TO_INDEX, true); final TaskActionClient actionClient = createActionClient(task); final TaskToolbox toolbox = createTaskToolbox(task, actionClient); prepareTaskForLocking(task); @@ -534,7 +535,7 @@ public void testRunInSequential() { final Interval interval = Intervals.of("2017-12/P1M"); final boolean appendToExisting = false; - final ParallelIndexSupervisorTask task = newTask(interval, appendToExisting, false); + final ParallelIndexSupervisorTask task = newTask(interval, false); task.addToContext(Tasks.FORCE_TIME_CHUNK_LOCK_KEY, lockGranularity == LockGranularity.TIME_CHUNK); Assert.assertEquals(TaskState.SUCCESS, getIndexingServiceClient().runAndWait(task).getStatusCode()); assertShardSpec(task, lockGranularity, appendToExisting, Collections.emptyList()); @@ -583,7 +584,7 @@ public void testRunInSequential() @Test public void testPublishEmptySegments() { - final ParallelIndexSupervisorTask task = newTask(Intervals.of("2020-12/P1M"), false, true); + final ParallelIndexSupervisorTask task = newTask(Intervals.of("2020-12/P1M"), true); task.addToContext(Tasks.FORCE_TIME_CHUNK_LOCK_KEY, lockGranularity == LockGranularity.TIME_CHUNK); Assert.assertEquals(TaskState.SUCCESS, getIndexingServiceClient().runAndWait(task).getStatusCode()); } @@ -598,40 +599,9 @@ public void testWith1MaxNumConcurrentSubTasks() Granularities.DAY, appendToExisting, true, - new ParallelIndexTuningConfig( - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - 1, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null - ), + TuningConfigBuilder.forParallelIndexTask() + .withMaxNumConcurrentSubTasks(1) + .build(), VALID_INPUT_SOURCE_FILTER ); task.addToContext(Tasks.FORCE_TIME_CHUNK_LOCK_KEY, lockGranularity == LockGranularity.TIME_CHUNK); @@ -724,40 +694,10 @@ public void testMaxLocksWith1MaxNumConcurrentSubTasks() Granularities.DAY, appendToExisting, true, - new ParallelIndexTuningConfig( - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - 1, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - 0, - null - ), + TuningConfigBuilder.forParallelIndexTask() + .withMaxNumConcurrentSubTasks(1) + .withMaxAllowedLockCount(0) + .build(), VALID_INPUT_SOURCE_FILTER ); task.addToContext(Tasks.FORCE_TIME_CHUNK_LOCK_KEY, lockGranularity == LockGranularity.TIME_CHUNK); @@ -786,40 +726,10 @@ public void testMaxLocksWith2MaxNumConcurrentSubTasks() Granularities.DAY, appendToExisting, true, - new ParallelIndexTuningConfig( - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - 2, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - 0, - null - ), + TuningConfigBuilder.forParallelIndexTask() + .withMaxNumConcurrentSubTasks(2) + .withMaxAllowedLockCount(0) + .build(), VALID_INPUT_SOURCE_FILTER ); task.addToContext(Tasks.FORCE_TIME_CHUNK_LOCK_KEY, lockGranularity == LockGranularity.TIME_CHUNK); @@ -1006,11 +916,10 @@ public void testIngestBothExplicitAndImplicitDimsSchemaDiscovery() throws IOExce private ParallelIndexSupervisorTask newTask( @Nullable Interval interval, - boolean appendToExisting, boolean splittableInputSource ) { - return newTask(interval, Granularities.DAY, appendToExisting, splittableInputSource); + return newTask(interval, Granularities.DAY, false, splittableInputSource); } private ParallelIndexSupervisorTask newTask( diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java index 91d74a2bd677..7e421519963f 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java @@ -76,7 +76,6 @@ import org.apache.druid.indexing.common.task.IndexTask; import org.apache.druid.indexing.common.task.IndexTask.IndexIOConfig; import org.apache.druid.indexing.common.task.IndexTask.IndexIngestionSpec; -import org.apache.druid.indexing.common.task.IndexTask.IndexTuningConfig; import org.apache.druid.indexing.common.task.KillUnusedSegmentsTask; import org.apache.druid.indexing.common.task.NoopTask; import org.apache.druid.indexing.common.task.NoopTaskContextEnricher; @@ -84,6 +83,7 @@ import org.apache.druid.indexing.common.task.Task; import org.apache.druid.indexing.common.task.TaskResource; import org.apache.druid.indexing.common.task.TestAppenderatorsManager; +import org.apache.druid.indexing.common.task.TuningConfigBuilder; import org.apache.druid.indexing.overlord.config.DefaultTaskConfig; import org.apache.druid.indexing.overlord.config.TaskLockConfig; import org.apache.druid.indexing.overlord.config.TaskQueueConfig; @@ -730,33 +730,13 @@ public void testIndexTask() null ), new IndexIOConfig(null, new MockInputSource(), new NoopInputFormat(), false, false), - new IndexTuningConfig( - null, - 10000, - null, - 10, - null, - null, - null, - null, - null, - null, - null, - indexSpec, - null, - 3, - false, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null - ) + TuningConfigBuilder.forIndexTask() + .withMaxRowsPerSegment(10000) + .withMaxRowsInMemory(100) + .withIndexSpec(indexSpec) + .withMaxPendingPersists(3) + .withForceGuaranteedRollup(false) + .build() ), null ); @@ -815,33 +795,13 @@ public void testIndexTaskFailure() mapper ), new IndexIOConfig(null, new MockExceptionInputSource(), new NoopInputFormat(), false, false), - new IndexTuningConfig( - null, - 10000, - null, - 10, - null, - null, - null, - null, - null, - null, - null, - indexSpec, - null, - 3, - false, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null - ) + TuningConfigBuilder.forIndexTask() + .withMaxRowsPerSegment(10000) + .withMaxRowsInMemory(10) + .withIndexSpec(indexSpec) + .withMaxPendingPersists(3) + .withForceGuaranteedRollup(false) + .build() ), null ); @@ -1259,33 +1219,11 @@ public void testResumeTasks() throws Exception null ), new IndexIOConfig(null, new MockInputSource(), new NoopInputFormat(), false, false), - new IndexTuningConfig( - null, - 10000, - null, - 10, - null, - null, - null, - null, - null, - null, - null, - indexSpec, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null - ) + TuningConfigBuilder.forIndexTask() + .withMaxRowsPerSegment(10000) + .withMaxRowsInMemory(10) + .withIndexSpec(indexSpec) + .build() ), null ); @@ -1371,33 +1309,13 @@ public void testUnifiedAppenderatorsManagerCleanup() throws Exception null ), new IndexIOConfig(null, new MockInputSource(), new NoopInputFormat(), false, false), - new IndexTuningConfig( - null, - 10000, - null, - 10, - null, - null, - null, - null, - null, - null, - null, - indexSpec, - null, - 3, - false, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null - ) + TuningConfigBuilder.forIndexTask() + .withMaxRowsPerSegment(10000) + .withMaxRowsInMemory(10) + .withIndexSpec(indexSpec) + .withMaxPendingPersists(3) + .withForceGuaranteedRollup(false) + .build() ), null ); From d6c760f7ce74b5864cd7d0750736f1609d171fdf Mon Sep 17 00:00:00 2001 From: AmatyaAvadhanula Date: Mon, 15 Jul 2024 14:07:53 +0530 Subject: [PATCH 2/7] Do not kill segments with referenced load specs from deep storage (#16667) Do not kill segments with referenced load specs from deep storage --- .../ActionBasedPublishedSegmentRetriever.java | 2 +- .../RetrieveUpgradedFromSegmentIdsAction.java | 90 +++++++ .../RetrieveUpgradedToSegmentIdsAction.java | 95 ++++++++ .../indexing/common/actions/TaskAction.java | 2 + .../actions/UpgradedFromSegmentsResponse.java | 44 ++++ .../actions/UpgradedToSegmentsResponse.java | 46 ++++ .../common/task/KillUnusedSegmentsTask.java | 150 ++++++++++-- .../common/task/IngestionTestBase.java | 13 +- .../task/KillUnusedSegmentsTaskTest.java | 227 +++++++++++++++++- .../indexing/test/TestDataSegmentKiller.java | 13 +- ...TestIndexerMetadataStorageCoordinator.java | 18 ++ .../IndexerMetadataStorageCoordinator.java | 17 ++ .../IndexerSQLMetadataStorageCoordinator.java | 118 ++++++++- .../druid/metadata/PendingSegmentRecord.java | 5 +- .../druid/metadata/SQLMetadataConnector.java | 10 + .../metadata/SqlSegmentsMetadataQuery.java | 13 +- .../druid/server/http/DataSegmentPlus.java | 23 +- ...exerSQLMetadataStorageCoordinatorTest.java | 197 ++++++++++++++- ...SqlMetadataStorageCoordinatorTestBase.java | 49 ++++ .../server/http/DataSegmentPlusTest.java | 3 +- .../server/http/MetadataResourceTest.java | 2 +- 21 files changed, 1080 insertions(+), 57 deletions(-) create mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/common/actions/RetrieveUpgradedFromSegmentIdsAction.java create mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/common/actions/RetrieveUpgradedToSegmentIdsAction.java create mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/common/actions/UpgradedFromSegmentsResponse.java create mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/common/actions/UpgradedToSegmentsResponse.java diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/appenderator/ActionBasedPublishedSegmentRetriever.java b/indexing-service/src/main/java/org/apache/druid/indexing/appenderator/ActionBasedPublishedSegmentRetriever.java index ba5cf923b12e..bb349cc97902 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/appenderator/ActionBasedPublishedSegmentRetriever.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/appenderator/ActionBasedPublishedSegmentRetriever.java @@ -79,7 +79,7 @@ public Set findPublishedSegments(Set segmentIds) throws catch (Exception e) { log.warn( e, - "Could not retrieve published segment IDs[%s] using task action[segmentListById]." + "Could not retrieve published segment IDs[%s] using task action[retrieveSegmentsById]." + " Overlord maybe on an older version, retrying with action[segmentListUsed]." + " This task may fail to publish segments if there is a concurrent replace happening.", serializedSegmentIds diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/RetrieveUpgradedFromSegmentIdsAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/RetrieveUpgradedFromSegmentIdsAction.java new file mode 100644 index 000000000000..67f7ae6e1317 --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/RetrieveUpgradedFromSegmentIdsAction.java @@ -0,0 +1,90 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.common.actions; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.core.type.TypeReference; +import org.apache.druid.indexing.common.task.Task; + +import java.util.Set; + +/** + * Task action to retrieve the segment IDs from which a given set of segments were upgraded. + */ +public class RetrieveUpgradedFromSegmentIdsAction implements TaskAction +{ + private final String dataSource; + private final Set segmentIds; + + @JsonCreator + public RetrieveUpgradedFromSegmentIdsAction( + @JsonProperty("dataSource") String dataSource, + @JsonProperty("segmentIds") Set segmentIds + ) + { + this.dataSource = dataSource; + this.segmentIds = segmentIds; + } + + @JsonProperty + public String getDataSource() + { + return dataSource; + } + + @JsonProperty + public Set getSegmentIds() + { + return segmentIds; + } + + @Override + public TypeReference getReturnTypeReference() + { + return new TypeReference() + { + }; + } + + @Override + public UpgradedFromSegmentsResponse perform(Task task, TaskActionToolbox toolbox) + { + return new UpgradedFromSegmentsResponse( + toolbox.getIndexerMetadataStorageCoordinator() + .retrieveUpgradedFromSegmentIds(dataSource, segmentIds) + ); + } + + @Override + public boolean isAudited() + { + return false; + } + + @Override + public String toString() + { + return getClass().getSimpleName() + "{" + + "dataSource='" + dataSource + '\'' + + ", segmentIds=" + segmentIds + + '}'; + } +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/RetrieveUpgradedToSegmentIdsAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/RetrieveUpgradedToSegmentIdsAction.java new file mode 100644 index 000000000000..412c9604d114 --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/RetrieveUpgradedToSegmentIdsAction.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.indexing.common.actions; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.core.type.TypeReference; +import org.apache.druid.indexing.common.task.Task; + +import java.util.Set; + +/** + * Task action to determine the set of all segments containing the same load spec given the parent id.
+ * Returns a map from a segment ID to a set containing: + *
    + *
  1. all segment IDs that were upgraded from it AND are still present in the metadata store
  2. + *
  3. the segment ID itself if and only if it is still present in the metadata store
  4. + *
+ */ +public class RetrieveUpgradedToSegmentIdsAction implements TaskAction +{ + private final String dataSource; + private final Set segmentIds; + + @JsonCreator + public RetrieveUpgradedToSegmentIdsAction( + @JsonProperty("dataSource") String dataSource, + @JsonProperty("segmentIds") Set segmentIds + ) + { + this.dataSource = dataSource; + this.segmentIds = segmentIds; + } + + @JsonProperty + public String getDataSource() + { + return dataSource; + } + + @JsonProperty + public Set getSegmentIds() + { + return segmentIds; + } + + @Override + public TypeReference getReturnTypeReference() + { + return new TypeReference() + { + }; + } + + @Override + public UpgradedToSegmentsResponse perform(Task task, TaskActionToolbox toolbox) + { + return new UpgradedToSegmentsResponse( + toolbox.getIndexerMetadataStorageCoordinator() + .retrieveUpgradedToSegmentIds(dataSource, segmentIds) + ); + } + + @Override + public boolean isAudited() + { + return false; + } + + @Override + public String toString() + { + return getClass().getSimpleName() + "{" + + "dataSource='" + dataSource + '\'' + + ", segmentIds=" + segmentIds + + '}'; + } +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TaskAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TaskAction.java index 4606bd597a8d..973a83ecee43 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TaskAction.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TaskAction.java @@ -39,6 +39,8 @@ @JsonSubTypes.Type(name = "segmentTransactionalAppend", value = SegmentTransactionalAppendAction.class), @JsonSubTypes.Type(name = "segmentTransactionalReplace", value = SegmentTransactionalReplaceAction.class), @JsonSubTypes.Type(name = "retrieveSegmentsById", value = RetrieveSegmentsByIdAction.class), + @JsonSubTypes.Type(name = "retrieveUpgradedFromSegmentIds", value = RetrieveUpgradedFromSegmentIdsAction.class), + @JsonSubTypes.Type(name = "retrieveUpgradedToSegmentIds", value = RetrieveUpgradedToSegmentIdsAction.class), @JsonSubTypes.Type(name = "segmentListUsed", value = RetrieveUsedSegmentsAction.class), @JsonSubTypes.Type(name = "segmentListUnused", value = RetrieveUnusedSegmentsAction.class), @JsonSubTypes.Type(name = "markSegmentsAsUnused", value = MarkSegmentsAsUnusedAction.class), diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/UpgradedFromSegmentsResponse.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/UpgradedFromSegmentsResponse.java new file mode 100644 index 000000000000..5f0f1775f16a --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/UpgradedFromSegmentsResponse.java @@ -0,0 +1,44 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.common.actions; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +import java.util.Map; + +public class UpgradedFromSegmentsResponse +{ + private final Map upgradedFromSegmentIds; + + @JsonCreator + public UpgradedFromSegmentsResponse( + @JsonProperty("upgradedFromSegmentIds") Map upgradedFromSegmentIds + ) + { + this.upgradedFromSegmentIds = upgradedFromSegmentIds; + } + + @JsonProperty + public Map getUpgradedFromSegmentIds() + { + return upgradedFromSegmentIds; + } +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/UpgradedToSegmentsResponse.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/UpgradedToSegmentsResponse.java new file mode 100644 index 000000000000..e9bf33a97cee --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/UpgradedToSegmentsResponse.java @@ -0,0 +1,46 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.common.actions; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +import java.util.Map; +import java.util.Set; + +public class UpgradedToSegmentsResponse +{ + + private final Map> upgradedToSegmentIds; + + @JsonCreator + public UpgradedToSegmentsResponse( + @JsonProperty("upgradedToSegmentIds") Map> upgradedToSegmentIds + ) + { + this.upgradedToSegmentIds = upgradedToSegmentIds; + } + + @JsonProperty + public Map> getUpgradedToSegmentIds() + { + return upgradedToSegmentIds; + } +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/KillUnusedSegmentsTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/KillUnusedSegmentsTask.java index fe49569a3bbf..e1f6d2915eea 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/KillUnusedSegmentsTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/KillUnusedSegmentsTask.java @@ -35,11 +35,14 @@ import org.apache.druid.indexing.common.TaskLockType; import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.common.actions.RetrieveUnusedSegmentsAction; +import org.apache.druid.indexing.common.actions.RetrieveUpgradedFromSegmentIdsAction; +import org.apache.druid.indexing.common.actions.RetrieveUpgradedToSegmentIdsAction; import org.apache.druid.indexing.common.actions.RetrieveUsedSegmentsAction; import org.apache.druid.indexing.common.actions.SegmentNukeAction; import org.apache.druid.indexing.common.actions.TaskActionClient; import org.apache.druid.indexing.common.actions.TaskLocks; import org.apache.druid.indexing.common.actions.TimeChunkLockTryAcquireAction; +import org.apache.druid.indexing.common.actions.UpgradedToSegmentsResponse; import org.apache.druid.indexing.overlord.Segments; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.StringUtils; @@ -47,6 +50,8 @@ import org.apache.druid.server.lookup.cache.LookupLoadingSpec; import org.apache.druid.server.security.ResourceAction; import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.SegmentId; +import org.apache.druid.utils.CollectionUtils; import org.joda.time.DateTime; import org.joda.time.Interval; @@ -54,6 +59,7 @@ import javax.annotation.Nullable; import java.io.IOException; import java.util.ArrayList; +import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; @@ -63,9 +69,23 @@ import java.util.stream.Collectors; /** + *

* The client representation of this task is {@link ClientKillUnusedSegmentsTaskQuery}. * JSON serialization fields of this class must correspond to those of {@link * ClientKillUnusedSegmentsTaskQuery}, except for {@link #id} and {@link #context} fields. + *

+ * The Kill task fetches the set of used segments for the interval and computes the set of their load specs.
+ * Until `limit` segments have been processed in total or all segments for the interval have been nuked: + *

    + *
  1. Fetch at most `batchSize` unused segments from the metadata store.
  2. + *
  3. Determine the mapping from these segments to their parents *before* nuking the segments.
  4. + *
  5. Nuke the batch of unused segments from the metadata store.
  6. + *
  7. Determine the mapping of the set of parents to all their children.
  8. + *
  9. Check if unused or parent segments exist.
  10. + *
  11. Find the unreferenced segments.
  12. + *
  13. Filter the set of unreferenced segments using load specs from the set of used segments.
  14. + *
  15. Kill the filtered set of segments from deep storage.
  16. + *
*/ public class KillUnusedSegmentsTask extends AbstractFixedIntervalTask { @@ -76,7 +96,7 @@ public class KillUnusedSegmentsTask extends AbstractFixedIntervalTask * Default nuke batch size. This is a small enough size that we still get value from batching, while * yielding as quickly as possible. In one real cluster environment backed with mysql, ~2000rows/sec, * with batch size of 100, means a batch should only less than a second for the task lock, and depending - * on the segment store latency, unoptimised S3 cleanups typically take 5-10 seconds per 100. Over time + * on the segment store latency, unoptimised S3 cleanups typically take 5-10 seconds per 100. Over time, * we expect the S3 cleanup to get quicker, so this should be < 1 second, which means we'll be yielding * the task lockbox every 1-2 seconds. */ @@ -97,13 +117,15 @@ public class KillUnusedSegmentsTask extends AbstractFixedIntervalTask /** * Maximum number of segments that can be killed. */ - @Nullable private final Integer limit; + @Nullable + private final Integer limit; /** * The maximum used status last updated time. Any segments with * {@code used_status_last_updated} no later than this time will be included in the kill task. */ - @Nullable private final DateTime maxUsedStatusLastUpdatedTime; + @Nullable + private final DateTime maxUsedStatusLastUpdatedTime; @JsonCreator public KillUnusedSegmentsTask( @@ -196,18 +218,17 @@ public TaskStatus runTask(TaskToolbox toolbox) throws Exception numTotalBatches != null ? StringUtils.format(" in [%d] batches.", numTotalBatches) : "." ); + final TaskActionClient taskActionClient = toolbox.getTaskActionClient(); RetrieveUsedSegmentsAction retrieveUsedSegmentsAction = new RetrieveUsedSegmentsAction( getDataSource(), ImmutableList.of(getInterval()), Segments.INCLUDING_OVERSHADOWED ); // Fetch the load specs of all segments overlapping with the unused segment intervals - final Set> usedSegmentLoadSpecs = - new HashSet<>(toolbox.getTaskActionClient().submit(retrieveUsedSegmentsAction) - .stream() - .map(DataSegment::getLoadSpec) - .collect(Collectors.toSet()) - ); + final Set> usedSegmentLoadSpecs = taskActionClient.submit(retrieveUsedSegmentsAction) + .stream() + .map(DataSegment::getLoadSpec) + .collect(Collectors.toSet()); do { if (nextBatchSize <= 0) { @@ -231,20 +252,47 @@ public TaskStatus runTask(TaskToolbox toolbox) throws Exception ); } - // Kill segments - // Order is important here: we want the nuke action to clean up the metadata records _before_ the - // segments are removed from storage, this helps maintain that we will always have a storage segment if - // the metadata segment is present. If the segment nuke throws an exception, then the segment cleanup is - // abandoned. + // Kill segments. Order is important here: + // Retrieve the segment upgrade infos for the batch _before_ the segments are nuked + // We then want the nuke action to clean up the metadata records _before_ the segments are removed from storage. + // This helps maintain that we will always have a storage segment if the metadata segment is present. + // Determine the subset of segments to be killed from deep storage based on loadspecs. + // If the segment nuke throws an exception, then the segment cleanup is abandoned. + + // Determine upgraded segment ids before nuking + final Set segmentIds = unusedSegments.stream() + .map(DataSegment::getId) + .map(SegmentId::toString) + .collect(Collectors.toSet()); + final Map upgradedFromSegmentIds = new HashMap<>(); + try { + upgradedFromSegmentIds.putAll( + taskActionClient.submit( + new RetrieveUpgradedFromSegmentIdsAction(getDataSource(), segmentIds) + ).getUpgradedFromSegmentIds() + ); + } + catch (Exception e) { + LOG.warn( + e, + "Could not retrieve parent segment ids using task action[retrieveUpgradedFromSegmentIds]." + + " Overlord may be on an older version." + ); + } - toolbox.getTaskActionClient().submit(new SegmentNukeAction(new HashSet<>(unusedSegments))); + // Nuke Segments + taskActionClient.submit(new SegmentNukeAction(new HashSet<>(unusedSegments))); - // Kill segments from the deep storage only if their load specs are not being used by any used segments - final List segmentsToBeKilled = unusedSegments - .stream() - .filter(unusedSegment -> unusedSegment.getLoadSpec() == null - || !usedSegmentLoadSpecs.contains(unusedSegment.getLoadSpec())) - .collect(Collectors.toList()); + // Determine segments to be killed + final List segmentsToBeKilled + = getKillableSegments(unusedSegments, upgradedFromSegmentIds, usedSegmentLoadSpecs, taskActionClient); + + final Set segmentsNotKilled = new HashSet<>(unusedSegments); + segmentsToBeKilled.forEach(segmentsNotKilled::remove); + LOG.infoSegments( + segmentsNotKilled, + "Skipping segment kill from deep storage as their load specs are referenced by other segments." + ); toolbox.getDataSegmentKiller().kill(segmentsToBeKilled); numBatchesProcessed++; @@ -253,7 +301,7 @@ public TaskStatus runTask(TaskToolbox toolbox) throws Exception LOG.info("Processed [%d] batches for kill task[%s].", numBatchesProcessed, getId()); nextBatchSize = computeNextBatchSize(numSegmentsKilled); - } while (unusedSegments.size() != 0 && (null == numTotalBatches || numBatchesProcessed < numTotalBatches)); + } while (!unusedSegments.isEmpty() && (null == numTotalBatches || numBatchesProcessed < numTotalBatches)); final String taskId = getId(); LOG.info( @@ -300,6 +348,64 @@ private NavigableMap> getNonRevokedTaskLockMap(TaskActi return taskLockMap; } + /** + * Determines subset of segments without referenced load specs that can be safely killed by + * looking at the segment upgrades and used segment load specs + * @param unusedSegments input segments + * @param upgradedFromSegmentIds segment to parent mapping + * @param usedSegmentLoadSpecs load specs of used segments + * @param taskActionClient task action client + * @return list of segments to kill from deep storage + */ + private List getKillableSegments( + List unusedSegments, + Map upgradedFromSegmentIds, + Set> usedSegmentLoadSpecs, + TaskActionClient taskActionClient + ) + { + + // Determine parentId for each unused segment + final Map> parentIdToUnusedSegments = new HashMap<>(); + for (DataSegment segment : unusedSegments) { + final String segmentId = segment.getId().toString(); + parentIdToUnusedSegments.computeIfAbsent( + upgradedFromSegmentIds.getOrDefault(segmentId, segmentId), + k -> new HashSet<>() + ).add(segment); + } + + // Check if the parent or any of its children exist in metadata store + try { + UpgradedToSegmentsResponse response = taskActionClient.submit( + new RetrieveUpgradedToSegmentIdsAction(getDataSource(), parentIdToUnusedSegments.keySet()) + ); + if (response != null && response.getUpgradedToSegmentIds() != null) { + response.getUpgradedToSegmentIds().forEach((parent, children) -> { + if (!CollectionUtils.isNullOrEmpty(children)) { + // Do not kill segment if its parent or any of its siblings still exist in metadata store + parentIdToUnusedSegments.remove(parent); + } + }); + } + } + catch (Exception e) { + LOG.warn( + e, + "Could not retrieve referenced ids using task action[retrieveUpgradedToSegmentIds]." + + " Overlord may be on an older version." + ); + } + + // Filter using the used segment load specs as segment upgrades predate the above task action + return parentIdToUnusedSegments.values() + .stream() + .flatMap(Set::stream) + .filter(segment -> !usedSegmentLoadSpecs.contains(segment.getLoadSpec())) + .collect(Collectors.toList()); + } + + @Override public LookupLoadingSpec getLookupLoadingSpec() { diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IngestionTestBase.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IngestionTestBase.java index 133ced3907dc..d6687efbf34e 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IngestionTestBase.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IngestionTestBase.java @@ -61,6 +61,7 @@ import org.apache.druid.indexing.overlord.TaskStorage; import org.apache.druid.indexing.overlord.autoscaling.ScalingStats; import org.apache.druid.indexing.overlord.supervisor.SupervisorManager; +import org.apache.druid.indexing.test.TestDataSegmentKiller; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.RE; @@ -81,7 +82,6 @@ import org.apache.druid.segment.join.NoopJoinableFactory; import org.apache.druid.segment.loading.LocalDataSegmentPusher; import org.apache.druid.segment.loading.LocalDataSegmentPusherConfig; -import org.apache.druid.segment.loading.NoopDataSegmentKiller; import org.apache.druid.segment.loading.SegmentCacheManager; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; import org.apache.druid.segment.metadata.SegmentSchemaCache; @@ -130,6 +130,7 @@ public abstract class IngestionTestBase extends InitializedNullHandlingTest private SegmentSchemaManager segmentSchemaManager; private SegmentSchemaCache segmentSchemaCache; private SupervisorManager supervisorManager; + private TestDataSegmentKiller dataSegmentKiller; protected File reportsFile; @Before @@ -169,6 +170,7 @@ public void setUpIngestionTestBase() throws IOException lockbox = new TaskLockbox(taskStorage, storageCoordinator); segmentCacheManagerFactory = new SegmentCacheManagerFactory(TestIndex.INDEX_IO, getObjectMapper()); reportsFile = temporaryFolder.newFile(); + dataSegmentKiller = new TestDataSegmentKiller(); } @After @@ -243,6 +245,11 @@ public RowIngestionMetersFactory getRowIngestionMetersFactory() return testUtils.getRowIngestionMetersFactory(); } + public TestDataSegmentKiller getDataSegmentKiller() + { + return dataSegmentKiller; + } + public TaskActionToolbox createTaskActionToolbox() { storageCoordinator.start(); @@ -265,7 +272,7 @@ public TaskToolbox createTaskToolbox(TaskConfig config, Task task, SupervisorMan .taskExecutorNode(new DruidNode("druid/middlemanager", "localhost", false, 8091, null, true, false)) .taskActionClient(createActionClient(task)) .segmentPusher(new LocalDataSegmentPusher(new LocalDataSegmentPusherConfig())) - .dataSegmentKiller(new NoopDataSegmentKiller()) + .dataSegmentKiller(dataSegmentKiller) .joinableFactory(NoopJoinableFactory.INSTANCE) .jsonMapper(objectMapper) .taskWorkDir(baseDir) @@ -450,7 +457,7 @@ public ListenableFuture run(Task task) .taskExecutorNode(new DruidNode("druid/middlemanager", "localhost", false, 8091, null, true, false)) .taskActionClient(taskActionClient) .segmentPusher(new LocalDataSegmentPusher(new LocalDataSegmentPusherConfig())) - .dataSegmentKiller(new NoopDataSegmentKiller()) + .dataSegmentKiller(dataSegmentKiller) .joinableFactory(NoopJoinableFactory.INSTANCE) .jsonMapper(objectMapper) .taskWorkDir(baseDir) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/KillUnusedSegmentsTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/KillUnusedSegmentsTaskTest.java index f888ace5a54d..fe2b5a51c86a 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/KillUnusedSegmentsTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/KillUnusedSegmentsTaskTest.java @@ -36,6 +36,7 @@ import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.JodaUtils; +import org.apache.druid.metadata.IndexerSqlMetadataStorageCoordinatorTestBase; import org.apache.druid.server.lookup.cache.LookupLoadingSpec; import org.apache.druid.timeline.DataSegment; import org.assertj.core.api.Assertions; @@ -72,10 +73,10 @@ public void setup() taskRunner = new TestTaskRunner(); final String version = DateTimes.nowUtc().toString(); - segment1 = newSegment(Intervals.of("2019-01-01/2019-02-01"), version); - segment2 = newSegment(Intervals.of("2019-02-01/2019-03-01"), version); - segment3 = newSegment(Intervals.of("2019-03-01/2019-04-01"), version); - segment4 = newSegment(Intervals.of("2019-04-01/2019-05-01"), version); + segment1 = newSegment(Intervals.of("2019-01-01/2019-02-01"), version).withLoadSpec(ImmutableMap.of("k", 1)); + segment2 = newSegment(Intervals.of("2019-02-01/2019-03-01"), version).withLoadSpec(ImmutableMap.of("k", 2)); + segment3 = newSegment(Intervals.of("2019-03-01/2019-04-01"), version).withLoadSpec(ImmutableMap.of("k", 3)); + segment4 = newSegment(Intervals.of("2019-04-01/2019-05-01"), version).withLoadSpec(ImmutableMap.of("k", 4)); } @Test @@ -125,6 +126,212 @@ public void testKill() throws Exception new KillTaskReport.Stats(1, 2), getReportedStats() ); + Assert.assertEquals(ImmutableSet.of(segment3), getDataSegmentKiller().getKilledSegments()); + } + + @Test + public void testKillSegmentsDeleteUnreferencedSiblings() throws Exception + { + final Map upgradeSegmentMapping = ImmutableMap.of( + segment1.getId().toString(), + "nonExistentParent", + segment2.getId().toString(), + "nonExistentParent" + ); + insertUsedSegments(ImmutableSet.of(segment1, segment2), upgradeSegmentMapping); + getStorageCoordinator().markSegmentsAsUnusedWithinInterval(DATA_SOURCE, Intervals.ETERNITY); + + + final KillUnusedSegmentsTask task = new KillUnusedSegmentsTaskBuilder() + .dataSource(DATA_SOURCE) + .interval(Intervals.ETERNITY) + .build(); + + Assert.assertEquals(TaskState.SUCCESS, taskRunner.run(task).get().getStatusCode()); + + final List observedUnusedSegments = + getMetadataStorageCoordinator().retrieveUnusedSegmentsForInterval( + DATA_SOURCE, + Intervals.ETERNITY, + null, + null, + null + ); + + Assert.assertEquals(Collections.emptyList(), observedUnusedSegments); + + Assert.assertEquals( + new KillTaskReport.Stats(2, 2), + getReportedStats() + ); + Assert.assertEquals(ImmutableSet.of(segment1, segment2), getDataSegmentKiller().getKilledSegments()); + } + + @Test + public void testKillSegmentsDoNotDeleteReferencedSibling() throws Exception + { + final Map upgradeSegmentMapping = ImmutableMap.of( + segment1.getId().toString(), + "nonExistentParent", + segment2.getId().toString(), + "nonExistentParent" + ); + insertUsedSegments(ImmutableSet.of(segment1, segment2), upgradeSegmentMapping); + getStorageCoordinator().markSegmentsAsUnusedWithinInterval(DATA_SOURCE, Intervals.ETERNITY); + + + final KillUnusedSegmentsTask task = new KillUnusedSegmentsTaskBuilder() + .dataSource(DATA_SOURCE) + .interval(segment1.getInterval()) + .build(); + + Assert.assertEquals(TaskState.SUCCESS, taskRunner.run(task).get().getStatusCode()); + + final List observedUnusedSegments = + getMetadataStorageCoordinator().retrieveUnusedSegmentsForInterval( + DATA_SOURCE, + Intervals.ETERNITY, + null, + null, + null + ); + + Assert.assertEquals(Collections.singletonList(segment2), observedUnusedSegments); + + Assert.assertEquals( + new KillTaskReport.Stats(0, 2), + getReportedStats() + ); + Assert.assertEquals(Collections.emptySet(), getDataSegmentKiller().getKilledSegments()); + } + + @Test + public void testKillSegmentsDoNotDeleteParentWithReferencedChildren() throws Exception + { + final Map upgradeSegmentMapping = ImmutableMap.of( + segment1.getId().toString(), + segment3.getId().toString(), + segment2.getId().toString(), + segment3.getId().toString() + ); + insertUsedSegments(ImmutableSet.of(segment1, segment2, segment3), upgradeSegmentMapping); + getSegmentsMetadataManager().markSegmentAsUnused(segment2.getId()); + getSegmentsMetadataManager().markSegmentAsUnused(segment3.getId()); + + + final KillUnusedSegmentsTask task = new KillUnusedSegmentsTaskBuilder() + .dataSource(DATA_SOURCE) + .interval(Intervals.ETERNITY) + .build(); + + Assert.assertEquals(TaskState.SUCCESS, taskRunner.run(task).get().getStatusCode()); + + final List observedUnusedSegments = + getMetadataStorageCoordinator().retrieveUnusedSegmentsForInterval( + DATA_SOURCE, + Intervals.ETERNITY, + null, + null, + null + ); + Assert.assertEquals(ImmutableList.of(), observedUnusedSegments); + Assertions.assertThat( + getMetadataStorageCoordinator().retrieveUsedSegmentsForInterval( + DATA_SOURCE, + Intervals.ETERNITY, + Segments.ONLY_VISIBLE + ) + ).containsExactlyInAnyOrder(segment1); + + Assert.assertEquals( + new KillTaskReport.Stats(0, 2), + getReportedStats() + ); + Assert.assertEquals(Collections.emptySet(), getDataSegmentKiller().getKilledSegments()); + } + + @Test + public void testKillSegmentsDoNotDeleteChildrenWithReferencedParent() throws Exception + { + final Map upgradeSegmentMapping = ImmutableMap.of( + segment1.getId().toString(), + segment3.getId().toString(), + segment2.getId().toString(), + segment3.getId().toString() + ); + insertUsedSegments(ImmutableSet.of(segment1, segment2, segment3), upgradeSegmentMapping); + getSegmentsMetadataManager().markSegmentAsUnused(segment1.getId()); + getSegmentsMetadataManager().markSegmentAsUnused(segment2.getId()); + + + final KillUnusedSegmentsTask task = new KillUnusedSegmentsTaskBuilder() + .dataSource(DATA_SOURCE) + .interval(Intervals.ETERNITY) + .build(); + + Assert.assertEquals(TaskState.SUCCESS, taskRunner.run(task).get().getStatusCode()); + + final List observedUnusedSegments = + getMetadataStorageCoordinator().retrieveUnusedSegmentsForInterval( + DATA_SOURCE, + Intervals.ETERNITY, + null, + null, + null + ); + Assert.assertEquals(ImmutableList.of(), observedUnusedSegments); + Assertions.assertThat( + getMetadataStorageCoordinator().retrieveUsedSegmentsForInterval( + DATA_SOURCE, + Intervals.ETERNITY, + Segments.ONLY_VISIBLE + ) + ).containsExactlyInAnyOrder(segment3); + + Assert.assertEquals( + new KillTaskReport.Stats(0, 2), + getReportedStats() + ); + Assert.assertEquals(Collections.emptySet(), getDataSegmentKiller().getKilledSegments()); + } + + @Test + public void testKillSegmentsDeleteChildrenAndParent() throws Exception + { + final Map upgradeSegmentMapping = ImmutableMap.of( + segment1.getId().toString(), + segment3.getId().toString(), + segment2.getId().toString(), + segment3.getId().toString() + ); + insertUsedSegments(ImmutableSet.of(segment1, segment2, segment3), upgradeSegmentMapping); + getSegmentsMetadataManager().markSegmentAsUnused(segment1.getId()); + getSegmentsMetadataManager().markSegmentAsUnused(segment2.getId()); + getSegmentsMetadataManager().markSegmentAsUnused(segment3.getId()); + + + final KillUnusedSegmentsTask task = new KillUnusedSegmentsTaskBuilder() + .dataSource(DATA_SOURCE) + .interval(Intervals.ETERNITY) + .build(); + + Assert.assertEquals(TaskState.SUCCESS, taskRunner.run(task).get().getStatusCode()); + + final List observedUnusedSegments = + getMetadataStorageCoordinator().retrieveUnusedSegmentsForInterval( + DATA_SOURCE, + Intervals.ETERNITY, + null, + null, + null + ); + Assert.assertEquals(ImmutableList.of(), observedUnusedSegments); + + Assert.assertEquals( + new KillTaskReport.Stats(3, 2), + getReportedStats() + ); + Assert.assertEquals(ImmutableSet.of(segment1, segment2, segment3), getDataSegmentKiller().getKilledSegments()); } @Test @@ -1247,4 +1454,16 @@ private static DataSegment newSegment(Interval interval, String version, Map segments, Map upgradedFromSegmentIdMap) + { + final String table = derbyConnectorRule.metadataTablesConfigSupplier().get().getSegmentsTable(); + IndexerSqlMetadataStorageCoordinatorTestBase.insertUsedSegments( + segments, + upgradedFromSegmentIdMap, + derbyConnectorRule.getConnector(), + table, + getObjectMapper() + ); + } } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/test/TestDataSegmentKiller.java b/indexing-service/src/test/java/org/apache/druid/indexing/test/TestDataSegmentKiller.java index 33421eb1a5cb..92581f6dd1ed 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/test/TestDataSegmentKiller.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/test/TestDataSegmentKiller.java @@ -22,12 +22,18 @@ import org.apache.druid.segment.loading.DataSegmentKiller; import org.apache.druid.timeline.DataSegment; +import java.util.HashSet; +import java.util.Set; + public class TestDataSegmentKiller implements DataSegmentKiller { + + private final Set killedSegments = new HashSet<>(); + @Override public void kill(DataSegment segment) { - // do nothing + killedSegments.add(segment); } @Override @@ -35,4 +41,9 @@ public void killAll() { throw new UnsupportedOperationException("not implemented"); } + + public Set getKilledSegments() + { + return killedSegments; + } } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/test/TestIndexerMetadataStorageCoordinator.java b/indexing-service/src/test/java/org/apache/druid/indexing/test/TestIndexerMetadataStorageCoordinator.java index 61a57e948423..d2055d6e0c99 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/test/TestIndexerMetadataStorageCoordinator.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/test/TestIndexerMetadataStorageCoordinator.java @@ -314,6 +314,24 @@ public List getPendingSegments(String datasource, Interval throw new UnsupportedOperationException(); } + @Override + public Map retrieveUpgradedFromSegmentIds( + final String dataSource, + final Set segmentIds + ) + { + return Collections.emptyMap(); + } + + @Override + public Map> retrieveUpgradedToSegmentIds( + final String dataSource, + final Set segmentIds + ) + { + return Collections.emptyMap(); + } + public Set getPublished() { return ImmutableSet.copyOf(published); diff --git a/server/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java b/server/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java index c055a8d9e9f4..83b4ac7e474c 100644 --- a/server/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java +++ b/server/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java @@ -473,4 +473,21 @@ SegmentPublishResult commitMetadataOnly( * @return List of pending segment records */ List getPendingSegments(String datasource, Interval interval); + + /** + * Map from a segment ID to the segment ID from which it was upgraded + * There should be no entry in the map for an original non-upgraded segment + * @param dataSource data source + * @param segmentIds ids of segments + */ + Map retrieveUpgradedFromSegmentIds(String dataSource, Set segmentIds); + + /** + * Map from a segment ID to a set containing + * 1) all segment IDs that were upgraded from it AND are still present in the metadata store + * 2) the segment ID itself if and only if it is still present in the metadata store + * @param dataSource data source + * @param segmentIds ids of the first segments which had the corresponding load spec + */ + Map> retrieveUpgradedToSegmentIds(String dataSource, Set segmentIds); } diff --git a/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java b/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java index fd6377289081..54f75ccb9201 100644 --- a/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java +++ b/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java @@ -564,6 +564,7 @@ public SegmentPublishResult commitReplaceSegments( createNewIdsOfAppendSegmentsAfterReplace(handle, replaceSegments, locksHeldByReplaceTask); Map upgradeSegmentMetadata = new HashMap<>(); + final Map upgradedFromSegmentIdMap = new HashMap<>(); for (DataSegmentPlus dataSegmentPlus : upgradedSegments) { segmentsToInsert.add(dataSegmentPlus.getDataSegment()); if (dataSegmentPlus.getSchemaFingerprint() != null && dataSegmentPlus.getNumRows() != null) { @@ -572,6 +573,12 @@ public SegmentPublishResult commitReplaceSegments( new SegmentMetadata(dataSegmentPlus.getNumRows(), dataSegmentPlus.getSchemaFingerprint()) ); } + if (dataSegmentPlus.getUpgradedFromSegmentId() != null) { + upgradedFromSegmentIdMap.put( + dataSegmentPlus.getDataSegment().getId().toString(), + dataSegmentPlus.getUpgradedFromSegmentId() + ); + } } SegmentPublishResult result = SegmentPublishResult.ok( insertSegments( @@ -579,7 +586,8 @@ public SegmentPublishResult commitReplaceSegments( segmentsToInsert, segmentSchemaMapping, upgradeSegmentMetadata, - Collections.emptyMap() + Collections.emptyMap(), + upgradedFromSegmentIdMap ), upgradePendingSegmentsOverlappingWith(segmentsToInsert) ); @@ -1408,6 +1416,7 @@ private SegmentPublishResult commitAppendSegmentsAndMetadataInTransaction( final Set allSegmentsToInsert = new HashSet<>(appendSegments); final Map newVersionSegmentToParent = new HashMap<>(); final Map segmentIdMap = new HashMap<>(); + final Map upgradedFromSegmentIdMap = new HashMap<>(); appendSegments.forEach(segment -> segmentIdMap.put(segment.getId().toString(), segment)); segmentIdsForNewVersions.forEach( pendingSegment -> { @@ -1415,6 +1424,7 @@ private SegmentPublishResult commitAppendSegmentsAndMetadataInTransaction( final DataSegment oldSegment = segmentIdMap.get(pendingSegment.getUpgradedFromSegmentId()); final SegmentId newVersionSegmentId = pendingSegment.getId().asSegmentId(); newVersionSegmentToParent.put(newVersionSegmentId, oldSegment.getId()); + upgradedFromSegmentIdMap.put(newVersionSegmentId.toString(), oldSegment.getId().toString()); allSegmentsToInsert.add( new DataSegment( pendingSegment.getId().asSegmentId(), @@ -1473,7 +1483,8 @@ private SegmentPublishResult commitAppendSegmentsAndMetadataInTransaction( allSegmentsToInsert, segmentSchemaMapping, Collections.emptyMap(), - newVersionSegmentToParent + newVersionSegmentToParent, + upgradedFromSegmentIdMap ) ); }, @@ -2092,7 +2103,8 @@ private Set announceHistoricalSegmentBatch( .bind("version", segment.getVersion()) .bind("used", usedSegments.contains(segment)) .bind("payload", jsonMapper.writeValueAsBytes(segment)) - .bind("used_status_last_updated", now); + .bind("used_status_last_updated", now) + .bind("upgraded_from_segment_id", (String) null); if (schemaPersistEnabled) { Long numRows = null; @@ -2217,6 +2229,11 @@ private Set createNewIdsOfAppendSegmentsAfterReplace( .shardSpec(shardSpec) .build(); + // When the segment already has an upgraded_from_segment_id, reuse it for its children + final String upgradedFromSegmentId = oldSegmentMetadata.getUpgradedFromSegmentId() == null + ? oldSegmentMetadata.getDataSegment().getId().toString() + : oldSegmentMetadata.getUpgradedFromSegmentId(); + upgradedSegments.add( new DataSegmentPlus( dataSegment, @@ -2224,7 +2241,9 @@ private Set createNewIdsOfAppendSegmentsAfterReplace( null, null, oldSegmentMetadata.getSchemaFingerprint(), - oldSegmentMetadata.getNumRows()) + oldSegmentMetadata.getNumRows(), + upgradedFromSegmentId + ) ); } @@ -2266,7 +2285,8 @@ private Set insertSegments( Set segments, @Nullable SegmentSchemaMapping segmentSchemaMapping, Map upgradeSegmentMetadata, - Map newVersionForAppendToParent + Map newVersionForAppendToParent, + Map upgradedFromSegmentIdMap ) throws IOException { boolean shouldPersistSchema = shouldPersistSchema(segmentSchemaMapping); @@ -2302,7 +2322,8 @@ private Set insertSegments( .bind("version", segment.getVersion()) .bind("used", true) .bind("payload", jsonMapper.writeValueAsBytes(segment)) - .bind("used_status_last_updated", now); + .bind("used_status_last_updated", now) + .bind("upgraded_from_segment_id", upgradedFromSegmentIdMap.get(segment.getId().toString())); if (schemaPersistEnabled) { SegmentMetadata segmentMetadata = @@ -2449,9 +2470,9 @@ private String buildSqlToInsertSegments() { String insertStatement = "INSERT INTO %1$s (id, dataSource, created_date, start, %2$send%2$s," - + " partitioned, version, used, payload, used_status_last_updated %3$s) " + + " partitioned, version, used, payload, used_status_last_updated, upgraded_from_segment_id %3$s) " + "VALUES (:id, :dataSource, :created_date, :start, :end," - + " :partitioned, :version, :used, :payload, :used_status_last_updated %4$s)"; + + " :partitioned, :version, :used, :payload, :used_status_last_updated, :upgraded_from_segment_id %4$s)"; if (schemaPersistEnabled) { return StringUtils.format( @@ -2923,6 +2944,87 @@ public int deleteUpgradeSegmentsForTask(final String taskId) ); } + @Override + public Map retrieveUpgradedFromSegmentIds( + final String dataSource, + final Set segmentIds + ) + { + if (segmentIds.isEmpty()) { + return Collections.emptyMap(); + } + + final List segmentIdList = ImmutableList.copyOf(segmentIds); + final String sql = StringUtils.format( + "SELECT id, upgraded_from_segment_id FROM %s WHERE dataSource = :dataSource %s", + dbTables.getSegmentsTable(), + SqlSegmentsMetadataQuery.getParameterizedInConditionForColumn("id", segmentIdList) + ); + final Map upgradedFromSegmentIds = new HashMap<>(); + connector.retryWithHandle( + handle -> { + Query> query = handle.createQuery(sql) + .bind("dataSource", dataSource); + SqlSegmentsMetadataQuery.bindColumnValuesToQueryWithInCondition("id", segmentIdList, query); + return query.map((index, r, ctx) -> { + final String id = r.getString(1); + final String upgradedFromSegmentId = r.getString(2); + if (upgradedFromSegmentId != null) { + upgradedFromSegmentIds.put(id, upgradedFromSegmentId); + } + return null; + }).list(); + } + ); + return upgradedFromSegmentIds; + } + + @Override + public Map> retrieveUpgradedToSegmentIds( + final String dataSource, + final Set segmentIds + ) + { + if (segmentIds.isEmpty()) { + return Collections.emptyMap(); + } + + final List upgradedFromSegmentIdList = ImmutableList.copyOf(segmentIds); + final String sql = StringUtils.format( + "SELECT id, upgraded_from_segment_id FROM %s WHERE dataSource = :dataSource %s", + dbTables.getSegmentsTable(), + SqlSegmentsMetadataQuery.getParameterizedInConditionForColumn( + "upgraded_from_segment_id", + upgradedFromSegmentIdList + ) + ); + final Map> upgradedToSegmentIds = new HashMap<>(); + retrieveSegmentsById(dataSource, segmentIds) + .stream() + .map(DataSegment::getId) + .map(SegmentId::toString) + .forEach(id -> upgradedToSegmentIds.computeIfAbsent(id, k -> new HashSet<>()).add(id)); + connector.retryWithHandle( + handle -> { + Query> query = handle.createQuery(sql) + .bind("dataSource", dataSource); + SqlSegmentsMetadataQuery.bindColumnValuesToQueryWithInCondition( + "upgraded_from_segment_id", + upgradedFromSegmentIdList, + query + ); + return query.map((index, r, ctx) -> { + final String upgradedToId = r.getString(1); + final String id = r.getString(2); + upgradedToSegmentIds.computeIfAbsent(id, k -> new HashSet<>()) + .add(upgradedToId); + return null; + }).list(); + } + ); + return upgradedToSegmentIds; + } + private static class PendingSegmentsRecord { private final String sequenceName; diff --git a/server/src/main/java/org/apache/druid/metadata/PendingSegmentRecord.java b/server/src/main/java/org/apache/druid/metadata/PendingSegmentRecord.java index bfbaad18ef1a..f117fe7f28bf 100644 --- a/server/src/main/java/org/apache/druid/metadata/PendingSegmentRecord.java +++ b/server/src/main/java/org/apache/druid/metadata/PendingSegmentRecord.java @@ -40,7 +40,10 @@ *
  • id -> id (Unique identifier for pending segment)
  • *
  • sequence_name -> sequenceName (sequence name used for segment allocation)
  • *
  • sequence_prev_id -> sequencePrevId (previous segment id used for segment allocation)
  • - *
  • upgraded_from_segment_id -> upgradedFromSegmentId (Id of the root segment from which this was upgraded)
  • + *
  • upgraded_from_segment_id -> upgradedFromSegmentId + * (ID of the segment which was upgraded to create the current segment. + * If the former was itself created as a result of an upgrade, then this ID + * must refer to the original non-upgraded segment in the hierarchy.)
  • *
  • task_allocator_id -> taskAllocatorId (Associates a task / task group / replica group with the pending segment)
  • * */ diff --git a/server/src/main/java/org/apache/druid/metadata/SQLMetadataConnector.java b/server/src/main/java/org/apache/druid/metadata/SQLMetadataConnector.java index 2d315d19fc8b..dc87b9fc2fd4 100644 --- a/server/src/main/java/org/apache/druid/metadata/SQLMetadataConnector.java +++ b/server/src/main/java/org/apache/druid/metadata/SQLMetadataConnector.java @@ -587,6 +587,8 @@ protected void alterSegmentTable() Map columnNameTypes = new HashMap<>(); columnNameTypes.put("used_status_last_updated", "VARCHAR(255)"); + columnNameTypes.put("upgraded_from_segment_id", "VARCHAR(255)"); + if (centralizedDatasourceSchemaConfig.isEnabled()) { columnNameTypes.put("schema_fingerprint", "VARCHAR(255)"); columnNameTypes.put("num_rows", "BIGINT"); @@ -619,6 +621,14 @@ protected void alterSegmentTable() } alterTable(tableName, alterCommands); + + final Set createdIndexSet = getIndexOnTable(tableName); + createIndex( + tableName, + StringUtils.format("idx_%1$s_datasource_upgraded_from_segment_id", tableName), + ImmutableList.of("dataSource", "upgraded_from_segment_id"), + createdIndexSet + ); } @Override diff --git a/server/src/main/java/org/apache/druid/metadata/SqlSegmentsMetadataQuery.java b/server/src/main/java/org/apache/druid/metadata/SqlSegmentsMetadataQuery.java index f14cc9950505..fc1c84a70371 100644 --- a/server/src/main/java/org/apache/druid/metadata/SqlSegmentsMetadataQuery.java +++ b/server/src/main/java/org/apache/druid/metadata/SqlSegmentsMetadataQuery.java @@ -286,7 +286,7 @@ private List retrieveSegmentBatchById( if (includeSchemaInfo) { final Query> query = handle.createQuery( StringUtils.format( - "SELECT payload, used, schema_fingerprint, num_rows FROM %s WHERE dataSource = :dataSource %s", + "SELECT payload, used, schema_fingerprint, num_rows, upgraded_from_segment_id FROM %s WHERE dataSource = :dataSource %s", dbTables.getSegmentsTable(), getParameterizedInConditionForColumn("id", segmentIds) ) ); @@ -306,7 +306,8 @@ private List retrieveSegmentBatchById( null, r.getBoolean(2), schemaFingerprint, - numRows + numRows, + r.getString(5) ); } ) @@ -314,7 +315,7 @@ private List retrieveSegmentBatchById( } else { final Query> query = handle.createQuery( StringUtils.format( - "SELECT payload, used FROM %s WHERE dataSource = :dataSource %s", + "SELECT payload, used, upgraded_from_segment_id FROM %s WHERE dataSource = :dataSource %s", dbTables.getSegmentsTable(), getParameterizedInConditionForColumn("id", segmentIds) ) ); @@ -331,7 +332,8 @@ private List retrieveSegmentBatchById( null, r.getBoolean(2), null, - null + null, + r.getString(3) ) ) .iterator(); @@ -864,6 +866,7 @@ private ResultIterator getDataSegmentPlusResultIterator(Query values, final SQLStatement query diff --git a/server/src/main/java/org/apache/druid/server/http/DataSegmentPlus.java b/server/src/main/java/org/apache/druid/server/http/DataSegmentPlus.java index 9841e09a1a72..bfda5cbf3ad4 100644 --- a/server/src/main/java/org/apache/druid/server/http/DataSegmentPlus.java +++ b/server/src/main/java/org/apache/druid/server/http/DataSegmentPlus.java @@ -36,6 +36,8 @@ *
  • {@link DataSegmentPlus#createdDate} - The time when the segment was created.
  • *
  • {@link DataSegmentPlus#usedStatusLastUpdatedDate} - The time when the segments * used status was last updated.
  • + *
  • {@link DataSegmentPlus#upgradedFromSegmentId} - The segment id to which the same load spec originally belonged. + * Load specs can be shared as a result of segment version upgrades.
  • * *

    * This class closely resembles the row structure of the {@link MetadataStorageTablesConfig#getSegmentsTable()}. @@ -53,6 +55,9 @@ public class DataSegmentPlus private final String schemaFingerprint; private final Long numRows; + @Nullable + private final String upgradedFromSegmentId; + @JsonCreator public DataSegmentPlus( @JsonProperty("dataSegment") final DataSegment dataSegment, @@ -60,7 +65,8 @@ public DataSegmentPlus( @JsonProperty("usedStatusLastUpdatedDate") @Nullable final DateTime usedStatusLastUpdatedDate, @JsonProperty("used") @Nullable final Boolean used, @JsonProperty("schemaFingerprint") @Nullable final String schemaFingerprint, - @JsonProperty("numRows") @Nullable final Long numRows + @JsonProperty("numRows") @Nullable final Long numRows, + @JsonProperty("upgradedFromSegmentId") @Nullable final String upgradedFromSegmentId ) { this.dataSegment = dataSegment; @@ -69,6 +75,7 @@ public DataSegmentPlus( this.used = used; this.schemaFingerprint = schemaFingerprint; this.numRows = numRows; + this.upgradedFromSegmentId = upgradedFromSegmentId; } @Nullable @@ -112,6 +119,13 @@ public Long getNumRows() return numRows; } + @Nullable + @JsonProperty + public String getUpgradedFromSegmentId() + { + return upgradedFromSegmentId; + } + @Override public boolean equals(Object o) { @@ -127,7 +141,8 @@ public boolean equals(Object o) && Objects.equals(usedStatusLastUpdatedDate, that.getUsedStatusLastUpdatedDate()) && Objects.equals(used, that.getUsed()) && Objects.equals(schemaFingerprint, that.getSchemaFingerprint()) - && Objects.equals(numRows, that.getNumRows()); + && Objects.equals(numRows, that.getNumRows()) + && Objects.equals(upgradedFromSegmentId, that.getUpgradedFromSegmentId()); } @Override @@ -139,7 +154,8 @@ public int hashCode() usedStatusLastUpdatedDate, used, schemaFingerprint, - numRows + numRows, + upgradedFromSegmentId ); } @@ -153,6 +169,7 @@ public String toString() ", used=" + getUsed() + ", schemaFingerprint=" + getSchemaFingerprint() + ", numRows=" + getNumRows() + + ", upgradedFromSegmentId=" + getUpgradedFromSegmentId() + '}'; } } diff --git a/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java b/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java index 222c1ece89fb..f352d5e2609d 100644 --- a/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java +++ b/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java @@ -138,8 +138,10 @@ public void testCommitAppendSegments() final String v1 = "2023-01-01"; final String v2 = "2023-01-02"; final String v3 = "2023-01-03"; + final String alreadyUpgradedVersion = "2023-02-01"; final String lockVersion = "2024-01-01"; + final String taskAllocatorId = "appendTask"; final String replaceTaskId = "replaceTask1"; final ReplaceTaskLock replaceLock = new ReplaceTaskLock( replaceTaskId, @@ -148,6 +150,7 @@ public void testCommitAppendSegments() ); final Set appendSegments = new HashSet<>(); + final List pendingSegmentsForTask = new ArrayList<>(); final Set expectedSegmentsToUpgrade = new HashSet<>(); for (int i = 0; i < 10; i++) { final DataSegment segment = createSegment( @@ -157,6 +160,31 @@ public void testCommitAppendSegments() ); appendSegments.add(segment); expectedSegmentsToUpgrade.add(segment); + // Add the same segment + pendingSegmentsForTask.add( + new PendingSegmentRecord( + SegmentIdWithShardSpec.fromDataSegment(segment), + v1, + segment.getId().toString(), + null, + taskAllocatorId + ) + ); + // Add upgraded pending segment + pendingSegmentsForTask.add( + new PendingSegmentRecord( + new SegmentIdWithShardSpec( + DS.WIKI, + Intervals.of("2023-01-01/2023-02-01"), + alreadyUpgradedVersion, + new NumberedShardSpec(i, 0) + ), + alreadyUpgradedVersion, + segment.getId().toString(), + segment.getId().toString(), + taskAllocatorId + ) + ); } for (int i = 0; i < 10; i++) { @@ -167,6 +195,31 @@ public void testCommitAppendSegments() ); appendSegments.add(segment); expectedSegmentsToUpgrade.add(segment); + // Add the same segment + pendingSegmentsForTask.add( + new PendingSegmentRecord( + SegmentIdWithShardSpec.fromDataSegment(segment), + v2, + segment.getId().toString(), + null, + taskAllocatorId + ) + ); + // Add upgraded pending segment + pendingSegmentsForTask.add( + new PendingSegmentRecord( + new SegmentIdWithShardSpec( + DS.WIKI, + Intervals.of("2023-01-01/2023-02-01"), + alreadyUpgradedVersion, + new NumberedShardSpec(10 + i, 0) + ), + alreadyUpgradedVersion, + segment.getId().toString(), + segment.getId().toString(), + taskAllocatorId + ) + ); } for (int i = 0; i < 10; i++) { @@ -176,23 +229,78 @@ public void testCommitAppendSegments() new LinearShardSpec(i) ); appendSegments.add(segment); + // Add the same segment + pendingSegmentsForTask.add( + new PendingSegmentRecord( + SegmentIdWithShardSpec.fromDataSegment(segment), + v3, + segment.getId().toString(), + null, + taskAllocatorId + ) + ); + // Add upgraded pending segment + pendingSegmentsForTask.add( + new PendingSegmentRecord( + new SegmentIdWithShardSpec( + DS.WIKI, + Intervals.of("2023-01-01/2023-02-01"), + alreadyUpgradedVersion, + new NumberedShardSpec(20 + i, 0) + ), + alreadyUpgradedVersion, + segment.getId().toString(), + segment.getId().toString(), + taskAllocatorId + ) + ); } + derbyConnector.retryWithHandle( + handle -> coordinator.insertPendingSegmentsIntoMetastore(handle, pendingSegmentsForTask, DS.WIKI, false) + ); + final Map segmentToReplaceLock = expectedSegmentsToUpgrade.stream() .collect(Collectors.toMap(s -> s, s -> replaceLock)); // Commit the segment and verify the results SegmentPublishResult commitResult - = coordinator.commitAppendSegments(appendSegments, segmentToReplaceLock, "append", null); + = coordinator.commitAppendSegments(appendSegments, segmentToReplaceLock, taskAllocatorId, null); Assert.assertTrue(commitResult.isSuccess()); - Assert.assertEquals(appendSegments, commitResult.getSegments()); - // Verify the segments present in the metadata store - Assert.assertEquals( - appendSegments, - ImmutableSet.copyOf(retrieveUsedSegments(derbyConnectorRule.metadataTablesConfigSupplier().get())) + Set allCommittedSegments + = new HashSet<>(retrieveUsedSegments(derbyConnectorRule.metadataTablesConfigSupplier().get())); + Map upgradedFromSegmentIdMap = coordinator.retrieveUpgradedFromSegmentIds( + DS.WIKI, + allCommittedSegments.stream().map(DataSegment::getId).map(SegmentId::toString).collect(Collectors.toSet()) ); + // Verify the segments present in the metadata store + Assert.assertTrue(allCommittedSegments.containsAll(appendSegments)); + for (DataSegment segment : appendSegments) { + Assert.assertNull(upgradedFromSegmentIdMap.get(segment.getId().toString())); + } + allCommittedSegments.removeAll(appendSegments); + + // Verify the commit of upgraded pending segments + Assert.assertEquals(appendSegments.size(), allCommittedSegments.size()); + Map segmentMap = new HashMap<>(); + for (DataSegment segment : appendSegments) { + segmentMap.put(segment.getId().toString(), segment); + } + for (DataSegment segment : allCommittedSegments) { + for (PendingSegmentRecord pendingSegmentRecord : pendingSegmentsForTask) { + if (pendingSegmentRecord.getId().asSegmentId().toString().equals(segment.getId().toString())) { + DataSegment upgradedFromSegment = segmentMap.get(pendingSegmentRecord.getUpgradedFromSegmentId()); + Assert.assertNotNull(upgradedFromSegment); + Assert.assertEquals(segment.getLoadSpec(), upgradedFromSegment.getLoadSpec()); + Assert.assertEquals( + pendingSegmentRecord.getUpgradedFromSegmentId(), + upgradedFromSegmentIdMap.get(segment.getId().toString()) + ); + } + } + } // Verify entries in the segment task lock table final Set expectedUpgradeSegmentIds @@ -290,12 +398,24 @@ public void testCommitReplaceSegments() retrieveUsedSegmentIds(derbyConnectorRule.metadataTablesConfigSupplier().get()).size() ); - final Set usedSegments = new HashSet<>(retrieveUsedSegments(derbyConnectorRule.metadataTablesConfigSupplier().get())); + final Set usedSegments + = new HashSet<>(retrieveUsedSegments(derbyConnectorRule.metadataTablesConfigSupplier().get())); + + final Map upgradedFromSegmentIdMap = coordinator.retrieveUpgradedFromSegmentIds( + "foo", + usedSegments.stream().map(DataSegment::getId).map(SegmentId::toString).collect(Collectors.toSet()) + ); Assert.assertTrue(usedSegments.containsAll(segmentsAppendedWithReplaceLock)); + for (DataSegment appendSegment : segmentsAppendedWithReplaceLock) { + Assert.assertNull(upgradedFromSegmentIdMap.get(appendSegment.getId().toString())); + } usedSegments.removeAll(segmentsAppendedWithReplaceLock); Assert.assertTrue(usedSegments.containsAll(replacingSegments)); + for (DataSegment replaceSegment : replacingSegments) { + Assert.assertNull(upgradedFromSegmentIdMap.get(replaceSegment.getId().toString())); + } usedSegments.removeAll(replacingSegments); Assert.assertEquals(segmentsAppendedWithReplaceLock.size(), usedSegments.size()); @@ -303,6 +423,10 @@ public void testCommitReplaceSegments() boolean hasBeenCarriedForward = false; for (DataSegment appendedSegment : segmentsAppendedWithReplaceLock) { if (appendedSegment.getLoadSpec().equals(segmentReplicaWithNewVersion.getLoadSpec())) { + Assert.assertEquals( + appendedSegment.getId().toString(), + upgradedFromSegmentIdMap.get(segmentReplicaWithNewVersion.getId().toString()) + ); hasBeenCarriedForward = true; break; } @@ -3300,4 +3424,63 @@ public void testRetrieveUnusedSegmentsForExactIntervalAndVersion() unusedSegmentIdsForIntervalAndVersion.get(0) ); } + + @Test + public void testRetrieveUpgradedFromSegmentIds() + { + final String datasource = defaultSegment.getDataSource(); + final Map upgradedFromSegmentIdMap = new HashMap<>(); + upgradedFromSegmentIdMap.put(defaultSegment2.getId().toString(), defaultSegment.getId().toString()); + insertUsedSegments(ImmutableSet.of(defaultSegment, defaultSegment2), upgradedFromSegmentIdMap); + coordinator.markSegmentsAsUnusedWithinInterval(datasource, Intervals.ETERNITY); + upgradedFromSegmentIdMap.clear(); + upgradedFromSegmentIdMap.put(defaultSegment3.getId().toString(), defaultSegment.getId().toString()); + insertUsedSegments(ImmutableSet.of(defaultSegment3, defaultSegment4), upgradedFromSegmentIdMap); + + Map expected = new HashMap<>(); + expected.put(defaultSegment2.getId().toString(), defaultSegment.getId().toString()); + expected.put(defaultSegment3.getId().toString(), defaultSegment.getId().toString()); + + Set segmentIds = new HashSet<>(); + segmentIds.add(defaultSegment.getId().toString()); + segmentIds.add(defaultSegment2.getId().toString()); + segmentIds.add(defaultSegment3.getId().toString()); + segmentIds.add(defaultSegment4.getId().toString()); + Assert.assertEquals( + expected, + coordinator.retrieveUpgradedFromSegmentIds(datasource, segmentIds) + ); + } + + @Test + public void testRetrieveUpgradedToSegmentIds() + { + final String datasource = defaultSegment.getDataSource(); + final Map upgradedFromSegmentIdMap = new HashMap<>(); + upgradedFromSegmentIdMap.put(defaultSegment2.getId().toString(), defaultSegment.getId().toString()); + insertUsedSegments(ImmutableSet.of(defaultSegment, defaultSegment2), upgradedFromSegmentIdMap); + coordinator.markSegmentsAsUnusedWithinInterval(datasource, Intervals.ETERNITY); + upgradedFromSegmentIdMap.clear(); + upgradedFromSegmentIdMap.put(defaultSegment3.getId().toString(), defaultSegment.getId().toString()); + insertUsedSegments(ImmutableSet.of(defaultSegment3, defaultSegment4), upgradedFromSegmentIdMap); + + Map> expected = new HashMap<>(); + expected.put(defaultSegment.getId().toString(), new HashSet<>()); + expected.get(defaultSegment.getId().toString()).add(defaultSegment.getId().toString()); + expected.get(defaultSegment.getId().toString()).add(defaultSegment2.getId().toString()); + expected.get(defaultSegment.getId().toString()).add(defaultSegment3.getId().toString()); + + Set upgradedIds = new HashSet<>(); + upgradedIds.add(defaultSegment.getId().toString()); + Assert.assertEquals( + expected, + coordinator.retrieveUpgradedToSegmentIds(datasource, upgradedIds) + ); + } + + private void insertUsedSegments(Set segments, Map upgradedFromSegmentIdMap) + { + final String table = derbyConnectorRule.metadataTablesConfigSupplier().get().getSegmentsTable(); + insertUsedSegments(segments, upgradedFromSegmentIdMap, derbyConnector, table, mapper); + } } diff --git a/server/src/test/java/org/apache/druid/metadata/IndexerSqlMetadataStorageCoordinatorTestBase.java b/server/src/test/java/org/apache/druid/metadata/IndexerSqlMetadataStorageCoordinatorTestBase.java index ce0e06860583..2076e5ffa461 100644 --- a/server/src/test/java/org/apache/druid/metadata/IndexerSqlMetadataStorageCoordinatorTestBase.java +++ b/server/src/test/java/org/apache/druid/metadata/IndexerSqlMetadataStorageCoordinatorTestBase.java @@ -58,6 +58,7 @@ import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.Set; import java.util.concurrent.atomic.AtomicLong; import java.util.function.Function; @@ -322,6 +323,8 @@ protected DataSegment createSegment(Interval interval, String version, ShardSpec .version(version) .shardSpec(shardSpec) .size(100) + // hash to get a unique load spec as segmentId has not yet been generated + .loadSpec(ImmutableMap.of("hash", Objects.hash(interval, version, shardSpec))) .build(); } @@ -559,4 +562,50 @@ protected void insertIntoUpgradeSegmentsTable(Map } ); } + + public static void insertUsedSegments( + Set dataSegments, + Map upgradedFromSegmentIdMap, + SQLMetadataConnector connector, + String table, + ObjectMapper jsonMapper + ) + { + connector.retryWithHandle( + handle -> { + PreparedBatch preparedBatch = handle.prepareBatch( + StringUtils.format( + "INSERT INTO %1$s (id, dataSource, created_date, start, %2$send%2$s, partitioned, version," + + " used, payload, used_status_last_updated, upgraded_from_segment_id) " + + "VALUES (:id, :dataSource, :created_date, :start, :end, :partitioned, :version," + + " :used, :payload, :used_status_last_updated, :upgraded_from_segment_id)", + table, + connector.getQuoteString() + ) + ); + for (DataSegment segment : dataSegments) { + String id = segment.getId().toString(); + preparedBatch.add() + .bind("id", id) + .bind("dataSource", segment.getDataSource()) + .bind("created_date", DateTimes.nowUtc().toString()) + .bind("start", segment.getInterval().getStart().toString()) + .bind("end", segment.getInterval().getEnd().toString()) + .bind("partitioned", !(segment.getShardSpec() instanceof NoneShardSpec)) + .bind("version", segment.getVersion()) + .bind("used", true) + .bind("payload", jsonMapper.writeValueAsBytes(segment)) + .bind("used_status_last_updated", DateTimes.nowUtc().toString()) + .bind("upgraded_from_segment_id", upgradedFromSegmentIdMap.get(segment.getId().toString())); + } + + final int[] affectedRows = preparedBatch.execute(); + final boolean succeeded = Arrays.stream(affectedRows).allMatch(eachAffectedRows -> eachAffectedRows == 1); + if (!succeeded) { + throw new ISE("Failed to publish segments to DB"); + } + return true; + } + ); + } } diff --git a/server/src/test/java/org/apache/druid/server/http/DataSegmentPlusTest.java b/server/src/test/java/org/apache/druid/server/http/DataSegmentPlusTest.java index 0f20fc96bdcc..b963f4337081 100644 --- a/server/src/test/java/org/apache/druid/server/http/DataSegmentPlusTest.java +++ b/server/src/test/java/org/apache/druid/server/http/DataSegmentPlusTest.java @@ -100,6 +100,7 @@ public void testSerde() throws JsonProcessingException usedStatusLastUpdatedDate, null, null, + null, null ); @@ -108,7 +109,7 @@ public void testSerde() throws JsonProcessingException JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT ); - Assert.assertEquals(6, objectMap.size()); + Assert.assertEquals(7, objectMap.size()); final Map segmentObjectMap = MAPPER.readValue( MAPPER.writeValueAsString(segmentPlus.getDataSegment()), JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT diff --git a/server/src/test/java/org/apache/druid/server/http/MetadataResourceTest.java b/server/src/test/java/org/apache/druid/server/http/MetadataResourceTest.java index 4d6bbf5929be..9c52d639300c 100644 --- a/server/src/test/java/org/apache/druid/server/http/MetadataResourceTest.java +++ b/server/src/test/java/org/apache/druid/server/http/MetadataResourceTest.java @@ -77,7 +77,7 @@ public class MetadataResourceTest .toArray(new DataSegment[0]); private final List segmentsPlus = Arrays.stream(segments) - .map(s -> new DataSegmentPlus(s, DateTimes.nowUtc(), DateTimes.nowUtc(), null, null, null)) + .map(s -> new DataSegmentPlus(s, DateTimes.nowUtc(), DateTimes.nowUtc(), null, null, null, null)) .collect(Collectors.toList()); private HttpServletRequest request; private SegmentsMetadataManager segmentsMetadataManager; From 209f8a95468e8450e3d1365d7f0eda98a35732df Mon Sep 17 00:00:00 2001 From: Laksh Singla Date: Mon, 15 Jul 2024 15:00:17 +0530 Subject: [PATCH 3/7] Deserialize complex dimensions in group by queries to their respective types when reading from spilled files and cached results (#16620) Like #16511, but for keys that have been spilled or cached during the grouping process --- .../druid/jackson/AggregatorsModule.java | 15 +- .../apache/druid/query/QueryToolChest.java | 21 ++- .../DataSourceQueryQueryToolChest.java | 8 + .../groupby/GroupByQueryQueryToolChest.java | 106 +++++++---- .../query/groupby/epinephelinae/Grouper.java | 11 ++ .../epinephelinae/RowBasedGrouperHelper.java | 135 ++++++++++++-- .../epinephelinae/RowBasedKeySerdeHelper.java | 5 + .../epinephelinae/SpillingGrouper.java | 2 +- .../SegmentMetadataQueryQueryToolChest.java | 11 ++ .../search/SearchQueryQueryToolChest.java | 10 ++ .../TimeBoundaryQueryQueryToolChest.java | 12 ++ .../TimeseriesQueryQueryToolChest.java | 12 ++ .../query/topn/TopNQueryQueryToolChest.java | 13 +- .../ObjectStrategyComplexTypeStrategy.java | 6 +- .../druid/segment/column/TypeStrategies.java | 30 ++++ .../druid/segment/column/TypeStrategy.java | 2 +- .../aggregation/AggregationTestHelper.java | 8 +- .../ComplexDimensionGroupByQueryTest.java | 164 ++++++++++++++++++ .../GroupByQueryQueryToolChestTest.java | 94 ++++++++-- .../query/groupby/GroupByQueryRunnerTest.java | 5 +- .../druid/client/CachingClusteredClient.java | 2 +- .../druid/client/CachingQueryRunner.java | 2 +- .../query/ResultLevelCachingQueryRunner.java | 2 +- .../druid/client/CachingQueryRunnerTest.java | 15 +- 24 files changed, 594 insertions(+), 97 deletions(-) create mode 100644 processing/src/test/java/org/apache/druid/query/groupby/ComplexDimensionGroupByQueryTest.java diff --git a/processing/src/main/java/org/apache/druid/jackson/AggregatorsModule.java b/processing/src/main/java/org/apache/druid/jackson/AggregatorsModule.java index f7aca511e17d..200e6fcb1394 100644 --- a/processing/src/main/java/org/apache/druid/jackson/AggregatorsModule.java +++ b/processing/src/main/java/org/apache/druid/jackson/AggregatorsModule.java @@ -83,6 +83,16 @@ public AggregatorsModule() { super("AggregatorFactories"); + registerComplexMetricsAndSerde(); + + setMixInAnnotation(AggregatorFactory.class, AggregatorFactoryMixin.class); + setMixInAnnotation(PostAggregator.class, PostAggregatorMixin.class); + + addSerializer(DoubleMeanHolder.class, DoubleMeanHolder.Serializer.INSTANCE); + } + + public static void registerComplexMetricsAndSerde() + { ComplexMetrics.registerSerde(HyperUniquesSerde.TYPE_NAME, new HyperUniquesSerde()); ComplexMetrics.registerSerde(PreComputedHyperUniquesSerde.TYPE_NAME, new PreComputedHyperUniquesSerde()); ComplexMetrics.registerSerde( @@ -102,11 +112,6 @@ public AggregatorsModule() SerializablePairLongLongComplexMetricSerde.TYPE_NAME, new SerializablePairLongLongComplexMetricSerde() ); - - setMixInAnnotation(AggregatorFactory.class, AggregatorFactoryMixin.class); - setMixInAnnotation(PostAggregator.class, PostAggregatorMixin.class); - - addSerializer(DoubleMeanHolder.class, DoubleMeanHolder.Serializer.INSTANCE); } @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type") diff --git a/processing/src/main/java/org/apache/druid/query/QueryToolChest.java b/processing/src/main/java/org/apache/druid/query/QueryToolChest.java index b0678f247c9d..fa394beec43a 100644 --- a/processing/src/main/java/org/apache/druid/query/QueryToolChest.java +++ b/processing/src/main/java/org/apache/druid/query/QueryToolChest.java @@ -251,19 +251,36 @@ public Function makePostComputeManipulatorFn(QueryType q */ public abstract TypeReference getResultTypeReference(); + /** + * Like {@link #getCacheStrategy(Query, ObjectMapper)} but the caller doesn't supply the object mapper for deserializing + * and converting the cached data to desired type. It's upto the individual implementations to decide the appropriate action in that case. + * It can either throw an exception outright or decide if the query requires the object mapper for proper downstream processing and + * work with the generic java types if not. + *

    + * @deprecated Use {@link #getCacheStrategy(Query, ObjectMapper)} instead + */ + @Deprecated + @Nullable + public CacheStrategy getCacheStrategy(QueryType query) + { + return null; + } + /** * Returns a CacheStrategy to be used to load data into the cache and remove it from the cache. *

    * This is optional. If it returns null, caching is effectively disabled for the query. * * @param query The query whose results might be cached + * @param mapper Object mapper to convert the deserialized generic java objects to desired types. It can be nullable + * to preserve backward compatibility. * @param The type of object that will be stored in the cache * @return A CacheStrategy that can be used to populate and read from the Cache */ @Nullable - public CacheStrategy getCacheStrategy(QueryType query) + public CacheStrategy getCacheStrategy(QueryType query, @Nullable ObjectMapper mapper) { - return null; + return getCacheStrategy(query); } /** diff --git a/processing/src/main/java/org/apache/druid/query/datasourcemetadata/DataSourceQueryQueryToolChest.java b/processing/src/main/java/org/apache/druid/query/datasourcemetadata/DataSourceQueryQueryToolChest.java index dbe8922f2e9b..21fb5c53afcc 100644 --- a/processing/src/main/java/org/apache/druid/query/datasourcemetadata/DataSourceQueryQueryToolChest.java +++ b/processing/src/main/java/org/apache/druid/query/datasourcemetadata/DataSourceQueryQueryToolChest.java @@ -20,6 +20,7 @@ package org.apache.druid.query.datasourcemetadata; import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Function; import com.google.common.base.Functions; import com.google.inject.Inject; @@ -38,6 +39,7 @@ import org.apache.druid.query.context.ResponseContext; import org.apache.druid.timeline.LogicalSegment; +import javax.annotation.Nullable; import java.util.List; import java.util.stream.Collectors; @@ -119,4 +121,10 @@ public CacheStrategy getCacheStrategy(DataSourceMetadataQuery query) { return null; } + + @Override + public CacheStrategy getCacheStrategy(DataSourceMetadataQuery query, @Nullable ObjectMapper mapper) + { + return null; + } } diff --git a/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryQueryToolChest.java b/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryQueryToolChest.java index b19b479c26d1..d69e09c9ff0b 100644 --- a/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryQueryToolChest.java +++ b/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryQueryToolChest.java @@ -77,8 +77,10 @@ import org.apache.druid.segment.column.NullableTypeStrategy; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.nested.StructuredData; import org.joda.time.DateTime; +import javax.annotation.Nullable; import java.io.Closeable; import java.io.IOException; import java.util.ArrayList; @@ -471,7 +473,7 @@ public void serialize( // Deserializer that can deserialize either array- or map-based rows. final JsonDeserializer deserializer = new JsonDeserializer() { - final Class[] dimensionClasses = createDimensionClasses(); + final Class[] dimensionClasses = createDimensionClasses(query); boolean containsComplexDimensions = query.getDimensions() .stream() .anyMatch( @@ -524,30 +526,6 @@ public ResultRow deserialize(final JsonParser jp, final DeserializationContext c return ResultRow.of(objectArray); } } - - private Class[] createDimensionClasses() - { - final List queryDimensions = query.getDimensions(); - final Class[] classes = new Class[queryDimensions.size()]; - for (int i = 0; i < queryDimensions.size(); ++i) { - final ColumnType dimensionOutputType = queryDimensions.get(i).getOutputType(); - if (dimensionOutputType.is(ValueType.COMPLEX)) { - NullableTypeStrategy nullableTypeStrategy = dimensionOutputType.getNullableStrategy(); - if (!nullableTypeStrategy.groupable()) { - throw DruidException.defensive( - "Ungroupable dimension [%s] with type [%s] found in the query.", - queryDimensions.get(i).getDimension(), - dimensionOutputType - ); - } - classes[i] = nullableTypeStrategy.getClazz(); - } else { - classes[i] = Object.class; - } - } - return classes; - } - }; class GroupByResultRowModule extends SimpleModule @@ -597,9 +575,32 @@ public Sequence run(QueryPlus queryPlus, ResponseContext r ); } + @Nullable @Override - public CacheStrategy getCacheStrategy(final GroupByQuery query) + public CacheStrategy getCacheStrategy(GroupByQuery query) { + return getCacheStrategy(query, null); + } + + @Override + public CacheStrategy getCacheStrategy( + final GroupByQuery query, + @Nullable final ObjectMapper mapper + ) + { + + for (DimensionSpec dimension : query.getDimensions()) { + if (dimension.getOutputType().is(ValueType.COMPLEX) && !dimension.getOutputType().equals(ColumnType.NESTED_DATA)) { + if (mapper == null) { + throw DruidException.defensive( + "Cannot deserialize complex dimension of type[%s] from result cache if object mapper is not provided", + dimension.getOutputType().getComplexTypeName() + ); + } + } + } + final Class[] dimensionClasses = createDimensionClasses(query); + return new CacheStrategy() { private static final byte CACHE_STRATEGY_VERSION = 0x1; @@ -726,13 +727,29 @@ public ResultRow apply(Object input) int dimPos = 0; while (dimsIter.hasNext() && results.hasNext()) { final DimensionSpec dimensionSpec = dimsIter.next(); - - // Must convert generic Jackson-deserialized type into the proper type. - resultRow.set( - dimensionStart + dimPos, - DimensionHandlerUtils.convertObjectToType(results.next(), dimensionSpec.getOutputType()) - ); - + final Object dimensionObject = results.next(); + final Object dimensionObjectCasted; + + final ColumnType outputType = dimensionSpec.getOutputType(); + + // Must convert generic Jackson-deserialized type into the proper type. The downstream functions expect the + // dimensions to be of appropriate types for further processing like merging and comparing. + if (outputType.is(ValueType.COMPLEX)) { + // Json columns can interpret generic data objects appropriately, hence they are wrapped as is in StructuredData. + // They don't need to converted them from Object.class to StructuredData.class using object mapper as that is an + // expensive operation that will be wasteful. + if (outputType.equals(ColumnType.NESTED_DATA)) { + dimensionObjectCasted = StructuredData.wrap(dimensionObject); + } else { + dimensionObjectCasted = mapper.convertValue(dimensionObject, dimensionClasses[dimPos]); + } + } else { + dimensionObjectCasted = DimensionHandlerUtils.convertObjectToType( + dimensionObject, + dimensionSpec.getOutputType() + ); + } + resultRow.set(dimensionStart + dimPos, dimensionObjectCasted); dimPos++; } @@ -861,4 +878,27 @@ private static BitSet extractionsToRewrite(GroupByQuery query) return retVal; } + + private static Class[] createDimensionClasses(final GroupByQuery query) + { + final List queryDimensions = query.getDimensions(); + final Class[] classes = new Class[queryDimensions.size()]; + for (int i = 0; i < queryDimensions.size(); ++i) { + final ColumnType dimensionOutputType = queryDimensions.get(i).getOutputType(); + if (dimensionOutputType.is(ValueType.COMPLEX)) { + NullableTypeStrategy nullableTypeStrategy = dimensionOutputType.getNullableStrategy(); + if (!nullableTypeStrategy.groupable()) { + throw DruidException.defensive( + "Ungroupable dimension [%s] with type [%s] found in the query.", + queryDimensions.get(i).getDimension(), + dimensionOutputType + ); + } + classes[i] = nullableTypeStrategy.getClazz(); + } else { + classes[i] = Object.class; + } + } + return classes; + } } diff --git a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/Grouper.java b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/Grouper.java index 591624f1ab80..0f3faedb707c 100644 --- a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/Grouper.java +++ b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/Grouper.java @@ -19,6 +19,7 @@ package org.apache.druid.query.groupby.epinephelinae; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Preconditions; import org.apache.druid.java.util.common.parsers.CloseableIterator; import org.apache.druid.query.aggregation.AggregatorFactory; @@ -232,6 +233,16 @@ interface KeySerde */ BufferComparator bufferComparatorWithAggregators(AggregatorFactory[] aggregatorFactories, int[] aggregatorOffsets); + /** + * Decorates the object mapper enabling it to read and write query results' grouping keys. It is used by the + * {@link SpillingGrouper} to preserve the types of the dimensions after serializing and deserializing them on the + * spilled files. + */ + default ObjectMapper decorateObjectMapper(ObjectMapper spillMapper) + { + return spillMapper; + } + /** * Reset the keySerde to its initial state. After this method is called, {@link #readFromByteBuffer} * and {@link #bufferComparator()} may no longer work properly on previously-serialized keys. diff --git a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/RowBasedGrouperHelper.java b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/RowBasedGrouperHelper.java index 491c28d41427..da8a0e046230 100644 --- a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/RowBasedGrouperHelper.java +++ b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/RowBasedGrouperHelper.java @@ -19,9 +19,14 @@ package org.apache.druid.query.groupby.epinephelinae; -import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonValue; +import com.fasterxml.jackson.core.JsonParser; +import com.fasterxml.jackson.core.JsonToken; +import com.fasterxml.jackson.core.ObjectCodec; +import com.fasterxml.jackson.databind.DeserializationContext; +import com.fasterxml.jackson.databind.JsonDeserializer; import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.module.SimpleModule; import com.google.common.base.Preconditions; import com.google.common.base.Supplier; import com.google.common.primitives.Ints; @@ -84,6 +89,7 @@ import javax.annotation.Nullable; import java.io.Closeable; +import java.io.IOException; import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Arrays; @@ -666,22 +672,6 @@ public static class RowBasedKey this.key = key; } - @JsonCreator - public static RowBasedKey fromJsonArray(final Object[] key) - { - // Type info is lost during serde: - // Floats may be deserialized as doubles, Longs may be deserialized as integers, convert them back - for (int i = 0; i < key.length; i++) { - if (key[i] instanceof Integer) { - key[i] = ((Integer) key[i]).longValue(); - } else if (key[i] instanceof Double) { - key[i] = ((Double) key[i]).floatValue(); - } - } - - return new RowBasedKey(key); - } - @JsonValue public Object[] getKey() { @@ -1371,6 +1361,65 @@ public Grouper.BufferComparator bufferComparatorWithAggregators( ); } + @Override + public ObjectMapper decorateObjectMapper(ObjectMapper spillMapper) + { + + final JsonDeserializer deserializer = new JsonDeserializer() + { + @Override + public RowBasedKey deserialize( + JsonParser jp, + DeserializationContext deserializationContext + ) throws IOException + { + if (!jp.isExpectedStartArrayToken()) { + throw DruidException.defensive("Expected array start token, received [%s]", jp.getCurrentToken()); + } + jp.nextToken(); + + final ObjectCodec codec = jp.getCodec(); + final int timestampAdjustment = includeTimestamp ? 1 : 0; + final int dimsToRead = timestampAdjustment + serdeHelpers.length; + int dimsReadSoFar = 0; + final Object[] objects = new Object[dimsToRead]; + + if (includeTimestamp) { + DruidException.conditionalDefensive( + jp.currentToken() != JsonToken.END_ARRAY, + "Unexpected end of array when deserializing timestamp from the spilled files" + ); + objects[dimsReadSoFar] = codec.readValue(jp, Long.class); + + ++dimsReadSoFar; + jp.nextToken(); + } + + while (jp.currentToken() != JsonToken.END_ARRAY) { + objects[dimsReadSoFar] = + codec.readValue(jp, serdeHelpers[dimsReadSoFar - timestampAdjustment].getClazz()); + + ++dimsReadSoFar; + jp.nextToken(); + } + + return new RowBasedKey(objects); + } + }; + + class SpillModule extends SimpleModule + { + public SpillModule() + { + addDeserializer(RowBasedKey.class, deserializer); + } + } + + final ObjectMapper newObjectMapper = spillMapper.copy(); + newObjectMapper.registerModule(new SpillModule()); + return newObjectMapper; + } + @Override public void reset() { @@ -1588,6 +1637,7 @@ private class GenericRowBasedKeySerdeHelper extends DictionaryBuildingSingleValu { final BufferComparator bufferComparator; final String columnTypeName; + final Class clazz; final List dictionary; final Object2IntMap reverseDictionary; @@ -1613,6 +1663,7 @@ public GenericRowBasedKeySerdeHelper( dictionary.get(lhsBuffer.getInt(lhsPosition + keyBufferPosition)), dictionary.get(rhsBuffer.getInt(rhsPosition + keyBufferPosition)) ); + clazz = columnType.getNullableStrategy().getClazz(); } // Asserts that we don't entertain any complex types without a typename, to prevent intermixing dictionaries of @@ -1645,6 +1696,12 @@ public Object2IntMap getReverseDictionary() { return reverseDictionary; } + + @Override + public Class getClazz() + { + return clazz; + } } @@ -1726,6 +1783,14 @@ public Object2IntMap getReverseDictionary() { return reverseDictionary; } + + @Override + public Class getClazz() + { + // Jackson deserializes Object[] containing longs to Object[] containing string if Object[].class is returned + // Therefore we are using Object.class + return Object.class; + } } private class ArrayStringRowBasedKeySerdeHelper extends DictionaryBuildingSingleValuedRowBasedKeySerdeHelper @@ -1770,6 +1835,12 @@ public Object2IntMap getReverseDictionary() { return reverseStringArrayDictionary; } + + @Override + public Class getClazz() + { + return Object[].class; + } } private abstract class AbstractStringRowBasedKeySerdeHelper implements RowBasedKeySerdeHelper @@ -1819,6 +1890,12 @@ public BufferComparator getBufferComparator() { return bufferComparator; } + + @Override + public Class getClazz() + { + return String.class; + } } private class DynamicDictionaryStringRowBasedKeySerdeHelper extends AbstractStringRowBasedKeySerdeHelper @@ -1937,6 +2014,12 @@ public BufferComparator getBufferComparator() { return bufferComparator; } + + @Override + public Class getClazz() + { + return Long.class; + } } private class FloatRowBasedKeySerdeHelper implements RowBasedKeySerdeHelper @@ -1982,6 +2065,12 @@ public BufferComparator getBufferComparator() { return bufferComparator; } + + @Override + public Class getClazz() + { + return Float.class; + } } private class DoubleRowBasedKeySerdeHelper implements RowBasedKeySerdeHelper @@ -2027,6 +2116,12 @@ public BufferComparator getBufferComparator() { return bufferComparator; } + + @Override + public Class getClazz() + { + return Double.class; + } } // This class is only used when SQL compatible null handling is enabled. @@ -2082,6 +2177,12 @@ public BufferComparator getBufferComparator() { return comparator; } + + @Override + public Class getClazz() + { + return delegate.getClazz(); + } } } } diff --git a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/RowBasedKeySerdeHelper.java b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/RowBasedKeySerdeHelper.java index 1cb29d23bc06..71372ca238ba 100644 --- a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/RowBasedKeySerdeHelper.java +++ b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/RowBasedKeySerdeHelper.java @@ -65,4 +65,9 @@ interface RowBasedKeySerdeHelper * Return a {@link BufferComparator} to compare keys stored in ByteBuffer. */ BufferComparator getBufferComparator(); + + /** + * Returns the expected class of the key which used to deserialize the objects correctly from the spilled files. + */ + Class getClazz(); } diff --git a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/SpillingGrouper.java b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/SpillingGrouper.java index 4e9b96102a16..d8a7760c11de 100644 --- a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/SpillingGrouper.java +++ b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/SpillingGrouper.java @@ -152,7 +152,7 @@ public SpillingGrouper( } this.aggregatorFactories = aggregatorFactories; this.temporaryStorage = temporaryStorage; - this.spillMapper = spillMapper; + this.spillMapper = keySerde.decorateObjectMapper(spillMapper); this.spillingAllowed = spillingAllowed; this.sortHasNonGroupingFields = sortHasNonGroupingFields; } diff --git a/processing/src/main/java/org/apache/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java b/processing/src/main/java/org/apache/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java index 912ecb1ac322..fd8d7e7009c9 100644 --- a/processing/src/main/java/org/apache/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java +++ b/processing/src/main/java/org/apache/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java @@ -20,6 +20,7 @@ package org.apache.druid.query.metadata; import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Function; import com.google.common.base.Functions; @@ -62,6 +63,7 @@ import org.joda.time.DateTime; import org.joda.time.Interval; +import javax.annotation.Nullable; import java.util.ArrayList; import java.util.Arrays; import java.util.Comparator; @@ -184,6 +186,15 @@ public TypeReference getResultTypeReference() @Override public CacheStrategy getCacheStrategy(final SegmentMetadataQuery query) + { + return getCacheStrategy(query, null); + } + + @Override + public CacheStrategy getCacheStrategy( + final SegmentMetadataQuery query, + @Nullable final ObjectMapper objectMapper + ) { return new CacheStrategy() { diff --git a/processing/src/main/java/org/apache/druid/query/search/SearchQueryQueryToolChest.java b/processing/src/main/java/org/apache/druid/query/search/SearchQueryQueryToolChest.java index b390cd83a58d..c15e1d0d99c4 100644 --- a/processing/src/main/java/org/apache/druid/query/search/SearchQueryQueryToolChest.java +++ b/processing/src/main/java/org/apache/druid/query/search/SearchQueryQueryToolChest.java @@ -20,6 +20,7 @@ package org.apache.druid.query.search; import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Function; import com.google.common.base.Functions; @@ -124,6 +125,15 @@ public TypeReference> getResultTypeReference() @Override public CacheStrategy, Object, SearchQuery> getCacheStrategy(final SearchQuery query) + { + return getCacheStrategy(query, null); + } + + @Override + public CacheStrategy, Object, SearchQuery> getCacheStrategy( + final SearchQuery query, + @Nullable final ObjectMapper objectMapper + ) { return new CacheStrategy, Object, SearchQuery>() diff --git a/processing/src/main/java/org/apache/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java b/processing/src/main/java/org/apache/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java index 9087dd26a885..eab5e0f5abcf 100644 --- a/processing/src/main/java/org/apache/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java +++ b/processing/src/main/java/org/apache/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java @@ -20,6 +20,7 @@ package org.apache.druid.query.timeboundary; import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Function; import com.google.common.base.Functions; @@ -47,6 +48,7 @@ import org.apache.druid.segment.column.RowSignature; import org.apache.druid.timeline.LogicalSegment; +import javax.annotation.Nullable; import java.nio.ByteBuffer; import java.util.Comparator; import java.util.List; @@ -163,6 +165,16 @@ public TypeReference> getResultTypeReference() @Override public CacheStrategy, Object, TimeBoundaryQuery> getCacheStrategy(final TimeBoundaryQuery query) + { + return getCacheStrategy(query, null); + } + + + @Override + public CacheStrategy, Object, TimeBoundaryQuery> getCacheStrategy( + final TimeBoundaryQuery query, + @Nullable final ObjectMapper objectMapper + ) { return new CacheStrategy, Object, TimeBoundaryQuery>() { diff --git a/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryQueryToolChest.java b/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryQueryToolChest.java index 17a2f8be956b..67c36fe76030 100644 --- a/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryQueryToolChest.java +++ b/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryQueryToolChest.java @@ -20,6 +20,7 @@ package org.apache.druid.query.timeseries; import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Function; import com.google.common.base.Preconditions; @@ -65,6 +66,7 @@ import org.apache.druid.segment.column.RowSignature; import org.joda.time.DateTime; +import javax.annotation.Nullable; import java.io.Closeable; import java.util.ArrayList; import java.util.Collections; @@ -276,6 +278,16 @@ public TypeReference> getResultTypeReference() @Override public CacheStrategy, Object, TimeseriesQuery> getCacheStrategy(final TimeseriesQuery query) + { + return getCacheStrategy(query, null); + } + + + @Override + public CacheStrategy, Object, TimeseriesQuery> getCacheStrategy( + final TimeseriesQuery query, + @Nullable final ObjectMapper objectMapper + ) { return new CacheStrategy, Object, TimeseriesQuery>() { diff --git a/processing/src/main/java/org/apache/druid/query/topn/TopNQueryQueryToolChest.java b/processing/src/main/java/org/apache/druid/query/topn/TopNQueryQueryToolChest.java index 25a4284aa427..21bc336438ac 100644 --- a/processing/src/main/java/org/apache/druid/query/topn/TopNQueryQueryToolChest.java +++ b/processing/src/main/java/org/apache/druid/query/topn/TopNQueryQueryToolChest.java @@ -20,6 +20,7 @@ package org.apache.druid.query.topn; import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Function; import com.google.common.collect.Iterables; @@ -64,6 +65,7 @@ import org.apache.druid.segment.column.RowSignature; import org.joda.time.DateTime; +import javax.annotation.Nullable; import java.io.Closeable; import java.util.ArrayList; import java.util.Collections; @@ -268,9 +270,18 @@ public TypeReference> getResultTypeReference() return TYPE_REFERENCE; } + @Nullable + @Override + public CacheStrategy, Object, TopNQuery> getCacheStrategy(TopNQuery query) + { + return getCacheStrategy(query, null); + } @Override - public CacheStrategy, Object, TopNQuery> getCacheStrategy(final TopNQuery query) + public CacheStrategy, Object, TopNQuery> getCacheStrategy( + final TopNQuery query, + @Nullable final ObjectMapper objectMapper + ) { return new CacheStrategy, Object, TopNQuery>() { diff --git a/processing/src/main/java/org/apache/druid/segment/column/ObjectStrategyComplexTypeStrategy.java b/processing/src/main/java/org/apache/druid/segment/column/ObjectStrategyComplexTypeStrategy.java index b274e55282ea..f80a1cdcf8dc 100644 --- a/processing/src/main/java/org/apache/druid/segment/column/ObjectStrategyComplexTypeStrategy.java +++ b/processing/src/main/java/org/apache/druid/segment/column/ObjectStrategyComplexTypeStrategy.java @@ -123,7 +123,7 @@ public boolean groupable() public int hashCode(T o) { if (hashStrategy == null) { - throw DruidException.defensive("hashStrategy not provided"); + throw DruidException.defensive("Type [%s] is not groupable", typeSignature.asTypeString()); } return hashStrategy.hashCode(o); } @@ -132,7 +132,7 @@ public int hashCode(T o) public boolean equals(T a, T b) { if (hashStrategy == null) { - throw DruidException.defensive("hashStrategy not provided"); + throw DruidException.defensive("Type [%s] is not groupable", typeSignature.asTypeString()); } return hashStrategy.equals(a, b); } @@ -141,7 +141,7 @@ public boolean equals(T a, T b) public Class getClazz() { if (clazz == null) { - throw DruidException.defensive("hashStrategy not provided"); + throw DruidException.defensive("Type [%s] is not groupable", typeSignature.asTypeString()); } return clazz; } diff --git a/processing/src/main/java/org/apache/druid/segment/column/TypeStrategies.java b/processing/src/main/java/org/apache/druid/segment/column/TypeStrategies.java index bae29179b4d5..7ac8def99ec1 100644 --- a/processing/src/main/java/org/apache/druid/segment/column/TypeStrategies.java +++ b/processing/src/main/java/org/apache/druid/segment/column/TypeStrategies.java @@ -299,6 +299,12 @@ public boolean equals(Long a, Long b) { return a.equals(b); } + + @Override + public Class getClazz() + { + return Long.class; + } } /** @@ -368,6 +374,12 @@ public boolean equals(Float a, Float b) { return a.equals(b); } + + @Override + public Class getClazz() + { + return Float.class; + } } /** @@ -438,6 +450,12 @@ public boolean equals(Double a, Double b) { return a.equals(b); } + + @Override + public Class getClazz() + { + return Double.class; + } } /** @@ -519,6 +537,12 @@ public boolean equals(String a, String b) { return a.equals(b); } + + @Override + public Class getClazz() + { + return String.class; + } } /** @@ -664,5 +688,11 @@ public boolean equals(@Nullable Object[] a, @Nullable Object[] b) return false; } } + + @Override + public Class getClazz() + { + return Object[].class; + } } } diff --git a/processing/src/main/java/org/apache/druid/segment/column/TypeStrategy.java b/processing/src/main/java/org/apache/druid/segment/column/TypeStrategy.java index c5cff1a0b2f2..075fceca473f 100644 --- a/processing/src/main/java/org/apache/druid/segment/column/TypeStrategy.java +++ b/processing/src/main/java/org/apache/druid/segment/column/TypeStrategy.java @@ -225,6 +225,6 @@ default boolean equals(T a, T b) */ default Class getClazz() { - throw DruidException.defensive("Not implemented. It is only implemented for complex dimensions which are groupable()"); + throw DruidException.defensive("Not implemented. Check groupable() first"); } } diff --git a/processing/src/test/java/org/apache/druid/query/aggregation/AggregationTestHelper.java b/processing/src/test/java/org/apache/druid/query/aggregation/AggregationTestHelper.java index 526a62c813fb..2ad9f90148a8 100644 --- a/processing/src/test/java/org/apache/druid/query/aggregation/AggregationTestHelper.java +++ b/processing/src/test/java/org/apache/druid/query/aggregation/AggregationTestHelper.java @@ -766,7 +766,7 @@ public Object accumulate(Object accumulated, Object in) String resultStr = mapper.writer().writeValueAsString(yielder); List resultRows = Lists.transform( - readQueryResultArrayFromString(resultStr), + readQueryResultArrayFromString(resultStr, queryPlus.getQuery()), toolChest.makePreComputeManipulatorFn( queryPlus.getQuery(), MetricManipulatorFns.deserializing() @@ -798,11 +798,13 @@ public Object accumulate(Object accumulated, Object in) }; } - private List readQueryResultArrayFromString(String str) throws Exception + private List readQueryResultArrayFromString(String str, Query query) throws Exception { List result = new ArrayList(); - JsonParser jp = mapper.getFactory().createParser(str); + ObjectMapper decoratedMapper = toolChest.decorateObjectMapper(mapper, query); + + JsonParser jp = decoratedMapper.getFactory().createParser(str); if (jp.nextToken() != JsonToken.START_ARRAY) { throw new IAE("not an array [%s]", str); diff --git a/processing/src/test/java/org/apache/druid/query/groupby/ComplexDimensionGroupByQueryTest.java b/processing/src/test/java/org/apache/druid/query/groupby/ComplexDimensionGroupByQueryTest.java new file mode 100644 index 000000000000..bc1ecbb0ddc8 --- /dev/null +++ b/processing/src/test/java/org/apache/druid/query/groupby/ComplexDimensionGroupByQueryTest.java @@ -0,0 +1,164 @@ +/* + * 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.query.groupby; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.granularity.Granularities; +import org.apache.druid.java.util.common.guava.Sequence; +import org.apache.druid.java.util.common.guava.Sequences; +import org.apache.druid.query.QueryContexts; +import org.apache.druid.query.aggregation.AggregationTestHelper; +import org.apache.druid.query.aggregation.CountAggregatorFactory; +import org.apache.druid.query.aggregation.SerializablePairLongString; +import org.apache.druid.query.aggregation.SerializablePairLongStringComplexMetricSerde; +import org.apache.druid.query.dimension.DefaultDimensionSpec; +import org.apache.druid.segment.RowBasedSegment; +import org.apache.druid.segment.Segment; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.column.RowSignature; +import org.apache.druid.timeline.SegmentId; +import org.junit.Assert; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.Map; + +@RunWith(Parameterized.class) +public class ComplexDimensionGroupByQueryTest +{ + private final QueryContexts.Vectorize vectorize; + private final AggregationTestHelper helper; + private final List segments; + + @Rule + public final TemporaryFolder tempFolder = new TemporaryFolder(); + + public ComplexDimensionGroupByQueryTest(GroupByQueryConfig config, String vectorize) + { + this.vectorize = QueryContexts.Vectorize.fromString(vectorize); + this.helper = AggregationTestHelper.createGroupByQueryAggregationTestHelper( + Collections.emptyList(), + config, + tempFolder + ); + Sequence rows = Sequences.simple( + ImmutableList.of( + new Object[]{new SerializablePairLongString(1L, "abc")}, + new Object[]{new SerializablePairLongString(1L, "abc")}, + new Object[]{new SerializablePairLongString(1L, "def")}, + new Object[]{new SerializablePairLongString(1L, "abc")}, + new Object[]{new SerializablePairLongString(1L, "ghi")}, + new Object[]{new SerializablePairLongString(1L, "def")}, + new Object[]{new SerializablePairLongString(1L, "abc")}, + new Object[]{new SerializablePairLongString(1L, "pqr")}, + new Object[]{new SerializablePairLongString(1L, "xyz")}, + new Object[]{new SerializablePairLongString(1L, "foo")}, + new Object[]{new SerializablePairLongString(1L, "bar")} + ) + ); + RowSignature rowSignature = RowSignature.builder() + .add( + "pair", + ColumnType.ofComplex(SerializablePairLongStringComplexMetricSerde.TYPE_NAME) + ) + .build(); + + this.segments = Collections.singletonList( + new RowBasedSegment<>( + SegmentId.dummy("dummy"), + rows, + columnName -> { + final int columnNumber = rowSignature.indexOf(columnName); + return row -> columnNumber >= 0 ? row[columnNumber] : null; + }, + rowSignature + ) + ); + } + + @Parameterized.Parameters(name = "config = {0}, vectorize = {1}") + public static Collection constructorFeeder() + { + final List constructors = new ArrayList<>(); + for (GroupByQueryConfig config : GroupByQueryRunnerTest.testConfigs()) { + for (String vectorize : new String[]{"false", "force"}) { + constructors.add(new Object[]{config, vectorize}); + } + } + return constructors; + } + + public Map getContext() + { + return ImmutableMap.of( + QueryContexts.VECTORIZE_KEY, vectorize.toString(), + QueryContexts.VECTORIZE_VIRTUAL_COLUMNS_KEY, "true" + ); + } + + @Test + public void testGroupByOnPairClass() + { + GroupByQuery groupQuery = GroupByQuery.builder() + .setDataSource("test_datasource") + .setGranularity(Granularities.ALL) + .setInterval(Intervals.ETERNITY) + .setDimensions(new DefaultDimensionSpec( + "pair", + "pair", + ColumnType.ofComplex(SerializablePairLongStringComplexMetricSerde.TYPE_NAME) + )) + .setAggregatorSpecs(new CountAggregatorFactory("count")) + .setContext(getContext()) + .build(); + + if (vectorize == QueryContexts.Vectorize.FORCE) { + // Cannot vectorize group by on complex dimension + Assert.assertThrows( + RuntimeException.class, + () -> helper.runQueryOnSegmentsObjs(segments, groupQuery).toList() + ); + } else { + List resultRows = helper.runQueryOnSegmentsObjs(segments, groupQuery).toList(); + + Assert.assertArrayEquals( + new ResultRow[]{ + ResultRow.of(new SerializablePairLongString(1L, "abc"), 4L), + ResultRow.of(new SerializablePairLongString(1L, "bar"), 1L), + ResultRow.of(new SerializablePairLongString(1L, "def"), 2L), + ResultRow.of(new SerializablePairLongString(1L, "foo"), 1L), + ResultRow.of(new SerializablePairLongString(1L, "ghi"), 1L), + ResultRow.of(new SerializablePairLongString(1L, "pqr"), 1L), + ResultRow.of(new SerializablePairLongString(1L, "xyz"), 1L) + }, + resultRows.toArray() + ); + } + } +} diff --git a/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryQueryToolChestTest.java b/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryQueryToolChestTest.java index f43bbce9d978..7279ca938bd8 100644 --- a/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryQueryToolChestTest.java +++ b/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryQueryToolChestTest.java @@ -33,6 +33,7 @@ import org.apache.druid.collections.StupidPool; import org.apache.druid.common.config.NullHandling; import org.apache.druid.data.input.Row; +import org.apache.druid.jackson.AggregatorsModule; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.Granularities; @@ -97,6 +98,7 @@ public class GroupByQueryQueryToolChestTest extends InitializedNullHandlingTest public static void setUpClass() { NullHandling.initializeForTests(); + AggregatorsModule.registerComplexMetricsAndSerde(); } @Test @@ -130,11 +132,13 @@ public void testResultLevelCacheKeyWithPostAggregate() .setGranularity(QueryRunnerTestHelper.DAY_GRAN) .build(); + final ObjectMapper mapper = TestHelper.makeJsonMapper(); + final CacheStrategy strategy1 = - new GroupByQueryQueryToolChest(null, null).getCacheStrategy(query1); + new GroupByQueryQueryToolChest(null, null).getCacheStrategy(query1, mapper); final CacheStrategy strategy2 = - new GroupByQueryQueryToolChest(null, null).getCacheStrategy(query2); + new GroupByQueryQueryToolChest(null, null).getCacheStrategy(query2, mapper); Assert.assertTrue(Arrays.equals(strategy1.computeCacheKey(query1), strategy2.computeCacheKey(query2))); Assert.assertFalse(Arrays.equals( @@ -190,11 +194,12 @@ public void testResultLevelCacheKeyWithLimitSpec() ) .build(); + final ObjectMapper mapper = TestHelper.makeJsonMapper(); final CacheStrategy strategy1 = - new GroupByQueryQueryToolChest(null, null).getCacheStrategy(query1); + new GroupByQueryQueryToolChest(null, null).getCacheStrategy(query1, mapper); final CacheStrategy strategy2 = - new GroupByQueryQueryToolChest(null, null).getCacheStrategy(query2); + new GroupByQueryQueryToolChest(null, null).getCacheStrategy(query2, mapper); Assert.assertTrue(Arrays.equals(strategy1.computeCacheKey(query1), strategy2.computeCacheKey(query2))); Assert.assertFalse(Arrays.equals( @@ -252,11 +257,12 @@ public void testResultLevelCacheKeyWithHavingSpec() .setHavingSpec(new GreaterThanHavingSpec(QueryRunnerTestHelper.UNIQUE_METRIC, 10)) .build(); + final ObjectMapper mapper = TestHelper.makeJsonMapper(); final CacheStrategy strategy1 = - new GroupByQueryQueryToolChest(null, null).getCacheStrategy(query1); + new GroupByQueryQueryToolChest(null, null).getCacheStrategy(query1, mapper); final CacheStrategy strategy2 = - new GroupByQueryQueryToolChest(null, null).getCacheStrategy(query2); + new GroupByQueryQueryToolChest(null, null).getCacheStrategy(query2, mapper); Assert.assertTrue(Arrays.equals(strategy1.computeCacheKey(query1), strategy2.computeCacheKey(query2))); Assert.assertFalse(Arrays.equals( @@ -336,11 +342,12 @@ public void testResultLevelCacheKeyWithAndHavingSpec() .setHavingSpec(andHavingSpec2) .build(); + final ObjectMapper mapper = TestHelper.makeJsonMapper(); final CacheStrategy strategy1 = - new GroupByQueryQueryToolChest(null, null).getCacheStrategy(query1); + new GroupByQueryQueryToolChest(null, null).getCacheStrategy(query1, mapper); final CacheStrategy strategy2 = - new GroupByQueryQueryToolChest(null, null).getCacheStrategy(query2); + new GroupByQueryQueryToolChest(null, null).getCacheStrategy(query2, mapper); Assert.assertTrue(Arrays.equals(strategy1.computeCacheKey(query1), strategy2.computeCacheKey(query2))); Assert.assertFalse(Arrays.equals( @@ -427,11 +434,12 @@ public void testResultLevelCacheKeyWithHavingDimFilterHavingSpec() .setHavingSpec(havingSpec2) .build(); + final ObjectMapper mapper = TestHelper.makeJsonMapper(); final CacheStrategy strategy1 = - new GroupByQueryQueryToolChest(null, null).getCacheStrategy(query1); + new GroupByQueryQueryToolChest(null, null).getCacheStrategy(query1, mapper); final CacheStrategy strategy2 = - new GroupByQueryQueryToolChest(null, null).getCacheStrategy(query2); + new GroupByQueryQueryToolChest(null, null).getCacheStrategy(query2, mapper); Assert.assertTrue(Arrays.equals(strategy1.computeCacheKey(query1), strategy2.computeCacheKey(query2))); Assert.assertFalse(Arrays.equals( @@ -490,11 +498,12 @@ public void testResultLevelCacheKeyWithSubTotalsSpec() )) .build(); + final ObjectMapper mapper = TestHelper.makeJsonMapper(); final CacheStrategy strategy1 = - new GroupByQueryQueryToolChest(null, null).getCacheStrategy(query1); + new GroupByQueryQueryToolChest(null, null).getCacheStrategy(query1, mapper); final CacheStrategy strategy2 = - new GroupByQueryQueryToolChest(null, null).getCacheStrategy(query2); + new GroupByQueryQueryToolChest(null, null).getCacheStrategy(query2, mapper); Assert.assertTrue(Arrays.equals(strategy1.computeCacheKey(query1), strategy2.computeCacheKey(query2))); Assert.assertFalse(Arrays.equals( @@ -512,6 +521,48 @@ public void testCacheStrategy() throws Exception doTestCacheStrategy(ColumnType.LONG, 2L); } + @Test + public void testComplexDimensionCacheStrategy() throws IOException + { + final GroupByQuery query1 = GroupByQuery + .builder() + .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) + .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) + .setDimensions(ImmutableList.of( + new DefaultDimensionSpec( + "test", + "test", + ColumnType.ofComplex(SerializablePairLongStringComplexMetricSerde.TYPE_NAME) + ) + )) + .setAggregatorSpecs(QueryRunnerTestHelper.ROWS_COUNT) + .setGranularity(QueryRunnerTestHelper.DAY_GRAN) + .build(); + + ObjectMapper objectMapper = TestHelper.makeJsonMapper(); + + CacheStrategy strategy = + new GroupByQueryQueryToolChest(null, null).getCacheStrategy(query1, objectMapper); + + // test timestamps that result in integer size millis + final ResultRow result1 = ResultRow.of( + 123L, + new SerializablePairLongString(123L, "abc"), + 1 + ); + + Object preparedValue = strategy.prepareForSegmentLevelCache().apply(result1); + + Object fromCacheValue = objectMapper.readValue( + objectMapper.writeValueAsBytes(preparedValue), + strategy.getCacheObjectClazz() + ); + + ResultRow fromCacheResult = strategy.pullFromSegmentLevelCache().apply(fromCacheValue); + + Assert.assertEquals(result1, fromCacheResult); + } + @Test public void testMultiColumnCacheStrategy() throws Exception { @@ -538,8 +589,9 @@ public void testMultiColumnCacheStrategy() throws Exception .setGranularity(QueryRunnerTestHelper.DAY_GRAN) .build(); + final ObjectMapper mapper = TestHelper.makeJsonMapper(); CacheStrategy strategy = - new GroupByQueryQueryToolChest(null, null).getCacheStrategy(query1); + new GroupByQueryQueryToolChest(null, null).getCacheStrategy(query1, mapper); // test timestamps that result in integer size millis final ResultRow result1 = ResultRow.of( @@ -1054,8 +1106,9 @@ private void doTestCacheStrategy(final ColumnType valueType, final Object dimVal .setGranularity(QueryRunnerTestHelper.DAY_GRAN) .build(); + final ObjectMapper mapper = TestHelper.makeJsonMapper(); CacheStrategy strategy = - new GroupByQueryQueryToolChest(null, null).getCacheStrategy(query1); + new GroupByQueryQueryToolChest(null, null).getCacheStrategy(query1, mapper); // test timestamps that result in integer size millis final ResultRow result1 = ResultRow.of( @@ -1147,11 +1200,12 @@ public void testQueryCacheKeyWithLimitSpec() .setGranularity(QueryRunnerTestHelper.DAY_GRAN) .build(); + final ObjectMapper mapper = TestHelper.makeJsonMapper(); final CacheStrategy strategy1 = - new GroupByQueryQueryToolChest(null, null).getCacheStrategy(query1); + new GroupByQueryQueryToolChest(null, null).getCacheStrategy(query1, mapper); final CacheStrategy strategy2 = - new GroupByQueryQueryToolChest(null, null).getCacheStrategy(query2); + new GroupByQueryQueryToolChest(null, null).getCacheStrategy(query2, mapper); Assert.assertFalse(Arrays.equals(strategy1.computeCacheKey(query1), strategy2.computeCacheKey(query2))); Assert.assertFalse(Arrays.equals( @@ -1183,11 +1237,12 @@ public void testQueryCacheKeyWithLimitSpecPushDownUsingContext() .overrideContext(ImmutableMap.of(GroupByQueryConfig.CTX_KEY_APPLY_LIMIT_PUSH_DOWN, "false")) .build(); + final ObjectMapper mapper = TestHelper.makeJsonMapper(); final CacheStrategy strategy1 = - new GroupByQueryQueryToolChest(null, null).getCacheStrategy(query1); + new GroupByQueryQueryToolChest(null, null).getCacheStrategy(query1, mapper); final CacheStrategy strategy2 = - new GroupByQueryQueryToolChest(null, null).getCacheStrategy(query2); + new GroupByQueryQueryToolChest(null, null).getCacheStrategy(query2, mapper); Assert.assertFalse(Arrays.equals(strategy1.computeCacheKey(query1), strategy2.computeCacheKey(query2))); Assert.assertTrue( @@ -1245,7 +1300,8 @@ public String getFormatString() QueryRunnerTestHelper.NOOP_QUERYWATCHER ); final GroupByQueryQueryToolChest queryToolChest = new GroupByQueryQueryToolChest(groupingEngine, groupByResourcesReservationPool); - CacheStrategy cacheStrategy = queryToolChest.getCacheStrategy(query); + final ObjectMapper mapper = TestHelper.makeJsonMapper(); + CacheStrategy cacheStrategy = queryToolChest.getCacheStrategy(query, mapper); Assert.assertTrue( "result level cache on broker server for GroupByStrategyV2 should be enabled", cacheStrategy.isCacheable(query, false, false) diff --git a/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerTest.java b/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerTest.java index d4dc87341306..a5dbb49bca51 100644 --- a/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerTest.java +++ b/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerTest.java @@ -33,6 +33,7 @@ import org.apache.druid.common.config.NullHandling; import org.apache.druid.data.input.Row; import org.apache.druid.data.input.Rows; +import org.apache.druid.error.DruidException; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.HumanReadableBytes; import org.apache.druid.java.util.common.IAE; @@ -9965,7 +9966,6 @@ public void testGroupByLongColumn() @Test public void testGroupByComplexColumn() { - cannotVectorize(); GroupByQuery query = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) @@ -9979,7 +9979,8 @@ public void testGroupByComplexColumn() .setGranularity(QueryRunnerTestHelper.ALL_GRAN) .build(); - expectedException.expect(RuntimeException.class); + expectedException.expect(DruidException.class); + expectedException.expectMessage("Type [COMPLEX] is not groupable"); GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); } 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 7bcb4c2ce038..5fa34d6699d8 100644 --- a/server/src/main/java/org/apache/druid/client/CachingClusteredClient.java +++ b/server/src/main/java/org/apache/druid/client/CachingClusteredClient.java @@ -275,7 +275,7 @@ private class SpecificQueryRunnable this.responseContext = responseContext; this.query = queryPlus.getQuery(); this.toolChest = warehouse.getToolChest(query); - this.strategy = toolChest.getCacheStrategy(query); + this.strategy = toolChest.getCacheStrategy(query, objectMapper); this.dataSourceAnalysis = query.getDataSource().getAnalysis(); this.useCache = CacheUtil.isUseSegmentCache(query, strategy, cacheConfig, CacheUtil.ServerType.BROKER); diff --git a/server/src/main/java/org/apache/druid/client/CachingQueryRunner.java b/server/src/main/java/org/apache/druid/client/CachingQueryRunner.java index 9bb9f474dd9a..41d4bb4ea639 100644 --- a/server/src/main/java/org/apache/druid/client/CachingQueryRunner.java +++ b/server/src/main/java/org/apache/druid/client/CachingQueryRunner.java @@ -86,7 +86,7 @@ public CachingQueryRunner( public Sequence run(QueryPlus queryPlus, ResponseContext responseContext) { Query query = queryPlus.getQuery(); - final CacheStrategy strategy = toolChest.getCacheStrategy(query); + final CacheStrategy strategy = toolChest.getCacheStrategy(query, mapper); final boolean populateCache = canPopulateCache(query, strategy); final boolean useCache = canUseCache(query, strategy); diff --git a/server/src/main/java/org/apache/druid/query/ResultLevelCachingQueryRunner.java b/server/src/main/java/org/apache/druid/query/ResultLevelCachingQueryRunner.java index 182faba7a09c..0af6ebca3ede 100644 --- a/server/src/main/java/org/apache/druid/query/ResultLevelCachingQueryRunner.java +++ b/server/src/main/java/org/apache/druid/query/ResultLevelCachingQueryRunner.java @@ -73,7 +73,7 @@ public ResultLevelCachingQueryRunner( this.cache = cache; this.cacheConfig = cacheConfig; this.query = query; - this.strategy = queryToolChest.getCacheStrategy(query); + this.strategy = queryToolChest.getCacheStrategy(query, objectMapper); this.populateResultCache = CacheUtil.isPopulateResultCache( query, strategy, diff --git a/server/src/test/java/org/apache/druid/client/CachingQueryRunnerTest.java b/server/src/test/java/org/apache/druid/client/CachingQueryRunnerTest.java index a4375a61900a..7208ab2fc4ba 100644 --- a/server/src/test/java/org/apache/druid/client/CachingQueryRunnerTest.java +++ b/server/src/test/java/org/apache/druid/client/CachingQueryRunnerTest.java @@ -68,6 +68,7 @@ import org.apache.druid.query.topn.TopNQueryConfig; import org.apache.druid.query.topn.TopNQueryQueryToolChest; import org.apache.druid.query.topn.TopNResultValue; +import org.apache.druid.testing.InitializedNullHandlingTest; import org.easymock.EasyMock; import org.joda.time.DateTime; import org.junit.Assert; @@ -90,7 +91,7 @@ import java.util.concurrent.atomic.AtomicBoolean; @RunWith(Parameterized.class) -public class CachingQueryRunnerTest +public class CachingQueryRunnerTest extends InitializedNullHandlingTest { @Parameterized.Parameters(name = "numBackgroundThreads={0}") public static Iterable constructorFeeder() @@ -222,8 +223,8 @@ public void testNullCacheKeyPrefix() Cache cache = EasyMock.mock(Cache.class); EasyMock.replay(cache); CachingQueryRunner queryRunner = makeCachingQueryRunner(null, cache, toolchest, Sequences.empty()); - Assert.assertFalse(queryRunner.canPopulateCache(query, toolchest.getCacheStrategy(query))); - Assert.assertFalse(queryRunner.canUseCache(query, toolchest.getCacheStrategy(query))); + Assert.assertFalse(queryRunner.canPopulateCache(query, toolchest.getCacheStrategy(query, null))); + Assert.assertFalse(queryRunner.canUseCache(query, toolchest.getCacheStrategy(query, null))); queryRunner.run(QueryPlus.wrap(query)); EasyMock.verifyUnexpectedCalls(cache); } @@ -243,7 +244,7 @@ public void testNullStrategy() QueryToolChest toolchest = EasyMock.mock(QueryToolChest.class); Cache cache = EasyMock.mock(Cache.class); - EasyMock.expect(toolchest.getCacheStrategy(query)).andReturn(null); + EasyMock.expect(toolchest.getCacheStrategy(EasyMock.eq(query), EasyMock.anyObject())).andReturn(null); EasyMock.replay(cache, toolchest); CachingQueryRunner queryRunner = makeCachingQueryRunner(new byte[0], cache, toolchest, Sequences.empty()); Assert.assertFalse(queryRunner.canPopulateCache(query, null)); @@ -339,7 +340,7 @@ public void doMonitor(ServiceEmitter emitter) resultSeq ); - CacheStrategy cacheStrategy = toolchest.getCacheStrategy(query); + CacheStrategy cacheStrategy = toolchest.getCacheStrategy(query, null); Cache.NamedKey cacheKey = CacheUtil.computeSegmentCacheKey( CACHE_ID, SEGMENT_DESCRIPTOR, @@ -383,7 +384,7 @@ private void testUseCache( byte[] cacheKeyPrefix = RandomUtils.nextBytes(10); - CacheStrategy cacheStrategy = toolchest.getCacheStrategy(query); + CacheStrategy cacheStrategy = toolchest.getCacheStrategy(query, null); Cache.NamedKey cacheKey = CacheUtil.computeSegmentCacheKey( CACHE_ID, SEGMENT_DESCRIPTOR, @@ -399,7 +400,7 @@ private void testUseCache( toolchest, Sequences.empty() ); - Assert.assertTrue(runner.canUseCache(query, toolchest.getCacheStrategy(query))); + Assert.assertTrue(runner.canUseCache(query, toolchest.getCacheStrategy(query, null))); List results = runner.run(QueryPlus.wrap(query)).toList(); Assert.assertEquals(expectedResults.toString(), results.toString()); } From 64104533acf744bf3b596a9770beb17aedc374df Mon Sep 17 00:00:00 2001 From: Rishabh Singh <6513075+findingrish@users.noreply.github.com> Date: Mon, 15 Jul 2024 15:02:59 +0530 Subject: [PATCH 4/7] Enable querying entirely cold datasources (#16676) Add ability to query entirely cold datasources. --- .../client/coordinator/CoordinatorClient.java | 5 + .../coordinator/CoordinatorClientImpl.java | 13 + .../AbstractSegmentMetadataCache.java | 16 +- .../CoordinatorSegmentMetadataCache.java | 220 +++++++- .../server/coordinator/DruidCoordinator.java | 3 + .../coordinator/NoopCoordinatorClient.java | 6 + ...inatorSegmentDataCacheConcurrencyTest.java | 23 +- .../CoordinatorSegmentMetadataCacheTest.java | 479 +++++++++++++++++- .../schema/BrokerSegmentMetadataCache.java | 41 +- .../BrokerSegmentMetadataCacheTest.java | 89 +++- 10 files changed, 853 insertions(+), 42 deletions(-) diff --git a/server/src/main/java/org/apache/druid/client/coordinator/CoordinatorClient.java b/server/src/main/java/org/apache/druid/client/coordinator/CoordinatorClient.java index fdf16b2ac505..edeb16665ba4 100644 --- a/server/src/main/java/org/apache/druid/client/coordinator/CoordinatorClient.java +++ b/server/src/main/java/org/apache/druid/client/coordinator/CoordinatorClient.java @@ -69,4 +69,9 @@ public interface CoordinatorClient * Returns a new instance backed by a ServiceClient which follows the provided retryPolicy */ CoordinatorClient withRetryPolicy(ServiceRetryPolicy retryPolicy); + + /** + * Retrieves list of datasources with used segments. + */ + ListenableFuture> fetchDataSourcesWithUsedSegments(); } diff --git a/server/src/main/java/org/apache/druid/client/coordinator/CoordinatorClientImpl.java b/server/src/main/java/org/apache/druid/client/coordinator/CoordinatorClientImpl.java index 4c795c9dbd47..fc3deee12ed3 100644 --- a/server/src/main/java/org/apache/druid/client/coordinator/CoordinatorClientImpl.java +++ b/server/src/main/java/org/apache/druid/client/coordinator/CoordinatorClientImpl.java @@ -188,4 +188,17 @@ public CoordinatorClientImpl withRetryPolicy(ServiceRetryPolicy retryPolicy) { return new CoordinatorClientImpl(client.withRetryPolicy(retryPolicy), jsonMapper); } + + @Override + public ListenableFuture> fetchDataSourcesWithUsedSegments() + { + final String path = "/druid/coordinator/v1/metadata/datasources"; + return FutureUtils.transform( + client.asyncRequest( + new RequestBuilder(HttpMethod.GET, path), + new BytesFullResponseHandler() + ), + holder -> JacksonUtils.readValue(jsonMapper, holder.getContent(), new TypeReference>() {}) + ); + } } diff --git a/server/src/main/java/org/apache/druid/segment/metadata/AbstractSegmentMetadataCache.java b/server/src/main/java/org/apache/druid/segment/metadata/AbstractSegmentMetadataCache.java index 9cb2297db828..88e6ee97b983 100644 --- a/server/src/main/java/org/apache/druid/segment/metadata/AbstractSegmentMetadataCache.java +++ b/server/src/main/java/org/apache/druid/segment/metadata/AbstractSegmentMetadataCache.java @@ -200,7 +200,7 @@ public abstract class AbstractSegmentMetadataCache tables = new ConcurrentHashMap<>(); + protected final ConcurrentHashMap tables = new ConcurrentHashMap<>(); /** * This lock coordinates the access from multiple threads to those variables guarded by this lock. @@ -269,9 +269,10 @@ protected void cacheExecLoop() final boolean wasRecentFailure = DateTimes.utc(lastFailure) .plus(config.getMetadataRefreshPeriod()) .isAfterNow(); + if (isServerViewInitialized && !wasRecentFailure && - (!segmentsNeedingRefresh.isEmpty() || !dataSourcesNeedingRebuild.isEmpty()) && + shouldRefresh() && (refreshImmediately || nextRefresh < System.currentTimeMillis())) { // We need to do a refresh. Break out of the waiting loop. break; @@ -334,6 +335,7 @@ protected void cacheExecLoop() } } + /** * Lifecycle start method. */ @@ -361,6 +363,15 @@ public void refreshWaitCondition() throws InterruptedException // noop } + /** + * Refresh is executed only when there are segments or datasources needing refresh. + */ + @SuppressWarnings("GuardedBy") + protected boolean shouldRefresh() + { + return (!segmentsNeedingRefresh.isEmpty() || !dataSourcesNeedingRebuild.isEmpty()); + } + public void awaitInitialization() throws InterruptedException { initialized.await(); @@ -373,6 +384,7 @@ public void awaitInitialization() throws InterruptedException * * @return schema information for the given datasource */ + @Nullable public T getDatasource(String name) { return tables.get(name); diff --git a/server/src/main/java/org/apache/druid/segment/metadata/CoordinatorSegmentMetadataCache.java b/server/src/main/java/org/apache/druid/segment/metadata/CoordinatorSegmentMetadataCache.java index dad0b78ea778..3a4f548b8ba9 100644 --- a/server/src/main/java/org/apache/druid/segment/metadata/CoordinatorSegmentMetadataCache.java +++ b/server/src/main/java/org/apache/druid/segment/metadata/CoordinatorSegmentMetadataCache.java @@ -20,19 +20,27 @@ package org.apache.druid.segment.metadata; import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Supplier; +import com.google.common.collect.ImmutableMap; import com.google.common.collect.Maps; import com.google.common.collect.Sets; +import com.google.common.util.concurrent.ThreadFactoryBuilder; import com.google.inject.Inject; import org.apache.druid.client.CoordinatorServerView; +import org.apache.druid.client.ImmutableDruidDataSource; import org.apache.druid.client.InternalQueryConfig; import org.apache.druid.client.ServerView; import org.apache.druid.client.TimelineServerView; import org.apache.druid.guice.ManageLifecycle; import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.Stopwatch; +import org.apache.druid.java.util.common.StringUtils; 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.java.util.emitter.service.ServiceEmitter; +import org.apache.druid.metadata.SegmentsMetadataManagerConfig; +import org.apache.druid.metadata.SqlSegmentsMetadataManager; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.metadata.metadata.SegmentAnalysis; import org.apache.druid.segment.SchemaPayloadPlus; @@ -41,21 +49,30 @@ import org.apache.druid.segment.realtime.appenderator.SegmentSchemas; import org.apache.druid.server.QueryLifecycleFactory; import org.apache.druid.server.coordination.DruidServerMetadata; +import org.apache.druid.server.coordinator.loading.SegmentReplicaCount; +import org.apache.druid.server.coordinator.loading.SegmentReplicationStatus; import org.apache.druid.server.security.Escalator; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.SegmentId; import javax.annotation.Nullable; import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; import java.util.Collections; +import java.util.HashMap; import java.util.HashSet; import java.util.LinkedHashMap; import java.util.List; import java.util.Map; import java.util.Optional; import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentSkipListMap; +import java.util.concurrent.Executors; import java.util.concurrent.Future; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; /** @@ -71,17 +88,36 @@ *
    • Metadata query is executed only for those non-realtime segments for which the schema is not cached.
    • *
    • Datasources marked for refresh are then rebuilt.
    * + *

    + * It is important to note that the datasource schema returned in {@link #getDatasource} & {@link #getDataSourceInformationMap()} + * also includes columns from cold segments. + * Cold segments are processed in a separate thread and datasource schema from cold segments is separately stored. + *

    */ @ManageLifecycle public class CoordinatorSegmentMetadataCache extends AbstractSegmentMetadataCache { private static final EmittingLogger log = new EmittingLogger(CoordinatorSegmentMetadataCache.class); + private static final Long COLD_SCHEMA_PERIOD_MULTIPLIER = 3L; + private static final Long COLD_SCHEMA_SLOWNESS_THRESHOLD_MILLIS = TimeUnit.SECONDS.toMillis(50); private final SegmentMetadataCacheConfig config; private final ColumnTypeMergePolicy columnTypeMergePolicy; private final SegmentSchemaCache segmentSchemaCache; private final SegmentSchemaBackFillQueue segmentSchemaBackfillQueue; + private final SqlSegmentsMetadataManager sqlSegmentsMetadataManager; + private volatile SegmentReplicationStatus segmentReplicationStatus = null; + + // Datasource schema built from only cold segments. + private final ConcurrentHashMap coldSchemaTable = new ConcurrentHashMap<>(); + + // Period for cold schema processing thread. This is a multiple of segment polling period. + // Cold schema processing runs slower than the segment poll to save processing cost of all segments. + // The downside is a delay in columns from cold segment reflecting in the datasource schema. + private final long coldSchemaExecPeriodMillis; + private final ScheduledExecutorService coldSchemaExec; private @Nullable Future cacheExecFuture = null; + private @Nullable Future coldSchemaExecFuture = null; @Inject public CoordinatorSegmentMetadataCache( @@ -92,7 +128,9 @@ public CoordinatorSegmentMetadataCache( InternalQueryConfig internalQueryConfig, ServiceEmitter emitter, SegmentSchemaCache segmentSchemaCache, - SegmentSchemaBackFillQueue segmentSchemaBackfillQueue + SegmentSchemaBackFillQueue segmentSchemaBackfillQueue, + SqlSegmentsMetadataManager sqlSegmentsMetadataManager, + Supplier segmentsMetadataManagerConfigSupplier ) { super(queryLifecycleFactory, config, escalator, internalQueryConfig, emitter); @@ -100,6 +138,15 @@ public CoordinatorSegmentMetadataCache( this.columnTypeMergePolicy = config.getMetadataColumnTypeMergePolicy(); this.segmentSchemaCache = segmentSchemaCache; this.segmentSchemaBackfillQueue = segmentSchemaBackfillQueue; + this.sqlSegmentsMetadataManager = sqlSegmentsMetadataManager; + this.coldSchemaExecPeriodMillis = + segmentsMetadataManagerConfigSupplier.get().getPollDuration().getMillis() * COLD_SCHEMA_PERIOD_MULTIPLIER; + coldSchemaExec = Executors.newSingleThreadScheduledExecutor( + new ThreadFactoryBuilder() + .setNameFormat("DruidColdSchema-ScheduledExecutor-%d") + .setDaemon(false) + .build() + ); initServerViewTimelineCallback(serverView); } @@ -168,11 +215,15 @@ public void stop() { callbackExec.shutdownNow(); cacheExec.shutdownNow(); + coldSchemaExec.shutdownNow(); segmentSchemaCache.onLeaderStop(); segmentSchemaBackfillQueue.onLeaderStop(); if (cacheExecFuture != null) { cacheExecFuture.cancel(true); } + if (coldSchemaExecFuture != null) { + coldSchemaExecFuture.cancel(true); + } } public void onLeaderStart() @@ -181,6 +232,12 @@ public void onLeaderStart() try { segmentSchemaBackfillQueue.onLeaderStart(); cacheExecFuture = cacheExec.submit(this::cacheExecLoop); + coldSchemaExecFuture = coldSchemaExec.schedule( + this::coldDatasourceSchemaExec, + coldSchemaExecPeriodMillis, + TimeUnit.MILLISECONDS + ); + if (config.isAwaitInitializationOnStart()) { awaitInitialization(); } @@ -196,6 +253,9 @@ public void onLeaderStop() if (cacheExecFuture != null) { cacheExecFuture.cancel(true); } + if (coldSchemaExecFuture != null) { + coldSchemaExecFuture.cancel(true); + } segmentSchemaCache.onLeaderStop(); segmentSchemaBackfillQueue.onLeaderStop(); } @@ -209,6 +269,11 @@ public synchronized void refreshWaitCondition() throws InterruptedException segmentSchemaCache.awaitInitialization(); } + public void updateSegmentReplicationStatus(SegmentReplicationStatus segmentReplicationStatus) + { + this.segmentReplicationStatus = segmentReplicationStatus; + } + @Override protected void unmarkSegmentAsMutable(SegmentId segmentId) { @@ -336,6 +401,62 @@ public AvailableSegmentMetadata getAvailableSegmentMetadata(String datasource, S return availableSegmentMetadata; } + @Override + public DataSourceInformation getDatasource(String name) + { + return getMergedDatasourceInformation(tables.get(name), coldSchemaTable.get(name)).orElse(null); + } + + @Override + public Map getDataSourceInformationMap() + { + Map hot = new HashMap<>(tables); + Map cold = new HashMap<>(coldSchemaTable); + Set combinedDatasources = new HashSet<>(hot.keySet()); + combinedDatasources.addAll(cold.keySet()); + ImmutableMap.Builder combined = ImmutableMap.builder(); + + for (String dataSource : combinedDatasources) { + getMergedDatasourceInformation(hot.get(dataSource), cold.get(dataSource)) + .ifPresent(merged -> combined.put( + dataSource, + merged + )); + } + + return combined.build(); + } + + private Optional getMergedDatasourceInformation( + final DataSourceInformation hot, + final DataSourceInformation cold + ) + { + if (hot == null && cold == null) { + return Optional.empty(); + } else if (hot != null && cold == null) { + return Optional.of(hot); + } else if (hot == null && cold != null) { + return Optional.of(cold); + } else { + final Map columnTypes = new LinkedHashMap<>(); + + List signatures = new ArrayList<>(); + // hot datasource schema takes precedence + signatures.add(hot.getRowSignature()); + signatures.add(cold.getRowSignature()); + + for (RowSignature signature : signatures) { + mergeRowSignature(columnTypes, signature); + } + + final RowSignature.Builder builder = RowSignature.builder(); + columnTypes.forEach(builder::add); + + return Optional.of(new DataSourceInformation(hot.getDataSource(), builder.build())); + } + } + /** * Executes SegmentMetadataQuery to fetch schema information for each segment in the refresh list. * The schema information for individual segments is combined to construct a table schema, which is then cached. @@ -382,6 +503,7 @@ public void refresh(final Set segmentsToRefresh, final Set da // Rebuild the datasources. for (String dataSource : dataSourcesToRebuild) { final RowSignature rowSignature = buildDataSourceRowSignature(dataSource); + if (rowSignature == null) { log.info("RowSignature null for dataSource [%s], implying that it no longer exists. All metadata removed.", dataSource); tables.remove(dataSource); @@ -419,6 +541,94 @@ private Set filterSegmentWithCachedSchema(Set segmentIds) return cachedSegments; } + @Nullable + private Integer getReplicationFactor(SegmentId segmentId) + { + if (segmentReplicationStatus == null) { + return null; + } + SegmentReplicaCount replicaCountsInCluster = segmentReplicationStatus.getReplicaCountsInCluster(segmentId); + return replicaCountsInCluster == null ? null : replicaCountsInCluster.required(); + } + + @VisibleForTesting + protected void coldDatasourceSchemaExec() + { + Stopwatch stopwatch = Stopwatch.createStarted(); + + Set dataSourceWithColdSegmentSet = new HashSet<>(); + + int datasources = 0; + int segments = 0; + int dataSourceWithColdSegments = 0; + + Collection immutableDataSources = + sqlSegmentsMetadataManager.getImmutableDataSourcesWithAllUsedSegments(); + + for (ImmutableDruidDataSource dataSource : immutableDataSources) { + datasources++; + Collection dataSegments = dataSource.getSegments(); + + final Map columnTypes = new LinkedHashMap<>(); + + for (DataSegment segment : dataSegments) { + Integer replicationFactor = getReplicationFactor(segment.getId()); + if (replicationFactor != null && replicationFactor != 0) { + continue; + } + Optional optionalSchema = segmentSchemaCache.getSchemaForSegment(segment.getId()); + if (optionalSchema.isPresent()) { + RowSignature rowSignature = optionalSchema.get().getSchemaPayload().getRowSignature(); + mergeRowSignature(columnTypes, rowSignature); + } + segments++; + } + + if (columnTypes.isEmpty()) { + // this datasource doesn't have any cold segment + continue; + } + + final RowSignature.Builder builder = RowSignature.builder(); + columnTypes.forEach(builder::add); + + RowSignature coldSignature = builder.build(); + + String dataSourceName = dataSource.getName(); + dataSourceWithColdSegmentSet.add(dataSourceName); + dataSourceWithColdSegments++; + + log.debug("[%s] signature from cold segments is [%s]", dataSourceName, coldSignature); + + coldSchemaTable.put(dataSourceName, new DataSourceInformation(dataSourceName, coldSignature)); + } + + // remove any stale datasource from the map + coldSchemaTable.keySet().retainAll(dataSourceWithColdSegmentSet); + + String executionStatsLog = StringUtils.format( + "Cold schema processing took [%d] millis. " + + "Processed total [%d] datasources, [%d] segments. Found [%d] datasources with cold segments.", + stopwatch.millisElapsed(), datasources, segments, dataSourceWithColdSegments + ); + if (stopwatch.millisElapsed() > COLD_SCHEMA_SLOWNESS_THRESHOLD_MILLIS) { + log.info(executionStatsLog); + } else { + log.debug(executionStatsLog); + } + } + + private void mergeRowSignature(final Map columnTypes, final RowSignature signature) + { + for (String column : signature.getColumnNames()) { + final ColumnType columnType = + signature.getColumnType(column) + .orElseThrow(() -> new ISE("Encountered null type for column [%s]", column)); + + columnTypes.compute(column, (c, existingType) -> columnTypeMergePolicy.merge(existingType, columnType)); + } + } + @VisibleForTesting @Nullable @Override @@ -434,13 +644,7 @@ public RowSignature buildDataSourceRowSignature(final String dataSource) Optional optionalSchema = segmentSchemaCache.getSchemaForSegment(segmentId); if (optionalSchema.isPresent()) { RowSignature rowSignature = optionalSchema.get().getSchemaPayload().getRowSignature(); - for (String column : rowSignature.getColumnNames()) { - final ColumnType columnType = - rowSignature.getColumnType(column) - .orElseThrow(() -> new ISE("Encountered null type for column [%s]", column)); - - columnTypes.compute(column, (c, existingType) -> columnTypeMergePolicy.merge(existingType, columnType)); - } + mergeRowSignature(columnTypes, rowSignature); } else { // mark it for refresh, however, this case shouldn't arise by design markSegmentAsNeedRefresh(segmentId); 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 36cfac8089c4..9710bda79b44 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 @@ -816,6 +816,9 @@ public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params) { broadcastSegments = params.getBroadcastSegments(); segmentReplicationStatus = params.getSegmentReplicationStatus(); + if (coordinatorSegmentMetadataCache != null) { + coordinatorSegmentMetadataCache.updateSegmentReplicationStatus(segmentReplicationStatus); + } // Collect stats for unavailable and under-replicated segments final CoordinatorRunStats stats = params.getCoordinatorStats(); diff --git a/server/src/test/java/org/apache/druid/client/coordinator/NoopCoordinatorClient.java b/server/src/test/java/org/apache/druid/client/coordinator/NoopCoordinatorClient.java index 5aee343a851b..58f5af58a3e7 100644 --- a/server/src/test/java/org/apache/druid/client/coordinator/NoopCoordinatorClient.java +++ b/server/src/test/java/org/apache/druid/client/coordinator/NoopCoordinatorClient.java @@ -75,4 +75,10 @@ public CoordinatorClient withRetryPolicy(ServiceRetryPolicy retryPolicy) // Ignore retryPolicy for the test client. return this; } + + @Override + public ListenableFuture> fetchDataSourcesWithUsedSegments() + { + throw new UnsupportedOperationException(); + } } diff --git a/server/src/test/java/org/apache/druid/segment/metadata/CoordinatorSegmentDataCacheConcurrencyTest.java b/server/src/test/java/org/apache/druid/segment/metadata/CoordinatorSegmentDataCacheConcurrencyTest.java index 81f65acf84ae..4cc4ac38184b 100644 --- a/server/src/test/java/org/apache/druid/segment/metadata/CoordinatorSegmentDataCacheConcurrencyTest.java +++ b/server/src/test/java/org/apache/druid/segment/metadata/CoordinatorSegmentDataCacheConcurrencyTest.java @@ -20,6 +20,8 @@ package org.apache.druid.segment.metadata; import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Supplier; +import com.google.common.base.Suppliers; import com.google.common.collect.ImmutableList; import com.google.common.collect.Sets; import org.apache.druid.client.BrokerServerView; @@ -39,6 +41,8 @@ import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.common.concurrent.ScheduledExecutors; import org.apache.druid.java.util.emitter.service.ServiceEmitter; +import org.apache.druid.metadata.SegmentsMetadataManagerConfig; +import org.apache.druid.metadata.SqlSegmentsMetadataManager; import org.apache.druid.metadata.TestDerbyConnector; import org.apache.druid.query.QueryRunner; import org.apache.druid.query.TableDataSource; @@ -61,16 +65,19 @@ import org.apache.druid.timeline.SegmentId; import org.apache.druid.timeline.partition.NumberedShardSpec; import org.easymock.EasyMock; +import org.joda.time.Period; import org.junit.After; import org.junit.Assert; import org.junit.Before; import org.junit.Rule; import org.junit.Test; +import org.mockito.Mockito; import javax.annotation.Nullable; import java.io.File; import java.util.ArrayList; import java.util.Collection; +import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.List; @@ -101,6 +108,8 @@ public class CoordinatorSegmentDataCacheConcurrencyTest extends SegmentMetadataC private TestSegmentMetadataQueryWalker walker; private SegmentSchemaCache segmentSchemaCache; private SegmentSchemaBackFillQueue backFillQueue; + private SqlSegmentsMetadataManager sqlSegmentsMetadataManager; + private Supplier segmentsMetadataManagerConfigSupplier; private final ObjectMapper mapper = TestHelper.makeJsonMapper(); @Before @@ -190,6 +199,12 @@ public CallbackAction segmentSchemasAnnounced(SegmentSchemas segmentSchemas) } ); + sqlSegmentsMetadataManager = Mockito.mock(SqlSegmentsMetadataManager.class); + Mockito.when(sqlSegmentsMetadataManager.getImmutableDataSourcesWithAllUsedSegments()).thenReturn(Collections.emptyList()); + SegmentsMetadataManagerConfig metadataManagerConfig = Mockito.mock(SegmentsMetadataManagerConfig.class); + Mockito.when(metadataManagerConfig.getPollDuration()).thenReturn(Period.millis(1000)); + segmentsMetadataManagerConfigSupplier = Suppliers.ofInstance(metadataManagerConfig); + inventoryView.init(); initLatch.await(); exec = Execs.multiThreaded(4, "DruidSchemaConcurrencyTest-%d"); @@ -227,7 +242,9 @@ public void testSegmentMetadataRefreshAndInventoryViewAddSegmentAndBrokerServerV new InternalQueryConfig(), new NoopServiceEmitter(), segmentSchemaCache, - backFillQueue + backFillQueue, + sqlSegmentsMetadataManager, + segmentsMetadataManagerConfigSupplier ) { @Override @@ -341,7 +358,9 @@ public void testSegmentMetadataRefreshAndDruidSchemaGetSegmentMetadata() new InternalQueryConfig(), new NoopServiceEmitter(), segmentSchemaCache, - backFillQueue + backFillQueue, + sqlSegmentsMetadataManager, + segmentsMetadataManagerConfigSupplier ) { @Override diff --git a/server/src/test/java/org/apache/druid/segment/metadata/CoordinatorSegmentMetadataCacheTest.java b/server/src/test/java/org/apache/druid/segment/metadata/CoordinatorSegmentMetadataCacheTest.java index e5b6db1d42df..ef1fb1e8eddf 100644 --- a/server/src/test/java/org/apache/druid/segment/metadata/CoordinatorSegmentMetadataCacheTest.java +++ b/server/src/test/java/org/apache/druid/segment/metadata/CoordinatorSegmentMetadataCacheTest.java @@ -22,11 +22,14 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; +import com.google.common.base.Supplier; +import com.google.common.base.Suppliers; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Sets; import org.apache.druid.client.DruidServer; +import org.apache.druid.client.ImmutableDruidDataSource; import org.apache.druid.client.InternalQueryConfig; import org.apache.druid.data.input.InputRow; import org.apache.druid.java.util.common.Intervals; @@ -37,6 +40,8 @@ import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.java.util.metrics.StubServiceEmitter; import org.apache.druid.metadata.MetadataStorageTablesConfig; +import org.apache.druid.metadata.SegmentsMetadataManagerConfig; +import org.apache.druid.metadata.SqlSegmentsMetadataManager; import org.apache.druid.metadata.TestDerbyConnector; import org.apache.druid.query.DruidMetrics; import org.apache.druid.query.QueryContexts; @@ -66,6 +71,8 @@ import org.apache.druid.server.QueryResponse; import org.apache.druid.server.coordination.DruidServerMetadata; import org.apache.druid.server.coordination.ServerType; +import org.apache.druid.server.coordinator.loading.SegmentReplicaCount; +import org.apache.druid.server.coordinator.loading.SegmentReplicationStatus; import org.apache.druid.server.metrics.NoopServiceEmitter; import org.apache.druid.server.security.Access; import org.apache.druid.server.security.AllowAllAuthenticator; @@ -74,18 +81,23 @@ import org.apache.druid.timeline.SegmentId; import org.apache.druid.timeline.partition.LinearShardSpec; import org.easymock.EasyMock; +import org.joda.time.Period; import org.junit.After; import org.junit.Assert; import org.junit.Before; import org.junit.Test; +import org.mockito.ArgumentMatchers; +import org.mockito.Mockito; import org.skife.jdbi.v2.StatementContext; import java.io.File; import java.io.IOException; import java.sql.ResultSet; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collections; import java.util.EnumSet; +import java.util.HashMap; import java.util.HashSet; import java.util.LinkedHashMap; import java.util.LinkedHashSet; @@ -106,12 +118,19 @@ public class CoordinatorSegmentMetadataCacheTest extends CoordinatorSegmentMetad private CoordinatorSegmentMetadataCache runningSchema; private CountDownLatch buildTableLatch = new CountDownLatch(1); private CountDownLatch markDataSourceLatch = new CountDownLatch(1); + private SqlSegmentsMetadataManager sqlSegmentsMetadataManager; + private Supplier segmentsMetadataManagerConfigSupplier; @Before @Override public void setUp() throws Exception { super.setUp(); + sqlSegmentsMetadataManager = Mockito.mock(SqlSegmentsMetadataManager.class); + Mockito.when(sqlSegmentsMetadataManager.getImmutableDataSourcesWithAllUsedSegments()).thenReturn(Collections.emptyList()); + SegmentsMetadataManagerConfig metadataManagerConfig = Mockito.mock(SegmentsMetadataManagerConfig.class); + Mockito.when(metadataManagerConfig.getPollDuration()).thenReturn(Period.millis(1000)); + segmentsMetadataManagerConfigSupplier = Suppliers.ofInstance(metadataManagerConfig); } @After @@ -132,6 +151,7 @@ public CoordinatorSegmentMetadataCache buildSchemaMarkAndTableLatch() throws Int public CoordinatorSegmentMetadataCache buildSchemaMarkAndTableLatch(SegmentMetadataCacheConfig config) throws InterruptedException { Preconditions.checkState(runningSchema == null); + runningSchema = new CoordinatorSegmentMetadataCache( getQueryLifecycleFactory(walker), serverView, @@ -140,7 +160,9 @@ public CoordinatorSegmentMetadataCache buildSchemaMarkAndTableLatch(SegmentMetad new InternalQueryConfig(), new NoopServiceEmitter(), segmentSchemaCache, - backFillQueue + backFillQueue, + sqlSegmentsMetadataManager, + segmentsMetadataManagerConfigSupplier ) { @Override @@ -178,7 +200,7 @@ public void testGetTableMap() throws InterruptedException public void testGetTableMapFoo() throws InterruptedException { CoordinatorSegmentMetadataCache schema = buildSchemaMarkAndTableLatch(); - verifyFooDSSchema(schema); + verifyFooDSSchema(schema, 6); } @Test @@ -312,7 +334,9 @@ public void testAllDatasourcesRebuiltOnDatasourceRemoval() throws IOException, I new InternalQueryConfig(), new NoopServiceEmitter(), segmentSchemaCache, - backFillQueue + backFillQueue, + sqlSegmentsMetadataManager, + segmentsMetadataManagerConfigSupplier ) { @Override @@ -523,7 +547,9 @@ public void testSegmentAddedCallbackAddNewHistoricalSegment() throws Interrupted new InternalQueryConfig(), new NoopServiceEmitter(), segmentSchemaCache, - backFillQueue + backFillQueue, + sqlSegmentsMetadataManager, + segmentsMetadataManagerConfigSupplier ) { @Override @@ -558,6 +584,11 @@ public void testSegmentAddedCallbackAddExistingSegment() throws InterruptedExcep { String datasource = "newSegmentAddTest"; CountDownLatch addSegmentLatch = new CountDownLatch(2); + SqlSegmentsMetadataManager sqlSegmentsMetadataManager = Mockito.mock(SqlSegmentsMetadataManager.class); + Mockito.when(sqlSegmentsMetadataManager.getImmutableDataSourcesWithAllUsedSegments()).thenReturn(Collections.emptyList()); + SegmentsMetadataManagerConfig metadataManagerConfig = Mockito.mock(SegmentsMetadataManagerConfig.class); + Mockito.when(metadataManagerConfig.getPollDuration()).thenReturn(Period.millis(1000)); + Supplier segmentsMetadataManagerConfigSupplier = Suppliers.ofInstance(metadataManagerConfig); CoordinatorSegmentMetadataCache schema = new CoordinatorSegmentMetadataCache( getQueryLifecycleFactory(walker), serverView, @@ -566,7 +597,9 @@ public void testSegmentAddedCallbackAddExistingSegment() throws InterruptedExcep new InternalQueryConfig(), new NoopServiceEmitter(), segmentSchemaCache, - backFillQueue + backFillQueue, + sqlSegmentsMetadataManager, + segmentsMetadataManagerConfigSupplier ) { @Override @@ -605,6 +638,11 @@ public void testSegmentAddedCallbackAddNewRealtimeSegment() throws InterruptedEx { String datasource = "newSegmentAddTest"; CountDownLatch addSegmentLatch = new CountDownLatch(1); + SqlSegmentsMetadataManager sqlSegmentsMetadataManager = Mockito.mock(SqlSegmentsMetadataManager.class); + Mockito.when(sqlSegmentsMetadataManager.getImmutableDataSourcesWithAllUsedSegments()).thenReturn(Collections.emptyList()); + SegmentsMetadataManagerConfig metadataManagerConfig = Mockito.mock(SegmentsMetadataManagerConfig.class); + Mockito.when(metadataManagerConfig.getPollDuration()).thenReturn(Period.millis(1000)); + Supplier segmentsMetadataManagerConfigSupplier = Suppliers.ofInstance(metadataManagerConfig); CoordinatorSegmentMetadataCache schema = new CoordinatorSegmentMetadataCache( getQueryLifecycleFactory(walker), serverView, @@ -613,7 +651,9 @@ public void testSegmentAddedCallbackAddNewRealtimeSegment() throws InterruptedEx new InternalQueryConfig(), new NoopServiceEmitter(), segmentSchemaCache, - backFillQueue + backFillQueue, + sqlSegmentsMetadataManager, + segmentsMetadataManagerConfigSupplier ) { @Override @@ -649,6 +689,11 @@ public void testSegmentAddedCallbackAddNewBroadcastSegment() throws InterruptedE { String datasource = "newSegmentAddTest"; CountDownLatch addSegmentLatch = new CountDownLatch(1); + SqlSegmentsMetadataManager sqlSegmentsMetadataManager = Mockito.mock(SqlSegmentsMetadataManager.class); + Mockito.when(sqlSegmentsMetadataManager.getImmutableDataSourcesWithAllUsedSegments()).thenReturn(Collections.emptyList()); + SegmentsMetadataManagerConfig metadataManagerConfig = Mockito.mock(SegmentsMetadataManagerConfig.class); + Mockito.when(metadataManagerConfig.getPollDuration()).thenReturn(Period.millis(1000)); + Supplier segmentsMetadataManagerConfigSupplier = Suppliers.ofInstance(metadataManagerConfig); CoordinatorSegmentMetadataCache schema = new CoordinatorSegmentMetadataCache( getQueryLifecycleFactory(walker), serverView, @@ -657,7 +702,9 @@ public void testSegmentAddedCallbackAddNewBroadcastSegment() throws InterruptedE new InternalQueryConfig(), new NoopServiceEmitter(), segmentSchemaCache, - backFillQueue + backFillQueue, + sqlSegmentsMetadataManager, + segmentsMetadataManagerConfigSupplier ) { @Override @@ -698,7 +745,9 @@ public void testSegmentRemovedCallbackEmptyDataSourceAfterRemove() throws Interr new InternalQueryConfig(), new NoopServiceEmitter(), segmentSchemaCache, - backFillQueue + backFillQueue, + sqlSegmentsMetadataManager, + segmentsMetadataManagerConfigSupplier ) { @Override @@ -756,7 +805,9 @@ public void testSegmentRemovedCallbackNonEmptyDataSourceAfterRemove() throws Int new InternalQueryConfig(), new NoopServiceEmitter(), segmentSchemaCache, - backFillQueue + backFillQueue, + sqlSegmentsMetadataManager, + segmentsMetadataManagerConfigSupplier ) { @Override @@ -817,7 +868,9 @@ public void testServerSegmentRemovedCallbackRemoveUnknownSegment() throws Interr new InternalQueryConfig(), new NoopServiceEmitter(), segmentSchemaCache, - backFillQueue + backFillQueue, + sqlSegmentsMetadataManager, + segmentsMetadataManagerConfigSupplier ) { @Override @@ -852,7 +905,9 @@ public void testServerSegmentRemovedCallbackRemoveBrokerSegment() throws Interru new InternalQueryConfig(), new NoopServiceEmitter(), segmentSchemaCache, - backFillQueue + backFillQueue, + sqlSegmentsMetadataManager, + segmentsMetadataManagerConfigSupplier ) { @Override @@ -900,7 +955,9 @@ public void testServerSegmentRemovedCallbackRemoveHistoricalSegment() throws Int new InternalQueryConfig(), new NoopServiceEmitter(), segmentSchemaCache, - backFillQueue + backFillQueue, + sqlSegmentsMetadataManager, + segmentsMetadataManagerConfigSupplier ) { @Override @@ -972,7 +1029,9 @@ public void testRunSegmentMetadataQueryWithContext() throws Exception internalQueryConfig, new NoopServiceEmitter(), segmentSchemaCache, - backFillQueue + backFillQueue, + sqlSegmentsMetadataManager, + segmentsMetadataManagerConfigSupplier ); Map queryContext = ImmutableMap.of( @@ -1141,7 +1200,9 @@ public void testRefreshShouldEmitMetrics() throws InterruptedException, IOExcept new InternalQueryConfig(), emitter, segmentSchemaCache, - backFillQueue + backFillQueue, + sqlSegmentsMetadataManager, + segmentsMetadataManagerConfigSupplier ) { @Override @@ -1306,7 +1367,9 @@ public void testRealtimeSchemaAnnouncement() throws InterruptedException, IOExce new InternalQueryConfig(), new NoopServiceEmitter(), segmentSchemaCache, - backFillQueue + backFillQueue, + sqlSegmentsMetadataManager, + segmentsMetadataManagerConfigSupplier ) { @Override void updateSchemaForRealtimeSegments(SegmentSchemas segmentSchemas) @@ -1385,7 +1448,9 @@ public void testRealtimeSchemaAnnouncementDataSourceSchemaUpdated() throws Inter new InternalQueryConfig(), new NoopServiceEmitter(), segmentSchemaCache, - backFillQueue + backFillQueue, + sqlSegmentsMetadataManager, + segmentsMetadataManagerConfigSupplier ) { @Override public void refresh(Set segmentsToRefresh, Set dataSourcesToRebuild) @@ -1565,7 +1630,9 @@ public void testSchemaBackfilling() throws InterruptedException new InternalQueryConfig(), new NoopServiceEmitter(), segmentSchemaCache, - backFillQueue + backFillQueue, + sqlSegmentsMetadataManager, + segmentsMetadataManagerConfigSupplier ) { @Override public Set refreshSegmentsForDataSource(String dataSource, Set segments) @@ -1594,7 +1661,7 @@ public void refresh(Set segmentsToRefresh, Set dataSourcesToR Assert.assertEquals(0, refreshCount.get()); // verify that datasource schema is built - verifyFooDSSchema(schema); + verifyFooDSSchema(schema, 6); serverView.addSegment(segment3, ServerType.HISTORICAL); @@ -1721,12 +1788,384 @@ public void testSameSegmentAddedOnMultipleServer() throws InterruptedException, Assert.assertEquals(existingMetadata.getNumReplicas(), currentMetadata.getNumReplicas()); } - private void verifyFooDSSchema(CoordinatorSegmentMetadataCache schema) + private CoordinatorSegmentMetadataCache setupForColdDatasourceSchemaTest() + { + // foo has both hot and cold segments + DataSegment coldSegment = + DataSegment.builder() + .dataSource(DATASOURCE1) + .interval(Intervals.of("1998/P2Y")) + .version("1") + .shardSpec(new LinearShardSpec(0)) + .size(0) + .build(); + + // cold has only cold segments + DataSegment singleColdSegment = + DataSegment.builder() + .dataSource("cold") + .interval(Intervals.of("2000/P2Y")) + .version("1") + .shardSpec(new LinearShardSpec(0)) + .size(0) + .build(); + + ImmutableMap.Builder segmentStatsMap = new ImmutableMap.Builder<>(); + segmentStatsMap.put(coldSegment.getId(), new SegmentMetadata(20L, "foo-fingerprint")); + segmentStatsMap.put(singleColdSegment.getId(), new SegmentMetadata(20L, "cold-fingerprint")); + ImmutableMap.Builder schemaPayloadMap = new ImmutableMap.Builder<>(); + schemaPayloadMap.put( + "foo-fingerprint", + new SchemaPayload(RowSignature.builder() + .add("dim1", ColumnType.STRING) + .add("c1", ColumnType.STRING) + .add("c2", ColumnType.LONG) + .build()) + ); + schemaPayloadMap.put( + "cold-fingerprint", + new SchemaPayload( + RowSignature.builder() + .add("f1", ColumnType.STRING) + .add("f2", ColumnType.DOUBLE) + .build() + ) + ); + + segmentSchemaCache.updateFinalizedSegmentSchema( + new SegmentSchemaCache.FinalizedSegmentSchemaInfo(segmentStatsMap.build(), schemaPayloadMap.build()) + ); + + List druidDataSources = new ArrayList<>(); + Map segmentMap = new HashMap<>(); + segmentMap.put(coldSegment.getId(), coldSegment); + segmentMap.put(segment1.getId(), segment1); + segmentMap.put(segment2.getId(), segment2); + druidDataSources.add(new ImmutableDruidDataSource( + coldSegment.getDataSource(), + Collections.emptyMap(), + segmentMap + )); + druidDataSources.add(new ImmutableDruidDataSource( + singleColdSegment.getDataSource(), + Collections.emptyMap(), + Collections.singletonMap(singleColdSegment.getId(), singleColdSegment) + )); + + Mockito.when( + sqlSegmentsMetadataManager.getImmutableDataSourcesWithAllUsedSegments()) + .thenReturn(druidDataSources); + + CoordinatorSegmentMetadataCache schema = new CoordinatorSegmentMetadataCache( + getQueryLifecycleFactory(walker), + serverView, + SEGMENT_CACHE_CONFIG_DEFAULT, + new NoopEscalator(), + new InternalQueryConfig(), + new NoopServiceEmitter(), + segmentSchemaCache, + backFillQueue, + sqlSegmentsMetadataManager, + segmentsMetadataManagerConfigSupplier + ); + + SegmentReplicaCount zeroSegmentReplicaCount = Mockito.mock(SegmentReplicaCount.class); + SegmentReplicaCount nonZeroSegmentReplicaCount = Mockito.mock(SegmentReplicaCount.class); + Mockito.when(zeroSegmentReplicaCount.required()).thenReturn(0); + Mockito.when(nonZeroSegmentReplicaCount.required()).thenReturn(1); + SegmentReplicationStatus segmentReplicationStatus = Mockito.mock(SegmentReplicationStatus.class); + Mockito.when(segmentReplicationStatus.getReplicaCountsInCluster(ArgumentMatchers.eq(coldSegment.getId()))) + .thenReturn(zeroSegmentReplicaCount); + Mockito.when(segmentReplicationStatus.getReplicaCountsInCluster(ArgumentMatchers.eq(singleColdSegment.getId()))) + .thenReturn(zeroSegmentReplicaCount); + Mockito.when(segmentReplicationStatus.getReplicaCountsInCluster(ArgumentMatchers.eq(segment1.getId()))) + .thenReturn(nonZeroSegmentReplicaCount); + + Mockito.when(segmentReplicationStatus.getReplicaCountsInCluster(ArgumentMatchers.eq(segment2.getId()))) + .thenReturn(nonZeroSegmentReplicaCount); + + schema.updateSegmentReplicationStatus(segmentReplicationStatus); + schema.updateSegmentReplicationStatus(segmentReplicationStatus); + + return schema; + } + + @Test + public void testColdDatasourceSchema_refreshAfterColdSchemaExec() throws IOException + { + CoordinatorSegmentMetadataCache schema = setupForColdDatasourceSchemaTest(); + + schema.coldDatasourceSchemaExec(); + + Assert.assertEquals(new HashSet<>(Arrays.asList("foo", "cold")), schema.getDataSourceInformationMap().keySet()); + + // verify that cold schema for both foo and cold is present + RowSignature fooSignature = schema.getDatasource("foo").getRowSignature(); + List columnNames = fooSignature.getColumnNames(); + + // verify that foo schema doesn't contain columns from hot segments + Assert.assertEquals(3, columnNames.size()); + + Assert.assertEquals("dim1", columnNames.get(0)); + Assert.assertEquals(ColumnType.STRING, fooSignature.getColumnType(columnNames.get(0)).get()); + + Assert.assertEquals("c1", columnNames.get(1)); + Assert.assertEquals(ColumnType.STRING, fooSignature.getColumnType(columnNames.get(1)).get()); + + Assert.assertEquals("c2", columnNames.get(2)); + Assert.assertEquals(ColumnType.LONG, fooSignature.getColumnType(columnNames.get(2)).get()); + + RowSignature coldSignature = schema.getDatasource("cold").getRowSignature(); + columnNames = coldSignature.getColumnNames(); + Assert.assertEquals("f1", columnNames.get(0)); + Assert.assertEquals(ColumnType.STRING, coldSignature.getColumnType(columnNames.get(0)).get()); + + Assert.assertEquals("f2", columnNames.get(1)); + Assert.assertEquals(ColumnType.DOUBLE, coldSignature.getColumnType(columnNames.get(1)).get()); + + Set segmentIds = new HashSet<>(); + segmentIds.add(segment1.getId()); + segmentIds.add(segment2.getId()); + + schema.refresh(segmentIds, new HashSet<>()); + + Assert.assertEquals(new HashSet<>(Arrays.asList("foo", "cold")), schema.getDataSourceInformationMap().keySet()); + + coldSignature = schema.getDatasource("cold").getRowSignature(); + columnNames = coldSignature.getColumnNames(); + Assert.assertEquals("f1", columnNames.get(0)); + Assert.assertEquals(ColumnType.STRING, coldSignature.getColumnType(columnNames.get(0)).get()); + + Assert.assertEquals("f2", columnNames.get(1)); + Assert.assertEquals(ColumnType.DOUBLE, coldSignature.getColumnType(columnNames.get(1)).get()); + + // foo now contains schema from both hot and cold segments + verifyFooDSSchema(schema, 8); + RowSignature rowSignature = schema.getDatasource("foo").getRowSignature(); + + // cold columns should be present at the end + columnNames = rowSignature.getColumnNames(); + Assert.assertEquals("c1", columnNames.get(6)); + Assert.assertEquals(ColumnType.STRING, rowSignature.getColumnType(columnNames.get(6)).get()); + + Assert.assertEquals("c2", columnNames.get(7)); + Assert.assertEquals(ColumnType.LONG, rowSignature.getColumnType(columnNames.get(7)).get()); + } + + @Test + public void testColdDatasourceSchema_coldSchemaExecAfterRefresh() throws IOException + { + CoordinatorSegmentMetadataCache schema = setupForColdDatasourceSchemaTest(); + + Set segmentIds = new HashSet<>(); + segmentIds.add(segment1.getId()); + segmentIds.add(segment2.getId()); + + schema.refresh(segmentIds, new HashSet<>()); + // cold datasource shouldn't be present + Assert.assertEquals(Collections.singleton("foo"), schema.getDataSourceInformationMap().keySet()); + + // cold columns shouldn't be present + verifyFooDSSchema(schema, 6); + Assert.assertNull(schema.getDatasource("cold")); + + schema.coldDatasourceSchemaExec(); + + // could datasource should be present now + Assert.assertEquals(new HashSet<>(Arrays.asList("foo", "cold")), schema.getDataSourceInformationMap().keySet()); + + RowSignature coldSignature = schema.getDatasource("cold").getRowSignature(); + List columnNames = coldSignature.getColumnNames(); + Assert.assertEquals("f1", columnNames.get(0)); + Assert.assertEquals(ColumnType.STRING, coldSignature.getColumnType(columnNames.get(0)).get()); + + Assert.assertEquals("f2", columnNames.get(1)); + Assert.assertEquals(ColumnType.DOUBLE, coldSignature.getColumnType(columnNames.get(1)).get()); + + // columns from cold datasource should be present + verifyFooDSSchema(schema, 8); + RowSignature rowSignature = schema.getDatasource("foo").getRowSignature(); + + columnNames = rowSignature.getColumnNames(); + Assert.assertEquals("c1", columnNames.get(6)); + Assert.assertEquals(ColumnType.STRING, rowSignature.getColumnType(columnNames.get(6)).get()); + + Assert.assertEquals("c2", columnNames.get(7)); + Assert.assertEquals(ColumnType.LONG, rowSignature.getColumnType(columnNames.get(7)).get()); + } + + @Test + public void testColdDatasourceSchema_verifyStaleDatasourceRemoved() + { + DataSegment coldSegmentAlpha = + DataSegment.builder() + .dataSource("alpha") + .interval(Intervals.of("2000/P2Y")) + .version("1") + .shardSpec(new LinearShardSpec(0)) + .size(0) + .build(); + + DataSegment coldSegmentBeta = + DataSegment.builder() + .dataSource("beta") + .interval(Intervals.of("2000/P2Y")) + .version("1") + .shardSpec(new LinearShardSpec(0)) + .size(0) + .build(); + + DataSegment coldSegmentGamma = + DataSegment.builder() + .dataSource("gamma") + .interval(Intervals.of("2000/P2Y")) + .version("1") + .shardSpec(new LinearShardSpec(0)) + .size(0) + .build(); + + DataSegment hotSegmentGamma = + DataSegment.builder() + .dataSource("gamma") + .interval(Intervals.of("2001/P2Y")) + .version("1") + .shardSpec(new LinearShardSpec(0)) + .size(0) + .build(); + + ImmutableMap.Builder segmentStatsMap = new ImmutableMap.Builder<>(); + segmentStatsMap.put(coldSegmentAlpha.getId(), new SegmentMetadata(20L, "cold")); + segmentStatsMap.put(coldSegmentBeta.getId(), new SegmentMetadata(20L, "cold")); + segmentStatsMap.put(hotSegmentGamma.getId(), new SegmentMetadata(20L, "hot")); + segmentStatsMap.put(coldSegmentGamma.getId(), new SegmentMetadata(20L, "cold")); + + ImmutableMap.Builder schemaPayloadMap = new ImmutableMap.Builder<>(); + schemaPayloadMap.put( + "cold", + new SchemaPayload(RowSignature.builder() + .add("dim1", ColumnType.STRING) + .add("c1", ColumnType.STRING) + .add("c2", ColumnType.LONG) + .build()) + ); + schemaPayloadMap.put( + "hot", + new SchemaPayload(RowSignature.builder() + .add("c3", ColumnType.STRING) + .add("c4", ColumnType.STRING) + .build()) + ); + segmentSchemaCache.updateFinalizedSegmentSchema( + new SegmentSchemaCache.FinalizedSegmentSchemaInfo(segmentStatsMap.build(), schemaPayloadMap.build()) + ); + + List druidDataSources = new ArrayList<>(); + druidDataSources.add( + new ImmutableDruidDataSource( + "alpha", + Collections.emptyMap(), + Collections.singletonMap(coldSegmentAlpha.getId(), coldSegmentAlpha) + ) + ); + + Map gammaSegments = new HashMap<>(); + gammaSegments.put(hotSegmentGamma.getId(), hotSegmentGamma); + gammaSegments.put(coldSegmentGamma.getId(), coldSegmentGamma); + + druidDataSources.add( + new ImmutableDruidDataSource( + "gamma", + Collections.emptyMap(), + gammaSegments + ) + ); + + Mockito.when(sqlSegmentsMetadataManager.getImmutableDataSourcesWithAllUsedSegments()) + .thenReturn(druidDataSources); + + CoordinatorSegmentMetadataCache schema = new CoordinatorSegmentMetadataCache( + getQueryLifecycleFactory(walker), + serverView, + SEGMENT_CACHE_CONFIG_DEFAULT, + new NoopEscalator(), + new InternalQueryConfig(), + new NoopServiceEmitter(), + segmentSchemaCache, + backFillQueue, + sqlSegmentsMetadataManager, + segmentsMetadataManagerConfigSupplier + ); + + SegmentReplicaCount zeroSegmentReplicaCount = Mockito.mock(SegmentReplicaCount.class); + SegmentReplicaCount nonZeroSegmentReplicaCount = Mockito.mock(SegmentReplicaCount.class); + Mockito.when(zeroSegmentReplicaCount.required()).thenReturn(0); + Mockito.when(nonZeroSegmentReplicaCount.required()).thenReturn(1); + SegmentReplicationStatus segmentReplicationStatus = Mockito.mock(SegmentReplicationStatus.class); + Mockito.when(segmentReplicationStatus.getReplicaCountsInCluster(ArgumentMatchers.eq(coldSegmentAlpha.getId()))) + .thenReturn(zeroSegmentReplicaCount); + Mockito.when(segmentReplicationStatus.getReplicaCountsInCluster(ArgumentMatchers.eq(coldSegmentBeta.getId()))) + .thenReturn(zeroSegmentReplicaCount); + Mockito.when(segmentReplicationStatus.getReplicaCountsInCluster(ArgumentMatchers.eq(coldSegmentGamma.getId()))) + .thenReturn(zeroSegmentReplicaCount); + + Mockito.when(segmentReplicationStatus.getReplicaCountsInCluster(ArgumentMatchers.eq(hotSegmentGamma.getId()))) + .thenReturn(nonZeroSegmentReplicaCount); + + schema.updateSegmentReplicationStatus(segmentReplicationStatus); + + schema.coldDatasourceSchemaExec(); + // alpha has only 1 cold segment + Assert.assertNotNull(schema.getDatasource("alpha")); + // gamma has both hot and cold segment + Assert.assertNotNull(schema.getDatasource("gamma")); + // assert that cold schema for gamma doesn't contain any columns from hot segment + RowSignature rowSignature = schema.getDatasource("gamma").getRowSignature(); + Assert.assertTrue(rowSignature.contains("dim1")); + Assert.assertTrue(rowSignature.contains("c1")); + Assert.assertTrue(rowSignature.contains("c2")); + Assert.assertFalse(rowSignature.contains("c3")); + Assert.assertFalse(rowSignature.contains("c4")); + + Assert.assertEquals(new HashSet<>(Arrays.asList("alpha", "gamma")), schema.getDataSourceInformationMap().keySet()); + + druidDataSources.clear(); + druidDataSources.add( + new ImmutableDruidDataSource( + "beta", + Collections.emptyMap(), + Collections.singletonMap(coldSegmentBeta.getId(), coldSegmentBeta) + ) + ); + + druidDataSources.add( + new ImmutableDruidDataSource( + "gamma", + Collections.emptyMap(), + Collections.singletonMap(hotSegmentGamma.getId(), hotSegmentGamma) + ) + ); + + Mockito.when(sqlSegmentsMetadataManager.getImmutableDataSourcesWithAllUsedSegments()) + .thenReturn(druidDataSources); + + schema.coldDatasourceSchemaExec(); + Assert.assertNotNull(schema.getDatasource("beta")); + // alpha doesn't have any segments + Assert.assertNull(schema.getDatasource("alpha")); + // gamma just has 1 hot segment + Assert.assertNull(schema.getDatasource("gamma")); + + Assert.assertNull(schema.getDatasource("doesnotexist")); + + Assert.assertEquals(Collections.singleton("beta"), schema.getDataSourceInformationMap().keySet()); + } + + private void verifyFooDSSchema(CoordinatorSegmentMetadataCache schema, int columns) { final DataSourceInformation fooDs = schema.getDatasource("foo"); final RowSignature fooRowSignature = fooDs.getRowSignature(); List columnNames = fooRowSignature.getColumnNames(); - Assert.assertEquals(6, columnNames.size()); + Assert.assertEquals(columns, columnNames.size()); Assert.assertEquals("__time", columnNames.get(0)); Assert.assertEquals(ColumnType.LONG, fooRowSignature.getColumnType(columnNames.get(0)).get()); diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/schema/BrokerSegmentMetadataCache.java b/sql/src/main/java/org/apache/druid/sql/calcite/schema/BrokerSegmentMetadataCache.java index 7974ed460ebb..628b6ea39781 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/schema/BrokerSegmentMetadataCache.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/schema/BrokerSegmentMetadataCache.java @@ -173,6 +173,16 @@ public void stop() callbackExec.shutdownNow(); } + /** + * Execute refresh on the broker in each cycle if CentralizedDatasourceSchema is enabled + * else if there are segments or datasources to be refreshed. + */ + @Override + protected boolean shouldRefresh() + { + return centralizedDatasourceSchemaConfig.isEnabled() || super.shouldRefresh(); + } + /** * Refreshes the set of segments in two steps: *
      @@ -196,6 +206,11 @@ public void refresh(final Set segmentsToRefresh, final Set da // segmentMetadataInfo keys should be a superset of all other sets including datasources to refresh final Set dataSourcesToQuery = new HashSet<>(segmentMetadataInfo.keySet()); + // this is the complete set of datasources polled from the Coordinator + final Set polledDatasources = queryDataSources(); + + dataSourcesToQuery.addAll(polledDatasources); + log.debug("Querying schema for [%s] datasources from Coordinator.", dataSourcesToQuery); // Fetch datasource information from the Coordinator @@ -227,14 +242,7 @@ public void refresh(final Set segmentsToRefresh, final Set da // Remove those datasource for which we received schema from the Coordinator. dataSourcesToRebuild.removeAll(polledDataSourceMetadata.keySet()); - if (centralizedDatasourceSchemaConfig.isEnabled()) { - // this is a hacky way to ensure refresh is executed even if there are no new segments to refresh - // once, CentralizedDatasourceSchema feature is GA, brokers should simply poll schema for all datasources - dataSourcesNeedingRebuild.addAll(segmentMetadataInfo.keySet()); - } else { - dataSourcesNeedingRebuild.clear(); - } - log.debug("DatasourcesNeedingRebuild are [%s]", dataSourcesNeedingRebuild); + dataSourcesNeedingRebuild.clear(); } // Rebuild the datasources. @@ -267,6 +275,23 @@ protected void removeSegmentAction(SegmentId segmentId) // noop, no additional action needed when segment is removed. } + private Set queryDataSources() + { + Set dataSources = new HashSet<>(); + + try { + Set polled = FutureUtils.getUnchecked(coordinatorClient.fetchDataSourcesWithUsedSegments(), true); + if (polled != null) { + dataSources.addAll(polled); + } + } + catch (Exception e) { + log.debug(e, "Failed to query datasources from the Coordinator."); + } + + return dataSources; + } + private Map queryDataSourceInformation(Set dataSourcesToQuery) { Stopwatch stopwatch = Stopwatch.createStarted(); diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/schema/BrokerSegmentMetadataCacheTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/schema/BrokerSegmentMetadataCacheTest.java index 23b2759286c5..65610ce99f28 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/schema/BrokerSegmentMetadataCacheTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/schema/BrokerSegmentMetadataCacheTest.java @@ -74,11 +74,13 @@ import org.apache.druid.sql.calcite.table.DatasourceTable; import org.apache.druid.sql.calcite.table.DruidTable; import org.apache.druid.sql.calcite.util.CalciteTests; +import org.apache.druid.sql.calcite.util.TestTimelineServerView; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.SegmentId; import org.apache.druid.timeline.partition.LinearShardSpec; import org.apache.druid.timeline.partition.NumberedShardSpec; import org.easymock.EasyMock; +import org.joda.time.Period; import org.junit.After; import org.junit.Assert; import org.junit.Before; @@ -332,6 +334,9 @@ public void testBrokerPollsAllDSSchema() throws InterruptedException ArgumentCaptor> argumentCaptor = ArgumentCaptor.forClass(Set.class); CoordinatorClient coordinatorClient = Mockito.mock(CoordinatorClient.class); Mockito.when(coordinatorClient.fetchDataSourceInformation(argumentCaptor.capture())).thenReturn(Futures.immediateFuture(null)); + + Set datsources = Sets.newHashSet(DATASOURCE1, DATASOURCE2, DATASOURCE3, SOME_DATASOURCE, "xyz", "coldDS"); + Mockito.when(coordinatorClient.fetchDataSourcesWithUsedSegments()).thenReturn(Futures.immediateFuture(datsources)); BrokerSegmentMetadataCache schema = new BrokerSegmentMetadataCache( CalciteTests.createMockQueryLifecycleFactory(walker, conglomerate), serverView, @@ -347,7 +352,7 @@ public void testBrokerPollsAllDSSchema() throws InterruptedException schema.start(); schema.awaitInitialization(); - Assert.assertEquals(Sets.newHashSet(DATASOURCE1, DATASOURCE2, DATASOURCE3, SOME_DATASOURCE), argumentCaptor.getValue()); + Assert.assertEquals(datsources, argumentCaptor.getValue()); refreshLatch = new CountDownLatch(1); serverView.addSegment(newSegment("xyz", 0), ServerType.HISTORICAL); @@ -355,7 +360,87 @@ public void testBrokerPollsAllDSSchema() throws InterruptedException refreshLatch.await(WAIT_TIMEOUT_SECS, TimeUnit.SECONDS); // verify that previously refreshed are included in the last coordinator poll - Assert.assertEquals(Sets.newHashSet(DATASOURCE1, DATASOURCE2, DATASOURCE3, SOME_DATASOURCE, "xyz"), argumentCaptor.getValue()); + Assert.assertEquals(datsources, argumentCaptor.getValue()); + } + + @Test + public void testRefreshOnEachCycleCentralizedDatasourceSchemaEnabled() throws InterruptedException + { + CentralizedDatasourceSchemaConfig config = CentralizedDatasourceSchemaConfig.create(); + config.setEnabled(true); + + serverView = new TestTimelineServerView(walker.getSegments(), Collections.emptyList()); + druidServers = serverView.getDruidServers(); + + BrokerSegmentMetadataCacheConfig metadataCacheConfig = BrokerSegmentMetadataCacheConfig.create("PT1S"); + metadataCacheConfig.setMetadataRefreshPeriod(Period.parse("PT0.001S")); + BrokerSegmentMetadataCache schema = new BrokerSegmentMetadataCache( + CalciteTests.createMockQueryLifecycleFactory(walker, conglomerate), + serverView, + SEGMENT_CACHE_CONFIG_DEFAULT, + new NoopEscalator(), + new InternalQueryConfig(), + new NoopServiceEmitter(), + new PhysicalDatasourceMetadataFactory(globalTableJoinable, segmentManager), + new NoopCoordinatorClient(), + config + ) { + @Override + public void refresh(Set segmentsToRefresh, Set dataSourcesToRebuild) + throws IOException + { + super.refresh(segmentsToRefresh, dataSourcesToRebuild); + refreshLatch.countDown(); + } + }; + + // refresh should be executed more than once, with the feature disabled refresh should be executed only once + refreshLatch = new CountDownLatch(3); + schema.start(); + schema.awaitInitialization(); + + refreshLatch.await(WAIT_TIMEOUT_SECS, TimeUnit.SECONDS); + + Assert.assertEquals(0, refreshLatch.getCount()); + } + + @Test + public void testRefreshOnEachCycleCentralizedDatasourceSchemaDisabled() throws InterruptedException + { + BrokerSegmentMetadataCacheConfig metadataCacheConfig = BrokerSegmentMetadataCacheConfig.create("PT1S"); + metadataCacheConfig.setMetadataRefreshPeriod(Period.parse("PT0.001S")); + + serverView = new TestTimelineServerView(walker.getSegments(), Collections.emptyList()); + druidServers = serverView.getDruidServers(); + + BrokerSegmentMetadataCache schema = new BrokerSegmentMetadataCache( + CalciteTests.createMockQueryLifecycleFactory(walker, conglomerate), + serverView, + SEGMENT_CACHE_CONFIG_DEFAULT, + new NoopEscalator(), + new InternalQueryConfig(), + new NoopServiceEmitter(), + new PhysicalDatasourceMetadataFactory(globalTableJoinable, segmentManager), + new NoopCoordinatorClient(), + CentralizedDatasourceSchemaConfig.create() + ) { + @Override + public void refresh(Set segmentsToRefresh, Set dataSourcesToRebuild) + throws IOException + { + super.refresh(segmentsToRefresh, dataSourcesToRebuild); + refreshLatch.countDown(); + } + }; + + // refresh should be executed only once + refreshLatch = new CountDownLatch(3); + schema.start(); + schema.awaitInitialization(); + + refreshLatch.await(WAIT_TIMEOUT_SECS, TimeUnit.SECONDS); + + Assert.assertEquals(2, refreshLatch.getCount()); } @Test From 78a4a09d0181fc641f024d0c22dc6534803e7c76 Mon Sep 17 00:00:00 2001 From: Sree Charan Manamala Date: Mon, 15 Jul 2024 16:13:27 +0530 Subject: [PATCH 5/7] Window Function offset correction for RAC (#16718) * When an ArrayList RAC creates a child RAC, the start and end offsets need to have the offset of parent's start offset * Defaults the 2nd window bound to CURRENT ROW when only a single bound is specified * Removes the windowingStrictValidation warning and throws a hard exception when Order By alongside RANGE clause is not provided with UNBOUNDED or CURRENT ROW as both bounds --- .../sql/VarianceSqlAggregatorTest.java | 3 +- .../org/apache/druid/query/QueryContext.java | 8 -- .../org/apache/druid/query/QueryContexts.java | 2 - .../rowsandcols/ArrayListRowsAndColumns.java | 4 +- .../apache/druid/query/QueryContextsTest.java | 9 -- .../ArrayListRowsAndColumnsTest.java | 39 ++++++++- .../calcite/planner/DruidSqlValidator.java | 32 ++++---- .../druid/sql/calcite/CalciteQueryTest.java | 2 +- .../sql/calcite/CalciteWindowQueryTest.java | 12 +-- .../window/defaultBoundCurrentRow.sqlTest | 82 +++++++++++++++++++ .../tests/window/range_handling.sqlTest | 17 ++-- ...ipediaAggregationsMultipleOrdering.sqlTest | 8 +- ...iaAggregationsMultipleOrderingDesc.sqlTest | 2 +- .../wikipediaFramedAggregations.sqlTest | 6 +- 14 files changed, 158 insertions(+), 68 deletions(-) create mode 100644 sql/src/test/resources/calcite/tests/window/defaultBoundCurrentRow.sqlTest diff --git a/extensions-core/stats/src/test/java/org/apache/druid/query/aggregation/variance/sql/VarianceSqlAggregatorTest.java b/extensions-core/stats/src/test/java/org/apache/druid/query/aggregation/variance/sql/VarianceSqlAggregatorTest.java index f3e6406ecf38..687e26e9ff27 100644 --- a/extensions-core/stats/src/test/java/org/apache/druid/query/aggregation/variance/sql/VarianceSqlAggregatorTest.java +++ b/extensions-core/stats/src/test/java/org/apache/druid/query/aggregation/variance/sql/VarianceSqlAggregatorTest.java @@ -712,8 +712,7 @@ public void testOverWindow() + "group by dim4, dim5, mod(m1, 3)") .queryContext(ImmutableMap.of( PlannerContext.CTX_ENABLE_WINDOW_FNS, true, - QueryContexts.ENABLE_DEBUG, true, - QueryContexts.WINDOWING_STRICT_VALIDATION, false + QueryContexts.ENABLE_DEBUG, true )) .expectedResults(ImmutableList.of( new Object[]{"a", "aa", 1.0D, 0.0D}, diff --git a/processing/src/main/java/org/apache/druid/query/QueryContext.java b/processing/src/main/java/org/apache/druid/query/QueryContext.java index daa6760f8f32..8b2f7e27e686 100644 --- a/processing/src/main/java/org/apache/druid/query/QueryContext.java +++ b/processing/src/main/java/org/apache/druid/query/QueryContext.java @@ -613,14 +613,6 @@ public boolean isTimeBoundaryPlanningEnabled() ); } - public boolean isWindowingStrictValidation() - { - return getBoolean( - QueryContexts.WINDOWING_STRICT_VALIDATION, - QueryContexts.DEFAULT_WINDOWING_STRICT_VALIDATION - ); - } - public boolean isCatalogValidationEnabled() { return getBoolean( diff --git a/processing/src/main/java/org/apache/druid/query/QueryContexts.java b/processing/src/main/java/org/apache/druid/query/QueryContexts.java index 61520a04bc28..402001efe7d0 100644 --- a/processing/src/main/java/org/apache/druid/query/QueryContexts.java +++ b/processing/src/main/java/org/apache/druid/query/QueryContexts.java @@ -87,7 +87,6 @@ public class QueryContexts public static final String SERIALIZE_DATE_TIME_AS_LONG_INNER_KEY = "serializeDateTimeAsLongInner"; public static final String UNCOVERED_INTERVALS_LIMIT_KEY = "uncoveredIntervalsLimit"; public static final String MIN_TOP_N_THRESHOLD = "minTopNThreshold"; - public static final String WINDOWING_STRICT_VALIDATION = "windowingStrictValidation"; public static final String CATALOG_VALIDATION_ENABLED = "catalogValidationEnabled"; // Unique identifier for the query, that is used to map the global shared resources (specifically merge buffers) to the // query's runtime @@ -126,7 +125,6 @@ public class QueryContexts public static final int DEFAULT_IN_FUNCTION_THRESHOLD = 100; public static final int DEFAULT_IN_FUNCTION_EXPR_THRESHOLD = 2; public static final boolean DEFAULT_ENABLE_TIME_BOUNDARY_PLANNING = false; - public static final boolean DEFAULT_WINDOWING_STRICT_VALIDATION = true; public static final boolean DEFAULT_CATALOG_VALIDATION_ENABLED = true; @SuppressWarnings("unused") // Used by Jackson serialization diff --git a/processing/src/main/java/org/apache/druid/query/rowsandcols/ArrayListRowsAndColumns.java b/processing/src/main/java/org/apache/druid/query/rowsandcols/ArrayListRowsAndColumns.java index 6f5460095113..04f9eddbff0c 100644 --- a/processing/src/main/java/org/apache/druid/query/rowsandcols/ArrayListRowsAndColumns.java +++ b/processing/src/main/java/org/apache/druid/query/rowsandcols/ArrayListRowsAndColumns.java @@ -259,8 +259,8 @@ private ArrayListRowsAndColumns limited(int startOffset, int endOffset) rowSignature, extraColumns, columnNames, - startOffset, - endOffset + this.startOffset + startOffset, + this.startOffset + endOffset ); } diff --git a/processing/src/test/java/org/apache/druid/query/QueryContextsTest.java b/processing/src/test/java/org/apache/druid/query/QueryContextsTest.java index 2fa414188a11..554ca7b8f4a5 100644 --- a/processing/src/test/java/org/apache/druid/query/QueryContextsTest.java +++ b/processing/src/test/java/org/apache/druid/query/QueryContextsTest.java @@ -151,15 +151,6 @@ public void testDefaultPlanTimeBoundarySql() ); } - @Test - public void testDefaultWindowingStrictValidation() - { - Assert.assertEquals( - QueryContexts.DEFAULT_WINDOWING_STRICT_VALIDATION, - QueryContext.empty().isWindowingStrictValidation() - ); - } - @Test public void testCatalogValidationEnabled() { diff --git a/processing/src/test/java/org/apache/druid/query/rowsandcols/ArrayListRowsAndColumnsTest.java b/processing/src/test/java/org/apache/druid/query/rowsandcols/ArrayListRowsAndColumnsTest.java index dd492ae407f6..b709678ec17e 100644 --- a/processing/src/test/java/org/apache/druid/query/rowsandcols/ArrayListRowsAndColumnsTest.java +++ b/processing/src/test/java/org/apache/druid/query/rowsandcols/ArrayListRowsAndColumnsTest.java @@ -19,12 +19,18 @@ package org.apache.druid.query.rowsandcols; +import com.google.common.collect.ImmutableMap; import org.apache.druid.java.util.common.ISE; import org.apache.druid.query.rowsandcols.column.ColumnAccessor; +import org.apache.druid.query.rowsandcols.column.IntArrayColumn; import org.apache.druid.segment.column.RowSignature; +import org.junit.Assert; +import org.junit.Test; import javax.annotation.Nonnull; import java.util.ArrayList; +import java.util.Collections; +import java.util.Objects; import java.util.function.Function; public class ArrayListRowsAndColumnsTest extends RowsAndColumnsTestBase @@ -35,7 +41,10 @@ public ArrayListRowsAndColumnsTest() } @Nonnull - public static Function> MAKER = input -> { + public static Function> MAKER = input -> buildRAC(input); + + public static ArrayListRowsAndColumns buildRAC(MapOfColumnsRowsAndColumns input) + { ArrayList rows = new ArrayList<>(input.numRows()); ArrayList cols = new ArrayList<>(input.getColumnNames()); @@ -47,7 +56,7 @@ public ArrayListRowsAndColumnsTest() for (int colIndex = 0; colIndex < cols.size(); ++colIndex) { String col = cols.get(colIndex); - final ColumnAccessor column = input.findColumn(col).toAccessor(); + final ColumnAccessor column = Objects.requireNonNull(input.findColumn(col)).toAccessor(); sigBob.add(col, column.getType()); for (int i = 0; i < column.numRows(); ++i) { @@ -66,5 +75,29 @@ public ArrayListRowsAndColumnsTest() }, sigBob.build() ); - }; + } + + @Test + public void testChildRAC() + { + MapOfColumnsRowsAndColumns input = MapOfColumnsRowsAndColumns.fromMap( + ImmutableMap.of( + "colA", new IntArrayColumn(new int[]{1, 1, 1, 1, 2, 2, 2, 2, 2, 2}), + "colB", new IntArrayColumn(new int[]{3, 3, 4, 4, 5, 5, 5, 6, 6, 7}) + ) + ); + + ArrayListRowsAndColumns rac = ArrayListRowsAndColumnsTest.buildRAC(input); + ArrayList childRACs = rac.toClusteredGroupPartitioner() + .partitionOnBoundaries(Collections.singletonList("colA")); + + Assert.assertEquals(2, childRACs.size()); + ArrayListRowsAndColumns childRAC = (ArrayListRowsAndColumns) childRACs.get(1); + ArrayListRowsAndColumns curChildRAC = (ArrayListRowsAndColumns) childRAC.toClusteredGroupPartitioner() + .partitionOnBoundaries(Collections.singletonList( + "colB")) + .get(0); + + Assert.assertEquals(5, curChildRAC.findColumn("colB").toAccessor().getInt(0)); + } } diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/planner/DruidSqlValidator.java b/sql/src/main/java/org/apache/druid/sql/calcite/planner/DruidSqlValidator.java index d399786676c8..75778daf5593 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/planner/DruidSqlValidator.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/planner/DruidSqlValidator.java @@ -61,7 +61,6 @@ import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.query.QueryContext; -import org.apache.druid.query.QueryContexts; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.Types; import org.apache.druid.segment.column.ValueType; @@ -123,7 +122,6 @@ public void validateWindow(SqlNode windowOrId, SqlValidatorScope scope, @Nullabl throw Util.unexpected(windowOrId.getKind()); } - @Nullable SqlNode lowerBound = targetWindow.getLowerBound(); @Nullable @@ -135,6 +133,17 @@ public void validateWindow(SqlNode windowOrId, SqlValidatorScope scope, @Nullabl ); } + if (lowerBound != null && upperBound == null) { + if (lowerBound.getKind() == SqlKind.FOLLOWING || SqlWindow.isUnboundedFollowing(lowerBound)) { + upperBound = lowerBound; + lowerBound = SqlWindow.createCurrentRow(SqlParserPos.ZERO); + } else { + upperBound = SqlWindow.createCurrentRow(SqlParserPos.ZERO); + } + targetWindow.setLowerBound(lowerBound); + targetWindow.setUpperBound(upperBound); + } + boolean hasBounds = lowerBound != null || upperBound != null; if (call.getKind() == SqlKind.NTILE && hasBounds) { throw buildCalciteContextException( @@ -152,18 +161,13 @@ public void validateWindow(SqlNode windowOrId, SqlValidatorScope scope, @Nullabl } } - if (plannerContext.queryContext().isWindowingStrictValidation()) { - if (!targetWindow.isRows() && - (!isUnboundedOrCurrent(lowerBound) || !isUnboundedOrCurrent(upperBound))) { - // this limitation can be lifted when https://github.com/apache/druid/issues/15767 is addressed - throw buildCalciteContextException( - StringUtils.format( - "The query contains a window frame which may return incorrect results. To disregard this warning, set [%s] to false in the query context.", - QueryContexts.WINDOWING_STRICT_VALIDATION - ), - windowOrId - ); - } + if (!targetWindow.isRows() && + (!isUnboundedOrCurrent(lowerBound) || !isUnboundedOrCurrent(upperBound))) { + // this limitation can be lifted when https://github.com/apache/druid/issues/15767 is addressed + throw buildCalciteContextException( + "Order By with RANGE clause currently supports only UNBOUNDED or CURRENT ROW. Use ROWS clause instead.", + windowOrId + ); } super.validateWindow(windowOrId, scope, call); 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 ec40fb3f871d..2ae095d41c74 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 @@ -15546,7 +15546,7 @@ public void testUnSupportedRangeBounds() .queryContext(ImmutableMap.of(PlannerContext.CTX_ENABLE_WINDOW_FNS, true)) .sql("SELECT dim1,ROW_NUMBER() OVER (ORDER BY dim1 RANGE BETWEEN 3 PRECEDING AND 2 FOLLOWING) from druid.foo") .run()); - assertThat(e, invalidSqlIs("The query contains a window frame which may return incorrect results. To disregard this warning, set [windowingStrictValidation] to false in the query context. (line [1], column [31])")); + assertThat(e, invalidSqlIs("Order By with RANGE clause currently supports only UNBOUNDED or CURRENT ROW. Use ROWS clause instead. (line [1], column [31])")); } @Test diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteWindowQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteWindowQueryTest.java index cdc4bc9cbf98..b3d657b148f1 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteWindowQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteWindowQueryTest.java @@ -201,8 +201,7 @@ public void windowQueryTest(String filename) throws Exception .sql(testCase.getSql()) .queryContext(ImmutableMap.of( PlannerContext.CTX_ENABLE_WINDOW_FNS, true, - QueryContexts.ENABLE_DEBUG, true, - QueryContexts.WINDOWING_STRICT_VALIDATION, false + QueryContexts.ENABLE_DEBUG, true )) .addCustomVerification(QueryVerification.ofResults(testCase)) .run(); @@ -224,8 +223,7 @@ public void windowQueryTestWithCustomContextMaxSubqueryBytes(String filename) th .sql(testCase.getSql()) .queryContext(ImmutableMap.of(QueryContexts.ENABLE_DEBUG, true, PlannerContext.CTX_ENABLE_WINDOW_FNS, true, - QueryContexts.MAX_SUBQUERY_BYTES_KEY, "100000", - QueryContexts.WINDOWING_STRICT_VALIDATION, false + QueryContexts.MAX_SUBQUERY_BYTES_KEY, "100000" ) ) .addCustomVerification(QueryVerification.ofResults(testCase)) @@ -246,8 +244,7 @@ public void testEmptyWindowInSubquery() ) .queryContext(ImmutableMap.of( PlannerContext.CTX_ENABLE_WINDOW_FNS, true, - QueryContexts.ENABLE_DEBUG, true, - QueryContexts.WINDOWING_STRICT_VALIDATION, false + QueryContexts.ENABLE_DEBUG, true )) .expectedResults(ImmutableList.of( new Object[]{1L}, @@ -269,8 +266,7 @@ public void testWindow() "FROM \"wikipedia\"") .queryContext(ImmutableMap.of( PlannerContext.CTX_ENABLE_WINDOW_FNS, true, - QueryContexts.ENABLE_DEBUG, true, - QueryContexts.WINDOWING_STRICT_VALIDATION, false + QueryContexts.ENABLE_DEBUG, true )) .expectedResults(ImmutableList.of( new Object[]{1L, 1L} diff --git a/sql/src/test/resources/calcite/tests/window/defaultBoundCurrentRow.sqlTest b/sql/src/test/resources/calcite/tests/window/defaultBoundCurrentRow.sqlTest new file mode 100644 index 000000000000..d5a324c9e2d8 --- /dev/null +++ b/sql/src/test/resources/calcite/tests/window/defaultBoundCurrentRow.sqlTest @@ -0,0 +1,82 @@ +type: "operatorValidation" + +sql: | + SELECT + dim2, + count(*) OVER (partition by dim2 ORDER BY dim1 ROWS UNBOUNDED PRECEDING), + count(*) OVER (partition by dim2 ORDER BY dim1 ROWS 1 PRECEDING), + count(*) OVER (partition by dim2 ORDER BY dim1 ROWS CURRENT ROW), + count(*) OVER (partition by dim2 ORDER BY dim1 ROWS 1 FOLLOWING), + count(*) OVER (partition by dim2 ORDER BY dim1 ROWS UNBOUNDED FOLLOWING) + FROM numfoo + WHERE dim2 IN ('a', 'abc') + GROUP BY dim2, dim1 + +expectedOperators: + - {"type":"naiveSort","columns":[{"column":"_d1","direction":"ASC"},{"column":"_d0","direction":"ASC"}]} + - { type: "naivePartition", partitionColumns: [ "_d1" ] } + - type: "window" + processor: + type: "framedAgg" + frame: + peerType: "ROWS" + lowUnbounded: true + lowOffset: 0 + uppUnbounded: false + uppOffset: 0 + orderBy: null + aggregations: + - { type: "count", name: "w0" } + - type: "window" + processor: + type: "framedAgg" + frame: + peerType: "ROWS" + lowUnbounded: false + lowOffset: -1 + uppUnbounded: false + uppOffset: 0 + orderBy: null + aggregations: + - { type: "count", name: "w1" } + - type: "window" + processor: + type: "framedAgg" + frame: + peerType: "ROWS" + lowUnbounded: false + lowOffset: 0 + uppUnbounded: false + uppOffset: 0 + orderBy: null + aggregations: + - { type: "count", name: "w2" } + - type: "window" + processor: + type: "framedAgg" + frame: + peerType: "ROWS" + lowUnbounded: false + lowOffset: 0 + uppUnbounded: false + uppOffset: 1 + orderBy: null + aggregations: + - { type: "count", name: "w3" } + - type: "window" + processor: + type: "framedAgg" + frame: + peerType: "ROWS" + lowUnbounded: false + lowOffset: 0 + uppUnbounded: true + uppOffset: 0 + orderBy: null + aggregations: + - { type: "count", name: "w4" } + +expectedResults: + - ["a",1,1,1,2,2] + - ["a",2,2,1,1,1] + - ["abc",1,1,1,1,1] diff --git a/sql/src/test/resources/calcite/tests/window/range_handling.sqlTest b/sql/src/test/resources/calcite/tests/window/range_handling.sqlTest index 717778d1fe28..405886af2303 100644 --- a/sql/src/test/resources/calcite/tests/window/range_handling.sqlTest +++ b/sql/src/test/resources/calcite/tests/window/range_handling.sqlTest @@ -9,16 +9,13 @@ sql: | COUNT(1) OVER (ORDER BY FLOOR(m1/3) RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW), 'postfix', COUNT(1) OVER (ORDER BY FLOOR(m1/3) ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING), - COUNT(1) OVER (ORDER BY FLOOR(m1/3) RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING), - 'k(1)', - COUNT(1) OVER (ORDER BY FLOOR(m1/3) ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING), - COUNT(1) OVER (ORDER BY FLOOR(m1/3) RANGE BETWEEN 1 PRECEDING AND 1 FOLLOWING) + COUNT(1) OVER (ORDER BY FLOOR(m1/3) RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) FROM foo expectedResults: - - [0.0,1,"prefix",1,2,"postfix",6,6,"k(1)",2,5] - - [0.0,1,"prefix",2,2,"postfix",5,6,"k(1)",3,5] - - [1.0,2,"prefix",3,5,"postfix",4,4,"k(1)",3,6] - - [1.0,2,"prefix",4,5,"postfix",3,4,"k(1)",3,6] - - [1.0,2,"prefix",5,5,"postfix",2,4,"k(1)",3,6] - - [2.0,3,"prefix",6,6,"postfix",1,1,"k(1)",2,4] + - [0.0,1,"prefix",1,2,"postfix",6,6] + - [0.0,1,"prefix",2,2,"postfix",5,6] + - [1.0,2,"prefix",3,5,"postfix",4,4] + - [1.0,2,"prefix",4,5,"postfix",3,4] + - [1.0,2,"prefix",5,5,"postfix",2,4] + - [2.0,3,"prefix",6,6,"postfix",1,1] diff --git a/sql/src/test/resources/calcite/tests/window/wikipediaAggregationsMultipleOrdering.sqlTest b/sql/src/test/resources/calcite/tests/window/wikipediaAggregationsMultipleOrdering.sqlTest index b8da0dbd6580..cc59868482a2 100644 --- a/sql/src/test/resources/calcite/tests/window/wikipediaAggregationsMultipleOrdering.sqlTest +++ b/sql/src/test/resources/calcite/tests/window/wikipediaAggregationsMultipleOrdering.sqlTest @@ -5,7 +5,7 @@ sql: | countryIsoCode, CAST (FLOOR(__time TO HOUR) AS BIGINT) t, SUM(delta) delta, - SUM(SUM(delta)) OVER (PARTITION BY countryIsoCode ORDER BY CAST (FLOOR(__time TO HOUR) AS BIGINT) RANGE BETWEEN 3 PRECEDING AND 2 FOLLOWING) windowedDelta, + SUM(SUM(delta)) OVER (PARTITION BY countryIsoCode ORDER BY CAST (FLOOR(__time TO HOUR) AS BIGINT) ROWS BETWEEN 3 PRECEDING AND 2 FOLLOWING) windowedDelta, ROW_NUMBER() OVER (PARTITION BY CAST (FLOOR(__time TO HOUR) AS BIGINT) ORDER BY SUM(delta)) AS hourlyRank FROM wikipedia GROUP BY 1, 2 @@ -16,14 +16,12 @@ expectedOperators: processor: type: "framedAgg" frame: - peerType: "RANGE" + peerType: "ROWS" lowUnbounded: false lowOffset: -3 uppUnbounded: false uppOffset: 2 - orderBy: - - column: d1 - direction: ASC + orderBy: null aggregations: - { type: "longSum", name: "w0", fieldName: "a0" } - { type: "naiveSort", columns: [ { column: "d1", direction: "ASC" }, { column: "a0", direction: "ASC"} ]} diff --git a/sql/src/test/resources/calcite/tests/window/wikipediaAggregationsMultipleOrderingDesc.sqlTest b/sql/src/test/resources/calcite/tests/window/wikipediaAggregationsMultipleOrderingDesc.sqlTest index 0180f6153130..b16aa21d303e 100644 --- a/sql/src/test/resources/calcite/tests/window/wikipediaAggregationsMultipleOrderingDesc.sqlTest +++ b/sql/src/test/resources/calcite/tests/window/wikipediaAggregationsMultipleOrderingDesc.sqlTest @@ -5,7 +5,7 @@ sql: | countryIsoCode, CAST (FLOOR(__time TO HOUR) AS BIGINT) t, SUM(delta) delta, - SUM(SUM(delta)) OVER (PARTITION BY countryIsoCode ORDER BY CAST (FLOOR(__time TO HOUR) AS BIGINT) DESC RANGE BETWEEN 3 PRECEDING AND 2 FOLLOWING) windowedDelta, + SUM(SUM(delta)) OVER (PARTITION BY countryIsoCode ORDER BY CAST (FLOOR(__time TO HOUR) AS BIGINT) DESC ROWS BETWEEN 3 PRECEDING AND 2 FOLLOWING) windowedDelta, ROW_NUMBER() OVER (PARTITION BY CAST (FLOOR(__time TO HOUR) AS BIGINT) ORDER BY SUM(delta) DESC) AS hourlyRank FROM wikipedia GROUP BY 1, 2 diff --git a/sql/src/test/resources/calcite/tests/window/wikipediaFramedAggregations.sqlTest b/sql/src/test/resources/calcite/tests/window/wikipediaFramedAggregations.sqlTest index 07cfadfb680f..c25f1ff03528 100644 --- a/sql/src/test/resources/calcite/tests/window/wikipediaFramedAggregations.sqlTest +++ b/sql/src/test/resources/calcite/tests/window/wikipediaFramedAggregations.sqlTest @@ -5,7 +5,7 @@ sql: | countryIsoCode, CAST (FLOOR(__time TO HOUR) AS BIGINT) t, SUM(delta) delta, - SUM(SUM(delta)) OVER (PARTITION BY countryIsoCode ORDER BY CAST (FLOOR(__time TO HOUR) AS BIGINT) RANGE BETWEEN 3 PRECEDING AND 2 FOLLOWING) windowedDelta + SUM(SUM(delta)) OVER (PARTITION BY countryIsoCode ORDER BY CAST (FLOOR(__time TO HOUR) AS BIGINT) ROWS BETWEEN 3 PRECEDING AND 2 FOLLOWING) windowedDelta FROM wikipedia GROUP BY 1, 2 @@ -15,12 +15,12 @@ expectedOperators: processor: type: "framedAgg" frame: - peerType: "RANGE" + peerType: "ROWS" lowUnbounded: false lowOffset: -3 uppUnbounded: false uppOffset: 2 - orderBy: [ {column: "d1", direction: ASC} ] + orderBy: null aggregations: - { type: "longSum", name: "w0", fieldName: "a0" } From 6891866c4341a66074db3908204bcf673f30143f Mon Sep 17 00:00:00 2001 From: AmatyaAvadhanula Date: Mon, 15 Jul 2024 18:24:23 +0530 Subject: [PATCH 6/7] Process retrieval of parent and child segment ids in batches (#16734) --- .../IndexerSQLMetadataStorageCoordinator.java | 99 ++++++++++--------- ...exerSQLMetadataStorageCoordinatorTest.java | 93 +++++++++++++++++ 2 files changed, 144 insertions(+), 48 deletions(-) diff --git a/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java b/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java index 54f75ccb9201..ecfad572e745 100644 --- a/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java +++ b/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java @@ -2954,28 +2954,30 @@ public Map retrieveUpgradedFromSegmentIds( return Collections.emptyMap(); } - final List segmentIdList = ImmutableList.copyOf(segmentIds); - final String sql = StringUtils.format( - "SELECT id, upgraded_from_segment_id FROM %s WHERE dataSource = :dataSource %s", - dbTables.getSegmentsTable(), - SqlSegmentsMetadataQuery.getParameterizedInConditionForColumn("id", segmentIdList) - ); final Map upgradedFromSegmentIds = new HashMap<>(); - connector.retryWithHandle( - handle -> { - Query> query = handle.createQuery(sql) - .bind("dataSource", dataSource); - SqlSegmentsMetadataQuery.bindColumnValuesToQueryWithInCondition("id", segmentIdList, query); - return query.map((index, r, ctx) -> { - final String id = r.getString(1); - final String upgradedFromSegmentId = r.getString(2); - if (upgradedFromSegmentId != null) { - upgradedFromSegmentIds.put(id, upgradedFromSegmentId); - } - return null; - }).list(); - } - ); + final List> partitions = Lists.partition(ImmutableList.copyOf(segmentIds), 100); + for (List partition : partitions) { + final String sql = StringUtils.format( + "SELECT id, upgraded_from_segment_id FROM %s WHERE dataSource = :dataSource %s", + dbTables.getSegmentsTable(), + SqlSegmentsMetadataQuery.getParameterizedInConditionForColumn("id", partition) + ); + connector.retryWithHandle( + handle -> { + Query> query = handle.createQuery(sql) + .bind("dataSource", dataSource); + SqlSegmentsMetadataQuery.bindColumnValuesToQueryWithInCondition("id", partition, query); + return query.map((index, r, ctx) -> { + final String id = r.getString(1); + final String upgradedFromSegmentId = r.getString(2); + if (upgradedFromSegmentId != null) { + upgradedFromSegmentIds.put(id, upgradedFromSegmentId); + } + return null; + }).list(); + } + ); + } return upgradedFromSegmentIds; } @@ -2989,39 +2991,40 @@ public Map> retrieveUpgradedToSegmentIds( return Collections.emptyMap(); } - final List upgradedFromSegmentIdList = ImmutableList.copyOf(segmentIds); - final String sql = StringUtils.format( - "SELECT id, upgraded_from_segment_id FROM %s WHERE dataSource = :dataSource %s", - dbTables.getSegmentsTable(), - SqlSegmentsMetadataQuery.getParameterizedInConditionForColumn( - "upgraded_from_segment_id", - upgradedFromSegmentIdList - ) - ); final Map> upgradedToSegmentIds = new HashMap<>(); retrieveSegmentsById(dataSource, segmentIds) .stream() .map(DataSegment::getId) .map(SegmentId::toString) .forEach(id -> upgradedToSegmentIds.computeIfAbsent(id, k -> new HashSet<>()).add(id)); - connector.retryWithHandle( - handle -> { - Query> query = handle.createQuery(sql) - .bind("dataSource", dataSource); - SqlSegmentsMetadataQuery.bindColumnValuesToQueryWithInCondition( - "upgraded_from_segment_id", - upgradedFromSegmentIdList, - query - ); - return query.map((index, r, ctx) -> { - final String upgradedToId = r.getString(1); - final String id = r.getString(2); - upgradedToSegmentIds.computeIfAbsent(id, k -> new HashSet<>()) - .add(upgradedToId); - return null; - }).list(); - } - ); + + final List> partitions = Lists.partition(ImmutableList.copyOf(segmentIds), 100); + for (List partition : partitions) { + final String sql = StringUtils.format( + "SELECT id, upgraded_from_segment_id FROM %s WHERE dataSource = :dataSource %s", + dbTables.getSegmentsTable(), + SqlSegmentsMetadataQuery.getParameterizedInConditionForColumn("upgraded_from_segment_id", partition) + ); + + connector.retryWithHandle( + handle -> { + Query> query = handle.createQuery(sql) + .bind("dataSource", dataSource); + SqlSegmentsMetadataQuery.bindColumnValuesToQueryWithInCondition( + "upgraded_from_segment_id", + partition, + query + ); + return query.map((index, r, ctx) -> { + final String upgradedToId = r.getString(1); + final String id = r.getString(2); + upgradedToSegmentIds.computeIfAbsent(id, k -> new HashSet<>()) + .add(upgradedToId); + return null; + }).list(); + } + ); + } return upgradedToSegmentIds; } diff --git a/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java b/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java index f352d5e2609d..6eccbccaa849 100644 --- a/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java +++ b/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java @@ -3452,6 +3452,48 @@ public void testRetrieveUpgradedFromSegmentIds() ); } + @Test + public void testRetrieveUpgradedFromSegmentIdsInBatches() + { + final int size = 500; + final int batchSize = 100; + + List segments = new ArrayList<>(); + for (int i = 0; i < size; i++) { + segments.add( + new DataSegment( + "DS", + Intervals.ETERNITY, + "v " + (i % 5), + ImmutableMap.of("num", i / 5), + ImmutableList.of("dim"), + ImmutableList.of("agg"), + new NumberedShardSpec(i / 5, 0), + 0, + 100L + ) + ); + } + Map expected = new HashMap<>(); + for (int i = 0; i < batchSize; i++) { + for (int j = 1; j < 5; j++) { + expected.put( + segments.get(5 * i + j).getId().toString(), + segments.get(5 * i).getId().toString() + ); + } + } + insertUsedSegments(ImmutableSet.copyOf(segments), expected); + + Map actual = coordinator.retrieveUpgradedFromSegmentIds( + "DS", + segments.stream().map(DataSegment::getId).map(SegmentId::toString).collect(Collectors.toSet()) + ); + + Assert.assertEquals(400, actual.size()); + Assert.assertEquals(expected, actual); + } + @Test public void testRetrieveUpgradedToSegmentIds() { @@ -3478,6 +3520,57 @@ public void testRetrieveUpgradedToSegmentIds() ); } + @Test + public void testRetrieveUpgradedToSegmentIdsInBatches() + { + final int size = 500; + final int batchSize = 100; + + List segments = new ArrayList<>(); + for (int i = 0; i < size; i++) { + segments.add( + new DataSegment( + "DS", + Intervals.ETERNITY, + "v " + (i % 5), + ImmutableMap.of("num", i / 5), + ImmutableList.of("dim"), + ImmutableList.of("agg"), + new NumberedShardSpec(i / 5, 0), + 0, + 100L + ) + ); + } + + Map> expected = new HashMap<>(); + for (DataSegment segment : segments) { + final String id = segment.getId().toString(); + expected.put(id, new HashSet<>()); + expected.get(id).add(id); + } + Map upgradeMap = new HashMap<>(); + for (int i = 0; i < batchSize; i++) { + for (int j = 1; j < 5; j++) { + upgradeMap.put( + segments.get(5 * i + j).getId().toString(), + segments.get(5 * i).getId().toString() + ); + expected.get(segments.get(5 * i).getId().toString()) + .add(segments.get(5 * i + j).getId().toString()); + } + } + insertUsedSegments(ImmutableSet.copyOf(segments), upgradeMap); + + Map> actual = coordinator.retrieveUpgradedToSegmentIds( + "DS", + segments.stream().map(DataSegment::getId).map(SegmentId::toString).collect(Collectors.toSet()) + ); + + Assert.assertEquals(500, actual.size()); + Assert.assertEquals(expected, actual); + } + private void insertUsedSegments(Set segments, Map upgradedFromSegmentIdMap) { final String table = derbyConnectorRule.metadataTablesConfigSupplier().get().getSegmentsTable(); From 6cf6838eb974634ce5ae664d5ca877d8c804036a Mon Sep 17 00:00:00 2001 From: Adithya Chakilam <35785271+adithyachakilam@users.noreply.github.com> Date: Mon, 15 Jul 2024 16:35:11 -0500 Subject: [PATCH 7/7] kubernetes-overlord-extension: Fix tasks not being shutdown (#16711) --- .../k8s/overlord/KubernetesTaskRunner.java | 14 ++++++++++---- .../k8s/overlord/KubernetesTaskRunnerTest.java | 18 ++++++++++++++++++ 2 files changed, 28 insertions(+), 4 deletions(-) diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunner.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunner.java index 080a0fdaa986..c324b49e13a2 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunner.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunner.java @@ -443,11 +443,17 @@ public Collection getPendingTasks() @Override public TaskLocation getTaskLocation(String taskId) { - final KubernetesWorkItem workItem = tasks.get(taskId); - if (workItem == null) { + try { + final KubernetesWorkItem workItem = tasks.get(taskId); + if (workItem == null) { + return TaskLocation.unknown(); + } else { + return workItem.getLocation(); + } + } + catch (Exception e) { + log.warn("Unable to find location for task [%s]", taskId); return TaskLocation.unknown(); - } else { - return workItem.getLocation(); } } diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerTest.java b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerTest.java index 391db70afb28..67a5278c6a32 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerTest.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerTest.java @@ -654,6 +654,24 @@ public TaskLocation getLocation() Assert.assertEquals(TaskLocation.create("host", 0, 1, false), taskLocation); } + @Test + public void test_getTaskLocation_throws() + { + KubernetesWorkItem workItem = new KubernetesWorkItem(task, null) + { + @Override + public TaskLocation getLocation() + { + throw new RuntimeException(); + } + }; + + runner.tasks.put(task.getId(), workItem); + + TaskLocation taskLocation = runner.getTaskLocation(task.getId()); + Assert.assertEquals(TaskLocation.unknown(), taskLocation); + } + @Test public void test_getTaskLocation_noTaskFound() {