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 9aba7f75bacd..ae7e9c6dda7b 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 @@ -22,13 +22,22 @@ 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.HashedPartitionsSpec; +import org.apache.druid.indexer.partitions.PartitionsSpec; import org.apache.druid.indexing.common.task.IndexTask; 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.HumanReadableBytes; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.query.DruidMetrics; @@ -39,9 +48,12 @@ import org.apache.druid.segment.metadata.IndexingStateCache; import org.apache.druid.segment.metadata.IndexingStateFingerprintMapper; import org.apache.druid.segment.transform.CompactionTransformSpec; +import org.apache.druid.server.compaction.CompactionCandidateSearchPolicy; +import org.apache.druid.server.compaction.MostFragmentedIntervalFirstPolicy; 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; @@ -54,22 +66,31 @@ 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.Period; import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.MethodSource; +import org.testcontainers.shaded.org.jetbrains.annotations.Nullable; 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 { + protected final KafkaResource kafkaServer = new KafkaResource(); private final EmbeddedBroker broker = new EmbeddedBroker(); private final EmbeddedIndexer indexer = new EmbeddedIndexer() .setServerMemory(2_000_000_000L) @@ -95,29 +116,43 @@ 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 + ) .addServer(coordinator) .addServer(overlord) .addServer(indexer) .addServer(historical) .addServer(broker) + .addResource(kafkaServer) .addServer(new EmbeddedRouter()); } - 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( @@ -138,27 +173,9 @@ public void test_ingestDayGranularity_andCompactToMonthGranularity_andCompactToY new UserCompactionTaskGranularityConfig(Granularities.MONTH, null, null) ) .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 - ) + UserCompactionTaskQueryTuningConfig.builder() + .partitionsSpec(new DimensionRangePartitionsSpec(null, 5000, List.of("item"), false)) + .build() ) .build(); @@ -179,27 +196,9 @@ public void test_ingestDayGranularity_andCompactToMonthGranularity_andCompactToY new UserCompactionTaskGranularityConfig(Granularities.YEAR, null, null) ) .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 - ) + UserCompactionTaskQueryTuningConfig.builder() + .partitionsSpec(new DimensionRangePartitionsSpec(null, 5000, List.of("item"), false)) + .build() ) .build(); @@ -214,6 +213,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) @@ -244,27 +362,9 @@ public void test_compaction_withPersistLastCompactionStateFalse_storesOnlyFinger new UserCompactionTaskGranularityConfig(Granularities.MONTH, null, null) ) .withTuningConfig( - new UserCompactionTaskQueryTuningConfig( - null, - null, - null, - null, - null, - new DimensionRangePartitionsSpec(1000, null, List.of("item"), false), - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null - ) + UserCompactionTaskQueryTuningConfig.builder() + .partitionsSpec(new DimensionRangePartitionsSpec(1000, null, List.of("item"), false)) + .build() ) .build(); @@ -279,7 +379,7 @@ public void test_compaction_withPersistLastCompactionStateFalse_storesOnlyFinger * 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. @@ -291,7 +391,7 @@ public void test_compactionWithTransformFilteringAllRows_createsTombstones( String partitionType ) { - configureCompaction(compactionEngine); + configureCompaction(compactionEngine, null); runIngestionAtGranularity( "DAY", @@ -327,52 +427,17 @@ 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 - ) + UserCompactionTaskQueryTuningConfig.builder() + .partitionsSpec(new DimensionRangePartitionsSpec(null, 5000, List.of("item"), false)) + .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 - ) + UserCompactionTaskQueryTuningConfig.builder() + .partitionsSpec(new HashedPartitionsSpec(null, null, null)) + .maxNumConcurrentSubTasks(2) + .build() ); } @@ -434,9 +499,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() 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 89eb331df49f..4482097bf045 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 ) { return CompactionConfigValidationResult.success(); @@ -99,7 +101,7 @@ public CompactionConfigValidationResult validateCompactionTask( */ @VisibleForTesting static List createIngestionSpecs( - Map intervalDataSchemaMap, + Map inputSchemas, final TaskToolbox toolbox, final CompactionIOConfig ioConfig, final PartitionConfigurationManager partitionConfigurationManager, @@ -109,18 +111,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()); } @@ -179,7 +181,7 @@ private static ParallelIndexIOConfig createIoConfig( @Override public TaskStatus runCompactionTasks( CompactionTask compactionTask, - Map intervalDataSchemaMap, + Map intervalDataSchemaMap, TaskToolbox taskToolbox ) throws Exception { @@ -321,8 +323,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 24b6e1f6af40..5c1b2ff16fcf 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.CompactionSlotManager; import org.apache.druid.server.compaction.DataSourceCompactibleSegmentIterator; -import org.apache.druid.server.compaction.NewestSegmentFirstPolicy; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; import org.apache.druid.server.coordinator.duty.CompactSegments; import org.apache.druid.timeline.CompactionState; @@ -133,9 +132,7 @@ DataSourceCompactibleSegmentIterator getCompactibleCandidates( config, timeline, Intervals.complementOf(searchInterval), - // This policy is used only while creating jobs - // The actual order of jobs is determined by the policy used in CompactionJobQueue - new NewestSegmentFirstPolicy(null), + params.getClusterCompactionConfig().getCompactionPolicy(), params.getFingerprintMapper() ); 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 9446ac664f29..53697b0ae9c1 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; @@ -43,7 +44,6 @@ import org.apache.druid.server.compaction.CompactionCandidateSearchPolicy; import org.apache.druid.server.compaction.CompactionSlotManager; 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.coordinator.AutoCompactionSnapshot; import org.apache.druid.server.coordinator.ClusterCompactionConfig; @@ -217,7 +217,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 +267,6 @@ public Map getSnapshots() */ private boolean startJobIfPendingAndReady( CompactionJob job, - CompactionCandidateSearchPolicy policy, List pendingJobs, CompactionSlotManager slotManager ) @@ -282,18 +281,17 @@ private boolean startJobIfPendingAndReady( } // Check if the job is already running, completed or skipped - final CompactionStatus compactionStatus = getCurrentStatusForJob(job, policy); - switch (compactionStatus.getState()) { - case RUNNING: + final CompactionCandidate.TaskState candidateState = getCurrentTaskStateForJob(job); + switch (candidateState) { + case TASK_IN_PROGRESS: return false; - case COMPLETE: + case RECENTLY_COMPLETED: snapshotBuilder.moveFromPendingToCompleted(candidate); return false; - case SKIPPED: - snapshotBuilder.moveFromPendingToSkipped(candidate); - return false; - default: + case READY: break; + default: + throw DruidException.defensive("unknown compaction candidate state[%s]", candidateState); } // Check if enough compaction task slots are available @@ -378,12 +376,10 @@ private void persistPendingIndexingState(CompactionJob job) } } - public CompactionStatus getCurrentStatusForJob(CompactionJob job, CompactionCandidateSearchPolicy policy) + public CompactionCandidate.TaskState getCurrentTaskStateForJob(CompactionJob job) { - final CompactionStatus compactionStatus = statusTracker.computeCompactionStatus(job.getCandidate(), policy); - final CompactionCandidate candidatesWithStatus = job.getCandidate().withCurrentStatus(null); - statusTracker.onCompactionStatusComputed(candidatesWithStatus, null); - return compactionStatus; + statusTracker.onCompactionCandidates(job.getCandidate(), null); + return statusTracker.computeCompactionTaskState(job.getCandidate()); } public static CompactionConfigValidationResult validateCompactionJob(BatchIndexingJob job) 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..6e06c962046a 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 @@ -499,7 +499,7 @@ public CompactionSimulateResult simulateRunWithConfigUpdate(ClusterCompactionCon updateRequest.getEngine() ); } else { - return new CompactionSimulateResult(Collections.emptyMap()); + return new CompactionSimulateResult(Collections.emptyMap(), 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 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 1c04c7b5bd2b..842227efb01e 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 f6dece2ba9da..4cf7dc4820a6 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 acb0dba5027b..f56c109cda5d 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; @@ -787,7 +788,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)), @@ -801,7 +802,7 @@ public void testCreateIngestionSchema() throws IOException ); final List ingestionSpecs = NativeCompactionRunner.createIngestionSpecs( - dataSchemasForIntervals, + inputSchemas, toolbox, new CompactionIOConfig(null, false, null), new PartitionConfigurationManager(TUNING_CONFIG), @@ -850,7 +851,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)), @@ -864,7 +865,7 @@ public void testCreateIngestionSchemaWithTargetPartitionSize() throws IOExceptio ); final List ingestionSpecs = NativeCompactionRunner.createIngestionSpecs( - dataSchemasForIntervals, + inputSchemas, toolbox, new CompactionIOConfig(null, false, null), new PartitionConfigurationManager(tuningConfig), @@ -914,7 +915,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)), @@ -928,7 +929,7 @@ public void testCreateIngestionSchemaWithMaxTotalRows() throws IOException ); final List ingestionSpecs = NativeCompactionRunner.createIngestionSpecs( - dataSchemasForIntervals, + inputSchemas, toolbox, new CompactionIOConfig(null, false, null), new PartitionConfigurationManager(tuningConfig), @@ -979,7 +980,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)), @@ -993,7 +994,7 @@ public void testCreateIngestionSchemaWithNumShards() throws IOException ); final List ingestionSpecs = NativeCompactionRunner.createIngestionSpecs( - dataSchemasForIntervals, + inputSchemas, toolbox, new CompactionIOConfig(null, false, null), new PartitionConfigurationManager(tuningConfig), @@ -1051,7 +1052,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)), @@ -1065,7 +1066,7 @@ public void testCreateIngestionSchemaWithCustomDimensionsSpec() throws IOExcepti ); final List ingestionSpecs = NativeCompactionRunner.createIngestionSpecs( - dataSchemasForIntervals, + inputSchemas, toolbox, new CompactionIOConfig(null, false, null), new PartitionConfigurationManager(TUNING_CONFIG), @@ -1103,7 +1104,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)), @@ -1117,7 +1118,7 @@ public void testCreateIngestionSchemaWithCustomMetricsSpec() throws IOException ); final List ingestionSpecs = NativeCompactionRunner.createIngestionSpecs( - dataSchemasForIntervals, + inputSchemas, toolbox, new CompactionIOConfig(null, false, null), new PartitionConfigurationManager(TUNING_CONFIG), @@ -1148,7 +1149,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)), @@ -1162,7 +1163,7 @@ public void testCreateIngestionSchemaWithCustomSegments() throws IOException ); final List ingestionSpecs = NativeCompactionRunner.createIngestionSpecs( - dataSchemasForIntervals, + inputSchemas, toolbox, new CompactionIOConfig(null, false, null), new PartitionConfigurationManager(TUNING_CONFIG), @@ -1200,7 +1201,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)), @@ -1214,7 +1215,7 @@ public void testCreateIngestionSchemaWithDifferentSegmentSet() throws IOExceptio ); NativeCompactionRunner.createIngestionSpecs( - dataSchemasForIntervals, + inputSchemas, toolbox, new CompactionIOConfig(null, false, null), new PartitionConfigurationManager(TUNING_CONFIG), @@ -1232,7 +1233,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)), @@ -1246,7 +1247,7 @@ public void testMissingMetadata() throws IOException ); NativeCompactionRunner.createIngestionSpecs( - dataSchemasForIntervals, + inputSchemas, toolbox, new CompactionIOConfig(null, false, null), new PartitionConfigurationManager(TUNING_CONFIG), @@ -1275,7 +1276,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)), @@ -1289,7 +1290,7 @@ public void testSegmentGranularityAndNullQueryGranularity() throws IOException ); final List ingestionSpecs = NativeCompactionRunner.createIngestionSpecs( - dataSchemasForIntervals, + inputSchemas, toolbox, new CompactionIOConfig(null, false, null), new PartitionConfigurationManager(TUNING_CONFIG), @@ -1321,7 +1322,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)), @@ -1334,7 +1335,7 @@ public void testQueryGranularityAndNullSegmentGranularity() throws IOException false ); final List ingestionSpecs = NativeCompactionRunner.createIngestionSpecs( - dataSchemasForIntervals, + inputSchemas, toolbox, new CompactionIOConfig(null, false, null), new PartitionConfigurationManager(TUNING_CONFIG), @@ -1364,7 +1365,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)), @@ -1382,7 +1383,7 @@ public void testQueryGranularityAndSegmentGranularityNonNull() throws IOExceptio ); final List ingestionSpecs = NativeCompactionRunner.createIngestionSpecs( - dataSchemasForIntervals, + inputSchemas, toolbox, new CompactionIOConfig(null, false, null), new PartitionConfigurationManager(TUNING_CONFIG), @@ -1417,7 +1418,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)), @@ -1431,7 +1432,7 @@ public void testNullGranularitySpec() throws IOException ); final List ingestionSpecs = NativeCompactionRunner.createIngestionSpecs( - dataSchemasForIntervals, + inputSchemas, toolbox, new CompactionIOConfig(null, false, null), new PartitionConfigurationManager(TUNING_CONFIG), @@ -1464,7 +1465,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)), @@ -1478,7 +1479,7 @@ public void testGranularitySpecWithNullQueryGranularityAndNullSegmentGranularity ); final List ingestionSpecs = NativeCompactionRunner.createIngestionSpecs( - dataSchemasForIntervals, + inputSchemas, toolbox, new CompactionIOConfig(null, false, null), new PartitionConfigurationManager(TUNING_CONFIG), @@ -1511,7 +1512,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)), @@ -1525,7 +1526,7 @@ public void testGranularitySpecWithNotNullRollup() ); final List ingestionSpecs = NativeCompactionRunner.createIngestionSpecs( - dataSchemasForIntervals, + inputSchemas, toolbox, new CompactionIOConfig(null, false, null), new PartitionConfigurationManager(TUNING_CONFIG), @@ -1543,7 +1544,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)), @@ -1557,7 +1558,7 @@ public void testGranularitySpecWithNullRollup() ); final List ingestionSpecs = NativeCompactionRunner.createIngestionSpecs( - dataSchemasForIntervals, + inputSchemas, toolbox, new CompactionIOConfig(null, false, null), new PartitionConfigurationManager(TUNING_CONFIG), @@ -1577,7 +1578,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)), @@ -1589,7 +1590,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()); } @@ -2214,7 +2215,7 @@ private static class TestMSQCompactionRunner implements CompactionRunner @Override public TaskStatus runCompactionTasks( CompactionTask compactionTask, - Map intervalDataSchemaMap, + Map inputSchemas, TaskToolbox taskToolbox ) { @@ -2230,7 +2231,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/compact/OverlordCompactionSchedulerTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/compact/OverlordCompactionSchedulerTest.java index 4c98b52f48c1..78389263b5fd 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/compact/OverlordCompactionSchedulerTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/compact/OverlordCompactionSchedulerTest.java @@ -66,9 +66,9 @@ import org.apache.druid.segment.TestIndex; import org.apache.druid.segment.metadata.HeapMemoryIndexingStateStorage; import org.apache.druid.segment.metadata.IndexingStateCache; +import org.apache.druid.server.compaction.CompactionCandidate; import org.apache.druid.server.compaction.CompactionSimulateResult; import org.apache.druid.server.compaction.CompactionStatistics; -import org.apache.druid.server.compaction.CompactionStatus; import org.apache.druid.server.compaction.CompactionStatusTracker; import org.apache.druid.server.compaction.Table; import org.apache.druid.server.coordinator.AutoCompactionSnapshot; @@ -451,7 +451,7 @@ public void test_simulateRunWithConfigUpdate() new ClusterCompactionConfig(null, null, null, null, null, null) ); Assert.assertEquals(1, simulateResult.getCompactionStates().size()); - final Table pendingCompactionTable = simulateResult.getCompactionStates().get(CompactionStatus.State.PENDING); + final Table pendingCompactionTable = simulateResult.getCompactionStates().get(CompactionCandidate.TaskState.READY); Assert.assertEquals( Arrays.asList("dataSource", "interval", "numSegments", "bytes", "maxTaskSlots", "reasonToCompact"), pendingCompactionTable.getColumnNames() 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 2310ff84072d..bea3fcafc66c 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; @@ -152,7 +153,7 @@ public MSQCompactionRunner( @Override public CompactionConfigValidationResult validateCompactionTask( CompactionTask compactionTask, - Map intervalToDataSchemaMap + Map intervalToDataSchemaMap ) { if (intervalToDataSchemaMap.size() > 1) { @@ -233,11 +234,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( @@ -256,21 +257,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); + QuerySegmentSpec segmentSpec = inputSchema.getKey(); + DataSchema dataSchema = inputSchema.getValue(); + Map inputColToVirtualCol = + getVirtualColumns(dataSchema, JodaUtils.umbrellaInterval(segmentSpec.getIntervals())); 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()); @@ -500,7 +502,7 @@ private static Map buildQueryContext( private Query buildScanQuery( CompactionTask compactionTask, - Interval interval, + QuerySegmentSpec segmentSpec, DataSchema dataSchema, Map inputColToVirtualCol ) @@ -512,7 +514,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)); @@ -631,7 +633,7 @@ private Map getVirtualColumns(DataSchema dataSchema, Inte private Query buildGroupByQuery( CompactionTask compactionTask, - Interval interval, + QuerySegmentSpec segmentSpec, DataSchema dataSchema, Map inputColToVirtualCol ) @@ -667,7 +669,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 eeb24bb62f16..365b104b86be 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; @@ -576,6 +584,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 ebeba9329458..04ca6ca4555c 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: + *

+ * + * @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 4a72e01e7c46..7a44198f708b 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 @@ -2354,8 +2395,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( @@ -2373,8 +2414,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()) { @@ -2457,8 +2498,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) @@ -2730,10 +2771,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); @@ -2744,7 +2785,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/BaseCandidateSearchPolicy.java b/server/src/main/java/org/apache/druid/server/compaction/BaseCandidateSearchPolicy.java index 2a9107132623..951f00ba788f 100644 --- a/server/src/main/java/org/apache/druid/server/compaction/BaseCandidateSearchPolicy.java +++ b/server/src/main/java/org/apache/druid/server/compaction/BaseCandidateSearchPolicy.java @@ -68,12 +68,12 @@ public final int compareCandidates(CompactionCandidate o1, CompactionCandidate o } @Override - public Eligibility checkEligibilityForCompaction( - CompactionCandidate candidate, - CompactionTaskStatus latestTaskStatus + public CompactionCandidate createCandidate( + CompactionCandidate.ProposedCompaction proposedCompaction, + CompactionStatus eligibility ) { - return Eligibility.OK; + return CompactionMode.FULL_COMPACTION.createCandidate(proposedCompaction, eligibility); } /** 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..c2845fa46f7b 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 @@ -19,6 +19,7 @@ package org.apache.druid.server.compaction; +import com.google.common.base.Preconditions; import org.apache.druid.error.InvalidInput; import org.apache.druid.java.util.common.JodaUtils; import org.apache.druid.java.util.common.granularity.Granularity; @@ -28,6 +29,7 @@ import javax.annotation.Nullable; import java.util.List; +import java.util.Objects; import java.util.Set; import java.util.stream.Collectors; @@ -37,58 +39,182 @@ */ public class CompactionCandidate { - private final List segments; - private final Interval umbrellaInterval; - private final Interval compactionInterval; - private final String dataSource; - private final long totalBytes; - private final int numIntervals; - - private final CompactionStatus currentStatus; - - public static CompactionCandidate from( - List segments, - @Nullable Granularity targetSegmentGranularity - ) + /** + * Non-empty list of segments of a datasource being proposed for compaction. + * A proposed compaction typically contains all the segments of a single time chunk. + */ + public static class ProposedCompaction { - if (segments == null || segments.isEmpty()) { - throw InvalidInput.exception("Segments to compact must be non-empty"); + private final List segments; + private final Interval umbrellaInterval; + private final Interval compactionInterval; + private final String dataSource; + private final long totalBytes; + private final int numIntervals; + + public static ProposedCompaction from( + List segments, + @Nullable Granularity targetSegmentGranularity + ) + { + if (segments == null || segments.isEmpty()) { + throw InvalidInput.exception("Segments to compact must be non-empty"); + } + + final Set segmentIntervals = + segments.stream().map(DataSegment::getInterval).collect(Collectors.toSet()); + final Interval umbrellaInterval = JodaUtils.umbrellaInterval(segmentIntervals); + final Interval compactionInterval = + targetSegmentGranularity == null + ? umbrellaInterval + : JodaUtils.umbrellaInterval(targetSegmentGranularity.getIterable(umbrellaInterval)); + + return new ProposedCompaction( + segments, + umbrellaInterval, + compactionInterval, + segmentIntervals.size() + ); } - final Set segmentIntervals = - segments.stream().map(DataSegment::getInterval).collect(Collectors.toSet()); - final Interval umbrellaInterval = JodaUtils.umbrellaInterval(segmentIntervals); - final Interval compactionInterval = - targetSegmentGranularity == null - ? umbrellaInterval - : JodaUtils.umbrellaInterval(targetSegmentGranularity.getIterable(umbrellaInterval)); - - return new CompactionCandidate( - segments, - umbrellaInterval, - compactionInterval, - segmentIntervals.size(), - null - ); + ProposedCompaction( + List segments, + Interval umbrellaInterval, + Interval compactionInterval, + int numDistinctSegmentIntervals + ) + { + this.segments = segments; + this.totalBytes = segments.stream().mapToLong(DataSegment::getSize).sum(); + + this.umbrellaInterval = umbrellaInterval; + this.compactionInterval = compactionInterval; + + this.numIntervals = numDistinctSegmentIntervals; + this.dataSource = segments.get(0).getDataSource(); + } + + /** + * @return Non-empty list of segments that make up this proposed compaction. + */ + public List getSegments() + { + return segments; + } + + public long getTotalBytes() + { + return totalBytes; + } + + public int numSegments() + { + return segments.size(); + } + + /** + * Umbrella interval of all the segments in this proposed compaction. This typically + * corresponds to a single time chunk in the segment timeline. + */ + public Interval getUmbrellaInterval() + { + return umbrellaInterval; + } + + /** + * Interval aligned to the target segment granularity used for the compaction + * task. This interval completely contains the {@link #umbrellaInterval}. + */ + public Interval getCompactionInterval() + { + return compactionInterval; + } + + public String getDataSource() + { + return dataSource; + } + + public CompactionStatistics getStats() + { + return CompactionStatistics.create(totalBytes, numSegments(), numIntervals); + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + ProposedCompaction that = (ProposedCompaction) o; + return totalBytes == that.totalBytes + && numIntervals == that.numIntervals + && segments.equals(that.segments) + && umbrellaInterval.equals(that.umbrellaInterval) + && compactionInterval.equals(that.compactionInterval) + && dataSource.equals(that.dataSource); + } + + @Override + public int hashCode() + { + return Objects.hash(segments, umbrellaInterval, compactionInterval, dataSource, totalBytes, numIntervals); + } + + @Override + public String toString() + { + return "ProposedCompaction{" + + "datasource=" + dataSource + + ", umbrellaInterval=" + umbrellaInterval + + ", compactionInterval=" + compactionInterval + + ", numIntervals=" + numIntervals + + ", segments=" + SegmentUtils.commaSeparatedIdentifiers(segments) + + ", totalSize=" + totalBytes + + '}'; + } } - private CompactionCandidate( - List segments, - Interval umbrellaInterval, - Interval compactionInterval, - int numDistinctSegmentIntervals, - @Nullable CompactionStatus currentStatus - ) + /** + * Used by {@link CompactionStatusTracker#computeCompactionTaskState(CompactionCandidate)}. + * The callsite then determines whether to launch compaction task or not. + */ + public enum TaskState { - this.segments = segments; - this.totalBytes = segments.stream().mapToLong(DataSegment::getSize).sum(); + // no other compaction candidate is running, we can start a new task + READY, + // compaction candidate is already running under a task + TASK_IN_PROGRESS, + // compaction candidate has recently been completed, and the segment timeline has not yet updated after that + RECENTLY_COMPLETED + } + + private final ProposedCompaction proposedCompaction; - this.umbrellaInterval = umbrellaInterval; - this.compactionInterval = compactionInterval; + private final CompactionStatus eligibility; + @Nullable + private final String policyNote; + private final CompactionMode mode; + + CompactionCandidate( + ProposedCompaction proposedCompaction, + CompactionStatus eligibility, + @Nullable String policyNote, + CompactionMode mode + ) + { + this.proposedCompaction = Preconditions.checkNotNull(proposedCompaction, "proposedCompaction"); + this.eligibility = Preconditions.checkNotNull(eligibility, "eligibility"); + this.policyNote = policyNote; + this.mode = Preconditions.checkNotNull(mode, "mode"); + } - this.numIntervals = numDistinctSegmentIntervals; - this.dataSource = segments.get(0).getDataSource(); - this.currentStatus = currentStatus; + public ProposedCompaction getProposedCompaction() + { + return proposedCompaction; } /** @@ -96,17 +222,17 @@ private CompactionCandidate( */ public List getSegments() { - return segments; + return proposedCompaction.getSegments(); } public long getTotalBytes() { - return totalBytes; + return proposedCompaction.getTotalBytes(); } public int numSegments() { - return segments.size(); + return proposedCompaction.numSegments(); } /** @@ -115,67 +241,52 @@ public int numSegments() */ public Interval getUmbrellaInterval() { - return umbrellaInterval; + return proposedCompaction.getUmbrellaInterval(); } /** * Interval aligned to the target segment granularity used for the compaction - * task. This interval completely contains the {@link #umbrellaInterval}. + * task. This interval completely contains the {@link #getUmbrellaInterval()}. */ public Interval getCompactionInterval() { - return compactionInterval; + return proposedCompaction.getCompactionInterval(); } public String getDataSource() { - return dataSource; + return proposedCompaction.getDataSource(); } public CompactionStatistics getStats() { - return CompactionStatistics.create(totalBytes, numSegments(), numIntervals); - } - - @Nullable - public CompactionStatistics getCompactedStats() - { - return (currentStatus == null || currentStatus.getCompactedStats() == null) - ? null : currentStatus.getCompactedStats(); + return proposedCompaction.getStats(); } @Nullable - public CompactionStatistics getUncompactedStats() + public String getPolicyNote() { - return (currentStatus == null || currentStatus.getUncompactedStats() == null) - ? null : currentStatus.getUncompactedStats(); + return policyNote; } - /** - * Current compaction status of the time chunk corresponding to this candidate. - */ - @Nullable - public CompactionStatus getCurrentStatus() + public CompactionMode getMode() { - return currentStatus; + return mode; } - /** - * Creates a copy of this CompactionCandidate object with the given status. - */ - public CompactionCandidate withCurrentStatus(CompactionStatus status) + public CompactionStatus getEligibility() { - return new CompactionCandidate(segments, umbrellaInterval, compactionInterval, numIntervals, status); + return eligibility; } @Override public String toString() { return "SegmentsToCompact{" + - "datasource=" + dataSource + - ", segments=" + SegmentUtils.commaSeparatedIdentifiers(segments) + - ", totalSize=" + totalBytes + - ", currentStatus=" + currentStatus + + ", proposedCompaction=" + proposedCompaction + + ", eligibility=" + eligibility + + ", policyNote=" + policyNote + + ", mode=" + mode + '}'; } } 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 bfb69787dd84..3afe296297b1 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 @@ -21,11 +21,8 @@ import com.fasterxml.jackson.annotation.JsonSubTypes; import com.fasterxml.jackson.annotation.JsonTypeInfo; -import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.server.coordinator.duty.CompactSegments; -import java.util.Objects; - /** * Policy used by {@link CompactSegments} duty to pick segments for compaction. */ @@ -48,74 +45,11 @@ public interface CompactionCandidateSearchPolicy int compareCandidates(CompactionCandidate candidateA, CompactionCandidate candidateB); /** - * Checks if the given {@link CompactionCandidate} is eligible for compaction - * in the current iteration. A policy may implement this method to skip - * compacting intervals or segments that do not fulfil some required criteria. + * Creates a {@link CompactionCandidate} after applying policy-specific checks to the proposed compaction candidate. * - * @return {@link Eligibility#OK} only if eligible. - */ - Eligibility checkEligibilityForCompaction( - CompactionCandidate candidate, - CompactionTaskStatus latestTaskStatus - ); - - /** - * Describes the eligibility of an interval for compaction. + * @param candidate the proposed compaction + * @param eligibility initial eligibility from compaction config checks + * @return final compaction candidate */ - class Eligibility - { - public static final Eligibility OK = new Eligibility(true, null); - - private final boolean eligible; - private final String reason; - - private Eligibility(boolean eligible, String reason) - { - this.eligible = eligible; - this.reason = reason; - } - - public boolean isEligible() - { - return eligible; - } - - public String getReason() - { - return reason; - } - - public static Eligibility fail(String messageFormat, Object... args) - { - return new Eligibility(false, StringUtils.format(messageFormat, args)); - } - - @Override - public boolean equals(Object object) - { - if (this == object) { - return true; - } - if (object == null || getClass() != object.getClass()) { - return false; - } - Eligibility that = (Eligibility) object; - return eligible == that.eligible && Objects.equals(reason, that.reason); - } - - @Override - public int hashCode() - { - return Objects.hash(eligible, reason); - } - - @Override - public String toString() - { - return "Eligibility{" + - "eligible=" + eligible + - ", reason='" + reason + '\'' + - '}'; - } - } + CompactionCandidate createCandidate(CompactionCandidate.ProposedCompaction candidate, CompactionStatus eligibility); } 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 new file mode 100644 index 000000000000..e89a5b2a7329 --- /dev/null +++ b/server/src/main/java/org/apache/druid/server/compaction/CompactionMode.java @@ -0,0 +1,115 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.server.compaction; + +import org.apache.druid.error.DruidException; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.timeline.DataSegment; + +import javax.annotation.Nullable; +import java.util.Objects; + +public enum CompactionMode +{ + FULL_COMPACTION { + @Override + public CompactionCandidate createCandidate( + CompactionCandidate.ProposedCompaction proposedCompaction, + CompactionStatus eligibility, + @Nullable String policyNote + ) + { + return new CompactionCandidate(proposedCompaction, eligibility, policyNote, this); + } + }, + INCREMENTAL_COMPACTION { + @Override + public CompactionCandidate createCandidate( + CompactionCandidate.ProposedCompaction proposedCompaction, + CompactionStatus eligibility, + @Nullable String policyNote + ) + { + CompactionCandidate.ProposedCompaction newProposed = new CompactionCandidate.ProposedCompaction( + Objects.requireNonNull(eligibility.getUncompactedSegments()), + proposedCompaction.getUmbrellaInterval(), + proposedCompaction.getCompactionInterval(), + Math.toIntExact(eligibility.getUncompactedSegments() + .stream() + .map(DataSegment::getInterval) + .distinct() + .count()) + ); + return new CompactionCandidate(newProposed, eligibility, policyNote, this); + } + }, + NOT_APPLICABLE; + + public CompactionCandidate createCandidate( + CompactionCandidate.ProposedCompaction proposedCompaction, + CompactionStatus eligibility + ) + { + return createCandidate(proposedCompaction, eligibility, null); + } + + public CompactionCandidate createCandidate( + CompactionCandidate.ProposedCompaction proposedCompaction, + CompactionStatus eligibility, + @Nullable String policyNote + ) + { + throw DruidException.defensive("Cannot create compaction candidate with mode[%s]", this); + } + + public static CompactionCandidate failWithPolicyCheck( + CompactionCandidate.ProposedCompaction proposedCompaction, + CompactionStatus eligibility, + String reasonFormat, + Object... args + ) + { + return new CompactionCandidate( + proposedCompaction, + eligibility, + StringUtils.format(reasonFormat, args), + CompactionMode.NOT_APPLICABLE + ); + } + + public static CompactionCandidate notEligible( + CompactionCandidate.ProposedCompaction proposedCompaction, + String reason + ) + { + // CompactionStatus returns an ineligible reason, have not even got to policy check yet + return new CompactionCandidate( + proposedCompaction, + CompactionStatus.notEligible(reason), + null, + CompactionMode.NOT_APPLICABLE + ); + } + + public static CompactionCandidate complete(CompactionCandidate.ProposedCompaction proposedCompaction) + { + return new CompactionCandidate(proposedCompaction, CompactionStatus.COMPLETE, null, CompactionMode.NOT_APPLICABLE); + } +} diff --git a/server/src/main/java/org/apache/druid/server/compaction/CompactionRunSimulator.java b/server/src/main/java/org/apache/druid/server/compaction/CompactionRunSimulator.java index bab65f90bf94..1ffa48036ff9 100644 --- a/server/src/main/java/org/apache/druid/server/compaction/CompactionRunSimulator.java +++ b/server/src/main/java/org/apache/druid/server/compaction/CompactionRunSimulator.java @@ -25,6 +25,8 @@ import org.apache.druid.client.indexing.ClientCompactionTaskQueryTuningConfig; import org.apache.druid.client.indexing.IndexingTotalWorkerCapacityInfo; import org.apache.druid.client.indexing.TaskPayloadResponse; +import org.apache.druid.common.guava.GuavaUtils; +import org.apache.druid.error.DruidException; import org.apache.druid.indexer.CompactionEngine; import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexer.TaskStatusPlus; @@ -87,36 +89,81 @@ public CompactionSimulateResult simulateRunWithConfig( // account for the active tasks final CompactionStatusTracker simulationStatusTracker = new CompactionStatusTracker() { + + @Override + public void onSkippedCandidate( + CompactionCandidate candidateSegments, + DataSourceCompactionConfig config + ) + { + skippedIntervals.addRow(createRow( + candidateSegments, + null, + GuavaUtils.firstNonNull( + candidateSegments.getPolicyNote(), + candidateSegments.getEligibility().getReason() + ) + )); + } + @Override - public CompactionStatus computeCompactionStatus( - CompactionCandidate candidate, - CompactionCandidateSearchPolicy searchPolicy + public void onCompactionCandidates( + CompactionCandidate candidateSegments, + DataSourceCompactionConfig config ) { - return statusTracker.computeCompactionStatus(candidate, searchPolicy); + switch (candidateSegments.getMode()) { + case NOT_APPLICABLE: + skippedIntervals.addRow(createRow( + candidateSegments, + null, + GuavaUtils.firstNonNull( + candidateSegments.getPolicyNote(), + candidateSegments.getEligibility().getReason() + ) + )); + break; + case INCREMENTAL_COMPACTION: + case FULL_COMPACTION: + queuedIntervals.addRow(createRow( + candidateSegments, + ClientCompactionTaskQueryTuningConfig.from(config), + GuavaUtils.firstNonNull( + candidateSegments.getPolicyNote(), + candidateSegments.getEligibility().getReason() + ) + )); + break; + default: + throw DruidException.defensive("unexpected compaction mode[%s]", candidateSegments.getMode()); + } } @Override - public void onCompactionStatusComputed( + public void onCompactionTaskStateComputed( CompactionCandidate candidateSegments, + CompactionCandidate.TaskState taskState, DataSourceCompactionConfig config ) { - final CompactionStatus status = candidateSegments.getCurrentStatus(); - if (status == null) { - // do nothing - } else if (status.getState() == CompactionStatus.State.COMPLETE) { - compactedIntervals.addRow( - createRow(candidateSegments, null, null) - ); - } else if (status.getState() == CompactionStatus.State.RUNNING) { - runningIntervals.addRow( - createRow(candidateSegments, ClientCompactionTaskQueryTuningConfig.from(config), status.getReason()) - ); - } else if (status.getState() == CompactionStatus.State.SKIPPED) { - skippedIntervals.addRow( - createRow(candidateSegments, null, status.getReason()) - ); + switch (taskState) { + case RECENTLY_COMPLETED: + compactedIntervals.addRow(createRow(candidateSegments, null, null)); + break; + case TASK_IN_PROGRESS: + runningIntervals.addRow(createRow( + candidateSegments, + ClientCompactionTaskQueryTuningConfig.from(config), + GuavaUtils.firstNonNull( + candidateSegments.getPolicyNote(), + candidateSegments.getEligibility().getReason() + ) + )); + break; + case READY: + break; + default: + throw DruidException.defensive("unknown compaction task state[%s]", taskState); } } @@ -124,10 +171,11 @@ public void onCompactionStatusComputed( public void onTaskSubmitted(String taskId, CompactionCandidate candidateSegments) { // Add a row for each task in order of submission - final CompactionStatus status = candidateSegments.getCurrentStatus(); - queuedIntervals.addRow( - createRow(candidateSegments, null, status == null ? "" : status.getReason()) + final String reason = GuavaUtils.firstNonNull( + candidateSegments.getPolicyNote(), + candidateSegments.getEligibility().getReason() ); + queuedIntervals.addRow(createRow(candidateSegments, null, reason)); } }; @@ -150,21 +198,18 @@ public void onTaskSubmitted(String taskId, CompactionCandidate candidateSegments stats ); - final Map compactionStates = new HashMap<>(); + final Map compactionStates = new HashMap<>(); if (!compactedIntervals.isEmpty()) { - compactionStates.put(CompactionStatus.State.COMPLETE, compactedIntervals); + compactionStates.put(CompactionCandidate.TaskState.RECENTLY_COMPLETED, compactedIntervals); } if (!runningIntervals.isEmpty()) { - compactionStates.put(CompactionStatus.State.RUNNING, runningIntervals); + compactionStates.put(CompactionCandidate.TaskState.TASK_IN_PROGRESS, runningIntervals); } if (!queuedIntervals.isEmpty()) { - compactionStates.put(CompactionStatus.State.PENDING, queuedIntervals); - } - if (!skippedIntervals.isEmpty()) { - compactionStates.put(CompactionStatus.State.SKIPPED, skippedIntervals); + compactionStates.put(CompactionCandidate.TaskState.READY, queuedIntervals); } - return new CompactionSimulateResult(compactionStates); + return new CompactionSimulateResult(compactionStates, skippedIntervals); } private Object[] createRow( diff --git a/server/src/main/java/org/apache/druid/server/compaction/CompactionSimulateResult.java b/server/src/main/java/org/apache/druid/server/compaction/CompactionSimulateResult.java index 7a48ccf0e5ba..6b91c03ff40d 100644 --- a/server/src/main/java/org/apache/druid/server/compaction/CompactionSimulateResult.java +++ b/server/src/main/java/org/apache/druid/server/compaction/CompactionSimulateResult.java @@ -26,19 +26,28 @@ public class CompactionSimulateResult { - private final Map compactionStates; + private final Map compactionStates; + private final Table skippedIntervals; @JsonCreator public CompactionSimulateResult( - @JsonProperty("compactionStates") Map compactionStates + @JsonProperty("compactionStates") Map compactionStates, + @JsonProperty("skippedIntervals") Table skippedIntervals ) { this.compactionStates = compactionStates; + this.skippedIntervals = skippedIntervals; } @JsonProperty - public Map getCompactionStates() + public Map getCompactionStates() { return compactionStates; } + + @JsonProperty + public Table getSkippedIntervals() + { + return skippedIntervals; + } } 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 99e1eef21465..4b9f6a9efdf3 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 @@ -19,10 +19,13 @@ package org.apache.druid.server.compaction; +import com.google.common.base.Strings; import org.apache.commons.lang3.ArrayUtils; import org.apache.druid.client.indexing.ClientCompactionTaskQueryTuningConfig; import org.apache.druid.common.config.Configs; import org.apache.druid.data.input.impl.DimensionSchema; +import org.apache.druid.error.DruidException; +import org.apache.druid.error.InvalidInput; import org.apache.druid.indexer.partitions.DimensionRangePartitionsSpec; import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; import org.apache.druid.indexer.partitions.HashedPartitionsSpec; @@ -48,28 +51,27 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.Set; import java.util.function.Function; import java.util.stream.Collectors; /** - * Represents the status of compaction for a given {@link CompactionCandidate}. + * Describes the eligibility of an interval for compaction. */ public class CompactionStatus { - private static final Logger log = new Logger(CompactionStatus.class); - - private static final CompactionStatus COMPLETE = new CompactionStatus(State.COMPLETE, null, null, null); + public static final CompactionStatus COMPLETE = new CompactionStatus(State.COMPLETE, "", null, null, null); public enum State { - COMPLETE, PENDING, RUNNING, SKIPPED + COMPLETE, ELIGIBLE, NOT_ELIGIBLE } /** * List of checks performed to determine if compaction is already complete based on indexing state fingerprints. */ - private static final List> FINGERPRINT_CHECKS = List.of( + static final List> FINGERPRINT_CHECKS = List.of( Evaluator::allFingerprintedCandidatesHaveExpectedFingerprint ); @@ -78,7 +80,7 @@ public enum State *

    * The order of the checks must be honored while evaluating them. */ - private static final List> CHECKS = Arrays.asList( + static final List> CHECKS = Arrays.asList( Evaluator::partitionsSpecIsUpToDate, Evaluator::indexSpecIsUpToDate, Evaluator::segmentGranularityIsUpToDate, @@ -90,32 +92,64 @@ public enum State Evaluator::projectionsAreUpToDate ); + public static CompactionStatus notEligible(String messageFormat, Object... args) + { + return new CompactionStatus(State.NOT_ELIGIBLE, StringUtils.format(messageFormat, args), null, null, null); + } + private final State state; private final String reason; - private final CompactionStatistics compactedStats; - private final CompactionStatistics uncompactedStats; + + @Nullable + private final CompactionStatistics compacted; + @Nullable + private final CompactionStatistics uncompacted; + @Nullable + private final List uncompactedSegments; private CompactionStatus( State state, String reason, - CompactionStatistics compactedStats, - CompactionStatistics uncompactedStats + @Nullable CompactionStatistics compacted, + @Nullable CompactionStatistics uncompacted, + @Nullable List uncompactedSegments ) { this.state = state; this.reason = reason; - this.compactedStats = compactedStats; - this.uncompactedStats = uncompactedStats; + switch (state) { + case COMPLETE: + break; + case NOT_ELIGIBLE: + InvalidInput.conditionalException( + !Strings.isNullOrEmpty(reason), + "must provide a reason why compaction not eligible" + ); + break; + case ELIGIBLE: + InvalidInput.conditionalException(compacted != null, "must provide compacted stats for compaction"); + InvalidInput.conditionalException(uncompacted != null, "must provide uncompacted stats for compaction"); + InvalidInput.conditionalException( + uncompactedSegments != null, + "must provide uncompactedSegments for compaction" + ); + break; + default: + throw DruidException.defensive("unexpected compaction status state[%s]", state); + } + this.compacted = compacted; + this.uncompacted = uncompacted; + this.uncompactedSegments = uncompactedSegments; } - public boolean isComplete() + static CompactionStatusBuilder builder(State state, String reason) { - return state == State.COMPLETE; + return new CompactionStatusBuilder(state, reason); } - public boolean isSkipped() + public State getState() { - return state == State.SKIPPED; + return state; } public String getReason() @@ -123,58 +157,90 @@ public String getReason() return reason; } - public State getState() + @Nullable + public CompactionStatistics getUncompactedStats() { - return state; + return uncompacted; } + @Nullable public CompactionStatistics getCompactedStats() { - return compactedStats; + return compacted; } - public CompactionStatistics getUncompactedStats() + @Nullable + public List getUncompactedSegments() { - return uncompactedStats; + return uncompactedSegments; + } + + /** + * Evaluates a compaction candidate to determine its eligibility and compaction status. + *

    + * This method performs a two-stage evaluation: + *

      + *
    1. First, uses {@link Evaluator} to check if the candidate needs compaction + * based on the compaction config (e.g., checking segment granularity, partitions spec, etc.)
    2. + *
    3. Then, applies the search policy to determine if this candidate should be compacted in the + * current run (e.g., checking minimum segment count, bytes, or other policy criteria)
    4. + *
    + * + * @param proposedCompaction the compaction candidate to evaluate + * @param config the compaction configuration for the datasource + * @param fingerprintMapper mapper for indexing state fingerprints + * @return a new {@link CompactionCandidate} with updated eligibility and status. For incremental + * compaction, returns a candidate containing only the uncompacted segments. + */ + public static CompactionStatus compute( + CompactionCandidate.ProposedCompaction proposedCompaction, + DataSourceCompactionConfig config, + IndexingStateFingerprintMapper fingerprintMapper + ) + { + return new Evaluator(proposedCompaction, config, fingerprintMapper).evaluate(); } @Override - public String toString() + public boolean equals(Object object) { - return "CompactionStatus{" + - "state=" + state + - ", reason=" + reason + - ", compactedStats=" + compactedStats + - ", uncompactedStats=" + uncompactedStats + - '}'; + if (this == object) { + return true; + } + if (object == null || getClass() != object.getClass()) { + return false; + } + CompactionStatus that = (CompactionStatus) object; + return state == that.state + && Objects.equals(reason, that.reason) + && Objects.equals(compacted, that.compacted) + && Objects.equals(uncompacted, that.uncompacted) + && Objects.equals(uncompactedSegments, that.uncompactedSegments); } - public static CompactionStatus pending(String reasonFormat, Object... args) + @Override + public int hashCode() { - return new CompactionStatus(State.PENDING, StringUtils.format(reasonFormat, args), null, null); + return Objects.hash(state, reason, compacted, uncompacted, uncompactedSegments); } - public static CompactionStatus pending( - CompactionStatistics compactedStats, - CompactionStatistics uncompactedStats, - String reasonFormat, - Object... args - ) + @Override + public String toString() { - return new CompactionStatus( - State.PENDING, - StringUtils.format(reasonFormat, args), - compactedStats, - uncompactedStats - ); + return "CompactionStatus{" + + "state=" + state + + ", reason='" + reason + '\'' + + ", compacted=" + compacted + + ", uncompacted=" + uncompacted + + ", uncompactedSegments=" + uncompactedSegments + + '}'; } /** - * Computes compaction status for the given field. The status is assumed to be - * COMPLETE (i.e. no further compaction is required) if the configured value - * of the field is null or equal to the current value. + * Returns a 'mismatch' reason to be eligible for compaction if config doesn't match, NULL if config matches. */ - private static CompactionStatus completeIfNullOrEqual( + @Nullable + private static String getConfigMismatchReason( String field, T configured, T current, @@ -182,20 +248,20 @@ private static CompactionStatus completeIfNullOrEqual( ) { if (configured == null || configured.equals(current)) { - return COMPLETE; + return null; } else { return configChanged(field, configured, current, stringFunction); } } - private static CompactionStatus configChanged( + private static String configChanged( String field, T target, T current, Function stringFunction ) { - return CompactionStatus.pending( + return StringUtils.format( "'%s' mismatch: required[%s], current[%s]", field, target == null ? null : stringFunction.apply(target), @@ -241,41 +307,6 @@ 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); - } - - public static CompactionStatus running(String message) - { - return new CompactionStatus(State.RUNNING, message, null, null); - } - - /** - * Determines the CompactionStatus of the given candidate segments by evaluating - * the {@link #CHECKS} one by one. If any check returns an incomplete status, - * further checks are still performed to determine the number of uncompacted - * segments but only the first incomplete status is returned. - */ - static CompactionStatus compute( - CompactionCandidate candidateSegments, - DataSourceCompactionConfig config, - @Nullable IndexingStateFingerprintMapper fingerprintMapper - ) - { - final CompactionState expectedState = config.toCompactionState(); - String expectedFingerprint; - if (fingerprintMapper == null) { - expectedFingerprint = null; - } else { - expectedFingerprint = fingerprintMapper.generateFingerprint( - config.getDataSource(), - expectedState - ); - } - return new Evaluator(candidateSegments, config, expectedFingerprint, fingerprintMapper).evaluate(); - } - @Nullable public static PartitionsSpec findPartitionsSpecFromConfig(ClientCompactionTaskQueryTuningConfig tuningConfig) { @@ -346,8 +377,10 @@ static DimensionRangePartitionsSpec getEffectiveRangePartitionsSpec(DimensionRan */ private static class Evaluator { + private static final Logger log = new Logger(Evaluator.class); + private final DataSourceCompactionConfig compactionConfig; - private final CompactionCandidate candidateSegments; + private final CompactionCandidate.ProposedCompaction proposedCompaction; private final ClientCompactionTaskQueryTuningConfig tuningConfig; private final UserCompactionTaskGranularityConfig configuredGranularitySpec; @@ -357,47 +390,63 @@ private static class Evaluator private final Map> unknownStateToSegments = new HashMap<>(); @Nullable - private final String targetFingerprint; private final IndexingStateFingerprintMapper fingerprintMapper; + @Nullable + private final String targetFingerprint; private Evaluator( - CompactionCandidate candidateSegments, + CompactionCandidate.ProposedCompaction proposedCompaction, DataSourceCompactionConfig compactionConfig, - @Nullable String targetFingerprint, @Nullable IndexingStateFingerprintMapper fingerprintMapper ) { - this.candidateSegments = candidateSegments; + this.proposedCompaction = proposedCompaction; this.compactionConfig = compactionConfig; this.tuningConfig = ClientCompactionTaskQueryTuningConfig.from(compactionConfig); this.configuredGranularitySpec = compactionConfig.getGranularitySpec(); - this.targetFingerprint = targetFingerprint; this.fingerprintMapper = fingerprintMapper; + if (fingerprintMapper == null) { + targetFingerprint = null; + } else { + targetFingerprint = fingerprintMapper.generateFingerprint( + compactionConfig.getDataSource(), + compactionConfig.toCompactionState() + ); + } } + /** + * Evaluates the compaction status of candidate segments through a multi-step process: + *
      + *
    1. Validates input bytes are within limits
    2. + *
    3. Categorizes segments by compaction state (fingerprinted, uncompacted, or unknown)
    4. + *
    5. Performs fingerprint-based validation if available (fast path)
    6. + *
    7. Runs detailed checks against unknown states via {@link CompactionStatus#CHECKS}
    8. + *
    + * + * @return Pair of eligibility status and compaction status with reason for first failed check + */ private CompactionStatus evaluate() { - final CompactionStatus inputBytesCheck = inputBytesAreWithinLimit(); - if (inputBytesCheck.isSkipped()) { - return inputBytesCheck; + final String inputBytesCheck = inputBytesAreWithinLimit(); + if (inputBytesCheck != null) { + return CompactionStatus.notEligible(inputBytesCheck); } List reasonsForCompaction = new ArrayList<>(); - CompactionStatus compactedOnceCheck = segmentsHaveBeenCompactedAtLeastOnce(); - if (!compactedOnceCheck.isComplete()) { - reasonsForCompaction.add(compactedOnceCheck.getReason()); + String compactedOnceCheck = segmentsHaveBeenCompactedAtLeastOnce(); + if (compactedOnceCheck != null) { + reasonsForCompaction.add(compactedOnceCheck); } if (fingerprintMapper != null && targetFingerprint != null) { // First try fingerprint-based evaluation (fast path) - CompactionStatus fingerprintStatus = FINGERPRINT_CHECKS.stream() - .map(f -> f.apply(this)) - .filter(status -> !status.isComplete()) - .findFirst().orElse(COMPLETE); + FINGERPRINT_CHECKS.stream() + .map(f -> f.apply(this)) + .filter(Objects::nonNull) + .findFirst() + .ifPresent(reasonsForCompaction::add); - if (!fingerprintStatus.isComplete()) { - reasonsForCompaction.add(fingerprintStatus.getReason()); - } } if (!unknownStateToSegments.isEmpty()) { @@ -405,13 +454,12 @@ private CompactionStatus evaluate() reasonsForCompaction.addAll( CHECKS.stream() .map(f -> f.apply(this)) - .filter(status -> !status.isComplete()) - .map(CompactionStatus::getReason) + .filter(Objects::nonNull) .collect(Collectors.toList()) ); // Any segments left in unknownStateToSegments passed all checks and are considered compacted - this.compactedSegments.addAll( + compactedSegments.addAll( unknownStateToSegments .values() .stream() @@ -421,13 +469,12 @@ private CompactionStatus evaluate() } if (reasonsForCompaction.isEmpty()) { - return COMPLETE; + return CompactionStatus.COMPLETE; } else { - return CompactionStatus.pending( - createStats(this.compactedSegments), - createStats(uncompactedSegments), - reasonsForCompaction.get(0) - ); + return builder(State.ELIGIBLE, reasonsForCompaction.get(0)).compacted(createStats(compactedSegments)) + .uncompacted(createStats(uncompactedSegments)) + .uncompactedSegments(uncompactedSegments) + .build(); } } @@ -439,7 +486,7 @@ private CompactionStatus evaluate() * {@link #unknownStateToSegments} where their indexing states will be analyzed. *

    */ - private CompactionStatus allFingerprintedCandidatesHaveExpectedFingerprint() + private String allFingerprintedCandidatesHaveExpectedFingerprint() { Map> mismatchedFingerprintToSegmentMap = new HashMap<>(); for (DataSegment segment : fingerprintedSegments) { @@ -457,18 +504,18 @@ private CompactionStatus allFingerprintedCandidatesHaveExpectedFingerprint() if (mismatchedFingerprintToSegmentMap.isEmpty()) { // All fingerprinted segments have the expected fingerprint - compaction is complete - return COMPLETE; + return null; } if (fingerprintMapper == null) { // Cannot evaluate further without a fingerprint mapper uncompactedSegments.addAll( mismatchedFingerprintToSegmentMap.values() - .stream() - .flatMap(List::stream) - .collect(Collectors.toList()) + .stream() + .flatMap(List::stream) + .collect(Collectors.toList()) ); - return CompactionStatus.pending("Segments have a mismatched fingerprint and no fingerprint mapper is available"); + return "Segments have a mismatched fingerprint and no fingerprint mapper is available"; } boolean fingerprintedSegmentWithoutCachedStateFound = false; @@ -490,14 +537,15 @@ private CompactionStatus allFingerprintedCandidatesHaveExpectedFingerprint() } segments.addAll(e.getValue()); return segments; - }); + } + ); } } if (fingerprintedSegmentWithoutCachedStateFound) { - return CompactionStatus.pending("One or more fingerprinted segments do not have a cached indexing state"); + return "One or more fingerprinted segments do not have a cached indexing state"; } else { - return COMPLETE; + return null; } } @@ -505,9 +553,9 @@ private CompactionStatus allFingerprintedCandidatesHaveExpectedFingerprint() * Checks if all the segments have been compacted at least once and groups them into uncompacted, fingerprinted, or * non-fingerprinted. */ - private CompactionStatus segmentsHaveBeenCompactedAtLeastOnce() + private String segmentsHaveBeenCompactedAtLeastOnce() { - for (DataSegment segment : candidateSegments.getSegments()) { + for (DataSegment segment : proposedCompaction.getSegments()) { final String fingerprint = segment.getIndexingStateFingerprint(); final CompactionState segmentState = segment.getLastCompactionState(); if (fingerprint != null) { @@ -520,58 +568,58 @@ private CompactionStatus segmentsHaveBeenCompactedAtLeastOnce() } if (uncompactedSegments.isEmpty()) { - return COMPLETE; + return null; } else { - return CompactionStatus.pending("not compacted yet"); + return "not compacted yet"; } } - private CompactionStatus partitionsSpecIsUpToDate() + private String partitionsSpecIsUpToDate() { return evaluateForAllCompactionStates(this::partitionsSpecIsUpToDate); } - private CompactionStatus indexSpecIsUpToDate() + private String indexSpecIsUpToDate() { return evaluateForAllCompactionStates(this::indexSpecIsUpToDate); } - private CompactionStatus projectionsAreUpToDate() + private String projectionsAreUpToDate() { return evaluateForAllCompactionStates(this::projectionsAreUpToDate); } - private CompactionStatus segmentGranularityIsUpToDate() + private String segmentGranularityIsUpToDate() { return evaluateForAllCompactionStates(this::segmentGranularityIsUpToDate); } - private CompactionStatus rollupIsUpToDate() + private String rollupIsUpToDate() { return evaluateForAllCompactionStates(this::rollupIsUpToDate); } - private CompactionStatus queryGranularityIsUpToDate() + private String queryGranularityIsUpToDate() { return evaluateForAllCompactionStates(this::queryGranularityIsUpToDate); } - private CompactionStatus dimensionsSpecIsUpToDate() + private String dimensionsSpecIsUpToDate() { return evaluateForAllCompactionStates(this::dimensionsSpecIsUpToDate); } - private CompactionStatus metricsSpecIsUpToDate() + private String metricsSpecIsUpToDate() { return evaluateForAllCompactionStates(this::metricsSpecIsUpToDate); } - private CompactionStatus transformSpecFilterIsUpToDate() + private String transformSpecFilterIsUpToDate() { return evaluateForAllCompactionStates(this::transformSpecFilterIsUpToDate); } - private CompactionStatus partitionsSpecIsUpToDate(CompactionState lastCompactionState) + private String partitionsSpecIsUpToDate(CompactionState lastCompactionState) { PartitionsSpec existingPartionsSpec = lastCompactionState.getPartitionsSpec(); if (existingPartionsSpec instanceof DimensionRangePartitionsSpec) { @@ -579,9 +627,10 @@ private CompactionStatus partitionsSpecIsUpToDate(CompactionState lastCompaction } else if (existingPartionsSpec instanceof DynamicPartitionsSpec) { existingPartionsSpec = new DynamicPartitionsSpec( existingPartionsSpec.getMaxRowsPerSegment(), - ((DynamicPartitionsSpec) existingPartionsSpec).getMaxTotalRowsOr(Long.MAX_VALUE)); + ((DynamicPartitionsSpec) existingPartionsSpec).getMaxTotalRowsOr(Long.MAX_VALUE) + ); } - return CompactionStatus.completeIfNullOrEqual( + return getConfigMismatchReason( "partitionsSpec", findPartitionsSpecFromConfig(tuningConfig), existingPartionsSpec, @@ -589,9 +638,9 @@ private CompactionStatus partitionsSpecIsUpToDate(CompactionState lastCompaction ); } - private CompactionStatus indexSpecIsUpToDate(CompactionState lastCompactionState) + private String indexSpecIsUpToDate(CompactionState lastCompactionState) { - return CompactionStatus.completeIfNullOrEqual( + return getConfigMismatchReason( "indexSpec", Configs.valueOrDefault(tuningConfig.getIndexSpec(), IndexSpec.getDefault()).getEffectiveSpec(), lastCompactionState.getIndexSpec().getEffectiveSpec(), @@ -599,9 +648,9 @@ private CompactionStatus indexSpecIsUpToDate(CompactionState lastCompactionState ); } - private CompactionStatus projectionsAreUpToDate(CompactionState lastCompactionState) + private String projectionsAreUpToDate(CompactionState lastCompactionState) { - return CompactionStatus.completeIfNullOrEqual( + return getConfigMismatchReason( "projections", compactionConfig.getProjections(), lastCompactionState.getProjections(), @@ -609,24 +658,24 @@ private CompactionStatus projectionsAreUpToDate(CompactionState lastCompactionSt ); } - private CompactionStatus inputBytesAreWithinLimit() + @Nullable + private String inputBytesAreWithinLimit() { final long inputSegmentSize = compactionConfig.getInputSegmentSizeBytes(); - if (candidateSegments.getTotalBytes() > inputSegmentSize) { - return CompactionStatus.skipped( + if (proposedCompaction.getTotalBytes() > inputSegmentSize) { + return StringUtils.format( "'inputSegmentSize' exceeded: Total segment size[%d] is larger than allowed inputSegmentSize[%d]", - candidateSegments.getTotalBytes(), inputSegmentSize + proposedCompaction.getTotalBytes(), inputSegmentSize ); - } else { - return COMPLETE; } + return null; } - private CompactionStatus segmentGranularityIsUpToDate(CompactionState lastCompactionState) + private String segmentGranularityIsUpToDate(CompactionState lastCompactionState) { if (configuredGranularitySpec == null || configuredGranularitySpec.getSegmentGranularity() == null) { - return COMPLETE; + return null; } final Granularity configuredSegmentGranularity = configuredGranularitySpec.getSegmentGranularity(); @@ -635,7 +684,7 @@ private CompactionStatus segmentGranularityIsUpToDate(CompactionState lastCompac = existingGranularitySpec == null ? null : existingGranularitySpec.getSegmentGranularity(); if (configuredSegmentGranularity.equals(existingSegmentGranularity)) { - return COMPLETE; + return null; } else if (existingSegmentGranularity == null) { // Candidate segments were compacted without segment granularity specified // Check if the segments already have the desired segment granularity @@ -644,13 +693,13 @@ private CompactionStatus segmentGranularityIsUpToDate(CompactionState lastCompac segment -> !configuredSegmentGranularity.isAligned(segment.getInterval()) ); if (needsCompaction) { - return CompactionStatus.pending( + return StringUtils.format( "segmentGranularity: segments do not align with target[%s]", - asString(configuredSegmentGranularity) + CompactionStatus.asString(configuredSegmentGranularity) ); } } else { - return CompactionStatus.configChanged( + return configChanged( "segmentGranularity", configuredSegmentGranularity, existingSegmentGranularity, @@ -658,17 +707,17 @@ private CompactionStatus segmentGranularityIsUpToDate(CompactionState lastCompac ); } - return COMPLETE; + return null; } - private CompactionStatus rollupIsUpToDate(CompactionState lastCompactionState) + private String rollupIsUpToDate(CompactionState lastCompactionState) { if (configuredGranularitySpec == null) { - return COMPLETE; + return null; } else { final UserCompactionTaskGranularityConfig existingGranularitySpec = getGranularitySpec(lastCompactionState); - return CompactionStatus.completeIfNullOrEqual( + return getConfigMismatchReason( "rollup", configuredGranularitySpec.isRollup(), existingGranularitySpec == null ? null : existingGranularitySpec.isRollup(), @@ -677,14 +726,14 @@ private CompactionStatus rollupIsUpToDate(CompactionState lastCompactionState) } } - private CompactionStatus queryGranularityIsUpToDate(CompactionState lastCompactionState) + private String queryGranularityIsUpToDate(CompactionState lastCompactionState) { if (configuredGranularitySpec == null) { - return COMPLETE; + return null; } else { final UserCompactionTaskGranularityConfig existingGranularitySpec = getGranularitySpec(lastCompactionState); - return CompactionStatus.completeIfNullOrEqual( + return getConfigMismatchReason( "queryGranularity", configuredGranularitySpec.getQueryGranularity(), existingGranularitySpec == null ? null : existingGranularitySpec.getQueryGranularity(), @@ -698,10 +747,10 @@ private CompactionStatus queryGranularityIsUpToDate(CompactionState lastCompacti * which can create a mismatch between expected and actual order of dimensions. Partition dimensions are separately * covered in {@link Evaluator#partitionsSpecIsUpToDate()} check. */ - private CompactionStatus dimensionsSpecIsUpToDate(CompactionState lastCompactionState) + private String dimensionsSpecIsUpToDate(CompactionState lastCompactionState) { if (compactionConfig.getDimensionsSpec() == null) { - return COMPLETE; + return null; } else { List existingDimensions = getNonPartitioningDimensions( lastCompactionState.getDimensionsSpec() == null @@ -717,7 +766,7 @@ private CompactionStatus dimensionsSpecIsUpToDate(CompactionState lastCompaction ? IndexSpec.getDefault() : compactionConfig.getTuningConfig().getIndexSpec() ); - return CompactionStatus.completeIfNullOrEqual( + return getConfigMismatchReason( "dimensionsSpec", configuredDimensions, existingDimensions, @@ -726,11 +775,11 @@ private CompactionStatus dimensionsSpecIsUpToDate(CompactionState lastCompaction } } - private CompactionStatus metricsSpecIsUpToDate(CompactionState lastCompactionState) + private String metricsSpecIsUpToDate(CompactionState lastCompactionState) { final AggregatorFactory[] configuredMetricsSpec = compactionConfig.getMetricsSpec(); if (ArrayUtils.isEmpty(configuredMetricsSpec)) { - return COMPLETE; + return null; } final List metricSpecList = lastCompactionState.getMetricsSpec(); @@ -739,25 +788,25 @@ private CompactionStatus metricsSpecIsUpToDate(CompactionState lastCompactionSta ? null : metricSpecList.toArray(new AggregatorFactory[0]); if (existingMetricsSpec == null || !Arrays.deepEquals(configuredMetricsSpec, existingMetricsSpec)) { - return CompactionStatus.configChanged( + return configChanged( "metricsSpec", configuredMetricsSpec, existingMetricsSpec, Arrays::toString ); } else { - return COMPLETE; + return null; } } - private CompactionStatus transformSpecFilterIsUpToDate(CompactionState lastCompactionState) + private String transformSpecFilterIsUpToDate(CompactionState lastCompactionState) { if (compactionConfig.getTransformSpec() == null) { - return COMPLETE; + return null; } CompactionTransformSpec existingTransformSpec = lastCompactionState.getTransformSpec(); - return CompactionStatus.completeIfNullOrEqual( + return getConfigMismatchReason( "transformSpec filter", compactionConfig.getTransformSpec().getFilter(), existingTransformSpec == null ? null : existingTransformSpec.getFilter(), @@ -772,22 +821,20 @@ private CompactionStatus transformSpecFilterIsUpToDate(CompactionState lastCompa * * @return The first status which is not COMPLETE. */ - private CompactionStatus evaluateForAllCompactionStates( - Function check - ) + private String evaluateForAllCompactionStates(Function check) { - CompactionStatus firstIncompleteStatus = null; + String firstIncomplete = null; for (CompactionState state : List.copyOf(unknownStateToSegments.keySet())) { - final CompactionStatus status = check.apply(state); - if (!status.isComplete()) { + final String eligibleReason = check.apply(state); + if (eligibleReason != null) { uncompactedSegments.addAll(unknownStateToSegments.remove(state)); - if (firstIncompleteStatus == null) { - firstIncompleteStatus = status; + if (firstIncomplete == null) { + firstIncomplete = eligibleReason; } } } - return firstIncompleteStatus == null ? COMPLETE : firstIncompleteStatus; + return firstIncomplete; } private static UserCompactionTaskGranularityConfig getGranularitySpec( @@ -805,4 +852,42 @@ private static CompactionStatistics createStats(List segments) return CompactionStatistics.create(totalBytes, segments.size(), segmentIntervals.size()); } } + + static class CompactionStatusBuilder + { + private State state; + private CompactionStatistics compacted; + private CompactionStatistics uncompacted; + private List uncompactedSegments; + private String reason; + + CompactionStatusBuilder(State state, String reason) + { + this.state = state; + this.reason = reason; + } + + CompactionStatusBuilder compacted(CompactionStatistics compacted) + { + this.compacted = compacted; + return this; + } + + CompactionStatusBuilder uncompacted(CompactionStatistics uncompacted) + { + this.uncompacted = uncompacted; + return this; + } + + CompactionStatusBuilder uncompactedSegments(List uncompactedSegments) + { + this.uncompactedSegments = uncompactedSegments; + return this; + } + + CompactionStatus build() + { + return new CompactionStatus(state, reason, compacted, uncompacted, uncompactedSegments); + } + } } 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 1dc409e7361e..fb5dd40056e7 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 @@ -80,15 +80,12 @@ public Set getSubmittedTaskIds() * This method assumes that the given candidate is eligible for compaction * based on the current compaction config/supervisor of the datasource. */ - public CompactionStatus computeCompactionStatus( - CompactionCandidate candidate, - CompactionCandidateSearchPolicy searchPolicy - ) + public CompactionCandidate.TaskState computeCompactionTaskState(CompactionCandidate candidate) { // Skip intervals that already have a running task final CompactionTaskStatus lastTaskStatus = getLatestTaskStatus(candidate); if (lastTaskStatus != null && lastTaskStatus.getState() == TaskState.RUNNING) { - return CompactionStatus.running("Task for interval is already running"); + return CompactionCandidate.TaskState.TASK_IN_PROGRESS; } // Skip intervals that have been recently compacted if segment timeline is not updated yet @@ -96,27 +93,35 @@ public CompactionStatus computeCompactionStatus( if (lastTaskStatus != null && lastTaskStatus.getState() == TaskState.SUCCESS && snapshotTime != null && snapshotTime.isBefore(lastTaskStatus.getUpdatedTime())) { - return CompactionStatus.skipped( - "Segment timeline not updated since last compaction task succeeded" - ); + return CompactionCandidate.TaskState.RECENTLY_COMPLETED; } - // 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 CompactionCandidate.TaskState.READY; } /** * Tracks the latest compaction status of the given compaction candidates. * Used only by the {@link CompactionRunSimulator}. */ - public void onCompactionStatusComputed( + public void onSkippedCandidate( + CompactionCandidate candidateSegments, + DataSourceCompactionConfig config + ) + { + // Nothing to do, used by simulator + } + + public void onCompactionCandidates( + CompactionCandidate candidateSegments, + DataSourceCompactionConfig config + ) + { + // Nothing to do, used by simulator + } + + public void onCompactionTaskStateComputed( CompactionCandidate candidateSegments, + CompactionCandidate.TaskState taskState, DataSourceCompactionConfig config ) { diff --git a/server/src/main/java/org/apache/druid/server/compaction/DataSourceCompactibleSegmentIterator.java b/server/src/main/java/org/apache/druid/server/compaction/DataSourceCompactibleSegmentIterator.java index 1994e87a6388..26ae52b23eda 100644 --- a/server/src/main/java/org/apache/druid/server/compaction/DataSourceCompactibleSegmentIterator.java +++ b/server/src/main/java/org/apache/druid/server/compaction/DataSourceCompactibleSegmentIterator.java @@ -24,9 +24,11 @@ import com.google.common.base.Preconditions; import com.google.common.collect.Iterators; import com.google.common.collect.Lists; +import org.apache.druid.error.DruidException; import org.apache.druid.java.util.common.DateTimes; 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.guava.Comparators; import org.apache.druid.java.util.common.logger.Logger; @@ -80,6 +82,7 @@ public class DataSourceCompactibleSegmentIterator implements CompactionSegmentIt // run of the compaction job and skip any interval that was already previously compacted. private final Set queuedIntervals = new HashSet<>(); + private final CompactionCandidateSearchPolicy searchPolicy; private final PriorityQueue queue; public DataSourceCompactibleSegmentIterator( @@ -92,6 +95,7 @@ public DataSourceCompactibleSegmentIterator( { this.config = config; this.dataSource = config.getDataSource(); + this.searchPolicy = searchPolicy; this.queue = new PriorityQueue<>(searchPolicy::compareCandidates); this.fingerprintMapper = indexingStateFingerprintMapper; @@ -121,9 +125,11 @@ private void populateQueue(SegmentTimeline timeline, List skipInterval if (!partialEternitySegments.isEmpty()) { // Do not use the target segment granularity in the CompactionCandidate // as Granularities.getIterable() will cause OOM due to the above issue - CompactionCandidate candidatesWithStatus = CompactionCandidate - .from(partialEternitySegments, null) - .withCurrentStatus(CompactionStatus.skipped("Segments have partial-eternity intervals")); + CompactionCandidate candidatesWithStatus = + CompactionMode.notEligible( + CompactionCandidate.ProposedCompaction.from(partialEternitySegments, null), + "Segments have partial-eternity intervals" + ); skippedSegments.add(candidatesWithStatus); return; } @@ -329,17 +335,41 @@ private void findAndEnqueueSegmentsToCompact(CompactibleSegmentIterator compacti continue; } - final CompactionCandidate candidates = CompactionCandidate.from(segments, config.getSegmentGranularity()); - final CompactionStatus compactionStatus = CompactionStatus.compute(candidates, config, fingerprintMapper); - final CompactionCandidate candidatesWithStatus = candidates.withCurrentStatus(compactionStatus); + CompactionCandidate.ProposedCompaction proposed = + CompactionCandidate.ProposedCompaction.from(segments, config.getSegmentGranularity()); + final CompactionStatus eligibility = CompactionStatus.compute(proposed, config, fingerprintMapper); + final CompactionCandidate candidate; + switch (eligibility.getState()) { + case COMPLETE: + candidate = CompactionMode.complete(proposed); + break; + case NOT_ELIGIBLE: + candidate = CompactionMode.notEligible(proposed, eligibility.getReason()); + break; + case ELIGIBLE: + candidate = searchPolicy.createCandidate(proposed, eligibility); + break; + default: + throw DruidException.defensive("unknown compaction state[%s]", eligibility.getState()); + } - if (compactionStatus.isComplete()) { - compactedSegments.add(candidatesWithStatus); - } else if (compactionStatus.isSkipped()) { - skippedSegments.add(candidatesWithStatus); - } else if (!queuedIntervals.contains(candidates.getUmbrellaInterval())) { - queue.add(candidatesWithStatus); - queuedIntervals.add(candidates.getUmbrellaInterval()); + switch (candidate.getMode()) { + case INCREMENTAL_COMPACTION: + case FULL_COMPACTION: + if (!queuedIntervals.contains(candidate.getProposedCompaction().getUmbrellaInterval())) { + queue.add(candidate); + queuedIntervals.add(candidate.getProposedCompaction().getUmbrellaInterval()); + } + break; + case NOT_APPLICABLE: + if (CompactionStatus.State.COMPLETE.equals(candidate.getEligibility().getState())) { + compactedSegments.add(candidate); + } else { + skippedSegments.add(candidate); + } + break; + default: + throw DruidException.defensive("Unexpected compaction mode[%s]", candidate.getMode()); } } } @@ -372,16 +402,17 @@ private List findInitialSearchInterval( timeline.findNonOvershadowedObjectsInInterval(skipInterval, Partitions.ONLY_COMPLETE) ); if (!CollectionUtils.isNullOrEmpty(segments)) { - final CompactionCandidate candidates = CompactionCandidate.from(segments, config.getSegmentGranularity()); + final CompactionCandidate.ProposedCompaction candidates = + CompactionCandidate.ProposedCompaction.from(segments, config.getSegmentGranularity()); - final CompactionStatus reason; + final String skipReason; if (candidates.getCompactionInterval().overlaps(latestSkipInterval)) { - reason = CompactionStatus.skipped("skip offset from latest[%s]", skipOffset); + skipReason = StringUtils.format("skip offset from latest[%s]", skipOffset); } else { - reason = CompactionStatus.skipped("interval locked by another task"); + skipReason = "interval locked by another task"; } - final CompactionCandidate candidatesWithStatus = candidates.withCurrentStatus(reason); + final CompactionCandidate candidatesWithStatus = CompactionMode.notEligible(candidates, skipReason); skippedSegments.add(candidatesWithStatus); } } @@ -436,7 +467,8 @@ static Interval computeLatestSkipInterval( if (configuredSegmentGranularity == null) { return new Interval(skipOffsetFromLatest, latestDataTimestamp); } else { - DateTime skipFromLastest = new DateTime(latestDataTimestamp, latestDataTimestamp.getZone()).minus(skipOffsetFromLatest); + DateTime skipFromLastest = + new DateTime(latestDataTimestamp, latestDataTimestamp.getZone()).minus(skipOffsetFromLatest); DateTime skipOffsetBucketToSegmentGranularity = configuredSegmentGranularity.bucketStart(skipFromLastest); return new Interval(skipOffsetBucketToSegmentGranularity, latestDataTimestamp); } diff --git a/server/src/main/java/org/apache/druid/server/compaction/FixedIntervalOrderPolicy.java b/server/src/main/java/org/apache/druid/server/compaction/FixedIntervalOrderPolicy.java index 24a2f001afe3..279beca7a654 100644 --- a/server/src/main/java/org/apache/druid/server/compaction/FixedIntervalOrderPolicy.java +++ b/server/src/main/java/org/apache/druid/server/compaction/FixedIntervalOrderPolicy.java @@ -52,21 +52,25 @@ public List getEligibleCandidates() @Override public int compareCandidates(CompactionCandidate candidateA, CompactionCandidate candidateB) { - return findIndex(candidateA) - findIndex(candidateB); + return findIndex(candidateA.getProposedCompaction()) - findIndex(candidateB.getProposedCompaction()); } @Override - public Eligibility checkEligibilityForCompaction( - CompactionCandidate candidate, - CompactionTaskStatus latestTaskStatus + public CompactionCandidate createCandidate( + CompactionCandidate.ProposedCompaction candidate, + CompactionStatus eligibility ) { return findIndex(candidate) < Integer.MAX_VALUE - ? Eligibility.OK - : Eligibility.fail("Datasource/Interval is not in the list of 'eligibleCandidates'"); + ? CompactionMode.FULL_COMPACTION.createCandidate(candidate, eligibility) + : CompactionMode.failWithPolicyCheck( + candidate, + eligibility, + "Datasource/Interval is not in the list of 'eligibleCandidates'" + ); } - private int findIndex(CompactionCandidate candidate) + private int findIndex(CompactionCandidate.ProposedCompaction candidate) { int index = 0; for (Candidate eligibleCandidate : eligibleCandidates) { 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..da7f331991dd 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 @@ -25,6 +25,7 @@ import org.apache.druid.error.InvalidInput; import org.apache.druid.guice.annotations.UnstableApi; import org.apache.druid.java.util.common.HumanReadableBytes; +import org.apache.druid.java.util.common.StringUtils; import javax.annotation.Nullable; import java.util.Comparator; @@ -47,6 +48,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 +56,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 +73,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,10 +119,21 @@ 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() { - return this::compare; + return Comparator.comparing(o -> Objects.requireNonNull(o.getEligibility()), this::compare); } @Override @@ -124,7 +148,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,22 +163,25 @@ 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) + private int compare(CompactionStatus candidateA, CompactionStatus candidateB) { final double fragmentationDiff = computeFragmentationIndex(candidateB) - computeFragmentationIndex(candidateA); @@ -157,36 +189,56 @@ private int compare(CompactionCandidate candidateA, CompactionCandidate candidat } @Override - public Eligibility checkEligibilityForCompaction( - CompactionCandidate candidate, - CompactionTaskStatus latestTaskStatus + public CompactionCandidate createCandidate( + CompactionCandidate.ProposedCompaction candidate, + CompactionStatus eligibility ) { - final CompactionStatistics uncompacted = candidate.getUncompactedStats(); - if (uncompacted == null) { - return Eligibility.OK; - } else if (uncompacted.getNumSegments() < 1) { - return Eligibility.fail("No uncompacted segments in interval"); + final CompactionStatistics uncompacted = Objects.requireNonNull(eligibility.getUncompactedStats()); + + if (uncompacted.getNumSegments() < 1) { + return CompactionMode.failWithPolicyCheck(candidate, eligibility, "No uncompacted segments in interval"); } else if (uncompacted.getNumSegments() < minUncompactedCount) { - return Eligibility.fail( + return CompactionMode.failWithPolicyCheck( + candidate, + eligibility, "Uncompacted segments[%,d] in interval must be at least [%,d]", - uncompacted.getNumSegments(), minUncompactedCount + uncompacted.getNumSegments(), + minUncompactedCount ); } else if (uncompacted.getTotalBytes() < minUncompactedBytes.getBytes()) { - return Eligibility.fail( + return CompactionMode.failWithPolicyCheck( + candidate, + eligibility, "Uncompacted bytes[%,d] in interval must be at least [%,d]", - uncompacted.getTotalBytes(), minUncompactedBytes.getBytes() + uncompacted.getTotalBytes(), + minUncompactedBytes.getBytes() ); } final long avgSegmentSize = (uncompacted.getTotalBytes() / uncompacted.getNumSegments()); if (avgSegmentSize > maxAverageUncompactedBytesPerSegment.getBytes()) { - return Eligibility.fail( + return CompactionMode.failWithPolicyCheck( + candidate, + eligibility, "Average size[%,d] of uncompacted segments in interval must be at most [%,d]", - avgSegmentSize, maxAverageUncompactedBytesPerSegment.getBytes() + avgSegmentSize, + maxAverageUncompactedBytesPerSegment.getBytes() + ); + } + + final double uncompactedBytesRatio = (double) uncompacted.getTotalBytes() / + (uncompacted.getTotalBytes() + eligibility.getCompactedStats() + .getTotalBytes()); + if (uncompactedBytesRatio < incrementalCompactionUncompactedRatioThreshold) { + String policyNote = StringUtils.format( + "Uncompacted bytes ratio[%.2f] is below threshold[%.2f]", + uncompactedBytesRatio, + incrementalCompactionUncompactedRatioThreshold ); + return CompactionMode.INCREMENTAL_COMPACTION.createCandidate(candidate, eligibility, policyNote); } else { - return Eligibility.OK; + return CompactionMode.FULL_COMPACTION.createCandidate(candidate, eligibility); } } @@ -197,9 +249,9 @@ public Eligibility checkEligibilityForCompaction( * A higher fragmentation index causes the candidate to be higher in priority * for compaction. */ - private double computeFragmentationIndex(CompactionCandidate candidate) + private double computeFragmentationIndex(CompactionStatus eligibility) { - final CompactionStatistics uncompacted = candidate.getUncompactedStats(); + final CompactionStatistics uncompacted = eligibility.getUncompactedStats(); if (uncompacted == null || uncompacted.getNumSegments() < 1 || uncompacted.getTotalBytes() < 1) { return 0; } 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..6e4ea70e360e 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 @@ -32,8 +32,10 @@ import org.apache.druid.client.indexing.ClientCompactionTaskQuery; import org.apache.druid.client.indexing.ClientCompactionTaskQueryTuningConfig; import org.apache.druid.common.guava.FutureUtils; +import org.apache.druid.common.guava.GuavaUtils; 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; @@ -43,10 +45,10 @@ 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; -import org.apache.druid.server.compaction.CompactionStatus; import org.apache.druid.server.compaction.CompactionStatusTracker; import org.apache.druid.server.compaction.PriorityBasedCompactionSegmentIterator; import org.apache.druid.server.coordinator.AutoCompactionSnapshot; @@ -192,7 +194,6 @@ public void run( compactionSnapshotBuilder, slotManager, iterator, - policy, defaultEngine ); @@ -228,7 +229,6 @@ private int submitCompactionTasks( CompactionSnapshotBuilder snapshotBuilder, CompactionSlotManager slotManager, CompactionSegmentIterator iterator, - CompactionCandidateSearchPolicy policy, CompactionEngine defaultEngine ) { @@ -244,20 +244,18 @@ private int submitCompactionTasks( final String dataSourceName = entry.getDataSource(); final DataSourceCompactionConfig config = compactionConfigs.get(dataSourceName); - final CompactionStatus compactionStatus = - statusTracker.computeCompactionStatus(entry, policy); - final CompactionCandidate candidatesWithStatus = entry.withCurrentStatus(compactionStatus); - statusTracker.onCompactionStatusComputed(candidatesWithStatus, config); - - if (compactionStatus.isComplete()) { - snapshotBuilder.addToComplete(candidatesWithStatus); - continue; - } else if (compactionStatus.isSkipped()) { - snapshotBuilder.addToSkipped(candidatesWithStatus); - continue; - } else { - // As these segments will be compacted, we will aggregate the statistic to the Compacted statistics - snapshotBuilder.addToComplete(entry); + final CompactionCandidate.TaskState compactionTaskState = statusTracker.computeCompactionTaskState(entry); + statusTracker.onCompactionTaskStateComputed(entry, compactionTaskState, config); + + switch (compactionTaskState) { + case READY: + case TASK_IN_PROGRESS: + case RECENTLY_COMPLETED: + // As these segments will be compacted, we will aggregate the statistic to the Compacted statistics + snapshotBuilder.addToComplete(entry); + break; + default: + throw DruidException.defensive("unexpected task state[%s]", compactionTaskState); } final ClientCompactionTaskQuery taskPayload = createCompactionTask( @@ -371,8 +369,11 @@ public static ClientCompactionTaskQuery createCompactionTask( } final Map autoCompactionContext = newAutoCompactionContext(config.getTaskContext()); - if (candidate.getCurrentStatus() != null) { - autoCompactionContext.put(COMPACTION_REASON_KEY, candidate.getCurrentStatus().getReason()); + if (CompactionMode.NOT_APPLICABLE.equals(candidate.getMode())) { + autoCompactionContext.put( + COMPACTION_REASON_KEY, + GuavaUtils.firstNonNull(candidate.getPolicyNote(), candidate.getEligibility().getReason()) + ); } autoCompactionContext.put(STORE_COMPACTION_STATE_KEY, storeCompactionStatePerSegment); @@ -418,7 +419,7 @@ private void updateCompactionSnapshotStats( } iterator.getCompactedSegments().forEach(snapshotBuilder::addToComplete); iterator.getSkippedSegments().forEach(entry -> { - statusTracker.onCompactionStatusComputed(entry, datasourceToConfig.get(entry.getDataSource())); + statusTracker.onSkippedCandidate(entry, datasourceToConfig.get(entry.getDataSource())); snapshotBuilder.addToSkipped(entry); }); @@ -463,14 +464,20 @@ private static ClientCompactionTaskQuery compactSegments( context.put("priority", compactionTaskPriority); final String taskId = IdUtils.newTaskId(TASK_ID_PREFIX, ClientCompactionTaskQuery.TYPE, dataSource, null); + final ClientCompactionIntervalSpec clientCompactionIntervalSpec; + switch (entry.getMode()) { + case FULL_COMPACTION: + case INCREMENTAL_COMPACTION: + clientCompactionIntervalSpec = new ClientCompactionIntervalSpec(entry.getCompactionInterval(), null, null); + break; + default: + throw DruidException.defensive("Unexpected compaction mode[%s]", entry.getMode()); + } 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..0c4d8d5d7b45 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,24 @@ package org.apache.druid.client.indexing; -import com.google.common.collect.ImmutableList; +import com.fasterxml.jackson.databind.ObjectMapper; +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.server.compaction.CompactionCandidate.ProposedCompaction; 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 { @@ -73,7 +78,7 @@ public class ClientCompactionIntervalSpecTest public void testFromSegmentWithNoSegmentGranularity() { // The umbrella interval of segments is 2015-02-12/2015-04-14 - CompactionCandidate actual = CompactionCandidate.from(ImmutableList.of(dataSegment1, dataSegment2, dataSegment3), null); + ProposedCompaction actual = ProposedCompaction.from(List.of(dataSegment1, dataSegment2, dataSegment3), null); Assert.assertEquals(Intervals.of("2015-02-12/2015-04-14"), actual.getCompactionInterval()); } @@ -81,7 +86,7 @@ public void testFromSegmentWithNoSegmentGranularity() public void testFromSegmentWitSegmentGranularitySameAsSegment() { // The umbrella interval of segments is 2015-04-11/2015-04-12 - CompactionCandidate actual = CompactionCandidate.from(ImmutableList.of(dataSegment1), Granularities.DAY); + ProposedCompaction actual = ProposedCompaction.from(List.of(dataSegment1), Granularities.DAY); Assert.assertEquals(Intervals.of("2015-04-11/2015-04-12"), actual.getCompactionInterval()); } @@ -89,7 +94,10 @@ public void testFromSegmentWitSegmentGranularitySameAsSegment() public void testFromSegmentWithCoarserSegmentGranularity() { // The umbrella interval of segments is 2015-02-12/2015-04-14 - CompactionCandidate actual = CompactionCandidate.from(ImmutableList.of(dataSegment1, dataSegment2, dataSegment3), Granularities.YEAR); + ProposedCompaction actual = ProposedCompaction.from( + List.of(dataSegment1, dataSegment2, dataSegment3), + Granularities.YEAR + ); // The compaction interval should be expanded to start of the year and end of the year to cover the segmentGranularity Assert.assertEquals(Intervals.of("2015-01-01/2016-01-01"), actual.getCompactionInterval()); } @@ -98,7 +106,10 @@ public void testFromSegmentWithCoarserSegmentGranularity() public void testFromSegmentWithFinerSegmentGranularityAndUmbrellaIntervalAlign() { // The umbrella interval of segments is 2015-02-12/2015-04-14 - CompactionCandidate actual = CompactionCandidate.from(ImmutableList.of(dataSegment1, dataSegment2, dataSegment3), Granularities.DAY); + ProposedCompaction actual = ProposedCompaction.from( + List.of(dataSegment1, dataSegment2, dataSegment3), + Granularities.DAY + ); // The segmentGranularity of DAY align with the umbrella interval (umbrella interval can be evenly divide into the segmentGranularity) Assert.assertEquals(Intervals.of("2015-02-12/2015-04-14"), actual.getCompactionInterval()); } @@ -107,9 +118,48 @@ public void testFromSegmentWithFinerSegmentGranularityAndUmbrellaIntervalAlign() public void testFromSegmentWithFinerSegmentGranularityAndUmbrellaIntervalNotAlign() { // The umbrella interval of segments is 2015-02-12/2015-04-14 - CompactionCandidate actual = CompactionCandidate.from(ImmutableList.of(dataSegment1, dataSegment2, dataSegment3), Granularities.WEEK); + ProposedCompaction actual = ProposedCompaction.from( + List.of(dataSegment1, dataSegment2, dataSegment3), + Granularities.WEEK + ); // The segmentGranularity of WEEK does not align with the umbrella interval (umbrella interval cannot be evenly divide into the segmentGranularity) // 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 0beecb318956..99567b8cafda 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/CompactionCandidateTest.java b/server/src/test/java/org/apache/druid/server/compaction/CompactionCandidateTest.java new file mode 100644 index 000000000000..d17c7ec3a415 --- /dev/null +++ b/server/src/test/java/org/apache/druid/server/compaction/CompactionCandidateTest.java @@ -0,0 +1,121 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.server.compaction; + +import org.apache.druid.error.DruidException; +import org.apache.druid.java.util.common.granularity.Granularities; +import org.apache.druid.server.coordinator.CreateDataSegments; +import org.apache.druid.timeline.DataSegment; +import org.junit.Assert; +import org.junit.Test; + +import java.util.Collections; +import java.util.List; + +public class CompactionCandidateTest +{ + private static final String DATASOURCE = "test_datasource"; + + @Test + public void testConstructorAndGetters() + { + List segments = createTestSegments(3); + CompactionCandidate.ProposedCompaction proposed = CompactionCandidate.ProposedCompaction.from(segments, null); + CompactionStatus eligibility = CompactionStatus.notEligible("test reason"); + + CompactionCandidate candidate = new CompactionCandidate(proposed, eligibility, null, CompactionMode.FULL_COMPACTION); + + Assert.assertEquals(proposed, candidate.getProposedCompaction()); + Assert.assertEquals(eligibility, candidate.getEligibility()); + Assert.assertEquals(segments, candidate.getSegments()); + Assert.assertEquals(DATASOURCE, candidate.getDataSource()); + Assert.assertEquals(3, candidate.numSegments()); + } + + @Test + public void testProposedCompactionFrom() + { + List segments = createTestSegments(3); + + CompactionCandidate.ProposedCompaction proposed = + CompactionCandidate.ProposedCompaction.from(segments, null); + + Assert.assertEquals(segments, proposed.getSegments()); + Assert.assertEquals(DATASOURCE, proposed.getDataSource()); + Assert.assertEquals(3, proposed.numSegments()); + Assert.assertNotNull(proposed.getUmbrellaInterval()); + Assert.assertNotNull(proposed.getCompactionInterval()); + Assert.assertNotNull(proposed.getStats()); + } + + @Test + public void testProposedCompactionWithTargetGranularity() + { + List segments = createTestSegments(5); + + CompactionCandidate.ProposedCompaction proposed = + CompactionCandidate.ProposedCompaction.from(segments, Granularities.MONTH); + + Assert.assertEquals(segments, proposed.getSegments()); + Assert.assertEquals(5, proposed.numSegments()); + Assert.assertNotNull(proposed.getUmbrellaInterval()); + Assert.assertNotNull(proposed.getCompactionInterval()); + } + + @Test + public void testProposedCompactionThrowsOnNullOrEmptySegments() + { + Assert.assertThrows( + DruidException.class, + () -> CompactionCandidate.ProposedCompaction.from(null, null) + ); + + Assert.assertThrows( + DruidException.class, + () -> CompactionCandidate.ProposedCompaction.from(Collections.emptyList(), null) + ); + } + + @Test + public void testDelegationMethods() + { + List segments = createTestSegments(3); + CompactionCandidate.ProposedCompaction proposed = CompactionCandidate.ProposedCompaction.from(segments, null); + CompactionCandidate candidate = new CompactionCandidate( + proposed, + CompactionStatus.notEligible("test"), + null, + CompactionMode.FULL_COMPACTION + ); + + Assert.assertEquals(proposed.getTotalBytes(), candidate.getTotalBytes()); + Assert.assertEquals(proposed.getUmbrellaInterval(), candidate.getUmbrellaInterval()); + Assert.assertEquals(proposed.getCompactionInterval(), candidate.getCompactionInterval()); + Assert.assertEquals(proposed.getStats().getTotalBytes(), candidate.getStats().getTotalBytes()); + } + + private static List createTestSegments(int count) + { + return CreateDataSegments.ofDatasource(DATASOURCE) + .forIntervals(count, Granularities.DAY) + .startingAt("2024-01-01") + .eachOfSizeInMb(100); + } +} diff --git a/server/src/test/java/org/apache/druid/server/compaction/CompactionRunSimulatorTest.java b/server/src/test/java/org/apache/druid/server/compaction/CompactionRunSimulatorTest.java index 56ec8525bb83..c97827e6491a 100644 --- a/server/src/test/java/org/apache/druid/server/compaction/CompactionRunSimulatorTest.java +++ b/server/src/test/java/org/apache/druid/server/compaction/CompactionRunSimulatorTest.java @@ -80,13 +80,13 @@ public void testSimulateClusterCompactionConfigUpdate() Assert.assertNotNull(simulateResult); - final Map compactionStates = simulateResult.getCompactionStates(); + final Map compactionStates = simulateResult.getCompactionStates(); Assert.assertNotNull(compactionStates); - Assert.assertNull(compactionStates.get(CompactionStatus.State.COMPLETE)); - Assert.assertNull(compactionStates.get(CompactionStatus.State.RUNNING)); + Assert.assertNull(compactionStates.get(CompactionCandidate.TaskState.RECENTLY_COMPLETED)); + Assert.assertNull(compactionStates.get(CompactionCandidate.TaskState.TASK_IN_PROGRESS)); - final Table queuedTable = compactionStates.get(CompactionStatus.State.PENDING); + final Table queuedTable = compactionStates.get(CompactionCandidate.TaskState.READY); Assert.assertEquals( Arrays.asList("dataSource", "interval", "numSegments", "bytes", "maxTaskSlots", "reasonToCompact"), queuedTable.getColumnNames() @@ -106,7 +106,7 @@ public void testSimulateClusterCompactionConfigUpdate() queuedTable.getRows() ); - final Table skippedTable = compactionStates.get(CompactionStatus.State.SKIPPED); + final Table skippedTable = simulateResult.getSkippedIntervals(); Assert.assertEquals( Arrays.asList("dataSource", "interval", "numSegments", "bytes", "reasonToSkip"), skippedTable.getColumnNames() @@ -153,13 +153,13 @@ public void testSimulate_withFixedIntervalOrderPolicy() Assert.assertNotNull(simulateResult); - final Map compactionStates = simulateResult.getCompactionStates(); + final Map compactionStates = simulateResult.getCompactionStates(); Assert.assertNotNull(compactionStates); - Assert.assertNull(compactionStates.get(CompactionStatus.State.COMPLETE)); - Assert.assertNull(compactionStates.get(CompactionStatus.State.RUNNING)); + Assert.assertNull(compactionStates.get(CompactionCandidate.TaskState.RECENTLY_COMPLETED)); + Assert.assertNull(compactionStates.get(CompactionCandidate.TaskState.TASK_IN_PROGRESS)); - final Table pendingTable = compactionStates.get(CompactionStatus.State.PENDING); + final Table pendingTable = compactionStates.get(CompactionCandidate.TaskState.READY); Assert.assertEquals( List.of("dataSource", "interval", "numSegments", "bytes", "maxTaskSlots", "reasonToCompact"), pendingTable.getColumnNames() @@ -172,24 +172,23 @@ public void testSimulate_withFixedIntervalOrderPolicy() pendingTable.getRows() ); - final Table skippedTable = compactionStates.get(CompactionStatus.State.SKIPPED); + final Table skippedTable = simulateResult.getSkippedIntervals(); Assert.assertEquals( List.of("dataSource", "interval", "numSegments", "bytes", "reasonToSkip"), skippedTable.getColumnNames() ); - final String rejectedMessage - = "Rejected by search policy: Datasource/Interval is not in the list of 'eligibleCandidates'"; + final String rejectedMessage = "Datasource/Interval is not in the list of 'eligibleCandidates'"; Assert.assertEquals( List.of( - List.of("wiki", Intervals.of("2013-01-02/P1D"), 10, 1_000_000_000L, 1, rejectedMessage), - List.of("wiki", Intervals.of("2013-01-03/P1D"), 10, 1_000_000_000L, 1, rejectedMessage), + List.of("wiki", Intervals.of("2013-01-10/P1D"), 10, 1_000_000_000L, 1, "skip offset from latest[P1D]"), + List.of("wiki", Intervals.of("2013-01-09/P1D"), 10, 1_000_000_000L, 1, rejectedMessage), List.of("wiki", Intervals.of("2013-01-07/P1D"), 10, 1_000_000_000L, 1, rejectedMessage), - List.of("wiki", Intervals.of("2013-01-05/P1D"), 10, 1_000_000_000L, 1, rejectedMessage), List.of("wiki", Intervals.of("2013-01-06/P1D"), 10, 1_000_000_000L, 1, rejectedMessage), - List.of("wiki", Intervals.of("2013-01-01/P1D"), 10, 1_000_000_000L, 1, rejectedMessage), - List.of("wiki", Intervals.of("2013-01-09/P1D"), 10, 1_000_000_000L, 1, rejectedMessage), - List.of("wiki", Intervals.of("2013-01-10/P1D"), 10, 1_000_000_000L, 1, "skip offset from latest[P1D]") - ), + List.of("wiki", Intervals.of("2013-01-05/P1D"), 10, 1_000_000_000L, 1, rejectedMessage), + List.of("wiki", Intervals.of("2013-01-03/P1D"), 10, 1_000_000_000L, 1, rejectedMessage), + List.of("wiki", Intervals.of("2013-01-02/P1D"), 10, 1_000_000_000L, 1, rejectedMessage), + List.of("wiki", Intervals.of("2013-01-01/P1D"), 10, 1_000_000_000L, 1, rejectedMessage) + ), skippedTable.getRows() ); } diff --git a/server/src/test/java/org/apache/druid/server/compaction/CompactionStatusBuilderTest.java b/server/src/test/java/org/apache/druid/server/compaction/CompactionStatusBuilderTest.java new file mode 100644 index 000000000000..576489596297 --- /dev/null +++ b/server/src/test/java/org/apache/druid/server/compaction/CompactionStatusBuilderTest.java @@ -0,0 +1,183 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.server.compaction; + +import org.apache.druid.error.DruidException; +import org.apache.druid.java.util.common.granularity.Granularities; +import org.apache.druid.server.coordinator.CreateDataSegments; +import org.apache.druid.timeline.DataSegment; +import org.junit.Assert; +import org.junit.Test; + +import java.util.Collections; +import java.util.List; + +public class CompactionStatusBuilderTest +{ + private static final String DATASOURCE = "test_datasource"; + + @Test + public void testNotEligible() + { + CompactionStatus eligibility = CompactionStatus.notEligible("test reason: %s", "failure"); + + Assert.assertEquals(CompactionStatus.State.NOT_ELIGIBLE, eligibility.getState()); + Assert.assertEquals("test reason: failure", eligibility.getReason()); + Assert.assertNull(eligibility.getCompactedStats()); + Assert.assertNull(eligibility.getUncompactedStats()); + Assert.assertNull(eligibility.getUncompactedSegments()); + } + + @Test + public void testBuilderWithCompactionStats() + { + CompactionStatistics compactedStats = CompactionStatistics.create(1000, 5, 2); + CompactionStatistics uncompactedStats = CompactionStatistics.create(500, 3, 1); + List uncompactedSegments = createTestSegments(3); + + CompactionStatus eligibility = + CompactionStatus.builder(CompactionStatus.State.ELIGIBLE, "needs full compaction") + .compacted(compactedStats) + .uncompacted(uncompactedStats) + .uncompactedSegments(uncompactedSegments) + .build(); + + Assert.assertEquals(CompactionStatus.State.ELIGIBLE, eligibility.getState()); + Assert.assertEquals("needs full compaction", eligibility.getReason()); + Assert.assertEquals(compactedStats, eligibility.getCompactedStats()); + Assert.assertEquals(uncompactedStats, eligibility.getUncompactedStats()); + Assert.assertEquals(uncompactedSegments, eligibility.getUncompactedSegments()); + } + + @Test + public void testEqualsAndHashCode() + { + // Test with simple eligibility objects (same state and reason) + CompactionStatus simple1 = CompactionStatus.notEligible("reason"); + CompactionStatus simple2 = CompactionStatus.notEligible("reason"); + Assert.assertEquals(simple1, simple2); + Assert.assertEquals(simple1.hashCode(), simple2.hashCode()); + + // Test with different reasons + CompactionStatus differentReason = CompactionStatus.notEligible("different"); + Assert.assertNotEquals(simple1, differentReason); + + // Test with different states + CompactionStatus differentState = CompactionStatus.COMPLETE; + Assert.assertNotEquals(simple1, differentState); + + // Test with full compaction eligibility (with stats and segments) + CompactionStatistics stats1 = CompactionStatistics.create(1000, 5, 2); + CompactionStatistics stats2 = CompactionStatistics.create(500, 3, 1); + List segments = createTestSegments(3); + + CompactionStatus withStats1 = + CompactionStatus.builder(CompactionStatus.State.ELIGIBLE, "reason") + .compacted(stats1) + .uncompacted(stats2) + .uncompactedSegments(segments) + .build(); + + CompactionStatus withStats2 = + CompactionStatus.builder(CompactionStatus.State.ELIGIBLE, "reason") + .compacted(stats1) + .uncompacted(stats2) + .uncompactedSegments(segments) + .build(); + + // Same values - should be equal + Assert.assertEquals(withStats1, withStats2); + Assert.assertEquals(withStats1.hashCode(), withStats2.hashCode()); + + // Test with different compacted stats + CompactionStatistics differentStats = CompactionStatistics.create(2000, 10, 5); + CompactionStatus differentCompactedStats = + CompactionStatus.builder(CompactionStatus.State.ELIGIBLE, "reason") + .compacted(differentStats) + .uncompacted(stats2) + .uncompactedSegments(segments) + .build(); + Assert.assertNotEquals(withStats1, differentCompactedStats); + + // Test with different uncompacted stats + CompactionStatus differentUncompactedStats = + CompactionStatus.builder(CompactionStatus.State.ELIGIBLE, "reason") + .compacted(stats1) + .uncompacted(differentStats) + .uncompactedSegments(segments) + .build(); + Assert.assertNotEquals(withStats1, differentUncompactedStats); + + // Test with different segment lists + List differentSegments = createTestSegments(5); + CompactionStatus differentSegmentList = + CompactionStatus.builder(CompactionStatus.State.ELIGIBLE, "reason") + .compacted(stats1) + .uncompacted(stats2) + .uncompactedSegments(differentSegments) + .build(); + Assert.assertNotEquals(withStats1, differentSegmentList); + } + + @Test + public void testBuilderRequiresReasonForNotEligible() + { + Assert.assertThrows( + DruidException.class, + () -> CompactionStatus.builder(CompactionStatus.State.NOT_ELIGIBLE, null).build() + ); + } + + @Test + public void testBuilderRequiresStatsForFullCompaction() + { + Assert.assertThrows( + DruidException.class, + () -> CompactionStatus.builder(CompactionStatus.State.ELIGIBLE, "reason").build() + ); + + Assert.assertThrows( + DruidException.class, + () -> CompactionStatus.builder(CompactionStatus.State.ELIGIBLE, "reason") + .compacted(CompactionStatistics.create(1000, 5, 2)) + .build() + ); + + Assert.assertThrows( + DruidException.class, + () -> CompactionStatus.builder(CompactionStatus.State.ELIGIBLE, "reason") + .compacted(CompactionStatistics.create(1000, 5, 2)) + .uncompacted(CompactionStatistics.create(500, 3, 1)) + .build() + ); + } + + private static List createTestSegments(int count) + { + if (count == 0) { + return Collections.emptyList(); + } + + return CreateDataSegments.ofDatasource(DATASOURCE) + .forIntervals(count, Granularities.DAY) + .startingAt("2024-01-01") + .eachOfSizeInMb(100); + } +} diff --git a/server/src/test/java/org/apache/druid/server/compaction/CompactionStatusTest.java b/server/src/test/java/org/apache/druid/server/compaction/CompactionStatusTest.java index 2b274e805d4c..cb82f64dd6d5 100644 --- a/server/src/test/java/org/apache/druid/server/compaction/CompactionStatusTest.java +++ b/server/src/test/java/org/apache/druid/server/compaction/CompactionStatusTest.java @@ -99,9 +99,7 @@ public void testFindPartitionsSpecWhenGivenIsNull() { final ClientCompactionTaskQueryTuningConfig tuningConfig = ClientCompactionTaskQueryTuningConfig.from(null); - Assert.assertNull( - CompactionStatus.findPartitionsSpecFromConfig(tuningConfig) - ); + Assert.assertNull(CompactionStatus.findPartitionsSpecFromConfig(tuningConfig)); } @Test @@ -173,9 +171,7 @@ public void testFindPartitionsSpecFromConfigWithDeprecatedMaxRowsPerSegmentAndMa .build(); Assert.assertEquals( new DynamicPartitionsSpec(100, 1000L), - CompactionStatus.findPartitionsSpecFromConfig( - ClientCompactionTaskQueryTuningConfig.from(config) - ) + CompactionStatus.findPartitionsSpecFromConfig(ClientCompactionTaskQueryTuningConfig.from(config)) ); } @@ -221,7 +217,7 @@ public void testFindPartitionsSpecWhenGivenIsRangeWithTargetRows() @Test public void testStatusWhenLastCompactionStateIsNull() { - verifyCompactionStatusIsPendingBecause( + verifyCompactionIsEligibleBecause( null, InlineSchemaDataSourceCompactionConfig.builder().forDataSource(TestDataSource.WIKI).build(), "not compacted yet" @@ -232,7 +228,7 @@ public void testStatusWhenLastCompactionStateIsNull() public void testStatusWhenLastCompactionStateIsEmpty() { final PartitionsSpec requiredPartitionsSpec = new DynamicPartitionsSpec(5_000_000, null); - verifyCompactionStatusIsPendingBecause( + verifyCompactionIsEligibleBecause( new CompactionState(null, null, null, null, null, null, null), InlineSchemaDataSourceCompactionConfig .builder() @@ -257,7 +253,7 @@ public void testStatusOnPartitionsSpecMismatch() .forDataSource(TestDataSource.WIKI) .build(); - verifyCompactionStatusIsPendingBecause( + verifyCompactionIsEligibleBecause( lastCompactionState, compactionConfig, "'partitionsSpec' mismatch: required['dynamic' with 5,000,000 rows]," @@ -287,7 +283,7 @@ public void testStatusOnIndexSpecMismatch() .withTuningConfig(createTuningConfig(currentPartitionsSpec, null)) .build(); - verifyCompactionStatusIsPendingBecause( + verifyCompactionIsEligibleBecause( lastCompactionState, compactionConfig, "'indexSpec' mismatch: " @@ -329,7 +325,7 @@ public void testStatusOnSegmentGranularityMismatch() .withGranularitySpec(new UserCompactionTaskGranularityConfig(Granularities.DAY, null, null)) .build(); - verifyCompactionStatusIsPendingBecause( + verifyCompactionIsEligibleBecause( lastCompactionState, compactionConfig, "'segmentGranularity' mismatch: required[DAY], current[HOUR]" @@ -362,11 +358,11 @@ public void testStatusWhenLastCompactionStateSameAsRequired() final DataSegment segment = DataSegment.builder(WIKI_SEGMENT).lastCompactionState(lastCompactionState).build(); final CompactionStatus status = CompactionStatus.compute( - CompactionCandidate.from(List.of(segment), Granularities.HOUR), + CompactionCandidate.ProposedCompaction.from(List.of(segment), Granularities.HOUR), compactionConfig, fingerprintMapper ); - Assert.assertTrue(status.isComplete()); + Assert.assertEquals(CompactionStatus.State.COMPLETE, status.getState()); } @Test @@ -412,11 +408,11 @@ public void testStatusWhenProjectionsMatch() final DataSegment segment = DataSegment.builder(WIKI_SEGMENT).lastCompactionState(lastCompactionState).build(); final CompactionStatus status = CompactionStatus.compute( - CompactionCandidate.from(List.of(segment), Granularities.HOUR), + CompactionCandidate.ProposedCompaction.from(List.of(segment), Granularities.HOUR), compactionConfig, fingerprintMapper ); - Assert.assertTrue(status.isComplete()); + Assert.assertEquals(CompactionStatus.COMPLETE, status); } @Test @@ -467,11 +463,12 @@ public void testStatusWhenProjectionsMismatch() final DataSegment segment = DataSegment.builder(WIKI_SEGMENT).lastCompactionState(lastCompactionState).build(); final CompactionStatus status = CompactionStatus.compute( - CompactionCandidate.from(List.of(segment), Granularities.HOUR), + CompactionCandidate.ProposedCompaction.from(List.of(segment), Granularities.HOUR), compactionConfig, fingerprintMapper ); - Assert.assertFalse(status.isComplete()); + Assert.assertEquals(CompactionStatus.State.ELIGIBLE, status.getState()); + Assert.assertTrue(status.getReason().contains("'projections' mismatch")); } @Test @@ -521,11 +518,11 @@ public void testStatusWhenAutoSchemaMatch() final DataSegment segment = DataSegment.builder(WIKI_SEGMENT).lastCompactionState(lastCompactionState).build(); final CompactionStatus status = CompactionStatus.compute( - CompactionCandidate.from(List.of(segment), null), + CompactionCandidate.ProposedCompaction.from(List.of(segment), null), compactionConfig, fingerprintMapper ); - Assert.assertTrue(status.isComplete()); + Assert.assertEquals(CompactionStatus.COMPLETE, status); } @Test @@ -575,11 +572,12 @@ public void testStatusWhenAutoSchemaMismatch() final DataSegment segment = DataSegment.builder(WIKI_SEGMENT).lastCompactionState(lastCompactionState).build(); final CompactionStatus status = CompactionStatus.compute( - CompactionCandidate.from(List.of(segment), null), + CompactionCandidate.ProposedCompaction.from(List.of(segment), null), compactionConfig, fingerprintMapper ); - Assert.assertFalse(status.isComplete()); + Assert.assertEquals(CompactionStatus.State.ELIGIBLE, status.getState()); + Assert.assertTrue(status.getReason().contains("'dimensionsSpec' mismatch")); } @Test @@ -607,7 +605,7 @@ public void test_evaluate_needsCompactionWhenAllSegmentsHaveUnexpectedIndexingSt syncCacheFromManager(); verifyEvaluationNeedsCompactionBecauseWithCustomSegments( - CompactionCandidate.from(segments, null), + CompactionCandidate.ProposedCompaction.from(segments, null), compactionConfig, "'segmentGranularity' mismatch: required[DAY], current[HOUR]" ); @@ -643,7 +641,7 @@ public void test_evaluate_needsCompactionWhenSomeSegmentsHaveUnexpectedIndexingS syncCacheFromManager(); verifyEvaluationNeedsCompactionBecauseWithCustomSegments( - CompactionCandidate.from(segments, null), + CompactionCandidate.ProposedCompaction.from(segments, null), compactionConfig, "'segmentGranularity' mismatch: required[DAY], current[HOUR]" ); @@ -666,11 +664,11 @@ public void test_evaluate_noCompacationIfUnexpectedFingerprintHasExpectedIndexin syncCacheFromManager(); final CompactionStatus status = CompactionStatus.compute( - CompactionCandidate.from(segments, null), + CompactionCandidate.ProposedCompaction.from(segments, null), compactionConfig, fingerprintMapper ); - Assert.assertTrue(status.isComplete()); + Assert.assertEquals(CompactionStatus.COMPLETE, status); } @Test @@ -686,7 +684,7 @@ public void test_evaluate_needsCompactionWhenUnexpectedFingerprintAndNoFingerpri .build(); verifyEvaluationNeedsCompactionBecauseWithCustomSegments( - CompactionCandidate.from(segments, null), + CompactionCandidate.ProposedCompaction.from(segments, null), compactionConfig, "One or more fingerprinted segments do not have a cached indexing state" ); @@ -711,11 +709,11 @@ public void test_evaluate_noCompactionWhenAllSegmentsHaveExpectedIndexingStateFi ); final CompactionStatus status = CompactionStatus.compute( - CompactionCandidate.from(segments, null), + CompactionCandidate.ProposedCompaction.from(segments, null), compactionConfig, fingerprintMapper ); - Assert.assertTrue(status.isComplete()); + Assert.assertEquals(CompactionStatus.COMPLETE, status); } @Test @@ -740,7 +738,7 @@ public void test_evaluate_needsCompactionWhenNonFingerprintedSegmentsFailChecksO verifyEvaluationNeedsCompactionBecauseWithCustomSegments( - CompactionCandidate.from(segments, null), + CompactionCandidate.ProposedCompaction.from(segments, null), compactionConfig, "'segmentGranularity' mismatch: required[DAY], current[HOUR]" ); @@ -765,11 +763,11 @@ public void test_evaluate_noCompactionWhenNonFingerprintedSegmentsPassChecksOnLa ); final CompactionStatus status = CompactionStatus.compute( - CompactionCandidate.from(segments, null), + CompactionCandidate.ProposedCompaction.from(segments, null), compactionConfig, fingerprintMapper ); - Assert.assertTrue(status.isComplete()); + Assert.assertEquals(CompactionStatus.COMPLETE, status); } // ============================ @@ -795,13 +793,12 @@ public void test_evaluate_isSkippedWhenInputBytesExceedLimit() ); final CompactionStatus status = CompactionStatus.compute( - CompactionCandidate.from(segments, null), + CompactionCandidate.ProposedCompaction.from(segments, null), compactionConfig, fingerprintMapper ); - Assert.assertFalse(status.isComplete()); - Assert.assertTrue(status.isSkipped()); + Assert.assertEquals(CompactionStatus.State.NOT_ELIGIBLE, status.getState()); Assert.assertTrue(status.getReason().contains("'inputSegmentSize' exceeded")); Assert.assertTrue(status.getReason().contains("200000000")); Assert.assertTrue(status.getReason().contains("150000000")); @@ -812,22 +809,22 @@ public void test_evaluate_isSkippedWhenInputBytesExceedLimit() * Allows customization of the segments in the compaction candidate. */ private void verifyEvaluationNeedsCompactionBecauseWithCustomSegments( - CompactionCandidate candidate, + CompactionCandidate.ProposedCompaction proposedCompaction, DataSourceCompactionConfig compactionConfig, String expectedReason ) { final CompactionStatus status = CompactionStatus.compute( - candidate, + proposedCompaction, compactionConfig, fingerprintMapper ); - Assert.assertFalse(status.isComplete()); + Assert.assertEquals(CompactionStatus.State.ELIGIBLE, status.getState()); Assert.assertEquals(expectedReason, status.getReason()); } - private void verifyCompactionStatusIsPendingBecause( + private void verifyCompactionIsEligibleBecause( CompactionState lastCompactionState, DataSourceCompactionConfig compactionConfig, String expectedReason @@ -838,12 +835,12 @@ private void verifyCompactionStatusIsPendingBecause( .lastCompactionState(lastCompactionState) .build(); final CompactionStatus status = CompactionStatus.compute( - CompactionCandidate.from(List.of(segment), null), + CompactionCandidate.ProposedCompaction.from(List.of(segment), null), compactionConfig, fingerprintMapper ); - Assert.assertFalse(status.isComplete()); + Assert.assertEquals(CompactionStatus.State.ELIGIBLE, status.getState()); Assert.assertEquals(expectedReason, status.getReason()); } diff --git a/server/src/test/java/org/apache/druid/server/compaction/CompactionStatusTrackerTest.java b/server/src/test/java/org/apache/druid/server/compaction/CompactionStatusTrackerTest.java index 1314a1a0bc79..855ea57d2753 100644 --- a/server/src/test/java/org/apache/druid/server/compaction/CompactionStatusTrackerTest.java +++ b/server/src/test/java/org/apache/druid/server/compaction/CompactionStatusTrackerTest.java @@ -22,6 +22,7 @@ import org.apache.druid.indexer.TaskState; import org.apache.druid.indexer.TaskStatus; import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.segment.TestDataSource; import org.apache.druid.server.coordinator.CreateDataSegments; import org.apache.druid.timeline.DataSegment; @@ -29,6 +30,7 @@ import org.junit.Before; import org.junit.Test; +import javax.annotation.Nullable; import java.util.List; public class CompactionStatusTrackerTest @@ -47,8 +49,7 @@ public void setup() @Test public void testGetLatestTaskStatusForSubmittedTask() { - final CompactionCandidate candidateSegments - = CompactionCandidate.from(List.of(WIKI_SEGMENT), null); + final CompactionCandidate candidateSegments = createCandidate(List.of(WIKI_SEGMENT), null); statusTracker.onTaskSubmitted("task1", candidateSegments); CompactionTaskStatus status = statusTracker.getLatestTaskStatus(candidateSegments); @@ -58,8 +59,7 @@ public void testGetLatestTaskStatusForSubmittedTask() @Test public void testGetLatestTaskStatusForSuccessfulTask() { - final CompactionCandidate candidateSegments - = CompactionCandidate.from(List.of(WIKI_SEGMENT), null); + final CompactionCandidate candidateSegments = createCandidate(List.of(WIKI_SEGMENT), null); statusTracker.onTaskSubmitted("task1", candidateSegments); statusTracker.onTaskFinished("task1", TaskStatus.success("task1")); @@ -70,8 +70,7 @@ public void testGetLatestTaskStatusForSuccessfulTask() @Test public void testGetLatestTaskStatusForFailedTask() { - final CompactionCandidate candidateSegments - = CompactionCandidate.from(List.of(WIKI_SEGMENT), null); + final CompactionCandidate candidateSegments = createCandidate(List.of(WIKI_SEGMENT), null); statusTracker.onTaskSubmitted("task1", candidateSegments); statusTracker.onTaskFinished("task1", TaskStatus.failure("task1", "some failure")); @@ -83,8 +82,7 @@ public void testGetLatestTaskStatusForFailedTask() @Test public void testGetLatestTaskStatusForRepeatedlyFailingTask() { - final CompactionCandidate candidateSegments - = CompactionCandidate.from(List.of(WIKI_SEGMENT), null); + final CompactionCandidate candidateSegments = createCandidate(List.of(WIKI_SEGMENT), null); statusTracker.onTaskSubmitted("task1", candidateSegments); statusTracker.onTaskFinished("task1", TaskStatus.failure("task1", "some failure")); @@ -102,33 +100,43 @@ public void testGetLatestTaskStatusForRepeatedlyFailingTask() } @Test - public void testComputeCompactionStatusForSuccessfulTask() + public void testComputeCompactionTaskStateForSuccessfulTask() { final NewestSegmentFirstPolicy policy = new NewestSegmentFirstPolicy(null); - final CompactionCandidate candidateSegments - = CompactionCandidate.from(List.of(WIKI_SEGMENT), null); + final CompactionCandidate candidateSegments = createCandidate(List.of(WIKI_SEGMENT), null); // Verify that interval is originally eligible for compaction - CompactionStatus status - = statusTracker.computeCompactionStatus(candidateSegments, policy); - Assert.assertEquals(CompactionStatus.State.PENDING, status.getState()); - Assert.assertEquals("Not compacted yet", status.getReason()); + CompactionCandidate.TaskState status = statusTracker.computeCompactionTaskState(candidateSegments); + Assert.assertEquals(CompactionCandidate.TaskState.READY, status); // Verify that interval is skipped for compaction after task has finished statusTracker.onSegmentTimelineUpdated(DateTimes.nowUtc().minusMinutes(1)); statusTracker.onTaskSubmitted("task1", candidateSegments); statusTracker.onTaskFinished("task1", TaskStatus.success("task1")); - status = statusTracker.computeCompactionStatus(candidateSegments, policy); - Assert.assertEquals(CompactionStatus.State.SKIPPED, status.getState()); - Assert.assertEquals( - "Segment timeline not updated since last compaction task succeeded", - status.getReason() - ); + status = statusTracker.computeCompactionTaskState(candidateSegments); + Assert.assertEquals(CompactionCandidate.TaskState.RECENTLY_COMPLETED, status); // Verify that interval becomes eligible again after timeline has been updated statusTracker.onSegmentTimelineUpdated(DateTimes.nowUtc()); - status = statusTracker.computeCompactionStatus(candidateSegments, policy); - Assert.assertEquals(CompactionStatus.State.PENDING, status.getState()); + status = statusTracker.computeCompactionTaskState(candidateSegments); + Assert.assertEquals(CompactionCandidate.TaskState.READY, status); + } + + private static CompactionCandidate createCandidate( + List segments, + @Nullable Granularity targetSegmentGranularity + ) + { + CompactionCandidate.ProposedCompaction proposedCompaction = CompactionCandidate.ProposedCompaction.from( + segments, + targetSegmentGranularity + ); + CompactionStatus status = CompactionStatus.builder(CompactionStatus.State.ELIGIBLE, "approve without check") + .compacted(CompactionStatistics.create(1, 1, 1)) + .uncompacted(CompactionStatistics.create(1, 1, 1)) + .uncompactedSegments(List.of()) + .build(); + return CompactionMode.FULL_COMPACTION.createCandidate(proposedCompaction, status); } } 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..b2d7c5c7afa7 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 @@ -22,8 +22,8 @@ import nl.jqno.equalsverifier.EqualsVerifier; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.HumanReadableBytes; -import org.apache.druid.segment.TestDataSource; -import org.apache.druid.server.coordinator.CreateDataSegments; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.segment.TestSegmentUtils; import org.apache.druid.timeline.DataSegment; import org.junit.Test; import org.junit.jupiter.api.Assertions; @@ -34,12 +34,19 @@ public class MostFragmentedIntervalFirstPolicyTest { private static final DataSegment SEGMENT = - CreateDataSegments.ofDatasource(TestDataSource.WIKI).eachOfSizeInMb(100).get(0); + TestSegmentUtils.makeSegment("foo", "1", Intervals.ETERNITY); + private static final DataSegment SEGMENT2 = + TestSegmentUtils.makeSegment("foo", "2", Intervals.ETERNITY); + private static final CompactionCandidate.ProposedCompaction PROPOSED_COMPACTION = + CompactionCandidate.ProposedCompaction.from(List.of(SEGMENT, SEGMENT2), null); + + private static final CompactionStatistics DUMMY_COMPACTION_STATS = CompactionStatistics.create(1L, 1L, 1L); @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()); @@ -47,72 +54,80 @@ public void test_thresholdValues_ofDefaultPolicy() } @Test - public void test_checkEligibilityForCompaction_fails_ifUncompactedCountLessThanCutoff() + public void test_createCandidate_fails_ifUncompactedCountLessThanCutoff() { final int minUncompactedCount = 10_000; final MostFragmentedIntervalFirstPolicy policy = new MostFragmentedIntervalFirstPolicy( minUncompactedCount, HumanReadableBytes.valueOf(1), HumanReadableBytes.valueOf(10_000), + null, null ); + final CompactionStatus eligibility1 = + eligibilityBuilder().compacted(DUMMY_COMPACTION_STATS).uncompacted(createStats(1, 100L)).build(); + final CompactionCandidate candidate1 = policy.createCandidate(PROPOSED_COMPACTION, eligibility1); Assertions.assertEquals( - CompactionCandidateSearchPolicy.Eligibility.fail( - "Uncompacted segments[1] in interval must be at least [10,000]" - ), - policy.checkEligibilityForCompaction(createCandidate(1, 100L), null) - ); - Assertions.assertEquals( - CompactionCandidateSearchPolicy.Eligibility.OK, - policy.checkEligibilityForCompaction(createCandidate(10_001, 100L), null) + "Uncompacted segments[1] in interval must be at least [10,000]", + candidate1.getPolicyNote() ); + Assertions.assertEquals(CompactionMode.NOT_APPLICABLE, candidate1.getMode()); + + final CompactionStatus eligibility2 = + eligibilityBuilder().compacted(DUMMY_COMPACTION_STATS).uncompacted(createStats(10_001, 100L)).build(); + final CompactionCandidate candidate2 = policy.createCandidate(PROPOSED_COMPACTION, eligibility2); + Assertions.assertEquals(CompactionMode.FULL_COMPACTION, candidate2.getMode()); } @Test - public void test_checkEligibilityForCompaction_fails_ifUncompactedBytesLessThanCutoff() + public void test_createCandidate_fails_ifUncompactedBytesLessThanCutoff() { final HumanReadableBytes minUncompactedBytes = HumanReadableBytes.valueOf(10_000); final MostFragmentedIntervalFirstPolicy policy = new MostFragmentedIntervalFirstPolicy( 1, minUncompactedBytes, HumanReadableBytes.valueOf(10_000), + null, null ); - Assertions.assertEquals( - CompactionCandidateSearchPolicy.Eligibility.fail( - "Uncompacted bytes[100] in interval must be at least [10,000]" - ), - policy.checkEligibilityForCompaction(createCandidate(1, 100L), null) - ); - Assertions.assertEquals( - CompactionCandidateSearchPolicy.Eligibility.OK, - policy.checkEligibilityForCompaction(createCandidate(100, 10_000L), null) - ); + final CompactionStatus eligibility1 = + eligibilityBuilder().compacted(DUMMY_COMPACTION_STATS).uncompacted(createStats(1, 100L)).build(); + final CompactionCandidate candidate1 = policy.createCandidate(PROPOSED_COMPACTION, eligibility1); + Assertions.assertEquals("Uncompacted bytes[100] in interval must be at least [10,000]", candidate1.getPolicyNote()); + Assertions.assertEquals(CompactionMode.NOT_APPLICABLE, candidate1.getMode()); + + final CompactionStatus eligibility2 = + eligibilityBuilder().compacted(DUMMY_COMPACTION_STATS).uncompacted(createStats(100, 10_000L)).build(); + final CompactionCandidate candidate2 = policy.createCandidate(PROPOSED_COMPACTION, eligibility2); + Assertions.assertEquals(CompactionMode.FULL_COMPACTION, candidate2.getMode()); } @Test - public void test_checkEligibilityForCompaction_fails_ifAvgSegmentSizeGreaterThanCutoff() + public void test_createCandidate_fails_ifAvgSegmentSizeGreaterThanCutoff() { final HumanReadableBytes maxAvgSegmentSize = HumanReadableBytes.valueOf(100); final MostFragmentedIntervalFirstPolicy policy = new MostFragmentedIntervalFirstPolicy( 1, HumanReadableBytes.valueOf(100), maxAvgSegmentSize, + null, null ); + final CompactionStatus eligibility1 = + eligibilityBuilder().compacted(DUMMY_COMPACTION_STATS).uncompacted(createStats(1, 10_000L)).build(); + final CompactionCandidate candidate1 = policy.createCandidate(PROPOSED_COMPACTION, eligibility1); Assertions.assertEquals( - CompactionCandidateSearchPolicy.Eligibility.fail( - "Average size[10,000] of uncompacted segments in interval must be at most [100]" - ), - policy.checkEligibilityForCompaction(createCandidate(1, 10_000L), null) - ); - Assertions.assertEquals( - CompactionCandidateSearchPolicy.Eligibility.OK, - policy.checkEligibilityForCompaction(createCandidate(1, 100L), null) + "Average size[10,000] of uncompacted segments in interval must be at most [100]", + candidate1.getPolicyNote() ); + Assertions.assertEquals(CompactionMode.NOT_APPLICABLE, candidate1.getMode()); + final CompactionStatus eligibility2 = + eligibilityBuilder().compacted(DUMMY_COMPACTION_STATS).uncompacted(createStats(1, 100L)).build(); + final CompactionCandidate candidate2 = policy.createCandidate(PROPOSED_COMPACTION, eligibility2); + Assertions.assertEquals(CompactionMode.FULL_COMPACTION, candidate2.getMode()); } @Test @@ -122,15 +137,17 @@ public void test_policy_favorsIntervalWithMoreUncompactedSegments_ifTotalBytesIs 1, HumanReadableBytes.valueOf(1), HumanReadableBytes.valueOf(10_000), + null, null ); - final CompactionCandidate candidateA = createCandidate(1, 1000L); - final CompactionCandidate candidateB = createCandidate(2, 500L); - - verifyCandidateIsEligible(candidateA, policy); - verifyCandidateIsEligible(candidateB, policy); + final CompactionStatus eligibility1 = + eligibilityBuilder().compacted(DUMMY_COMPACTION_STATS).uncompacted(createStats(1, 1_000L)).build(); + final CompactionStatus eligibility2 = + eligibilityBuilder().compacted(DUMMY_COMPACTION_STATS).uncompacted(createStats(2, 500L)).build(); + final CompactionCandidate candidateA = policy.createCandidate(PROPOSED_COMPACTION, eligibility1); + final CompactionCandidate candidateB = policy.createCandidate(PROPOSED_COMPACTION, eligibility2); Assertions.assertTrue(policy.compareCandidates(candidateA, candidateB) > 0); Assertions.assertTrue(policy.compareCandidates(candidateB, candidateA) < 0); } @@ -142,15 +159,17 @@ public void test_policy_favorsIntervalWithMoreUncompactedSegments_ifAverageSizeI 1, HumanReadableBytes.valueOf(1), HumanReadableBytes.valueOf(10_000), + null, null ); - final CompactionCandidate candidateA = createCandidate(1, 1000L); - final CompactionCandidate candidateB = createCandidate(2, 1000L); - - verifyCandidateIsEligible(candidateA, policy); - verifyCandidateIsEligible(candidateB, policy); + final CompactionStatus eligibility1 = + eligibilityBuilder().compacted(DUMMY_COMPACTION_STATS).uncompacted(createStats(1, 1000L)).build(); + final CompactionStatus eligibility2 = + eligibilityBuilder().compacted(DUMMY_COMPACTION_STATS).uncompacted(createStats(2, 1000L)).build(); + final CompactionCandidate candidateA = policy.createCandidate(PROPOSED_COMPACTION, eligibility1); + final CompactionCandidate candidateB = policy.createCandidate(PROPOSED_COMPACTION, eligibility2); Assertions.assertTrue(policy.compareCandidates(candidateA, candidateB) > 0); Assertions.assertTrue(policy.compareCandidates(candidateB, candidateA) < 0); } @@ -162,15 +181,17 @@ public void test_policy_favorsIntervalWithSmallerSegments_ifCountIsEqual() 1, HumanReadableBytes.valueOf(1), HumanReadableBytes.valueOf(10_000), + null, null ); - final CompactionCandidate candidateA = createCandidate(10, 500L); - final CompactionCandidate candidateB = createCandidate(10, 1000L); - - verifyCandidateIsEligible(candidateA, policy); - verifyCandidateIsEligible(candidateB, policy); + final CompactionStatus eligibility1 = + eligibilityBuilder().compacted(DUMMY_COMPACTION_STATS).uncompacted(createStats(10, 500L)).build(); + final CompactionStatus eligibility2 = + eligibilityBuilder().compacted(DUMMY_COMPACTION_STATS).uncompacted(createStats(10, 1000L)).build(); + final CompactionCandidate candidateA = policy.createCandidate(PROPOSED_COMPACTION, eligibility1); + final CompactionCandidate candidateB = policy.createCandidate(PROPOSED_COMPACTION, eligibility2); Assertions.assertTrue(policy.compareCandidates(candidateA, candidateB) < 0); Assertions.assertTrue(policy.compareCandidates(candidateB, candidateA) > 0); } @@ -182,15 +203,17 @@ public void test_compareCandidates_returnsZeroIfSegmentCountAndAvgSizeScaleEquiv 100, HumanReadableBytes.valueOf(1), HumanReadableBytes.valueOf(100), + null, null ); - final CompactionCandidate candidateA = createCandidate(100, 25); - final CompactionCandidate candidateB = createCandidate(400, 100); - - verifyCandidateIsEligible(candidateA, policy); - verifyCandidateIsEligible(candidateB, policy); + final CompactionStatus eligibility1 = + eligibilityBuilder().compacted(DUMMY_COMPACTION_STATS).uncompacted(createStats(100, 25)).build(); + final CompactionStatus eligibility2 = + eligibilityBuilder().compacted(DUMMY_COMPACTION_STATS).uncompacted(createStats(400, 100)).build(); + final CompactionCandidate candidateA = policy.createCandidate(PROPOSED_COMPACTION, eligibility1); + final CompactionCandidate candidateB = policy.createCandidate(PROPOSED_COMPACTION, eligibility2); Assertions.assertEquals(0, policy.compareCandidates(candidateA, candidateB)); Assertions.assertEquals(0, policy.compareCandidates(candidateB, candidateA)); } @@ -211,6 +234,7 @@ public void test_serde_allFieldsSet() throws IOException 1, HumanReadableBytes.valueOf(2), HumanReadableBytes.valueOf(3), + 0.5, "foo" ); final DefaultObjectMapper mapper = new DefaultObjectMapper(); @@ -222,30 +246,96 @@ 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); } - private CompactionCandidate createCandidate(int numSegments, long avgSizeBytes) + @Test + public void test_createCandidate_returnsIncrementalCompaction_whenRatioBelowThreshold() { - final CompactionStatistics dummyCompactedStats = CompactionStatistics.create(1L, 1L, 1L); - final CompactionStatistics uncompactedStats = CompactionStatistics.create( - avgSizeBytes * numSegments, - numSegments, - 1L + // 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 eligibility = eligibilityBuilder() + .compacted(compacted) + .uncompacted(uncompacted) + .uncompactedSegments(List.of(SEGMENT)) + .build(); + + final CompactionCandidate candidate = policy.createCandidate(PROPOSED_COMPACTION, eligibility); + Assertions.assertEquals("Uncompacted bytes ratio[0.25] is below threshold[0.50]", candidate.getPolicyNote()); + Assertions.assertEquals(CompactionMode.INCREMENTAL_COMPACTION, candidate.getMode()); + Assertions.assertEquals( + CompactionCandidate.ProposedCompaction.from(List.of(SEGMENT), null), + candidate.getProposedCompaction() ); - return CompactionCandidate.from(List.of(SEGMENT), null) - .withCurrentStatus(CompactionStatus.pending(dummyCompactedStats, uncompactedStats, "")); } - private void verifyCandidateIsEligible(CompactionCandidate candidate, MostFragmentedIntervalFirstPolicy policy) + @Test + public void test_createCandidate_returnsFullCompaction_whenRatioAboveThreshold() { - Assertions.assertEquals( - CompactionCandidateSearchPolicy.Eligibility.OK, - policy.checkEligibilityForCompaction(candidate, null) + // Set threshold to 0.5 (50%) + final MostFragmentedIntervalFirstPolicy policy = new MostFragmentedIntervalFirstPolicy( + 1, + HumanReadableBytes.valueOf(1), + HumanReadableBytes.valueOf(10_000), + 0.5, + null ); + + final CompactionStatus eligibility = + eligibilityBuilder() + .compacted(CompactionStatistics.create(500L, 5, 1)) + .uncompacted(CompactionStatistics.create(600L, 100, 1)) + .build(); + final CompactionCandidate candidate = policy.createCandidate(PROPOSED_COMPACTION, eligibility); + + Assertions.assertEquals(CompactionMode.FULL_COMPACTION, candidate.getMode()); + } + + @Test + public void test_createCandidate_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 eligibility = + eligibilityBuilder() + .compacted(CompactionStatistics.create(1_000L, 10, 1)) + .uncompacted(CompactionStatistics.create(100L, 100, 1)) + .build(); + final CompactionCandidate candidate = policy.createCandidate(PROPOSED_COMPACTION, eligibility); + + Assertions.assertEquals(CompactionMode.FULL_COMPACTION, candidate.getMode()); + } + + private CompactionStatistics createStats(int numSegments, long avgSizeBytes) + { + return CompactionStatistics.create(avgSizeBytes * numSegments, numSegments, 1L); + } + + private static CompactionStatus.CompactionStatusBuilder eligibilityBuilder() + { + return CompactionStatus.builder(CompactionStatus.State.ELIGIBLE, "approve") + .uncompactedSegments(List.of()); } } 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 68cef0430d2f..5fec43a2fc7e 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 @@ -80,7 +80,7 @@ import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.segment.indexing.BatchIOConfig; import org.apache.druid.segment.transform.CompactionTransformSpec; -import org.apache.druid.server.compaction.CompactionCandidate; +import org.apache.druid.server.compaction.CompactionCandidate.ProposedCompaction; import org.apache.druid.server.compaction.CompactionCandidateSearchPolicy; import org.apache.druid.server.compaction.CompactionSlotManager; import org.apache.druid.server.compaction.CompactionStatusTracker; @@ -101,6 +101,7 @@ import org.apache.druid.server.coordinator.stats.Stats; import org.apache.druid.timeline.CompactionState; 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.partition.HashBasedNumberedShardSpec; @@ -217,7 +218,10 @@ public void setup() final String dataSource = DATA_SOURCE_PREFIX + i; for (int j : new int[]{0, 1, 2, 3, 7, 8}) { for (int k = 0; k < PARTITION_PER_TIME_INTERVAL; k++) { - List segmentForDatasource = datasourceToSegments.computeIfAbsent(dataSource, key -> new ArrayList<>()); + List segmentForDatasource = datasourceToSegments.computeIfAbsent( + dataSource, + key -> new ArrayList<>() + ); DataSegment dataSegment = createSegment(dataSource, j, true, k); allSegments.add(dataSegment); segmentForDatasource.add(dataSegment); @@ -250,17 +254,10 @@ private DataSegment createSegment(String dataSource, int startDay, boolean befor startDay + 2 ) ); - return new DataSegment( - dataSource, - interval, - "version", - null, - Collections.emptyList(), - Collections.emptyList(), - shardSpec, - 0, - 10L - ); + return DataSegment.builder(SegmentId.of(dataSource, interval, "version", partition)) + .shardSpec(shardSpec) + .size(10L) + .build(); } @Test @@ -865,8 +862,12 @@ public void testCompactWithGranularitySpec() .withTuningConfig(getTuningConfig(3)) .withEngine(engine) .withGranularitySpec( - new UserCompactionTaskGranularityConfig(Granularities.YEAR, null, null) - ) + new UserCompactionTaskGranularityConfig( + Granularities.YEAR, + null, + null + ) + ) .build() ); doCompactSegments(compactSegments, compactionConfigs); @@ -876,7 +877,7 @@ public void testCompactWithGranularitySpec() // All segments is compact at the same time since we changed the segment granularity to YEAR and all segment // are within the same year Assert.assertEquals( - CompactionCandidate.from(datasourceToSegments.get(dataSource), Granularities.YEAR).getCompactionInterval(), + ProposedCompaction.from(datasourceToSegments.get(dataSource), Granularities.YEAR).getCompactionInterval(), taskPayload.getIoConfig().getInputSpec().getInterval() ); @@ -901,10 +902,10 @@ public void testCompactWithDimensionSpec() .withSkipOffsetFromLatest(new Period("PT0H")) // smaller than segment interval .withTuningConfig(getTuningConfig(3)) .withDimensionsSpec( - new UserCompactionTaskDimensionsConfig( - DimensionsSpec.getDefaultSchemas(ImmutableList.of("bar", "foo")) - ) - ) + new UserCompactionTaskDimensionsConfig( + DimensionsSpec.getDefaultSchemas(ImmutableList.of("bar", "foo")) + ) + ) .withEngine(engine) .build() ); @@ -968,10 +969,10 @@ public void testCompactWithProjections() .withSkipOffsetFromLatest(new Period("PT0H")) // smaller than segment interval .withTuningConfig(getTuningConfig(3)) .withDimensionsSpec( - new UserCompactionTaskDimensionsConfig( - DimensionsSpec.getDefaultSchemas(ImmutableList.of("bar", "foo")) - ) - ) + new UserCompactionTaskDimensionsConfig( + DimensionsSpec.getDefaultSchemas(ImmutableList.of("bar", "foo")) + ) + ) .withProjections(projections) .withEngine(engine) .build() @@ -1056,8 +1057,12 @@ public void testCompactWithRollupInGranularitySpec() .withSkipOffsetFromLatest(new Period("PT0H")) // smaller than segment interval .withTuningConfig(getTuningConfig(3)) .withGranularitySpec( - new UserCompactionTaskGranularityConfig(Granularities.YEAR, null, true) - ) + new UserCompactionTaskGranularityConfig( + Granularities.YEAR, + null, + true + ) + ) .withEngine(engine) .build() ); @@ -1067,7 +1072,7 @@ public void testCompactWithRollupInGranularitySpec() // All segments is compact at the same time since we changed the segment granularity to YEAR and all segment // are within the same year Assert.assertEquals( - CompactionCandidate.from(datasourceToSegments.get(dataSource), Granularities.YEAR).getCompactionInterval(), + ProposedCompaction.from(datasourceToSegments.get(dataSource), Granularities.YEAR).getCompactionInterval(), taskPayload.getIoConfig().getInputSpec().getInterval() ); @@ -1102,6 +1107,7 @@ public void testCompactWithGranularitySpecConflictWithActiveCompactionTask() new ClientCompactionIOConfig( new ClientCompactionIntervalSpec( Intervals.of("2000/2099"), + null, "testSha256OfSortedSegmentIds" ), null @@ -1146,8 +1152,12 @@ public void testCompactWithGranularitySpecConflictWithActiveCompactionTask() .withSkipOffsetFromLatest(new Period("PT0H")) // smaller than segment interval .withTuningConfig(getTuningConfig(3)) .withGranularitySpec( - new UserCompactionTaskGranularityConfig(Granularities.YEAR, null, null) - ) + new UserCompactionTaskGranularityConfig( + Granularities.YEAR, + null, + null + ) + ) .withEngine(engine) .build() ); @@ -1162,7 +1172,7 @@ public void testCompactWithGranularitySpecConflictWithActiveCompactionTask() // All segments is compact at the same time since we changed the segment granularity to YEAR and all segment // are within the same year Assert.assertEquals( - CompactionCandidate.from(datasourceToSegments.get(dataSource), Granularities.YEAR).getCompactionInterval(), + ProposedCompaction.from(datasourceToSegments.get(dataSource), Granularities.YEAR).getCompactionInterval(), taskPayload.getIoConfig().getInputSpec().getInterval() ); @@ -1288,10 +1298,10 @@ public void testCompactWithTransformSpec() .withSkipOffsetFromLatest(new Period("PT0H")) // smaller than segment interval .withTuningConfig(getTuningConfig(3)) .withTransformSpec( - new CompactionTransformSpec( - new SelectorDimFilter("dim1", "foo", null) - ) - ) + new CompactionTransformSpec( + new SelectorDimFilter("dim1", "foo", null) + ) + ) .withEngine(engine) .build() ); @@ -1328,7 +1338,7 @@ public void testCompactWithoutCustomSpecs() @Test public void testCompactWithMetricsSpec() { - AggregatorFactory[] aggregatorFactories = new AggregatorFactory[] {new CountAggregatorFactory("cnt")}; + AggregatorFactory[] aggregatorFactories = new AggregatorFactory[]{new CountAggregatorFactory("cnt")}; final OverlordClient mockClient = Mockito.mock(OverlordClient.class); final ArgumentCaptor payloadCaptor = setUpMockClient(mockClient); final CompactSegments compactSegments = new CompactSegments(statusTracker, mockClient); @@ -1358,30 +1368,26 @@ public void testDetermineSegmentGranularityFromSegmentsToCompact() String dataSourceName = DATA_SOURCE_PREFIX + 1; List segments = new ArrayList<>(); segments.add( - new DataSegment( - dataSourceName, - Intervals.of("2017-01-01T00:00:00/2017-01-02T00:00:00"), - "1", - null, - ImmutableList.of(), - ImmutableList.of(), - shardSpecFactory.apply(0, 2), - 0, - 10L - ) + DataSegment.builder(SegmentId.of( + dataSourceName, + Intervals.of("2017-01-01T00:00:00/2017-01-02T00:00:00"), + "1", + 0 + )) + .shardSpec(shardSpecFactory.apply(0, 2)) + .size(10L) + .build() ); segments.add( - new DataSegment( - dataSourceName, - Intervals.of("2017-01-01T00:00:00/2017-01-02T00:00:00"), - "1", - null, - ImmutableList.of(), - ImmutableList.of(), - shardSpecFactory.apply(1, 2), - 0, - 10L - ) + DataSegment.builder(SegmentId.of( + dataSourceName, + Intervals.of("2017-01-01T00:00:00/2017-01-02T00:00:00"), + "1", + 1 + )) + .shardSpec(shardSpecFactory.apply(1, 2)) + .size(10L) + .build() ); dataSources = DataSourcesSnapshot.fromUsedSegments(segments); @@ -1403,7 +1409,7 @@ public void testDetermineSegmentGranularityFromSegmentsToCompact() ClientCompactionTaskQuery taskPayload = (ClientCompactionTaskQuery) payloadCaptor.getValue(); Assert.assertEquals( - CompactionCandidate.from(segments, Granularities.DAY).getCompactionInterval(), + ProposedCompaction.from(segments, Granularities.DAY).getCompactionInterval(), taskPayload.getIoConfig().getInputSpec().getInterval() ); @@ -1418,30 +1424,26 @@ public void testDetermineSegmentGranularityFromSegmentGranularityInCompactionCon String dataSourceName = DATA_SOURCE_PREFIX + 1; List segments = new ArrayList<>(); segments.add( - new DataSegment( - dataSourceName, - Intervals.of("2017-01-01T00:00:00/2017-01-02T00:00:00"), - "1", - null, - ImmutableList.of(), - ImmutableList.of(), - shardSpecFactory.apply(0, 2), - 0, - 10L - ) + DataSegment.builder(SegmentId.of( + dataSourceName, + Intervals.of("2017-01-01T00:00:00/2017-01-02T00:00:00"), + "1", + 0 + )) + .shardSpec(shardSpecFactory.apply(0, 2)) + .size(10L) + .build() ); segments.add( - new DataSegment( - dataSourceName, - Intervals.of("2017-01-01T00:00:00/2017-01-02T00:00:00"), - "1", - null, - ImmutableList.of(), - ImmutableList.of(), - shardSpecFactory.apply(1, 2), - 0, - 10L - ) + DataSegment.builder(SegmentId.of( + dataSourceName, + Intervals.of("2017-01-01T00:00:00/2017-01-02T00:00:00"), + "1", + 1 + )) + .shardSpec(shardSpecFactory.apply(1, 2)) + .size(10L) + .build() ); dataSources = DataSourcesSnapshot.fromUsedSegments(segments); @@ -1457,8 +1459,12 @@ public void testDetermineSegmentGranularityFromSegmentGranularityInCompactionCon .withSkipOffsetFromLatest(new Period("PT0H")) // smaller than segment interval .withTuningConfig(getTuningConfig(3)) .withGranularitySpec( - new UserCompactionTaskGranularityConfig(Granularities.YEAR, null, null) - ) + new UserCompactionTaskGranularityConfig( + Granularities.YEAR, + null, + null + ) + ) .withEngine(engine) .build() ); @@ -1466,7 +1472,7 @@ public void testDetermineSegmentGranularityFromSegmentGranularityInCompactionCon ClientCompactionTaskQuery taskPayload = (ClientCompactionTaskQuery) payloadCaptor.getValue(); Assert.assertEquals( - CompactionCandidate.from(segments, Granularities.YEAR).getCompactionInterval(), + ProposedCompaction.from(segments, Granularities.YEAR).getCompactionInterval(), taskPayload.getIoConfig().getInputSpec().getInterval() ); @@ -1490,7 +1496,7 @@ public void testCompactWithMetricsSpecShouldSetPreserveExistingMetricsTrue() .withInputSegmentSizeBytes(500L) .withSkipOffsetFromLatest(new Period("PT0H")) // smaller than segment interval .withTuningConfig(getTuningConfig(3)) - .withMetricsSpec(new AggregatorFactory[] {new CountAggregatorFactory("cnt")}) + .withMetricsSpec(new AggregatorFactory[]{new CountAggregatorFactory("cnt")}) .withEngine(engine) .build() ); @@ -1545,17 +1551,20 @@ private void verifySnapshot( { Map autoCompactionSnapshots = compactSegments.getAutoCompactionSnapshot(); AutoCompactionSnapshot snapshot = autoCompactionSnapshots.get(dataSourceName); - Assert.assertEquals(dataSourceName, snapshot.getDataSource()); - Assert.assertEquals(scheduleStatus, snapshot.getScheduleStatus()); - Assert.assertEquals(expectedByteCountAwaitingCompaction, snapshot.getBytesAwaitingCompaction()); - Assert.assertEquals(expectedByteCountCompressed, snapshot.getBytesCompacted()); - Assert.assertEquals(expectedByteCountSkipped, snapshot.getBytesSkipped()); - Assert.assertEquals(expectedIntervalCountAwaitingCompaction, snapshot.getIntervalCountAwaitingCompaction()); - Assert.assertEquals(expectedIntervalCountCompressed, snapshot.getIntervalCountCompacted()); - Assert.assertEquals(expectedIntervalCountSkipped, snapshot.getIntervalCountSkipped()); - Assert.assertEquals(expectedSegmentCountAwaitingCompaction, snapshot.getSegmentCountAwaitingCompaction()); - Assert.assertEquals(expectedSegmentCountCompressed, snapshot.getSegmentCountCompacted()); - Assert.assertEquals(expectedSegmentCountSkipped, snapshot.getSegmentCountSkipped()); + Assert.assertEquals(new AutoCompactionSnapshot( + dataSourceName, + scheduleStatus, + null, + expectedByteCountAwaitingCompaction, + expectedByteCountCompressed, + expectedByteCountSkipped, + expectedSegmentCountAwaitingCompaction, + expectedSegmentCountCompressed, + expectedSegmentCountSkipped, + expectedIntervalCountAwaitingCompaction, + expectedIntervalCountCompressed, + expectedIntervalCountSkipped + ), snapshot); } private void doCompactionAndAssertCompactSegmentStatistics(CompactSegments compactSegments, int compactionRunCount) @@ -1632,7 +1641,10 @@ private CoordinatorRunStats doCompactSegments(CompactSegments compactSegments) return doCompactSegments(compactSegments, (Integer) null); } - private CoordinatorRunStats doCompactSegments(CompactSegments compactSegments, @Nullable Integer numCompactionTaskSlots) + private CoordinatorRunStats doCompactSegments( + CompactSegments compactSegments, + @Nullable Integer numCompactionTaskSlots + ) { return doCompactSegments(compactSegments, createCompactionConfigs(), numCompactionTaskSlots); } @@ -1725,7 +1737,8 @@ private void assertCompactSegments( = dataSources.getUsedSegmentsTimelinesPerDataSource(); for (int i = 0; i < 3; i++) { final String dataSource = DATA_SOURCE_PREFIX + i; - List> holders = dataSourceToTimeline.get(dataSource).lookup(expectedInterval); + List> holders = + dataSourceToTimeline.get(dataSource).lookup(expectedInterval); Assert.assertEquals(1, holders.size()); List> chunks = Lists.newArrayList(holders.get(0).getObject()); Assert.assertEquals(2, chunks.size()); @@ -1821,10 +1834,10 @@ private List createCompactionConfigs( .withTuningConfig(getTuningConfig(maxNumConcurrentSubTasksForNative)) .withEngine(engine) .withTaskContext( - maxNumTasksForMSQ == null - ? null - : ImmutableMap.of(ClientMSQContext.CTX_MAX_NUM_TASKS, maxNumTasksForMSQ) - ) + maxNumTasksForMSQ == null + ? null + : Map.of(ClientMSQContext.CTX_MAX_NUM_TASKS, maxNumTasksForMSQ) + ) .build() ); } @@ -1925,7 +1938,8 @@ private void compactSegments( if (clientCompactionTaskQuery.getTuningConfig().getPartitionsSpec() instanceof DynamicPartitionsSpec) { compactionPartitionsSpec = new DynamicPartitionsSpec( clientCompactionTaskQuery.getTuningConfig().getPartitionsSpec().getMaxRowsPerSegment(), - ((DynamicPartitionsSpec) clientCompactionTaskQuery.getTuningConfig().getPartitionsSpec()).getMaxTotalRowsOr(Long.MAX_VALUE) + ((DynamicPartitionsSpec) clientCompactionTaskQuery.getTuningConfig().getPartitionsSpec()).getMaxTotalRowsOr( + Long.MAX_VALUE) ); } else { compactionPartitionsSpec = clientCompactionTaskQuery.getTuningConfig().getPartitionsSpec(); @@ -1937,40 +1951,39 @@ private void compactSegments( } for (int i = 0; i < 2; i++) { - DataSegment compactSegment = new DataSegment( - segments.get(0).getDataSource(), - compactInterval, - version, - null, - segments.get(0).getDimensions(), - segments.get(0).getMetrics(), - shardSpecFactory.apply(i, 2), - new CompactionState( - compactionPartitionsSpec, - clientCompactionTaskQuery.getDimensionsSpec() == null ? null : new DimensionsSpec( - clientCompactionTaskQuery.getDimensionsSpec().getDimensions() - ), - metricsSpec, - clientCompactionTaskQuery.getTransformSpec(), - jsonMapper.convertValue( - ImmutableMap.of( - "bitmap", - ImmutableMap.of("type", "roaring"), - "dimensionCompression", - "lz4", - "metricCompression", - "lz4", - "longEncoding", - "longs" - ), - IndexSpec.class - ), - jsonMapper.convertValue(ImmutableMap.of(), GranularitySpec.class), - null - ), - 1, - segmentSize - ); + DataSegment compactSegment = + DataSegment.builder(SegmentId.of(segments.get(0).getDataSource(), compactInterval, version, i)) + .dimensions(segments.get(0).getDimensions()) + .metrics(segments.get(0).getMetrics()) + .shardSpec(shardSpecFactory.apply(i, 2)) + .lastCompactionState( + new CompactionState( + compactionPartitionsSpec, + clientCompactionTaskQuery.getDimensionsSpec() == null ? null : new DimensionsSpec( + clientCompactionTaskQuery.getDimensionsSpec().getDimensions() + ), + metricsSpec, + clientCompactionTaskQuery.getTransformSpec(), + jsonMapper.convertValue( + ImmutableMap.of( + "bitmap", + ImmutableMap.of("type", "roaring"), + "dimensionCompression", + "lz4", + "metricCompression", + "lz4", + "longEncoding", + "longs" + ), + IndexSpec.class + ), + jsonMapper.convertValue(ImmutableMap.of(), GranularitySpec.class), + null + ) + ) + .binaryVersion(1) + .size(segmentSize) + .build(); timeline.add( compactInterval,