From 682dd8ba71c92d887ee3cc50152db68bcf0929aa Mon Sep 17 00:00:00 2001 From: cecemei Date: Wed, 25 Feb 2026 19:02:29 -0800 Subject: [PATCH 01/26] incremental-compaction-mode --- .../compact/CompactionSupervisorTest.java | 285 ++++++++++++------ .../common/actions/SegmentUpgradeAction.java | 87 ++++++ .../indexing/common/actions/TaskAction.java | 1 + .../common/task/CompactionIntervalSpec.java | 50 ++- .../common/task/CompactionRunner.java | 6 +- .../indexing/common/task/CompactionTask.java | 89 +++++- .../common/task/NativeCompactionRunner.java | 36 +-- .../CompactionConfigBasedJobTemplate.java | 23 +- .../indexing/compact/CompactionJobQueue.java | 31 +- .../actions/SegmentUpgradeActionTest.java | 140 +++++++++ .../ClientCompactionTaskQuerySerdeTest.java | 2 +- .../common/task/CompactionTaskRunBase.java | 13 +- .../common/task/CompactionTaskTest.java | 73 ++--- .../task/NativeCompactionRunnerTest.java | 14 +- ...TestIndexerMetadataStorageCoordinator.java | 13 +- .../WikipediaStreamEventStreamGenerator.java | 18 ++ .../msq/indexing/MSQCompactionRunner.java | 32 +- .../msq/exec/MSQCompactionTaskRunTest.java | 165 ++++++++++ .../msq/indexing/MSQCompactionRunnerTest.java | 18 +- .../partition/DimensionRangeShardSpec.java | 18 ++ .../timeline/partition/LinearShardSpec.java | 12 + .../timeline/partition/NumberedShardSpec.java | 18 ++ .../druid/timeline/partition/ShardSpec.java | 27 ++ .../ClientCompactionIntervalSpec.java | 35 ++- .../IndexerMetadataStorageCoordinator.java | 23 ++ .../IndexerSQLMetadataStorageCoordinator.java | 135 ++++++--- .../compaction/CompactionCandidate.java | 7 + .../CompactionCandidateSearchPolicy.java | 5 + .../server/compaction/CompactionMode.java | 3 +- .../server/compaction/CompactionStatus.java | 32 +- .../compaction/CompactionStatusTracker.java | 30 +- .../MostFragmentedIntervalFirstPolicy.java | 57 +++- .../UserCompactionTaskQueryTuningConfig.java | 167 ++++++++++ .../coordinator/duty/CompactSegments.java | 31 +- .../ClientCompactionIntervalSpecTest.java | 42 +++ ...exerSQLMetadataStorageCoordinatorTest.java | 146 ++++++++- ...MostFragmentedIntervalFirstPolicyTest.java | 100 +++++- ...erCompactionTaskQueryTuningConfigTest.java | 79 ++--- .../coordinator/duty/CompactSegmentsTest.java | 1 + 39 files changed, 1727 insertions(+), 337 deletions(-) create mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentUpgradeAction.java create mode 100644 indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentUpgradeActionTest.java diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionSupervisorTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionSupervisorTest.java index e09be28eb517..5c85fc35c8f2 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionSupervisorTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionSupervisorTest.java @@ -23,6 +23,9 @@ import org.apache.druid.catalog.guice.CatalogClientModule; import org.apache.druid.catalog.guice.CatalogCoordinatorModule; import org.apache.druid.common.utils.IdUtils; +import org.apache.druid.data.input.impl.DimensionsSpec; +import org.apache.druid.data.input.impl.StringDimensionSchema; +import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.indexer.CompactionEngine; import org.apache.druid.indexer.partitions.DimensionRangePartitionsSpec; import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; @@ -32,9 +35,14 @@ import org.apache.druid.indexing.common.task.TaskBuilder; import org.apache.druid.indexing.compact.CascadingReindexingTemplate; import org.apache.druid.indexing.compact.CompactionSupervisorSpec; +import org.apache.druid.indexing.kafka.KafkaIndexTaskModule; +import org.apache.druid.indexing.kafka.simulate.KafkaResource; +import org.apache.druid.indexing.kafka.supervisor.KafkaSupervisorSpec; +import org.apache.druid.indexing.kafka.supervisor.KafkaSupervisorSpecBuilder; import org.apache.druid.indexing.overlord.Segments; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.java.util.common.HumanReadableBytes; import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.Granularities; @@ -53,7 +61,9 @@ import org.apache.druid.segment.metadata.IndexingStateFingerprintMapper; import org.apache.druid.segment.transform.CompactionTransformSpec; import org.apache.druid.segment.virtual.ExpressionVirtualColumn; +import org.apache.druid.server.compaction.CompactionCandidateSearchPolicy; import org.apache.druid.server.compaction.InlineReindexingRuleProvider; +import org.apache.druid.server.compaction.MostFragmentedIntervalFirstPolicy; import org.apache.druid.server.compaction.ReindexingDeletionRule; import org.apache.druid.server.compaction.ReindexingIOConfigRule; import org.apache.druid.server.compaction.ReindexingSegmentGranularityRule; @@ -61,6 +71,7 @@ import org.apache.druid.server.coordinator.ClusterCompactionConfig; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; import org.apache.druid.server.coordinator.InlineSchemaDataSourceCompactionConfig; +import org.apache.druid.server.coordinator.UserCompactionTaskDimensionsConfig; import org.apache.druid.server.coordinator.UserCompactionTaskGranularityConfig; import org.apache.druid.server.coordinator.UserCompactionTaskIOConfig; import org.apache.druid.server.coordinator.UserCompactionTaskQueryTuningConfig; @@ -73,6 +84,11 @@ import org.apache.druid.testing.embedded.EmbeddedRouter; import org.apache.druid.testing.embedded.indexing.MoreResources; import org.apache.druid.testing.embedded.junit5.EmbeddedClusterTestBase; +import org.apache.druid.testing.tools.EventSerializer; +import org.apache.druid.testing.tools.JsonEventSerializer; +import org.apache.druid.testing.tools.StreamGenerator; +import org.apache.druid.testing.tools.WikipediaStreamEventStreamGenerator; +import org.apache.kafka.clients.producer.ProducerRecord; import org.hamcrest.Matcher; import org.hamcrest.Matchers; import org.joda.time.DateTime; @@ -83,17 +99,20 @@ import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.MethodSource; +import javax.annotation.Nullable; import java.nio.charset.StandardCharsets; import java.time.Duration; import java.util.ArrayList; import java.util.List; import java.util.Map; +import java.util.stream.Collectors; /** * Embedded test that runs compaction supervisors of various types. */ public class CompactionSupervisorTest extends EmbeddedClusterTestBase { + private final KafkaResource kafkaServer = new KafkaResource(); private final EmbeddedBroker broker = new EmbeddedBroker(); private final EmbeddedIndexer indexer = new EmbeddedIndexer() .setServerMemory(2_000_000_000L) @@ -119,7 +138,12 @@ public EmbeddedDruidCluster createCluster() "[\"org.apache.druid.query.policy.NoRestrictionPolicy\"]" ) .addCommonProperty("druid.policy.enforcer.type", "restrictAllTables") - .addExtensions(CatalogClientModule.class, CatalogCoordinatorModule.class) + .addExtensions( + CatalogClientModule.class, + CatalogCoordinatorModule.class, + KafkaIndexTaskModule.class + ) + .addResource(kafkaServer) .addServer(coordinator) .addServer(overlord) .addServer(indexer) @@ -129,19 +153,28 @@ public EmbeddedDruidCluster createCluster() } - private void configureCompaction(CompactionEngine compactionEngine) + private void configureCompaction(CompactionEngine compactionEngine, @Nullable CompactionCandidateSearchPolicy policy) { final UpdateResponse updateResponse = cluster.callApi().onLeaderOverlord( - o -> o.updateClusterCompactionConfig(new ClusterCompactionConfig(1.0, 100, null, true, compactionEngine, true)) + o -> o.updateClusterCompactionConfig(new ClusterCompactionConfig( + 1.0, + 100, + policy, + true, + compactionEngine, + true + )) ); Assertions.assertTrue(updateResponse.isSuccess()); } @MethodSource("getEngine") @ParameterizedTest(name = "compactionEngine={0}") - public void test_ingestDayGranularity_andCompactToMonthGranularity_andCompactToYearGranularity_withInlineConfig(CompactionEngine compactionEngine) + public void test_ingestDayGranularity_andCompactToMonthGranularity_andCompactToYearGranularity_withInlineConfig( + CompactionEngine compactionEngine + ) { - configureCompaction(compactionEngine); + configureCompaction(compactionEngine, null); // Ingest data at DAY granularity and verify runIngestionAtGranularity( @@ -202,6 +235,125 @@ public void test_ingestDayGranularity_andCompactToMonthGranularity_andCompactToY verifyCompactedSegmentsHaveFingerprints(yearGranConfig); } + @Test + public void test_incrementalCompactionWithMSQ() throws Exception + { + configureCompaction( + CompactionEngine.MSQ, + new MostFragmentedIntervalFirstPolicy(2, new HumanReadableBytes("1KiB"), null, 0.8, null) + ); + KafkaSupervisorSpecBuilder kafkaSupervisorSpecBuilder = MoreResources.Supervisor.KAFKA_JSON + .get() + .withDataSchema(schema -> schema.withTimestamp(new TimestampSpec("timestamp", "iso", null)) + .withDimensions(DimensionsSpec.builder().useSchemaDiscovery(true).build())) + .withTuningConfig(tuningConfig -> tuningConfig.withMaxRowsPerSegment(1)) + .withIoConfig(ioConfig -> ioConfig.withConsumerProperties(kafkaServer.consumerProperties()).withTaskCount(2)); + + // Set up first topic and supervisor + final String topic1 = IdUtils.getRandomId(); + kafkaServer.createTopicWithPartitions(topic1, 1); + final KafkaSupervisorSpec supervisor1 = kafkaSupervisorSpecBuilder.withId(topic1).build(dataSource, topic1); + cluster.callApi().postSupervisor(supervisor1); + + final int totalRowCount = publish1kRecords(topic1, true) + publish1kRecords(topic1, false); + waitUntilPublishedRecordsAreIngested(totalRowCount); + + // Before compaction + Assertions.assertEquals(4, getNumSegmentsWith(Granularities.HOUR)); + + PartitionsSpec partitionsSpec = new DimensionRangePartitionsSpec(null, 5000, List.of("page"), false); + // Create a compaction config with DAY granularity + InlineSchemaDataSourceCompactionConfig dayGranularityConfig = + InlineSchemaDataSourceCompactionConfig + .builder() + .forDataSource(dataSource) + .withSkipOffsetFromLatest(Period.seconds(0)) + .withGranularitySpec(new UserCompactionTaskGranularityConfig(Granularities.DAY, null, false)) + .withDimensionsSpec(new UserCompactionTaskDimensionsConfig( + WikipediaStreamEventStreamGenerator.dimensions() + .stream() + .map(StringDimensionSchema::new) + .collect(Collectors.toUnmodifiableList()))) + .withTaskContext(Map.of("useConcurrentLocks", true)) + .withIoConfig(new UserCompactionTaskIOConfig(true)) + .withTuningConfig(UserCompactionTaskQueryTuningConfig.builder().partitionsSpec(partitionsSpec).build()) + .build(); + + runCompactionWithSpec(dayGranularityConfig); + Thread.sleep(2_000L); + waitForAllCompactionTasksToFinish(); + + pauseCompaction(dayGranularityConfig); + Assertions.assertEquals(0, getNumSegmentsWith(Granularities.HOUR)); + Assertions.assertEquals(1, getNumSegmentsWith(Granularities.DAY)); + + verifyCompactedSegmentsHaveFingerprints(dayGranularityConfig); + + // Set up another topic and supervisor + final String topic2 = IdUtils.getRandomId(); + kafkaServer.createTopicWithPartitions(topic2, 1); + final KafkaSupervisorSpec supervisor2 = kafkaSupervisorSpecBuilder.withId(topic2).build(dataSource, topic2); + cluster.callApi().postSupervisor(supervisor2); + + // published another 1k + final int appendedRowCount = publish1kRecords(topic2, true); + indexer.latchableEmitter().flush(); + waitUntilPublishedRecordsAreIngested(appendedRowCount); + + Assertions.assertEquals(0, getNumSegmentsWith(Granularities.HOUR)); + // 1 compacted segment + 2 appended segment + Assertions.assertEquals(3, getNumSegmentsWith(Granularities.DAY)); + + runCompactionWithSpec(dayGranularityConfig); + Thread.sleep(2_000L); + waitForAllCompactionTasksToFinish(); + + // performed incremental compaction + Assertions.assertEquals(2, getNumSegmentsWith(Granularities.DAY)); + + // Tear down both topics and supervisors + kafkaServer.deleteTopic(topic1); + cluster.callApi().postSupervisor(supervisor1.createSuspendedSpec()); + kafkaServer.deleteTopic(topic2); + cluster.callApi().postSupervisor(supervisor2.createSuspendedSpec()); + } + + protected void waitUntilPublishedRecordsAreIngested(int expectedRowCount) + { + indexer.latchableEmitter().waitForEventAggregate( + event -> event.hasMetricName("ingest/events/processed") + .hasDimension(DruidMetrics.DATASOURCE, dataSource), + agg -> agg.hasSumAtLeast(expectedRowCount) + ); + + final int totalEventsProcessed = indexer + .latchableEmitter() + .getMetricValues("ingest/events/processed", Map.of(DruidMetrics.DATASOURCE, dataSource)) + .stream() + .mapToInt(Number::intValue) + .sum(); + Assertions.assertEquals(expectedRowCount, totalEventsProcessed); + } + + protected int publish1kRecords(String topic, boolean useTransactions) + { + final EventSerializer serializer = new JsonEventSerializer(overlord.bindings().jsonMapper()); + final StreamGenerator streamGenerator = new WikipediaStreamEventStreamGenerator(serializer, 100, 100); + List records = streamGenerator.generateEvents(10); + + ArrayList> producerRecords = new ArrayList<>(); + for (byte[] record : records) { + producerRecords.add(new ProducerRecord<>(topic, record)); + } + + if (useTransactions) { + kafkaServer.produceRecordsToTopic(producerRecords); + } else { + kafkaServer.produceRecordsWithoutTransaction(producerRecords); + } + return producerRecords.size(); + } + @MethodSource("getEngine") @ParameterizedTest(name = "compactionEngine={0}") public void test_compaction_withPersistLastCompactionStateFalse_storesOnlyFingerprint(CompactionEngine compactionEngine) @@ -314,15 +466,19 @@ public void test_cascadingCompactionTemplate_multiplePeriodsApplyDifferentCompac null ); - InlineReindexingRuleProvider.Builder ruleProvider = InlineReindexingRuleProvider.builder() - .segmentGranularityRules(List.of(hourRule, dayRule)) - .tuningConfigRules(List.of(tuningConfigRule)) - .deletionRules(List.of(deletionRule)); + InlineReindexingRuleProvider.Builder ruleProvider = + InlineReindexingRuleProvider.builder() + .segmentGranularityRules(List.of(hourRule, dayRule)) + .tuningConfigRules(List.of(tuningConfigRule)) + .deletionRules(List.of(deletionRule)); if (compactionEngine == CompactionEngine.NATIVE) { - ruleProvider = ruleProvider.ioConfigRules( - List.of(new ReindexingIOConfigRule("dropExisting", null, Period.days(7), new UserCompactionTaskIOConfig(true))) - ); + ruleProvider = ruleProvider.ioConfigRules(List.of(new ReindexingIOConfigRule( + "dropExisting", + null, + Period.days(7), + new UserCompactionTaskIOConfig(true) + ))); } CascadingReindexingTemplate cascadingReindexingTemplate = new CascadingReindexingTemplate( @@ -351,7 +507,7 @@ public void test_cascadingReindexing_withVirtualColumnOnNestedData_filtersCorrec { // Virtual Columns on nested data is only supported with MSQ compaction engine right now. CompactionEngine compactionEngine = CompactionEngine.MSQ; - configureCompaction(compactionEngine); + configureCompaction(compactionEngine, null); String jsonDataWithNestedColumn = "{\"timestamp\":\"2025-06-01T00:00:00.000Z\",\"item\":\"shirt\",\"value\":105," @@ -432,7 +588,7 @@ public void test_cascadingReindexing_withVirtualColumnOnNestedData_filtersCorrec * Tests that when a compaction task filters out all rows using a transform spec, * tombstones are created to properly drop the old segments. This test covers both * hash and range partitioning strategies. - * + *

* This regression test addresses a bug where compaction with transforms that filter * all rows would succeed but not create tombstones, leaving old segments visible * and causing indefinite compaction retries. @@ -444,7 +600,7 @@ public void test_compactionWithTransformFilteringAllRows_createsTombstones( String partitionType ) { - configureCompaction(compactionEngine); + configureCompaction(compactionEngine, null); runIngestionAtGranularity( "DAY", @@ -480,54 +636,15 @@ public void test_compactionWithTransformFilteringAllRows_createsTombstones( // Add partitioning spec based on test parameter if ("range".equals(partitionType)) { - builder.withTuningConfig( - new UserCompactionTaskQueryTuningConfig( - null, - null, - null, - null, - null, - new DimensionRangePartitionsSpec(null, 5000, List.of("item"), false), - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null - ) - ); + PartitionsSpec partitionsSpec = new DimensionRangePartitionsSpec(null, 5000, List.of("item"), false); + builder.withTuningConfig(UserCompactionTaskQueryTuningConfig.builder().partitionsSpec(partitionsSpec).build()); } else { // Hash partitioning - builder.withTuningConfig( - new UserCompactionTaskQueryTuningConfig( - null, - null, - null, - null, - null, - new HashedPartitionsSpec(null, null, null), - null, - null, - null, - null, - null, - 2, - null, - null, - null, - null, - null, - null, - null - ) - ); + PartitionsSpec partitionsSpec = new HashedPartitionsSpec(null, null, null); + builder.withTuningConfig(UserCompactionTaskQueryTuningConfig.builder() + .partitionsSpec(partitionsSpec) + .maxNumConcurrentSubTasks(2) + .build()); } InlineSchemaDataSourceCompactionConfig compactionConfig = builder.build(); @@ -547,12 +664,8 @@ public void test_compactionWithTransformFilteringAllRows_createsTombstones( private int getTotalRowCount() { String sql = StringUtils.format("SELECT COUNT(*) as cnt FROM \"%s\"", dataSource); - String result = cluster.callApi().onAnyBroker(b -> b.submitSqlQuery(new ClientSqlQuery(sql, null, false, false, false, null, null))); - List> rows = JacksonUtils.readValue( - new DefaultObjectMapper(), - result.getBytes(StandardCharsets.UTF_8), - new TypeReference<>() {} - ); + ClientSqlQuery clientSqlQuery = new ClientSqlQuery(sql, null, false, false, false, null, null); + List> rows = parse(cluster.callApi().onAnyBroker(b -> b.submitSqlQuery(clientSqlQuery))); return ((Number) rows.get(0).get("cnt")).intValue(); } @@ -565,12 +678,8 @@ private void verifyNoRowsWithNestedValue(String nestedColumn, String field, Stri field, value ); - String result = cluster.callApi().onAnyBroker(b -> b.submitSqlQuery(new ClientSqlQuery(sql, null, false, false, false, null, null))); - List> rows = JacksonUtils.readValue( - new DefaultObjectMapper(), - result.getBytes(StandardCharsets.UTF_8), - new TypeReference<>() {} - ); + ClientSqlQuery clientSqlQuery = new ClientSqlQuery(sql, null, false, false, false, null, null); + List> rows = parse(cluster.callApi().onAnyBroker(b -> b.submitSqlQuery(clientSqlQuery))); Assertions.assertEquals( 0, ((Number) rows.get(0).get("cnt")).intValue(), @@ -641,9 +750,12 @@ private void verifyCompactedSegmentsHaveFingerprints(DataSourceCompactionConfig private void runCompactionWithSpec(DataSourceCompactionConfig config) { - final CompactionSupervisorSpec compactionSupervisor - = new CompactionSupervisorSpec(config, false, null); - cluster.callApi().postSupervisor(compactionSupervisor); + cluster.callApi().postSupervisor(new CompactionSupervisorSpec(config, false, null)); + } + + private void pauseCompaction(DataSourceCompactionConfig config) + { + cluster.callApi().postSupervisor(new CompactionSupervisorSpec(config, true, null)); } private void waitForAllCompactionTasksToFinish() @@ -727,7 +839,7 @@ private void verifyEventCountOlderThan(Period period, String dimension, String v DateTime now = DateTimes.nowUtc(); DateTime threshold = now.minus(period); - ClientSqlQuery query = new ClientSqlQuery( + ClientSqlQuery clientSqlQuery = new ClientSqlQuery( StringUtils.format( "SELECT COUNT(*) as cnt FROM \"%s\" WHERE %s = '%s' AND __time < MILLIS_TO_TIMESTAMP(%d)", dataSource, @@ -742,14 +854,7 @@ private void verifyEventCountOlderThan(Period period, String dimension, String v null, null ); - - final String resultAsJson = cluster.callApi().onAnyBroker(b -> b.submitSqlQuery(query)); - - List> result = JacksonUtils.readValue( - new DefaultObjectMapper(), - resultAsJson.getBytes(StandardCharsets.UTF_8), - new TypeReference<>() {} - ); + List> result = parse(cluster.callApi().onAnyBroker(b -> b.submitSqlQuery(clientSqlQuery))); Assertions.assertEquals(1, result.size()); Assertions.assertEquals( @@ -790,4 +895,14 @@ private UserCompactionTaskQueryTuningConfig createTuningConfigWithPartitionsSpec ); } + private static List> parse(String resultAsJson) + { + return JacksonUtils.readValue( + new DefaultObjectMapper(), + resultAsJson.getBytes(StandardCharsets.UTF_8), + new TypeReference<>() + { + } + ); + } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentUpgradeAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentUpgradeAction.java new file mode 100644 index 000000000000..ebc2348d450e --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentUpgradeAction.java @@ -0,0 +1,87 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.common.actions; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.core.type.TypeReference; +import org.apache.druid.indexing.common.task.Task; +import org.apache.druid.java.util.common.IAE; +import org.apache.druid.metadata.ReplaceTaskLock; +import org.apache.druid.timeline.DataSegment; + +import java.util.List; +import java.util.Map; +import java.util.Set; + +public class SegmentUpgradeAction implements TaskAction +{ + private final String dataSource; + private final List upgradeSegments; + + @JsonCreator + public SegmentUpgradeAction( + @JsonProperty("dataSource") String dataSource, + @JsonProperty("upgradeSegments") List upgradeSegments + ) + { + this.dataSource = dataSource; + this.upgradeSegments = upgradeSegments; + } + + @JsonProperty + public String getDataSource() + { + return dataSource; + } + + @JsonProperty + public List getUpgradeSegments() + { + return upgradeSegments; + } + + @Override + public TypeReference getReturnTypeReference() + { + return new TypeReference<>() + { + }; + } + + @Override + public Integer perform(Task task, TaskActionToolbox toolbox) + { + final String datasource = task.getDataSource(); + final Map segmentToReplaceLock + = TaskLocks.findReplaceLocksCoveringSegments(datasource, toolbox.getTaskLockbox(), Set.copyOf(upgradeSegments)); + + if (segmentToReplaceLock.size() < upgradeSegments.size()) { + throw new IAE( + "Not all segments are hold by a replace lock, only [%d] segments out of total segments[%d] are hold by repalce lock", + segmentToReplaceLock.size(), + upgradeSegments.size() + ); + } + + return toolbox.getIndexerMetadataStorageCoordinator() + .insertIntoUpgradeSegmentsTable(segmentToReplaceLock); + } +} 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 8b1d863530a9..dd2c1b7936b3 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 @@ -37,6 +37,7 @@ @JsonSubTypes.Type(name = "segmentTransactionalInsert", value = SegmentTransactionalInsertAction.class), @JsonSubTypes.Type(name = "segmentTransactionalAppend", value = SegmentTransactionalAppendAction.class), @JsonSubTypes.Type(name = "segmentTransactionalReplace", value = SegmentTransactionalReplaceAction.class), + @JsonSubTypes.Type(name = "segmentUpgrade", value = SegmentUpgradeAction.class), @JsonSubTypes.Type(name = "retrieveSegmentsById", value = RetrieveSegmentsByIdAction.class), @JsonSubTypes.Type(name = "retrieveUpgradedFromSegmentIds", value = RetrieveUpgradedFromSegmentIdsAction.class), @JsonSubTypes.Type(name = "retrieveUpgradedToSegmentIds", value = RetrieveUpgradedToSegmentIdsAction.class), diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionIntervalSpec.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionIntervalSpec.java index 40a4d775e031..b76f49ec40ae 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionIntervalSpec.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionIntervalSpec.java @@ -20,10 +20,12 @@ package org.apache.druid.indexing.common.task; import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonInclude; import com.fasterxml.jackson.annotation.JsonProperty; import org.apache.druid.indexing.common.LockGranularity; import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.JodaUtils; +import org.apache.druid.query.SegmentDescriptor; import org.apache.druid.segment.SegmentUtils; import org.apache.druid.timeline.DataSegment; import org.joda.time.Interval; @@ -42,11 +44,29 @@ public class CompactionIntervalSpec implements CompactionInputSpec private final Interval interval; @Nullable + private final List uncompactedSegments; + /** + * Optional hash of all segment IDs for validation. When set, this is used in {@link #validateSegments} to verify + * that the segments haven't changed since this spec was created. + *

+ * Note: This hash is computed and validated against ALL segments overlapping the interval, not just the + * uncompactedSegments. This is because compaction operates on all segments within the interval - compacted + * segments may need to be rewritten alongside uncompacted ones to maintain proper partitioning and sort order. + * Therefore, the validation check must apply to all segments to ensure correctness. + */ + @Nullable private final String sha256OfSortedSegmentIds; + public CompactionIntervalSpec(Interval interval, String sha256OfSortedSegmentIds) + { + this(interval, null, sha256OfSortedSegmentIds); + } + @JsonCreator public CompactionIntervalSpec( @JsonProperty("interval") Interval interval, + @JsonProperty("uncompactedSegments") @Nullable + List uncompactedSegments, @JsonProperty("sha256OfSortedSegmentIds") @Nullable String sha256OfSortedSegmentIds ) { @@ -54,6 +74,22 @@ public CompactionIntervalSpec( throw new IAE("Interval[%s] is empty, must specify a nonempty interval", interval); } this.interval = interval; + if (uncompactedSegments == null) { + // all segments within interval are included, pass check + } else if (uncompactedSegments.isEmpty()) { + throw new IAE("Can not supply empty segments as input, please use either null or non-empty segments."); + } else if (interval != null) { + List segmentsNotInInterval = + uncompactedSegments.stream().filter(s -> !interval.contains(s.getInterval())).collect(Collectors.toList()); + if (!segmentsNotInInterval.isEmpty()) { + throw new IAE( + "Can not supply segments outside interval[%s], got segments[%s].", + interval, + segmentsNotInInterval + ); + } + } + this.uncompactedSegments = uncompactedSegments; this.sha256OfSortedSegmentIds = sha256OfSortedSegmentIds; } @@ -63,6 +99,14 @@ public Interval getInterval() return interval; } + @Nullable + @JsonProperty + @JsonInclude(JsonInclude.Include.NON_NULL) + public List getUncompactedSegments() + { + return uncompactedSegments; + } + @Nullable @JsonProperty public String getSha256OfSortedSegmentIds() @@ -105,13 +149,14 @@ public boolean equals(Object o) } CompactionIntervalSpec that = (CompactionIntervalSpec) o; return Objects.equals(interval, that.interval) && + Objects.equals(uncompactedSegments, that.uncompactedSegments) && Objects.equals(sha256OfSortedSegmentIds, that.sha256OfSortedSegmentIds); } @Override public int hashCode() { - return Objects.hash(interval, sha256OfSortedSegmentIds); + return Objects.hash(interval, uncompactedSegments, sha256OfSortedSegmentIds); } @Override @@ -119,7 +164,8 @@ public String toString() { return "CompactionIntervalSpec{" + "interval=" + interval + - ", sha256OfSegmentIds='" + sha256OfSortedSegmentIds + '\'' + + ", uncompactedSegments=" + uncompactedSegments + + ", sha256OfSortedSegmentIds='" + sha256OfSortedSegmentIds + '\'' + '}'; } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionRunner.java index 0abaeed8eb27..baff4a528b0c 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionRunner.java @@ -24,9 +24,9 @@ import org.apache.druid.client.indexing.ClientCompactionRunnerInfo; import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexing.common.TaskToolbox; +import org.apache.druid.query.spec.QuerySegmentSpec; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.server.coordinator.CompactionConfigValidationResult; -import org.joda.time.Interval; import java.util.Map; @@ -47,7 +47,7 @@ public interface CompactionRunner */ TaskStatus runCompactionTasks( CompactionTask compactionTask, - Map intervalDataSchemaMap, + Map inputSchemas, TaskToolbox taskToolbox ) throws Exception; @@ -59,7 +59,7 @@ TaskStatus runCompactionTasks( */ CompactionConfigValidationResult validateCompactionTask( CompactionTask compactionTask, - Map intervalToDataSchemaMap + Map inputSchemas ); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java index 65bdee9a04a9..97416c2ec202 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java @@ -27,6 +27,8 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; +import com.google.common.base.Predicate; +import com.google.common.base.Predicates; import com.google.common.collect.BiMap; import com.google.common.collect.HashBiMap; import com.google.common.collect.ImmutableList; @@ -56,6 +58,7 @@ import org.apache.druid.indexing.common.SegmentCacheManagerFactory; import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.common.actions.RetrieveUsedSegmentsAction; +import org.apache.druid.indexing.common.actions.SegmentUpgradeAction; import org.apache.druid.indexing.common.actions.TaskActionClient; import org.apache.druid.indexing.common.task.IndexTask.IndexTuningConfig; import org.apache.druid.indexing.common.task.batch.parallel.ParallelIndexTuningConfig; @@ -77,7 +80,11 @@ import org.apache.druid.java.util.emitter.service.ServiceMetricEvent; import org.apache.druid.query.Order; import org.apache.druid.query.OrderBy; +import org.apache.druid.query.SegmentDescriptor; import org.apache.druid.query.aggregation.AggregatorFactory; +import org.apache.druid.query.spec.MultipleIntervalSegmentSpec; +import org.apache.druid.query.spec.MultipleSpecificSegmentSpec; +import org.apache.druid.query.spec.QuerySegmentSpec; import org.apache.druid.segment.AggregateProjectionMetadata; import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.Metadata; @@ -127,6 +134,7 @@ import java.util.function.Supplier; import java.util.stream.Collectors; import java.util.stream.IntStream; +import java.util.stream.StreamSupport; /** * The client representation of this task is {@link ClientCompactionTaskQuery}. JSON @@ -518,7 +526,7 @@ public TaskStatus runTask(TaskToolbox toolbox) throws Exception { emitMetric(toolbox.getEmitter(), "ingest/count", 1); - final Map intervalDataSchemas = createDataSchemasForIntervals( + final Map inputSchemas = createInputDataSchemas( toolbox, getTaskLockHelper().getLockGranularityToUse(), segmentProvider, @@ -534,12 +542,12 @@ public TaskStatus runTask(TaskToolbox toolbox) throws Exception registerResourceCloserOnAbnormalExit(compactionRunner.getCurrentSubTaskHolder()); CompactionConfigValidationResult supportsCompactionConfig = compactionRunner.validateCompactionTask( this, - intervalDataSchemas + inputSchemas ); if (!supportsCompactionConfig.isValid()) { throw InvalidInput.exception("Compaction spec not supported. Reason[%s].", supportsCompactionConfig.getReason()); } - return compactionRunner.runCompactionTasks(this, intervalDataSchemas, toolbox); + return compactionRunner.runCompactionTasks(this, inputSchemas, toolbox); } /** @@ -549,7 +557,7 @@ public TaskStatus runTask(TaskToolbox toolbox) throws Exception * determine schemas. */ @VisibleForTesting - static Map createDataSchemasForIntervals( + static Map createInputDataSchemas( final TaskToolbox toolbox, final LockGranularity lockGranularityInUse, final SegmentProvider segmentProvider, @@ -572,8 +580,21 @@ static Map createDataSchemasForIntervals( return Collections.emptyMap(); } + if (segmentProvider.incrementalCompaction) { + Iterable segmentsNotCompletelyWithinin = + Iterables.filter(timelineSegments, s -> !segmentProvider.interval.contains(s.getInterval())); + if (segmentsNotCompletelyWithinin.iterator().hasNext()) { + throw new ISE( + "Incremental compaction doesn't allow segments not completely within interval[%s]", + segmentProvider.interval + ); + } + } + if (granularitySpec == null || granularitySpec.getSegmentGranularity() == null) { - Map intervalDataSchemaMap = new HashMap<>(); + Map inputSchemas = new HashMap<>(); + // if segment is already compacted in incremental compaction, they need to be upgraded directly, supported in MSQ + List upgradeSegments = new ArrayList<>(); // original granularity final Map> intervalToSegments = new TreeMap<>( @@ -581,8 +602,15 @@ static Map createDataSchemasForIntervals( ); for (final DataSegment dataSegment : timelineSegments) { - intervalToSegments.computeIfAbsent(dataSegment.getInterval(), k -> new ArrayList<>()) - .add(dataSegment); + if (segmentProvider.segmentsToUpgradePredicate.test(dataSegment)) { + upgradeSegments.add(dataSegment); + } else { + intervalToSegments.computeIfAbsent(dataSegment.getInterval(), k -> new ArrayList<>()) + .add(dataSegment); + } + } + if (!upgradeSegments.isEmpty()) { + toolbox.getTaskActionClient().submit(new SegmentUpgradeAction(segmentProvider.dataSource, upgradeSegments)); } // unify overlapping intervals to ensure overlapping segments compacting in the same indexSpec @@ -627,18 +655,34 @@ static Map createDataSchemasForIntervals( projections, needMultiValuedColumns ); - intervalDataSchemaMap.put(interval, dataSchema); + inputSchemas.put( + segmentProvider.incrementalCompaction + ? new MultipleSpecificSegmentSpec(segmentsToCompact.stream() + .map(DataSegment::toDescriptor) + .collect(Collectors.toList())) + : new MultipleIntervalSegmentSpec(List.of(interval)), dataSchema); } - return intervalDataSchemaMap; + return inputSchemas; } else { // given segment granularity + List upgradeSegments = StreamSupport.stream(timelineSegments.spliterator(), false) + .filter(segmentProvider.segmentsToUpgradePredicate) + .collect(Collectors.toList()); + if (!upgradeSegments.isEmpty()) { + toolbox.getTaskActionClient().submit(new SegmentUpgradeAction(segmentProvider.dataSource, upgradeSegments)); + } + + final Iterable segmentsToCompact = Iterables.filter( + timelineSegments, + segmentProvider.segmentsToCompactPredicate + ); final DataSchema dataSchema = createDataSchema( toolbox.getEmitter(), metricBuilder, segmentProvider.dataSource, umbrellaInterval(timelineSegments, segmentProvider), lazyFetchSegments( - timelineSegments, + segmentsToCompact, toolbox.getSegmentCacheManager() ), dimensionsSpec, @@ -648,7 +692,11 @@ static Map createDataSchemasForIntervals( projections, needMultiValuedColumns ); - return Collections.singletonMap(segmentProvider.interval, dataSchema); + return Map.of(segmentProvider.incrementalCompaction + ? new MultipleSpecificSegmentSpec(StreamSupport.stream(segmentsToCompact.spliterator(), false) + .map(DataSegment::toDescriptor) + .collect(Collectors.toList())) + : new MultipleIntervalSegmentSpec(List.of(segmentProvider.interval)), dataSchema); } } @@ -658,8 +706,7 @@ private static Iterable retrieveRelevantTimelineHolders( LockGranularity lockGranularityInUse ) throws IOException { - final List usedSegments = - segmentProvider.findSegments(toolbox.getTaskActionClient()); + final List usedSegments = segmentProvider.findSegments(toolbox.getTaskActionClient()); segmentProvider.checkSegments(lockGranularityInUse, usedSegments); final List> timelineSegments = SegmentTimeline .forSegments(usedSegments) @@ -1222,11 +1269,27 @@ static class SegmentProvider private final CompactionInputSpec inputSpec; private final Interval interval; + private final boolean incrementalCompaction; + private final Predicate segmentsToUpgradePredicate; + private final Predicate segmentsToCompactPredicate; + SegmentProvider(String dataSource, CompactionInputSpec inputSpec) { this.dataSource = Preconditions.checkNotNull(dataSource); this.inputSpec = inputSpec; this.interval = inputSpec.findInterval(dataSource); + if (inputSpec instanceof CompactionIntervalSpec + && ((CompactionIntervalSpec) inputSpec).getUncompactedSegments() != null) { + incrementalCompaction = true; + Set uncompactedSegments = Set.copyOf(((CompactionIntervalSpec) inputSpec).getUncompactedSegments()); + this.segmentsToUpgradePredicate = s -> !uncompactedSegments.contains(s.toDescriptor()) + && this.interval.contains(s.getInterval()); + this.segmentsToCompactPredicate = Predicates.not(this.segmentsToUpgradePredicate); + } else { + incrementalCompaction = false; + this.segmentsToUpgradePredicate = Predicates.alwaysFalse(); + this.segmentsToCompactPredicate = Predicates.alwaysTrue(); + } } List findSegments(TaskActionClient actionClient) throws IOException diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/NativeCompactionRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/NativeCompactionRunner.java index 559cdc1c6403..b171a977f2df 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/NativeCompactionRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/NativeCompactionRunner.java @@ -37,9 +37,11 @@ import org.apache.druid.indexing.input.DruidInputSource; import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.JodaUtils; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.query.spec.QuerySegmentSpec; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.server.coordinator.CompactionConfigValidationResult; import org.apache.druid.server.coordinator.duty.CompactSegments; @@ -86,7 +88,7 @@ public CurrentSubTaskHolder getCurrentSubTaskHolder() @Override public CompactionConfigValidationResult validateCompactionTask( CompactionTask compactionTask, - Map intervalDataSchemaMap + Map inputSchemas ) { // Virtual columns in filter rules are not supported by native compaction @@ -107,7 +109,7 @@ public CompactionConfigValidationResult validateCompactionTask( */ @VisibleForTesting static List createIngestionSpecs( - Map intervalDataSchemaMap, + Map inputSchemas, final TaskToolbox toolbox, final CompactionIOConfig ioConfig, final PartitionConfigurationManager partitionConfigurationManager, @@ -117,18 +119,18 @@ static List createIngestionSpecs( { final CompactionTask.CompactionTuningConfig compactionTuningConfig = partitionConfigurationManager.computeTuningConfig(); - return intervalDataSchemaMap.entrySet().stream().map((dataSchema) -> new ParallelIndexIngestionSpec( - dataSchema.getValue(), - createIoConfig( - toolbox, - dataSchema.getValue(), - dataSchema.getKey(), - coordinatorClient, - segmentCacheManagerFactory, - ioConfig - ), - compactionTuningConfig - ) + return inputSchemas.entrySet().stream().map((dataSchema) -> new ParallelIndexIngestionSpec( + dataSchema.getValue(), + createIoConfig( + toolbox, + dataSchema.getValue(), + JodaUtils.umbrellaInterval(dataSchema.getKey().getIntervals()), + coordinatorClient, + segmentCacheManagerFactory, + ioConfig + ), + compactionTuningConfig + ) ).collect(Collectors.toList()); } @@ -187,7 +189,7 @@ private static ParallelIndexIOConfig createIoConfig( @Override public TaskStatus runCompactionTasks( CompactionTask compactionTask, - Map intervalDataSchemaMap, + Map intervalDataSchemaMap, TaskToolbox taskToolbox ) throws Exception { @@ -329,8 +331,8 @@ static class PartitionConfigurationManager CompactionTask.CompactionTuningConfig computeTuningConfig() { CompactionTask.CompactionTuningConfig newTuningConfig = tuningConfig == null - ? CompactionTask.CompactionTuningConfig.defaultConfig() - : tuningConfig; + ? CompactionTask.CompactionTuningConfig.defaultConfig() + : tuningConfig; PartitionsSpec partitionsSpec = newTuningConfig.getGivenOrDefaultPartitionsSpec(); if (partitionsSpec instanceof DynamicPartitionsSpec) { final DynamicPartitionsSpec dynamicPartitionsSpec = (DynamicPartitionsSpec) partitionsSpec; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionConfigBasedJobTemplate.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionConfigBasedJobTemplate.java index 25643231b2c8..fd282701844b 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionConfigBasedJobTemplate.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionConfigBasedJobTemplate.java @@ -26,6 +26,7 @@ import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.server.compaction.CompactionCandidate; +import org.apache.druid.server.compaction.CompactionCandidateSearchPolicy; import org.apache.druid.server.compaction.CompactionSlotManager; import org.apache.druid.server.compaction.DataSourceCompactibleSegmentIterator; import org.apache.druid.server.compaction.NewestSegmentFirstPolicy; @@ -91,12 +92,30 @@ public List createCompactionJobs( // Create a job for each CompactionCandidate while (segmentIterator.hasNext()) { final CompactionCandidate candidate = segmentIterator.next(); + final CompactionCandidateSearchPolicy.Eligibility eligibility = + params.getClusterCompactionConfig().getCompactionPolicy().checkEligibilityForCompaction(candidate, null); + if (!eligibility.isEligible()) { + continue; + } + final CompactionCandidate finalCandidate; + switch (eligibility.getMode()) { + case FULL_COMPACTION: + finalCandidate = candidate; + break; + case INCREMENTAL_COMPACTION: + finalCandidate = CompactionCandidate.from(candidate.getUncompactedSegments(), null) + .withCurrentStatus(candidate.getCurrentStatus()); + break; + default: + throw DruidException.defensive("unexpected compaction mode[%s]", eligibility.getMode()); + } // Allow template-specific customization of the config per candidate DataSourceCompactionConfig finalConfig = configOptimizer.optimizeConfig(config, candidate, params); ClientCompactionTaskQuery taskPayload = CompactSegments.createCompactionTask( - candidate, + finalCandidate, + eligibility.getMode(), finalConfig, params.getClusterCompactionConfig().getEngine(), indexingStateFingerprint, @@ -105,7 +124,7 @@ public List createCompactionJobs( jobs.add( new CompactionJob( taskPayload, - candidate, + finalCandidate, CompactionSlotManager.computeSlotsRequiredForTask(taskPayload), indexingStateFingerprint, compactionState diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionJobQueue.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionJobQueue.java index 97cd15808f9f..c2db60c28629 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionJobQueue.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionJobQueue.java @@ -25,6 +25,7 @@ import org.apache.druid.client.indexing.ClientCompactionTaskQuery; import org.apache.druid.client.indexing.ClientTaskQuery; import org.apache.druid.common.guava.FutureUtils; +import org.apache.druid.error.DruidException; import org.apache.druid.indexer.TaskState; import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexing.common.actions.TaskActionClientFactory; @@ -45,6 +46,7 @@ import org.apache.druid.server.compaction.CompactionSnapshotBuilder; import org.apache.druid.server.compaction.CompactionStatus; import org.apache.druid.server.compaction.CompactionStatusTracker; +import org.apache.druid.server.compaction.CompactionTaskStatus; import org.apache.druid.server.coordinator.AutoCompactionSnapshot; import org.apache.druid.server.coordinator.ClusterCompactionConfig; import org.apache.druid.server.coordinator.CompactionConfigValidationResult; @@ -217,7 +219,7 @@ public void runReadyJobs() final List pendingJobs = new ArrayList<>(); while (!queue.isEmpty()) { final CompactionJob job = queue.poll(); - if (startJobIfPendingAndReady(job, searchPolicy, pendingJobs, slotManager)) { + if (startJobIfPendingAndReady(job, pendingJobs, slotManager)) { runStats.add(Stats.Compaction.SUBMITTED_TASKS, RowKey.of(Dimension.DATASOURCE, job.getDataSource()), 1); } } @@ -267,7 +269,6 @@ public Map getSnapshots() */ private boolean startJobIfPendingAndReady( CompactionJob job, - CompactionCandidateSearchPolicy policy, List pendingJobs, CompactionSlotManager slotManager ) @@ -282,18 +283,20 @@ private boolean startJobIfPendingAndReady( } // Check if the job is already running, completed or skipped - final CompactionStatus compactionStatus = statusTracker.computeCompactionStatus(job.getCandidate(), policy); - switch (compactionStatus.getState()) { - case RUNNING: - return false; - case COMPLETE: - snapshotBuilder.moveFromPendingToCompleted(candidate); - return false; - case SKIPPED: - snapshotBuilder.moveFromPendingToSkipped(candidate); - return false; - default: - break; + final CompactionTaskStatus lastTaskStatus = statusTracker.getLatestTaskStatus(candidate); + final CompactionStatus compactionStatus = statusTracker.deriveCompactionStatus(lastTaskStatus); + if (compactionStatus != null) { + switch (compactionStatus.getState()) { + case RUNNING: + return false; + case SKIPPED: + snapshotBuilder.moveFromPendingToSkipped(candidate); + return false; + case PENDING: + case COMPLETE: + default: + throw DruidException.defensive("unexpected derived compaction state[%s]", compactionStatus.getState()); + } } // Check if enough compaction task slots are available diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentUpgradeActionTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentUpgradeActionTest.java new file mode 100644 index 000000000000..b3fc36a837ea --- /dev/null +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentUpgradeActionTest.java @@ -0,0 +1,140 @@ +/* + * 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 org.apache.druid.error.DruidException; +import org.apache.druid.indexing.common.TaskLockType; +import org.apache.druid.indexing.common.task.NoopTask; +import org.apache.druid.indexing.common.task.Task; +import org.apache.druid.indexing.overlord.TimeChunkLockRequest; +import org.apache.druid.java.util.common.IAE; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.SegmentId; +import org.apache.druid.timeline.partition.LinearShardSpec; +import org.joda.time.Interval; +import org.junit.Assert; +import org.junit.Rule; +import org.junit.Test; + +import java.util.List; + +public class SegmentUpgradeActionTest +{ + @Rule + public TaskActionTestKit actionTestKit = new TaskActionTestKit(); + + private static final String DATA_SOURCE = "test_dataSource"; + private static final Interval INTERVAL_2026_01 = Intervals.of("2026-01-01/2026-01-02"); + private static final Interval INTERVAL_2026_02 = Intervals.of("2026-01-02/2026-01-03"); + private static final String VERSION = "2026-01-01T00:00:00.000Z"; + + private static final DataSegment SEGMENT1 = + DataSegment.builder(SegmentId.of(DATA_SOURCE, INTERVAL_2026_01, VERSION, 0)) + .shardSpec(new LinearShardSpec(0)) + .build(); + + private static final DataSegment SEGMENT2 = + DataSegment.builder(SegmentId.of(DATA_SOURCE, INTERVAL_2026_01, VERSION, 1)) + .shardSpec(new LinearShardSpec(1)) + .build(); + + private static final DataSegment SEGMENT3 = + DataSegment.builder(SegmentId.of(DATA_SOURCE, INTERVAL_2026_02, VERSION, 0)) + .shardSpec(new LinearShardSpec(0)) + .build(); + + @Test + public void test_segmentsSuccessfullyInsertedIntoUpgradeTable() throws Exception + { + final Task task = NoopTask.forDatasource(DATA_SOURCE); + actionTestKit.getTaskLockbox().add(task); + actionTestKit.getTaskLockbox() + .lock(task, new TimeChunkLockRequest(TaskLockType.REPLACE, task, INTERVAL_2026_01, null), 5000); + actionTestKit.getTaskLockbox() + .lock(task, new TimeChunkLockRequest(TaskLockType.REPLACE, task, INTERVAL_2026_02, null), 5000); + + final SegmentUpgradeAction action = new SegmentUpgradeAction(DATA_SOURCE, List.of(SEGMENT1, SEGMENT2, SEGMENT3)); + + final Integer insertedCount = action.perform(task, actionTestKit.getTaskActionToolbox()); + Assert.assertEquals(3, insertedCount.intValue()); + final int deletedCount = actionTestKit.getMetadataStorageCoordinator().deleteUpgradeSegmentsForTask(task.getId()); + Assert.assertEquals(3, deletedCount); + } + + @Test + public void test_failsWhenSegmentsNotCoveredByReplaceLock() throws Exception + { + final Task task = NoopTask.forDatasource(DATA_SOURCE); + actionTestKit.getTaskLockbox().add(task); + actionTestKit.getTaskLockbox() + .lock(task, new TimeChunkLockRequest(TaskLockType.REPLACE, task, INTERVAL_2026_01, null), 5000); + + final SegmentUpgradeAction action = new SegmentUpgradeAction(DATA_SOURCE, List.of(SEGMENT1, SEGMENT2, SEGMENT3)); + + IAE exception = Assert.assertThrows(IAE.class, () -> action.perform(task, actionTestKit.getTaskActionToolbox())); + Assert.assertTrue(exception.getMessage().contains("Not all segments are hold by a replace lock")); + } + + @Test + public void test_failsWithExclusiveLockInsteadOfReplaceLock() throws Exception + { + final Task task = NoopTask.forDatasource(DATA_SOURCE); + actionTestKit.getTaskLockbox().add(task); + actionTestKit.getTaskLockbox() + .lock(task, new TimeChunkLockRequest(TaskLockType.EXCLUSIVE, task, INTERVAL_2026_01, null), 5000); + + final SegmentUpgradeAction action = new SegmentUpgradeAction(DATA_SOURCE, List.of(SEGMENT1, SEGMENT2)); + + IAE exception = Assert.assertThrows(IAE.class, () -> action.perform(task, actionTestKit.getTaskActionToolbox())); + Assert.assertTrue(exception.getMessage().contains("Not all segments are hold by a replace lock")); + } + + @Test + public void test_emptySegmentsList() + { + final Task task = NoopTask.forDatasource(DATA_SOURCE); + actionTestKit.getTaskLockbox().add(task); + + final SegmentUpgradeAction action = new SegmentUpgradeAction(DATA_SOURCE, List.of()); + + DruidException exception = Assert.assertThrows( + DruidException.class, + () -> action.perform(task, actionTestKit.getTaskActionToolbox()) + ); + Assert.assertTrue(exception.getMessage().contains("No segment to commit")); + } + + @Test + public void test_singleSegmentUpgrade() throws Exception + { + final Task task = NoopTask.forDatasource(DATA_SOURCE); + actionTestKit.getTaskLockbox().add(task); + actionTestKit.getTaskLockbox() + .lock(task, new TimeChunkLockRequest(TaskLockType.REPLACE, task, INTERVAL_2026_01, null), 5000); + + final SegmentUpgradeAction action = new SegmentUpgradeAction(DATA_SOURCE, List.of(SEGMENT1)); + + final Integer insertedCount = action.perform(task, actionTestKit.getTaskActionToolbox()); + Assert.assertEquals(1, insertedCount.intValue()); + final int deletedCount = actionTestKit.getMetadataStorageCoordinator().deleteUpgradeSegmentsForTask(task.getId()); + Assert.assertEquals(1, deletedCount); + } +} 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 4136d082aa51..aa9b006dd8ae 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 @@ -301,7 +301,7 @@ private ClientCompactionTaskQuery createCompactionTaskQuery(String id, Compactio id, "datasource", new ClientCompactionIOConfig( - new ClientCompactionIntervalSpec(Intervals.of("2019/2020"), "testSha256OfSortedSegmentIds"), true + new ClientCompactionIntervalSpec(Intervals.of("2019/2020"), null, "testSha256OfSortedSegmentIds"), true ), new ClientCompactionTaskQueryTuningConfig( 100, diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunBase.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunBase.java index 50b53e614886..8679824df2ba 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunBase.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunBase.java @@ -1523,6 +1523,17 @@ protected IndexTask buildIndexTask( Interval interval, boolean appendToExisting ) throws Exception + { + return buildIndexTask(DEFAULT_SEGMENT_GRAN, parseSpec, rows, interval, appendToExisting); + } + + protected IndexTask buildIndexTask( + Granularity segmentGranularity, + ParseSpec parseSpec, + List rows, + Interval interval, + boolean appendToExisting + ) throws Exception { File tmpDir = temporaryFolder.newFolder(); File tmpFile = File.createTempFile("druid", "index", tmpDir); @@ -1540,7 +1551,7 @@ protected IndexTask buildIndexTask( tmpDir, parseSpec, null, - new UniformGranularitySpec(DEFAULT_SEGMENT_GRAN, DEFAULT_QUERY_GRAN, List.of(interval)), + new UniformGranularitySpec(segmentGranularity, DEFAULT_QUERY_GRAN, List.of(interval)), IndexTaskTest.createTuningConfig(2, 2, 2L, null, false, true), appendToExisting, false 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 2b20fcfa8259..3a51d5b9fe21 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 @@ -97,6 +97,7 @@ import org.apache.druid.query.aggregation.firstlast.last.DoubleLastAggregatorFactory; import org.apache.druid.query.expression.TestExprMacroTable; import org.apache.druid.query.filter.SelectorDimFilter; +import org.apache.druid.query.spec.QuerySegmentSpec; import org.apache.druid.segment.AutoTypeColumnSchema; import org.apache.druid.segment.IndexIO; import org.apache.druid.segment.IndexMergerV9; @@ -788,7 +789,7 @@ public void testSegmentProviderFindSegmentsWithEmptySegmentsThrowException() @Test public void testCreateIngestionSchema() throws IOException { - final Map dataSchemasForIntervals = CompactionTask.createDataSchemasForIntervals( + final Map inputSchemas = CompactionTask.createInputDataSchemas( toolbox, LockGranularity.TIME_CHUNK, new SegmentProvider(DATA_SOURCE, new CompactionIntervalSpec(COMPACTION_INTERVAL, null)), @@ -802,7 +803,7 @@ public void testCreateIngestionSchema() throws IOException ); final List ingestionSpecs = NativeCompactionRunner.createIngestionSpecs( - dataSchemasForIntervals, + inputSchemas, toolbox, new CompactionIOConfig(null, false, null), new PartitionConfigurationManager(TUNING_CONFIG), @@ -851,7 +852,7 @@ public void testCreateIngestionSchemaWithTargetPartitionSize() throws IOExceptio .withMaxNumConcurrentSubTasks(10) .build(); - final Map dataSchemasForIntervals = CompactionTask.createDataSchemasForIntervals( + final Map inputSchemas = CompactionTask.createInputDataSchemas( toolbox, LockGranularity.TIME_CHUNK, new SegmentProvider(DATA_SOURCE, new CompactionIntervalSpec(COMPACTION_INTERVAL, null)), @@ -865,7 +866,7 @@ public void testCreateIngestionSchemaWithTargetPartitionSize() throws IOExceptio ); final List ingestionSpecs = NativeCompactionRunner.createIngestionSpecs( - dataSchemasForIntervals, + inputSchemas, toolbox, new CompactionIOConfig(null, false, null), new PartitionConfigurationManager(tuningConfig), @@ -915,7 +916,7 @@ public void testCreateIngestionSchemaWithMaxTotalRows() throws IOException .withPushTimeout(5000L) .build(); - final Map dataSchemasForIntervals = CompactionTask.createDataSchemasForIntervals( + final Map inputSchemas = CompactionTask.createInputDataSchemas( toolbox, LockGranularity.TIME_CHUNK, new SegmentProvider(DATA_SOURCE, new CompactionIntervalSpec(COMPACTION_INTERVAL, null)), @@ -929,7 +930,7 @@ public void testCreateIngestionSchemaWithMaxTotalRows() throws IOException ); final List ingestionSpecs = NativeCompactionRunner.createIngestionSpecs( - dataSchemasForIntervals, + inputSchemas, toolbox, new CompactionIOConfig(null, false, null), new PartitionConfigurationManager(tuningConfig), @@ -980,7 +981,7 @@ public void testCreateIngestionSchemaWithNumShards() throws IOException .withMaxNumConcurrentSubTasks(10) .build(); - final Map dataSchemasForIntervals = CompactionTask.createDataSchemasForIntervals( + final Map inputSchemas = CompactionTask.createInputDataSchemas( toolbox, LockGranularity.TIME_CHUNK, new SegmentProvider(DATA_SOURCE, new CompactionIntervalSpec(COMPACTION_INTERVAL, null)), @@ -994,7 +995,7 @@ public void testCreateIngestionSchemaWithNumShards() throws IOException ); final List ingestionSpecs = NativeCompactionRunner.createIngestionSpecs( - dataSchemasForIntervals, + inputSchemas, toolbox, new CompactionIOConfig(null, false, null), new PartitionConfigurationManager(tuningConfig), @@ -1052,7 +1053,7 @@ public void testCreateIngestionSchemaWithCustomDimensionsSpec() throws IOExcepti ) ); - final Map dataSchemasForIntervals = CompactionTask.createDataSchemasForIntervals( + final Map inputSchemas = CompactionTask.createInputDataSchemas( toolbox, LockGranularity.TIME_CHUNK, new SegmentProvider(DATA_SOURCE, new CompactionIntervalSpec(COMPACTION_INTERVAL, null)), @@ -1066,7 +1067,7 @@ public void testCreateIngestionSchemaWithCustomDimensionsSpec() throws IOExcepti ); final List ingestionSpecs = NativeCompactionRunner.createIngestionSpecs( - dataSchemasForIntervals, + inputSchemas, toolbox, new CompactionIOConfig(null, false, null), new PartitionConfigurationManager(TUNING_CONFIG), @@ -1104,7 +1105,7 @@ public void testCreateIngestionSchemaWithCustomMetricsSpec() throws IOException new DoubleMaxAggregatorFactory("custom_double_max", "agg_4") }; - final Map dataSchemasForIntervals = CompactionTask.createDataSchemasForIntervals( + final Map inputSchemas = CompactionTask.createInputDataSchemas( toolbox, LockGranularity.TIME_CHUNK, new SegmentProvider(DATA_SOURCE, new CompactionIntervalSpec(COMPACTION_INTERVAL, null)), @@ -1118,7 +1119,7 @@ public void testCreateIngestionSchemaWithCustomMetricsSpec() throws IOException ); final List ingestionSpecs = NativeCompactionRunner.createIngestionSpecs( - dataSchemasForIntervals, + inputSchemas, toolbox, new CompactionIOConfig(null, false, null), new PartitionConfigurationManager(TUNING_CONFIG), @@ -1149,7 +1150,7 @@ public void testCreateIngestionSchemaWithCustomMetricsSpec() throws IOException @Test public void testCreateIngestionSchemaWithCustomSegments() throws IOException { - final Map dataSchemasForIntervals = CompactionTask.createDataSchemasForIntervals( + final Map inputSchemas = CompactionTask.createInputDataSchemas( toolbox, LockGranularity.TIME_CHUNK, new SegmentProvider(DATA_SOURCE, new CompactionIntervalSpec(COMPACTION_INTERVAL, null)), @@ -1163,7 +1164,7 @@ public void testCreateIngestionSchemaWithCustomSegments() throws IOException ); final List ingestionSpecs = NativeCompactionRunner.createIngestionSpecs( - dataSchemasForIntervals, + inputSchemas, toolbox, new CompactionIOConfig(null, false, null), new PartitionConfigurationManager(TUNING_CONFIG), @@ -1201,7 +1202,7 @@ public void testCreateIngestionSchemaWithDifferentSegmentSet() throws IOExceptio Collections.sort(segments); // Remove one segment in the middle segments.remove(segments.size() / 2); - final Map dataSchemasForIntervals = CompactionTask.createDataSchemasForIntervals( + final Map inputSchemas = CompactionTask.createInputDataSchemas( toolbox, LockGranularity.TIME_CHUNK, new SegmentProvider(DATA_SOURCE, SpecificSegmentsSpec.fromSegments(segments)), @@ -1215,7 +1216,7 @@ public void testCreateIngestionSchemaWithDifferentSegmentSet() throws IOExceptio ); NativeCompactionRunner.createIngestionSpecs( - dataSchemasForIntervals, + inputSchemas, toolbox, new CompactionIOConfig(null, false, null), new PartitionConfigurationManager(TUNING_CONFIG), @@ -1233,7 +1234,7 @@ public void testMissingMetadata() throws IOException final TestIndexIO indexIO = (TestIndexIO) toolbox.getIndexIO(); indexIO.removeMetadata(Iterables.getFirst(indexIO.getQueryableIndexMap().keySet(), null)); final List segments = new ArrayList<>(SEGMENTS); - final Map dataSchemasForIntervals = CompactionTask.createDataSchemasForIntervals( + final Map inputSchemas = CompactionTask.createInputDataSchemas( toolbox, LockGranularity.TIME_CHUNK, new SegmentProvider(DATA_SOURCE, SpecificSegmentsSpec.fromSegments(segments)), @@ -1247,7 +1248,7 @@ public void testMissingMetadata() throws IOException ); NativeCompactionRunner.createIngestionSpecs( - dataSchemasForIntervals, + inputSchemas, toolbox, new CompactionIOConfig(null, false, null), new PartitionConfigurationManager(TUNING_CONFIG), @@ -1276,7 +1277,7 @@ public void testEmptyInterval() @Test public void testSegmentGranularityAndNullQueryGranularity() throws IOException { - final Map dataSchemasForIntervals = CompactionTask.createDataSchemasForIntervals( + final Map inputSchemas = CompactionTask.createInputDataSchemas( toolbox, LockGranularity.TIME_CHUNK, new SegmentProvider(DATA_SOURCE, new CompactionIntervalSpec(COMPACTION_INTERVAL, null)), @@ -1290,7 +1291,7 @@ public void testSegmentGranularityAndNullQueryGranularity() throws IOException ); final List ingestionSpecs = NativeCompactionRunner.createIngestionSpecs( - dataSchemasForIntervals, + inputSchemas, toolbox, new CompactionIOConfig(null, false, null), new PartitionConfigurationManager(TUNING_CONFIG), @@ -1322,7 +1323,7 @@ public void testSegmentGranularityAndNullQueryGranularity() throws IOException @Test public void testQueryGranularityAndNullSegmentGranularity() throws IOException { - final Map dataSchemasForIntervals = CompactionTask.createDataSchemasForIntervals( + final Map inputSchemas = CompactionTask.createInputDataSchemas( toolbox, LockGranularity.TIME_CHUNK, new SegmentProvider(DATA_SOURCE, new CompactionIntervalSpec(COMPACTION_INTERVAL, null)), @@ -1335,7 +1336,7 @@ public void testQueryGranularityAndNullSegmentGranularity() throws IOException false ); final List ingestionSpecs = NativeCompactionRunner.createIngestionSpecs( - dataSchemasForIntervals, + inputSchemas, toolbox, new CompactionIOConfig(null, false, null), new PartitionConfigurationManager(TUNING_CONFIG), @@ -1365,7 +1366,7 @@ public void testQueryGranularityAndNullSegmentGranularity() throws IOException @Test public void testQueryGranularityAndSegmentGranularityNonNull() throws IOException { - final Map dataSchemasForIntervals = CompactionTask.createDataSchemasForIntervals( + final Map inputSchemas = CompactionTask.createInputDataSchemas( toolbox, LockGranularity.TIME_CHUNK, new SegmentProvider(DATA_SOURCE, new CompactionIntervalSpec(COMPACTION_INTERVAL, null)), @@ -1383,7 +1384,7 @@ public void testQueryGranularityAndSegmentGranularityNonNull() throws IOExceptio ); final List ingestionSpecs = NativeCompactionRunner.createIngestionSpecs( - dataSchemasForIntervals, + inputSchemas, toolbox, new CompactionIOConfig(null, false, null), new PartitionConfigurationManager(TUNING_CONFIG), @@ -1418,7 +1419,7 @@ public void testQueryGranularityAndSegmentGranularityNonNull() throws IOExceptio @Test public void testNullGranularitySpec() throws IOException { - final Map dataSchemasForIntervals = CompactionTask.createDataSchemasForIntervals( + final Map inputSchemas = CompactionTask.createInputDataSchemas( toolbox, LockGranularity.TIME_CHUNK, new SegmentProvider(DATA_SOURCE, new CompactionIntervalSpec(COMPACTION_INTERVAL, null)), @@ -1432,7 +1433,7 @@ public void testNullGranularitySpec() throws IOException ); final List ingestionSpecs = NativeCompactionRunner.createIngestionSpecs( - dataSchemasForIntervals, + inputSchemas, toolbox, new CompactionIOConfig(null, false, null), new PartitionConfigurationManager(TUNING_CONFIG), @@ -1465,7 +1466,7 @@ public void testNullGranularitySpec() throws IOException public void testGranularitySpecWithNullQueryGranularityAndNullSegmentGranularity() throws IOException { - final Map dataSchemasForIntervals = CompactionTask.createDataSchemasForIntervals( + final Map inputSchemas = CompactionTask.createInputDataSchemas( toolbox, LockGranularity.TIME_CHUNK, new SegmentProvider(DATA_SOURCE, new CompactionIntervalSpec(COMPACTION_INTERVAL, null)), @@ -1479,7 +1480,7 @@ public void testGranularitySpecWithNullQueryGranularityAndNullSegmentGranularity ); final List ingestionSpecs = NativeCompactionRunner.createIngestionSpecs( - dataSchemasForIntervals, + inputSchemas, toolbox, new CompactionIOConfig(null, false, null), new PartitionConfigurationManager(TUNING_CONFIG), @@ -1512,7 +1513,7 @@ public void testGranularitySpecWithNullQueryGranularityAndNullSegmentGranularity public void testGranularitySpecWithNotNullRollup() throws IOException { - final Map dataSchemasForIntervals = CompactionTask.createDataSchemasForIntervals( + final Map inputSchemas = CompactionTask.createInputDataSchemas( toolbox, LockGranularity.TIME_CHUNK, new SegmentProvider(DATA_SOURCE, new CompactionIntervalSpec(COMPACTION_INTERVAL, null)), @@ -1526,7 +1527,7 @@ public void testGranularitySpecWithNotNullRollup() ); final List ingestionSpecs = NativeCompactionRunner.createIngestionSpecs( - dataSchemasForIntervals, + inputSchemas, toolbox, new CompactionIOConfig(null, false, null), new PartitionConfigurationManager(TUNING_CONFIG), @@ -1544,7 +1545,7 @@ public void testGranularitySpecWithNotNullRollup() public void testGranularitySpecWithNullRollup() throws IOException { - final Map dataSchemasForIntervals = CompactionTask.createDataSchemasForIntervals( + final Map inputSchemas = CompactionTask.createInputDataSchemas( toolbox, LockGranularity.TIME_CHUNK, new SegmentProvider(DATA_SOURCE, new CompactionIntervalSpec(COMPACTION_INTERVAL, null)), @@ -1558,7 +1559,7 @@ public void testGranularitySpecWithNullRollup() ); final List ingestionSpecs = NativeCompactionRunner.createIngestionSpecs( - dataSchemasForIntervals, + inputSchemas, toolbox, new CompactionIOConfig(null, false, null), new PartitionConfigurationManager(TUNING_CONFIG), @@ -1578,7 +1579,7 @@ public void testGranularitySpecWithNullRollup() public void testMultiValuedDimensionsProcessing() throws IOException { - final Map dataSchemasForIntervals = CompactionTask.createDataSchemasForIntervals( + final Map inputSchemas = CompactionTask.createInputDataSchemas( toolbox, LockGranularity.TIME_CHUNK, new SegmentProvider(DATA_SOURCE, new CompactionIntervalSpec(COMPACTION_INTERVAL, null)), @@ -1590,7 +1591,7 @@ public void testMultiValuedDimensionsProcessing() METRIC_BUILDER, true ); - for (DataSchema dataSchema : dataSchemasForIntervals.values()) { + for (DataSchema dataSchema : inputSchemas.values()) { Assert.assertTrue(dataSchema instanceof CombinedDataSchema); Assert.assertTrue(((CombinedDataSchema) dataSchema).getMultiValuedDimensions().isEmpty()); } @@ -2220,7 +2221,7 @@ private static class TestMSQCompactionRunner implements CompactionRunner @Override public TaskStatus runCompactionTasks( CompactionTask compactionTask, - Map intervalDataSchemaMap, + Map inputSchemas, TaskToolbox taskToolbox ) { @@ -2236,7 +2237,7 @@ public CurrentSubTaskHolder getCurrentSubTaskHolder() @Override public CompactionConfigValidationResult validateCompactionTask( CompactionTask compactionTask, - Map intervalToDataSchemaMap + Map inputSchemas ) { return null; diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/NativeCompactionRunnerTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/NativeCompactionRunnerTest.java index dcb801ab4ecf..f13e98d3051b 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/NativeCompactionRunnerTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/NativeCompactionRunnerTest.java @@ -23,13 +23,13 @@ import org.apache.druid.indexing.common.SegmentCacheManagerFactory; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.query.expression.TestExprMacroTable; +import org.apache.druid.query.spec.QuerySegmentSpec; import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.transform.CompactionTransformSpec; import org.apache.druid.segment.virtual.ExpressionVirtualColumn; import org.apache.druid.server.coordinator.CompactionConfigValidationResult; -import org.joda.time.Interval; import org.junit.Assert; import org.junit.Test; import org.mockito.Mockito; @@ -60,11 +60,11 @@ public void testVirtualColumnsInTransformSpecAreNotSupported() CompactionTransformSpec transformSpec = new CompactionTransformSpec(null, virtualColumns); CompactionTask compactionTask = createCompactionTask(transformSpec); - Map intervalDataschemas = Collections.emptyMap(); + Map inputSchemas = Collections.emptyMap(); CompactionConfigValidationResult validationResult = NATIVE_COMPACTION_RUNNER.validateCompactionTask( compactionTask, - intervalDataschemas + inputSchemas ); Assert.assertFalse(validationResult.isValid()); @@ -78,11 +78,11 @@ public void testVirtualColumnsInTransformSpecAreNotSupported() public void testNoVirtualColumnsIsValid() { CompactionTask compactionTask = createCompactionTask(null); - Map intervalDataschemas = Collections.emptyMap(); + Map inputSchemas = Collections.emptyMap(); CompactionConfigValidationResult validationResult = NATIVE_COMPACTION_RUNNER.validateCompactionTask( compactionTask, - intervalDataschemas + inputSchemas ); Assert.assertTrue(validationResult.isValid()); @@ -94,11 +94,11 @@ public void testEmptyVirtualColumnsIsValid() CompactionTransformSpec transformSpec = new CompactionTransformSpec(null, VirtualColumns.EMPTY); CompactionTask compactionTask = createCompactionTask(transformSpec); - Map intervalDataschemas = Collections.emptyMap(); + Map inputSchemas = Collections.emptyMap(); CompactionConfigValidationResult validationResult = NATIVE_COMPACTION_RUNNER.validateCompactionTask( compactionTask, - intervalDataschemas + inputSchemas ); Assert.assertTrue(validationResult.isValid()); 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 ee9e85204e07..00cddb943c79 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 @@ -125,7 +125,10 @@ public Set retrieveAllUsedSegments(String dataSource, Segments visi } @Override - public List> retrieveUsedSegmentsAndCreatedDates(String dataSource, List intervals) + public List> retrieveUsedSegmentsAndCreatedDates( + String dataSource, + List intervals + ) { return List.of(); } @@ -259,6 +262,14 @@ public SegmentPublishResult commitAppendSegmentsAndMetadata( return SegmentPublishResult.ok(commitSegments(appendSegments, segmentSchemaMapping)); } + @Override + public int insertIntoUpgradeSegmentsTable( + Map segmentToReplaceLock + ) + { + throw new UnsupportedOperationException("not implemented"); + } + @Override public SegmentPublishResult commitSegmentsAndMetadata( Set segments, diff --git a/integration-tests-ex/tools/src/main/java/org/apache/druid/testing/tools/WikipediaStreamEventStreamGenerator.java b/integration-tests-ex/tools/src/main/java/org/apache/druid/testing/tools/WikipediaStreamEventStreamGenerator.java index b26c999ed83e..28ecfdf06941 100644 --- a/integration-tests-ex/tools/src/main/java/org/apache/druid/testing/tools/WikipediaStreamEventStreamGenerator.java +++ b/integration-tests-ex/tools/src/main/java/org/apache/druid/testing/tools/WikipediaStreamEventStreamGenerator.java @@ -68,4 +68,22 @@ List> newEvent(int i, DateTime timestamp) event.add(Pair.of("delta", i)); return Collections.unmodifiableList(event); } + + public static List dimensions() + { + return List.of( + "page", + "language", + "user", + "unpatrolled", + "newPage", + "robot", + "anonymous", + COL_UNIQUE_NAMESPACE, + "continent", + "country", + "region", + "city" + ); + } } diff --git a/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java b/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java index e65a1994f4aa..07dec885a77f 100644 --- a/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java +++ b/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java @@ -38,6 +38,7 @@ import org.apache.druid.indexing.common.task.CompactionRunner; import org.apache.druid.indexing.common.task.CompactionTask; import org.apache.druid.indexing.common.task.CurrentSubTaskHolder; +import org.apache.druid.java.util.common.JodaUtils; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.AllGranularity; import org.apache.druid.java.util.common.granularity.Granularities; @@ -65,7 +66,7 @@ import org.apache.druid.query.groupby.GroupByQueryConfig; import org.apache.druid.query.groupby.orderby.OrderByColumnSpec; import org.apache.druid.query.policy.PolicyEnforcer; -import org.apache.druid.query.spec.MultipleIntervalSegmentSpec; +import org.apache.druid.query.spec.QuerySegmentSpec; import org.apache.druid.segment.VirtualColumn; import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.column.ColumnHolder; @@ -153,7 +154,7 @@ public MSQCompactionRunner( @Override public CompactionConfigValidationResult validateCompactionTask( CompactionTask compactionTask, - Map intervalToDataSchemaMap + Map intervalToDataSchemaMap ) { if (intervalToDataSchemaMap.size() > 1) { @@ -234,11 +235,11 @@ public CurrentSubTaskHolder getCurrentSubTaskHolder() @Override public TaskStatus runCompactionTasks( CompactionTask compactionTask, - Map intervalDataSchemas, + Map inputSchemas, TaskToolbox taskToolbox ) throws Exception { - List msqControllerTasks = createMsqControllerTasks(compactionTask, intervalDataSchemas); + List msqControllerTasks = createMsqControllerTasks(compactionTask, inputSchemas); if (msqControllerTasks.isEmpty()) { String msg = StringUtils.format( @@ -257,21 +258,22 @@ public TaskStatus runCompactionTasks( public List createMsqControllerTasks( CompactionTask compactionTask, - Map intervalDataSchemas + Map inputSchemas ) throws JsonProcessingException { final List msqControllerTasks = new ArrayList<>(); - for (Map.Entry intervalDataSchema : intervalDataSchemas.entrySet()) { + for (Map.Entry inputSchema : inputSchemas.entrySet()) { Query query; - Interval interval = intervalDataSchema.getKey(); - DataSchema dataSchema = intervalDataSchema.getValue(); - Map inputColToVirtualCol = getVirtualColumns(dataSchema, interval, compactionTask.getTransformSpec()); + QuerySegmentSpec segmentSpec = inputSchema.getKey(); + DataSchema dataSchema = inputSchema.getValue(); + Map inputColToVirtualCol = + getVirtualColumns(dataSchema, JodaUtils.umbrellaInterval(segmentSpec.getIntervals()), compactionTask.getTransformSpec()); if (isGroupBy(dataSchema)) { - query = buildGroupByQuery(compactionTask, interval, dataSchema, inputColToVirtualCol); + query = buildGroupByQuery(compactionTask, segmentSpec, dataSchema, inputColToVirtualCol); } else { - query = buildScanQuery(compactionTask, interval, dataSchema, inputColToVirtualCol); + query = buildScanQuery(compactionTask, segmentSpec, dataSchema, inputColToVirtualCol); } QueryContext compactionTaskContext = new QueryContext(compactionTask.getContext()); @@ -501,7 +503,7 @@ private static Map buildQueryContext( private Query buildScanQuery( CompactionTask compactionTask, - Interval interval, + QuerySegmentSpec segmentSpec, DataSchema dataSchema, Map inputColToVirtualCol ) @@ -513,7 +515,7 @@ private Query buildScanQuery( .columns(rowSignature.getColumnNames()) .virtualColumns(virtualColumns) .columnTypes(rowSignature.getColumnTypes()) - .intervals(new MultipleIntervalSegmentSpec(Collections.singletonList(interval))) + .intervals(segmentSpec) .filters(dataSchema.getTransformSpec().getFilter()) .context(buildQueryContext(compactionTask.getContext(), dataSchema)); @@ -639,7 +641,7 @@ private Map getVirtualColumns(DataSchema dataSchema, Inte private Query buildGroupByQuery( CompactionTask compactionTask, - Interval interval, + QuerySegmentSpec segmentSpec, DataSchema dataSchema, Map inputColToVirtualCol ) @@ -675,7 +677,7 @@ private Query buildGroupByQuery( .setAggregatorSpecs(Arrays.asList(dataSchema.getAggregators())) .setPostAggregatorSpecs(postAggregators) .setContext(buildQueryContext(compactionTask.getContext(), dataSchema)) - .setInterval(interval); + .setQuerySegmentSpec(segmentSpec); if (compactionTask.getTuningConfig() != null && compactionTask.getTuningConfig().getPartitionsSpec() != null) { getOrderBySpec(compactionTask.getTuningConfig().getPartitionsSpec()).forEach(builder::addOrderByColumn); diff --git a/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQCompactionTaskRunTest.java b/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQCompactionTaskRunTest.java index 0193dd463312..e45ca07116b0 100644 --- a/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQCompactionTaskRunTest.java +++ b/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQCompactionTaskRunTest.java @@ -21,6 +21,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableList; import com.google.common.collect.Iterables; import com.google.common.util.concurrent.Futures; import com.google.inject.Guice; @@ -36,7 +37,9 @@ import org.apache.druid.guice.annotations.Json; import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexer.granularity.UniformGranularitySpec; +import org.apache.druid.indexer.partitions.DimensionRangePartitionsSpec; import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; +import org.apache.druid.indexer.partitions.PartitionsSpec; import org.apache.druid.indexing.common.LockGranularity; import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.common.TestUtils; @@ -46,6 +49,8 @@ import org.apache.druid.indexing.common.task.CompactionTaskRunBase; import org.apache.druid.indexing.common.task.IndexTask; import org.apache.druid.indexing.common.task.Tasks; +import org.apache.druid.indexing.common.task.TuningConfigBuilder; +import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.StringUtils; @@ -62,6 +67,7 @@ import org.apache.druid.msq.test.MSQTestControllerContext; import org.apache.druid.query.ForwardingQueryProcessingPool; import org.apache.druid.query.QueryProcessingPool; +import org.apache.druid.query.SegmentDescriptor; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.LongMaxAggregatorFactory; import org.apache.druid.query.expression.TestExprMacroTable; @@ -76,6 +82,7 @@ import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.QueryableIndexSegment; import org.apache.druid.segment.ReferenceCountedSegmentProvider; +import org.apache.druid.segment.indexing.TuningConfig; import org.apache.druid.segment.loading.AcquireSegmentAction; import org.apache.druid.segment.loading.AcquireSegmentResult; import org.apache.druid.segment.loading.DataSegmentPusher; @@ -89,6 +96,7 @@ import org.apache.druid.timeline.CompactionState; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.partition.NumberedShardSpec; +import org.apache.druid.timeline.partition.ShardSpec; import org.joda.time.Interval; import org.junit.Assert; import org.junit.Assume; @@ -569,6 +577,163 @@ public void testMSQCompactionWithConcurrentAppendAppendLocksFirst() throws Excep verifyTaskSuccessRowsAndSchemaMatch(finalResult, 19); } + @Test + public void testIncrementalCompaction() throws Exception + { + Assume.assumeTrue(lockGranularity == LockGranularity.TIME_CHUNK); + Assume.assumeTrue("Incremental compaction depends on concurrent lock", useConcurrentLocks); + verifyTaskSuccessRowsAndSchemaMatch(runIndexTask(), TOTAL_TEST_ROWS); + + final CompactionTask compactionTask1 = + compactionTaskBuilder(segmentGranularity).interval(inputInterval, true).build(); + + final Pair resultPair1 = runTask(compactionTask1); + verifyTaskSuccessRowsAndSchemaMatch(resultPair1, TOTAL_TEST_ROWS); + verifyCompactedSegment(List.copyOf(resultPair1.rhs.getSegments()), segmentGranularity, DEFAULT_QUERY_GRAN, false); + Assert.assertEquals(1, resultPair1.rhs.getSegments().size()); + final DataSegment compactedSegment1 = Iterables.getOnlyElement(resultPair1.rhs.getSegments()); + + Pair appendTask = runAppendTask(); + verifyTaskSuccessRowsAndSchemaMatch(appendTask, TOTAL_TEST_ROWS); + + List uncompacted = appendTask.rhs.getSegments() + .stream() + .map(DataSegment::toDescriptor) + .collect(Collectors.toList()); + final CompactionTask compactionTask2 = + compactionTaskBuilder(segmentGranularity) + .inputSpec(new CompactionIntervalSpec(inputInterval, uncompacted, null), true) + .build(); + final Pair resultPair2 = runTask(compactionTask2); + verifyTaskSuccessRowsAndSchemaMatch(resultPair2, TOTAL_TEST_ROWS); + Assert.assertEquals(1, resultPair2.rhs.getSegments().size()); + final DataSegment compactedSegment2 = Iterables.getOnlyElement(resultPair2.rhs.getSegments()); + + final List usedSegments = + coordinatorClient.fetchUsedSegments(DATA_SOURCE, List.of(Intervals.of("2014-01-01/2014-01-02"))) + .get() + .stream() + .map(DataSegment::toString) + .collect(Collectors.toList()); + Assert.assertEquals( + List.of( + compactedSegment2.withShardSpec(new NumberedShardSpec(0, 2)).toString(), + // shard spec in compactedSegment2 has been updated + compactedSegment1.toBuilder() + .shardSpec(new NumberedShardSpec(1, 2)) + .version(compactedSegment2.getVersion()) + .build() + .toString() // compactedSegment1 has been upgraded with the new version & shardSpec + ), usedSegments); + } + + @Test + public void testIncrementalCompactionRangePartition() throws Exception + { + List rows = ImmutableList.of( + "2014-01-01T00:00:10Z,a,1\n", + "2014-01-01T00:00:10Z,b,2\n", + "2014-01-01T00:00:10Z,c,3\n", + "2014-01-01T01:00:20Z,a,1\n", + "2014-01-01T01:00:20Z,b,2\n", + "2014-01-01T01:00:20Z,c,3\n", + "2014-01-01T02:00:30Z,a,1\n", + "2014-01-01T02:00:30Z,b,2\n", + "2014-01-01T02:00:30Z,c,3\n" + ); + Assume.assumeTrue(lockGranularity == LockGranularity.TIME_CHUNK); + Assume.assumeTrue("Incremental compaction depends on concurrent lock", useConcurrentLocks); + verifyTaskSuccessRowsAndSchemaMatch( + runTask(buildIndexTask(DEFAULT_PARSE_SPEC, rows, inputInterval, false)), + 9 + ); + + PartitionsSpec rangePartitionSpec = new DimensionRangePartitionsSpec(null, 3, List.of("dim"), false); + TuningConfig tuningConfig = TuningConfigBuilder.forCompactionTask() + .withMaxTotalRows(Long.MAX_VALUE) + .withPartitionsSpec(rangePartitionSpec) + .withForceGuaranteedRollup(true) + .build(); + final CompactionTask compactionTask1 = + compactionTaskBuilder(segmentGranularity).interval(inputInterval, true).tuningConfig(tuningConfig).build(); + + final Pair resultPair1 = runTask(compactionTask1); + verifyTaskSuccessRowsAndSchemaMatch(resultPair1, 9); + Assert.assertEquals(3, resultPair1.rhs.getSegments().size()); + + Pair appendTask = + runTask(buildIndexTask(DEFAULT_PARSE_SPEC, rows, inputInterval, true)); + verifyTaskSuccessRowsAndSchemaMatch(appendTask, 9); + + List uncompacted = appendTask.rhs.getSegments() + .stream() + .map(DataSegment::toDescriptor) + .collect(Collectors.toList()); + final CompactionTask compactionTask2 = + compactionTaskBuilder(segmentGranularity) + .inputSpec(new CompactionIntervalSpec(inputInterval, uncompacted, null), true) + .tuningConfig(tuningConfig) + .build(); + final Pair resultPair2 = runTask(compactionTask2); + verifyTaskSuccessRowsAndSchemaMatch(resultPair2, 9); + Assert.assertEquals(3, resultPair2.rhs.getSegments().size()); + + final List usedSegments = + coordinatorClient.fetchUsedSegments(DATA_SOURCE, List.of(Intervals.of("2014-01-01/2014-01-02"))).get(); + Assert.assertEquals(6, usedSegments.size()); + final List shards = usedSegments.stream().map(DataSegment::getShardSpec).collect(Collectors.toList()); + Assert.assertEquals(Set.of("range"), shards.stream().map(ShardSpec::getType).collect(Collectors.toSet())); + } + + @Test + public void testIncrementalCompactionOverlappingInterval() throws Exception + { + Assume.assumeTrue(lockGranularity == LockGranularity.TIME_CHUNK); + Assume.assumeTrue("Incremental compaction depends on concurrent lock", useConcurrentLocks); + + List rows = new ArrayList<>(); + rows.add("2014-01-01T00:00:10Z,a1,11\n"); + rows.add("2014-01-01T00:00:10Z,b1,12\n"); + rows.add("2014-01-01T00:00:10Z,c1,13\n"); + rows.add("2014-01-01T06:00:20Z,a1,11\n"); + rows.add("2014-01-01T06:00:20Z,b1,12\n"); + rows.add("2014-01-01T06:00:20Z,c1,13\n"); + rows.add("2014-01-01T08:00:20Z,b1,12\n"); + rows.add("2014-01-01T08:00:20Z,c1,13\n"); + rows.add("2014-01-01T10:00:20Z,b1,12\n"); + rows.add("2014-01-01T10:00:20Z,c1,13\n"); + final IndexTask indexTask = buildIndexTask( + Granularities.SIX_HOUR, + DEFAULT_PARSE_SPEC, + rows, + TEST_INTERVAL_DAY, + true + ); + Pair indexTaskResult = runTask(indexTask); + // created 2 segments in HOUR 0 -> HOUR 6, and 4 segments in HOUR 6 -> HOUR12 + Assert.assertEquals(6, indexTaskResult.rhs.getSegments().size()); + verifyTaskSuccessRowsAndSchemaMatch(indexTaskResult, 10); + + // First compaction task to only compact 6 segments from indexTask. + final Interval compactionInterval = Intervals.of("2014-01-01T00:00:00Z/2014-01-01T08:00:00Z"); + final List uncompactedFromIndexTask = + indexTaskResult.rhs.getSegments() + .stream() + .filter(s -> compactionInterval.contains(s.getInterval())) + .map(DataSegment::toDescriptor) + .collect(Collectors.toList()); + + final CompactionTask compactionTask1 = + compactionTaskBuilder(Granularities.EIGHT_HOUR) + .inputSpec(new CompactionIntervalSpec(compactionInterval, uncompactedFromIndexTask, null), true) + .build(); + ISE e = Assert.assertThrows(ISE.class, () -> runTask(compactionTask1)); + Assert.assertEquals( + "Incremental compaction doesn't allow segments not completely within interval[2014-01-01T00:00:00.000Z/2014-01-01T08:00:00.000Z]", + e.getMessage() + ); + } + @Override protected CompactionState getDefaultCompactionState( Granularity segmentGranularity, diff --git a/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQCompactionRunnerTest.java b/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQCompactionRunnerTest.java index a906f3dc0016..fa124f44da35 100644 --- a/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQCompactionRunnerTest.java +++ b/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQCompactionRunnerTest.java @@ -65,6 +65,8 @@ import org.apache.druid.query.filter.SelectorDimFilter; import org.apache.druid.query.groupby.GroupByQuery; import org.apache.druid.query.scan.ScanQuery; +import org.apache.druid.query.spec.MultipleIntervalSegmentSpec; +import org.apache.druid.query.spec.QuerySegmentSpec; import org.apache.druid.segment.AutoTypeColumnSchema; import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.NestedDataColumnSchema; @@ -135,8 +137,8 @@ public class MSQCompactionRunnerTest new LongSumAggregatorFactory(LONG_DIMENSION.getName(), LONG_DIMENSION.getName()) ) .build(); - private static final Map INTERVAL_DATASCHEMAS = ImmutableMap.of( - COMPACTION_INTERVAL, + private static final Map INTERVAL_DATASCHEMAS = ImmutableMap.of( + new MultipleIntervalSegmentSpec(List.of(COMPACTION_INTERVAL)), new CombinedDataSchema( DATA_SOURCE, new TimestampSpec(TIMESTAMP_COLUMN, null, null), @@ -166,8 +168,8 @@ public class MSQCompactionRunnerTest @Test public void testMultipleDisjointCompactionIntervalsAreInvalid() { - Map intervalDataschemas = new HashMap<>(INTERVAL_DATASCHEMAS); - intervalDataschemas.put(Intervals.of("2017-07-01/2018-01-01"), null); + Map intervalDataschemas = new HashMap<>(INTERVAL_DATASCHEMAS); + intervalDataschemas.put(new MultipleIntervalSegmentSpec(List.of(Intervals.of("2017-07-01/2018-01-01"))), null); CompactionTask compactionTask = createCompactionTask( new HashedPartitionsSpec(3, null, ImmutableList.of("dummy")), null, @@ -387,7 +389,7 @@ public void testCompactionConfigWithoutMetricsSpecProducesCorrectSpec() throws J List msqControllerTasks = MSQ_COMPACTION_RUNNER.createMsqControllerTasks( taskCreatedWithTransformSpec, - Collections.singletonMap(COMPACTION_INTERVAL, dataSchema) + Map.of(new MultipleIntervalSegmentSpec(List.of(COMPACTION_INTERVAL)), dataSchema) ); MSQControllerTask msqControllerTask = Iterables.getOnlyElement(msqControllerTasks); @@ -471,7 +473,7 @@ public void testCompactionConfigWithSortOnNonTimeDimensionsProducesCorrectSpec() List msqControllerTasks = MSQ_COMPACTION_RUNNER.createMsqControllerTasks( taskCreatedWithTransformSpec, - Collections.singletonMap(COMPACTION_INTERVAL, dataSchema) + Map.of(new MultipleIntervalSegmentSpec(List.of(COMPACTION_INTERVAL)), dataSchema) ); LegacyMSQSpec actualMSQSpec = Iterables.getOnlyElement(msqControllerTasks).getQuerySpec(); @@ -520,7 +522,7 @@ public void testCompactionConfigWithMetricsSpecProducesCorrectSpec() throws Json List msqControllerTasks = MSQ_COMPACTION_RUNNER.createMsqControllerTasks( taskCreatedWithTransformSpec, - Collections.singletonMap(COMPACTION_INTERVAL, dataSchema) + Map.of(new MultipleIntervalSegmentSpec(List.of(COMPACTION_INTERVAL)), dataSchema) ); MSQControllerTask msqControllerTask = Iterables.getOnlyElement(msqControllerTasks); @@ -606,7 +608,7 @@ public void testCompactionConfigWithProjectionsProducesCorrectSpec() throws Json List msqControllerTasks = MSQ_COMPACTION_RUNNER.createMsqControllerTasks( taskCreatedWithTransformSpec, - Collections.singletonMap(COMPACTION_INTERVAL, dataSchema) + Map.of(new MultipleIntervalSegmentSpec(List.of(COMPACTION_INTERVAL)), dataSchema) ); MSQControllerTask msqControllerTask = Iterables.getOnlyElement(msqControllerTasks); diff --git a/processing/src/main/java/org/apache/druid/timeline/partition/DimensionRangeShardSpec.java b/processing/src/main/java/org/apache/druid/timeline/partition/DimensionRangeShardSpec.java index 4054a618a752..04963b19b8c5 100644 --- a/processing/src/main/java/org/apache/druid/timeline/partition/DimensionRangeShardSpec.java +++ b/processing/src/main/java/org/apache/druid/timeline/partition/DimensionRangeShardSpec.java @@ -104,6 +104,18 @@ public int getNumCorePartitions() return numCorePartitions; } + @Override + public ShardSpec withPartitionNum(int partitionNum1) + { + return new DimensionRangeShardSpec(dimensions, start, end, partitionNum1, numCorePartitions); + } + + @Override + public ShardSpec withCorePartitions(int partitions1) + { + return new DimensionRangeShardSpec(dimensions, start, end, partitionNum, partitions1); + } + public boolean isNumCorePartitionsUnknown() { return numCorePartitions == UNKNOWN_NUM_CORE_PARTITIONS; @@ -242,6 +254,12 @@ public boolean possibleInDomain(Map> domain) return true; } + @Override + public boolean isNumChunkSupported() + { + return !isNumCorePartitionsUnknown(); + } + @Override public PartitionChunk createChunk(T obj) { diff --git a/processing/src/main/java/org/apache/druid/timeline/partition/LinearShardSpec.java b/processing/src/main/java/org/apache/druid/timeline/partition/LinearShardSpec.java index 27659e5b5abf..f2d25b50311a 100644 --- a/processing/src/main/java/org/apache/druid/timeline/partition/LinearShardSpec.java +++ b/processing/src/main/java/org/apache/druid/timeline/partition/LinearShardSpec.java @@ -55,6 +55,18 @@ public int getNumCorePartitions() return 0; } + @Override + public ShardSpec withPartitionNum(int partitionNum1) + { + return new LinearShardSpec(partitionNum1); + } + + @Override + public ShardSpec withCorePartitions(int partitions1) + { + return this; + } + @Override public ShardSpecLookup getLookup(final List shardSpecs) { diff --git a/processing/src/main/java/org/apache/druid/timeline/partition/NumberedShardSpec.java b/processing/src/main/java/org/apache/druid/timeline/partition/NumberedShardSpec.java index aa3d5b3f800e..fdcb438f21c6 100644 --- a/processing/src/main/java/org/apache/druid/timeline/partition/NumberedShardSpec.java +++ b/processing/src/main/java/org/apache/druid/timeline/partition/NumberedShardSpec.java @@ -100,6 +100,24 @@ public int getNumCorePartitions() return partitions; } + @Override + public ShardSpec withPartitionNum(int partitionNum1) + { + return new NumberedShardSpec(partitionNum1, partitions); + } + + @Override + public ShardSpec withCorePartitions(int partitions1) + { + return new NumberedShardSpec(partitionNum, partitions1); + } + + @Override + public boolean isNumChunkSupported() + { + return true; + } + @Override public PartitionChunk createChunk(T obj) { diff --git a/processing/src/main/java/org/apache/druid/timeline/partition/ShardSpec.java b/processing/src/main/java/org/apache/druid/timeline/partition/ShardSpec.java index ffa2198f4497..1a64c5202fcd 100644 --- a/processing/src/main/java/org/apache/druid/timeline/partition/ShardSpec.java +++ b/processing/src/main/java/org/apache/druid/timeline/partition/ShardSpec.java @@ -23,6 +23,7 @@ import com.fasterxml.jackson.annotation.JsonSubTypes; import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.google.common.collect.RangeSet; +import org.apache.druid.error.DruidException; import java.util.List; import java.util.Map; @@ -56,6 +57,15 @@ }) public interface ShardSpec { + /** + * Returns whether {@link #createChunk} returns a {@link NumberedPartitionChunk} instance. + * This is necessary for supporting {@link PartitionHolder#isComplete()} if updating to a new corePartitions spec. + */ + default boolean isNumChunkSupported() + { + return false; + } + @JsonIgnore PartitionChunk createChunk(T obj); @@ -66,6 +76,22 @@ public interface ShardSpec int getNumCorePartitions(); + /** + * Creates a new ShardSpec with the specified partition number. + */ + default ShardSpec withPartitionNum(int partitionNum1) + { + throw DruidException.defensive("ShardSpec[%s] does not implement withPartitionNum", this.getClass().toString()); + } + + /** + * Creates a new ShardSpec with the specified number of core partitions. + */ + default ShardSpec withCorePartitions(int partitions) + { + throw DruidException.defensive("ShardSpec[%s] does not implement withCorePartitions", this.getClass().toString()); + } + /** * Returns the start root partition ID of the atomic update group which this segment belongs to. * @@ -119,6 +145,7 @@ default short getAtomicUpdateGroupSize() /** * if given domain ranges are not possible in this shard, return false; otherwise return true; + * * @return possibility of in domain */ @JsonIgnore diff --git a/server/src/main/java/org/apache/druid/client/indexing/ClientCompactionIntervalSpec.java b/server/src/main/java/org/apache/druid/client/indexing/ClientCompactionIntervalSpec.java index 7a7f65572319..46707e1ea55a 100644 --- a/server/src/main/java/org/apache/druid/client/indexing/ClientCompactionIntervalSpec.java +++ b/server/src/main/java/org/apache/druid/client/indexing/ClientCompactionIntervalSpec.java @@ -20,12 +20,16 @@ package org.apache.druid.client.indexing; import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonInclude; import com.fasterxml.jackson.annotation.JsonProperty; import org.apache.druid.java.util.common.IAE; +import org.apache.druid.query.SegmentDescriptor; import org.joda.time.Interval; import javax.annotation.Nullable; +import java.util.List; import java.util.Objects; +import java.util.stream.Collectors; /** * InputSpec for {@link ClientCompactionIOConfig}. @@ -38,11 +42,14 @@ public class ClientCompactionIntervalSpec private final Interval interval; @Nullable + private final List uncompactedSegments; + @Nullable private final String sha256OfSortedSegmentIds; @JsonCreator public ClientCompactionIntervalSpec( @JsonProperty("interval") Interval interval, + @JsonProperty("uncompactedSegments") @Nullable List uncompactedSegments, @JsonProperty("sha256OfSortedSegmentIds") @Nullable String sha256OfSortedSegmentIds ) { @@ -50,6 +57,22 @@ public ClientCompactionIntervalSpec( throw new IAE("Interval[%s] is empty, must specify a nonempty interval", interval); } this.interval = interval; + if (uncompactedSegments == null) { + // perform a full compaction + } else if (uncompactedSegments.isEmpty()) { + throw new IAE("Can not supply empty segments as input, please use either null or non-empty segments."); + } else if (interval != null) { + List segmentsNotInInterval = + uncompactedSegments.stream().filter(s -> !interval.contains(s.getInterval())).collect(Collectors.toList()); + if (!segmentsNotInInterval.isEmpty()) { + throw new IAE( + "Can not supply segments outside interval[%s], got segments[%s].", + interval, + segmentsNotInInterval + ); + } + } + this.uncompactedSegments = uncompactedSegments; this.sha256OfSortedSegmentIds = sha256OfSortedSegmentIds; } @@ -65,6 +88,14 @@ public Interval getInterval() return interval; } + @Nullable + @JsonProperty + @JsonInclude(JsonInclude.Include.NON_NULL) + public List getUncompactedSegments() + { + return uncompactedSegments; + } + @Nullable @JsonProperty public String getSha256OfSortedSegmentIds() @@ -83,13 +114,14 @@ public boolean equals(Object o) } ClientCompactionIntervalSpec that = (ClientCompactionIntervalSpec) o; return Objects.equals(interval, that.interval) && + Objects.equals(uncompactedSegments, that.uncompactedSegments) && Objects.equals(sha256OfSortedSegmentIds, that.sha256OfSortedSegmentIds); } @Override public int hashCode() { - return Objects.hash(interval, sha256OfSortedSegmentIds); + return Objects.hash(interval, uncompactedSegments, sha256OfSortedSegmentIds); } @Override @@ -97,6 +129,7 @@ public String toString() { return "ClientCompactionIntervalSpec{" + "interval=" + interval + + ", uncompactedSegments=" + uncompactedSegments + ", sha256OfSortedSegmentIds='" + sha256OfSortedSegmentIds + '\'' + '}'; } 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 c577bd3af146..876ffbe7f43a 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 @@ -387,6 +387,29 @@ SegmentPublishResult commitAppendSegmentsAndMetadata( @Nullable SegmentSchemaMapping segmentSchemaMapping ); + /** + * Inserts entries into the upgrade_segments table for segments that need to be upgraded + * to a new version without recompaction. This is used during incremental compaction when + * some segments in the interval don't require compaction but should be upgraded to match + * the version of newly compacted segments. + *

+ * In incremental compaction scenarios: + *

    + *
  • Some segments may already meet compaction criteria (e.g., already compacted, correct partitioning) + * and don't need to be rewritten.
  • + *
  • These segments are tracked in the upgrade_segments table via this method, mapped to their + * corresponding REPLACE lock.
  • + *
  • When the compaction task finishes via {@link #commitReplaceSegments}, these segments are + * upgraded directly to the new version to maintain version consistency across the interval.
  • + *
  • After the task completes, entries are cleaned up via {@link #deleteUpgradeSegmentsForTask}.
  • + *
+ * + * @param segmentToReplaceLock map from segment to its corresponding REPLACE lock, identifying + * which segments should be upgraded when the lock's task completes + * @return number of entries successfully inserted into the upgrade_segments table + */ + int insertIntoUpgradeSegmentsTable(Map segmentToReplaceLock); + /** * Commits segments and corresponding schema created by a REPLACE task. * This method also handles the segment upgrade scenarios that may result 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 dc0db5b56839..00d2424e1ec8 100644 --- a/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java +++ b/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java @@ -212,7 +212,10 @@ private Set doRetrieveUsedSegments( } @Override - public List> retrieveUsedSegmentsAndCreatedDates(String dataSource, List intervals) + public List> retrieveUsedSegmentsAndCreatedDates( + String dataSource, + List intervals + ) { return inReadOnlyDatasourceTransaction( dataSource, @@ -492,19 +495,16 @@ public SegmentPublishResult commitReplaceSegments( final SegmentPublishResult result = inReadWriteDatasourceTransaction( dataSource, transaction -> { - final Set segmentsToInsert = new HashSet<>(replaceSegments); - - Set upgradedSegments = createNewIdsOfAppendSegmentsAfterReplace( + final Pair, Set> newSegments = createNewSegmentsAfterReplace( dataSource, transaction, replaceSegments, locksHeldByReplaceTask ); - + final Set segmentsToInsert = newSegments.rhs; Map upgradeSegmentMetadata = new HashMap<>(); final Map upgradedFromSegmentIdMap = new HashMap<>(); - for (DataSegmentPlus dataSegmentPlus : upgradedSegments) { - segmentsToInsert.add(dataSegmentPlus.getDataSegment()); + for (DataSegmentPlus dataSegmentPlus : newSegments.lhs) { if (dataSegmentPlus.getSchemaFingerprint() != null && dataSegmentPlus.getNumRows() != null) { upgradeSegmentMetadata.put( dataSegmentPlus.getDataSegment().getId(), @@ -1250,7 +1250,7 @@ private SegmentPublishResult commitAppendSegmentsAndMetadataInTransaction( } } - insertIntoUpgradeSegmentsTable(transaction, appendSegmentToReplaceLock); + insertIntoUpgradeSegmentsTableDoWork(transaction, appendSegmentToReplaceLock); // Delete the pending segments to be committed in this transaction in batches of at most 100 int numDeletedPendingSegments = transaction.deletePendingSegments( @@ -1487,9 +1487,10 @@ private PendingSegmentRecord createNewPendingSegment( /** * Creates a new pending segment for the given datasource and interval. + * * @param partialShardSpec Shard spec info minus segment id stuff - * @param existingVersion Version of segments in interval, used to compute the version of the very first segment in - * interval + * @param existingVersion Version of segments in interval, used to compute the version of the very first segment in + * interval */ @Nullable private SegmentIdWithShardSpec createNewPendingSegment( @@ -1852,9 +1853,18 @@ protected Set insertSegments( } /** - * Creates new versions of segments appended while a "REPLACE" task was in progress. + * Creates upgraded versions of segments that were appended while a REPLACE task was in progress. + * Upgraded segments get new intervals, versions, and partition numbers to maintain consistency + * with the version created by the REPLACE task. + * + * @param dataSource The datasource being modified + * @param transaction The segment metadata transaction + * @param replaceSegments Segments being committed by the REPLACE task + * @param locksHeldByReplaceTask Replace locks held by the task + * @return Pair of (upgraded segments, all segments to insert with updated shard specs) + * @throws DruidException if a replace interval partially overlaps an appended segment */ - private Set createNewIdsOfAppendSegmentsAfterReplace( + private Pair, Set> createNewSegmentsAfterReplace( final String dataSource, final SegmentMetadataTransaction transaction, final Set replaceSegments, @@ -1864,20 +1874,22 @@ private Set createNewIdsOfAppendSegmentsAfterReplace( // If a "REPLACE" task has locked an interval, it would commit some segments // (or at least tombstones) in that interval (except in LEGACY_REPLACE ingestion mode) if (replaceSegments.isEmpty() || locksHeldByReplaceTask.isEmpty()) { - return Collections.emptySet(); + return Pair.of(Collections.emptySet(), Collections.emptySet()); } - // For each replace interval, find the number of core partitions and total partitions - final Map intervalToNumCorePartitions = new HashMap<>(); + // For each replace interval, find the current partition number final Map intervalToCurrentPartitionNum = new HashMap<>(); + // if numChunkNotSupported by all segments in an interval, we can't update the corePartitions in shardSpec + final Set numChunkNotSupported = new HashSet<>(); for (DataSegment segment : replaceSegments) { - intervalToNumCorePartitions.put(segment.getInterval(), segment.getShardSpec().getNumCorePartitions()); - int partitionNum = segment.getShardSpec().getPartitionNum(); intervalToCurrentPartitionNum.compute( segment.getInterval(), (i, value) -> value == null ? partitionNum : Math.max(value, partitionNum) ); + if (!segment.isTombstone() && !segment.getShardSpec().isNumChunkSupported()) { + numChunkNotSupported.add(segment.getInterval()); + } } // Find the segments that need to be upgraded @@ -1891,12 +1903,12 @@ private Set createNewIdsOfAppendSegmentsAfterReplace( = retrieveSegmentsById(dataSource, transaction, upgradeSegmentToLockVersion.keySet()); if (segmentsToUpgrade.isEmpty()) { - return Collections.emptySet(); + return Pair.of(Collections.emptySet(), replaceSegments); } - final Set replaceIntervals = intervalToNumCorePartitions.keySet(); - + final Set replaceIntervals = intervalToCurrentPartitionNum.keySet(); final Set upgradedSegments = new HashSet<>(); + final Set segmentsToInsert = new HashSet<>(replaceSegments); for (DataSegmentPlus oldSegmentMetadata : segmentsToUpgrade) { // Determine interval of the upgraded segment DataSegment oldSegment = oldSegmentMetadata.getDataSegment(); @@ -1925,15 +1937,16 @@ private Set createNewIdsOfAppendSegmentsAfterReplace( // but a (revoked) REPLACE lock covers this segment newInterval = oldInterval; } + if (!oldSegment.getShardSpec().isNumChunkSupported()) { + numChunkNotSupported.add(newInterval); + } // Compute shard spec of the upgraded segment final int partitionNum = intervalToCurrentPartitionNum.compute( newInterval, (i, value) -> value == null ? 0 : value + 1 ); - final int numCorePartitions = intervalToNumCorePartitions.get(newInterval); - ShardSpec shardSpec = new NumberedShardSpec(partitionNum, numCorePartitions); - + final ShardSpec shardSpec = oldSegment.getShardSpec().withPartitionNum(partitionNum); // Create upgraded segment with the correct interval, version and shard spec String lockVersion = upgradeSegmentToLockVersion.get(oldSegment.getId().toString()); DataSegment dataSegment = DataSegment.builder(oldSegment) @@ -1959,9 +1972,21 @@ private Set createNewIdsOfAppendSegmentsAfterReplace( oldSegmentMetadata.getIndexingStateFingerprint() ) ); + segmentsToInsert.add(dataSegment); } - return upgradedSegments; + // update corePartitions in shard spec + return Pair.of(upgradedSegments, segmentsToInsert.stream().map(segment -> { + Integer partitionNum = intervalToCurrentPartitionNum.get(segment.getInterval()); + if (!segment.isTombstone() + && !numChunkNotSupported.contains(segment.getInterval()) + && partitionNum != null + && partitionNum + 1 != segment.getShardSpec().getNumCorePartitions()) { + return segment.withShardSpec(segment.getShardSpec().withCorePartitions(partitionNum + 1)); + } else { + return segment; + } + }).collect(Collectors.toSet())); } /** @@ -1970,6 +1995,7 @@ private Set createNewIdsOfAppendSegmentsAfterReplace( *
  • The set of segments being committed is non-empty.
  • *
  • All segments belong to the same datasource.
  • * + * * @return Name of the common data source */ private String verifySegmentsToCommit(Collection segments) @@ -2097,17 +2123,27 @@ private SegmentMetadata getSegmentMetadataFromSchemaMappingOrUpgradeMetadata( return segmentMetadata; } + @Override + public int insertIntoUpgradeSegmentsTable(Map segmentToReplaceLock) + { + final String dataSource = verifySegmentsToCommit(segmentToReplaceLock.keySet()); + return inReadWriteDatasourceTransaction( + dataSource, + transaction -> insertIntoUpgradeSegmentsTableDoWork(transaction, segmentToReplaceLock) + ); + } + /** * Inserts entries into the upgrade_segments table in batches of size * {@link #MAX_NUM_SEGMENTS_TO_ANNOUNCE_AT_ONCE}. */ - private void insertIntoUpgradeSegmentsTable( + private int insertIntoUpgradeSegmentsTableDoWork( SegmentMetadataTransaction transaction, Map segmentToReplaceLock ) { if (segmentToReplaceLock.isEmpty()) { - return; + return 0; } final PreparedBatch batch = transaction.getHandle().prepareBatch( @@ -2118,6 +2154,7 @@ private void insertIntoUpgradeSegmentsTable( ) ); + int inserted = 0; final List>> partitions = Lists.partition( new ArrayList<>(segmentToReplaceLock.entrySet()), MAX_NUM_SEGMENTS_TO_ANNOUNCE_AT_ONCE @@ -2137,6 +2174,8 @@ private void insertIntoUpgradeSegmentsTable( for (int i = 0; i < partition.size(); ++i) { if (affectedAppendRows[i] != 1) { failedInserts.add(partition.get(i).getKey()); + } else { + inserted++; } } if (!failedInserts.isEmpty()) { @@ -2146,6 +2185,7 @@ private void insertIntoUpgradeSegmentsTable( ); } } + return inserted; } private List retrieveSegmentsById( @@ -2186,12 +2226,13 @@ private Map getAppendSegmentsCommittedDuringTask( ); ResultIterator> resultIterator = transaction.getHandle() - .createQuery(sql) - .bind("task_id", taskId) - .map( - (index, r, ctx) -> Pair.of(r.getString("segment_id"), r.getString("lock_version")) - ) - .iterator(); + .createQuery(sql) + .bind("task_id", taskId) + .map( + (index, r, ctx) -> Pair.of(r.getString( + "segment_id"), r.getString("lock_version")) + ) + .iterator(); final Map segmentIdToLockVersion = new HashMap<>(); while (resultIterator.hasNext()) { @@ -2246,8 +2287,8 @@ private Map getAppendSegmentsCommittedDuringTask( * {@link DataSourceMetadata#matches matches} the {@code endMetadata}, this * method returns immediately with success. * - * @param supervisorId The supervisor ID. Used as the PK for the corresponding metadata entry in the DB. - * @param dataSource The dataSource. Currently used only for logging purposes. + * @param supervisorId The supervisor ID. Used as the PK for the corresponding metadata entry in the DB. + * @param dataSource The dataSource. Currently used only for logging purposes. * @param startMetadata Current entry in the DB must * {@link DataSourceMetadata#matches match} this value. * @param endMetadata The updated entry will be equal to the current entry @@ -2359,8 +2400,8 @@ protected SegmentPublishResult updateDataSourceMetadataInTransaction( .execute(); publishResult = numRows == 1 - ? SegmentPublishResult.ok(Set.of()) - : SegmentPublishResult.retryableFailure("Insert failed"); + ? SegmentPublishResult.ok(Set.of()) + : SegmentPublishResult.retryableFailure("Insert failed"); } else { // Expecting a particular old metadata; use the SHA1 in a compare-and-swap UPDATE final String updateSql = StringUtils.format( @@ -2378,8 +2419,8 @@ protected SegmentPublishResult updateDataSourceMetadataInTransaction( .execute(); publishResult = numRows == 1 - ? SegmentPublishResult.ok(Set.of()) - : SegmentPublishResult.retryableFailure("Compare-and-swap update failed"); + ? SegmentPublishResult.ok(Set.of()) + : SegmentPublishResult.retryableFailure("Compare-and-swap update failed"); } if (publishResult.isSuccess()) { @@ -2462,8 +2503,8 @@ public int deleteSegments(final Set segments) final String dataSource = verifySegmentsToCommit(segments); final Set idsToDelete = segments.stream() - .map(DataSegment::getId) - .collect(Collectors.toSet()); + .map(DataSegment::getId) + .collect(Collectors.toSet()); int numDeletedSegments = inReadWriteDatasourceTransaction( dataSource, transaction -> transaction.deleteSegments(idsToDelete) @@ -2735,10 +2776,10 @@ private void markIndexingStateFingerprintsAsActive(Set segments) // Collect unique non-null indexing state fingerprints final List fingerprints = segments.stream() - .map(DataSegment::getIndexingStateFingerprint) - .filter(fp -> fp != null && !fp.isEmpty()) - .distinct() - .collect(Collectors.toList()); + .map(DataSegment::getIndexingStateFingerprint) + .filter(fp -> fp != null && !fp.isEmpty()) + .distinct() + .collect(Collectors.toList()); try { int rowsUpdated = indexingStateStorage.markIndexingStatesAsActive(fingerprints); @@ -2749,7 +2790,11 @@ private void markIndexingStateFingerprintsAsActive(Set segments) catch (Exception e) { // Log but don't fail the overall operation - the fingerprint will stay pending // and be cleaned up by the pending grace period - log.warn(e, "Failed to mark indexing states for the following fingerprints as active (Future segments publishes may remediate): %s", fingerprints); + log.warn( + e, + "Failed to mark indexing states for the following fingerprints as active (Future segments publishes may remediate): %s", + fingerprints + ); } } diff --git a/server/src/main/java/org/apache/druid/server/compaction/CompactionCandidate.java b/server/src/main/java/org/apache/druid/server/compaction/CompactionCandidate.java index af8b32ebe6db..4baf15611f87 100644 --- a/server/src/main/java/org/apache/druid/server/compaction/CompactionCandidate.java +++ b/server/src/main/java/org/apache/druid/server/compaction/CompactionCandidate.java @@ -151,6 +151,13 @@ public CompactionStatistics getUncompactedStats() ? null : currentStatus.getUncompactedStats(); } + @Nullable + public List getUncompactedSegments() + { + return (currentStatus == null || currentStatus.getUncompactedSegments() == null) + ? null : currentStatus.getUncompactedSegments(); + } + /** * Current compaction status of the time chunk corresponding to this candidate. */ diff --git a/server/src/main/java/org/apache/druid/server/compaction/CompactionCandidateSearchPolicy.java b/server/src/main/java/org/apache/druid/server/compaction/CompactionCandidateSearchPolicy.java index eeb5d612b0e4..87b8dc10f315 100644 --- a/server/src/main/java/org/apache/druid/server/compaction/CompactionCandidateSearchPolicy.java +++ b/server/src/main/java/org/apache/druid/server/compaction/CompactionCandidateSearchPolicy.java @@ -104,6 +104,11 @@ public static Eligibility fail(String messageFormat, Object... args) return new Eligibility(false, StringUtils.format(messageFormat, args), null); } + public static Eligibility incremental(String messageFormat, Object... args) + { + return new Eligibility(true, StringUtils.format(messageFormat, args), CompactionMode.INCREMENTAL_COMPACTION); + } + @Override public boolean equals(Object object) { diff --git a/server/src/main/java/org/apache/druid/server/compaction/CompactionMode.java b/server/src/main/java/org/apache/druid/server/compaction/CompactionMode.java index c4ccfb7be28f..1b037e476f9e 100644 --- a/server/src/main/java/org/apache/druid/server/compaction/CompactionMode.java +++ b/server/src/main/java/org/apache/druid/server/compaction/CompactionMode.java @@ -27,5 +27,6 @@ public enum CompactionMode /** * Indicates that all existing segments of the interval will be picked for compaction. */ - FULL_COMPACTION; + FULL_COMPACTION, + INCREMENTAL_COMPACTION; } diff --git a/server/src/main/java/org/apache/druid/server/compaction/CompactionStatus.java b/server/src/main/java/org/apache/druid/server/compaction/CompactionStatus.java index 10076a719241..0b0ba1ef51a9 100644 --- a/server/src/main/java/org/apache/druid/server/compaction/CompactionStatus.java +++ b/server/src/main/java/org/apache/druid/server/compaction/CompactionStatus.java @@ -59,7 +59,7 @@ public class CompactionStatus { private static final Logger log = new Logger(CompactionStatus.class); - private static final CompactionStatus COMPLETE = new CompactionStatus(State.COMPLETE, null, null, null); + private static final CompactionStatus COMPLETE = new CompactionStatus(State.COMPLETE, null, null, null, null); public static final String NEVER_COMPACTED_REASON = "not compacted yet"; public enum State @@ -93,20 +93,28 @@ public enum State private final State state; private final String reason; + + @Nullable private final CompactionStatistics compactedStats; + @Nullable private final CompactionStatistics uncompactedStats; + @Nullable + private final List uncompactedSegments; + private CompactionStatus( State state, String reason, - CompactionStatistics compactedStats, - CompactionStatistics uncompactedStats + @Nullable CompactionStatistics compactedStats, + @Nullable CompactionStatistics uncompactedStats, + @Nullable List uncompactedSegments ) { this.state = state; this.reason = reason; this.compactedStats = compactedStats; this.uncompactedStats = uncompactedStats; + this.uncompactedSegments = uncompactedSegments; } public boolean isComplete() @@ -139,6 +147,11 @@ public CompactionStatistics getUncompactedStats() return uncompactedStats; } + public List getUncompactedSegments() + { + return uncompactedSegments; + } + @Override public String toString() { @@ -152,12 +165,13 @@ public String toString() public static CompactionStatus pending(String reasonFormat, Object... args) { - return new CompactionStatus(State.PENDING, StringUtils.format(reasonFormat, args), null, null); + return new CompactionStatus(State.PENDING, StringUtils.format(reasonFormat, args), null, null, null); } public static CompactionStatus pending( CompactionStatistics compactedStats, CompactionStatistics uncompactedStats, + List uncompactedSegments, String reasonFormat, Object... args ) @@ -166,7 +180,8 @@ public static CompactionStatus pending( State.PENDING, StringUtils.format(reasonFormat, args), compactedStats, - uncompactedStats + uncompactedStats, + uncompactedSegments ); } @@ -244,12 +259,12 @@ private static String asString(PartitionsSpec partitionsSpec) public static CompactionStatus skipped(String reasonFormat, Object... args) { - return new CompactionStatus(State.SKIPPED, StringUtils.format(reasonFormat, args), null, null); + return new CompactionStatus(State.SKIPPED, StringUtils.format(reasonFormat, args), null, null, null); } public static CompactionStatus running(String message) { - return new CompactionStatus(State.RUNNING, message, null, null); + return new CompactionStatus(State.RUNNING, message, null, null, null); } /** @@ -426,7 +441,8 @@ private CompactionStatus evaluate() } else { return CompactionStatus.pending( createStats(this.compactedSegments), - createStats(uncompactedSegments), + createStats(this.uncompactedSegments), + this.uncompactedSegments, reasonsForCompaction.get(0) ); } diff --git a/server/src/main/java/org/apache/druid/server/compaction/CompactionStatusTracker.java b/server/src/main/java/org/apache/druid/server/compaction/CompactionStatusTracker.java index e98e3575f47b..47848c0a568c 100644 --- a/server/src/main/java/org/apache/druid/server/compaction/CompactionStatusTracker.java +++ b/server/src/main/java/org/apache/druid/server/compaction/CompactionStatusTracker.java @@ -91,8 +91,26 @@ public CompactionStatus computeCompactionStatus( CompactionCandidateSearchPolicy searchPolicy ) { - // Skip intervals that already have a running task final CompactionTaskStatus lastTaskStatus = getLatestTaskStatus(candidate); + CompactionStatus status = deriveCompactionStatus(lastTaskStatus); + if (status != null) { + return status; + } + + // Skip intervals that have been filtered out by the policy + final CompactionCandidateSearchPolicy.Eligibility eligibility + = searchPolicy.checkEligibilityForCompaction(candidate, lastTaskStatus); + if (eligibility.isEligible()) { + return CompactionStatus.pending("Not compacted yet"); + } else { + return CompactionStatus.skipped("Rejected by search policy: %s", eligibility.getReason()); + } + } + + @Nullable + public CompactionStatus deriveCompactionStatus(CompactionTaskStatus lastTaskStatus) + { + // Skip intervals that already have a running task if (lastTaskStatus != null && lastTaskStatus.getState() == TaskState.RUNNING) { return CompactionStatus.running("Task for interval is already running"); } @@ -106,15 +124,7 @@ public CompactionStatus computeCompactionStatus( "Segment timeline not updated since last compaction task succeeded" ); } - - // Skip intervals that have been filtered out by the policy - final CompactionCandidateSearchPolicy.Eligibility eligibility - = searchPolicy.checkEligibilityForCompaction(candidate, lastTaskStatus); - if (eligibility.isEligible()) { - return CompactionStatus.pending("Not compacted yet"); - } else { - return CompactionStatus.skipped("Rejected by search policy: %s", eligibility.getReason()); - } + return null; } /** diff --git a/server/src/main/java/org/apache/druid/server/compaction/MostFragmentedIntervalFirstPolicy.java b/server/src/main/java/org/apache/druid/server/compaction/MostFragmentedIntervalFirstPolicy.java index 345988ee7fc2..ac769cfde15d 100644 --- a/server/src/main/java/org/apache/druid/server/compaction/MostFragmentedIntervalFirstPolicy.java +++ b/server/src/main/java/org/apache/druid/server/compaction/MostFragmentedIntervalFirstPolicy.java @@ -47,6 +47,7 @@ public class MostFragmentedIntervalFirstPolicy extends BaseCandidateSearchPolicy private final int minUncompactedCount; private final HumanReadableBytes minUncompactedBytes; private final HumanReadableBytes maxAverageUncompactedBytesPerSegment; + private final double incrementalCompactionUncompactedRatioThreshold; @JsonCreator public MostFragmentedIntervalFirstPolicy( @@ -54,6 +55,8 @@ public MostFragmentedIntervalFirstPolicy( @JsonProperty("minUncompactedBytes") @Nullable HumanReadableBytes minUncompactedBytes, @JsonProperty("maxAverageUncompactedBytesPerSegment") @Nullable HumanReadableBytes maxAverageUncompactedBytesPerSegment, + @JsonProperty("incrementalCompactionUncompactedRatioThreshold") @Nullable + Double incrementalCompactionUncompactedRatioThreshold, @JsonProperty("priorityDatasource") @Nullable String priorityDatasource ) { @@ -69,11 +72,20 @@ public MostFragmentedIntervalFirstPolicy( "'minUncompactedCount'[%s] must be greater than 0", maxAverageUncompactedBytesPerSegment ); + InvalidInput.conditionalException( + incrementalCompactionUncompactedRatioThreshold == null + || (incrementalCompactionUncompactedRatioThreshold >= 0.0d + && incrementalCompactionUncompactedRatioThreshold < 1.0d), + "'incrementalCompactionUncompactedRatioThreshold'[%s] must be between 0.0 and 1.0", + incrementalCompactionUncompactedRatioThreshold + ); this.minUncompactedCount = Configs.valueOrDefault(minUncompactedCount, 100); this.minUncompactedBytes = Configs.valueOrDefault(minUncompactedBytes, SIZE_10_MB); this.maxAverageUncompactedBytesPerSegment = Configs.valueOrDefault(maxAverageUncompactedBytesPerSegment, SIZE_2_GB); + this.incrementalCompactionUncompactedRatioThreshold = + Configs.valueOrDefault(incrementalCompactionUncompactedRatioThreshold, 0.0d); } /** @@ -106,6 +118,17 @@ public HumanReadableBytes getMaxAverageUncompactedBytesPerSegment() return maxAverageUncompactedBytesPerSegment; } + /** + * Threshold ratio of uncompacted bytes to compacted bytes below which + * incremental compaction is eligible instead of full compaction. + * Default value is 0.0. + */ + @JsonProperty + public double getIncrementalCompactionUncompactedRatioThreshold() + { + return incrementalCompactionUncompactedRatioThreshold; + } + @Override protected Comparator getSegmentComparator() { @@ -124,7 +147,12 @@ public boolean equals(Object o) MostFragmentedIntervalFirstPolicy policy = (MostFragmentedIntervalFirstPolicy) o; return minUncompactedCount == policy.minUncompactedCount && Objects.equals(minUncompactedBytes, policy.minUncompactedBytes) - && Objects.equals(maxAverageUncompactedBytesPerSegment, policy.maxAverageUncompactedBytesPerSegment); + && Objects.equals(maxAverageUncompactedBytesPerSegment, policy.maxAverageUncompactedBytesPerSegment) + // Use Double.compare instead of == to handle NaN correctly and keep equals() consistent with hashCode() (especially for +0.0 vs -0.0). + && Double.compare( + incrementalCompactionUncompactedRatioThreshold, + policy.incrementalCompactionUncompactedRatioThreshold + ) == 0; } @Override @@ -134,19 +162,22 @@ public int hashCode() super.hashCode(), minUncompactedCount, minUncompactedBytes, - maxAverageUncompactedBytesPerSegment + maxAverageUncompactedBytesPerSegment, + incrementalCompactionUncompactedRatioThreshold ); } @Override public String toString() { - return "MostFragmentedIntervalFirstPolicy{" + - "minUncompactedCount=" + minUncompactedCount + - ", minUncompactedBytes=" + minUncompactedBytes + - ", maxAverageUncompactedBytesPerSegment=" + maxAverageUncompactedBytesPerSegment + - ", priorityDataSource='" + getPriorityDatasource() + '\'' + - '}'; + return + "MostFragmentedIntervalFirstPolicy{" + + "minUncompactedCount=" + minUncompactedCount + + ", minUncompactedBytes=" + minUncompactedBytes + + ", maxAverageUncompactedBytesPerSegment=" + maxAverageUncompactedBytesPerSegment + + ", incrementalCompactionUncompactedRatioThreshold=" + incrementalCompactionUncompactedRatioThreshold + + ", priorityDataSource='" + getPriorityDatasource() + '\'' + + '}'; } private int compare(CompactionCandidate candidateA, CompactionCandidate candidateB) @@ -185,6 +216,16 @@ public Eligibility checkEligibilityForCompaction( "Average size[%,d] of uncompacted segments in interval must be at most [%,d]", avgSegmentSize, maxAverageUncompactedBytesPerSegment.getBytes() ); + } + + final double uncompactedBytesRatio = (double) uncompacted.getTotalBytes() / + (uncompacted.getTotalBytes() + candidate.getCompactedStats().getTotalBytes()); + if (uncompactedBytesRatio < incrementalCompactionUncompactedRatioThreshold) { + return Eligibility.incremental( + "Uncompacted bytes ratio[%.2f] is below threshold[%.2f]", + uncompactedBytesRatio, + incrementalCompactionUncompactedRatioThreshold + ); } else { return Eligibility.OK; } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/UserCompactionTaskQueryTuningConfig.java b/server/src/main/java/org/apache/druid/server/coordinator/UserCompactionTaskQueryTuningConfig.java index 9c1bfb200648..ca93ef4b180e 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/UserCompactionTaskQueryTuningConfig.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/UserCompactionTaskQueryTuningConfig.java @@ -88,4 +88,171 @@ public Integer getMaxRowsPerSegment() { throw new UnsupportedOperationException(); } + + public static Builder builder() + { + return new Builder(); + } + + public static class Builder + { + private Integer maxRowsInMemory; + private AppendableIndexSpec appendableIndexSpec; + private Long maxBytesInMemory; + private Long maxTotalRows; + private SplitHintSpec splitHintSpec; + private PartitionsSpec partitionsSpec; + private IndexSpec indexSpec; + private IndexSpec indexSpecForIntermediatePersists; + private Integer maxPendingPersists; + private Long pushTimeout; + private SegmentWriteOutMediumFactory segmentWriteOutMediumFactory; + private Integer maxNumConcurrentSubTasks; + private Integer maxRetry; + private Long taskStatusCheckPeriodMs; + private Duration chatHandlerTimeout; + private Integer chatHandlerNumRetries; + private Integer maxNumSegmentsToMerge; + private Integer totalNumMergeTasks; + private Integer maxColumnsToMerge; + + public Builder maxRowsInMemory(Integer maxRowsInMemory) + { + this.maxRowsInMemory = maxRowsInMemory; + return this; + } + + public Builder appendableIndexSpec(AppendableIndexSpec appendableIndexSpec) + { + this.appendableIndexSpec = appendableIndexSpec; + return this; + } + + public Builder maxBytesInMemory(Long maxBytesInMemory) + { + this.maxBytesInMemory = maxBytesInMemory; + return this; + } + + public Builder maxTotalRows(Long maxTotalRows) + { + this.maxTotalRows = maxTotalRows; + return this; + } + + public Builder splitHintSpec(SplitHintSpec splitHintSpec) + { + this.splitHintSpec = splitHintSpec; + return this; + } + + public Builder partitionsSpec(PartitionsSpec partitionsSpec) + { + this.partitionsSpec = partitionsSpec; + return this; + } + + public Builder indexSpec(IndexSpec indexSpec) + { + this.indexSpec = indexSpec; + return this; + } + + public Builder indexSpecForIntermediatePersists(IndexSpec indexSpecForIntermediatePersists) + { + this.indexSpecForIntermediatePersists = indexSpecForIntermediatePersists; + return this; + } + + public Builder maxPendingPersists(Integer maxPendingPersists) + { + this.maxPendingPersists = maxPendingPersists; + return this; + } + + public Builder pushTimeout(Long pushTimeout) + { + this.pushTimeout = pushTimeout; + return this; + } + + public Builder segmentWriteOutMediumFactory(SegmentWriteOutMediumFactory segmentWriteOutMediumFactory) + { + this.segmentWriteOutMediumFactory = segmentWriteOutMediumFactory; + return this; + } + + public Builder maxNumConcurrentSubTasks(Integer maxNumConcurrentSubTasks) + { + this.maxNumConcurrentSubTasks = maxNumConcurrentSubTasks; + return this; + } + + public Builder maxRetry(Integer maxRetry) + { + this.maxRetry = maxRetry; + return this; + } + + public Builder taskStatusCheckPeriodMs(Long taskStatusCheckPeriodMs) + { + this.taskStatusCheckPeriodMs = taskStatusCheckPeriodMs; + return this; + } + + public Builder chatHandlerTimeout(Duration chatHandlerTimeout) + { + this.chatHandlerTimeout = chatHandlerTimeout; + return this; + } + + public Builder chatHandlerNumRetries(Integer chatHandlerNumRetries) + { + this.chatHandlerNumRetries = chatHandlerNumRetries; + return this; + } + + public Builder maxNumSegmentsToMerge(Integer maxNumSegmentsToMerge) + { + this.maxNumSegmentsToMerge = maxNumSegmentsToMerge; + return this; + } + + public Builder totalNumMergeTasks(Integer totalNumMergeTasks) + { + this.totalNumMergeTasks = totalNumMergeTasks; + return this; + } + + public Builder maxColumnsToMerge(Integer maxColumnsToMerge) + { + this.maxColumnsToMerge = maxColumnsToMerge; + return this; + } + + public UserCompactionTaskQueryTuningConfig build() + { + return new UserCompactionTaskQueryTuningConfig( + maxRowsInMemory, + appendableIndexSpec, + maxBytesInMemory, + maxTotalRows, + splitHintSpec, + partitionsSpec, + indexSpec, + indexSpecForIntermediatePersists, + maxPendingPersists, + pushTimeout, + segmentWriteOutMediumFactory, + maxNumConcurrentSubTasks, + maxRetry, + taskStatusCheckPeriodMs, + chatHandlerTimeout, + chatHandlerNumRetries, + maxNumSegmentsToMerge, + totalNumMergeTasks, + maxColumnsToMerge + ); + } + } } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java b/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java index c376aac34406..f34aae68dd04 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java @@ -34,15 +34,18 @@ import org.apache.druid.common.guava.FutureUtils; import org.apache.druid.common.utils.IdUtils; import org.apache.druid.data.input.impl.AggregateProjectionSpec; +import org.apache.druid.error.DruidException; import org.apache.druid.indexer.CompactionEngine; import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.common.granularity.GranularityType; import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.query.SegmentDescriptor; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.rpc.indexing.OverlordClient; import org.apache.druid.segment.transform.CompactionTransformSpec; import org.apache.druid.server.compaction.CompactionCandidate; import org.apache.druid.server.compaction.CompactionCandidateSearchPolicy; +import org.apache.druid.server.compaction.CompactionMode; import org.apache.druid.server.compaction.CompactionSegmentIterator; import org.apache.druid.server.compaction.CompactionSlotManager; import org.apache.druid.server.compaction.CompactionSnapshotBuilder; @@ -262,6 +265,7 @@ private int submitCompactionTasks( final ClientCompactionTaskQuery taskPayload = createCompactionTask( entry, + CompactionMode.FULL_COMPACTION, config, defaultEngine, null, @@ -291,6 +295,7 @@ private int submitCompactionTasks( */ public static ClientCompactionTaskQuery createCompactionTask( CompactionCandidate candidate, + CompactionMode compactionMode, DataSourceCompactionConfig config, CompactionEngine defaultEngine, String indexingStateFingerprint, @@ -380,6 +385,7 @@ public static ClientCompactionTaskQuery createCompactionTask( return compactSegments( candidate, + compactionMode, config.getTaskPriority(), ClientCompactionTaskQueryTuningConfig.from( config.getTuningConfig(), @@ -439,6 +445,7 @@ public Map getAutoCompactionSnapshot() private static ClientCompactionTaskQuery compactSegments( CompactionCandidate entry, + CompactionMode compactionMode, int compactionTaskPriority, ClientCompactionTaskQueryTuningConfig tuningConfig, ClientCompactionTaskGranularitySpec granularitySpec, @@ -463,14 +470,30 @@ private static ClientCompactionTaskQuery compactSegments( context.put("priority", compactionTaskPriority); final String taskId = IdUtils.newTaskId(TASK_ID_PREFIX, ClientCompactionTaskQuery.TYPE, dataSource, null); + final ClientCompactionIntervalSpec clientCompactionIntervalSpec; + switch (compactionMode) { + case FULL_COMPACTION: + clientCompactionIntervalSpec = new ClientCompactionIntervalSpec(entry.getCompactionInterval(), null, null); + break; + case INCREMENTAL_COMPACTION: + List uncompacted = entry.getUncompactedSegments() + .stream() + .map(DataSegment::toDescriptor) + .toList(); + clientCompactionIntervalSpec = new ClientCompactionIntervalSpec( + entry.getCompactionInterval(), + uncompacted, + null + ); + break; + default: + throw DruidException.defensive("unexpected compaction mode[%s]", compactionMode); + } return new ClientCompactionTaskQuery( taskId, dataSource, - new ClientCompactionIOConfig( - new ClientCompactionIntervalSpec(entry.getCompactionInterval(), null), - dropExisting - ), + new ClientCompactionIOConfig(clientCompactionIntervalSpec, dropExisting), tuningConfig, granularitySpec, dimensionsSpec, diff --git a/server/src/test/java/org/apache/druid/client/indexing/ClientCompactionIntervalSpecTest.java b/server/src/test/java/org/apache/druid/client/indexing/ClientCompactionIntervalSpecTest.java index 46ecc64d72d1..096a82c8c158 100644 --- a/server/src/test/java/org/apache/druid/client/indexing/ClientCompactionIntervalSpecTest.java +++ b/server/src/test/java/org/apache/druid/client/indexing/ClientCompactionIntervalSpecTest.java @@ -19,19 +19,25 @@ package org.apache.druid.client.indexing; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableList; +import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.granularity.Granularities; +import org.apache.druid.query.SegmentDescriptor; import org.apache.druid.segment.IndexIO; import org.apache.druid.server.compaction.CompactionCandidate; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.partition.NoneShardSpec; +import org.joda.time.Interval; import org.junit.Assert; import org.junit.Test; import java.util.ArrayList; import java.util.HashMap; +import java.util.List; public class ClientCompactionIntervalSpecTest { @@ -112,4 +118,40 @@ public void testFromSegmentWithFinerSegmentGranularityAndUmbrellaIntervalNotAlig // Hence the compaction interval is modified to aling with the segmentGranularity Assert.assertEquals(Intervals.of("2015-02-09/2015-04-20"), actual.getCompactionInterval()); } + + @Test + public void testClientCompactionIntervalSpec_throwsException_whenEmptySegmentsList() + { + Interval interval = Intervals.of("2015-04-11/2015-04-12"); + List emptySegments = List.of(); + + Assert.assertThrows( + IAE.class, + () -> new ClientCompactionIntervalSpec(interval, emptySegments, null) + ); + } + + @Test + public void testClientCompactionIntervalSpec_serde() throws Exception + { + ObjectMapper mapper = new DefaultObjectMapper(); + Interval interval = Intervals.of("2015-04-11/2015-04-12"); + List segments = List.of( + new SegmentDescriptor(Intervals.of("2015-04-11/2015-04-12"), "v1", 0) + ); + + // Test with uncompactedSegments (incremental compaction) + ClientCompactionIntervalSpec withSegments = new ClientCompactionIntervalSpec(interval, segments, "sha256hash"); + String json1 = mapper.writeValueAsString(withSegments); + ClientCompactionIntervalSpec deserialized1 = mapper.readValue(json1, ClientCompactionIntervalSpec.class); + Assert.assertEquals(withSegments, deserialized1); + Assert.assertEquals(segments, deserialized1.getUncompactedSegments()); + + // Test without uncompactedSegments (full compaction) + ClientCompactionIntervalSpec withoutSegments = new ClientCompactionIntervalSpec(interval, null, null); + String json2 = mapper.writeValueAsString(withoutSegments); + ClientCompactionIntervalSpec deserialized2 = mapper.readValue(json2, ClientCompactionIntervalSpec.class); + Assert.assertEquals(withoutSegments, deserialized2); + Assert.assertNull(deserialized2.getUncompactedSegments()); + } } 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 9add89949c83..8e166dbfcf91 100644 --- a/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java +++ b/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java @@ -74,6 +74,7 @@ import org.apache.druid.timeline.partition.NumberedShardSpec; import org.apache.druid.timeline.partition.PartialShardSpec; import org.apache.druid.timeline.partition.PartitionIds; +import org.apache.druid.timeline.partition.ShardSpec; import org.apache.druid.timeline.partition.SingleDimensionShardSpec; import org.apache.druid.timeline.partition.TombstoneShardSpec; import org.assertj.core.api.Assertions; @@ -559,14 +560,149 @@ public void testCommitReplaceSegments() ImmutableMap.of("path", "b-" + i), ImmutableList.of("dim1"), ImmutableList.of("m1"), - new NumberedShardSpec(i, 9), + new NumberedShardSpec(i - 1, 8), + 9, + 100 + ); + replacingSegments.add(segment); + } + + Assert.assertTrue(coordinator.commitReplaceSegments(replacingSegments, Set.of(replaceLock), null).isSuccess()); + + Assert.assertEquals( + 2L * segmentsAppendedWithReplaceLock.size() + replacingSegments.size(), + retrieveUsedSegmentIds(derbyConnectorRule.metadataTablesConfigSupplier().get()).size() + ); + + 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.assertEquals(usedSegments, coordinator.retrieveAllUsedSegments("foo", Segments.ONLY_VISIBLE)); + + 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()); + for (DataSegment segmentReplicaWithNewVersion : usedSegments) { + 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; + } + } + Assert.assertTrue(hasBeenCarriedForward); + } + + List pendingSegmentsInInterval = + coordinator.getPendingSegments("foo", Intervals.of("2023-01-01/2023-02-01")); + Assert.assertEquals(2, pendingSegmentsInInterval.size()); + final SegmentId rootPendingSegmentId = pendingSegmentInInterval.getId().asSegmentId(); + if (pendingSegmentsInInterval.get(0).getUpgradedFromSegmentId() == null) { + Assert.assertEquals(rootPendingSegmentId, pendingSegmentsInInterval.get(0).getId().asSegmentId()); + Assert.assertEquals(rootPendingSegmentId.toString(), pendingSegmentsInInterval.get(1).getUpgradedFromSegmentId()); + } else { + Assert.assertEquals(rootPendingSegmentId, pendingSegmentsInInterval.get(1).getId().asSegmentId()); + Assert.assertEquals(rootPendingSegmentId.toString(), pendingSegmentsInInterval.get(0).getUpgradedFromSegmentId()); + } + + List pendingSegmentsOutsideInterval = + coordinator.getPendingSegments("foo", Intervals.of("2023-04-01/2023-05-01")); + Assert.assertEquals(1, pendingSegmentsOutsideInterval.size()); + Assert.assertEquals( + pendingSegmentOutsideInterval.getId().asSegmentId(), pendingSegmentsOutsideInterval.get(0).getId().asSegmentId() + ); + } + + @Test + public void testCommitReplaceSegmentsWithUpdatedCorePartitions() + { + // this test is very similar to testCommitReplaceSegments, except both append/replace segments use DimensionRangeShardSpec + final ReplaceTaskLock replaceLock = new ReplaceTaskLock("g1", Intervals.of("2023-01-01/2023-02-01"), "2023-02-01"); + final Set segmentsAppendedWithReplaceLock = new HashSet<>(); + final Map appendedSegmentToReplaceLockMap = new HashMap<>(); + final PendingSegmentRecord pendingSegmentInInterval = PendingSegmentRecord.create( + new SegmentIdWithShardSpec( + "foo", + Intervals.of("2023-01-01/2023-01-02"), + "2023-01-02", + new NumberedShardSpec(100, 0) + ), + "", + "", + null, + "append" + ); + final PendingSegmentRecord pendingSegmentOutsideInterval = PendingSegmentRecord.create( + new SegmentIdWithShardSpec( + "foo", + Intervals.of("2023-04-01/2023-04-02"), + "2023-01-02", + new NumberedShardSpec(100, 0) + ), + "", + "", + null, + "append" + ); + for (int i = 1; i < 9; i++) { + final DataSegment segment = new DataSegment( + "foo", + Intervals.of("2023-01-0" + i + "/2023-01-0" + (i + 1)), + "2023-01-0" + i, + ImmutableMap.of("path", "a-" + i), + ImmutableList.of("dim1"), + ImmutableList.of("m1"), + new DimensionRangeShardSpec(List.of("dim1"), null, null, i - 1, 8), + 9, + 100 + ); + segmentsAppendedWithReplaceLock.add(segment); + appendedSegmentToReplaceLockMap.put(segment, replaceLock); + } + + segmentSchemaTestUtils.insertUsedSegments(segmentsAppendedWithReplaceLock, Collections.emptyMap()); + insertPendingSegments( + "foo", + List.of(pendingSegmentInInterval, pendingSegmentOutsideInterval), + true + ); + insertIntoUpgradeSegmentsTable(appendedSegmentToReplaceLockMap, derbyConnectorRule.metadataTablesConfigSupplier().get()); + + final Set replacingSegments = new HashSet<>(); + for (int i = 1; i < 9; i++) { + final DataSegment segment = new DataSegment( + "foo", + Intervals.of("2023-01-01/2023-02-01"), + "2023-02-01", + ImmutableMap.of("path", "b-" + i), + ImmutableList.of("dim1"), + ImmutableList.of("m1"), + new DimensionRangeShardSpec(List.of("dim1"), null, null, i - 1, 8), 9, 100 ); replacingSegments.add(segment); } - coordinator.commitReplaceSegments(replacingSegments, ImmutableSet.of(replaceLock), null); + Assert.assertTrue(coordinator.commitReplaceSegments(replacingSegments, Set.of(replaceLock), null).isSuccess()); Assert.assertEquals( 2L * segmentsAppendedWithReplaceLock.size() + replacingSegments.size(), @@ -587,6 +723,12 @@ public void testCommitReplaceSegments() } usedSegments.removeAll(segmentsAppendedWithReplaceLock); + Set fetched = coordinator.retrieveAllUsedSegments("foo", Segments.ONLY_VISIBLE); + Assert.assertEquals(usedSegments, fetched); + // all segments have the same corePartitions, exactly the size of replaced + appended + List shardSpecs = fetched.stream().map(DataSegment::getShardSpec).toList(); + Assert.assertTrue(shardSpecs.stream().allMatch(s -> s.getNumCorePartitions() == usedSegments.size())); + Assert.assertTrue(shardSpecs.stream().allMatch(s -> s instanceof DimensionRangeShardSpec)); Assert.assertTrue(usedSegments.containsAll(replacingSegments)); for (DataSegment replaceSegment : replacingSegments) { Assert.assertNull(upgradedFromSegmentIdMap.get(replaceSegment.getId().toString())); diff --git a/server/src/test/java/org/apache/druid/server/compaction/MostFragmentedIntervalFirstPolicyTest.java b/server/src/test/java/org/apache/druid/server/compaction/MostFragmentedIntervalFirstPolicyTest.java index 1b93bfa03a55..08134db66bb6 100644 --- a/server/src/test/java/org/apache/druid/server/compaction/MostFragmentedIntervalFirstPolicyTest.java +++ b/server/src/test/java/org/apache/druid/server/compaction/MostFragmentedIntervalFirstPolicyTest.java @@ -39,7 +39,8 @@ public class MostFragmentedIntervalFirstPolicyTest @Test public void test_thresholdValues_ofDefaultPolicy() { - final MostFragmentedIntervalFirstPolicy policy = new MostFragmentedIntervalFirstPolicy(null, null, null, null); + final MostFragmentedIntervalFirstPolicy policy = + new MostFragmentedIntervalFirstPolicy(null, null, null, null, null); Assertions.assertEquals(100, policy.getMinUncompactedCount()); Assertions.assertEquals(new HumanReadableBytes("10MiB"), policy.getMinUncompactedBytes()); Assertions.assertEquals(new HumanReadableBytes("2GiB"), policy.getMaxAverageUncompactedBytesPerSegment()); @@ -54,6 +55,7 @@ public void test_checkEligibilityForCompaction_fails_ifUncompactedCountLessThanC minUncompactedCount, HumanReadableBytes.valueOf(1), HumanReadableBytes.valueOf(10_000), + null, null ); @@ -77,6 +79,7 @@ public void test_checkEligibilityForCompaction_fails_ifUncompactedBytesLessThanC 1, minUncompactedBytes, HumanReadableBytes.valueOf(10_000), + null, null ); @@ -100,6 +103,7 @@ public void test_checkEligibilityForCompaction_fails_ifAvgSegmentSizeGreaterThan 1, HumanReadableBytes.valueOf(100), maxAvgSegmentSize, + null, null ); @@ -122,6 +126,7 @@ public void test_policy_favorsIntervalWithMoreUncompactedSegments_ifTotalBytesIs 1, HumanReadableBytes.valueOf(1), HumanReadableBytes.valueOf(10_000), + null, null ); @@ -142,6 +147,7 @@ public void test_policy_favorsIntervalWithMoreUncompactedSegments_ifAverageSizeI 1, HumanReadableBytes.valueOf(1), HumanReadableBytes.valueOf(10_000), + null, null ); @@ -162,6 +168,7 @@ public void test_policy_favorsIntervalWithSmallerSegments_ifCountIsEqual() 1, HumanReadableBytes.valueOf(1), HumanReadableBytes.valueOf(10_000), + null, null ); @@ -182,6 +189,7 @@ public void test_compareCandidates_returnsZeroIfSegmentCountAndAvgSizeScaleEquiv 100, HumanReadableBytes.valueOf(1), HumanReadableBytes.valueOf(100), + null, null ); @@ -211,6 +219,7 @@ public void test_serde_allFieldsSet() throws IOException 1, HumanReadableBytes.valueOf(2), HumanReadableBytes.valueOf(3), + 0.5, "foo" ); final DefaultObjectMapper mapper = new DefaultObjectMapper(); @@ -222,13 +231,93 @@ public void test_serde_allFieldsSet() throws IOException @Test public void test_serde_noFieldsSet() throws IOException { - final MostFragmentedIntervalFirstPolicy policy = new MostFragmentedIntervalFirstPolicy(null, null, null, null); + final MostFragmentedIntervalFirstPolicy policy = + new MostFragmentedIntervalFirstPolicy(null, null, null, null, null); final DefaultObjectMapper mapper = new DefaultObjectMapper(); final CompactionCandidateSearchPolicy policy2 = mapper.readValue(mapper.writeValueAsString(policy), CompactionCandidateSearchPolicy.class); Assertions.assertEquals(policy, policy2); } + @Test + public void test_compactionMode_returnsIncrementalCompaction_whenRatioBelowThreshold() + { + // Set threshold to 0.5 (50%) + final MostFragmentedIntervalFirstPolicy policy = new MostFragmentedIntervalFirstPolicy( + 1, + HumanReadableBytes.valueOf(1), + HumanReadableBytes.valueOf(10_000), + 0.5, + null + ); + + final CompactionStatistics compacted = CompactionStatistics.create(1200L, 10, 1L); + final CompactionStatistics uncompacted = CompactionStatistics.create(400L, 100, 1L); + final CompactionStatus status = CompactionStatus.pending(compacted, uncompacted, List.of(SEGMENT), ""); + + final CompactionCandidate candidate = CompactionCandidate.from(List.of(SEGMENT), null).withCurrentStatus(status); + final CompactionCandidateSearchPolicy.Eligibility eligibility = + policy.checkEligibilityForCompaction(candidate, null); + + Assertions.assertEquals(CompactionMode.INCREMENTAL_COMPACTION, eligibility.getMode()); + Assertions.assertTrue(eligibility.isEligible()); + } + + @Test + public void test_compactionMode_returnsFullCompaction_whenRatioAboveThreshold() + { + // Set threshold to 0.5 (50%) + final MostFragmentedIntervalFirstPolicy policy = new MostFragmentedIntervalFirstPolicy( + 1, + HumanReadableBytes.valueOf(1), + HumanReadableBytes.valueOf(10_000), + 0.5, + null + ); + + final CompactionStatus status = + CompactionStatus.pending( + CompactionStatistics.create(500L, 5, 1), + CompactionStatistics.create(600L, 100, 1), + List.of(), + "" + ); + final CompactionCandidate candidate = CompactionCandidate.from(List.of(SEGMENT), null).withCurrentStatus(status); + final CompactionCandidateSearchPolicy.Eligibility eligibility = + policy.checkEligibilityForCompaction(candidate, null); + + Assertions.assertEquals(CompactionMode.FULL_COMPACTION, eligibility.getMode()); + Assertions.assertTrue(eligibility.isEligible()); + } + + @Test + public void test_compactionMode_returnsFullCompaction_whenThresholdIsDefault() + { + // Default threshold is 0.0 + final MostFragmentedIntervalFirstPolicy policy = new MostFragmentedIntervalFirstPolicy( + 1, + HumanReadableBytes.valueOf(1), + HumanReadableBytes.valueOf(10_000), + null, + null + ); + + // With default threshold 0.0, any positive ratio >= 0.0, so always FULL_COMPACTION_ELIGIBLE + final CompactionStatus status = + CompactionStatus.pending( + CompactionStatistics.create(1_000L, 10, 1), + CompactionStatistics.create(100L, 100, 1), + List.of(), + "" + ); + final CompactionCandidate candidate = CompactionCandidate.from(List.of(SEGMENT), null).withCurrentStatus(status); + final CompactionCandidateSearchPolicy.Eligibility eligibility = + policy.checkEligibilityForCompaction(candidate, null); + + Assertions.assertEquals(CompactionMode.FULL_COMPACTION, eligibility.getMode()); + Assertions.assertTrue(eligibility.isEligible()); + } + private CompactionCandidate createCandidate(int numSegments, long avgSizeBytes) { final CompactionStatistics dummyCompactedStats = CompactionStatistics.create(1L, 1L, 1L); @@ -238,7 +327,12 @@ private CompactionCandidate createCandidate(int numSegments, long avgSizeBytes) 1L ); return CompactionCandidate.from(List.of(SEGMENT), null) - .withCurrentStatus(CompactionStatus.pending(dummyCompactedStats, uncompactedStats, "")); + .withCurrentStatus(CompactionStatus.pending( + dummyCompactedStats, + uncompactedStats, + List.of(), + "" + )); } private void verifyCandidateIsEligible(CompactionCandidate candidate, MostFragmentedIntervalFirstPolicy policy) diff --git a/server/src/test/java/org/apache/druid/server/coordinator/UserCompactionTaskQueryTuningConfigTest.java b/server/src/test/java/org/apache/druid/server/coordinator/UserCompactionTaskQueryTuningConfigTest.java index a92122e475d7..194948f4dfb9 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/UserCompactionTaskQueryTuningConfigTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/UserCompactionTaskQueryTuningConfigTest.java @@ -43,27 +43,7 @@ public class UserCompactionTaskQueryTuningConfigTest public void testSerdeNulls() throws IOException { final UserCompactionTaskQueryTuningConfig config = - new UserCompactionTaskQueryTuningConfig( - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null - ); + UserCompactionTaskQueryTuningConfig.builder().build(); final String json = OBJECT_MAPPER.writeValueAsString(config); // Check maxRowsPerSegment doesn't exist in the JSON string Assert.assertFalse(json.contains("maxRowsPerSegment")); @@ -75,35 +55,34 @@ public void testSerdeNulls() throws IOException @Test public void testSerde() throws IOException { - final UserCompactionTaskQueryTuningConfig tuningConfig = new UserCompactionTaskQueryTuningConfig( - 40000, - new OnheapIncrementalIndex.Spec(true), - 2000L, - null, - new SegmentsSplitHintSpec(new HumanReadableBytes(42L), null), - new DynamicPartitionsSpec(1000, 20000L), - IndexSpec.builder() - .withDimensionCompression(CompressionStrategy.LZ4) - .withMetricCompression(CompressionStrategy.LZ4) - .withLongEncoding(LongEncodingStrategy.LONGS) - .build(), - IndexSpec.builder() - .withDimensionCompression(CompressionStrategy.LZ4) - .withMetricCompression(CompressionStrategy.LZ4) - .withLongEncoding(LongEncodingStrategy.LONGS) - .build(), - 2, - 1000L, - TmpFileSegmentWriteOutMediumFactory.instance(), - 100, - 5, - 1000L, - new Duration(3000L), - 7, - 1000, - 100, - 2 - ); + final UserCompactionTaskQueryTuningConfig tuningConfig = UserCompactionTaskQueryTuningConfig.builder() + .maxRowsInMemory(40000) + .appendableIndexSpec(new OnheapIncrementalIndex.Spec(true)) + .maxBytesInMemory(2000L) + .splitHintSpec(new SegmentsSplitHintSpec(new HumanReadableBytes(42L), null)) + .partitionsSpec(new DynamicPartitionsSpec(1000, 20000L)) + .indexSpec(IndexSpec.builder() + .withDimensionCompression(CompressionStrategy.LZ4) + .withMetricCompression(CompressionStrategy.LZ4) + .withLongEncoding(LongEncodingStrategy.LONGS) + .build()) + .indexSpecForIntermediatePersists(IndexSpec.builder() + .withDimensionCompression(CompressionStrategy.LZ4) + .withMetricCompression(CompressionStrategy.LZ4) + .withLongEncoding(LongEncodingStrategy.LONGS) + .build()) + .maxPendingPersists(2) + .pushTimeout(1000L) + .segmentWriteOutMediumFactory(TmpFileSegmentWriteOutMediumFactory.instance()) + .maxNumConcurrentSubTasks(100) + .maxRetry(5) + .taskStatusCheckPeriodMs(1000L) + .chatHandlerTimeout(new Duration(3000L)) + .chatHandlerNumRetries(7) + .maxNumSegmentsToMerge(1000) + .totalNumMergeTasks(100) + .maxColumnsToMerge(2) + .build(); final String json = OBJECT_MAPPER.writeValueAsString(tuningConfig); final UserCompactionTaskQueryTuningConfig fromJson = diff --git a/server/src/test/java/org/apache/druid/server/coordinator/duty/CompactSegmentsTest.java b/server/src/test/java/org/apache/druid/server/coordinator/duty/CompactSegmentsTest.java index 4ea98e43ad57..11df140eccde 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/duty/CompactSegmentsTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/duty/CompactSegmentsTest.java @@ -1102,6 +1102,7 @@ public void testCompactWithGranularitySpecConflictWithActiveCompactionTask() new ClientCompactionIOConfig( new ClientCompactionIntervalSpec( Intervals.of("2000/2099"), + null, "testSha256OfSortedSegmentIds" ), null From 86ee6b81887527a7eaf2b8415f3b90f5d8f3333e Mon Sep 17 00:00:00 2001 From: cecemei Date: Thu, 26 Feb 2026 17:32:32 -0800 Subject: [PATCH 02/26] style --- .../common/actions/SegmentUpgradeAction.java | 16 +++++++ .../indexing/common/task/CompactionTask.java | 17 ++++--- .../indexing/compact/CompactionJobQueue.java | 2 +- .../actions/SegmentUpgradeActionTest.java | 46 +++++++++++++------ .../partition/DimensionRangeShardSpec.java | 8 ++-- .../timeline/partition/LinearShardSpec.java | 6 +-- .../timeline/partition/NumberedShardSpec.java | 8 ++-- .../druid/timeline/partition/ShardSpec.java | 2 +- .../IndexerSQLMetadataStorageCoordinator.java | 14 ++---- .../compaction/CompactionCandidate.java | 3 +- .../server/compaction/CompactionMode.java | 3 ++ .../MostFragmentedIntervalFirstPolicy.java | 9 ++-- 12 files changed, 85 insertions(+), 49 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentUpgradeAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentUpgradeAction.java index ebc2348d450e..124fac4bf416 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentUpgradeAction.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentUpgradeAction.java @@ -31,11 +31,27 @@ import java.util.Map; import java.util.Set; +/** + * Task action that records segments as being upgraded in the metadata store. + *

    + * This action is used during compaction to track which segments are being replaced. + * It validates that all segments to be upgraded are covered by + * {@link ReplaceTaskLock}s before inserting them into the upgrade segments table. + *

    + * The action will fail if any of the upgrade segments do not have a corresponding + * replace lock, ensuring that only properly locked segments can be marked for upgrade. + * + * @return the number of segments successfully inserted into the upgrade segments table + */ public class SegmentUpgradeAction implements TaskAction { private final String dataSource; private final List upgradeSegments; + /** + * @param dataSource the datasource containing the segments to upgrade + * @param upgradeSegments the list of segments to be recorded as upgraded + */ @JsonCreator public SegmentUpgradeAction( @JsonProperty("dataSource") String dataSource, diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java index 97416c2ec202..ddaa94bb3bc4 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java @@ -551,10 +551,14 @@ public TaskStatus runTask(TaskToolbox toolbox) throws Exception } /** - * Generate dataschema for segments in each interval. + * Creates input data schemas for compaction by grouping segments and generating {@link DataSchema}s. + * When segment granularity is not specified, preserves original granularity and creates a schema + * for each unified interval. When segment granularity is specified, creates a single schema for all + * segments. For incremental compaction, validates that all segments are completely within the target + * interval and submits already-compacted segments via {@link SegmentUpgradeAction} for direct upgrade. * - * @throws IOException if an exception occurs whie retrieving used segments to - * determine schemas. + * @return map from {@link QuerySegmentSpec} to {@link DataSchema} for each group of segments to compact + * @throws IOException if an exception occurs while retrieving segments */ @VisibleForTesting static Map createInputDataSchemas( @@ -665,9 +669,10 @@ static Map createInputDataSchemas( return inputSchemas; } else { // given segment granularity - List upgradeSegments = StreamSupport.stream(timelineSegments.spliterator(), false) - .filter(segmentProvider.segmentsToUpgradePredicate) - .collect(Collectors.toList()); + List upgradeSegments = Lists.newArrayList(Iterables.filter( + timelineSegments, + segmentProvider.segmentsToUpgradePredicate + )); if (!upgradeSegments.isEmpty()) { toolbox.getTaskActionClient().submit(new SegmentUpgradeAction(segmentProvider.dataSource, upgradeSegments)); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionJobQueue.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionJobQueue.java index c2db60c28629..3d3eca7b7765 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionJobQueue.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionJobQueue.java @@ -282,7 +282,7 @@ private boolean startJobIfPendingAndReady( return false; } - // Check if the job is already running, completed or skipped + // Check if the job is already running or skipped final CompactionTaskStatus lastTaskStatus = statusTracker.getLatestTaskStatus(candidate); final CompactionStatus compactionStatus = statusTracker.deriveCompactionStatus(lastTaskStatus); if (compactionStatus != null) { diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentUpgradeActionTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentUpgradeActionTest.java index b3fc36a837ea..0fdadf32fba5 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentUpgradeActionTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentUpgradeActionTest.java @@ -30,16 +30,32 @@ import org.apache.druid.timeline.SegmentId; import org.apache.druid.timeline.partition.LinearShardSpec; import org.joda.time.Interval; -import org.junit.Assert; -import org.junit.Rule; -import org.junit.Test; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; import java.util.List; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; + public class SegmentUpgradeActionTest { - @Rule - public TaskActionTestKit actionTestKit = new TaskActionTestKit(); + private TaskActionTestKit actionTestKit; + + @BeforeEach + public void setUp() + { + actionTestKit = new TaskActionTestKit(); + actionTestKit.before(); + } + + @AfterEach + public void tearDown() + { + actionTestKit.after(); + } private static final String DATA_SOURCE = "test_dataSource"; private static final Interval INTERVAL_2026_01 = Intervals.of("2026-01-01/2026-01-02"); @@ -74,9 +90,9 @@ public void test_segmentsSuccessfullyInsertedIntoUpgradeTable() throws Exception final SegmentUpgradeAction action = new SegmentUpgradeAction(DATA_SOURCE, List.of(SEGMENT1, SEGMENT2, SEGMENT3)); final Integer insertedCount = action.perform(task, actionTestKit.getTaskActionToolbox()); - Assert.assertEquals(3, insertedCount.intValue()); + assertEquals(3, insertedCount.intValue()); final int deletedCount = actionTestKit.getMetadataStorageCoordinator().deleteUpgradeSegmentsForTask(task.getId()); - Assert.assertEquals(3, deletedCount); + assertEquals(3, deletedCount); } @Test @@ -89,8 +105,8 @@ public void test_failsWhenSegmentsNotCoveredByReplaceLock() throws Exception final SegmentUpgradeAction action = new SegmentUpgradeAction(DATA_SOURCE, List.of(SEGMENT1, SEGMENT2, SEGMENT3)); - IAE exception = Assert.assertThrows(IAE.class, () -> action.perform(task, actionTestKit.getTaskActionToolbox())); - Assert.assertTrue(exception.getMessage().contains("Not all segments are hold by a replace lock")); + IAE exception = assertThrows(IAE.class, () -> action.perform(task, actionTestKit.getTaskActionToolbox())); + assertTrue(exception.getMessage().contains("Not all segments are hold by a replace lock")); } @Test @@ -103,8 +119,8 @@ public void test_failsWithExclusiveLockInsteadOfReplaceLock() throws Exception final SegmentUpgradeAction action = new SegmentUpgradeAction(DATA_SOURCE, List.of(SEGMENT1, SEGMENT2)); - IAE exception = Assert.assertThrows(IAE.class, () -> action.perform(task, actionTestKit.getTaskActionToolbox())); - Assert.assertTrue(exception.getMessage().contains("Not all segments are hold by a replace lock")); + IAE exception = assertThrows(IAE.class, () -> action.perform(task, actionTestKit.getTaskActionToolbox())); + assertTrue(exception.getMessage().contains("Not all segments are hold by a replace lock")); } @Test @@ -115,11 +131,11 @@ public void test_emptySegmentsList() final SegmentUpgradeAction action = new SegmentUpgradeAction(DATA_SOURCE, List.of()); - DruidException exception = Assert.assertThrows( + DruidException exception = assertThrows( DruidException.class, () -> action.perform(task, actionTestKit.getTaskActionToolbox()) ); - Assert.assertTrue(exception.getMessage().contains("No segment to commit")); + assertTrue(exception.getMessage().contains("No segment to commit")); } @Test @@ -133,8 +149,8 @@ public void test_singleSegmentUpgrade() throws Exception final SegmentUpgradeAction action = new SegmentUpgradeAction(DATA_SOURCE, List.of(SEGMENT1)); final Integer insertedCount = action.perform(task, actionTestKit.getTaskActionToolbox()); - Assert.assertEquals(1, insertedCount.intValue()); + assertEquals(1, insertedCount.intValue()); final int deletedCount = actionTestKit.getMetadataStorageCoordinator().deleteUpgradeSegmentsForTask(task.getId()); - Assert.assertEquals(1, deletedCount); + assertEquals(1, deletedCount); } } diff --git a/processing/src/main/java/org/apache/druid/timeline/partition/DimensionRangeShardSpec.java b/processing/src/main/java/org/apache/druid/timeline/partition/DimensionRangeShardSpec.java index 04963b19b8c5..381b63fe791a 100644 --- a/processing/src/main/java/org/apache/druid/timeline/partition/DimensionRangeShardSpec.java +++ b/processing/src/main/java/org/apache/druid/timeline/partition/DimensionRangeShardSpec.java @@ -105,15 +105,15 @@ public int getNumCorePartitions() } @Override - public ShardSpec withPartitionNum(int partitionNum1) + public ShardSpec withPartitionNum(int partitionNum) { - return new DimensionRangeShardSpec(dimensions, start, end, partitionNum1, numCorePartitions); + return new DimensionRangeShardSpec(dimensions, start, end, partitionNum, numCorePartitions); } @Override - public ShardSpec withCorePartitions(int partitions1) + public ShardSpec withCorePartitions(int partitions) { - return new DimensionRangeShardSpec(dimensions, start, end, partitionNum, partitions1); + return new DimensionRangeShardSpec(dimensions, start, end, partitionNum, partitions); } public boolean isNumCorePartitionsUnknown() diff --git a/processing/src/main/java/org/apache/druid/timeline/partition/LinearShardSpec.java b/processing/src/main/java/org/apache/druid/timeline/partition/LinearShardSpec.java index f2d25b50311a..4372ac8e1d7c 100644 --- a/processing/src/main/java/org/apache/druid/timeline/partition/LinearShardSpec.java +++ b/processing/src/main/java/org/apache/druid/timeline/partition/LinearShardSpec.java @@ -56,13 +56,13 @@ public int getNumCorePartitions() } @Override - public ShardSpec withPartitionNum(int partitionNum1) + public ShardSpec withPartitionNum(int partitionNum) { - return new LinearShardSpec(partitionNum1); + return new LinearShardSpec(partitionNum); } @Override - public ShardSpec withCorePartitions(int partitions1) + public ShardSpec withCorePartitions(int partitions) { return this; } diff --git a/processing/src/main/java/org/apache/druid/timeline/partition/NumberedShardSpec.java b/processing/src/main/java/org/apache/druid/timeline/partition/NumberedShardSpec.java index fdcb438f21c6..db9a501e0432 100644 --- a/processing/src/main/java/org/apache/druid/timeline/partition/NumberedShardSpec.java +++ b/processing/src/main/java/org/apache/druid/timeline/partition/NumberedShardSpec.java @@ -101,15 +101,15 @@ public int getNumCorePartitions() } @Override - public ShardSpec withPartitionNum(int partitionNum1) + public ShardSpec withPartitionNum(int partitionNum) { - return new NumberedShardSpec(partitionNum1, partitions); + return new NumberedShardSpec(partitionNum, partitions); } @Override - public ShardSpec withCorePartitions(int partitions1) + public ShardSpec withCorePartitions(int partitions) { - return new NumberedShardSpec(partitionNum, partitions1); + return new NumberedShardSpec(partitionNum, partitions); } @Override diff --git a/processing/src/main/java/org/apache/druid/timeline/partition/ShardSpec.java b/processing/src/main/java/org/apache/druid/timeline/partition/ShardSpec.java index 1a64c5202fcd..b7e97e232e52 100644 --- a/processing/src/main/java/org/apache/druid/timeline/partition/ShardSpec.java +++ b/processing/src/main/java/org/apache/druid/timeline/partition/ShardSpec.java @@ -79,7 +79,7 @@ default boolean isNumChunkSupported() /** * Creates a new ShardSpec with the specified partition number. */ - default ShardSpec withPartitionNum(int partitionNum1) + default ShardSpec withPartitionNum(int partitionNum) { throw DruidException.defensive("ShardSpec[%s] does not implement withPartitionNum", this.getClass().toString()); } 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 00d2424e1ec8..22d1f709096f 100644 --- a/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java +++ b/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java @@ -1853,16 +1853,12 @@ protected Set insertSegments( } /** - * Creates upgraded versions of segments that were appended while a REPLACE task was in progress. - * Upgraded segments get new intervals, versions, and partition numbers to maintain consistency - * with the version created by the REPLACE task. + * Retrieves segments from the upgrade segments table and creates upgraded versions with new intervals, + * versions, and partition numbers. Combines upgraded segments with replace segments and updates shard + * specs with correct core partition counts. * - * @param dataSource The datasource being modified - * @param transaction The segment metadata transaction - * @param replaceSegments Segments being committed by the REPLACE task - * @param locksHeldByReplaceTask Replace locks held by the task - * @return Pair of (upgraded segments, all segments to insert with updated shard specs) - * @throws DruidException if a replace interval partially overlaps an appended segment + * @return pair of (upgraded segments for metadata tracking, segments to insert into segment table) + * @throws DruidException if a replace interval partially overlaps a segment being upgraded */ private Pair, Set> createNewSegmentsAfterReplace( final String dataSource, diff --git a/server/src/main/java/org/apache/druid/server/compaction/CompactionCandidate.java b/server/src/main/java/org/apache/druid/server/compaction/CompactionCandidate.java index 4baf15611f87..3f56716d9b9e 100644 --- a/server/src/main/java/org/apache/druid/server/compaction/CompactionCandidate.java +++ b/server/src/main/java/org/apache/druid/server/compaction/CompactionCandidate.java @@ -154,8 +154,7 @@ public CompactionStatistics getUncompactedStats() @Nullable public List getUncompactedSegments() { - return (currentStatus == null || currentStatus.getUncompactedSegments() == null) - ? null : currentStatus.getUncompactedSegments(); + return currentStatus == null ? null : currentStatus.getUncompactedSegments(); } /** diff --git a/server/src/main/java/org/apache/druid/server/compaction/CompactionMode.java b/server/src/main/java/org/apache/druid/server/compaction/CompactionMode.java index 1b037e476f9e..82e9000683fb 100644 --- a/server/src/main/java/org/apache/druid/server/compaction/CompactionMode.java +++ b/server/src/main/java/org/apache/druid/server/compaction/CompactionMode.java @@ -28,5 +28,8 @@ public enum CompactionMode * Indicates that all existing segments of the interval will be picked for compaction. */ FULL_COMPACTION, + /** + * Indicates that only uncompacted segments of the interval will be picked for compaction. + */ INCREMENTAL_COMPACTION; } diff --git a/server/src/main/java/org/apache/druid/server/compaction/MostFragmentedIntervalFirstPolicy.java b/server/src/main/java/org/apache/druid/server/compaction/MostFragmentedIntervalFirstPolicy.java index ac769cfde15d..1a133f2e358c 100644 --- a/server/src/main/java/org/apache/druid/server/compaction/MostFragmentedIntervalFirstPolicy.java +++ b/server/src/main/java/org/apache/druid/server/compaction/MostFragmentedIntervalFirstPolicy.java @@ -149,10 +149,11 @@ public boolean equals(Object o) && Objects.equals(minUncompactedBytes, policy.minUncompactedBytes) && Objects.equals(maxAverageUncompactedBytesPerSegment, policy.maxAverageUncompactedBytesPerSegment) // Use Double.compare instead of == to handle NaN correctly and keep equals() consistent with hashCode() (especially for +0.0 vs -0.0). - && Double.compare( - incrementalCompactionUncompactedRatioThreshold, - policy.incrementalCompactionUncompactedRatioThreshold - ) == 0; + && + Double.compare( + incrementalCompactionUncompactedRatioThreshold, + policy.incrementalCompactionUncompactedRatioThreshold + ) == 0; } @Override From bdc28ac7fd0974d887771fa0ee3d83b01a1faa6f Mon Sep 17 00:00:00 2001 From: cecemei Date: Thu, 26 Feb 2026 17:49:27 -0800 Subject: [PATCH 03/26] pending --- .../indexing/compact/CompactionJobQueue.java | 26 +++++++++---------- .../compaction/CompactionStatusTracker.java | 5 ++-- 2 files changed, 15 insertions(+), 16 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionJobQueue.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionJobQueue.java index 3d3eca7b7765..7ec45dbb6943 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionJobQueue.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionJobQueue.java @@ -282,21 +282,21 @@ private boolean startJobIfPendingAndReady( return false; } - // Check if the job is already running or skipped + // Check if the job is already running or skipped or pending final CompactionTaskStatus lastTaskStatus = statusTracker.getLatestTaskStatus(candidate); final CompactionStatus compactionStatus = statusTracker.deriveCompactionStatus(lastTaskStatus); - if (compactionStatus != null) { - switch (compactionStatus.getState()) { - case RUNNING: - return false; - case SKIPPED: - snapshotBuilder.moveFromPendingToSkipped(candidate); - return false; - case PENDING: - case COMPLETE: - default: - throw DruidException.defensive("unexpected derived compaction state[%s]", compactionStatus.getState()); - } + + switch (compactionStatus.getState()) { + case RUNNING: + return false; + case SKIPPED: + snapshotBuilder.moveFromPendingToSkipped(candidate); + return false; + case PENDING: + break; + case COMPLETE: + default: + throw DruidException.defensive("unexpected derived compaction state[%s]", compactionStatus.getState()); } // Check if enough compaction task slots are available diff --git a/server/src/main/java/org/apache/druid/server/compaction/CompactionStatusTracker.java b/server/src/main/java/org/apache/druid/server/compaction/CompactionStatusTracker.java index 47848c0a568c..cdd52a4f917c 100644 --- a/server/src/main/java/org/apache/druid/server/compaction/CompactionStatusTracker.java +++ b/server/src/main/java/org/apache/druid/server/compaction/CompactionStatusTracker.java @@ -93,7 +93,7 @@ public CompactionStatus computeCompactionStatus( { final CompactionTaskStatus lastTaskStatus = getLatestTaskStatus(candidate); CompactionStatus status = deriveCompactionStatus(lastTaskStatus); - if (status != null) { + if (!CompactionStatus.State.PENDING.equals(status.getState())) { return status; } @@ -107,7 +107,6 @@ public CompactionStatus computeCompactionStatus( } } - @Nullable public CompactionStatus deriveCompactionStatus(CompactionTaskStatus lastTaskStatus) { // Skip intervals that already have a running task @@ -124,7 +123,7 @@ public CompactionStatus deriveCompactionStatus(CompactionTaskStatus lastTaskStat "Segment timeline not updated since last compaction task succeeded" ); } - return null; + return CompactionStatus.pending("Not compacted yet"); } /** From 9a09e4318320ae5b9780cb83bff1cddef6652b37 Mon Sep 17 00:00:00 2001 From: cecemei Date: Fri, 27 Feb 2026 11:56:21 -0800 Subject: [PATCH 04/26] format --- .../compact/CompactionSupervisorTest.java | 50 ++++++++++++------- .../common/actions/SegmentUpgradeAction.java | 4 +- .../indexing/common/actions/TaskAction.java | 2 +- .../CompactionConfigBasedJobTemplate.java | 4 +- .../actions/SegmentUpgradeActionTest.java | 11 ++-- .../IndexerMetadataStorageCoordinator.java | 15 +----- .../CompactionCandidateSearchPolicy.java | 4 +- .../server/compaction/CompactionMode.java | 4 +- .../MostFragmentedIntervalFirstPolicy.java | 48 ++++++++---------- .../coordinator/duty/CompactSegments.java | 6 +-- ...MostFragmentedIntervalFirstPolicyTest.java | 14 +++--- 11 files changed, 82 insertions(+), 80 deletions(-) diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionSupervisorTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionSupervisorTest.java index 5c85fc35c8f2..f07be29d4a16 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionSupervisorTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionSupervisorTest.java @@ -156,14 +156,7 @@ public EmbeddedDruidCluster createCluster() private void configureCompaction(CompactionEngine compactionEngine, @Nullable CompactionCandidateSearchPolicy policy) { final UpdateResponse updateResponse = cluster.callApi().onLeaderOverlord( - o -> o.updateClusterCompactionConfig(new ClusterCompactionConfig( - 1.0, - 100, - policy, - true, - compactionEngine, - true - )) + o -> o.updateClusterCompactionConfig(new ClusterCompactionConfig(1.0, 100, policy, true, compactionEngine, true)) ); Assertions.assertTrue(updateResponse.isSuccess()); } @@ -240,7 +233,7 @@ public void test_incrementalCompactionWithMSQ() throws Exception { configureCompaction( CompactionEngine.MSQ, - new MostFragmentedIntervalFirstPolicy(2, new HumanReadableBytes("1KiB"), null, 0.8, null) + new MostFragmentedIntervalFirstPolicy(2, new HumanReadableBytes("1KiB"), null, 80, null) ); KafkaSupervisorSpecBuilder kafkaSupervisorSpecBuilder = MoreResources.Supervisor.KAFKA_JSON .get() @@ -280,7 +273,6 @@ public void test_incrementalCompactionWithMSQ() throws Exception .build(); runCompactionWithSpec(dayGranularityConfig); - Thread.sleep(2_000L); waitForAllCompactionTasksToFinish(); pauseCompaction(dayGranularityConfig); @@ -300,22 +292,31 @@ public void test_incrementalCompactionWithMSQ() throws Exception indexer.latchableEmitter().flush(); waitUntilPublishedRecordsAreIngested(appendedRowCount); + // Tear down both topics and supervisors + kafkaServer.deleteTopic(topic1); + cluster.callApi().postSupervisor(supervisor1.createSuspendedSpec()); + kafkaServer.deleteTopic(topic2); + cluster.callApi().postSupervisor(supervisor2.createSuspendedSpec()); + long totalUsed = overlord.latchableEmitter().getMetricValues( + "segment/metadataCache/used/count", + Map.of(DruidMetrics.DATASOURCE, dataSource) + ).stream().reduce((first, second) -> second).orElse(0).longValue(); + Assertions.assertEquals(0, getNumSegmentsWith(Granularities.HOUR)); // 1 compacted segment + 2 appended segment Assertions.assertEquals(3, getNumSegmentsWith(Granularities.DAY)); runCompactionWithSpec(dayGranularityConfig); - Thread.sleep(2_000L); waitForAllCompactionTasksToFinish(); - // performed incremental compaction - Assertions.assertEquals(2, getNumSegmentsWith(Granularities.DAY)); + // wait for new segments have been updated to the cache + overlord.latchableEmitter().waitForEvent( + event -> event.hasMetricName("segment/metadataCache/used/count") + .hasDimension(DruidMetrics.DATASOURCE, dataSource) + .hasValueMatching(Matchers.greaterThan(totalUsed))); - // Tear down both topics and supervisors - kafkaServer.deleteTopic(topic1); - cluster.callApi().postSupervisor(supervisor1.createSuspendedSpec()); - kafkaServer.deleteTopic(topic2); - cluster.callApi().postSupervisor(supervisor2.createSuspendedSpec()); + // performed incremental compaction: 1 previously compacted segment + 1 incrementally compacted segment + Assertions.assertEquals(2, getNumSegmentsWith(Granularities.DAY)); } protected void waitUntilPublishedRecordsAreIngested(int expectedRowCount) @@ -767,11 +768,24 @@ private void waitForAllCompactionTasksToFinish() .hasValueMatching(Matchers.equalTo(0L)) ); + int compacted = overlord.latchableEmitter().getMetricValues( + "interval/compacted/count", + Map.of(DruidMetrics.DATASOURCE, dataSource) + ).stream().mapToInt(Number::intValue).sum(); + System.out.println("compacted " + compacted); + + int skipped = overlord.latchableEmitter().getMetricValues( + "interval/skipCompact/count", + Map.of(DruidMetrics.DATASOURCE, dataSource) + ).stream().mapToInt(Number::intValue).sum(); + System.out.println("skipped " + skipped); + // Wait for all submitted compaction jobs to finish int numSubmittedTasks = overlord.latchableEmitter().getMetricValues( "compact/task/count", Map.of(DruidMetrics.DATASOURCE, dataSource) ).stream().mapToInt(Number::intValue).sum(); + System.out.println("submitted task " + numSubmittedTasks); final Matcher taskTypeMatcher = Matchers.anyOf( Matchers.equalTo("query_controller"), diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentUpgradeAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentUpgradeAction.java index 124fac4bf416..989d39eddca8 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentUpgradeAction.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentUpgradeAction.java @@ -22,8 +22,8 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.core.type.TypeReference; +import org.apache.druid.error.InvalidInput; import org.apache.druid.indexing.common.task.Task; -import org.apache.druid.java.util.common.IAE; import org.apache.druid.metadata.ReplaceTaskLock; import org.apache.druid.timeline.DataSegment; @@ -90,7 +90,7 @@ public Integer perform(Task task, TaskActionToolbox toolbox) = TaskLocks.findReplaceLocksCoveringSegments(datasource, toolbox.getTaskLockbox(), Set.copyOf(upgradeSegments)); if (segmentToReplaceLock.size() < upgradeSegments.size()) { - throw new IAE( + throw InvalidInput.exception( "Not all segments are hold by a replace lock, only [%d] segments out of total segments[%d] are hold by repalce lock", segmentToReplaceLock.size(), upgradeSegments.size() 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 dd2c1b7936b3..9c5b35f12ba4 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 @@ -37,7 +37,7 @@ @JsonSubTypes.Type(name = "segmentTransactionalInsert", value = SegmentTransactionalInsertAction.class), @JsonSubTypes.Type(name = "segmentTransactionalAppend", value = SegmentTransactionalAppendAction.class), @JsonSubTypes.Type(name = "segmentTransactionalReplace", value = SegmentTransactionalReplaceAction.class), - @JsonSubTypes.Type(name = "segmentUpgrade", value = SegmentUpgradeAction.class), + @JsonSubTypes.Type(name = "markSegmentsToUpgrade", value = SegmentUpgradeAction.class), @JsonSubTypes.Type(name = "retrieveSegmentsById", value = RetrieveSegmentsByIdAction.class), @JsonSubTypes.Type(name = "retrieveUpgradedFromSegmentIds", value = RetrieveUpgradedFromSegmentIdsAction.class), @JsonSubTypes.Type(name = "retrieveUpgradedToSegmentIds", value = RetrieveUpgradedToSegmentIdsAction.class), diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionConfigBasedJobTemplate.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionConfigBasedJobTemplate.java index fd282701844b..96a37114efa5 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionConfigBasedJobTemplate.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionConfigBasedJobTemplate.java @@ -99,10 +99,10 @@ public List createCompactionJobs( } final CompactionCandidate finalCandidate; switch (eligibility.getMode()) { - case FULL_COMPACTION: + case ALL_SEGMENTS: finalCandidate = candidate; break; - case INCREMENTAL_COMPACTION: + case UNCOMPACTED_SEGMENTS_ONLY: finalCandidate = CompactionCandidate.from(candidate.getUncompactedSegments(), null) .withCurrentStatus(candidate.getCurrentStatus()); break; diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentUpgradeActionTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentUpgradeActionTest.java index 0fdadf32fba5..785a4cbff696 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentUpgradeActionTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentUpgradeActionTest.java @@ -24,7 +24,6 @@ import org.apache.druid.indexing.common.task.NoopTask; import org.apache.druid.indexing.common.task.Task; import org.apache.druid.indexing.overlord.TimeChunkLockRequest; -import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.SegmentId; @@ -105,7 +104,10 @@ public void test_failsWhenSegmentsNotCoveredByReplaceLock() throws Exception final SegmentUpgradeAction action = new SegmentUpgradeAction(DATA_SOURCE, List.of(SEGMENT1, SEGMENT2, SEGMENT3)); - IAE exception = assertThrows(IAE.class, () -> action.perform(task, actionTestKit.getTaskActionToolbox())); + DruidException exception = assertThrows( + DruidException.class, + () -> action.perform(task, actionTestKit.getTaskActionToolbox()) + ); assertTrue(exception.getMessage().contains("Not all segments are hold by a replace lock")); } @@ -119,7 +121,10 @@ public void test_failsWithExclusiveLockInsteadOfReplaceLock() throws Exception final SegmentUpgradeAction action = new SegmentUpgradeAction(DATA_SOURCE, List.of(SEGMENT1, SEGMENT2)); - IAE exception = assertThrows(IAE.class, () -> action.perform(task, actionTestKit.getTaskActionToolbox())); + DruidException exception = assertThrows( + DruidException.class, + () -> action.perform(task, actionTestKit.getTaskActionToolbox()) + ); assertTrue(exception.getMessage().contains("Not all segments are hold by a replace lock")); } 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 876ffbe7f43a..5ac25f6ad0b8 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 @@ -389,20 +389,7 @@ SegmentPublishResult commitAppendSegmentsAndMetadata( /** * Inserts entries into the upgrade_segments table for segments that need to be upgraded - * to a new version without recompaction. This is used during incremental compaction when - * some segments in the interval don't require compaction but should be upgraded to match - * the version of newly compacted segments. - *

    - * In incremental compaction scenarios: - *

      - *
    • Some segments may already meet compaction criteria (e.g., already compacted, correct partitioning) - * and don't need to be rewritten.
    • - *
    • These segments are tracked in the upgrade_segments table via this method, mapped to their - * corresponding REPLACE lock.
    • - *
    • When the compaction task finishes via {@link #commitReplaceSegments}, these segments are - * upgraded directly to the new version to maintain version consistency across the interval.
    • - *
    • After the task completes, entries are cleaned up via {@link #deleteUpgradeSegmentsForTask}.
    • - *
    + * to a new version when a REPLACE task finishes using {@link #commitReplaceSegments}. * * @param segmentToReplaceLock map from segment to its corresponding REPLACE lock, identifying * which segments should be upgraded when the lock's task completes diff --git a/server/src/main/java/org/apache/druid/server/compaction/CompactionCandidateSearchPolicy.java b/server/src/main/java/org/apache/druid/server/compaction/CompactionCandidateSearchPolicy.java index 87b8dc10f315..44889fb7e10c 100644 --- a/server/src/main/java/org/apache/druid/server/compaction/CompactionCandidateSearchPolicy.java +++ b/server/src/main/java/org/apache/druid/server/compaction/CompactionCandidateSearchPolicy.java @@ -65,7 +65,7 @@ Eligibility checkEligibilityForCompaction( */ class Eligibility { - public static final Eligibility OK = new Eligibility(true, null, CompactionMode.FULL_COMPACTION); + public static final Eligibility OK = new Eligibility(true, null, CompactionMode.ALL_SEGMENTS); private final boolean eligible; private final String reason; @@ -106,7 +106,7 @@ public static Eligibility fail(String messageFormat, Object... args) public static Eligibility incremental(String messageFormat, Object... args) { - return new Eligibility(true, StringUtils.format(messageFormat, args), CompactionMode.INCREMENTAL_COMPACTION); + return new Eligibility(true, StringUtils.format(messageFormat, args), CompactionMode.UNCOMPACTED_SEGMENTS_ONLY); } @Override diff --git a/server/src/main/java/org/apache/druid/server/compaction/CompactionMode.java b/server/src/main/java/org/apache/druid/server/compaction/CompactionMode.java index 82e9000683fb..83ba94ce80ca 100644 --- a/server/src/main/java/org/apache/druid/server/compaction/CompactionMode.java +++ b/server/src/main/java/org/apache/druid/server/compaction/CompactionMode.java @@ -27,9 +27,9 @@ public enum CompactionMode /** * Indicates that all existing segments of the interval will be picked for compaction. */ - FULL_COMPACTION, + ALL_SEGMENTS, /** * Indicates that only uncompacted segments of the interval will be picked for compaction. */ - INCREMENTAL_COMPACTION; + UNCOMPACTED_SEGMENTS_ONLY; } diff --git a/server/src/main/java/org/apache/druid/server/compaction/MostFragmentedIntervalFirstPolicy.java b/server/src/main/java/org/apache/druid/server/compaction/MostFragmentedIntervalFirstPolicy.java index 1a133f2e358c..840887ca1c18 100644 --- a/server/src/main/java/org/apache/druid/server/compaction/MostFragmentedIntervalFirstPolicy.java +++ b/server/src/main/java/org/apache/druid/server/compaction/MostFragmentedIntervalFirstPolicy.java @@ -47,7 +47,7 @@ public class MostFragmentedIntervalFirstPolicy extends BaseCandidateSearchPolicy private final int minUncompactedCount; private final HumanReadableBytes minUncompactedBytes; private final HumanReadableBytes maxAverageUncompactedBytesPerSegment; - private final double incrementalCompactionUncompactedRatioThreshold; + private final int minUncompactedBytesPercentForFullCompaction; @JsonCreator public MostFragmentedIntervalFirstPolicy( @@ -55,8 +55,8 @@ public MostFragmentedIntervalFirstPolicy( @JsonProperty("minUncompactedBytes") @Nullable HumanReadableBytes minUncompactedBytes, @JsonProperty("maxAverageUncompactedBytesPerSegment") @Nullable HumanReadableBytes maxAverageUncompactedBytesPerSegment, - @JsonProperty("incrementalCompactionUncompactedRatioThreshold") @Nullable - Double incrementalCompactionUncompactedRatioThreshold, + @JsonProperty("minUncompactedBytesPercentForFullCompaction") @Nullable + Integer minUncompactedBytesPercentForFullCompaction, @JsonProperty("priorityDatasource") @Nullable String priorityDatasource ) { @@ -73,19 +73,19 @@ public MostFragmentedIntervalFirstPolicy( maxAverageUncompactedBytesPerSegment ); InvalidInput.conditionalException( - incrementalCompactionUncompactedRatioThreshold == null - || (incrementalCompactionUncompactedRatioThreshold >= 0.0d - && incrementalCompactionUncompactedRatioThreshold < 1.0d), - "'incrementalCompactionUncompactedRatioThreshold'[%s] must be between 0.0 and 1.0", - incrementalCompactionUncompactedRatioThreshold + minUncompactedBytesPercentForFullCompaction == null + || (minUncompactedBytesPercentForFullCompaction >= 0 + && minUncompactedBytesPercentForFullCompaction < 100), + "'minUncompactedBytesPercentForFullCompaction'[%s] must be between 0 and 100", + minUncompactedBytesPercentForFullCompaction ); this.minUncompactedCount = Configs.valueOrDefault(minUncompactedCount, 100); this.minUncompactedBytes = Configs.valueOrDefault(minUncompactedBytes, SIZE_10_MB); this.maxAverageUncompactedBytesPerSegment = Configs.valueOrDefault(maxAverageUncompactedBytesPerSegment, SIZE_2_GB); - this.incrementalCompactionUncompactedRatioThreshold = - Configs.valueOrDefault(incrementalCompactionUncompactedRatioThreshold, 0.0d); + this.minUncompactedBytesPercentForFullCompaction = + Configs.valueOrDefault(minUncompactedBytesPercentForFullCompaction, 0); } /** @@ -119,14 +119,14 @@ public HumanReadableBytes getMaxAverageUncompactedBytesPerSegment() } /** - * Threshold ratio of uncompacted bytes to compacted bytes below which + * Threshold percentage of uncompacted bytes to total bytes below which * incremental compaction is eligible instead of full compaction. - * Default value is 0.0. + * Default value is 0. */ @JsonProperty - public double getIncrementalCompactionUncompactedRatioThreshold() + public int minUncompactedBytesPercentForFullCompaction() { - return incrementalCompactionUncompactedRatioThreshold; + return minUncompactedBytesPercentForFullCompaction; } @Override @@ -148,12 +148,7 @@ public boolean equals(Object o) return minUncompactedCount == policy.minUncompactedCount && Objects.equals(minUncompactedBytes, policy.minUncompactedBytes) && Objects.equals(maxAverageUncompactedBytesPerSegment, policy.maxAverageUncompactedBytesPerSegment) - // Use Double.compare instead of == to handle NaN correctly and keep equals() consistent with hashCode() (especially for +0.0 vs -0.0). - && - Double.compare( - incrementalCompactionUncompactedRatioThreshold, - policy.incrementalCompactionUncompactedRatioThreshold - ) == 0; + && minUncompactedBytesPercentForFullCompaction == policy.minUncompactedBytesPercentForFullCompaction; } @Override @@ -164,7 +159,7 @@ public int hashCode() minUncompactedCount, minUncompactedBytes, maxAverageUncompactedBytesPerSegment, - incrementalCompactionUncompactedRatioThreshold + minUncompactedBytesPercentForFullCompaction ); } @@ -176,7 +171,7 @@ public String toString() "minUncompactedCount=" + minUncompactedCount + ", minUncompactedBytes=" + minUncompactedBytes + ", maxAverageUncompactedBytesPerSegment=" + maxAverageUncompactedBytesPerSegment + - ", incrementalCompactionUncompactedRatioThreshold=" + incrementalCompactionUncompactedRatioThreshold + + ", minUncompactedBytesPercentForFullCompaction=" + minUncompactedBytesPercentForFullCompaction + ", priorityDataSource='" + getPriorityDatasource() + '\'' + '}'; } @@ -220,12 +215,13 @@ public Eligibility checkEligibilityForCompaction( } final double uncompactedBytesRatio = (double) uncompacted.getTotalBytes() / - (uncompacted.getTotalBytes() + candidate.getCompactedStats().getTotalBytes()); - if (uncompactedBytesRatio < incrementalCompactionUncompactedRatioThreshold) { + (uncompacted.getTotalBytes() + candidate.getCompactedStats().getTotalBytes()) + * 100; + if (uncompactedBytesRatio < minUncompactedBytesPercentForFullCompaction) { return Eligibility.incremental( - "Uncompacted bytes ratio[%.2f] is below threshold[%.2f]", + "Uncompacted bytes ratio[%.2f] is below threshold[%d]", uncompactedBytesRatio, - incrementalCompactionUncompactedRatioThreshold + minUncompactedBytesPercentForFullCompaction ); } else { return Eligibility.OK; diff --git a/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java b/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java index f34aae68dd04..5ca781e423c1 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java @@ -265,7 +265,7 @@ private int submitCompactionTasks( final ClientCompactionTaskQuery taskPayload = createCompactionTask( entry, - CompactionMode.FULL_COMPACTION, + CompactionMode.ALL_SEGMENTS, config, defaultEngine, null, @@ -472,10 +472,10 @@ private static ClientCompactionTaskQuery compactSegments( final String taskId = IdUtils.newTaskId(TASK_ID_PREFIX, ClientCompactionTaskQuery.TYPE, dataSource, null); final ClientCompactionIntervalSpec clientCompactionIntervalSpec; switch (compactionMode) { - case FULL_COMPACTION: + case ALL_SEGMENTS: clientCompactionIntervalSpec = new ClientCompactionIntervalSpec(entry.getCompactionInterval(), null, null); break; - case INCREMENTAL_COMPACTION: + case UNCOMPACTED_SEGMENTS_ONLY: List uncompacted = entry.getUncompactedSegments() .stream() .map(DataSegment::toDescriptor) diff --git a/server/src/test/java/org/apache/druid/server/compaction/MostFragmentedIntervalFirstPolicyTest.java b/server/src/test/java/org/apache/druid/server/compaction/MostFragmentedIntervalFirstPolicyTest.java index 08134db66bb6..7c7903a89bc0 100644 --- a/server/src/test/java/org/apache/druid/server/compaction/MostFragmentedIntervalFirstPolicyTest.java +++ b/server/src/test/java/org/apache/druid/server/compaction/MostFragmentedIntervalFirstPolicyTest.java @@ -219,7 +219,7 @@ public void test_serde_allFieldsSet() throws IOException 1, HumanReadableBytes.valueOf(2), HumanReadableBytes.valueOf(3), - 0.5, + 50, "foo" ); final DefaultObjectMapper mapper = new DefaultObjectMapper(); @@ -247,7 +247,7 @@ public void test_compactionMode_returnsIncrementalCompaction_whenRatioBelowThres 1, HumanReadableBytes.valueOf(1), HumanReadableBytes.valueOf(10_000), - 0.5, + 50, null ); @@ -259,7 +259,7 @@ public void test_compactionMode_returnsIncrementalCompaction_whenRatioBelowThres final CompactionCandidateSearchPolicy.Eligibility eligibility = policy.checkEligibilityForCompaction(candidate, null); - Assertions.assertEquals(CompactionMode.INCREMENTAL_COMPACTION, eligibility.getMode()); + Assertions.assertEquals(CompactionMode.UNCOMPACTED_SEGMENTS_ONLY, eligibility.getMode()); Assertions.assertTrue(eligibility.isEligible()); } @@ -271,7 +271,7 @@ public void test_compactionMode_returnsFullCompaction_whenRatioAboveThreshold() 1, HumanReadableBytes.valueOf(1), HumanReadableBytes.valueOf(10_000), - 0.5, + 50, null ); @@ -286,7 +286,7 @@ public void test_compactionMode_returnsFullCompaction_whenRatioAboveThreshold() final CompactionCandidateSearchPolicy.Eligibility eligibility = policy.checkEligibilityForCompaction(candidate, null); - Assertions.assertEquals(CompactionMode.FULL_COMPACTION, eligibility.getMode()); + Assertions.assertEquals(CompactionMode.ALL_SEGMENTS, eligibility.getMode()); Assertions.assertTrue(eligibility.isEligible()); } @@ -302,7 +302,7 @@ public void test_compactionMode_returnsFullCompaction_whenThresholdIsDefault() null ); - // With default threshold 0.0, any positive ratio >= 0.0, so always FULL_COMPACTION_ELIGIBLE + // With default threshold 0.0, any positive ratio >= 0.0, so always ALL_SEGMENTS_ELIGIBLE final CompactionStatus status = CompactionStatus.pending( CompactionStatistics.create(1_000L, 10, 1), @@ -314,7 +314,7 @@ public void test_compactionMode_returnsFullCompaction_whenThresholdIsDefault() final CompactionCandidateSearchPolicy.Eligibility eligibility = policy.checkEligibilityForCompaction(candidate, null); - Assertions.assertEquals(CompactionMode.FULL_COMPACTION, eligibility.getMode()); + Assertions.assertEquals(CompactionMode.ALL_SEGMENTS, eligibility.getMode()); Assertions.assertTrue(eligibility.isEligible()); } From 3a0bd79261ecd5a6af7298203f81833a5f2ccbf0 Mon Sep 17 00:00:00 2001 From: cecemei Date: Fri, 27 Feb 2026 17:31:45 -0800 Subject: [PATCH 05/26] build --- .../indexing/compact/CompactionConfigBasedJobTemplate.java | 7 +++++-- .../compaction/MostFragmentedIntervalFirstPolicyTest.java | 6 +++--- 2 files changed, 8 insertions(+), 5 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionConfigBasedJobTemplate.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionConfigBasedJobTemplate.java index 96a37114efa5..f4510fcbf72b 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionConfigBasedJobTemplate.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionConfigBasedJobTemplate.java @@ -103,8 +103,11 @@ public List createCompactionJobs( finalCandidate = candidate; break; case UNCOMPACTED_SEGMENTS_ONLY: - finalCandidate = CompactionCandidate.from(candidate.getUncompactedSegments(), null) - .withCurrentStatus(candidate.getCurrentStatus()); + finalCandidate = CompactionCandidate.from( + candidate.getUncompactedSegments(), + null, + candidate.getCurrentStatus() + ); break; default: throw DruidException.defensive("unexpected compaction mode[%s]", eligibility.getMode()); diff --git a/server/src/test/java/org/apache/druid/server/compaction/MostFragmentedIntervalFirstPolicyTest.java b/server/src/test/java/org/apache/druid/server/compaction/MostFragmentedIntervalFirstPolicyTest.java index 51a5deaea294..f443bd00c254 100644 --- a/server/src/test/java/org/apache/druid/server/compaction/MostFragmentedIntervalFirstPolicyTest.java +++ b/server/src/test/java/org/apache/druid/server/compaction/MostFragmentedIntervalFirstPolicyTest.java @@ -255,7 +255,7 @@ public void test_compactionMode_returnsIncrementalCompaction_whenRatioBelowThres final CompactionStatistics uncompacted = CompactionStatistics.create(400L, 100, 1L); final CompactionStatus status = CompactionStatus.pending(compacted, uncompacted, List.of(SEGMENT), ""); - final CompactionCandidate candidate = CompactionCandidate.from(List.of(SEGMENT), null).withCurrentStatus(status); + final CompactionCandidate candidate = CompactionCandidate.from(List.of(SEGMENT), null, status); final CompactionCandidateSearchPolicy.Eligibility eligibility = policy.checkEligibilityForCompaction(candidate, null); @@ -282,7 +282,7 @@ public void test_compactionMode_returnsFullCompaction_whenRatioAboveThreshold() List.of(), "" ); - final CompactionCandidate candidate = CompactionCandidate.from(List.of(SEGMENT), null).withCurrentStatus(status); + final CompactionCandidate candidate = CompactionCandidate.from(List.of(SEGMENT), null, status); final CompactionCandidateSearchPolicy.Eligibility eligibility = policy.checkEligibilityForCompaction(candidate, null); @@ -310,7 +310,7 @@ public void test_compactionMode_returnsFullCompaction_whenThresholdIsDefault() List.of(), "" ); - final CompactionCandidate candidate = CompactionCandidate.from(List.of(SEGMENT), null).withCurrentStatus(status); + final CompactionCandidate candidate = CompactionCandidate.from(List.of(SEGMENT), null, status); final CompactionCandidateSearchPolicy.Eligibility eligibility = policy.checkEligibilityForCompaction(candidate, null); From 3eac326b427c5d1d511fea452000faee96da2599 Mon Sep 17 00:00:00 2001 From: cecemei Date: Fri, 27 Feb 2026 18:20:05 -0800 Subject: [PATCH 06/26] UncompactedInputSpec --- .../compact/CompactionSupervisorTest.java | 2 +- .../common/task/CompactionInputSpec.java | 1 + .../common/task/CompactionIntervalSpec.java | 50 +------ .../indexing/common/task/CompactionTask.java | 17 ++- .../common/task/UncompactedInputSpec.java | 132 ++++++++++++++++++ .../msq/exec/MSQCompactionTaskRunTest.java | 13 +- .../ClientCompactionIntervalSpec.java | 7 +- .../coordinator/duty/CompactSegments.java | 12 +- ...MostFragmentedIntervalFirstPolicyTest.java | 8 +- 9 files changed, 163 insertions(+), 79 deletions(-) create mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/common/task/UncompactedInputSpec.java diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionSupervisorTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionSupervisorTest.java index f07be29d4a16..093695896d00 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionSupervisorTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionSupervisorTest.java @@ -229,7 +229,7 @@ public void test_ingestDayGranularity_andCompactToMonthGranularity_andCompactToY } @Test - public void test_incrementalCompactionWithMSQ() throws Exception + public void test_minorCompactionWithMSQ() throws Exception { configureCompaction( CompactionEngine.MSQ, diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionInputSpec.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionInputSpec.java index d73a0ffe39a0..a0ed3a903046 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionInputSpec.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionInputSpec.java @@ -34,6 +34,7 @@ @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type") @JsonSubTypes(value = { @Type(name = CompactionIntervalSpec.TYPE, value = CompactionIntervalSpec.class), + @Type(name = UncompactedInputSpec.TYPE, value = UncompactedInputSpec.class), @Type(name = SpecificSegmentsSpec.TYPE, value = SpecificSegmentsSpec.class) }) public interface CompactionInputSpec diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionIntervalSpec.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionIntervalSpec.java index b76f49ec40ae..40a4d775e031 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionIntervalSpec.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionIntervalSpec.java @@ -20,12 +20,10 @@ package org.apache.druid.indexing.common.task; import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonInclude; import com.fasterxml.jackson.annotation.JsonProperty; import org.apache.druid.indexing.common.LockGranularity; import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.JodaUtils; -import org.apache.druid.query.SegmentDescriptor; import org.apache.druid.segment.SegmentUtils; import org.apache.druid.timeline.DataSegment; import org.joda.time.Interval; @@ -44,29 +42,11 @@ public class CompactionIntervalSpec implements CompactionInputSpec private final Interval interval; @Nullable - private final List uncompactedSegments; - /** - * Optional hash of all segment IDs for validation. When set, this is used in {@link #validateSegments} to verify - * that the segments haven't changed since this spec was created. - *

    - * Note: This hash is computed and validated against ALL segments overlapping the interval, not just the - * uncompactedSegments. This is because compaction operates on all segments within the interval - compacted - * segments may need to be rewritten alongside uncompacted ones to maintain proper partitioning and sort order. - * Therefore, the validation check must apply to all segments to ensure correctness. - */ - @Nullable private final String sha256OfSortedSegmentIds; - public CompactionIntervalSpec(Interval interval, String sha256OfSortedSegmentIds) - { - this(interval, null, sha256OfSortedSegmentIds); - } - @JsonCreator public CompactionIntervalSpec( @JsonProperty("interval") Interval interval, - @JsonProperty("uncompactedSegments") @Nullable - List uncompactedSegments, @JsonProperty("sha256OfSortedSegmentIds") @Nullable String sha256OfSortedSegmentIds ) { @@ -74,22 +54,6 @@ public CompactionIntervalSpec( throw new IAE("Interval[%s] is empty, must specify a nonempty interval", interval); } this.interval = interval; - if (uncompactedSegments == null) { - // all segments within interval are included, pass check - } else if (uncompactedSegments.isEmpty()) { - throw new IAE("Can not supply empty segments as input, please use either null or non-empty segments."); - } else if (interval != null) { - List segmentsNotInInterval = - uncompactedSegments.stream().filter(s -> !interval.contains(s.getInterval())).collect(Collectors.toList()); - if (!segmentsNotInInterval.isEmpty()) { - throw new IAE( - "Can not supply segments outside interval[%s], got segments[%s].", - interval, - segmentsNotInInterval - ); - } - } - this.uncompactedSegments = uncompactedSegments; this.sha256OfSortedSegmentIds = sha256OfSortedSegmentIds; } @@ -99,14 +63,6 @@ public Interval getInterval() return interval; } - @Nullable - @JsonProperty - @JsonInclude(JsonInclude.Include.NON_NULL) - public List getUncompactedSegments() - { - return uncompactedSegments; - } - @Nullable @JsonProperty public String getSha256OfSortedSegmentIds() @@ -149,14 +105,13 @@ public boolean equals(Object o) } CompactionIntervalSpec that = (CompactionIntervalSpec) o; return Objects.equals(interval, that.interval) && - Objects.equals(uncompactedSegments, that.uncompactedSegments) && Objects.equals(sha256OfSortedSegmentIds, that.sha256OfSortedSegmentIds); } @Override public int hashCode() { - return Objects.hash(interval, uncompactedSegments, sha256OfSortedSegmentIds); + return Objects.hash(interval, sha256OfSortedSegmentIds); } @Override @@ -164,8 +119,7 @@ public String toString() { return "CompactionIntervalSpec{" + "interval=" + interval + - ", uncompactedSegments=" + uncompactedSegments + - ", sha256OfSortedSegmentIds='" + sha256OfSortedSegmentIds + '\'' + + ", sha256OfSegmentIds='" + sha256OfSortedSegmentIds + '\'' + '}'; } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java index ddaa94bb3bc4..f901de8695d3 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java @@ -584,7 +584,7 @@ static Map createInputDataSchemas( return Collections.emptyMap(); } - if (segmentProvider.incrementalCompaction) { + if (segmentProvider.minorCompaction) { Iterable segmentsNotCompletelyWithinin = Iterables.filter(timelineSegments, s -> !segmentProvider.interval.contains(s.getInterval())); if (segmentsNotCompletelyWithinin.iterator().hasNext()) { @@ -660,7 +660,7 @@ static Map createInputDataSchemas( needMultiValuedColumns ); inputSchemas.put( - segmentProvider.incrementalCompaction + segmentProvider.minorCompaction ? new MultipleSpecificSegmentSpec(segmentsToCompact.stream() .map(DataSegment::toDescriptor) .collect(Collectors.toList())) @@ -697,7 +697,7 @@ static Map createInputDataSchemas( projections, needMultiValuedColumns ); - return Map.of(segmentProvider.incrementalCompaction + return Map.of(segmentProvider.minorCompaction ? new MultipleSpecificSegmentSpec(StreamSupport.stream(segmentsToCompact.spliterator(), false) .map(DataSegment::toDescriptor) .collect(Collectors.toList())) @@ -1274,7 +1274,7 @@ static class SegmentProvider private final CompactionInputSpec inputSpec; private final Interval interval; - private final boolean incrementalCompaction; + private final boolean minorCompaction; private final Predicate segmentsToUpgradePredicate; private final Predicate segmentsToCompactPredicate; @@ -1283,15 +1283,14 @@ static class SegmentProvider this.dataSource = Preconditions.checkNotNull(dataSource); this.inputSpec = inputSpec; this.interval = inputSpec.findInterval(dataSource); - if (inputSpec instanceof CompactionIntervalSpec - && ((CompactionIntervalSpec) inputSpec).getUncompactedSegments() != null) { - incrementalCompaction = true; - Set uncompactedSegments = Set.copyOf(((CompactionIntervalSpec) inputSpec).getUncompactedSegments()); + if (inputSpec instanceof UncompactedInputSpec) { + minorCompaction = true; + Set uncompactedSegments = Set.copyOf(((UncompactedInputSpec) inputSpec).getUncompactedSegments()); this.segmentsToUpgradePredicate = s -> !uncompactedSegments.contains(s.toDescriptor()) && this.interval.contains(s.getInterval()); this.segmentsToCompactPredicate = Predicates.not(this.segmentsToUpgradePredicate); } else { - incrementalCompaction = false; + minorCompaction = false; this.segmentsToUpgradePredicate = Predicates.alwaysFalse(); this.segmentsToCompactPredicate = Predicates.alwaysTrue(); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/UncompactedInputSpec.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/UncompactedInputSpec.java new file mode 100644 index 000000000000..4833f1b5457f --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/UncompactedInputSpec.java @@ -0,0 +1,132 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.common.task; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.druid.indexing.common.LockGranularity; +import org.apache.druid.java.util.common.IAE; +import org.apache.druid.java.util.common.JodaUtils; +import org.apache.druid.query.SegmentDescriptor; +import org.apache.druid.timeline.DataSegment; +import org.joda.time.Interval; + +import java.util.List; +import java.util.Objects; +import java.util.stream.Collectors; + +/** + * Specifies uncompacted segments to compact within an interval. + * Used for minor compaction to compact only uncompacted segments while leaving compacted segments untouched. + */ +public class UncompactedInputSpec implements CompactionInputSpec +{ + public static final String TYPE = "uncompacted"; + + private final Interval interval; + private final List uncompactedSegments; + + @JsonCreator + public UncompactedInputSpec( + @JsonProperty("interval") Interval interval, + @JsonProperty("uncompactedSegments") List uncompactedSegments + ) + { + if (interval == null) { + throw new IAE("Uncompacted interval must not be null"); + } + if (interval.toDurationMillis() == 0) { + throw new IAE("Uncompacted interval[%s] is empty, must specify a nonempty interval", interval); + } + if (uncompactedSegments == null || uncompactedSegments.isEmpty()) { + throw new IAE("Uncompacted segments must not be null or empty"); + } + + // Validate that all segments are within the interval + List segmentsNotInInterval = + uncompactedSegments.stream().filter(s -> !interval.contains(s.getInterval())).collect(Collectors.toList()); + if (!segmentsNotInInterval.isEmpty()) { + throw new IAE( + "All uncompacted segments must be within interval[%s], got segments outside interval: %s", + interval, + segmentsNotInInterval + ); + } + + this.interval = interval; + this.uncompactedSegments = uncompactedSegments; + } + + @JsonProperty + public Interval getInterval() + { + return interval; + } + + @JsonProperty + public List getUncompactedSegments() + { + return uncompactedSegments; + } + + @Override + public Interval findInterval(String dataSource) + { + return interval; + } + + @Override + public boolean validateSegments(LockGranularity lockGranularityInUse, List latestSegments) + { + final Interval segmentsInterval = JodaUtils.umbrellaInterval( + latestSegments.stream().map(DataSegment::getInterval).collect(Collectors.toList()) + ); + return interval.overlaps(segmentsInterval); + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + UncompactedInputSpec that = (UncompactedInputSpec) o; + return Objects.equals(interval, that.interval) && + Objects.equals(uncompactedSegments, that.uncompactedSegments); + } + + @Override + public int hashCode() + { + return Objects.hash(interval, uncompactedSegments); + } + + @Override + public String toString() + { + return "UncompactedInputSpec{" + + "interval=" + interval + + ", uncompactedSegments=" + uncompactedSegments + + '}'; + } +} diff --git a/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQCompactionTaskRunTest.java b/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQCompactionTaskRunTest.java index e45ca07116b0..783e0d0bc715 100644 --- a/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQCompactionTaskRunTest.java +++ b/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQCompactionTaskRunTest.java @@ -50,6 +50,7 @@ import org.apache.druid.indexing.common.task.IndexTask; import org.apache.druid.indexing.common.task.Tasks; import org.apache.druid.indexing.common.task.TuningConfigBuilder; +import org.apache.druid.indexing.common.task.UncompactedInputSpec; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.Pair; @@ -578,7 +579,7 @@ public void testMSQCompactionWithConcurrentAppendAppendLocksFirst() throws Excep } @Test - public void testIncrementalCompaction() throws Exception + public void testMinorCompaction() throws Exception { Assume.assumeTrue(lockGranularity == LockGranularity.TIME_CHUNK); Assume.assumeTrue("Incremental compaction depends on concurrent lock", useConcurrentLocks); @@ -602,7 +603,7 @@ public void testIncrementalCompaction() throws Exception .collect(Collectors.toList()); final CompactionTask compactionTask2 = compactionTaskBuilder(segmentGranularity) - .inputSpec(new CompactionIntervalSpec(inputInterval, uncompacted, null), true) + .inputSpec(new UncompactedInputSpec(inputInterval, uncompacted), true) .build(); final Pair resultPair2 = runTask(compactionTask2); verifyTaskSuccessRowsAndSchemaMatch(resultPair2, TOTAL_TEST_ROWS); @@ -628,7 +629,7 @@ public void testIncrementalCompaction() throws Exception } @Test - public void testIncrementalCompactionRangePartition() throws Exception + public void testMinorCompactionRangePartition() throws Exception { List rows = ImmutableList.of( "2014-01-01T00:00:10Z,a,1\n", @@ -671,7 +672,7 @@ public void testIncrementalCompactionRangePartition() throws Exception .collect(Collectors.toList()); final CompactionTask compactionTask2 = compactionTaskBuilder(segmentGranularity) - .inputSpec(new CompactionIntervalSpec(inputInterval, uncompacted, null), true) + .inputSpec(new UncompactedInputSpec(inputInterval, uncompacted), true) .tuningConfig(tuningConfig) .build(); final Pair resultPair2 = runTask(compactionTask2); @@ -686,7 +687,7 @@ public void testIncrementalCompactionRangePartition() throws Exception } @Test - public void testIncrementalCompactionOverlappingInterval() throws Exception + public void testMinorCompactionOverlappingInterval() throws Exception { Assume.assumeTrue(lockGranularity == LockGranularity.TIME_CHUNK); Assume.assumeTrue("Incremental compaction depends on concurrent lock", useConcurrentLocks); @@ -725,7 +726,7 @@ public void testIncrementalCompactionOverlappingInterval() throws Exception final CompactionTask compactionTask1 = compactionTaskBuilder(Granularities.EIGHT_HOUR) - .inputSpec(new CompactionIntervalSpec(compactionInterval, uncompactedFromIndexTask, null), true) + .inputSpec(new UncompactedInputSpec(compactionInterval, uncompactedFromIndexTask), true) .build(); ISE e = Assert.assertThrows(ISE.class, () -> runTask(compactionTask1)); Assert.assertEquals( diff --git a/server/src/main/java/org/apache/druid/client/indexing/ClientCompactionIntervalSpec.java b/server/src/main/java/org/apache/druid/client/indexing/ClientCompactionIntervalSpec.java index 46707e1ea55a..366ad089069a 100644 --- a/server/src/main/java/org/apache/druid/client/indexing/ClientCompactionIntervalSpec.java +++ b/server/src/main/java/org/apache/druid/client/indexing/ClientCompactionIntervalSpec.java @@ -34,11 +34,12 @@ /** * InputSpec for {@link ClientCompactionIOConfig}. *

    - * Should be synchronized with org.apache.druid.indexing.common.task.CompactionIntervalSpec. + * Should be synchronized with org.apache.druid.indexing.common.task.CompactionIntervalSpec and org.apache.druid.indexing.common.task.UncompactedInputSpec. */ public class ClientCompactionIntervalSpec { - private static final String TYPE = "interval"; + private static final String TYPE_ALL_SEGMENTS = "interval"; + private static final String TYPE_UNCOMPACTED_SEGMENTS_ONLY = "uncompacted"; private final Interval interval; @Nullable @@ -79,7 +80,7 @@ public ClientCompactionIntervalSpec( @JsonProperty public String getType() { - return TYPE; + return (uncompactedSegments == null) ? TYPE_ALL_SEGMENTS : TYPE_UNCOMPACTED_SEGMENTS_ONLY; } @JsonProperty diff --git a/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java b/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java index 5ca781e423c1..108e74b1921b 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java @@ -470,21 +470,17 @@ private static ClientCompactionTaskQuery compactSegments( context.put("priority", compactionTaskPriority); final String taskId = IdUtils.newTaskId(TASK_ID_PREFIX, ClientCompactionTaskQuery.TYPE, dataSource, null); - final ClientCompactionIntervalSpec clientCompactionIntervalSpec; + final ClientCompactionIntervalSpec inputSpec; switch (compactionMode) { case ALL_SEGMENTS: - clientCompactionIntervalSpec = new ClientCompactionIntervalSpec(entry.getCompactionInterval(), null, null); + inputSpec = new ClientCompactionIntervalSpec(entry.getCompactionInterval(), null, null); break; case UNCOMPACTED_SEGMENTS_ONLY: List uncompacted = entry.getUncompactedSegments() .stream() .map(DataSegment::toDescriptor) .toList(); - clientCompactionIntervalSpec = new ClientCompactionIntervalSpec( - entry.getCompactionInterval(), - uncompacted, - null - ); + inputSpec = new ClientCompactionIntervalSpec(entry.getCompactionInterval(), uncompacted, null); break; default: throw DruidException.defensive("unexpected compaction mode[%s]", compactionMode); @@ -493,7 +489,7 @@ private static ClientCompactionTaskQuery compactSegments( return new ClientCompactionTaskQuery( taskId, dataSource, - new ClientCompactionIOConfig(clientCompactionIntervalSpec, dropExisting), + new ClientCompactionIOConfig(inputSpec, dropExisting), tuningConfig, granularitySpec, dimensionsSpec, diff --git a/server/src/test/java/org/apache/druid/server/compaction/MostFragmentedIntervalFirstPolicyTest.java b/server/src/test/java/org/apache/druid/server/compaction/MostFragmentedIntervalFirstPolicyTest.java index f443bd00c254..f242cec16805 100644 --- a/server/src/test/java/org/apache/druid/server/compaction/MostFragmentedIntervalFirstPolicyTest.java +++ b/server/src/test/java/org/apache/druid/server/compaction/MostFragmentedIntervalFirstPolicyTest.java @@ -240,7 +240,7 @@ public void test_serde_noFieldsSet() throws IOException } @Test - public void test_compactionMode_returnsIncrementalCompaction_whenRatioBelowThreshold() + public void test_compactionMode_returnsMinorCompactionMode_whenPercentageBelowThreshold() { // Set threshold to 0.5 (50%) final MostFragmentedIntervalFirstPolicy policy = new MostFragmentedIntervalFirstPolicy( @@ -264,7 +264,7 @@ public void test_compactionMode_returnsIncrementalCompaction_whenRatioBelowThres } @Test - public void test_compactionMode_returnsFullCompaction_whenRatioAboveThreshold() + public void test_compactionMode_returnsFullCompaction_whenPercentageAboveThreshold() { // Set threshold to 0.5 (50%) final MostFragmentedIntervalFirstPolicy policy = new MostFragmentedIntervalFirstPolicy( @@ -293,7 +293,7 @@ public void test_compactionMode_returnsFullCompaction_whenRatioAboveThreshold() @Test public void test_compactionMode_returnsFullCompaction_whenThresholdIsDefault() { - // Default threshold is 0.0 + // Default threshold is 0 final MostFragmentedIntervalFirstPolicy policy = new MostFragmentedIntervalFirstPolicy( 1, HumanReadableBytes.valueOf(1), @@ -302,7 +302,7 @@ public void test_compactionMode_returnsFullCompaction_whenThresholdIsDefault() null ); - // With default threshold 0.0, any positive ratio >= 0.0, so always ALL_SEGMENTS_ELIGIBLE + // With default threshold 0, any positive percentage >= 0, so always ALL_SEGMENTS_ELIGIBLE final CompactionStatus status = CompactionStatus.pending( CompactionStatistics.create(1_000L, 10, 1), From ba090ba3369ebf57f4dc7a2cb3a1ec60b9e15d50 Mon Sep 17 00:00:00 2001 From: cecemei Date: Fri, 27 Feb 2026 18:24:09 -0800 Subject: [PATCH 07/26] test --- .../common/task/UncompactedInputSpecTest.java | 104 ++++++++++++++++++ 1 file changed, 104 insertions(+) create mode 100644 indexing-service/src/test/java/org/apache/druid/indexing/common/task/UncompactedInputSpecTest.java diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/UncompactedInputSpecTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/UncompactedInputSpecTest.java new file mode 100644 index 000000000000..e486b9818759 --- /dev/null +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/UncompactedInputSpecTest.java @@ -0,0 +1,104 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.common.task; + +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.jackson.DefaultObjectMapper; +import org.apache.druid.java.util.common.IAE; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.query.SegmentDescriptor; +import org.joda.time.Interval; +import org.junit.Assert; +import org.junit.Test; + +import java.util.List; + +public class UncompactedInputSpecTest +{ + @Test + public void testSerde() throws Exception + { + ObjectMapper mapper = new DefaultObjectMapper(); + Interval interval = Intervals.of("2015-04-11/2015-04-12"); + List segments = List.of( + new SegmentDescriptor(Intervals.of("2015-04-11/2015-04-12"), "v1", 0) + ); + + UncompactedInputSpec spec = new UncompactedInputSpec(interval, segments); + String json = mapper.writeValueAsString(spec); + UncompactedInputSpec deserialized = mapper.readValue(json, UncompactedInputSpec.class); + + Assert.assertEquals(spec, deserialized); + Assert.assertEquals(interval, deserialized.getInterval()); + Assert.assertEquals(segments, deserialized.getUncompactedSegments()); + } + + @Test + public void testDeserializeFromClientFormat() throws Exception + { + ObjectMapper mapper = new DefaultObjectMapper(); + String clientJson = "{" + + "\"type\":\"uncompacted\"," + + "\"interval\":\"2015-04-11/2015-04-12\"," + + "\"uncompactedSegments\":[{\"itvl\":\"2015-04-11/2015-04-12\",\"ver\":\"v1\",\"part\":0}]" + + "}"; + + UncompactedInputSpec deserialized = mapper.readValue(clientJson, UncompactedInputSpec.class); + + Assert.assertEquals(Intervals.of("2015-04-11/2015-04-12"), deserialized.getInterval()); + Assert.assertEquals(1, deserialized.getUncompactedSegments().size()); + Assert.assertEquals( + new SegmentDescriptor(Intervals.of("2015-04-11/2015-04-12"), "v1", 0), + deserialized.getUncompactedSegments().get(0) + ); + } + + @Test + public void testThrowsExceptionWhenInvalidInterval() + { + List segments = List.of( + new SegmentDescriptor(Intervals.of("2015-04-11/2015-04-12"), "v1", 0) + ); + + Assert.assertThrows(IAE.class, () -> new UncompactedInputSpec(null, segments)); + + Interval emptyInterval = Intervals.of("2015-04-11/2015-04-11"); + Assert.assertThrows(IAE.class, () -> new UncompactedInputSpec(emptyInterval, segments)); + } + + @Test + public void testThrowsExceptionWhenInvalidSegments() + { + Interval interval = Intervals.of("2015-04-11/2015-04-12"); + Assert.assertThrows(IAE.class, () -> new UncompactedInputSpec(interval, null)); + Assert.assertThrows(IAE.class, () -> new UncompactedInputSpec(interval, List.of())); + } + + @Test + public void testThrowsExceptionWhenSegmentsOutsideInterval() + { + Interval interval = Intervals.of("2015-04-11/2015-04-12"); + List segments = List.of( + new SegmentDescriptor(Intervals.of("2015-05-11/2015-05-12"), "v1", 0) + ); + + Assert.assertThrows(IAE.class, () -> new UncompactedInputSpec(interval, segments)); + } +} From 64473a70cca048b4c7415f76ce7909e34fdd7ed4 Mon Sep 17 00:00:00 2001 From: cecemei Date: Fri, 27 Feb 2026 18:43:50 -0800 Subject: [PATCH 08/26] uncompacted --- .../embedded/compact/CompactionSupervisorTest.java | 12 ++---------- .../compact/CascadingReindexingTemplate.java | 2 +- .../compact/CompactionConfigBasedJobTemplate.java | 12 +++++++++--- .../druid/indexing/compact/CompactionScheduler.java | 6 ++++++ .../druid/indexing/compact/CompactionSupervisor.java | 3 ++- .../indexing/compact/CompactionSupervisorSpec.java | 5 +++-- .../compact/OverlordCompactionScheduler.java | 6 ++++++ 7 files changed, 29 insertions(+), 17 deletions(-) diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionSupervisorTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionSupervisorTest.java index 093695896d00..f2b5a0e03f92 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionSupervisorTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionSupervisorTest.java @@ -532,7 +532,7 @@ public void test_cascadingReindexing_withVirtualColumnOnNestedData_filtersCorrec cluster.callApi().runTask(task.withId(IdUtils.getRandomId()), overlord); cluster.callApi().waitForAllSegmentsToBeAvailable(dataSource, coordinator, broker); - Assertions.assertEquals(4, getTotalRowCount()); + Assertions.assertEquals("4", cluster.runSql("SELECT COUNT(*) FROM %s", dataSource)); VirtualColumns virtualColumns = VirtualColumns.create( new ExpressionVirtualColumn( @@ -579,7 +579,7 @@ public void test_cascadingReindexing_withVirtualColumnOnNestedData_filtersCorrec cluster.callApi().waitForAllSegmentsToBeAvailable(dataSource, coordinator, broker); // Verify: Should have 2 rows left (valueA appeared in 2 rows, both filtered out) - Assertions.assertEquals(2, getTotalRowCount()); + Assertions.assertEquals("2", cluster.runSql("SELECT COUNT(*) FROM %s", dataSource)); // Verify the correct rows were filtered verifyNoRowsWithNestedValue("extraInfo", "fieldA", "valueA"); @@ -662,14 +662,6 @@ public void test_compactionWithTransformFilteringAllRows_createsTombstones( ); } - private int getTotalRowCount() - { - String sql = StringUtils.format("SELECT COUNT(*) as cnt FROM \"%s\"", dataSource); - ClientSqlQuery clientSqlQuery = new ClientSqlQuery(sql, null, false, false, false, null, null); - List> rows = parse(cluster.callApi().onAnyBroker(b -> b.submitSqlQuery(clientSqlQuery))); - return ((Number) rows.get(0).get("cnt")).intValue(); - } - private void verifyNoRowsWithNestedValue(String nestedColumn, String field, String value) { String sql = StringUtils.format( diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CascadingReindexingTemplate.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CascadingReindexingTemplate.java index 16307fa15399..a7400dd71276 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CascadingReindexingTemplate.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CascadingReindexingTemplate.java @@ -310,7 +310,7 @@ protected CompactionJobTemplate createJobTemplateForInterval( InlineSchemaDataSourceCompactionConfig config ) { - return new CompactionConfigBasedJobTemplate(config, DELETION_RULE_OPTIMIZER); + return new CompactionConfigBasedJobTemplate(config, null, DELETION_RULE_OPTIMIZER); } /** diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionConfigBasedJobTemplate.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionConfigBasedJobTemplate.java index f4510fcbf72b..e0218b1928e5 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionConfigBasedJobTemplate.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionConfigBasedJobTemplate.java @@ -28,6 +28,7 @@ import org.apache.druid.server.compaction.CompactionCandidate; import org.apache.druid.server.compaction.CompactionCandidateSearchPolicy; import org.apache.druid.server.compaction.CompactionSlotManager; +import org.apache.druid.server.compaction.CompactionStatusTracker; import org.apache.druid.server.compaction.DataSourceCompactibleSegmentIterator; import org.apache.druid.server.compaction.NewestSegmentFirstPolicy; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; @@ -49,19 +50,22 @@ public class CompactionConfigBasedJobTemplate implements CompactionJobTemplate { private final DataSourceCompactionConfig config; + private final CompactionStatusTracker statusTracker; private final ReindexingConfigOptimizer configOptimizer; - public CompactionConfigBasedJobTemplate(DataSourceCompactionConfig config) + public CompactionConfigBasedJobTemplate(DataSourceCompactionConfig config, CompactionStatusTracker statusTracker) { - this(config, ReindexingConfigOptimizer.IDENTITY); + this(config, statusTracker, ReindexingConfigOptimizer.IDENTITY); } public CompactionConfigBasedJobTemplate( DataSourceCompactionConfig config, + CompactionStatusTracker statusTracker, ReindexingConfigOptimizer configOptimizer ) { this.config = config; + this.statusTracker = statusTracker; this.configOptimizer = configOptimizer; } @@ -93,7 +97,9 @@ public List createCompactionJobs( while (segmentIterator.hasNext()) { final CompactionCandidate candidate = segmentIterator.next(); final CompactionCandidateSearchPolicy.Eligibility eligibility = - params.getClusterCompactionConfig().getCompactionPolicy().checkEligibilityForCompaction(candidate, null); + params.getClusterCompactionConfig() + .getCompactionPolicy() + .checkEligibilityForCompaction(candidate, statusTracker.getLatestTaskStatus(candidate)); if (!eligibility.isEligible()) { continue; } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionScheduler.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionScheduler.java index 6f5ed1a7a6ef..c19771570139 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionScheduler.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionScheduler.java @@ -20,6 +20,7 @@ package org.apache.druid.indexing.compact; import org.apache.druid.server.compaction.CompactionSimulateResult; +import org.apache.druid.server.compaction.CompactionStatusTracker; import org.apache.druid.server.coordinator.AutoCompactionSnapshot; import org.apache.druid.server.coordinator.ClusterCompactionConfig; import org.apache.druid.server.coordinator.CompactionConfigValidationResult; @@ -60,6 +61,11 @@ public interface CompactionScheduler */ boolean isRunning(); + /** + * @return the tracker for monitoring compaction status across datasources. + */ + CompactionStatusTracker getCompactionStatusTracker(); + CompactionConfigValidationResult validateCompactionConfig(DataSourceCompactionConfig compactionConfig); /** diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionSupervisor.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionSupervisor.java index f9ba0eee6db7..636eeedc7bc6 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionSupervisor.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionSupervisor.java @@ -75,7 +75,8 @@ public List createJobs( CompactionJobParams jobParams ) { - return supervisorSpec.getTemplate().createCompactionJobs(inputSource, jobParams); + return supervisorSpec.getTemplate(scheduler.getCompactionStatusTracker()) + .createCompactionJobs(inputSource, jobParams); } @Override diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionSupervisorSpec.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionSupervisorSpec.java index d9002655f742..d295f68d85c3 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionSupervisorSpec.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionSupervisorSpec.java @@ -24,6 +24,7 @@ import com.fasterxml.jackson.annotation.JsonProperty; import org.apache.druid.common.config.Configs; import org.apache.druid.indexing.overlord.supervisor.SupervisorSpec; +import org.apache.druid.server.compaction.CompactionStatusTracker; import org.apache.druid.server.coordinator.CompactionConfigValidationResult; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; import org.apache.druid.server.security.ResourceAction; @@ -96,12 +97,12 @@ public CompactionSupervisor createSupervisor() /** * @return {@link CompactionJobTemplate} used to create jobs for the supervisor. */ - public CompactionJobTemplate getTemplate() + public CompactionJobTemplate getTemplate(CompactionStatusTracker statusTracker) { if (spec instanceof CascadingReindexingTemplate) { return (CascadingReindexingTemplate) spec; } else { - return new CompactionConfigBasedJobTemplate(spec); + return new CompactionConfigBasedJobTemplate(spec, statusTracker); } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/OverlordCompactionScheduler.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/OverlordCompactionScheduler.java index 11709e616c71..436790414576 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/compact/OverlordCompactionScheduler.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/OverlordCompactionScheduler.java @@ -259,6 +259,12 @@ public boolean isRunning() return started.get(); } + @Override + public CompactionStatusTracker getCompactionStatusTracker() + { + return statusTracker; + } + @Override public CompactionConfigValidationResult validateCompactionConfig(DataSourceCompactionConfig compactionConfig) { From 407e328a69ab93c6b506b2cfe1c3d442a283a5df Mon Sep 17 00:00:00 2001 From: cecemei Date: Fri, 27 Feb 2026 18:48:25 -0800 Subject: [PATCH 09/26] format --- .../compact/CompactionSupervisorTest.java | 40 ++++++++----------- .../common/actions/SegmentUpgradeAction.java | 4 +- .../indexing/common/actions/TaskAction.java | 2 +- .../indexing/common/task/CompactionTask.java | 8 ++-- .../actions/SegmentUpgradeActionTest.java | 12 +++--- 5 files changed, 29 insertions(+), 37 deletions(-) diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionSupervisorTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionSupervisorTest.java index f2b5a0e03f92..ca8f1816307b 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionSupervisorTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionSupervisorTest.java @@ -467,19 +467,24 @@ public void test_cascadingCompactionTemplate_multiplePeriodsApplyDifferentCompac null ); - InlineReindexingRuleProvider.Builder ruleProvider = - InlineReindexingRuleProvider.builder() - .segmentGranularityRules(List.of(hourRule, dayRule)) - .tuningConfigRules(List.of(tuningConfigRule)) - .deletionRules(List.of(deletionRule)); + InlineReindexingRuleProvider.Builder ruleProvider = InlineReindexingRuleProvider.builder() + .segmentGranularityRules(List.of( + hourRule, + dayRule + )) + .tuningConfigRules(List.of( + tuningConfigRule)) + .deletionRules(List.of(deletionRule)); if (compactionEngine == CompactionEngine.NATIVE) { - ruleProvider = ruleProvider.ioConfigRules(List.of(new ReindexingIOConfigRule( - "dropExisting", - null, - Period.days(7), - new UserCompactionTaskIOConfig(true) - ))); + ruleProvider = ruleProvider.ioConfigRules( + List.of(new ReindexingIOConfigRule( + "dropExisting", + null, + Period.days(7), + new UserCompactionTaskIOConfig(true) + )) + ); } CascadingReindexingTemplate cascadingReindexingTemplate = new CascadingReindexingTemplate( @@ -760,24 +765,11 @@ private void waitForAllCompactionTasksToFinish() .hasValueMatching(Matchers.equalTo(0L)) ); - int compacted = overlord.latchableEmitter().getMetricValues( - "interval/compacted/count", - Map.of(DruidMetrics.DATASOURCE, dataSource) - ).stream().mapToInt(Number::intValue).sum(); - System.out.println("compacted " + compacted); - - int skipped = overlord.latchableEmitter().getMetricValues( - "interval/skipCompact/count", - Map.of(DruidMetrics.DATASOURCE, dataSource) - ).stream().mapToInt(Number::intValue).sum(); - System.out.println("skipped " + skipped); - // Wait for all submitted compaction jobs to finish int numSubmittedTasks = overlord.latchableEmitter().getMetricValues( "compact/task/count", Map.of(DruidMetrics.DATASOURCE, dataSource) ).stream().mapToInt(Number::intValue).sum(); - System.out.println("submitted task " + numSubmittedTasks); final Matcher taskTypeMatcher = Matchers.anyOf( Matchers.equalTo("query_controller"), diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentUpgradeAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentUpgradeAction.java index 989d39eddca8..3326d710c059 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentUpgradeAction.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentUpgradeAction.java @@ -43,7 +43,7 @@ * * @return the number of segments successfully inserted into the upgrade segments table */ -public class SegmentUpgradeAction implements TaskAction +public class MarkSegmentToUpgradeAction implements TaskAction { private final String dataSource; private final List upgradeSegments; @@ -53,7 +53,7 @@ public class SegmentUpgradeAction implements TaskAction * @param upgradeSegments the list of segments to be recorded as upgraded */ @JsonCreator - public SegmentUpgradeAction( + public MarkSegmentToUpgradeAction( @JsonProperty("dataSource") String dataSource, @JsonProperty("upgradeSegments") List upgradeSegments ) 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 9c5b35f12ba4..1deab619f75f 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 @@ -37,7 +37,7 @@ @JsonSubTypes.Type(name = "segmentTransactionalInsert", value = SegmentTransactionalInsertAction.class), @JsonSubTypes.Type(name = "segmentTransactionalAppend", value = SegmentTransactionalAppendAction.class), @JsonSubTypes.Type(name = "segmentTransactionalReplace", value = SegmentTransactionalReplaceAction.class), - @JsonSubTypes.Type(name = "markSegmentsToUpgrade", value = SegmentUpgradeAction.class), + @JsonSubTypes.Type(name = "markSegmentsToUpgrade", value = MarkSegmentToUpgradeAction.class), @JsonSubTypes.Type(name = "retrieveSegmentsById", value = RetrieveSegmentsByIdAction.class), @JsonSubTypes.Type(name = "retrieveUpgradedFromSegmentIds", value = RetrieveUpgradedFromSegmentIdsAction.class), @JsonSubTypes.Type(name = "retrieveUpgradedToSegmentIds", value = RetrieveUpgradedToSegmentIdsAction.class), diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java index f901de8695d3..1ded3760db50 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java @@ -58,7 +58,7 @@ import org.apache.druid.indexing.common.SegmentCacheManagerFactory; import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.common.actions.RetrieveUsedSegmentsAction; -import org.apache.druid.indexing.common.actions.SegmentUpgradeAction; +import org.apache.druid.indexing.common.actions.MarkSegmentToUpgradeAction; import org.apache.druid.indexing.common.actions.TaskActionClient; import org.apache.druid.indexing.common.task.IndexTask.IndexTuningConfig; import org.apache.druid.indexing.common.task.batch.parallel.ParallelIndexTuningConfig; @@ -555,7 +555,7 @@ public TaskStatus runTask(TaskToolbox toolbox) throws Exception * When segment granularity is not specified, preserves original granularity and creates a schema * for each unified interval. When segment granularity is specified, creates a single schema for all * segments. For incremental compaction, validates that all segments are completely within the target - * interval and submits already-compacted segments via {@link SegmentUpgradeAction} for direct upgrade. + * interval and submits already-compacted segments via {@link MarkSegmentToUpgradeAction} for direct upgrade. * * @return map from {@link QuerySegmentSpec} to {@link DataSchema} for each group of segments to compact * @throws IOException if an exception occurs while retrieving segments @@ -614,7 +614,7 @@ static Map createInputDataSchemas( } } if (!upgradeSegments.isEmpty()) { - toolbox.getTaskActionClient().submit(new SegmentUpgradeAction(segmentProvider.dataSource, upgradeSegments)); + toolbox.getTaskActionClient().submit(new MarkSegmentToUpgradeAction(segmentProvider.dataSource, upgradeSegments)); } // unify overlapping intervals to ensure overlapping segments compacting in the same indexSpec @@ -674,7 +674,7 @@ static Map createInputDataSchemas( segmentProvider.segmentsToUpgradePredicate )); if (!upgradeSegments.isEmpty()) { - toolbox.getTaskActionClient().submit(new SegmentUpgradeAction(segmentProvider.dataSource, upgradeSegments)); + toolbox.getTaskActionClient().submit(new MarkSegmentToUpgradeAction(segmentProvider.dataSource, upgradeSegments)); } final Iterable segmentsToCompact = Iterables.filter( diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentUpgradeActionTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentUpgradeActionTest.java index 785a4cbff696..0a7edb653f8f 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentUpgradeActionTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentUpgradeActionTest.java @@ -39,7 +39,7 @@ import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; -public class SegmentUpgradeActionTest +public class MarkSegmentToUpgradeActionTest { private TaskActionTestKit actionTestKit; @@ -86,7 +86,7 @@ public void test_segmentsSuccessfullyInsertedIntoUpgradeTable() throws Exception actionTestKit.getTaskLockbox() .lock(task, new TimeChunkLockRequest(TaskLockType.REPLACE, task, INTERVAL_2026_02, null), 5000); - final SegmentUpgradeAction action = new SegmentUpgradeAction(DATA_SOURCE, List.of(SEGMENT1, SEGMENT2, SEGMENT3)); + final MarkSegmentToUpgradeAction action = new MarkSegmentToUpgradeAction(DATA_SOURCE, List.of(SEGMENT1, SEGMENT2, SEGMENT3)); final Integer insertedCount = action.perform(task, actionTestKit.getTaskActionToolbox()); assertEquals(3, insertedCount.intValue()); @@ -102,7 +102,7 @@ public void test_failsWhenSegmentsNotCoveredByReplaceLock() throws Exception actionTestKit.getTaskLockbox() .lock(task, new TimeChunkLockRequest(TaskLockType.REPLACE, task, INTERVAL_2026_01, null), 5000); - final SegmentUpgradeAction action = new SegmentUpgradeAction(DATA_SOURCE, List.of(SEGMENT1, SEGMENT2, SEGMENT3)); + final MarkSegmentToUpgradeAction action = new MarkSegmentToUpgradeAction(DATA_SOURCE, List.of(SEGMENT1, SEGMENT2, SEGMENT3)); DruidException exception = assertThrows( DruidException.class, @@ -119,7 +119,7 @@ public void test_failsWithExclusiveLockInsteadOfReplaceLock() throws Exception actionTestKit.getTaskLockbox() .lock(task, new TimeChunkLockRequest(TaskLockType.EXCLUSIVE, task, INTERVAL_2026_01, null), 5000); - final SegmentUpgradeAction action = new SegmentUpgradeAction(DATA_SOURCE, List.of(SEGMENT1, SEGMENT2)); + final MarkSegmentToUpgradeAction action = new MarkSegmentToUpgradeAction(DATA_SOURCE, List.of(SEGMENT1, SEGMENT2)); DruidException exception = assertThrows( DruidException.class, @@ -134,7 +134,7 @@ public void test_emptySegmentsList() final Task task = NoopTask.forDatasource(DATA_SOURCE); actionTestKit.getTaskLockbox().add(task); - final SegmentUpgradeAction action = new SegmentUpgradeAction(DATA_SOURCE, List.of()); + final MarkSegmentToUpgradeAction action = new MarkSegmentToUpgradeAction(DATA_SOURCE, List.of()); DruidException exception = assertThrows( DruidException.class, @@ -151,7 +151,7 @@ public void test_singleSegmentUpgrade() throws Exception actionTestKit.getTaskLockbox() .lock(task, new TimeChunkLockRequest(TaskLockType.REPLACE, task, INTERVAL_2026_01, null), 5000); - final SegmentUpgradeAction action = new SegmentUpgradeAction(DATA_SOURCE, List.of(SEGMENT1)); + final MarkSegmentToUpgradeAction action = new MarkSegmentToUpgradeAction(DATA_SOURCE, List.of(SEGMENT1)); final Integer insertedCount = action.perform(task, actionTestKit.getTaskActionToolbox()); assertEquals(1, insertedCount.intValue()); From df251871a3b09e939cc1fb6b319173d8cac48169 Mon Sep 17 00:00:00 2001 From: cecemei Date: Fri, 27 Feb 2026 18:51:20 -0800 Subject: [PATCH 10/26] format2 --- .../compact/CompactionSupervisorTest.java | 17 ++++------------- 1 file changed, 4 insertions(+), 13 deletions(-) diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionSupervisorTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionSupervisorTest.java index ca8f1816307b..f4a1362ef5f3 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionSupervisorTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionSupervisorTest.java @@ -468,22 +468,13 @@ public void test_cascadingCompactionTemplate_multiplePeriodsApplyDifferentCompac ); InlineReindexingRuleProvider.Builder ruleProvider = InlineReindexingRuleProvider.builder() - .segmentGranularityRules(List.of( - hourRule, - dayRule - )) - .tuningConfigRules(List.of( - tuningConfigRule)) - .deletionRules(List.of(deletionRule)); + .segmentGranularityRules(List.of(hourRule, dayRule)) + .tuningConfigRules(List.of(tuningConfigRule)) + .deletionRules(List.of(deletionRule)); if (compactionEngine == CompactionEngine.NATIVE) { ruleProvider = ruleProvider.ioConfigRules( - List.of(new ReindexingIOConfigRule( - "dropExisting", - null, - Period.days(7), - new UserCompactionTaskIOConfig(true) - )) + List.of(new ReindexingIOConfigRule("dropExisting", null, Period.days(7), new UserCompactionTaskIOConfig(true))) ); } From 0134e153283b559bdb35c3fe8ea5c22f82ee0aaf Mon Sep 17 00:00:00 2001 From: cecemei Date: Fri, 27 Feb 2026 18:54:07 -0800 Subject: [PATCH 11/26] format --- ...{SegmentUpgradeAction.java => MarkSegmentToUpgradeAction.java} | 0 ...UpgradeActionTest.java => MarkSegmentToUpgradeActionTest.java} | 0 2 files changed, 0 insertions(+), 0 deletions(-) rename indexing-service/src/main/java/org/apache/druid/indexing/common/actions/{SegmentUpgradeAction.java => MarkSegmentToUpgradeAction.java} (100%) rename indexing-service/src/test/java/org/apache/druid/indexing/common/actions/{SegmentUpgradeActionTest.java => MarkSegmentToUpgradeActionTest.java} (100%) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentUpgradeAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/MarkSegmentToUpgradeAction.java similarity index 100% rename from indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentUpgradeAction.java rename to indexing-service/src/main/java/org/apache/druid/indexing/common/actions/MarkSegmentToUpgradeAction.java diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentUpgradeActionTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/MarkSegmentToUpgradeActionTest.java similarity index 100% rename from indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentUpgradeActionTest.java rename to indexing-service/src/test/java/org/apache/druid/indexing/common/actions/MarkSegmentToUpgradeActionTest.java From c57b7f619ee17922c1812a932170f169fa525523 Mon Sep 17 00:00:00 2001 From: cecemei Date: Fri, 27 Feb 2026 20:20:11 -0800 Subject: [PATCH 12/26] fix --- .../compact/CompactionSupervisorTest.java | 7 +- .../indexing/common/task/CompactionTask.java | 2 +- .../compact/CascadingReindexingTemplate.java | 9 ++- .../CascadingReindexingTemplateTest.java | 69 +++++++++++++------ 4 files changed, 60 insertions(+), 27 deletions(-) diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionSupervisorTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionSupervisorTest.java index f4a1362ef5f3..81552a96f559 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionSupervisorTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionSupervisorTest.java @@ -62,6 +62,7 @@ import org.apache.druid.segment.transform.CompactionTransformSpec; import org.apache.druid.segment.virtual.ExpressionVirtualColumn; import org.apache.druid.server.compaction.CompactionCandidateSearchPolicy; +import org.apache.druid.server.compaction.CompactionStatusTracker; import org.apache.druid.server.compaction.InlineReindexingRuleProvider; import org.apache.druid.server.compaction.MostFragmentedIntervalFirstPolicy; import org.apache.druid.server.compaction.ReindexingDeletionRule; @@ -487,7 +488,8 @@ public void test_cascadingCompactionTemplate_multiplePeriodsApplyDifferentCompac null, null, null, - Granularities.HOUR + Granularities.HOUR, + overlord.bindings().getInstance(CompactionStatusTracker.class) ); runCompactionWithSpec(cascadingReindexingTemplate); waitForAllCompactionTasksToFinish(); @@ -566,7 +568,8 @@ public void test_cascadingReindexing_withVirtualColumnOnNestedData_filtersCorrec null, null, null, - Granularities.DAY + Granularities.DAY, + overlord.bindings().getInstance(CompactionStatusTracker.class) ); runCompactionWithSpec(cascadingTemplate); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java index 1ded3760db50..04eddd1b8a72 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java @@ -57,8 +57,8 @@ import org.apache.druid.indexing.common.LockGranularity; import org.apache.druid.indexing.common.SegmentCacheManagerFactory; import org.apache.druid.indexing.common.TaskToolbox; -import org.apache.druid.indexing.common.actions.RetrieveUsedSegmentsAction; import org.apache.druid.indexing.common.actions.MarkSegmentToUpgradeAction; +import org.apache.druid.indexing.common.actions.RetrieveUsedSegmentsAction; import org.apache.druid.indexing.common.actions.TaskActionClient; import org.apache.druid.indexing.common.task.IndexTask.IndexTuningConfig; import org.apache.druid.indexing.common.task.batch.parallel.ParallelIndexTuningConfig; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CascadingReindexingTemplate.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CascadingReindexingTemplate.java index a7400dd71276..eab651e2b7be 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CascadingReindexingTemplate.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CascadingReindexingTemplate.java @@ -19,6 +19,7 @@ package org.apache.druid.indexing.compact; +import com.fasterxml.jackson.annotation.JacksonInject; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.annotations.VisibleForTesting; @@ -32,6 +33,7 @@ import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.segment.transform.CompactionTransformSpec; +import org.apache.druid.server.compaction.CompactionStatusTracker; import org.apache.druid.server.compaction.IntervalGranularityInfo; import org.apache.druid.server.compaction.ReindexingRule; import org.apache.druid.server.compaction.ReindexingRuleProvider; @@ -99,6 +101,7 @@ public class CascadingReindexingTemplate implements CompactionJobTemplate, DataS private final Period skipOffsetFromLatest; private final Period skipOffsetFromNow; private final Granularity defaultSegmentGranularity; + private final CompactionStatusTracker statusTracker; @JsonCreator public CascadingReindexingTemplate( @@ -110,7 +113,8 @@ public CascadingReindexingTemplate( @JsonProperty("taskContext") @Nullable Map taskContext, @JsonProperty("skipOffsetFromLatest") @Nullable Period skipOffsetFromLatest, @JsonProperty("skipOffsetFromNow") @Nullable Period skipOffsetFromNow, - @JsonProperty("defaultSegmentGranularity") Granularity defaultSegmentGranularity + @JsonProperty("defaultSegmentGranularity") Granularity defaultSegmentGranularity, + @JacksonInject CompactionStatusTracker statusTracker ) { InvalidInput.conditionalException(dataSource != null, "'dataSource' cannot be null"); @@ -132,6 +136,7 @@ public CascadingReindexingTemplate( } this.skipOffsetFromNow = skipOffsetFromNow; this.skipOffsetFromLatest = skipOffsetFromLatest; + this.statusTracker = statusTracker; } @Override @@ -310,7 +315,7 @@ protected CompactionJobTemplate createJobTemplateForInterval( InlineSchemaDataSourceCompactionConfig config ) { - return new CompactionConfigBasedJobTemplate(config, null, DELETION_RULE_OPTIMIZER); + return new CompactionConfigBasedJobTemplate(config, statusTracker, DELETION_RULE_OPTIMIZER); } /** diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/compact/CascadingReindexingTemplateTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/compact/CascadingReindexingTemplateTest.java index 01b361004445..4019ad86bc48 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/compact/CascadingReindexingTemplateTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/compact/CascadingReindexingTemplateTest.java @@ -19,6 +19,7 @@ package org.apache.druid.indexing.compact; +import com.fasterxml.jackson.databind.InjectableValues; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableMap; import org.apache.druid.error.DruidException; @@ -30,6 +31,7 @@ import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.query.aggregation.AggregatorFactory; +import org.apache.druid.server.compaction.CompactionStatusTracker; import org.apache.druid.server.compaction.InlineReindexingRuleProvider; import org.apache.druid.server.compaction.IntervalGranularityInfo; import org.apache.druid.server.compaction.ReindexingDataSchemaRule; @@ -63,6 +65,7 @@ public class CascadingReindexingTemplateTest extends InitializedNullHandlingTest public void setUp() { OBJECT_MAPPER.registerModules(new SupervisorModule().getJacksonModules()); + OBJECT_MAPPER.setInjectableValues(new InjectableValues.Std().addValue(CompactionStatusTracker.class, null)); } @Test @@ -92,7 +95,8 @@ public void test_serde() throws Exception ImmutableMap.of("context_key", "context_value"), null, null, - Granularities.DAY + Granularities.DAY, + null ); final String json = OBJECT_MAPPER.writeValueAsString(template); @@ -127,7 +131,8 @@ public void test_serde_asDataSourceCompactionConfig() throws Exception ImmutableMap.of("key", "value"), null, null, - Granularities.HOUR + Granularities.HOUR, + null ); // Serialize and deserialize as DataSourceCompactionConfig interface @@ -162,7 +167,8 @@ public void test_createCompactionJobs_ruleProviderNotReady() null, null, null, - Granularities.DAY + Granularities.DAY, + null ); // Call createCompactionJobs - should return empty list without processing @@ -189,7 +195,8 @@ public void test_constructor_setBothSkipOffsetStrategiesThrowsException() null, Period.days(7), // skipOffsetFromLatest Period.days(3), // skipOffsetFromNow - Granularities.DAY + Granularities.DAY, + null ) ); @@ -214,7 +221,8 @@ public void test_constructor_nullDataSourceThrowsException() null, null, null, - Granularities.DAY + Granularities.DAY, + null ) ); @@ -236,7 +244,8 @@ public void test_constructor_nullRuleProviderThrowsException() null, null, null, - Granularities.DAY + Granularities.DAY, + null ) ); @@ -260,7 +269,8 @@ public void test_constructor_nullDefaultSegmentGranularityThrowsException() null, null, null, - null // null defaultSegmentGranularity + null, // null defaultSegmentGranularity, + null ) ); @@ -482,7 +492,8 @@ public void test_generateAlignedSearchIntervals_withGranularityAlignment() null, null, null, - Granularities.DAY + Granularities.DAY, + null ); List expected = List.of( @@ -573,7 +584,8 @@ public void test_generateAlignedSearchIntervals_withNonSegmentGranularityRuleSpl null, null, null, - Granularities.DAY + Granularities.DAY, + null ); List expected = List.of( @@ -673,7 +685,8 @@ public void test_generateAlignedSearchIntervals_withNoSegmentGranularityRules() null, null, null, - Granularities.DAY + Granularities.DAY, + null ); // When no segment granularity rules exist, a synthetic rule is created with the smallest period @@ -765,7 +778,8 @@ public void test_generateAlignedSearchIntervals_prependIntervalForShortNonSegmen null, null, null, - Granularities.HOUR + Granularities.HOUR, + null ); List expected = List.of( @@ -871,7 +885,8 @@ public void test_generateAlignedSearchIntervals() null, null, null, - Granularities.HOUR + Granularities.HOUR, + null ); List expected = List.of( @@ -941,7 +956,8 @@ public void test_generateAlignedSearchIntervals_noRulesThrowsException() null, null, null, - Granularities.DAY + Granularities.DAY, + null ); DruidException exception = Assertions.assertThrows( @@ -1006,7 +1022,8 @@ public void test_generateAlignedSearchIntervals_splitPointSnapsToExistingBoundar null, null, null, - Granularities.DAY + Granularities.DAY, + null ); List expected = List.of( @@ -1073,7 +1090,8 @@ public void test_generateAlignedSearchIntervals_prependAlignmentDoesNotExtendTim null, null, null, - Granularities.DAY + Granularities.DAY, + null ); List expected = List.of( @@ -1144,7 +1162,8 @@ public void test_generateAlignedSearchIntervals_duplicateSplitPointsFiltered() null, null, null, - Granularities.DAY + Granularities.DAY, + null ); List expected = List.of( @@ -1208,7 +1227,8 @@ public void test_generateAlignedSearchIntervals_singleRuleOnly() null, null, null, - Granularities.DAY + Granularities.DAY, + null ); List expected = List.of( @@ -1272,7 +1292,8 @@ public void test_generateAlignedSearchIntervals_zeroPeriodRuleAppliesImmediately null, null, null, - Granularities.DAY + Granularities.DAY, + null ); List expected = List.of( @@ -1357,7 +1378,8 @@ public void test_generateAlignedSearchIntervals_zeroPeriodRuleWithOtherRules() null, null, null, - Granularities.DAY + Granularities.DAY, + null ); List expected = List.of( @@ -1430,7 +1452,8 @@ public void test_generateAlignedSearchIntervals_failsWhenDefaultGranularityIsCoa null, null, null, - Granularities.MONTH // MONTH is coarser than HOUR! + Granularities.MONTH, // MONTH is coarser than HOUR! + null ); IllegalArgumentException exception = Assertions.assertThrows( @@ -1491,7 +1514,8 @@ public void test_generateAlignedSearchIntervals_failsWhenOlderRuleHasFinerGranul null, null, null, - Granularities.DAY + Granularities.DAY, + null ); IllegalArgumentException exception = Assertions.assertThrows( @@ -1524,7 +1548,8 @@ public TestCascadingReindexingTemplate( ) { super(dataSource, taskPriority, inputSegmentSizeBytes, ruleProvider, - engine, taskContext, skipOffsetFromLatest, skipOffsetFromNow, Granularities.DAY); + engine, taskContext, skipOffsetFromLatest, skipOffsetFromNow, Granularities.DAY, null + ); } public List getProcessedIntervals() From 1d19c8550b54af54c3361ae85530ad6c4ba40433 Mon Sep 17 00:00:00 2001 From: cecemei Date: Tue, 3 Mar 2026 13:34:08 -0800 Subject: [PATCH 13/26] review --- .../compact/CompactionSupervisorTest.java | 28 +++++----- .../common/task/CompactionInputSpec.java | 2 +- .../indexing/common/task/CompactionTask.java | 4 +- ...pec.java => MinorCompactionInputSpec.java} | 42 +++++++-------- ...java => MinorCompactionInputSpecTest.java} | 20 +++---- .../msq/exec/MSQCompactionTaskRunTest.java | 8 +-- .../IndexerSQLMetadataStorageCoordinator.java | 52 +++++++++++++------ .../druid/server/http/DataSegmentPlus.java | 2 +- 8 files changed, 92 insertions(+), 66 deletions(-) rename indexing-service/src/main/java/org/apache/druid/indexing/common/task/{UncompactedInputSpec.java => MinorCompactionInputSpec.java} (77%) rename indexing-service/src/test/java/org/apache/druid/indexing/common/task/{UncompactedInputSpecTest.java => MinorCompactionInputSpecTest.java} (77%) diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionSupervisorTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionSupervisorTest.java index 81552a96f559..0978c97af8dc 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionSupervisorTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionSupervisorTest.java @@ -229,8 +229,9 @@ public void test_ingestDayGranularity_andCompactToMonthGranularity_andCompactToY verifyCompactedSegmentsHaveFingerprints(yearGranConfig); } - @Test - public void test_minorCompactionWithMSQ() throws Exception + @MethodSource("getPartitionsSpec") + @ParameterizedTest(name = "partitionsSpec={0}") + public void test_minorCompactionWithMSQ(PartitionsSpec partitionsSpec) throws Exception { configureCompaction( CompactionEngine.MSQ, @@ -255,7 +256,6 @@ public void test_minorCompactionWithMSQ() throws Exception // Before compaction Assertions.assertEquals(4, getNumSegmentsWith(Granularities.HOUR)); - PartitionsSpec partitionsSpec = new DimensionRangePartitionsSpec(null, 5000, List.of("page"), false); // Create a compaction config with DAY granularity InlineSchemaDataSourceCompactionConfig dayGranularityConfig = InlineSchemaDataSourceCompactionConfig @@ -279,25 +279,19 @@ public void test_minorCompactionWithMSQ() throws Exception pauseCompaction(dayGranularityConfig); Assertions.assertEquals(0, getNumSegmentsWith(Granularities.HOUR)); Assertions.assertEquals(1, getNumSegmentsWith(Granularities.DAY)); + Assertions.assertEquals("2000", cluster.runSql("SELECT COUNT(*) FROM %s", dataSource)); verifyCompactedSegmentsHaveFingerprints(dayGranularityConfig); - // Set up another topic and supervisor - final String topic2 = IdUtils.getRandomId(); - kafkaServer.createTopicWithPartitions(topic2, 1); - final KafkaSupervisorSpec supervisor2 = kafkaSupervisorSpecBuilder.withId(topic2).build(dataSource, topic2); - cluster.callApi().postSupervisor(supervisor2); - // published another 1k - final int appendedRowCount = publish1kRecords(topic2, true); + final int appendedRowCount = publish1kRecords(topic1, true); indexer.latchableEmitter().flush(); waitUntilPublishedRecordsAreIngested(appendedRowCount); // Tear down both topics and supervisors kafkaServer.deleteTopic(topic1); cluster.callApi().postSupervisor(supervisor1.createSuspendedSpec()); - kafkaServer.deleteTopic(topic2); - cluster.callApi().postSupervisor(supervisor2.createSuspendedSpec()); + long totalUsed = overlord.latchableEmitter().getMetricValues( "segment/metadataCache/used/count", Map.of(DruidMetrics.DATASOURCE, dataSource) @@ -306,6 +300,7 @@ public void test_minorCompactionWithMSQ() throws Exception Assertions.assertEquals(0, getNumSegmentsWith(Granularities.HOUR)); // 1 compacted segment + 2 appended segment Assertions.assertEquals(3, getNumSegmentsWith(Granularities.DAY)); + Assertions.assertEquals("3000", cluster.runSql("SELECT COUNT(*) FROM %s", dataSource)); runCompactionWithSpec(dayGranularityConfig); waitForAllCompactionTasksToFinish(); @@ -318,6 +313,7 @@ public void test_minorCompactionWithMSQ() throws Exception // performed incremental compaction: 1 previously compacted segment + 1 incrementally compacted segment Assertions.assertEquals(2, getNumSegmentsWith(Granularities.DAY)); + Assertions.assertEquals("3000", cluster.runSql("SELECT COUNT(*) FROM %s", dataSource)); } protected void waitUntilPublishedRecordsAreIngested(int expectedRowCount) @@ -803,6 +799,14 @@ private void runIngestionAtGranularity( cluster.callApi().runTask(task, overlord); } + public static List getPartitionsSpec() + { + return List.of( + new DimensionRangePartitionsSpec(null, 5000, List.of("page"), false), + new DynamicPartitionsSpec(null, null) + ); + } + public static List getEngine() { return List.of(CompactionEngine.NATIVE, CompactionEngine.MSQ); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionInputSpec.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionInputSpec.java index a0ed3a903046..a2eab7bde473 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionInputSpec.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionInputSpec.java @@ -34,7 +34,7 @@ @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type") @JsonSubTypes(value = { @Type(name = CompactionIntervalSpec.TYPE, value = CompactionIntervalSpec.class), - @Type(name = UncompactedInputSpec.TYPE, value = UncompactedInputSpec.class), + @Type(name = MinorCompactionInputSpec.TYPE, value = MinorCompactionInputSpec.class), @Type(name = SpecificSegmentsSpec.TYPE, value = SpecificSegmentsSpec.class) }) public interface CompactionInputSpec diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java index 04eddd1b8a72..86f0f7a5a1b5 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java @@ -1283,9 +1283,9 @@ static class SegmentProvider this.dataSource = Preconditions.checkNotNull(dataSource); this.inputSpec = inputSpec; this.interval = inputSpec.findInterval(dataSource); - if (inputSpec instanceof UncompactedInputSpec) { + if (inputSpec instanceof MinorCompactionInputSpec) { minorCompaction = true; - Set uncompactedSegments = Set.copyOf(((UncompactedInputSpec) inputSpec).getUncompactedSegments()); + Set uncompactedSegments = Set.copyOf(((MinorCompactionInputSpec) inputSpec).getUncompactedSegments()); this.segmentsToUpgradePredicate = s -> !uncompactedSegments.contains(s.toDescriptor()) && this.interval.contains(s.getInterval()); this.segmentsToCompactPredicate = Predicates.not(this.segmentsToUpgradePredicate); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/UncompactedInputSpec.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/MinorCompactionInputSpec.java similarity index 77% rename from indexing-service/src/main/java/org/apache/druid/indexing/common/task/UncompactedInputSpec.java rename to indexing-service/src/main/java/org/apache/druid/indexing/common/task/MinorCompactionInputSpec.java index 4833f1b5457f..18bfa139fe84 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/UncompactedInputSpec.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/MinorCompactionInputSpec.java @@ -21,8 +21,8 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.druid.error.InvalidInput; import org.apache.druid.indexing.common.LockGranularity; -import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.JodaUtils; import org.apache.druid.query.SegmentDescriptor; import org.apache.druid.timeline.DataSegment; @@ -36,7 +36,7 @@ * Specifies uncompacted segments to compact within an interval. * Used for minor compaction to compact only uncompacted segments while leaving compacted segments untouched. */ -public class UncompactedInputSpec implements CompactionInputSpec +public class MinorCompactionInputSpec implements CompactionInputSpec { public static final String TYPE = "uncompacted"; @@ -44,31 +44,31 @@ public class UncompactedInputSpec implements CompactionInputSpec private final List uncompactedSegments; @JsonCreator - public UncompactedInputSpec( + public MinorCompactionInputSpec( @JsonProperty("interval") Interval interval, @JsonProperty("uncompactedSegments") List uncompactedSegments ) { - if (interval == null) { - throw new IAE("Uncompacted interval must not be null"); - } - if (interval.toDurationMillis() == 0) { - throw new IAE("Uncompacted interval[%s] is empty, must specify a nonempty interval", interval); - } - if (uncompactedSegments == null || uncompactedSegments.isEmpty()) { - throw new IAE("Uncompacted segments must not be null or empty"); - } + InvalidInput.conditionalException(interval != null, "Uncompacted interval must not be null"); + InvalidInput.conditionalException( + interval.toDurationMillis() > 0, + "Uncompacted interval[%s] is empty, must specify a nonempty interval", + interval + ); + InvalidInput.conditionalException( + uncompactedSegments != null && !uncompactedSegments.isEmpty(), + "Uncompacted segments must not be null or empty" + ); // Validate that all segments are within the interval List segmentsNotInInterval = uncompactedSegments.stream().filter(s -> !interval.contains(s.getInterval())).collect(Collectors.toList()); - if (!segmentsNotInInterval.isEmpty()) { - throw new IAE( - "All uncompacted segments must be within interval[%s], got segments outside interval: %s", - interval, - segmentsNotInInterval - ); - } + InvalidInput.conditionalException( + segmentsNotInInterval.isEmpty(), + "All uncompacted segments must be within interval[%s], got segments outside interval: %s", + interval, + segmentsNotInInterval + ); this.interval = interval; this.uncompactedSegments = uncompactedSegments; @@ -110,7 +110,7 @@ public boolean equals(Object o) if (o == null || getClass() != o.getClass()) { return false; } - UncompactedInputSpec that = (UncompactedInputSpec) o; + MinorCompactionInputSpec that = (MinorCompactionInputSpec) o; return Objects.equals(interval, that.interval) && Objects.equals(uncompactedSegments, that.uncompactedSegments); } @@ -124,7 +124,7 @@ public int hashCode() @Override public String toString() { - return "UncompactedInputSpec{" + + return "MinorCompactionInputSpec{" + "interval=" + interval + ", uncompactedSegments=" + uncompactedSegments + '}'; diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/UncompactedInputSpecTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/MinorCompactionInputSpecTest.java similarity index 77% rename from indexing-service/src/test/java/org/apache/druid/indexing/common/task/UncompactedInputSpecTest.java rename to indexing-service/src/test/java/org/apache/druid/indexing/common/task/MinorCompactionInputSpecTest.java index e486b9818759..c5a6f2fda033 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/UncompactedInputSpecTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/MinorCompactionInputSpecTest.java @@ -20,8 +20,8 @@ package org.apache.druid.indexing.common.task; import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.error.DruidException; import org.apache.druid.jackson.DefaultObjectMapper; -import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.query.SegmentDescriptor; import org.joda.time.Interval; @@ -30,7 +30,7 @@ import java.util.List; -public class UncompactedInputSpecTest +public class MinorCompactionInputSpecTest { @Test public void testSerde() throws Exception @@ -41,9 +41,9 @@ public void testSerde() throws Exception new SegmentDescriptor(Intervals.of("2015-04-11/2015-04-12"), "v1", 0) ); - UncompactedInputSpec spec = new UncompactedInputSpec(interval, segments); + MinorCompactionInputSpec spec = new MinorCompactionInputSpec(interval, segments); String json = mapper.writeValueAsString(spec); - UncompactedInputSpec deserialized = mapper.readValue(json, UncompactedInputSpec.class); + MinorCompactionInputSpec deserialized = mapper.readValue(json, MinorCompactionInputSpec.class); Assert.assertEquals(spec, deserialized); Assert.assertEquals(interval, deserialized.getInterval()); @@ -60,7 +60,7 @@ public void testDeserializeFromClientFormat() throws Exception + "\"uncompactedSegments\":[{\"itvl\":\"2015-04-11/2015-04-12\",\"ver\":\"v1\",\"part\":0}]" + "}"; - UncompactedInputSpec deserialized = mapper.readValue(clientJson, UncompactedInputSpec.class); + MinorCompactionInputSpec deserialized = mapper.readValue(clientJson, MinorCompactionInputSpec.class); Assert.assertEquals(Intervals.of("2015-04-11/2015-04-12"), deserialized.getInterval()); Assert.assertEquals(1, deserialized.getUncompactedSegments().size()); @@ -77,18 +77,18 @@ public void testThrowsExceptionWhenInvalidInterval() new SegmentDescriptor(Intervals.of("2015-04-11/2015-04-12"), "v1", 0) ); - Assert.assertThrows(IAE.class, () -> new UncompactedInputSpec(null, segments)); + Assert.assertThrows(DruidException.class, () -> new MinorCompactionInputSpec(null, segments)); Interval emptyInterval = Intervals.of("2015-04-11/2015-04-11"); - Assert.assertThrows(IAE.class, () -> new UncompactedInputSpec(emptyInterval, segments)); + Assert.assertThrows(DruidException.class, () -> new MinorCompactionInputSpec(emptyInterval, segments)); } @Test public void testThrowsExceptionWhenInvalidSegments() { Interval interval = Intervals.of("2015-04-11/2015-04-12"); - Assert.assertThrows(IAE.class, () -> new UncompactedInputSpec(interval, null)); - Assert.assertThrows(IAE.class, () -> new UncompactedInputSpec(interval, List.of())); + Assert.assertThrows(DruidException.class, () -> new MinorCompactionInputSpec(interval, null)); + Assert.assertThrows(DruidException.class, () -> new MinorCompactionInputSpec(interval, List.of())); } @Test @@ -99,6 +99,6 @@ public void testThrowsExceptionWhenSegmentsOutsideInterval() new SegmentDescriptor(Intervals.of("2015-05-11/2015-05-12"), "v1", 0) ); - Assert.assertThrows(IAE.class, () -> new UncompactedInputSpec(interval, segments)); + Assert.assertThrows(DruidException.class, () -> new MinorCompactionInputSpec(interval, segments)); } } diff --git a/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQCompactionTaskRunTest.java b/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQCompactionTaskRunTest.java index 783e0d0bc715..bcea86028fcf 100644 --- a/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQCompactionTaskRunTest.java +++ b/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQCompactionTaskRunTest.java @@ -48,9 +48,9 @@ import org.apache.druid.indexing.common.task.CompactionTask; import org.apache.druid.indexing.common.task.CompactionTaskRunBase; import org.apache.druid.indexing.common.task.IndexTask; +import org.apache.druid.indexing.common.task.MinorCompactionInputSpec; import org.apache.druid.indexing.common.task.Tasks; import org.apache.druid.indexing.common.task.TuningConfigBuilder; -import org.apache.druid.indexing.common.task.UncompactedInputSpec; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.Pair; @@ -603,7 +603,7 @@ public void testMinorCompaction() throws Exception .collect(Collectors.toList()); final CompactionTask compactionTask2 = compactionTaskBuilder(segmentGranularity) - .inputSpec(new UncompactedInputSpec(inputInterval, uncompacted), true) + .inputSpec(new MinorCompactionInputSpec(inputInterval, uncompacted), true) .build(); final Pair resultPair2 = runTask(compactionTask2); verifyTaskSuccessRowsAndSchemaMatch(resultPair2, TOTAL_TEST_ROWS); @@ -672,7 +672,7 @@ public void testMinorCompactionRangePartition() throws Exception .collect(Collectors.toList()); final CompactionTask compactionTask2 = compactionTaskBuilder(segmentGranularity) - .inputSpec(new UncompactedInputSpec(inputInterval, uncompacted), true) + .inputSpec(new MinorCompactionInputSpec(inputInterval, uncompacted), true) .tuningConfig(tuningConfig) .build(); final Pair resultPair2 = runTask(compactionTask2); @@ -726,7 +726,7 @@ public void testMinorCompactionOverlappingInterval() throws Exception final CompactionTask compactionTask1 = compactionTaskBuilder(Granularities.EIGHT_HOUR) - .inputSpec(new UncompactedInputSpec(compactionInterval, uncompactedFromIndexTask), true) + .inputSpec(new MinorCompactionInputSpec(compactionInterval, uncompactedFromIndexTask), true) .build(); ISE e = Assert.assertThrows(ISE.class, () -> runTask(compactionTask1)); Assert.assertEquals( 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 22d1f709096f..45c7a5999116 100644 --- a/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java +++ b/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java @@ -495,16 +495,15 @@ public SegmentPublishResult commitReplaceSegments( final SegmentPublishResult result = inReadWriteDatasourceTransaction( dataSource, transaction -> { - final Pair, Set> newSegments = createNewSegmentsAfterReplace( + final Set newSegments = createNewSegmentsAfterReplace( dataSource, transaction, replaceSegments, locksHeldByReplaceTask ); - final Set segmentsToInsert = newSegments.rhs; Map upgradeSegmentMetadata = new HashMap<>(); final Map upgradedFromSegmentIdMap = new HashMap<>(); - for (DataSegmentPlus dataSegmentPlus : newSegments.lhs) { + for (DataSegmentPlus dataSegmentPlus : newSegments) { if (dataSegmentPlus.getSchemaFingerprint() != null && dataSegmentPlus.getNumRows() != null) { upgradeSegmentMetadata.put( dataSegmentPlus.getDataSegment().getId(), @@ -518,6 +517,9 @@ public SegmentPublishResult commitReplaceSegments( ); } } + final Set segmentsToInsert = newSegments.stream() + .map(DataSegmentPlus::getDataSegment) + .collect(Collectors.toSet()); return SegmentPublishResult.ok( insertSegments( transaction, @@ -1860,7 +1862,7 @@ protected Set insertSegments( * @return pair of (upgraded segments for metadata tracking, segments to insert into segment table) * @throws DruidException if a replace interval partially overlaps a segment being upgraded */ - private Pair, Set> createNewSegmentsAfterReplace( + private Set createNewSegmentsAfterReplace( final String dataSource, final SegmentMetadataTransaction transaction, final Set replaceSegments, @@ -1870,7 +1872,7 @@ private Pair, Set> createNewSegmentsAfterRepla // If a "REPLACE" task has locked an interval, it would commit some segments // (or at least tombstones) in that interval (except in LEGACY_REPLACE ingestion mode) if (replaceSegments.isEmpty() || locksHeldByReplaceTask.isEmpty()) { - return Pair.of(Collections.emptySet(), Collections.emptySet()); + return Collections.emptySet(); } // For each replace interval, find the current partition number @@ -1893,17 +1895,19 @@ private Pair, Set> createNewSegmentsAfterRepla .map(ReplaceTaskLock::getSupervisorTaskId) .findFirst().orElse(null); final Map upgradeSegmentToLockVersion - = getAppendSegmentsCommittedDuringTask(transaction, taskId); + = getSegmentsCoveredByTaskLock(transaction, taskId); final List segmentsToUpgrade = retrieveSegmentsById(dataSource, transaction, upgradeSegmentToLockVersion.keySet()); if (segmentsToUpgrade.isEmpty()) { - return Pair.of(Collections.emptySet(), replaceSegments); + return replaceSegments.stream() + .map(s -> new DataSegmentPlus(s, null, null, null, null, null, null, null)) + .collect(Collectors.toSet()); } final Set replaceIntervals = intervalToCurrentPartitionNum.keySet(); - final Set upgradedSegments = new HashSet<>(); + final Map upgradedSegments = new HashMap<>(); final Set segmentsToInsert = new HashSet<>(replaceSegments); for (DataSegmentPlus oldSegmentMetadata : segmentsToUpgrade) { // Determine interval of the upgraded segment @@ -1956,7 +1960,8 @@ private Pair, Set> createNewSegmentsAfterRepla ? oldSegmentMetadata.getDataSegment().getId().toString() : oldSegmentMetadata.getUpgradedFromSegmentId(); - upgradedSegments.add( + upgradedSegments.put( + dataSegment, new DataSegmentPlus( dataSegment, null, @@ -1971,8 +1976,8 @@ private Pair, Set> createNewSegmentsAfterRepla segmentsToInsert.add(dataSegment); } - // update corePartitions in shard spec - return Pair.of(upgradedSegments, segmentsToInsert.stream().map(segment -> { + return segmentsToInsert.stream().map(segment -> { + // update corePartitions in shard spec Integer partitionNum = intervalToCurrentPartitionNum.get(segment.getInterval()); if (!segment.isTombstone() && !numChunkNotSupported.contains(segment.getInterval()) @@ -1982,7 +1987,24 @@ private Pair, Set> createNewSegmentsAfterRepla } else { return segment; } - }).collect(Collectors.toSet())); + }).map(s -> { + // wrap with DataSegmentPlus + if (upgradedSegments.containsKey(s)) { + DataSegmentPlus upgraded = upgradedSegments.get(s); + return new DataSegmentPlus( + s, + upgraded.getCreatedDate(), + upgraded.getUsedStatusLastUpdatedDate(), + upgraded.getUsed(), + upgraded.getSchemaFingerprint(), + upgraded.getNumRows(), + upgraded.getUpgradedFromSegmentId(), + upgraded.getIndexingStateFingerprint() + ); + } else { + return new DataSegmentPlus(s, null, null, null, null, null, null, null); + } + }).collect(Collectors.toSet()); } /** @@ -2205,13 +2227,13 @@ private List retrieveSegmentsById( } /** - * Finds the append segments that were covered by the given task REPLACE locks. - * These append segments must now be upgraded to the same version as the segments + * Finds segments were covered by the given task REPLACE locks. + * These segments must now be upgraded to the same version as the segments * being committed by this replace task. * * @return Map from append Segment ID to REPLACE lock version */ - private Map getAppendSegmentsCommittedDuringTask( + private Map getSegmentsCoveredByTaskLock( SegmentMetadataTransaction transaction, String taskId ) 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 e06241f6aae7..56d9155c6881 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 @@ -82,7 +82,7 @@ public DataSegmentPlus( this.upgradedFromSegmentId = upgradedFromSegmentId; this.indexingStateFingerprint = indexingStateFingerprint; } - + @Nullable @JsonProperty public DateTime getCreatedDate() From a7951bef80385ca7344cb76b527dee5e67424d6c Mon Sep 17 00:00:00 2001 From: Cece Mei Date: Tue, 3 Mar 2026 13:36:07 -0800 Subject: [PATCH 14/26] Update indexing-service/src/main/java/org/apache/druid/indexing/common/task/MinorCompactionInputSpec.java Co-authored-by: Kashif Faraz --- .../druid/indexing/common/task/MinorCompactionInputSpec.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/MinorCompactionInputSpec.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/MinorCompactionInputSpec.java index 18bfa139fe84..0176386bf075 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/MinorCompactionInputSpec.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/MinorCompactionInputSpec.java @@ -34,7 +34,7 @@ /** * Specifies uncompacted segments to compact within an interval. - * Used for minor compaction to compact only uncompacted segments while leaving compacted segments untouched. + * Used for MSQ-based minor compaction to compact only uncompacted segments while upgrading compacted segments (i.e. no change to physical segment files). */ public class MinorCompactionInputSpec implements CompactionInputSpec { From 4e888b75a27e2b3decae18ed6d0c0f4d7df2fdc8 Mon Sep 17 00:00:00 2001 From: Cece Mei Date: Tue, 3 Mar 2026 13:36:19 -0800 Subject: [PATCH 15/26] Update indexing-service/src/main/java/org/apache/druid/indexing/common/actions/MarkSegmentToUpgradeAction.java Co-authored-by: Kashif Faraz --- .../indexing/common/actions/MarkSegmentToUpgradeAction.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/MarkSegmentToUpgradeAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/MarkSegmentToUpgradeAction.java index 3326d710c059..c135698cda28 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/MarkSegmentToUpgradeAction.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/MarkSegmentToUpgradeAction.java @@ -91,7 +91,7 @@ public Integer perform(Task task, TaskActionToolbox toolbox) if (segmentToReplaceLock.size() < upgradeSegments.size()) { throw InvalidInput.exception( - "Not all segments are hold by a replace lock, only [%d] segments out of total segments[%d] are hold by repalce lock", + "Segments to upgrade must be covered by a REPLACE lock. Only [%d] out of [%d] segments are covered.", segmentToReplaceLock.size(), upgradeSegments.size() ); From 2ed63ae9118db532ec19d3fbfd1f6cf0dbb11856 Mon Sep 17 00:00:00 2001 From: cecemei Date: Tue, 3 Mar 2026 13:41:41 -0800 Subject: [PATCH 16/26] set --- .../common/actions/MarkSegmentToUpgradeAction.java | 11 +++++------ .../druid/indexing/common/task/CompactionTask.java | 5 +++-- .../actions/MarkSegmentToUpgradeActionTest.java | 12 ++++++------ 3 files changed, 14 insertions(+), 14 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/MarkSegmentToUpgradeAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/MarkSegmentToUpgradeAction.java index c135698cda28..ed524aa667f4 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/MarkSegmentToUpgradeAction.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/MarkSegmentToUpgradeAction.java @@ -27,7 +27,6 @@ import org.apache.druid.metadata.ReplaceTaskLock; import org.apache.druid.timeline.DataSegment; -import java.util.List; import java.util.Map; import java.util.Set; @@ -46,16 +45,16 @@ public class MarkSegmentToUpgradeAction implements TaskAction { private final String dataSource; - private final List upgradeSegments; + private final Set upgradeSegments; /** * @param dataSource the datasource containing the segments to upgrade - * @param upgradeSegments the list of segments to be recorded as upgraded + * @param upgradeSegments the set of segments to be recorded as upgraded */ @JsonCreator public MarkSegmentToUpgradeAction( @JsonProperty("dataSource") String dataSource, - @JsonProperty("upgradeSegments") List upgradeSegments + @JsonProperty("upgradeSegments") Set upgradeSegments ) { this.dataSource = dataSource; @@ -69,7 +68,7 @@ public String getDataSource() } @JsonProperty - public List getUpgradeSegments() + public Set getUpgradeSegments() { return upgradeSegments; } @@ -87,7 +86,7 @@ public Integer perform(Task task, TaskActionToolbox toolbox) { final String datasource = task.getDataSource(); final Map segmentToReplaceLock - = TaskLocks.findReplaceLocksCoveringSegments(datasource, toolbox.getTaskLockbox(), Set.copyOf(upgradeSegments)); + = TaskLocks.findReplaceLocksCoveringSegments(datasource, toolbox.getTaskLockbox(), upgradeSegments); if (segmentToReplaceLock.size() < upgradeSegments.size()) { throw InvalidInput.exception( diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java index 86f0f7a5a1b5..3087e3dfa94b 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java @@ -598,7 +598,7 @@ static Map createInputDataSchemas( if (granularitySpec == null || granularitySpec.getSegmentGranularity() == null) { Map inputSchemas = new HashMap<>(); // if segment is already compacted in incremental compaction, they need to be upgraded directly, supported in MSQ - List upgradeSegments = new ArrayList<>(); + Set upgradeSegments = new HashSet<>(); // original granularity final Map> intervalToSegments = new TreeMap<>( @@ -669,7 +669,8 @@ static Map createInputDataSchemas( return inputSchemas; } else { // given segment granularity - List upgradeSegments = Lists.newArrayList(Iterables.filter( + Set upgradeSegments = new HashSet<>(); + Iterables.addAll(upgradeSegments, Iterables.filter( timelineSegments, segmentProvider.segmentsToUpgradePredicate )); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/MarkSegmentToUpgradeActionTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/MarkSegmentToUpgradeActionTest.java index 0a7edb653f8f..df936e4dec8d 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/MarkSegmentToUpgradeActionTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/MarkSegmentToUpgradeActionTest.java @@ -33,7 +33,7 @@ import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; -import java.util.List; +import java.util.Set; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertThrows; @@ -86,7 +86,7 @@ public void test_segmentsSuccessfullyInsertedIntoUpgradeTable() throws Exception actionTestKit.getTaskLockbox() .lock(task, new TimeChunkLockRequest(TaskLockType.REPLACE, task, INTERVAL_2026_02, null), 5000); - final MarkSegmentToUpgradeAction action = new MarkSegmentToUpgradeAction(DATA_SOURCE, List.of(SEGMENT1, SEGMENT2, SEGMENT3)); + final MarkSegmentToUpgradeAction action = new MarkSegmentToUpgradeAction(DATA_SOURCE, Set.of(SEGMENT1, SEGMENT2, SEGMENT3)); final Integer insertedCount = action.perform(task, actionTestKit.getTaskActionToolbox()); assertEquals(3, insertedCount.intValue()); @@ -102,7 +102,7 @@ public void test_failsWhenSegmentsNotCoveredByReplaceLock() throws Exception actionTestKit.getTaskLockbox() .lock(task, new TimeChunkLockRequest(TaskLockType.REPLACE, task, INTERVAL_2026_01, null), 5000); - final MarkSegmentToUpgradeAction action = new MarkSegmentToUpgradeAction(DATA_SOURCE, List.of(SEGMENT1, SEGMENT2, SEGMENT3)); + final MarkSegmentToUpgradeAction action = new MarkSegmentToUpgradeAction(DATA_SOURCE, Set.of(SEGMENT1, SEGMENT2, SEGMENT3)); DruidException exception = assertThrows( DruidException.class, @@ -119,7 +119,7 @@ public void test_failsWithExclusiveLockInsteadOfReplaceLock() throws Exception actionTestKit.getTaskLockbox() .lock(task, new TimeChunkLockRequest(TaskLockType.EXCLUSIVE, task, INTERVAL_2026_01, null), 5000); - final MarkSegmentToUpgradeAction action = new MarkSegmentToUpgradeAction(DATA_SOURCE, List.of(SEGMENT1, SEGMENT2)); + final MarkSegmentToUpgradeAction action = new MarkSegmentToUpgradeAction(DATA_SOURCE, Set.of(SEGMENT1, SEGMENT2)); DruidException exception = assertThrows( DruidException.class, @@ -134,7 +134,7 @@ public void test_emptySegmentsList() final Task task = NoopTask.forDatasource(DATA_SOURCE); actionTestKit.getTaskLockbox().add(task); - final MarkSegmentToUpgradeAction action = new MarkSegmentToUpgradeAction(DATA_SOURCE, List.of()); + final MarkSegmentToUpgradeAction action = new MarkSegmentToUpgradeAction(DATA_SOURCE, Set.of()); DruidException exception = assertThrows( DruidException.class, @@ -151,7 +151,7 @@ public void test_singleSegmentUpgrade() throws Exception actionTestKit.getTaskLockbox() .lock(task, new TimeChunkLockRequest(TaskLockType.REPLACE, task, INTERVAL_2026_01, null), 5000); - final MarkSegmentToUpgradeAction action = new MarkSegmentToUpgradeAction(DATA_SOURCE, List.of(SEGMENT1)); + final MarkSegmentToUpgradeAction action = new MarkSegmentToUpgradeAction(DATA_SOURCE, Set.of(SEGMENT1)); final Integer insertedCount = action.perform(task, actionTestKit.getTaskActionToolbox()); assertEquals(1, insertedCount.intValue()); From ed32beb3e242325c76f775b54c1531507f5a6043 Mon Sep 17 00:00:00 2001 From: cecemei Date: Tue, 3 Mar 2026 17:07:04 -0800 Subject: [PATCH 17/26] test --- .../common/actions/MarkSegmentToUpgradeActionTest.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/MarkSegmentToUpgradeActionTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/MarkSegmentToUpgradeActionTest.java index df936e4dec8d..5f298b54388d 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/MarkSegmentToUpgradeActionTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/MarkSegmentToUpgradeActionTest.java @@ -108,7 +108,7 @@ public void test_failsWhenSegmentsNotCoveredByReplaceLock() throws Exception DruidException.class, () -> action.perform(task, actionTestKit.getTaskActionToolbox()) ); - assertTrue(exception.getMessage().contains("Not all segments are hold by a replace lock")); + assertTrue(exception.getMessage().contains("Segments to upgrade must be covered by a REPLACE lock")); } @Test @@ -125,7 +125,7 @@ public void test_failsWithExclusiveLockInsteadOfReplaceLock() throws Exception DruidException.class, () -> action.perform(task, actionTestKit.getTaskActionToolbox()) ); - assertTrue(exception.getMessage().contains("Not all segments are hold by a replace lock")); + assertTrue(exception.getMessage().contains("Segments to upgrade must be covered by a REPLACE lock")); } @Test From aaf5ad4c13fe10be741ddc1afc49d6eab4578670 Mon Sep 17 00:00:00 2001 From: cecemei Date: Wed, 4 Mar 2026 10:49:39 -0800 Subject: [PATCH 18/26] review --- .../compact/CompactionSupervisorTest.java | 6 +- .../indexing/common/task/CompactionTask.java | 92 +++++++++++-------- .../common/task/NativeCompactionRunner.java | 4 + .../compact/CascadingReindexingTemplate.java | 8 +- .../CompactionConfigBasedJobTemplate.java | 15 +-- .../indexing/compact/CompactionJobParams.java | 16 ++++ .../indexing/compact/CompactionJobQueue.java | 1 + .../indexing/compact/CompactionScheduler.java | 5 - .../compact/CompactionSupervisor.java | 3 +- .../compact/CompactionSupervisorSpec.java | 5 +- .../compact/OverlordCompactionScheduler.java | 6 -- .../CascadingReindexingTemplateTest.java | 69 +++++--------- .../partition/DimensionRangeShardSpec.java | 2 +- .../timeline/partition/NumberedShardSpec.java | 2 +- .../druid/timeline/partition/ShardSpec.java | 2 +- .../IndexerSQLMetadataStorageCoordinator.java | 31 +++---- .../compaction/CompactionCandidate.java | 2 +- 17 files changed, 127 insertions(+), 142 deletions(-) diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionSupervisorTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionSupervisorTest.java index 0978c97af8dc..050e88b6255e 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionSupervisorTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionSupervisorTest.java @@ -484,8 +484,7 @@ public void test_cascadingCompactionTemplate_multiplePeriodsApplyDifferentCompac null, null, null, - Granularities.HOUR, - overlord.bindings().getInstance(CompactionStatusTracker.class) + Granularities.HOUR ); runCompactionWithSpec(cascadingReindexingTemplate); waitForAllCompactionTasksToFinish(); @@ -564,8 +563,7 @@ public void test_cascadingReindexing_withVirtualColumnOnNestedData_filtersCorrec null, null, null, - Granularities.DAY, - overlord.bindings().getInstance(CompactionStatusTracker.class) + Granularities.DAY ); runCompactionWithSpec(cascadingTemplate); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java index 3087e3dfa94b..84f604543cc4 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java @@ -27,8 +27,6 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; -import com.google.common.base.Predicate; -import com.google.common.base.Predicates; import com.google.common.collect.BiMap; import com.google.common.collect.HashBiMap; import com.google.common.collect.ImmutableList; @@ -588,17 +586,19 @@ static Map createInputDataSchemas( Iterable segmentsNotCompletelyWithinin = Iterables.filter(timelineSegments, s -> !segmentProvider.interval.contains(s.getInterval())); if (segmentsNotCompletelyWithinin.iterator().hasNext()) { - throw new ISE( - "Incremental compaction doesn't allow segments not completely within interval[%s]", - segmentProvider.interval - ); + throw DruidException.forPersona(DruidException.Persona.USER) + .ofCategory(DruidException.Category.INVALID_INPUT) + .build( + "Incremental compaction doesn't allow segments not completely within interval[%s]", + segmentProvider.interval + ); } } if (granularitySpec == null || granularitySpec.getSegmentGranularity() == null) { Map inputSchemas = new HashMap<>(); // if segment is already compacted in incremental compaction, they need to be upgraded directly, supported in MSQ - Set upgradeSegments = new HashSet<>(); + Set segmentsToUpgrade = new HashSet<>(); // original granularity final Map> intervalToSegments = new TreeMap<>( @@ -606,15 +606,17 @@ static Map createInputDataSchemas( ); for (final DataSegment dataSegment : timelineSegments) { - if (segmentProvider.segmentsToUpgradePredicate.test(dataSegment)) { - upgradeSegments.add(dataSegment); + if (segmentProvider.shouldUpgradeSegment(dataSegment)) { + segmentsToUpgrade.add(dataSegment); } else { intervalToSegments.computeIfAbsent(dataSegment.getInterval(), k -> new ArrayList<>()) .add(dataSegment); } } - if (!upgradeSegments.isEmpty()) { - toolbox.getTaskActionClient().submit(new MarkSegmentToUpgradeAction(segmentProvider.dataSource, upgradeSegments)); + if (!segmentsToUpgrade.isEmpty()) { + log.info("Marking [%d]segments to upgrade", segmentsToUpgrade.size()); + toolbox.getTaskActionClient() + .submit(new MarkSegmentToUpgradeAction(segmentProvider.dataSource, segmentsToUpgrade)); } // unify overlapping intervals to ensure overlapping segments compacting in the same indexSpec @@ -659,28 +661,33 @@ static Map createInputDataSchemas( projections, needMultiValuedColumns ); - inputSchemas.put( - segmentProvider.minorCompaction - ? new MultipleSpecificSegmentSpec(segmentsToCompact.stream() - .map(DataSegment::toDescriptor) - .collect(Collectors.toList())) - : new MultipleIntervalSegmentSpec(List.of(interval)), dataSchema); + final QuerySegmentSpec querySegmentSpec; + if (segmentProvider.minorCompaction) { + querySegmentSpec = new MultipleSpecificSegmentSpec(segmentsToCompact.stream() + .map(DataSegment::toDescriptor) + .collect(Collectors.toList())); + } else { + querySegmentSpec = new MultipleIntervalSegmentSpec(List.of(interval)); + } + inputSchemas.put(querySegmentSpec, dataSchema); } return inputSchemas; } else { // given segment granularity - Set upgradeSegments = new HashSet<>(); - Iterables.addAll(upgradeSegments, Iterables.filter( + Set segmentsToUpgrade = new HashSet<>(); + Iterables.addAll(segmentsToUpgrade, Iterables.filter( timelineSegments, - segmentProvider.segmentsToUpgradePredicate + segmentProvider::shouldUpgradeSegment )); - if (!upgradeSegments.isEmpty()) { - toolbox.getTaskActionClient().submit(new MarkSegmentToUpgradeAction(segmentProvider.dataSource, upgradeSegments)); + if (!segmentsToUpgrade.isEmpty()) { + log.info("Marking [%d]segments to upgrade", segmentsToUpgrade.size()); + toolbox.getTaskActionClient() + .submit(new MarkSegmentToUpgradeAction(segmentProvider.dataSource, segmentsToUpgrade)); } final Iterable segmentsToCompact = Iterables.filter( timelineSegments, - segmentProvider.segmentsToCompactPredicate + segmentProvider::shouldCompactSegment ); final DataSchema dataSchema = createDataSchema( toolbox.getEmitter(), @@ -698,11 +705,15 @@ static Map createInputDataSchemas( projections, needMultiValuedColumns ); - return Map.of(segmentProvider.minorCompaction - ? new MultipleSpecificSegmentSpec(StreamSupport.stream(segmentsToCompact.spliterator(), false) - .map(DataSegment::toDescriptor) - .collect(Collectors.toList())) - : new MultipleIntervalSegmentSpec(List.of(segmentProvider.interval)), dataSchema); + final QuerySegmentSpec querySegmentSpec; + if (segmentProvider.minorCompaction) { + querySegmentSpec = new MultipleSpecificSegmentSpec(StreamSupport.stream(segmentsToCompact.spliterator(), false) + .map(DataSegment::toDescriptor) + .collect(Collectors.toList())); + } else { + querySegmentSpec = new MultipleIntervalSegmentSpec(List.of(segmentProvider.interval)); + } + return Map.of(querySegmentSpec, dataSchema); } } @@ -1276,8 +1287,7 @@ static class SegmentProvider private final Interval interval; private final boolean minorCompaction; - private final Predicate segmentsToUpgradePredicate; - private final Predicate segmentsToCompactPredicate; + private final Set uncompactedSegments; SegmentProvider(String dataSource, CompactionInputSpec inputSpec) { @@ -1286,17 +1296,27 @@ static class SegmentProvider this.interval = inputSpec.findInterval(dataSource); if (inputSpec instanceof MinorCompactionInputSpec) { minorCompaction = true; - Set uncompactedSegments = Set.copyOf(((MinorCompactionInputSpec) inputSpec).getUncompactedSegments()); - this.segmentsToUpgradePredicate = s -> !uncompactedSegments.contains(s.toDescriptor()) - && this.interval.contains(s.getInterval()); - this.segmentsToCompactPredicate = Predicates.not(this.segmentsToUpgradePredicate); + uncompactedSegments = Set.copyOf(((MinorCompactionInputSpec) inputSpec).getUncompactedSegments()); } else { minorCompaction = false; - this.segmentsToUpgradePredicate = Predicates.alwaysFalse(); - this.segmentsToCompactPredicate = Predicates.alwaysTrue(); + uncompactedSegments = null; } } + private boolean shouldUpgradeSegment(DataSegment s) + { + if (minorCompaction) { + return false; + } else { + return !uncompactedSegments.contains(s.toDescriptor()) && this.interval.contains(s.getInterval()); + } + } + + private boolean shouldCompactSegment(DataSegment s) + { + return !shouldUpgradeSegment(s); + } + List findSegments(TaskActionClient actionClient) throws IOException { return new ArrayList<>( diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/NativeCompactionRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/NativeCompactionRunner.java index b171a977f2df..afde45cdb136 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/NativeCompactionRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/NativeCompactionRunner.java @@ -99,6 +99,10 @@ public CompactionConfigValidationResult validateCompactionTask( "Virtual columns in filter rules are not supported by the Native compaction engine. Use MSQ compaction engine instead." ); } + if (compactionTask.getIoConfig().getInputSpec() instanceof MinorCompactionInputSpec) { + return CompactionConfigValidationResult.failure( + "Minor compaction is not supported by Native compaction engine. Use MSQ compaction engine instead."); + } return CompactionConfigValidationResult.success(); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CascadingReindexingTemplate.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CascadingReindexingTemplate.java index eab651e2b7be..1e5cadd17067 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CascadingReindexingTemplate.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CascadingReindexingTemplate.java @@ -33,7 +33,6 @@ import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.segment.transform.CompactionTransformSpec; -import org.apache.druid.server.compaction.CompactionStatusTracker; import org.apache.druid.server.compaction.IntervalGranularityInfo; import org.apache.druid.server.compaction.ReindexingRule; import org.apache.druid.server.compaction.ReindexingRuleProvider; @@ -101,7 +100,6 @@ public class CascadingReindexingTemplate implements CompactionJobTemplate, DataS private final Period skipOffsetFromLatest; private final Period skipOffsetFromNow; private final Granularity defaultSegmentGranularity; - private final CompactionStatusTracker statusTracker; @JsonCreator public CascadingReindexingTemplate( @@ -113,8 +111,7 @@ public CascadingReindexingTemplate( @JsonProperty("taskContext") @Nullable Map taskContext, @JsonProperty("skipOffsetFromLatest") @Nullable Period skipOffsetFromLatest, @JsonProperty("skipOffsetFromNow") @Nullable Period skipOffsetFromNow, - @JsonProperty("defaultSegmentGranularity") Granularity defaultSegmentGranularity, - @JacksonInject CompactionStatusTracker statusTracker + @JsonProperty("defaultSegmentGranularity") Granularity defaultSegmentGranularity ) { InvalidInput.conditionalException(dataSource != null, "'dataSource' cannot be null"); @@ -136,7 +133,6 @@ public CascadingReindexingTemplate( } this.skipOffsetFromNow = skipOffsetFromNow; this.skipOffsetFromLatest = skipOffsetFromLatest; - this.statusTracker = statusTracker; } @Override @@ -315,7 +311,7 @@ protected CompactionJobTemplate createJobTemplateForInterval( InlineSchemaDataSourceCompactionConfig config ) { - return new CompactionConfigBasedJobTemplate(config, statusTracker, DELETION_RULE_OPTIMIZER); + return new CompactionConfigBasedJobTemplate(config, DELETION_RULE_OPTIMIZER); } /** diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionConfigBasedJobTemplate.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionConfigBasedJobTemplate.java index e0218b1928e5..709e2198d088 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionConfigBasedJobTemplate.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionConfigBasedJobTemplate.java @@ -28,7 +28,6 @@ import org.apache.druid.server.compaction.CompactionCandidate; import org.apache.druid.server.compaction.CompactionCandidateSearchPolicy; import org.apache.druid.server.compaction.CompactionSlotManager; -import org.apache.druid.server.compaction.CompactionStatusTracker; import org.apache.druid.server.compaction.DataSourceCompactibleSegmentIterator; import org.apache.druid.server.compaction.NewestSegmentFirstPolicy; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; @@ -50,22 +49,16 @@ public class CompactionConfigBasedJobTemplate implements CompactionJobTemplate { private final DataSourceCompactionConfig config; - private final CompactionStatusTracker statusTracker; private final ReindexingConfigOptimizer configOptimizer; - public CompactionConfigBasedJobTemplate(DataSourceCompactionConfig config, CompactionStatusTracker statusTracker) + public CompactionConfigBasedJobTemplate(DataSourceCompactionConfig config) { - this(config, statusTracker, ReindexingConfigOptimizer.IDENTITY); + this(config, ReindexingConfigOptimizer.IDENTITY); } - public CompactionConfigBasedJobTemplate( - DataSourceCompactionConfig config, - CompactionStatusTracker statusTracker, - ReindexingConfigOptimizer configOptimizer - ) + public CompactionConfigBasedJobTemplate(DataSourceCompactionConfig config, ReindexingConfigOptimizer configOptimizer) { this.config = config; - this.statusTracker = statusTracker; this.configOptimizer = configOptimizer; } @@ -99,7 +92,7 @@ public List createCompactionJobs( final CompactionCandidateSearchPolicy.Eligibility eligibility = params.getClusterCompactionConfig() .getCompactionPolicy() - .checkEligibilityForCompaction(candidate, statusTracker.getLatestTaskStatus(candidate)); + .checkEligibilityForCompaction(candidate, params.getLatestTaskStatus(candidate)); if (!eligibility.isEligible()) { continue; } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionJobParams.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionJobParams.java index 36cd075922d5..b14e8734f2e5 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionJobParams.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionJobParams.java @@ -20,7 +20,9 @@ package org.apache.druid.indexing.compact; import org.apache.druid.segment.metadata.IndexingStateFingerprintMapper; +import org.apache.druid.server.compaction.CompactionCandidate; import org.apache.druid.server.compaction.CompactionSnapshotBuilder; +import org.apache.druid.server.compaction.CompactionTaskStatus; import org.apache.druid.server.coordinator.ClusterCompactionConfig; import org.apache.druid.timeline.SegmentTimeline; import org.joda.time.DateTime; @@ -32,6 +34,7 @@ public class CompactionJobParams { private final DateTime scheduleStartTime; private final TimelineProvider timelineProvider; + private final CompactionTaskStatusProvider compactionTaskStatusProvider; private final ClusterCompactionConfig clusterCompactionConfig; private final CompactionSnapshotBuilder snapshotBuilder; private final IndexingStateFingerprintMapper fingerprintMapper; @@ -40,6 +43,7 @@ public CompactionJobParams( DateTime scheduleStartTime, ClusterCompactionConfig clusterCompactionConfig, TimelineProvider timelineProvider, + CompactionTaskStatusProvider compactionTaskStatusProvider, CompactionSnapshotBuilder snapshotBuilder, IndexingStateFingerprintMapper indexingStateFingerprintMapper ) @@ -47,6 +51,7 @@ public CompactionJobParams( this.scheduleStartTime = scheduleStartTime; this.clusterCompactionConfig = clusterCompactionConfig; this.timelineProvider = timelineProvider; + this.compactionTaskStatusProvider = compactionTaskStatusProvider; this.snapshotBuilder = snapshotBuilder; this.fingerprintMapper = indexingStateFingerprintMapper; } @@ -79,6 +84,11 @@ public SegmentTimeline getTimeline(String dataSource) return timelineProvider.getTimelineForDataSource(dataSource); } + public CompactionTaskStatus getLatestTaskStatus(CompactionCandidate candidate) + { + return compactionTaskStatusProvider.getLatestTaskStatus(candidate); + } + /** * Used to build an {@link org.apache.druid.server.coordinator.AutoCompactionSnapshot} * for all the datasources at the end of the current run. During the run, as @@ -102,4 +112,10 @@ public interface TimelineProvider { SegmentTimeline getTimelineForDataSource(String dataSource); } + + @FunctionalInterface + public interface CompactionTaskStatusProvider + { + CompactionTaskStatus getLatestTaskStatus(CompactionCandidate candidate); + } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionJobQueue.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionJobQueue.java index 7ec45dbb6943..1dc29cfde6a8 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionJobQueue.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionJobQueue.java @@ -130,6 +130,7 @@ public CompactionJobQueue( DateTimes.nowUtc(), clusterCompactionConfig, dataSourcesSnapshot.getUsedSegmentsTimelinesPerDataSource()::get, + statusTracker::getLatestTaskStatus, snapshotBuilder, new DefaultIndexingStateFingerprintMapper(indexingStateCache, objectMapper) ); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionScheduler.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionScheduler.java index c19771570139..706f7c73a5b3 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionScheduler.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionScheduler.java @@ -61,11 +61,6 @@ public interface CompactionScheduler */ boolean isRunning(); - /** - * @return the tracker for monitoring compaction status across datasources. - */ - CompactionStatusTracker getCompactionStatusTracker(); - CompactionConfigValidationResult validateCompactionConfig(DataSourceCompactionConfig compactionConfig); /** diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionSupervisor.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionSupervisor.java index 636eeedc7bc6..f9ba0eee6db7 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionSupervisor.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionSupervisor.java @@ -75,8 +75,7 @@ public List createJobs( CompactionJobParams jobParams ) { - return supervisorSpec.getTemplate(scheduler.getCompactionStatusTracker()) - .createCompactionJobs(inputSource, jobParams); + return supervisorSpec.getTemplate().createCompactionJobs(inputSource, jobParams); } @Override diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionSupervisorSpec.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionSupervisorSpec.java index d295f68d85c3..d9002655f742 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionSupervisorSpec.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionSupervisorSpec.java @@ -24,7 +24,6 @@ import com.fasterxml.jackson.annotation.JsonProperty; import org.apache.druid.common.config.Configs; import org.apache.druid.indexing.overlord.supervisor.SupervisorSpec; -import org.apache.druid.server.compaction.CompactionStatusTracker; import org.apache.druid.server.coordinator.CompactionConfigValidationResult; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; import org.apache.druid.server.security.ResourceAction; @@ -97,12 +96,12 @@ public CompactionSupervisor createSupervisor() /** * @return {@link CompactionJobTemplate} used to create jobs for the supervisor. */ - public CompactionJobTemplate getTemplate(CompactionStatusTracker statusTracker) + public CompactionJobTemplate getTemplate() { if (spec instanceof CascadingReindexingTemplate) { return (CascadingReindexingTemplate) spec; } else { - return new CompactionConfigBasedJobTemplate(spec, statusTracker); + return new CompactionConfigBasedJobTemplate(spec); } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/OverlordCompactionScheduler.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/OverlordCompactionScheduler.java index 436790414576..11709e616c71 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/compact/OverlordCompactionScheduler.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/OverlordCompactionScheduler.java @@ -259,12 +259,6 @@ public boolean isRunning() return started.get(); } - @Override - public CompactionStatusTracker getCompactionStatusTracker() - { - return statusTracker; - } - @Override public CompactionConfigValidationResult validateCompactionConfig(DataSourceCompactionConfig compactionConfig) { diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/compact/CascadingReindexingTemplateTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/compact/CascadingReindexingTemplateTest.java index 4019ad86bc48..01b361004445 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/compact/CascadingReindexingTemplateTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/compact/CascadingReindexingTemplateTest.java @@ -19,7 +19,6 @@ package org.apache.druid.indexing.compact; -import com.fasterxml.jackson.databind.InjectableValues; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableMap; import org.apache.druid.error.DruidException; @@ -31,7 +30,6 @@ import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.query.aggregation.AggregatorFactory; -import org.apache.druid.server.compaction.CompactionStatusTracker; import org.apache.druid.server.compaction.InlineReindexingRuleProvider; import org.apache.druid.server.compaction.IntervalGranularityInfo; import org.apache.druid.server.compaction.ReindexingDataSchemaRule; @@ -65,7 +63,6 @@ public class CascadingReindexingTemplateTest extends InitializedNullHandlingTest public void setUp() { OBJECT_MAPPER.registerModules(new SupervisorModule().getJacksonModules()); - OBJECT_MAPPER.setInjectableValues(new InjectableValues.Std().addValue(CompactionStatusTracker.class, null)); } @Test @@ -95,8 +92,7 @@ public void test_serde() throws Exception ImmutableMap.of("context_key", "context_value"), null, null, - Granularities.DAY, - null + Granularities.DAY ); final String json = OBJECT_MAPPER.writeValueAsString(template); @@ -131,8 +127,7 @@ public void test_serde_asDataSourceCompactionConfig() throws Exception ImmutableMap.of("key", "value"), null, null, - Granularities.HOUR, - null + Granularities.HOUR ); // Serialize and deserialize as DataSourceCompactionConfig interface @@ -167,8 +162,7 @@ public void test_createCompactionJobs_ruleProviderNotReady() null, null, null, - Granularities.DAY, - null + Granularities.DAY ); // Call createCompactionJobs - should return empty list without processing @@ -195,8 +189,7 @@ public void test_constructor_setBothSkipOffsetStrategiesThrowsException() null, Period.days(7), // skipOffsetFromLatest Period.days(3), // skipOffsetFromNow - Granularities.DAY, - null + Granularities.DAY ) ); @@ -221,8 +214,7 @@ public void test_constructor_nullDataSourceThrowsException() null, null, null, - Granularities.DAY, - null + Granularities.DAY ) ); @@ -244,8 +236,7 @@ public void test_constructor_nullRuleProviderThrowsException() null, null, null, - Granularities.DAY, - null + Granularities.DAY ) ); @@ -269,8 +260,7 @@ public void test_constructor_nullDefaultSegmentGranularityThrowsException() null, null, null, - null, // null defaultSegmentGranularity, - null + null // null defaultSegmentGranularity ) ); @@ -492,8 +482,7 @@ public void test_generateAlignedSearchIntervals_withGranularityAlignment() null, null, null, - Granularities.DAY, - null + Granularities.DAY ); List expected = List.of( @@ -584,8 +573,7 @@ public void test_generateAlignedSearchIntervals_withNonSegmentGranularityRuleSpl null, null, null, - Granularities.DAY, - null + Granularities.DAY ); List expected = List.of( @@ -685,8 +673,7 @@ public void test_generateAlignedSearchIntervals_withNoSegmentGranularityRules() null, null, null, - Granularities.DAY, - null + Granularities.DAY ); // When no segment granularity rules exist, a synthetic rule is created with the smallest period @@ -778,8 +765,7 @@ public void test_generateAlignedSearchIntervals_prependIntervalForShortNonSegmen null, null, null, - Granularities.HOUR, - null + Granularities.HOUR ); List expected = List.of( @@ -885,8 +871,7 @@ public void test_generateAlignedSearchIntervals() null, null, null, - Granularities.HOUR, - null + Granularities.HOUR ); List expected = List.of( @@ -956,8 +941,7 @@ public void test_generateAlignedSearchIntervals_noRulesThrowsException() null, null, null, - Granularities.DAY, - null + Granularities.DAY ); DruidException exception = Assertions.assertThrows( @@ -1022,8 +1006,7 @@ public void test_generateAlignedSearchIntervals_splitPointSnapsToExistingBoundar null, null, null, - Granularities.DAY, - null + Granularities.DAY ); List expected = List.of( @@ -1090,8 +1073,7 @@ public void test_generateAlignedSearchIntervals_prependAlignmentDoesNotExtendTim null, null, null, - Granularities.DAY, - null + Granularities.DAY ); List expected = List.of( @@ -1162,8 +1144,7 @@ public void test_generateAlignedSearchIntervals_duplicateSplitPointsFiltered() null, null, null, - Granularities.DAY, - null + Granularities.DAY ); List expected = List.of( @@ -1227,8 +1208,7 @@ public void test_generateAlignedSearchIntervals_singleRuleOnly() null, null, null, - Granularities.DAY, - null + Granularities.DAY ); List expected = List.of( @@ -1292,8 +1272,7 @@ public void test_generateAlignedSearchIntervals_zeroPeriodRuleAppliesImmediately null, null, null, - Granularities.DAY, - null + Granularities.DAY ); List expected = List.of( @@ -1378,8 +1357,7 @@ public void test_generateAlignedSearchIntervals_zeroPeriodRuleWithOtherRules() null, null, null, - Granularities.DAY, - null + Granularities.DAY ); List expected = List.of( @@ -1452,8 +1430,7 @@ public void test_generateAlignedSearchIntervals_failsWhenDefaultGranularityIsCoa null, null, null, - Granularities.MONTH, // MONTH is coarser than HOUR! - null + Granularities.MONTH // MONTH is coarser than HOUR! ); IllegalArgumentException exception = Assertions.assertThrows( @@ -1514,8 +1491,7 @@ public void test_generateAlignedSearchIntervals_failsWhenOlderRuleHasFinerGranul null, null, null, - Granularities.DAY, - null + Granularities.DAY ); IllegalArgumentException exception = Assertions.assertThrows( @@ -1548,8 +1524,7 @@ public TestCascadingReindexingTemplate( ) { super(dataSource, taskPriority, inputSegmentSizeBytes, ruleProvider, - engine, taskContext, skipOffsetFromLatest, skipOffsetFromNow, Granularities.DAY, null - ); + engine, taskContext, skipOffsetFromLatest, skipOffsetFromNow, Granularities.DAY); } public List getProcessedIntervals() diff --git a/processing/src/main/java/org/apache/druid/timeline/partition/DimensionRangeShardSpec.java b/processing/src/main/java/org/apache/druid/timeline/partition/DimensionRangeShardSpec.java index 381b63fe791a..a0a850dae08a 100644 --- a/processing/src/main/java/org/apache/druid/timeline/partition/DimensionRangeShardSpec.java +++ b/processing/src/main/java/org/apache/druid/timeline/partition/DimensionRangeShardSpec.java @@ -255,7 +255,7 @@ public boolean possibleInDomain(Map> domain) } @Override - public boolean isNumChunkSupported() + public boolean canCreateNumberedPartitionChunk() { return !isNumCorePartitionsUnknown(); } diff --git a/processing/src/main/java/org/apache/druid/timeline/partition/NumberedShardSpec.java b/processing/src/main/java/org/apache/druid/timeline/partition/NumberedShardSpec.java index db9a501e0432..4ff607b0965e 100644 --- a/processing/src/main/java/org/apache/druid/timeline/partition/NumberedShardSpec.java +++ b/processing/src/main/java/org/apache/druid/timeline/partition/NumberedShardSpec.java @@ -113,7 +113,7 @@ public ShardSpec withCorePartitions(int partitions) } @Override - public boolean isNumChunkSupported() + public boolean canCreateNumberedPartitionChunk() { return true; } diff --git a/processing/src/main/java/org/apache/druid/timeline/partition/ShardSpec.java b/processing/src/main/java/org/apache/druid/timeline/partition/ShardSpec.java index b7e97e232e52..b91c96387a9d 100644 --- a/processing/src/main/java/org/apache/druid/timeline/partition/ShardSpec.java +++ b/processing/src/main/java/org/apache/druid/timeline/partition/ShardSpec.java @@ -61,7 +61,7 @@ public interface ShardSpec * Returns whether {@link #createChunk} returns a {@link NumberedPartitionChunk} instance. * This is necessary for supporting {@link PartitionHolder#isComplete()} if updating to a new corePartitions spec. */ - default boolean isNumChunkSupported() + default boolean canCreateNumberedPartitionChunk() { return false; } 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 45c7a5999116..fe4b4af34ecd 100644 --- a/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java +++ b/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java @@ -1252,7 +1252,7 @@ private SegmentPublishResult commitAppendSegmentsAndMetadataInTransaction( } } - insertIntoUpgradeSegmentsTableDoWork(transaction, appendSegmentToReplaceLock); + insertIntoUpgradeSegmentsTable(transaction, appendSegmentToReplaceLock); // Delete the pending segments to be committed in this transaction in batches of at most 100 int numDeletedPendingSegments = transaction.deletePendingSegments( @@ -1859,7 +1859,7 @@ protected Set insertSegments( * versions, and partition numbers. Combines upgraded segments with replace segments and updates shard * specs with correct core partition counts. * - * @return pair of (upgraded segments for metadata tracking, segments to insert into segment table) + * @return segments to insert into segment table * @throws DruidException if a replace interval partially overlaps a segment being upgraded */ private Set createNewSegmentsAfterReplace( @@ -1885,7 +1885,7 @@ private Set createNewSegmentsAfterReplace( segment.getInterval(), (i, value) -> value == null ? partitionNum : Math.max(value, partitionNum) ); - if (!segment.isTombstone() && !segment.getShardSpec().isNumChunkSupported()) { + if (!segment.isTombstone() && !segment.getShardSpec().canCreateNumberedPartitionChunk()) { numChunkNotSupported.add(segment.getInterval()); } } @@ -1937,7 +1937,7 @@ private Set createNewSegmentsAfterReplace( // but a (revoked) REPLACE lock covers this segment newInterval = oldInterval; } - if (!oldSegment.getShardSpec().isNumChunkSupported()) { + if (!oldSegment.getShardSpec().canCreateNumberedPartitionChunk()) { numChunkNotSupported.add(newInterval); } @@ -1979,10 +1979,7 @@ private Set createNewSegmentsAfterReplace( return segmentsToInsert.stream().map(segment -> { // update corePartitions in shard spec Integer partitionNum = intervalToCurrentPartitionNum.get(segment.getInterval()); - if (!segment.isTombstone() - && !numChunkNotSupported.contains(segment.getInterval()) - && partitionNum != null - && partitionNum + 1 != segment.getShardSpec().getNumCorePartitions()) { + if (!segment.isTombstone() && !numChunkNotSupported.contains(segment.getInterval()) && partitionNum != null) { return segment.withShardSpec(segment.getShardSpec().withCorePartitions(partitionNum + 1)); } else { return segment; @@ -2147,7 +2144,7 @@ public int insertIntoUpgradeSegmentsTable(Map segm final String dataSource = verifySegmentsToCommit(segmentToReplaceLock.keySet()); return inReadWriteDatasourceTransaction( dataSource, - transaction -> insertIntoUpgradeSegmentsTableDoWork(transaction, segmentToReplaceLock) + transaction -> insertIntoUpgradeSegmentsTable(transaction, segmentToReplaceLock) ); } @@ -2155,7 +2152,7 @@ public int insertIntoUpgradeSegmentsTable(Map segm * Inserts entries into the upgrade_segments table in batches of size * {@link #MAX_NUM_SEGMENTS_TO_ANNOUNCE_AT_ONCE}. */ - private int insertIntoUpgradeSegmentsTableDoWork( + private int insertIntoUpgradeSegmentsTable( SegmentMetadataTransaction transaction, Map segmentToReplaceLock ) @@ -2243,14 +2240,12 @@ private Map getSegmentsCoveredByTaskLock( dbTables.getUpgradeSegmentsTable() ); - ResultIterator> resultIterator = transaction.getHandle() - .createQuery(sql) - .bind("task_id", taskId) - .map( - (index, r, ctx) -> Pair.of(r.getString( - "segment_id"), r.getString("lock_version")) - ) - .iterator(); + ResultIterator> resultIterator = + transaction.getHandle() + .createQuery(sql) + .bind("task_id", taskId) + .map((index, r, ctx) -> Pair.of(r.getString("segment_id"), r.getString("lock_version"))) + .iterator(); final Map segmentIdToLockVersion = new HashMap<>(); while (resultIterator.hasNext()) { diff --git a/server/src/main/java/org/apache/druid/server/compaction/CompactionCandidate.java b/server/src/main/java/org/apache/druid/server/compaction/CompactionCandidate.java index 62f36353bac8..d3653ccd992a 100644 --- a/server/src/main/java/org/apache/druid/server/compaction/CompactionCandidate.java +++ b/server/src/main/java/org/apache/druid/server/compaction/CompactionCandidate.java @@ -170,7 +170,7 @@ public CompactionStatistics getUncompactedStats() @Nullable public List getUncompactedSegments() { - return currentStatus == null ? null : currentStatus.getUncompactedSegments(); + return currentStatus.getUncompactedSegments(); } /** From cd49144f2679280d617044cb4bbe64c205f0d752 Mon Sep 17 00:00:00 2001 From: cecemei Date: Wed, 4 Mar 2026 16:04:38 -0800 Subject: [PATCH 19/26] minor --- .../embedded/compact/CompactionSupervisorTest.java | 3 +-- .../apache/druid/indexing/common/task/CompactionTask.java | 6 +++--- .../indexing/compact/CascadingReindexingTemplate.java | 1 - .../druid/indexing/compact/CompactionScheduler.java | 1 - .../apache/druid/msq/exec/MSQCompactionTaskRunTest.java | 8 ++++---- .../compaction/MostFragmentedIntervalFirstPolicy.java | 2 +- .../client/indexing/ClientCompactionIntervalSpecTest.java | 2 +- 7 files changed, 10 insertions(+), 13 deletions(-) diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionSupervisorTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionSupervisorTest.java index 050e88b6255e..ff78d2c1ffce 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionSupervisorTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionSupervisorTest.java @@ -62,7 +62,6 @@ import org.apache.druid.segment.transform.CompactionTransformSpec; import org.apache.druid.segment.virtual.ExpressionVirtualColumn; import org.apache.druid.server.compaction.CompactionCandidateSearchPolicy; -import org.apache.druid.server.compaction.CompactionStatusTracker; import org.apache.druid.server.compaction.InlineReindexingRuleProvider; import org.apache.druid.server.compaction.MostFragmentedIntervalFirstPolicy; import org.apache.druid.server.compaction.ReindexingDeletionRule; @@ -311,7 +310,7 @@ public void test_minorCompactionWithMSQ(PartitionsSpec partitionsSpec) throws Ex .hasDimension(DruidMetrics.DATASOURCE, dataSource) .hasValueMatching(Matchers.greaterThan(totalUsed))); - // performed incremental compaction: 1 previously compacted segment + 1 incrementally compacted segment + // performed minor compaction: 1 previously compacted segment + 1 incrementally compacted segment Assertions.assertEquals(2, getNumSegmentsWith(Granularities.DAY)); Assertions.assertEquals("3000", cluster.runSql("SELECT COUNT(*) FROM %s", dataSource)); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java index 84f604543cc4..15d73c9f658b 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java @@ -552,7 +552,7 @@ public TaskStatus runTask(TaskToolbox toolbox) throws Exception * Creates input data schemas for compaction by grouping segments and generating {@link DataSchema}s. * When segment granularity is not specified, preserves original granularity and creates a schema * for each unified interval. When segment granularity is specified, creates a single schema for all - * segments. For incremental compaction, validates that all segments are completely within the target + * segments. For minor compaction, validates that all segments are completely within the target * interval and submits already-compacted segments via {@link MarkSegmentToUpgradeAction} for direct upgrade. * * @return map from {@link QuerySegmentSpec} to {@link DataSchema} for each group of segments to compact @@ -589,7 +589,7 @@ static Map createInputDataSchemas( throw DruidException.forPersona(DruidException.Persona.USER) .ofCategory(DruidException.Category.INVALID_INPUT) .build( - "Incremental compaction doesn't allow segments not completely within interval[%s]", + "Minor compaction doesn't allow segments not completely within interval[%s]", segmentProvider.interval ); } @@ -597,7 +597,7 @@ static Map createInputDataSchemas( if (granularitySpec == null || granularitySpec.getSegmentGranularity() == null) { Map inputSchemas = new HashMap<>(); - // if segment is already compacted in incremental compaction, they need to be upgraded directly, supported in MSQ + // if segment is already compacted in minor compaction, they need to be upgraded directly, supported in MSQ Set segmentsToUpgrade = new HashSet<>(); // original granularity diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CascadingReindexingTemplate.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CascadingReindexingTemplate.java index 1e5cadd17067..16307fa15399 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CascadingReindexingTemplate.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CascadingReindexingTemplate.java @@ -19,7 +19,6 @@ package org.apache.druid.indexing.compact; -import com.fasterxml.jackson.annotation.JacksonInject; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.annotations.VisibleForTesting; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionScheduler.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionScheduler.java index 706f7c73a5b3..6f5ed1a7a6ef 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionScheduler.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionScheduler.java @@ -20,7 +20,6 @@ package org.apache.druid.indexing.compact; import org.apache.druid.server.compaction.CompactionSimulateResult; -import org.apache.druid.server.compaction.CompactionStatusTracker; import org.apache.druid.server.coordinator.AutoCompactionSnapshot; import org.apache.druid.server.coordinator.ClusterCompactionConfig; import org.apache.druid.server.coordinator.CompactionConfigValidationResult; diff --git a/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQCompactionTaskRunTest.java b/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQCompactionTaskRunTest.java index bcea86028fcf..72a69fe6cad8 100644 --- a/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQCompactionTaskRunTest.java +++ b/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQCompactionTaskRunTest.java @@ -582,7 +582,7 @@ public void testMSQCompactionWithConcurrentAppendAppendLocksFirst() throws Excep public void testMinorCompaction() throws Exception { Assume.assumeTrue(lockGranularity == LockGranularity.TIME_CHUNK); - Assume.assumeTrue("Incremental compaction depends on concurrent lock", useConcurrentLocks); + Assume.assumeTrue("Minor compaction depends on concurrent lock", useConcurrentLocks); verifyTaskSuccessRowsAndSchemaMatch(runIndexTask(), TOTAL_TEST_ROWS); final CompactionTask compactionTask1 = @@ -643,7 +643,7 @@ public void testMinorCompactionRangePartition() throws Exception "2014-01-01T02:00:30Z,c,3\n" ); Assume.assumeTrue(lockGranularity == LockGranularity.TIME_CHUNK); - Assume.assumeTrue("Incremental compaction depends on concurrent lock", useConcurrentLocks); + Assume.assumeTrue("Minor compaction depends on concurrent lock", useConcurrentLocks); verifyTaskSuccessRowsAndSchemaMatch( runTask(buildIndexTask(DEFAULT_PARSE_SPEC, rows, inputInterval, false)), 9 @@ -690,7 +690,7 @@ public void testMinorCompactionRangePartition() throws Exception public void testMinorCompactionOverlappingInterval() throws Exception { Assume.assumeTrue(lockGranularity == LockGranularity.TIME_CHUNK); - Assume.assumeTrue("Incremental compaction depends on concurrent lock", useConcurrentLocks); + Assume.assumeTrue("Minor compaction depends on concurrent lock", useConcurrentLocks); List rows = new ArrayList<>(); rows.add("2014-01-01T00:00:10Z,a1,11\n"); @@ -730,7 +730,7 @@ public void testMinorCompactionOverlappingInterval() throws Exception .build(); ISE e = Assert.assertThrows(ISE.class, () -> runTask(compactionTask1)); Assert.assertEquals( - "Incremental compaction doesn't allow segments not completely within interval[2014-01-01T00:00:00.000Z/2014-01-01T08:00:00.000Z]", + "Minor compaction doesn't allow segments not completely within interval[2014-01-01T00:00:00.000Z/2014-01-01T08:00:00.000Z]", e.getMessage() ); } diff --git a/server/src/main/java/org/apache/druid/server/compaction/MostFragmentedIntervalFirstPolicy.java b/server/src/main/java/org/apache/druid/server/compaction/MostFragmentedIntervalFirstPolicy.java index 840887ca1c18..90d3ce40471e 100644 --- a/server/src/main/java/org/apache/druid/server/compaction/MostFragmentedIntervalFirstPolicy.java +++ b/server/src/main/java/org/apache/druid/server/compaction/MostFragmentedIntervalFirstPolicy.java @@ -120,7 +120,7 @@ public HumanReadableBytes getMaxAverageUncompactedBytesPerSegment() /** * Threshold percentage of uncompacted bytes to total bytes below which - * incremental compaction is eligible instead of full compaction. + * minor compaction is eligible instead of full compaction. * Default value is 0. */ @JsonProperty diff --git a/server/src/test/java/org/apache/druid/client/indexing/ClientCompactionIntervalSpecTest.java b/server/src/test/java/org/apache/druid/client/indexing/ClientCompactionIntervalSpecTest.java index 51e4ee2acd2d..0ab3e0593612 100644 --- a/server/src/test/java/org/apache/druid/client/indexing/ClientCompactionIntervalSpecTest.java +++ b/server/src/test/java/org/apache/druid/client/indexing/ClientCompactionIntervalSpecTest.java @@ -141,7 +141,7 @@ public void testClientCompactionIntervalSpec_serde() throws Exception new SegmentDescriptor(Intervals.of("2015-04-11/2015-04-12"), "v1", 0) ); - // Test with uncompactedSegments (incremental compaction) + // Test with uncompactedSegments (minor compaction) ClientCompactionIntervalSpec withSegments = new ClientCompactionIntervalSpec(interval, segments, "sha256hash"); String json1 = mapper.writeValueAsString(withSegments); ClientCompactionIntervalSpec deserialized1 = mapper.readValue(json1, ClientCompactionIntervalSpec.class); From f732719f316e7a554cf313e176531f5d6ae61627 Mon Sep 17 00:00:00 2001 From: cecemei Date: Wed, 4 Mar 2026 17:00:08 -0800 Subject: [PATCH 20/26] bug --- .../org/apache/druid/indexing/common/task/CompactionTask.java | 4 ++-- .../org/apache/druid/msq/exec/MSQCompactionTaskRunTest.java | 3 ++- 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java index 15d73c9f658b..3167fbebeedb 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java @@ -1306,9 +1306,9 @@ static class SegmentProvider private boolean shouldUpgradeSegment(DataSegment s) { if (minorCompaction) { - return false; - } else { return !uncompactedSegments.contains(s.toDescriptor()) && this.interval.contains(s.getInterval()); + } else { + return false; } } diff --git a/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQCompactionTaskRunTest.java b/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQCompactionTaskRunTest.java index 72a69fe6cad8..5b03780f3893 100644 --- a/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQCompactionTaskRunTest.java +++ b/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQCompactionTaskRunTest.java @@ -31,6 +31,7 @@ import com.google.inject.util.Providers; import org.apache.druid.client.indexing.ClientCompactionTaskGranularitySpec; import org.apache.druid.data.input.impl.DimensionsSpec; +import org.apache.druid.error.DruidException; import org.apache.druid.guice.DruidGuiceExtensions; import org.apache.druid.guice.LifecycleModule; import org.apache.druid.guice.SegmentWranglerModule; @@ -728,7 +729,7 @@ public void testMinorCompactionOverlappingInterval() throws Exception compactionTaskBuilder(Granularities.EIGHT_HOUR) .inputSpec(new MinorCompactionInputSpec(compactionInterval, uncompactedFromIndexTask), true) .build(); - ISE e = Assert.assertThrows(ISE.class, () -> runTask(compactionTask1)); + DruidException e = Assert.assertThrows(DruidException.class, () -> runTask(compactionTask1)); Assert.assertEquals( "Minor compaction doesn't allow segments not completely within interval[2014-01-01T00:00:00.000Z/2014-01-01T08:00:00.000Z]", e.getMessage() From e703f292a08a1ec0b95394cea12ddacb9365e2e3 Mon Sep 17 00:00:00 2001 From: cecemei Date: Wed, 4 Mar 2026 17:42:27 -0800 Subject: [PATCH 21/26] checkstyle --- .../java/org/apache/druid/msq/exec/MSQCompactionTaskRunTest.java | 1 - 1 file changed, 1 deletion(-) diff --git a/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQCompactionTaskRunTest.java b/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQCompactionTaskRunTest.java index 5b03780f3893..cee3a613a75d 100644 --- a/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQCompactionTaskRunTest.java +++ b/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQCompactionTaskRunTest.java @@ -52,7 +52,6 @@ import org.apache.druid.indexing.common.task.MinorCompactionInputSpec; import org.apache.druid.indexing.common.task.Tasks; import org.apache.druid.indexing.common.task.TuningConfigBuilder; -import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.StringUtils; From 1a055bae6ffb2fdf3f49de20401cc97df0bb5862 Mon Sep 17 00:00:00 2001 From: cecemei Date: Wed, 4 Mar 2026 17:55:45 -0800 Subject: [PATCH 22/26] format --- .../druid/indexing/common/task/CompactionTask.java | 7 +------ .../compact/CompactionConfigBasedJobTemplate.java | 5 ++++- .../metadata/IndexerSQLMetadataStorageCoordinator.java | 10 ++++------ 3 files changed, 9 insertions(+), 13 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java index 3167fbebeedb..647b750585fa 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java @@ -687,7 +687,7 @@ static Map createInputDataSchemas( final Iterable segmentsToCompact = Iterables.filter( timelineSegments, - segmentProvider::shouldCompactSegment + s -> !segmentProvider.shouldUpgradeSegment(s) ); final DataSchema dataSchema = createDataSchema( toolbox.getEmitter(), @@ -1312,11 +1312,6 @@ private boolean shouldUpgradeSegment(DataSegment s) } } - private boolean shouldCompactSegment(DataSegment s) - { - return !shouldUpgradeSegment(s); - } - List findSegments(TaskActionClient actionClient) throws IOException { return new ArrayList<>( diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionConfigBasedJobTemplate.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionConfigBasedJobTemplate.java index 709e2198d088..26fcb6354081 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionConfigBasedJobTemplate.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionConfigBasedJobTemplate.java @@ -56,7 +56,10 @@ public CompactionConfigBasedJobTemplate(DataSourceCompactionConfig config) this(config, ReindexingConfigOptimizer.IDENTITY); } - public CompactionConfigBasedJobTemplate(DataSourceCompactionConfig config, ReindexingConfigOptimizer configOptimizer) + public CompactionConfigBasedJobTemplate( + DataSourceCompactionConfig config, + ReindexingConfigOptimizer configOptimizer + ) { this.config = config; this.configOptimizer = configOptimizer; 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 fe4b4af34ecd..f66264bba76d 100644 --- a/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java +++ b/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java @@ -1489,10 +1489,9 @@ private PendingSegmentRecord createNewPendingSegment( /** * Creates a new pending segment for the given datasource and interval. - * * @param partialShardSpec Shard spec info minus segment id stuff - * @param existingVersion Version of segments in interval, used to compute the version of the very first segment in - * interval + * @param existingVersion Version of segments in interval, used to compute the version of the very first segment in + * interval */ @Nullable private SegmentIdWithShardSpec createNewPendingSegment( @@ -2010,7 +2009,6 @@ private Set createNewSegmentsAfterReplace( *
  • The set of segments being committed is non-empty.
  • *
  • All segments belong to the same datasource.
  • * - * * @return Name of the common data source */ private String verifySegmentsToCommit(Collection segments) @@ -2300,8 +2298,8 @@ private Map getSegmentsCoveredByTaskLock( * {@link DataSourceMetadata#matches matches} the {@code endMetadata}, this * method returns immediately with success. * - * @param supervisorId The supervisor ID. Used as the PK for the corresponding metadata entry in the DB. - * @param dataSource The dataSource. Currently used only for logging purposes. + * @param supervisorId The supervisor ID. Used as the PK for the corresponding metadata entry in the DB. + * @param dataSource The dataSource. Currently used only for logging purposes. * @param startMetadata Current entry in the DB must * {@link DataSourceMetadata#matches match} this value. * @param endMetadata The updated entry will be equal to the current entry From fb628eefb55a5913c9c014034191f1eec1a3d2b5 Mon Sep 17 00:00:00 2001 From: Cece Mei Date: Mon, 9 Mar 2026 11:19:47 -0700 Subject: [PATCH 23/26] Update server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java Co-authored-by: Kashif Faraz --- .../druid/metadata/IndexerSQLMetadataStorageCoordinator.java | 1 + 1 file changed, 1 insertion(+) 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 f66264bba76d..2e49e9165d04 100644 --- a/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java +++ b/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java @@ -1899,6 +1899,7 @@ private Set createNewSegmentsAfterReplace( final List segmentsToUpgrade = retrieveSegmentsById(dataSource, transaction, upgradeSegmentToLockVersion.keySet()); + // If there is nothing to upgrade, return the replaceSegments unchanged if (segmentsToUpgrade.isEmpty()) { return replaceSegments.stream() .map(s -> new DataSegmentPlus(s, null, null, null, null, null, null, null)) From 0131c043814a15300c47fd4907f48d421830ba2d Mon Sep 17 00:00:00 2001 From: cecemei Date: Mon, 9 Mar 2026 11:22:02 -0700 Subject: [PATCH 24/26] review --- .../druid/indexing/common/task/CompactionTask.java | 13 +++++++++++-- .../IndexerSQLMetadataStorageCoordinator.java | 4 ++-- 2 files changed, 13 insertions(+), 4 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java index 7811d5c0392b..274fe34c1a21 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java @@ -107,6 +107,7 @@ 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.timeline.SegmentTimeline; import org.apache.druid.timeline.TimelineObjectHolder; import org.apache.druid.timeline.VersionedIntervalTimeline; @@ -614,7 +615,11 @@ static Map createInputDataSchemas( } } if (!segmentsToUpgrade.isEmpty()) { - log.info("Marking [%d]segments to upgrade", segmentsToUpgrade.size()); + log.info( + "Marking [%d]segments to upgrade, showing the first 10 segments:%s", + segmentsToUpgrade.size(), + segmentsToUpgrade.stream().map(DataSegment::getId).map(SegmentId::toString).limit(10L) + ); toolbox.getTaskActionClient() .submit(new MarkSegmentToUpgradeAction(segmentProvider.dataSource, segmentsToUpgrade)); } @@ -680,7 +685,11 @@ static Map createInputDataSchemas( segmentProvider::shouldUpgradeSegment )); if (!segmentsToUpgrade.isEmpty()) { - log.info("Marking [%d]segments to upgrade", segmentsToUpgrade.size()); + log.info( + "Marking [%d]segments to upgrade, showing the first 10 segments:%s", + segmentsToUpgrade.size(), + segmentsToUpgrade.stream().map(DataSegment::getId).map(SegmentId::toString).limit(10L) + ); toolbox.getTaskActionClient() .submit(new MarkSegmentToUpgradeAction(segmentProvider.dataSource, segmentsToUpgrade)); } 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 2e49e9165d04..e344b714c353 100644 --- a/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java +++ b/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java @@ -1894,7 +1894,7 @@ private Set createNewSegmentsAfterReplace( .map(ReplaceTaskLock::getSupervisorTaskId) .findFirst().orElse(null); final Map upgradeSegmentToLockVersion - = getSegmentsCoveredByTaskLock(transaction, taskId); + = getSegmentsCoveredByReplaceTaskLock(transaction, taskId); final List segmentsToUpgrade = retrieveSegmentsById(dataSource, transaction, upgradeSegmentToLockVersion.keySet()); @@ -2229,7 +2229,7 @@ private List retrieveSegmentsById( * * @return Map from append Segment ID to REPLACE lock version */ - private Map getSegmentsCoveredByTaskLock( + private Map getSegmentsCoveredByReplaceTaskLock( SegmentMetadataTransaction transaction, String taskId ) From 826110cd53c34ae7c82216769ef01635674f966e Mon Sep 17 00:00:00 2001 From: cecemei Date: Mon, 9 Mar 2026 11:31:31 -0700 Subject: [PATCH 25/26] review --- .../compact/CompactionSupervisorTest.java | 66 +++++-------------- .../IndexerSQLMetadataStorageCoordinator.java | 2 +- 2 files changed, 17 insertions(+), 51 deletions(-) diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionSupervisorTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionSupervisorTest.java index 52481c86448a..2c68c0eeab6d 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionSupervisorTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionSupervisorTest.java @@ -19,7 +19,6 @@ package org.apache.druid.testing.embedded.compact; -import com.fasterxml.jackson.core.type.TypeReference; import org.apache.druid.catalog.guice.CatalogClientModule; import org.apache.druid.catalog.guice.CatalogCoordinatorModule; import org.apache.druid.common.utils.IdUtils; @@ -44,16 +43,15 @@ import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.HumanReadableBytes; import org.apache.druid.java.util.common.IAE; +import org.apache.druid.java.util.common.Numbers; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.granularity.Granularity; -import org.apache.druid.java.util.common.jackson.JacksonUtils; import org.apache.druid.java.util.emitter.service.ServiceMetricEvent; import org.apache.druid.query.DruidMetrics; import org.apache.druid.query.expression.TestExprMacroTable; import org.apache.druid.query.filter.EqualityFilter; import org.apache.druid.query.filter.NotDimFilter; -import org.apache.druid.query.http.ClientSqlQuery; import org.apache.druid.rpc.UpdateResponse; import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.column.ColumnType; @@ -103,7 +101,6 @@ import org.junit.jupiter.params.provider.MethodSource; import javax.annotation.Nullable; -import java.nio.charset.StandardCharsets; import java.time.Duration; import java.util.ArrayList; import java.util.List; @@ -281,7 +278,7 @@ public void test_minorCompactionWithMSQ(PartitionsSpec partitionsSpec) throws Ex pauseCompaction(dayGranularityConfig); Assertions.assertEquals(0, getNumSegmentsWith(Granularities.HOUR)); Assertions.assertEquals(1, getNumSegmentsWith(Granularities.DAY)); - Assertions.assertEquals("2000", cluster.runSql("SELECT COUNT(*) FROM %s", dataSource)); + Assertions.assertEquals(2000, getTotalRowCount()); verifyCompactedSegmentsHaveFingerprints(dayGranularityConfig); @@ -302,7 +299,7 @@ public void test_minorCompactionWithMSQ(PartitionsSpec partitionsSpec) throws Ex Assertions.assertEquals(0, getNumSegmentsWith(Granularities.HOUR)); // 1 compacted segment + 2 appended segment Assertions.assertEquals(3, getNumSegmentsWith(Granularities.DAY)); - Assertions.assertEquals("3000", cluster.runSql("SELECT COUNT(*) FROM %s", dataSource)); + Assertions.assertEquals(3000, getTotalRowCount()); runCompactionWithSpec(dayGranularityConfig); waitForAllCompactionTasksToFinish(); @@ -315,7 +312,7 @@ public void test_minorCompactionWithMSQ(PartitionsSpec partitionsSpec) throws Ex // performed minor compaction: 1 previously compacted segment + 1 incrementally compacted segment Assertions.assertEquals(2, getNumSegmentsWith(Granularities.DAY)); - Assertions.assertEquals("3000", cluster.runSql("SELECT COUNT(*) FROM %s", dataSource)); + Assertions.assertEquals(3000, getTotalRowCount()); } protected void waitUntilPublishedRecordsAreIngested(int expectedRowCount) @@ -521,7 +518,7 @@ public void test_cascadingReindexing_withVirtualColumnOnNestedData_filtersCorrec cluster.callApi().runTask(task.withId(IdUtils.getRandomId()), overlord); cluster.callApi().waitForAllSegmentsToBeAvailable(dataSource, coordinator, broker); - Assertions.assertEquals("4", cluster.runSql("SELECT COUNT(*) FROM %s", dataSource)); + Assertions.assertEquals(4, getTotalRowCount()); VirtualColumns virtualColumns = VirtualColumns.create( new ExpressionVirtualColumn( @@ -572,7 +569,7 @@ public void test_cascadingReindexing_withVirtualColumnOnNestedData_filtersCorrec cluster.callApi().waitForAllSegmentsToBeAvailable(dataSource, coordinator, broker); // Verify: Should have 2 rows left (valueA appeared in 2 rows, both filtered out) - Assertions.assertEquals("2", cluster.runSql("SELECT COUNT(*) FROM %s", dataSource)); + Assertions.assertEquals(2, getTotalRowCount()); // Verify the correct rows were filtered verifyNoRowsWithNestedValue("extraInfo", "fieldA", "valueA"); @@ -716,30 +713,21 @@ public void test_compaction_legacy_string_discovery_sparse_column( private int getTotalRowCount() { - String sql = StringUtils.format("SELECT COUNT(*) as cnt FROM \"%s\"", dataSource); - String result = cluster.callApi().onAnyBroker(b -> b.submitSqlQuery(new ClientSqlQuery(sql, null, false, false, false, null, null))); - List> rows = JacksonUtils.readValue( - new DefaultObjectMapper(), - result.getBytes(StandardCharsets.UTF_8), - new TypeReference<>() {} - ); - return ((Number) rows.get(0).get("cnt")).intValue(); + return Numbers.parseInt(cluster.runSql("SELECT COUNT(*) as cnt FROM \"%s\"", dataSource)); } private void verifyNoRowsWithNestedValue(String nestedColumn, String field, String value) { - String sql = StringUtils.format( + String result = cluster.runSql( "SELECT COUNT(*) as cnt FROM \"%s\" WHERE json_value(%s, '$.%s') = '%s'", dataSource, nestedColumn, field, value ); - ClientSqlQuery clientSqlQuery = new ClientSqlQuery(sql, null, false, false, false, null, null); - List> rows = parse(cluster.callApi().onAnyBroker(b -> b.submitSqlQuery(clientSqlQuery))); Assertions.assertEquals( 0, - ((Number) rows.get(0).get("cnt")).intValue(), + Numbers.parseInt(result), StringUtils.format("Expected no rows where %s.%s = '%s'", nestedColumn, field, value) ); } @@ -904,27 +892,16 @@ private void verifyEventCountOlderThan(Period period, String dimension, String v DateTime now = DateTimes.nowUtc(); DateTime threshold = now.minus(period); - ClientSqlQuery clientSqlQuery = new ClientSqlQuery( - StringUtils.format( - "SELECT COUNT(*) as cnt FROM \"%s\" WHERE %s = '%s' AND __time < MILLIS_TO_TIMESTAMP(%d)", - dataSource, - dimension, - value, - threshold.getMillis() - ), - null, - false, - false, - false, - null, - null + String result = cluster.runSql( + "SELECT COUNT(*) as cnt FROM \"%s\" WHERE %s = '%s' AND __time < MILLIS_TO_TIMESTAMP(%d)", + dataSource, + dimension, + value, + threshold.getMillis() ); - List> result = parse(cluster.callApi().onAnyBroker(b -> b.submitSqlQuery(clientSqlQuery))); - - Assertions.assertEquals(1, result.size()); Assertions.assertEquals( expectedCount, - result.get(0).get("cnt"), + Numbers.parseInt(result), StringUtils.format( "Expected %d events where %s='%s' older than %s", expectedCount, @@ -959,15 +936,4 @@ private UserCompactionTaskQueryTuningConfig createTuningConfigWithPartitionsSpec null ); } - - private static List> parse(String resultAsJson) - { - return JacksonUtils.readValue( - new DefaultObjectMapper(), - resultAsJson.getBytes(StandardCharsets.UTF_8), - new TypeReference<>() - { - } - ); - } } 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 e344b714c353..fbe30a31e88e 100644 --- a/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java +++ b/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java @@ -2223,7 +2223,7 @@ private List retrieveSegmentsById( } /** - * Finds segments were covered by the given task REPLACE locks. + * Finds segments that were covered by the given task REPLACE locks. * These segments must now be upgraded to the same version as the segments * being committed by this replace task. * From 4ceeb51bf07c338f35274555d8ae37c7ba65e9b1 Mon Sep 17 00:00:00 2001 From: cecemei Date: Tue, 10 Mar 2026 20:27:30 -0700 Subject: [PATCH 26/26] build --- .../apache/druid/msq/exec/MSQCompactionTaskRunTest.java | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQCompactionTaskRunTest.java b/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQCompactionTaskRunTest.java index 2cef2e3a39bf..3f182cef2531 100644 --- a/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQCompactionTaskRunTest.java +++ b/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQCompactionTaskRunTest.java @@ -592,7 +592,13 @@ public void testMinorCompaction() throws Exception final Pair resultPair1 = runTask(compactionTask1); verifyTaskSuccessRowsAndSchemaMatch(resultPair1, TOTAL_TEST_ROWS); - verifyCompactedSegment(List.copyOf(resultPair1.rhs.getSegments()), segmentGranularity, DEFAULT_QUERY_GRAN, false); + verifyCompactedSegment( + compactionTask1.getCompactionRunner(), + List.copyOf(resultPair1.rhs.getSegments()), + segmentGranularity, + DEFAULT_QUERY_GRAN, + false + ); Assert.assertEquals(1, resultPair1.rhs.getSegments().size()); final DataSegment compactedSegment1 = Iterables.getOnlyElement(resultPair1.rhs.getSegments());