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 fdfe1e1430bc..2c68c0eeab6d 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionSupervisorTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionSupervisorTest.java @@ -19,11 +19,12 @@ package org.apache.druid.testing.embedded.compact; -import com.fasterxml.jackson.core.type.TypeReference; import org.apache.druid.catalog.guice.CatalogClientModule; import org.apache.druid.catalog.guice.CatalogCoordinatorModule; import org.apache.druid.common.utils.IdUtils; import org.apache.druid.data.input.impl.DimensionsSpec; +import org.apache.druid.data.input.impl.StringDimensionSchema; +import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.indexer.CompactionEngine; import org.apache.druid.indexer.partitions.DimensionRangePartitionsSpec; import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; @@ -33,20 +34,24 @@ import org.apache.druid.indexing.common.task.TaskBuilder; import org.apache.druid.indexing.compact.CascadingReindexingTemplate; import org.apache.druid.indexing.compact.CompactionSupervisorSpec; +import org.apache.druid.indexing.kafka.KafkaIndexTaskModule; +import org.apache.druid.indexing.kafka.simulate.KafkaResource; +import org.apache.druid.indexing.kafka.supervisor.KafkaSupervisorSpec; +import org.apache.druid.indexing.kafka.supervisor.KafkaSupervisorSpecBuilder; import org.apache.druid.indexing.overlord.Segments; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.java.util.common.HumanReadableBytes; import org.apache.druid.java.util.common.IAE; +import org.apache.druid.java.util.common.Numbers; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.granularity.Granularity; -import org.apache.druid.java.util.common.jackson.JacksonUtils; import org.apache.druid.java.util.emitter.service.ServiceMetricEvent; import org.apache.druid.query.DruidMetrics; import org.apache.druid.query.expression.TestExprMacroTable; import org.apache.druid.query.filter.EqualityFilter; import org.apache.druid.query.filter.NotDimFilter; -import org.apache.druid.query.http.ClientSqlQuery; import org.apache.druid.rpc.UpdateResponse; import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.column.ColumnType; @@ -55,13 +60,16 @@ import org.apache.druid.segment.metadata.IndexingStateFingerprintMapper; import org.apache.druid.segment.transform.CompactionTransformSpec; import org.apache.druid.segment.virtual.ExpressionVirtualColumn; +import org.apache.druid.server.compaction.CompactionCandidateSearchPolicy; import org.apache.druid.server.compaction.InlineReindexingRuleProvider; +import org.apache.druid.server.compaction.MostFragmentedIntervalFirstPolicy; import org.apache.druid.server.compaction.ReindexingDeletionRule; import org.apache.druid.server.compaction.ReindexingSegmentGranularityRule; import org.apache.druid.server.compaction.ReindexingTuningConfigRule; 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; @@ -76,7 +84,12 @@ 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.druid.timeline.DataSegment; +import org.apache.kafka.clients.producer.ProducerRecord; import org.hamcrest.Matcher; import org.hamcrest.Matchers; import org.joda.time.DateTime; @@ -87,17 +100,19 @@ import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.MethodSource; -import java.nio.charset.StandardCharsets; +import javax.annotation.Nullable; import java.time.Duration; import java.util.ArrayList; import java.util.List; import java.util.Map; +import java.util.stream.Collectors; /** * Embedded test that runs compaction supervisors of various types. */ public class CompactionSupervisorTest extends EmbeddedClusterTestBase { + private final KafkaResource kafkaServer = new KafkaResource(); private final EmbeddedBroker broker = new EmbeddedBroker(); private final EmbeddedIndexer indexer = new EmbeddedIndexer() .setServerMemory(2_000_000_000L) @@ -123,7 +138,12 @@ public EmbeddedDruidCluster createCluster() "[\"org.apache.druid.query.policy.NoRestrictionPolicy\"]" ) .addCommonProperty("druid.policy.enforcer.type", "restrictAllTables") - .addExtensions(CatalogClientModule.class, CatalogCoordinatorModule.class) + .addExtensions( + CatalogClientModule.class, + CatalogCoordinatorModule.class, + KafkaIndexTaskModule.class + ) + .addResource(kafkaServer) .addServer(coordinator) .addServer(overlord) .addServer(indexer) @@ -133,19 +153,21 @@ public EmbeddedDruidCluster createCluster() } - private void configureCompaction(CompactionEngine compactionEngine) + private void configureCompaction(CompactionEngine compactionEngine, @Nullable CompactionCandidateSearchPolicy policy) { final UpdateResponse updateResponse = cluster.callApi().onLeaderOverlord( - o -> o.updateClusterCompactionConfig(new ClusterCompactionConfig(1.0, 100, null, true, compactionEngine, true)) + o -> o.updateClusterCompactionConfig(new ClusterCompactionConfig(1.0, 100, policy, true, compactionEngine, true)) ); Assertions.assertTrue(updateResponse.isSuccess()); } @MethodSource("getEngine") @ParameterizedTest(name = "compactionEngine={0}") - public void test_ingestDayGranularity_andCompactToMonthGranularity_andCompactToYearGranularity_withInlineConfig(CompactionEngine compactionEngine) + public void test_ingestDayGranularity_andCompactToMonthGranularity_andCompactToYearGranularity_withInlineConfig( + CompactionEngine compactionEngine + ) { - configureCompaction(compactionEngine); + configureCompaction(compactionEngine, null); // Ingest data at DAY granularity and verify runIngestionAtGranularity( @@ -206,6 +228,129 @@ public void test_ingestDayGranularity_andCompactToMonthGranularity_andCompactToY verifyCompactedSegmentsHaveFingerprints(yearGranConfig); } + @MethodSource("getPartitionsSpec") + @ParameterizedTest(name = "partitionsSpec={0}") + public void test_minorCompactionWithMSQ(PartitionsSpec partitionsSpec) throws Exception + { + configureCompaction( + CompactionEngine.MSQ, + new MostFragmentedIntervalFirstPolicy(2, new HumanReadableBytes("1KiB"), null, 80, 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)); + + // 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); + waitForAllCompactionTasksToFinish(); + + pauseCompaction(dayGranularityConfig); + Assertions.assertEquals(0, getNumSegmentsWith(Granularities.HOUR)); + Assertions.assertEquals(1, getNumSegmentsWith(Granularities.DAY)); + Assertions.assertEquals(2000, getTotalRowCount()); + + verifyCompactedSegmentsHaveFingerprints(dayGranularityConfig); + + // published another 1k + final int appendedRowCount = publish1kRecords(topic1, true); + indexer.latchableEmitter().flush(); + waitUntilPublishedRecordsAreIngested(appendedRowCount); + + // Tear down both topics and supervisors + kafkaServer.deleteTopic(topic1); + cluster.callApi().postSupervisor(supervisor1.createSuspendedSpec()); + + long totalUsed = overlord.latchableEmitter().getMetricValues( + "segment/metadataCache/used/count", + Map.of(DruidMetrics.DATASOURCE, dataSource) + ).stream().reduce((first, second) -> second).orElse(0).longValue(); + + Assertions.assertEquals(0, getNumSegmentsWith(Granularities.HOUR)); + // 1 compacted segment + 2 appended segment + Assertions.assertEquals(3, getNumSegmentsWith(Granularities.DAY)); + Assertions.assertEquals(3000, getTotalRowCount()); + + runCompactionWithSpec(dayGranularityConfig); + waitForAllCompactionTasksToFinish(); + + // wait for new segments have been updated to the cache + overlord.latchableEmitter().waitForEvent( + event -> event.hasMetricName("segment/metadataCache/used/count") + .hasDimension(DruidMetrics.DATASOURCE, dataSource) + .hasValueMatching(Matchers.greaterThan(totalUsed))); + + // performed minor compaction: 1 previously compacted segment + 1 incrementally compacted segment + Assertions.assertEquals(2, getNumSegmentsWith(Granularities.DAY)); + Assertions.assertEquals(3000, getTotalRowCount()); + } + + 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) @@ -349,7 +494,7 @@ public void test_cascadingReindexing_withVirtualColumnOnNestedData_filtersCorrec { // Virtual Columns on nested data is only supported with MSQ compaction engine right now. CompactionEngine compactionEngine = CompactionEngine.MSQ; - configureCompaction(compactionEngine); + configureCompaction(compactionEngine, null); String jsonDataWithNestedColumn = "{\"timestamp\":\"2025-06-01T00:00:00.000Z\",\"item\":\"shirt\",\"value\":105," @@ -443,7 +588,7 @@ public void test_cascadingReindexing_withVirtualColumnOnNestedData_filtersCorrec * Tests that when a compaction task filters out all rows using a transform spec, * tombstones are created to properly drop the old segments. This test covers both * hash and range partitioning strategies. - * + *

* This regression test addresses a bug where compaction with transforms that filter * all rows would succeed but not create tombstones, leaving old segments visible * and causing indefinite compaction retries. @@ -455,7 +600,7 @@ public void test_compactionWithTransformFilteringAllRows_createsTombstones( String partitionType ) { - configureCompaction(compactionEngine); + configureCompaction(compactionEngine, null); runIngestionAtGranularity( "DAY", @@ -491,54 +636,15 @@ public void test_compactionWithTransformFilteringAllRows_createsTombstones( // Add partitioning spec based on test parameter if ("range".equals(partitionType)) { - builder.withTuningConfig( - new UserCompactionTaskQueryTuningConfig( - null, - null, - null, - null, - null, - new DimensionRangePartitionsSpec(null, 5000, List.of("item"), false), - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null - ) - ); + PartitionsSpec partitionsSpec = new DimensionRangePartitionsSpec(null, 5000, List.of("item"), false); + builder.withTuningConfig(UserCompactionTaskQueryTuningConfig.builder().partitionsSpec(partitionsSpec).build()); } else { // Hash partitioning - builder.withTuningConfig( - new UserCompactionTaskQueryTuningConfig( - null, - null, - null, - null, - null, - new HashedPartitionsSpec(null, null, null), - null, - null, - null, - null, - null, - 2, - null, - null, - null, - null, - null, - null, - null - ) - ); + PartitionsSpec partitionsSpec = new HashedPartitionsSpec(null, null, null); + builder.withTuningConfig(UserCompactionTaskQueryTuningConfig.builder() + .partitionsSpec(partitionsSpec) + .maxNumConcurrentSubTasks(2) + .build()); } InlineSchemaDataSourceCompactionConfig compactionConfig = builder.build(); @@ -562,7 +668,7 @@ public void test_compaction_legacy_string_discovery_sparse_column( ) { // test for a bug encountered where ordering contained columns not in dimensions list - configureCompaction(compactionEngine); + configureCompaction(compactionEngine, null); String jsonallnull = """ {"timestamp": "2026-03-04T00:00:00", "string":[], "another_string": "a"} @@ -607,34 +713,21 @@ public void test_compaction_legacy_string_discovery_sparse_column( private int getTotalRowCount() { - String sql = StringUtils.format("SELECT COUNT(*) as cnt FROM \"%s\"", dataSource); - String result = cluster.callApi().onAnyBroker(b -> b.submitSqlQuery(new ClientSqlQuery(sql, null, false, false, false, null, null))); - List> rows = JacksonUtils.readValue( - new DefaultObjectMapper(), - result.getBytes(StandardCharsets.UTF_8), - new TypeReference<>() {} - ); - return ((Number) rows.get(0).get("cnt")).intValue(); + return Numbers.parseInt(cluster.runSql("SELECT COUNT(*) as cnt FROM \"%s\"", dataSource)); } private void verifyNoRowsWithNestedValue(String nestedColumn, String field, String value) { - String sql = StringUtils.format( + String result = cluster.runSql( "SELECT COUNT(*) as cnt FROM \"%s\" WHERE json_value(%s, '$.%s') = '%s'", dataSource, nestedColumn, field, value ); - String result = cluster.callApi().onAnyBroker(b -> b.submitSqlQuery(new ClientSqlQuery(sql, null, false, false, false, null, null))); - List> rows = JacksonUtils.readValue( - new DefaultObjectMapper(), - result.getBytes(StandardCharsets.UTF_8), - new TypeReference<>() {} - ); Assertions.assertEquals( 0, - ((Number) rows.get(0).get("cnt")).intValue(), + Numbers.parseInt(result), StringUtils.format("Expected no rows where %s.%s = '%s'", nestedColumn, field, value) ); } @@ -702,9 +795,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() @@ -760,6 +856,14 @@ private void runIngestionAtGranularity( cluster.callApi().runTask(task, overlord); } + public static List getPartitionsSpec() + { + return List.of( + new DimensionRangePartitionsSpec(null, 5000, List.of("page"), false), + new DynamicPartitionsSpec(null, null) + ); + } + public static List getEngine() { return List.of(CompactionEngine.NATIVE, CompactionEngine.MSQ); @@ -788,34 +892,16 @@ private void verifyEventCountOlderThan(Period period, String dimension, String v DateTime now = DateTimes.nowUtc(); DateTime threshold = now.minus(period); - ClientSqlQuery query = new ClientSqlQuery( - StringUtils.format( - "SELECT COUNT(*) as cnt FROM \"%s\" WHERE %s = '%s' AND __time < MILLIS_TO_TIMESTAMP(%d)", - dataSource, - dimension, - value, - threshold.getMillis() - ), - null, - false, - false, - false, - null, - null - ); - - final String resultAsJson = cluster.callApi().onAnyBroker(b -> b.submitSqlQuery(query)); - - List> result = JacksonUtils.readValue( - new DefaultObjectMapper(), - resultAsJson.getBytes(StandardCharsets.UTF_8), - new TypeReference<>() {} + String result = cluster.runSql( + "SELECT COUNT(*) as cnt FROM \"%s\" WHERE %s = '%s' AND __time < MILLIS_TO_TIMESTAMP(%d)", + dataSource, + dimension, + value, + threshold.getMillis() ); - - Assertions.assertEquals(1, result.size()); Assertions.assertEquals( expectedCount, - result.get(0).get("cnt"), + Numbers.parseInt(result), StringUtils.format( "Expected %d events where %s='%s' older than %s", expectedCount, @@ -850,5 +936,4 @@ private UserCompactionTaskQueryTuningConfig createTuningConfigWithPartitionsSpec null ); } - } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/MarkSegmentToUpgradeAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/MarkSegmentToUpgradeAction.java new file mode 100644 index 000000000000..ed524aa667f4 --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/MarkSegmentToUpgradeAction.java @@ -0,0 +1,102 @@ +/* + * 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.error.InvalidInput; +import org.apache.druid.indexing.common.task.Task; +import org.apache.druid.metadata.ReplaceTaskLock; +import org.apache.druid.timeline.DataSegment; + +import java.util.Map; +import java.util.Set; + +/** + * Task action that records segments as being upgraded in the metadata store. + *

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

+ * The action will fail if any of the upgrade segments do not have a corresponding + * replace lock, ensuring that only properly locked segments can be marked for upgrade. + * + * @return the number of segments successfully inserted into the upgrade segments table + */ +public class MarkSegmentToUpgradeAction implements TaskAction +{ + private final String dataSource; + private final Set upgradeSegments; + + /** + * @param dataSource the datasource containing the segments to upgrade + * @param upgradeSegments the set of segments to be recorded as upgraded + */ + @JsonCreator + public MarkSegmentToUpgradeAction( + @JsonProperty("dataSource") String dataSource, + @JsonProperty("upgradeSegments") Set upgradeSegments + ) + { + this.dataSource = dataSource; + this.upgradeSegments = upgradeSegments; + } + + @JsonProperty + public String getDataSource() + { + return dataSource; + } + + @JsonProperty + public Set 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(), upgradeSegments); + + if (segmentToReplaceLock.size() < upgradeSegments.size()) { + throw InvalidInput.exception( + "Segments to upgrade must be covered by a REPLACE lock. Only [%d] out of [%d] segments are covered.", + 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..1deab619f75f 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TaskAction.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TaskAction.java @@ -37,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 = "markSegmentsToUpgrade", value = MarkSegmentToUpgradeAction.class), @JsonSubTypes.Type(name = "retrieveSegmentsById", value = RetrieveSegmentsByIdAction.class), @JsonSubTypes.Type(name = "retrieveUpgradedFromSegmentIds", value = RetrieveUpgradedFromSegmentIdsAction.class), @JsonSubTypes.Type(name = "retrieveUpgradedToSegmentIds", value = RetrieveUpgradedToSegmentIdsAction.class), diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionInputSpec.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionInputSpec.java index d73a0ffe39a0..a2eab7bde473 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionInputSpec.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionInputSpec.java @@ -34,6 +34,7 @@ @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type") @JsonSubTypes(value = { @Type(name = CompactionIntervalSpec.TYPE, value = CompactionIntervalSpec.class), + @Type(name = MinorCompactionInputSpec.TYPE, value = MinorCompactionInputSpec.class), @Type(name = SpecificSegmentsSpec.TYPE, value = SpecificSegmentsSpec.class) }) public interface CompactionInputSpec diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/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 d7a224b05858..6cbdb1dc6a40 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 @@ -55,6 +55,7 @@ import org.apache.druid.indexing.common.LockGranularity; import org.apache.druid.indexing.common.SegmentCacheManagerFactory; import org.apache.druid.indexing.common.TaskToolbox; +import org.apache.druid.indexing.common.actions.MarkSegmentToUpgradeAction; import org.apache.druid.indexing.common.actions.RetrieveUsedSegmentsAction; import org.apache.druid.indexing.common.actions.TaskActionClient; import org.apache.druid.indexing.common.task.IndexTask.IndexTuningConfig; @@ -77,7 +78,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; @@ -102,6 +107,7 @@ import org.apache.druid.server.lookup.cache.LookupLoadingSpec; import org.apache.druid.server.security.ResourceAction; import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.SegmentId; import org.apache.druid.timeline.SegmentTimeline; import org.apache.druid.timeline.TimelineObjectHolder; import org.apache.druid.timeline.VersionedIntervalTimeline; @@ -127,6 +133,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 +525,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,22 +541,26 @@ 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); } /** - * Generate dataschema for segments in each interval. + * Creates input data schemas for compaction by grouping segments and generating {@link DataSchema}s. + * When segment granularity is not specified, preserves original granularity and creates a schema + * for each unified interval. When segment granularity is specified, creates a single schema for all + * segments. For minor compaction, validates that all segments are completely within the target + * interval and submits already-compacted segments via {@link MarkSegmentToUpgradeAction} for direct upgrade. * - * @throws IOException if an exception occurs whie retrieving used segments to - * determine schemas. + * @return map from {@link QuerySegmentSpec} to {@link DataSchema} for each group of segments to compact + * @throws IOException if an exception occurs while retrieving segments */ @VisibleForTesting - static Map createDataSchemasForIntervals( + static Map createInputDataSchemas( final TaskToolbox toolbox, final LockGranularity lockGranularityInUse, final SegmentProvider segmentProvider, @@ -572,8 +583,23 @@ static Map createDataSchemasForIntervals( return Collections.emptyMap(); } + if (segmentProvider.minorCompaction) { + Iterable segmentsNotCompletelyWithinin = + Iterables.filter(timelineSegments, s -> !segmentProvider.interval.contains(s.getInterval())); + if (segmentsNotCompletelyWithinin.iterator().hasNext()) { + throw DruidException.forPersona(DruidException.Persona.USER) + .ofCategory(DruidException.Category.INVALID_INPUT) + .build( + "Minor 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 minor compaction, they need to be upgraded directly, supported in MSQ + Set segmentsToUpgrade = new HashSet<>(); // original granularity final Map> intervalToSegments = new TreeMap<>( @@ -581,8 +607,21 @@ static Map createDataSchemasForIntervals( ); for (final DataSegment dataSegment : timelineSegments) { - intervalToSegments.computeIfAbsent(dataSegment.getInterval(), k -> new ArrayList<>()) - .add(dataSegment); + if (segmentProvider.shouldUpgradeSegment(dataSegment)) { + segmentsToUpgrade.add(dataSegment); + } else { + intervalToSegments.computeIfAbsent(dataSegment.getInterval(), k -> new ArrayList<>()) + .add(dataSegment); + } + } + if (!segmentsToUpgrade.isEmpty()) { + log.info( + "Marking [%d]segments to upgrade, showing the first 10 segments:%s", + segmentsToUpgrade.size(), + segmentsToUpgrade.stream().map(DataSegment::getId).map(SegmentId::toString).limit(10L) + ); + toolbox.getTaskActionClient() + .submit(new MarkSegmentToUpgradeAction(segmentProvider.dataSource, segmentsToUpgrade)); } // unify overlapping intervals to ensure overlapping segments compacting in the same indexSpec @@ -627,18 +666,45 @@ static Map createDataSchemasForIntervals( projections, needMultiValuedColumns ); - intervalDataSchemaMap.put(interval, dataSchema); + final QuerySegmentSpec querySegmentSpec; + if (segmentProvider.minorCompaction) { + querySegmentSpec = new MultipleSpecificSegmentSpec(segmentsToCompact.stream() + .map(DataSegment::toDescriptor) + .collect(Collectors.toList())); + } else { + querySegmentSpec = new MultipleIntervalSegmentSpec(List.of(interval)); + } + inputSchemas.put(querySegmentSpec, dataSchema); } - return intervalDataSchemaMap; + return inputSchemas; } else { // given segment granularity + Set segmentsToUpgrade = new HashSet<>(); + Iterables.addAll(segmentsToUpgrade, Iterables.filter( + timelineSegments, + segmentProvider::shouldUpgradeSegment + )); + if (!segmentsToUpgrade.isEmpty()) { + log.info( + "Marking [%d]segments to upgrade, showing the first 10 segments:%s", + segmentsToUpgrade.size(), + segmentsToUpgrade.stream().map(DataSegment::getId).map(SegmentId::toString).limit(10L) + ); + toolbox.getTaskActionClient() + .submit(new MarkSegmentToUpgradeAction(segmentProvider.dataSource, segmentsToUpgrade)); + } + + final Iterable segmentsToCompact = Iterables.filter( + timelineSegments, + s -> !segmentProvider.shouldUpgradeSegment(s) + ); final DataSchema dataSchema = createDataSchema( toolbox.getEmitter(), metricBuilder, segmentProvider.dataSource, JodaUtils.umbrellaInterval(Iterables.transform(timelineSegments, DataSegment::getInterval)), lazyFetchSegments( - timelineSegments, + segmentsToCompact, toolbox.getSegmentCacheManager() ), dimensionsSpec, @@ -648,7 +714,15 @@ static Map createDataSchemasForIntervals( projections, needMultiValuedColumns ); - return Collections.singletonMap(segmentProvider.interval, dataSchema); + final QuerySegmentSpec querySegmentSpec; + if (segmentProvider.minorCompaction) { + querySegmentSpec = new MultipleSpecificSegmentSpec(StreamSupport.stream(segmentsToCompact.spliterator(), false) + .map(DataSegment::toDescriptor) + .collect(Collectors.toList())); + } else { + querySegmentSpec = new MultipleIntervalSegmentSpec(List.of(segmentProvider.interval)); + } + return Map.of(querySegmentSpec, dataSchema); } } @@ -658,8 +732,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) @@ -1218,11 +1291,30 @@ static class SegmentProvider private final CompactionInputSpec inputSpec; private final Interval interval; + private final boolean minorCompaction; + private final Set uncompactedSegments; + SegmentProvider(String dataSource, CompactionInputSpec inputSpec) { this.dataSource = Preconditions.checkNotNull(dataSource); this.inputSpec = inputSpec; this.interval = inputSpec.findInterval(dataSource); + if (inputSpec instanceof MinorCompactionInputSpec) { + minorCompaction = true; + uncompactedSegments = Set.copyOf(((MinorCompactionInputSpec) inputSpec).getUncompactedSegments()); + } else { + minorCompaction = false; + uncompactedSegments = null; + } + } + + private boolean shouldUpgradeSegment(DataSegment s) + { + if (minorCompaction) { + return !uncompactedSegments.contains(s.toDescriptor()) && this.interval.contains(s.getInterval()); + } else { + return false; + } } List findSegments(TaskActionClient actionClient) throws IOException diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/MinorCompactionInputSpec.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/MinorCompactionInputSpec.java new file mode 100644 index 000000000000..0176386bf075 --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/MinorCompactionInputSpec.java @@ -0,0 +1,132 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.common.task; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.druid.error.InvalidInput; +import org.apache.druid.indexing.common.LockGranularity; +import org.apache.druid.java.util.common.JodaUtils; +import org.apache.druid.query.SegmentDescriptor; +import org.apache.druid.timeline.DataSegment; +import org.joda.time.Interval; + +import java.util.List; +import java.util.Objects; +import java.util.stream.Collectors; + +/** + * Specifies uncompacted segments to compact within an interval. + * Used for MSQ-based minor compaction to compact only uncompacted segments while upgrading compacted segments (i.e. no change to physical segment files). + */ +public class MinorCompactionInputSpec implements CompactionInputSpec +{ + public static final String TYPE = "uncompacted"; + + private final Interval interval; + private final List uncompactedSegments; + + @JsonCreator + public MinorCompactionInputSpec( + @JsonProperty("interval") Interval interval, + @JsonProperty("uncompactedSegments") List uncompactedSegments + ) + { + InvalidInput.conditionalException(interval != null, "Uncompacted interval must not be null"); + InvalidInput.conditionalException( + interval.toDurationMillis() > 0, + "Uncompacted interval[%s] is empty, must specify a nonempty interval", + interval + ); + InvalidInput.conditionalException( + uncompactedSegments != null && !uncompactedSegments.isEmpty(), + "Uncompacted segments must not be null or empty" + ); + + // Validate that all segments are within the interval + List segmentsNotInInterval = + uncompactedSegments.stream().filter(s -> !interval.contains(s.getInterval())).collect(Collectors.toList()); + InvalidInput.conditionalException( + segmentsNotInInterval.isEmpty(), + "All uncompacted segments must be within interval[%s], got segments outside interval: %s", + interval, + segmentsNotInInterval + ); + + this.interval = interval; + this.uncompactedSegments = uncompactedSegments; + } + + @JsonProperty + public Interval getInterval() + { + return interval; + } + + @JsonProperty + public List getUncompactedSegments() + { + return uncompactedSegments; + } + + @Override + public Interval findInterval(String dataSource) + { + return interval; + } + + @Override + public boolean validateSegments(LockGranularity lockGranularityInUse, List latestSegments) + { + final Interval segmentsInterval = JodaUtils.umbrellaInterval( + latestSegments.stream().map(DataSegment::getInterval).collect(Collectors.toList()) + ); + return interval.overlaps(segmentsInterval); + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + MinorCompactionInputSpec that = (MinorCompactionInputSpec) o; + return Objects.equals(interval, that.interval) && + Objects.equals(uncompactedSegments, that.uncompactedSegments); + } + + @Override + public int hashCode() + { + return Objects.hash(interval, uncompactedSegments); + } + + @Override + public String toString() + { + return "MinorCompactionInputSpec{" + + "interval=" + interval + + ", uncompactedSegments=" + uncompactedSegments + + '}'; + } +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/NativeCompactionRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/NativeCompactionRunner.java index 559cdc1c6403..afde45cdb136 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/NativeCompactionRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/NativeCompactionRunner.java @@ -37,9 +37,11 @@ import org.apache.druid.indexing.input.DruidInputSource; import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.JodaUtils; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.query.spec.QuerySegmentSpec; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.server.coordinator.CompactionConfigValidationResult; import org.apache.druid.server.coordinator.duty.CompactSegments; @@ -86,7 +88,7 @@ public CurrentSubTaskHolder getCurrentSubTaskHolder() @Override public CompactionConfigValidationResult validateCompactionTask( CompactionTask compactionTask, - Map intervalDataSchemaMap + Map inputSchemas ) { // Virtual columns in filter rules are not supported by native compaction @@ -97,6 +99,10 @@ public CompactionConfigValidationResult validateCompactionTask( "Virtual columns in filter rules are not supported by the Native compaction engine. Use MSQ compaction engine instead." ); } + if (compactionTask.getIoConfig().getInputSpec() instanceof MinorCompactionInputSpec) { + return CompactionConfigValidationResult.failure( + "Minor compaction is not supported by Native compaction engine. Use MSQ compaction engine instead."); + } return CompactionConfigValidationResult.success(); } @@ -107,7 +113,7 @@ public CompactionConfigValidationResult validateCompactionTask( */ @VisibleForTesting static List createIngestionSpecs( - Map intervalDataSchemaMap, + Map inputSchemas, final TaskToolbox toolbox, final CompactionIOConfig ioConfig, final PartitionConfigurationManager partitionConfigurationManager, @@ -117,18 +123,18 @@ static List createIngestionSpecs( { final CompactionTask.CompactionTuningConfig compactionTuningConfig = partitionConfigurationManager.computeTuningConfig(); - return intervalDataSchemaMap.entrySet().stream().map((dataSchema) -> new ParallelIndexIngestionSpec( - dataSchema.getValue(), - createIoConfig( - toolbox, - dataSchema.getValue(), - dataSchema.getKey(), - coordinatorClient, - segmentCacheManagerFactory, - ioConfig - ), - compactionTuningConfig - ) + return inputSchemas.entrySet().stream().map((dataSchema) -> new ParallelIndexIngestionSpec( + dataSchema.getValue(), + createIoConfig( + toolbox, + dataSchema.getValue(), + JodaUtils.umbrellaInterval(dataSchema.getKey().getIntervals()), + coordinatorClient, + segmentCacheManagerFactory, + ioConfig + ), + compactionTuningConfig + ) ).collect(Collectors.toList()); } @@ -187,7 +193,7 @@ private static ParallelIndexIOConfig createIoConfig( @Override public TaskStatus runCompactionTasks( CompactionTask compactionTask, - Map intervalDataSchemaMap, + Map intervalDataSchemaMap, TaskToolbox taskToolbox ) throws Exception { @@ -329,8 +335,8 @@ static class PartitionConfigurationManager CompactionTask.CompactionTuningConfig computeTuningConfig() { CompactionTask.CompactionTuningConfig newTuningConfig = tuningConfig == null - ? CompactionTask.CompactionTuningConfig.defaultConfig() - : tuningConfig; + ? CompactionTask.CompactionTuningConfig.defaultConfig() + : tuningConfig; PartitionsSpec partitionsSpec = newTuningConfig.getGivenOrDefaultPartitionsSpec(); if (partitionsSpec instanceof DynamicPartitionsSpec) { final DynamicPartitionsSpec dynamicPartitionsSpec = (DynamicPartitionsSpec) partitionsSpec; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionConfigBasedJobTemplate.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionConfigBasedJobTemplate.java index 25643231b2c8..26fcb6354081 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionConfigBasedJobTemplate.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionConfigBasedJobTemplate.java @@ -26,6 +26,7 @@ import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.server.compaction.CompactionCandidate; +import org.apache.druid.server.compaction.CompactionCandidateSearchPolicy; import org.apache.druid.server.compaction.CompactionSlotManager; import org.apache.druid.server.compaction.DataSourceCompactibleSegmentIterator; import org.apache.druid.server.compaction.NewestSegmentFirstPolicy; @@ -91,12 +92,35 @@ public List createCompactionJobs( // Create a job for each CompactionCandidate while (segmentIterator.hasNext()) { final CompactionCandidate candidate = segmentIterator.next(); + final CompactionCandidateSearchPolicy.Eligibility eligibility = + params.getClusterCompactionConfig() + .getCompactionPolicy() + .checkEligibilityForCompaction(candidate, params.getLatestTaskStatus(candidate)); + if (!eligibility.isEligible()) { + continue; + } + final CompactionCandidate finalCandidate; + switch (eligibility.getMode()) { + case ALL_SEGMENTS: + finalCandidate = candidate; + break; + case UNCOMPACTED_SEGMENTS_ONLY: + finalCandidate = CompactionCandidate.from( + candidate.getUncompactedSegments(), + null, + candidate.getCurrentStatus() + ); + break; + default: + throw DruidException.defensive("unexpected compaction mode[%s]", eligibility.getMode()); + } // Allow template-specific customization of the config per candidate DataSourceCompactionConfig finalConfig = configOptimizer.optimizeConfig(config, candidate, params); ClientCompactionTaskQuery taskPayload = CompactSegments.createCompactionTask( - candidate, + finalCandidate, + eligibility.getMode(), finalConfig, params.getClusterCompactionConfig().getEngine(), indexingStateFingerprint, @@ -105,7 +129,7 @@ public List createCompactionJobs( jobs.add( new CompactionJob( taskPayload, - candidate, + finalCandidate, CompactionSlotManager.computeSlotsRequiredForTask(taskPayload), indexingStateFingerprint, compactionState diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionJobParams.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionJobParams.java index 36cd075922d5..b14e8734f2e5 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionJobParams.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionJobParams.java @@ -20,7 +20,9 @@ package org.apache.druid.indexing.compact; import org.apache.druid.segment.metadata.IndexingStateFingerprintMapper; +import org.apache.druid.server.compaction.CompactionCandidate; import org.apache.druid.server.compaction.CompactionSnapshotBuilder; +import org.apache.druid.server.compaction.CompactionTaskStatus; import org.apache.druid.server.coordinator.ClusterCompactionConfig; import org.apache.druid.timeline.SegmentTimeline; import org.joda.time.DateTime; @@ -32,6 +34,7 @@ public class CompactionJobParams { private final DateTime scheduleStartTime; private final TimelineProvider timelineProvider; + private final CompactionTaskStatusProvider compactionTaskStatusProvider; private final ClusterCompactionConfig clusterCompactionConfig; private final CompactionSnapshotBuilder snapshotBuilder; private final IndexingStateFingerprintMapper fingerprintMapper; @@ -40,6 +43,7 @@ public CompactionJobParams( DateTime scheduleStartTime, ClusterCompactionConfig clusterCompactionConfig, TimelineProvider timelineProvider, + CompactionTaskStatusProvider compactionTaskStatusProvider, CompactionSnapshotBuilder snapshotBuilder, IndexingStateFingerprintMapper indexingStateFingerprintMapper ) @@ -47,6 +51,7 @@ public CompactionJobParams( this.scheduleStartTime = scheduleStartTime; this.clusterCompactionConfig = clusterCompactionConfig; this.timelineProvider = timelineProvider; + this.compactionTaskStatusProvider = compactionTaskStatusProvider; this.snapshotBuilder = snapshotBuilder; this.fingerprintMapper = indexingStateFingerprintMapper; } @@ -79,6 +84,11 @@ public SegmentTimeline getTimeline(String dataSource) return timelineProvider.getTimelineForDataSource(dataSource); } + public CompactionTaskStatus getLatestTaskStatus(CompactionCandidate candidate) + { + return compactionTaskStatusProvider.getLatestTaskStatus(candidate); + } + /** * Used to build an {@link org.apache.druid.server.coordinator.AutoCompactionSnapshot} * for all the datasources at the end of the current run. During the run, as @@ -102,4 +112,10 @@ public interface TimelineProvider { SegmentTimeline getTimelineForDataSource(String dataSource); } + + @FunctionalInterface + public interface CompactionTaskStatusProvider + { + CompactionTaskStatus getLatestTaskStatus(CompactionCandidate candidate); + } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionJobQueue.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionJobQueue.java index 97cd15808f9f..1dc29cfde6a8 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionJobQueue.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionJobQueue.java @@ -25,6 +25,7 @@ import org.apache.druid.client.indexing.ClientCompactionTaskQuery; import org.apache.druid.client.indexing.ClientTaskQuery; import org.apache.druid.common.guava.FutureUtils; +import org.apache.druid.error.DruidException; import org.apache.druid.indexer.TaskState; import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexing.common.actions.TaskActionClientFactory; @@ -45,6 +46,7 @@ import org.apache.druid.server.compaction.CompactionSnapshotBuilder; import org.apache.druid.server.compaction.CompactionStatus; import org.apache.druid.server.compaction.CompactionStatusTracker; +import org.apache.druid.server.compaction.CompactionTaskStatus; import org.apache.druid.server.coordinator.AutoCompactionSnapshot; import org.apache.druid.server.coordinator.ClusterCompactionConfig; import org.apache.druid.server.coordinator.CompactionConfigValidationResult; @@ -128,6 +130,7 @@ public CompactionJobQueue( DateTimes.nowUtc(), clusterCompactionConfig, dataSourcesSnapshot.getUsedSegmentsTimelinesPerDataSource()::get, + statusTracker::getLatestTaskStatus, snapshotBuilder, new DefaultIndexingStateFingerprintMapper(indexingStateCache, objectMapper) ); @@ -217,7 +220,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 +270,6 @@ public Map getSnapshots() */ private boolean startJobIfPendingAndReady( CompactionJob job, - CompactionCandidateSearchPolicy policy, List pendingJobs, CompactionSlotManager slotManager ) @@ -281,19 +283,21 @@ private boolean startJobIfPendingAndReady( return false; } - // Check if the job is already running, completed or skipped - final CompactionStatus compactionStatus = statusTracker.computeCompactionStatus(job.getCandidate(), policy); + // Check if the job is already running or skipped or pending + final CompactionTaskStatus lastTaskStatus = statusTracker.getLatestTaskStatus(candidate); + final CompactionStatus compactionStatus = statusTracker.deriveCompactionStatus(lastTaskStatus); + switch (compactionStatus.getState()) { case RUNNING: return false; - case COMPLETE: - snapshotBuilder.moveFromPendingToCompleted(candidate); - return false; case SKIPPED: snapshotBuilder.moveFromPendingToSkipped(candidate); return false; - default: + case PENDING: break; + case COMPLETE: + default: + throw DruidException.defensive("unexpected derived compaction state[%s]", compactionStatus.getState()); } // Check if enough compaction task slots are available diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/MarkSegmentToUpgradeActionTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/MarkSegmentToUpgradeActionTest.java new file mode 100644 index 000000000000..5f298b54388d --- /dev/null +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/MarkSegmentToUpgradeActionTest.java @@ -0,0 +1,161 @@ +/* + * 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.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.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +import java.util.Set; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; + +public class MarkSegmentToUpgradeActionTest +{ + private TaskActionTestKit actionTestKit; + + @BeforeEach + public void setUp() + { + actionTestKit = new TaskActionTestKit(); + actionTestKit.before(); + } + + @AfterEach + public void tearDown() + { + actionTestKit.after(); + } + + private static final String DATA_SOURCE = "test_dataSource"; + private static final Interval INTERVAL_2026_01 = Intervals.of("2026-01-01/2026-01-02"); + 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 MarkSegmentToUpgradeAction action = new MarkSegmentToUpgradeAction(DATA_SOURCE, Set.of(SEGMENT1, SEGMENT2, SEGMENT3)); + + final Integer insertedCount = action.perform(task, actionTestKit.getTaskActionToolbox()); + assertEquals(3, insertedCount.intValue()); + final int deletedCount = actionTestKit.getMetadataStorageCoordinator().deleteUpgradeSegmentsForTask(task.getId()); + 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 MarkSegmentToUpgradeAction action = new MarkSegmentToUpgradeAction(DATA_SOURCE, Set.of(SEGMENT1, SEGMENT2, SEGMENT3)); + + DruidException exception = assertThrows( + DruidException.class, + () -> action.perform(task, actionTestKit.getTaskActionToolbox()) + ); + assertTrue(exception.getMessage().contains("Segments to upgrade must be covered 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 MarkSegmentToUpgradeAction action = new MarkSegmentToUpgradeAction(DATA_SOURCE, Set.of(SEGMENT1, SEGMENT2)); + + DruidException exception = assertThrows( + DruidException.class, + () -> action.perform(task, actionTestKit.getTaskActionToolbox()) + ); + assertTrue(exception.getMessage().contains("Segments to upgrade must be covered by a REPLACE lock")); + } + + @Test + public void test_emptySegmentsList() + { + final Task task = NoopTask.forDatasource(DATA_SOURCE); + actionTestKit.getTaskLockbox().add(task); + + final MarkSegmentToUpgradeAction action = new MarkSegmentToUpgradeAction(DATA_SOURCE, Set.of()); + + DruidException exception = assertThrows( + DruidException.class, + () -> action.perform(task, actionTestKit.getTaskActionToolbox()) + ); + 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 MarkSegmentToUpgradeAction action = new MarkSegmentToUpgradeAction(DATA_SOURCE, Set.of(SEGMENT1)); + + final Integer insertedCount = action.perform(task, actionTestKit.getTaskActionToolbox()); + assertEquals(1, insertedCount.intValue()); + final int deletedCount = actionTestKit.getMetadataStorageCoordinator().deleteUpgradeSegmentsForTask(task.getId()); + assertEquals(1, deletedCount); + } +} diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ClientCompactionTaskQuerySerdeTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ClientCompactionTaskQuerySerdeTest.java index 4136d082aa51..aa9b006dd8ae 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ClientCompactionTaskQuerySerdeTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ClientCompactionTaskQuerySerdeTest.java @@ -301,7 +301,7 @@ private ClientCompactionTaskQuery createCompactionTaskQuery(String id, Compactio id, "datasource", new ClientCompactionIOConfig( - new ClientCompactionIntervalSpec(Intervals.of("2019/2020"), "testSha256OfSortedSegmentIds"), true + new ClientCompactionIntervalSpec(Intervals.of("2019/2020"), null, "testSha256OfSortedSegmentIds"), true ), new ClientCompactionTaskQueryTuningConfig( 100, diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunBase.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunBase.java index b8f14ae1a3de..3d7e1531f306 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 @@ -1590,6 +1590,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); @@ -1607,7 +1618,7 @@ protected IndexTask buildIndexTask( tmpDir, parseSpec, null, - new UniformGranularitySpec(DEFAULT_SEGMENT_GRAN, DEFAULT_QUERY_GRAN, List.of(interval)), + new UniformGranularitySpec(segmentGranularity, DEFAULT_QUERY_GRAN, List.of(interval)), IndexTaskTest.createTuningConfig(2, 2, 2L, null, false, true), appendToExisting, false diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java index 2b20fcfa8259..3a51d5b9fe21 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java @@ -97,6 +97,7 @@ import org.apache.druid.query.aggregation.firstlast.last.DoubleLastAggregatorFactory; import org.apache.druid.query.expression.TestExprMacroTable; import org.apache.druid.query.filter.SelectorDimFilter; +import org.apache.druid.query.spec.QuerySegmentSpec; import org.apache.druid.segment.AutoTypeColumnSchema; import org.apache.druid.segment.IndexIO; import org.apache.druid.segment.IndexMergerV9; @@ -788,7 +789,7 @@ public void testSegmentProviderFindSegmentsWithEmptySegmentsThrowException() @Test public void testCreateIngestionSchema() throws IOException { - final Map dataSchemasForIntervals = CompactionTask.createDataSchemasForIntervals( + final Map inputSchemas = CompactionTask.createInputDataSchemas( toolbox, LockGranularity.TIME_CHUNK, new SegmentProvider(DATA_SOURCE, new CompactionIntervalSpec(COMPACTION_INTERVAL, null)), @@ -802,7 +803,7 @@ public void testCreateIngestionSchema() throws IOException ); final List ingestionSpecs = NativeCompactionRunner.createIngestionSpecs( - dataSchemasForIntervals, + inputSchemas, toolbox, new CompactionIOConfig(null, false, null), new PartitionConfigurationManager(TUNING_CONFIG), @@ -851,7 +852,7 @@ public void testCreateIngestionSchemaWithTargetPartitionSize() throws IOExceptio .withMaxNumConcurrentSubTasks(10) .build(); - final Map dataSchemasForIntervals = CompactionTask.createDataSchemasForIntervals( + final Map inputSchemas = CompactionTask.createInputDataSchemas( toolbox, LockGranularity.TIME_CHUNK, new SegmentProvider(DATA_SOURCE, new CompactionIntervalSpec(COMPACTION_INTERVAL, null)), @@ -865,7 +866,7 @@ public void testCreateIngestionSchemaWithTargetPartitionSize() throws IOExceptio ); final List ingestionSpecs = NativeCompactionRunner.createIngestionSpecs( - dataSchemasForIntervals, + inputSchemas, toolbox, new CompactionIOConfig(null, false, null), new PartitionConfigurationManager(tuningConfig), @@ -915,7 +916,7 @@ public void testCreateIngestionSchemaWithMaxTotalRows() throws IOException .withPushTimeout(5000L) .build(); - final Map dataSchemasForIntervals = CompactionTask.createDataSchemasForIntervals( + final Map inputSchemas = CompactionTask.createInputDataSchemas( toolbox, LockGranularity.TIME_CHUNK, new SegmentProvider(DATA_SOURCE, new CompactionIntervalSpec(COMPACTION_INTERVAL, null)), @@ -929,7 +930,7 @@ public void testCreateIngestionSchemaWithMaxTotalRows() throws IOException ); final List ingestionSpecs = NativeCompactionRunner.createIngestionSpecs( - dataSchemasForIntervals, + inputSchemas, toolbox, new CompactionIOConfig(null, false, null), new PartitionConfigurationManager(tuningConfig), @@ -980,7 +981,7 @@ public void testCreateIngestionSchemaWithNumShards() throws IOException .withMaxNumConcurrentSubTasks(10) .build(); - final Map dataSchemasForIntervals = CompactionTask.createDataSchemasForIntervals( + final Map inputSchemas = CompactionTask.createInputDataSchemas( toolbox, LockGranularity.TIME_CHUNK, new SegmentProvider(DATA_SOURCE, new CompactionIntervalSpec(COMPACTION_INTERVAL, null)), @@ -994,7 +995,7 @@ public void testCreateIngestionSchemaWithNumShards() throws IOException ); final List ingestionSpecs = NativeCompactionRunner.createIngestionSpecs( - dataSchemasForIntervals, + inputSchemas, toolbox, new CompactionIOConfig(null, false, null), new PartitionConfigurationManager(tuningConfig), @@ -1052,7 +1053,7 @@ public void testCreateIngestionSchemaWithCustomDimensionsSpec() throws IOExcepti ) ); - final Map dataSchemasForIntervals = CompactionTask.createDataSchemasForIntervals( + final Map inputSchemas = CompactionTask.createInputDataSchemas( toolbox, LockGranularity.TIME_CHUNK, new SegmentProvider(DATA_SOURCE, new CompactionIntervalSpec(COMPACTION_INTERVAL, null)), @@ -1066,7 +1067,7 @@ public void testCreateIngestionSchemaWithCustomDimensionsSpec() throws IOExcepti ); final List ingestionSpecs = NativeCompactionRunner.createIngestionSpecs( - dataSchemasForIntervals, + inputSchemas, toolbox, new CompactionIOConfig(null, false, null), new PartitionConfigurationManager(TUNING_CONFIG), @@ -1104,7 +1105,7 @@ public void testCreateIngestionSchemaWithCustomMetricsSpec() throws IOException new DoubleMaxAggregatorFactory("custom_double_max", "agg_4") }; - final Map dataSchemasForIntervals = CompactionTask.createDataSchemasForIntervals( + final Map inputSchemas = CompactionTask.createInputDataSchemas( toolbox, LockGranularity.TIME_CHUNK, new SegmentProvider(DATA_SOURCE, new CompactionIntervalSpec(COMPACTION_INTERVAL, null)), @@ -1118,7 +1119,7 @@ public void testCreateIngestionSchemaWithCustomMetricsSpec() throws IOException ); final List ingestionSpecs = NativeCompactionRunner.createIngestionSpecs( - dataSchemasForIntervals, + inputSchemas, toolbox, new CompactionIOConfig(null, false, null), new PartitionConfigurationManager(TUNING_CONFIG), @@ -1149,7 +1150,7 @@ public void testCreateIngestionSchemaWithCustomMetricsSpec() throws IOException @Test public void testCreateIngestionSchemaWithCustomSegments() throws IOException { - final Map dataSchemasForIntervals = CompactionTask.createDataSchemasForIntervals( + final Map inputSchemas = CompactionTask.createInputDataSchemas( toolbox, LockGranularity.TIME_CHUNK, new SegmentProvider(DATA_SOURCE, new CompactionIntervalSpec(COMPACTION_INTERVAL, null)), @@ -1163,7 +1164,7 @@ public void testCreateIngestionSchemaWithCustomSegments() throws IOException ); final List ingestionSpecs = NativeCompactionRunner.createIngestionSpecs( - dataSchemasForIntervals, + inputSchemas, toolbox, new CompactionIOConfig(null, false, null), new PartitionConfigurationManager(TUNING_CONFIG), @@ -1201,7 +1202,7 @@ public void testCreateIngestionSchemaWithDifferentSegmentSet() throws IOExceptio Collections.sort(segments); // Remove one segment in the middle segments.remove(segments.size() / 2); - final Map dataSchemasForIntervals = CompactionTask.createDataSchemasForIntervals( + final Map inputSchemas = CompactionTask.createInputDataSchemas( toolbox, LockGranularity.TIME_CHUNK, new SegmentProvider(DATA_SOURCE, SpecificSegmentsSpec.fromSegments(segments)), @@ -1215,7 +1216,7 @@ public void testCreateIngestionSchemaWithDifferentSegmentSet() throws IOExceptio ); NativeCompactionRunner.createIngestionSpecs( - dataSchemasForIntervals, + inputSchemas, toolbox, new CompactionIOConfig(null, false, null), new PartitionConfigurationManager(TUNING_CONFIG), @@ -1233,7 +1234,7 @@ public void testMissingMetadata() throws IOException final TestIndexIO indexIO = (TestIndexIO) toolbox.getIndexIO(); indexIO.removeMetadata(Iterables.getFirst(indexIO.getQueryableIndexMap().keySet(), null)); final List segments = new ArrayList<>(SEGMENTS); - final Map dataSchemasForIntervals = CompactionTask.createDataSchemasForIntervals( + final Map inputSchemas = CompactionTask.createInputDataSchemas( toolbox, LockGranularity.TIME_CHUNK, new SegmentProvider(DATA_SOURCE, SpecificSegmentsSpec.fromSegments(segments)), @@ -1247,7 +1248,7 @@ public void testMissingMetadata() throws IOException ); NativeCompactionRunner.createIngestionSpecs( - dataSchemasForIntervals, + inputSchemas, toolbox, new CompactionIOConfig(null, false, null), new PartitionConfigurationManager(TUNING_CONFIG), @@ -1276,7 +1277,7 @@ public void testEmptyInterval() @Test public void testSegmentGranularityAndNullQueryGranularity() throws IOException { - final Map dataSchemasForIntervals = CompactionTask.createDataSchemasForIntervals( + final Map inputSchemas = CompactionTask.createInputDataSchemas( toolbox, LockGranularity.TIME_CHUNK, new SegmentProvider(DATA_SOURCE, new CompactionIntervalSpec(COMPACTION_INTERVAL, null)), @@ -1290,7 +1291,7 @@ public void testSegmentGranularityAndNullQueryGranularity() throws IOException ); final List ingestionSpecs = NativeCompactionRunner.createIngestionSpecs( - dataSchemasForIntervals, + inputSchemas, toolbox, new CompactionIOConfig(null, false, null), new PartitionConfigurationManager(TUNING_CONFIG), @@ -1322,7 +1323,7 @@ public void testSegmentGranularityAndNullQueryGranularity() throws IOException @Test public void testQueryGranularityAndNullSegmentGranularity() throws IOException { - final Map dataSchemasForIntervals = CompactionTask.createDataSchemasForIntervals( + final Map inputSchemas = CompactionTask.createInputDataSchemas( toolbox, LockGranularity.TIME_CHUNK, new SegmentProvider(DATA_SOURCE, new CompactionIntervalSpec(COMPACTION_INTERVAL, null)), @@ -1335,7 +1336,7 @@ public void testQueryGranularityAndNullSegmentGranularity() throws IOException false ); final List ingestionSpecs = NativeCompactionRunner.createIngestionSpecs( - dataSchemasForIntervals, + inputSchemas, toolbox, new CompactionIOConfig(null, false, null), new PartitionConfigurationManager(TUNING_CONFIG), @@ -1365,7 +1366,7 @@ public void testQueryGranularityAndNullSegmentGranularity() throws IOException @Test public void testQueryGranularityAndSegmentGranularityNonNull() throws IOException { - final Map dataSchemasForIntervals = CompactionTask.createDataSchemasForIntervals( + final Map inputSchemas = CompactionTask.createInputDataSchemas( toolbox, LockGranularity.TIME_CHUNK, new SegmentProvider(DATA_SOURCE, new CompactionIntervalSpec(COMPACTION_INTERVAL, null)), @@ -1383,7 +1384,7 @@ public void testQueryGranularityAndSegmentGranularityNonNull() throws IOExceptio ); final List ingestionSpecs = NativeCompactionRunner.createIngestionSpecs( - dataSchemasForIntervals, + inputSchemas, toolbox, new CompactionIOConfig(null, false, null), new PartitionConfigurationManager(TUNING_CONFIG), @@ -1418,7 +1419,7 @@ public void testQueryGranularityAndSegmentGranularityNonNull() throws IOExceptio @Test public void testNullGranularitySpec() throws IOException { - final Map dataSchemasForIntervals = CompactionTask.createDataSchemasForIntervals( + final Map inputSchemas = CompactionTask.createInputDataSchemas( toolbox, LockGranularity.TIME_CHUNK, new SegmentProvider(DATA_SOURCE, new CompactionIntervalSpec(COMPACTION_INTERVAL, null)), @@ -1432,7 +1433,7 @@ public void testNullGranularitySpec() throws IOException ); final List ingestionSpecs = NativeCompactionRunner.createIngestionSpecs( - dataSchemasForIntervals, + inputSchemas, toolbox, new CompactionIOConfig(null, false, null), new PartitionConfigurationManager(TUNING_CONFIG), @@ -1465,7 +1466,7 @@ public void testNullGranularitySpec() throws IOException public void testGranularitySpecWithNullQueryGranularityAndNullSegmentGranularity() throws IOException { - final Map dataSchemasForIntervals = CompactionTask.createDataSchemasForIntervals( + final Map inputSchemas = CompactionTask.createInputDataSchemas( toolbox, LockGranularity.TIME_CHUNK, new SegmentProvider(DATA_SOURCE, new CompactionIntervalSpec(COMPACTION_INTERVAL, null)), @@ -1479,7 +1480,7 @@ public void testGranularitySpecWithNullQueryGranularityAndNullSegmentGranularity ); final List ingestionSpecs = NativeCompactionRunner.createIngestionSpecs( - dataSchemasForIntervals, + inputSchemas, toolbox, new CompactionIOConfig(null, false, null), new PartitionConfigurationManager(TUNING_CONFIG), @@ -1512,7 +1513,7 @@ public void testGranularitySpecWithNullQueryGranularityAndNullSegmentGranularity public void testGranularitySpecWithNotNullRollup() throws IOException { - final Map dataSchemasForIntervals = CompactionTask.createDataSchemasForIntervals( + final Map inputSchemas = CompactionTask.createInputDataSchemas( toolbox, LockGranularity.TIME_CHUNK, new SegmentProvider(DATA_SOURCE, new CompactionIntervalSpec(COMPACTION_INTERVAL, null)), @@ -1526,7 +1527,7 @@ public void testGranularitySpecWithNotNullRollup() ); final List ingestionSpecs = NativeCompactionRunner.createIngestionSpecs( - dataSchemasForIntervals, + inputSchemas, toolbox, new CompactionIOConfig(null, false, null), new PartitionConfigurationManager(TUNING_CONFIG), @@ -1544,7 +1545,7 @@ public void testGranularitySpecWithNotNullRollup() public void testGranularitySpecWithNullRollup() throws IOException { - final Map dataSchemasForIntervals = CompactionTask.createDataSchemasForIntervals( + final Map inputSchemas = CompactionTask.createInputDataSchemas( toolbox, LockGranularity.TIME_CHUNK, new SegmentProvider(DATA_SOURCE, new CompactionIntervalSpec(COMPACTION_INTERVAL, null)), @@ -1558,7 +1559,7 @@ public void testGranularitySpecWithNullRollup() ); final List ingestionSpecs = NativeCompactionRunner.createIngestionSpecs( - dataSchemasForIntervals, + inputSchemas, toolbox, new CompactionIOConfig(null, false, null), new PartitionConfigurationManager(TUNING_CONFIG), @@ -1578,7 +1579,7 @@ public void testGranularitySpecWithNullRollup() public void testMultiValuedDimensionsProcessing() throws IOException { - final Map dataSchemasForIntervals = CompactionTask.createDataSchemasForIntervals( + final Map inputSchemas = CompactionTask.createInputDataSchemas( toolbox, LockGranularity.TIME_CHUNK, new SegmentProvider(DATA_SOURCE, new CompactionIntervalSpec(COMPACTION_INTERVAL, null)), @@ -1590,7 +1591,7 @@ public void testMultiValuedDimensionsProcessing() METRIC_BUILDER, true ); - for (DataSchema dataSchema : dataSchemasForIntervals.values()) { + for (DataSchema dataSchema : inputSchemas.values()) { Assert.assertTrue(dataSchema instanceof CombinedDataSchema); Assert.assertTrue(((CombinedDataSchema) dataSchema).getMultiValuedDimensions().isEmpty()); } @@ -2220,7 +2221,7 @@ private static class TestMSQCompactionRunner implements CompactionRunner @Override public TaskStatus runCompactionTasks( CompactionTask compactionTask, - Map intervalDataSchemaMap, + Map inputSchemas, TaskToolbox taskToolbox ) { @@ -2236,7 +2237,7 @@ public CurrentSubTaskHolder getCurrentSubTaskHolder() @Override public CompactionConfigValidationResult validateCompactionTask( CompactionTask compactionTask, - Map intervalToDataSchemaMap + Map inputSchemas ) { return null; diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/MinorCompactionInputSpecTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/MinorCompactionInputSpecTest.java new file mode 100644 index 000000000000..c5a6f2fda033 --- /dev/null +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/MinorCompactionInputSpecTest.java @@ -0,0 +1,104 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.common.task; + +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.error.DruidException; +import org.apache.druid.jackson.DefaultObjectMapper; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.query.SegmentDescriptor; +import org.joda.time.Interval; +import org.junit.Assert; +import org.junit.Test; + +import java.util.List; + +public class MinorCompactionInputSpecTest +{ + @Test + public void testSerde() throws Exception + { + ObjectMapper mapper = new DefaultObjectMapper(); + Interval interval = Intervals.of("2015-04-11/2015-04-12"); + List segments = List.of( + new SegmentDescriptor(Intervals.of("2015-04-11/2015-04-12"), "v1", 0) + ); + + MinorCompactionInputSpec spec = new MinorCompactionInputSpec(interval, segments); + String json = mapper.writeValueAsString(spec); + MinorCompactionInputSpec deserialized = mapper.readValue(json, MinorCompactionInputSpec.class); + + Assert.assertEquals(spec, deserialized); + Assert.assertEquals(interval, deserialized.getInterval()); + Assert.assertEquals(segments, deserialized.getUncompactedSegments()); + } + + @Test + public void testDeserializeFromClientFormat() throws Exception + { + ObjectMapper mapper = new DefaultObjectMapper(); + String clientJson = "{" + + "\"type\":\"uncompacted\"," + + "\"interval\":\"2015-04-11/2015-04-12\"," + + "\"uncompactedSegments\":[{\"itvl\":\"2015-04-11/2015-04-12\",\"ver\":\"v1\",\"part\":0}]" + + "}"; + + MinorCompactionInputSpec deserialized = mapper.readValue(clientJson, MinorCompactionInputSpec.class); + + Assert.assertEquals(Intervals.of("2015-04-11/2015-04-12"), deserialized.getInterval()); + Assert.assertEquals(1, deserialized.getUncompactedSegments().size()); + Assert.assertEquals( + new SegmentDescriptor(Intervals.of("2015-04-11/2015-04-12"), "v1", 0), + deserialized.getUncompactedSegments().get(0) + ); + } + + @Test + public void testThrowsExceptionWhenInvalidInterval() + { + List segments = List.of( + new SegmentDescriptor(Intervals.of("2015-04-11/2015-04-12"), "v1", 0) + ); + + Assert.assertThrows(DruidException.class, () -> new MinorCompactionInputSpec(null, segments)); + + Interval emptyInterval = Intervals.of("2015-04-11/2015-04-11"); + Assert.assertThrows(DruidException.class, () -> new MinorCompactionInputSpec(emptyInterval, segments)); + } + + @Test + public void testThrowsExceptionWhenInvalidSegments() + { + Interval interval = Intervals.of("2015-04-11/2015-04-12"); + Assert.assertThrows(DruidException.class, () -> new MinorCompactionInputSpec(interval, null)); + Assert.assertThrows(DruidException.class, () -> new MinorCompactionInputSpec(interval, List.of())); + } + + @Test + public void testThrowsExceptionWhenSegmentsOutsideInterval() + { + Interval interval = Intervals.of("2015-04-11/2015-04-12"); + List segments = List.of( + new SegmentDescriptor(Intervals.of("2015-05-11/2015-05-12"), "v1", 0) + ); + + Assert.assertThrows(DruidException.class, () -> new MinorCompactionInputSpec(interval, segments)); + } +} diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/NativeCompactionRunnerTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/NativeCompactionRunnerTest.java index dcb801ab4ecf..f13e98d3051b 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/NativeCompactionRunnerTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/NativeCompactionRunnerTest.java @@ -23,13 +23,13 @@ import org.apache.druid.indexing.common.SegmentCacheManagerFactory; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.query.expression.TestExprMacroTable; +import org.apache.druid.query.spec.QuerySegmentSpec; import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.transform.CompactionTransformSpec; import org.apache.druid.segment.virtual.ExpressionVirtualColumn; import org.apache.druid.server.coordinator.CompactionConfigValidationResult; -import org.joda.time.Interval; import org.junit.Assert; import org.junit.Test; import org.mockito.Mockito; @@ -60,11 +60,11 @@ public void testVirtualColumnsInTransformSpecAreNotSupported() CompactionTransformSpec transformSpec = new CompactionTransformSpec(null, virtualColumns); CompactionTask compactionTask = createCompactionTask(transformSpec); - Map intervalDataschemas = Collections.emptyMap(); + Map inputSchemas = Collections.emptyMap(); CompactionConfigValidationResult validationResult = NATIVE_COMPACTION_RUNNER.validateCompactionTask( compactionTask, - intervalDataschemas + inputSchemas ); Assert.assertFalse(validationResult.isValid()); @@ -78,11 +78,11 @@ public void testVirtualColumnsInTransformSpecAreNotSupported() public void testNoVirtualColumnsIsValid() { CompactionTask compactionTask = createCompactionTask(null); - Map intervalDataschemas = Collections.emptyMap(); + Map inputSchemas = Collections.emptyMap(); CompactionConfigValidationResult validationResult = NATIVE_COMPACTION_RUNNER.validateCompactionTask( compactionTask, - intervalDataschemas + inputSchemas ); Assert.assertTrue(validationResult.isValid()); @@ -94,11 +94,11 @@ public void testEmptyVirtualColumnsIsValid() CompactionTransformSpec transformSpec = new CompactionTransformSpec(null, VirtualColumns.EMPTY); CompactionTask compactionTask = createCompactionTask(transformSpec); - Map intervalDataschemas = Collections.emptyMap(); + Map inputSchemas = Collections.emptyMap(); CompactionConfigValidationResult validationResult = NATIVE_COMPACTION_RUNNER.validateCompactionTask( compactionTask, - intervalDataschemas + inputSchemas ); Assert.assertTrue(validationResult.isValid()); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/test/TestIndexerMetadataStorageCoordinator.java b/indexing-service/src/test/java/org/apache/druid/indexing/test/TestIndexerMetadataStorageCoordinator.java index ee9e85204e07..00cddb943c79 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/test/TestIndexerMetadataStorageCoordinator.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/test/TestIndexerMetadataStorageCoordinator.java @@ -125,7 +125,10 @@ public Set retrieveAllUsedSegments(String dataSource, Segments visi } @Override - public List> retrieveUsedSegmentsAndCreatedDates(String dataSource, List intervals) + public List> retrieveUsedSegmentsAndCreatedDates( + String dataSource, + List intervals + ) { return List.of(); } @@ -259,6 +262,14 @@ public SegmentPublishResult commitAppendSegmentsAndMetadata( return SegmentPublishResult.ok(commitSegments(appendSegments, segmentSchemaMapping)); } + @Override + public int insertIntoUpgradeSegmentsTable( + Map segmentToReplaceLock + ) + { + throw new UnsupportedOperationException("not implemented"); + } + @Override public SegmentPublishResult commitSegmentsAndMetadata( Set segments, diff --git a/integration-tests-ex/tools/src/main/java/org/apache/druid/testing/tools/WikipediaStreamEventStreamGenerator.java b/integration-tests-ex/tools/src/main/java/org/apache/druid/testing/tools/WikipediaStreamEventStreamGenerator.java index b26c999ed83e..28ecfdf06941 100644 --- a/integration-tests-ex/tools/src/main/java/org/apache/druid/testing/tools/WikipediaStreamEventStreamGenerator.java +++ b/integration-tests-ex/tools/src/main/java/org/apache/druid/testing/tools/WikipediaStreamEventStreamGenerator.java @@ -68,4 +68,22 @@ List> newEvent(int i, DateTime timestamp) event.add(Pair.of("delta", i)); return Collections.unmodifiableList(event); } + + public static List dimensions() + { + return List.of( + "page", + "language", + "user", + "unpatrolled", + "newPage", + "robot", + "anonymous", + COL_UNIQUE_NAMESPACE, + "continent", + "country", + "region", + "city" + ); + } } diff --git a/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java b/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java index e65a1994f4aa..07dec885a77f 100644 --- a/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java +++ b/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java @@ -38,6 +38,7 @@ import org.apache.druid.indexing.common.task.CompactionRunner; import org.apache.druid.indexing.common.task.CompactionTask; import org.apache.druid.indexing.common.task.CurrentSubTaskHolder; +import org.apache.druid.java.util.common.JodaUtils; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.AllGranularity; import org.apache.druid.java.util.common.granularity.Granularities; @@ -65,7 +66,7 @@ import org.apache.druid.query.groupby.GroupByQueryConfig; import org.apache.druid.query.groupby.orderby.OrderByColumnSpec; import org.apache.druid.query.policy.PolicyEnforcer; -import org.apache.druid.query.spec.MultipleIntervalSegmentSpec; +import org.apache.druid.query.spec.QuerySegmentSpec; import org.apache.druid.segment.VirtualColumn; import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.column.ColumnHolder; @@ -153,7 +154,7 @@ public MSQCompactionRunner( @Override public CompactionConfigValidationResult validateCompactionTask( CompactionTask compactionTask, - Map intervalToDataSchemaMap + Map intervalToDataSchemaMap ) { if (intervalToDataSchemaMap.size() > 1) { @@ -234,11 +235,11 @@ public CurrentSubTaskHolder getCurrentSubTaskHolder() @Override public TaskStatus runCompactionTasks( CompactionTask compactionTask, - Map intervalDataSchemas, + Map inputSchemas, TaskToolbox taskToolbox ) throws Exception { - List msqControllerTasks = createMsqControllerTasks(compactionTask, intervalDataSchemas); + List msqControllerTasks = createMsqControllerTasks(compactionTask, inputSchemas); if (msqControllerTasks.isEmpty()) { String msg = StringUtils.format( @@ -257,21 +258,22 @@ public TaskStatus runCompactionTasks( public List createMsqControllerTasks( CompactionTask compactionTask, - Map intervalDataSchemas + Map inputSchemas ) throws JsonProcessingException { final List msqControllerTasks = new ArrayList<>(); - for (Map.Entry intervalDataSchema : intervalDataSchemas.entrySet()) { + for (Map.Entry inputSchema : inputSchemas.entrySet()) { Query query; - Interval interval = intervalDataSchema.getKey(); - DataSchema dataSchema = intervalDataSchema.getValue(); - Map inputColToVirtualCol = getVirtualColumns(dataSchema, interval, compactionTask.getTransformSpec()); + QuerySegmentSpec segmentSpec = inputSchema.getKey(); + DataSchema dataSchema = inputSchema.getValue(); + Map inputColToVirtualCol = + getVirtualColumns(dataSchema, JodaUtils.umbrellaInterval(segmentSpec.getIntervals()), compactionTask.getTransformSpec()); if (isGroupBy(dataSchema)) { - query = buildGroupByQuery(compactionTask, interval, dataSchema, inputColToVirtualCol); + query = buildGroupByQuery(compactionTask, segmentSpec, dataSchema, inputColToVirtualCol); } else { - query = buildScanQuery(compactionTask, interval, dataSchema, inputColToVirtualCol); + query = buildScanQuery(compactionTask, segmentSpec, dataSchema, inputColToVirtualCol); } QueryContext compactionTaskContext = new QueryContext(compactionTask.getContext()); @@ -501,7 +503,7 @@ private static Map buildQueryContext( private Query buildScanQuery( CompactionTask compactionTask, - Interval interval, + QuerySegmentSpec segmentSpec, DataSchema dataSchema, Map inputColToVirtualCol ) @@ -513,7 +515,7 @@ private Query buildScanQuery( .columns(rowSignature.getColumnNames()) .virtualColumns(virtualColumns) .columnTypes(rowSignature.getColumnTypes()) - .intervals(new MultipleIntervalSegmentSpec(Collections.singletonList(interval))) + .intervals(segmentSpec) .filters(dataSchema.getTransformSpec().getFilter()) .context(buildQueryContext(compactionTask.getContext(), dataSchema)); @@ -639,7 +641,7 @@ private Map getVirtualColumns(DataSchema dataSchema, Inte private Query buildGroupByQuery( CompactionTask compactionTask, - Interval interval, + QuerySegmentSpec segmentSpec, DataSchema dataSchema, Map inputColToVirtualCol ) @@ -675,7 +677,7 @@ private Query buildGroupByQuery( .setAggregatorSpecs(Arrays.asList(dataSchema.getAggregators())) .setPostAggregatorSpecs(postAggregators) .setContext(buildQueryContext(compactionTask.getContext(), dataSchema)) - .setInterval(interval); + .setQuerySegmentSpec(segmentSpec); if (compactionTask.getTuningConfig() != null && compactionTask.getTuningConfig().getPartitionsSpec() != null) { getOrderBySpec(compactionTask.getTuningConfig().getPartitionsSpec()).forEach(builder::addOrderByColumn); diff --git a/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQCompactionTaskRunTest.java b/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQCompactionTaskRunTest.java index 758f27a741f4..3f182cef2531 100644 --- a/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQCompactionTaskRunTest.java +++ b/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQCompactionTaskRunTest.java @@ -22,6 +22,7 @@ import com.fasterxml.jackson.databind.InjectableValues; 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; @@ -31,13 +32,16 @@ import com.google.inject.util.Providers; import org.apache.druid.client.indexing.ClientCompactionTaskGranularitySpec; import org.apache.druid.data.input.impl.DimensionsSpec; +import org.apache.druid.error.DruidException; import org.apache.druid.guice.DruidGuiceExtensions; import org.apache.druid.guice.LifecycleModule; import org.apache.druid.guice.SegmentWranglerModule; 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,7 +50,9 @@ import org.apache.druid.indexing.common.task.CompactionTask; import org.apache.druid.indexing.common.task.CompactionTaskRunBase; import org.apache.druid.indexing.common.task.IndexTask; +import org.apache.druid.indexing.common.task.MinorCompactionInputSpec; import org.apache.druid.indexing.common.task.Tasks; +import org.apache.druid.indexing.common.task.TuningConfigBuilder; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.StringUtils; @@ -63,6 +69,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; @@ -77,6 +84,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; @@ -90,6 +98,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; @@ -571,6 +580,169 @@ public void testMSQCompactionWithConcurrentAppendAppendLocksFirst() throws Excep verifyTaskSuccessRowsAndSchemaMatch(finalResult, 19); } + @Test + public void testMinorCompaction() throws Exception + { + Assume.assumeTrue(lockGranularity == LockGranularity.TIME_CHUNK); + Assume.assumeTrue("Minor 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( + compactionTask1.getCompactionRunner(), + List.copyOf(resultPair1.rhs.getSegments()), + segmentGranularity, + DEFAULT_QUERY_GRAN, + false + ); + Assert.assertEquals(1, resultPair1.rhs.getSegments().size()); + final DataSegment compactedSegment1 = Iterables.getOnlyElement(resultPair1.rhs.getSegments()); + + 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 MinorCompactionInputSpec(inputInterval, uncompacted), 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 testMinorCompactionRangePartition() 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("Minor 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 MinorCompactionInputSpec(inputInterval, uncompacted), 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 testMinorCompactionOverlappingInterval() throws Exception + { + Assume.assumeTrue(lockGranularity == LockGranularity.TIME_CHUNK); + Assume.assumeTrue("Minor 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 MinorCompactionInputSpec(compactionInterval, uncompactedFromIndexTask), true) + .build(); + DruidException e = Assert.assertThrows(DruidException.class, () -> runTask(compactionTask1)); + Assert.assertEquals( + "Minor compaction doesn't allow segments not completely within interval[2014-01-01T00:00:00.000Z/2014-01-01T08:00:00.000Z]", + e.getMessage() + ); + } + @Override protected CompactionState getDefaultCompactionState( Granularity segmentGranularity, diff --git a/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQCompactionRunnerTest.java b/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQCompactionRunnerTest.java index a906f3dc0016..fa124f44da35 100644 --- a/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQCompactionRunnerTest.java +++ b/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQCompactionRunnerTest.java @@ -65,6 +65,8 @@ import org.apache.druid.query.filter.SelectorDimFilter; import org.apache.druid.query.groupby.GroupByQuery; import org.apache.druid.query.scan.ScanQuery; +import org.apache.druid.query.spec.MultipleIntervalSegmentSpec; +import org.apache.druid.query.spec.QuerySegmentSpec; import org.apache.druid.segment.AutoTypeColumnSchema; import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.NestedDataColumnSchema; @@ -135,8 +137,8 @@ public class MSQCompactionRunnerTest new LongSumAggregatorFactory(LONG_DIMENSION.getName(), LONG_DIMENSION.getName()) ) .build(); - private static final Map INTERVAL_DATASCHEMAS = ImmutableMap.of( - COMPACTION_INTERVAL, + private static final Map INTERVAL_DATASCHEMAS = ImmutableMap.of( + new MultipleIntervalSegmentSpec(List.of(COMPACTION_INTERVAL)), new CombinedDataSchema( DATA_SOURCE, new TimestampSpec(TIMESTAMP_COLUMN, null, null), @@ -166,8 +168,8 @@ public class MSQCompactionRunnerTest @Test public void testMultipleDisjointCompactionIntervalsAreInvalid() { - Map intervalDataschemas = new HashMap<>(INTERVAL_DATASCHEMAS); - intervalDataschemas.put(Intervals.of("2017-07-01/2018-01-01"), null); + Map intervalDataschemas = new HashMap<>(INTERVAL_DATASCHEMAS); + intervalDataschemas.put(new MultipleIntervalSegmentSpec(List.of(Intervals.of("2017-07-01/2018-01-01"))), null); CompactionTask compactionTask = createCompactionTask( new HashedPartitionsSpec(3, null, ImmutableList.of("dummy")), null, @@ -387,7 +389,7 @@ public void testCompactionConfigWithoutMetricsSpecProducesCorrectSpec() throws J List msqControllerTasks = MSQ_COMPACTION_RUNNER.createMsqControllerTasks( taskCreatedWithTransformSpec, - Collections.singletonMap(COMPACTION_INTERVAL, dataSchema) + Map.of(new MultipleIntervalSegmentSpec(List.of(COMPACTION_INTERVAL)), dataSchema) ); MSQControllerTask msqControllerTask = Iterables.getOnlyElement(msqControllerTasks); @@ -471,7 +473,7 @@ public void testCompactionConfigWithSortOnNonTimeDimensionsProducesCorrectSpec() List msqControllerTasks = MSQ_COMPACTION_RUNNER.createMsqControllerTasks( taskCreatedWithTransformSpec, - Collections.singletonMap(COMPACTION_INTERVAL, dataSchema) + Map.of(new MultipleIntervalSegmentSpec(List.of(COMPACTION_INTERVAL)), dataSchema) ); LegacyMSQSpec actualMSQSpec = Iterables.getOnlyElement(msqControllerTasks).getQuerySpec(); @@ -520,7 +522,7 @@ public void testCompactionConfigWithMetricsSpecProducesCorrectSpec() throws Json List msqControllerTasks = MSQ_COMPACTION_RUNNER.createMsqControllerTasks( taskCreatedWithTransformSpec, - Collections.singletonMap(COMPACTION_INTERVAL, dataSchema) + Map.of(new MultipleIntervalSegmentSpec(List.of(COMPACTION_INTERVAL)), dataSchema) ); MSQControllerTask msqControllerTask = Iterables.getOnlyElement(msqControllerTasks); @@ -606,7 +608,7 @@ public void testCompactionConfigWithProjectionsProducesCorrectSpec() throws Json List msqControllerTasks = MSQ_COMPACTION_RUNNER.createMsqControllerTasks( taskCreatedWithTransformSpec, - Collections.singletonMap(COMPACTION_INTERVAL, dataSchema) + Map.of(new MultipleIntervalSegmentSpec(List.of(COMPACTION_INTERVAL)), dataSchema) ); MSQControllerTask msqControllerTask = Iterables.getOnlyElement(msqControllerTasks); diff --git a/processing/src/main/java/org/apache/druid/timeline/partition/DimensionRangeShardSpec.java b/processing/src/main/java/org/apache/druid/timeline/partition/DimensionRangeShardSpec.java index 4054a618a752..a0a850dae08a 100644 --- a/processing/src/main/java/org/apache/druid/timeline/partition/DimensionRangeShardSpec.java +++ b/processing/src/main/java/org/apache/druid/timeline/partition/DimensionRangeShardSpec.java @@ -104,6 +104,18 @@ public int getNumCorePartitions() return numCorePartitions; } + @Override + public ShardSpec withPartitionNum(int partitionNum) + { + return new DimensionRangeShardSpec(dimensions, start, end, partitionNum, numCorePartitions); + } + + @Override + public ShardSpec withCorePartitions(int partitions) + { + return new DimensionRangeShardSpec(dimensions, start, end, partitionNum, partitions); + } + public boolean isNumCorePartitionsUnknown() { return numCorePartitions == UNKNOWN_NUM_CORE_PARTITIONS; @@ -242,6 +254,12 @@ public boolean possibleInDomain(Map> domain) return true; } + @Override + public boolean canCreateNumberedPartitionChunk() + { + 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..4372ac8e1d7c 100644 --- a/processing/src/main/java/org/apache/druid/timeline/partition/LinearShardSpec.java +++ b/processing/src/main/java/org/apache/druid/timeline/partition/LinearShardSpec.java @@ -55,6 +55,18 @@ public int getNumCorePartitions() return 0; } + @Override + public ShardSpec withPartitionNum(int partitionNum) + { + return new LinearShardSpec(partitionNum); + } + + @Override + public ShardSpec withCorePartitions(int partitions) + { + 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..4ff607b0965e 100644 --- a/processing/src/main/java/org/apache/druid/timeline/partition/NumberedShardSpec.java +++ b/processing/src/main/java/org/apache/druid/timeline/partition/NumberedShardSpec.java @@ -100,6 +100,24 @@ public int getNumCorePartitions() return partitions; } + @Override + public ShardSpec withPartitionNum(int partitionNum) + { + return new NumberedShardSpec(partitionNum, partitions); + } + + @Override + public ShardSpec withCorePartitions(int partitions) + { + return new NumberedShardSpec(partitionNum, partitions); + } + + @Override + public boolean canCreateNumberedPartitionChunk() + { + 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..b91c96387a9d 100644 --- a/processing/src/main/java/org/apache/druid/timeline/partition/ShardSpec.java +++ b/processing/src/main/java/org/apache/druid/timeline/partition/ShardSpec.java @@ -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 canCreateNumberedPartitionChunk() + { + 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 partitionNum) + { + 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..366ad089069a 100644 --- a/server/src/main/java/org/apache/druid/client/indexing/ClientCompactionIntervalSpec.java +++ b/server/src/main/java/org/apache/druid/client/indexing/ClientCompactionIntervalSpec.java @@ -20,29 +20,37 @@ 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}. *

- * Should be synchronized with org.apache.druid.indexing.common.task.CompactionIntervalSpec. + * Should be synchronized with org.apache.druid.indexing.common.task.CompactionIntervalSpec and org.apache.druid.indexing.common.task.UncompactedInputSpec. */ public class ClientCompactionIntervalSpec { - private static final String TYPE = "interval"; + private static final String TYPE_ALL_SEGMENTS = "interval"; + private static final String TYPE_UNCOMPACTED_SEGMENTS_ONLY = "uncompacted"; private final Interval interval; @Nullable + 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,13 +58,29 @@ 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; } @JsonProperty public String getType() { - return TYPE; + return (uncompactedSegments == null) ? TYPE_ALL_SEGMENTS : TYPE_UNCOMPACTED_SEGMENTS_ONLY; } @JsonProperty @@ -65,6 +89,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 +115,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 +130,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..5ac25f6ad0b8 100644 --- a/server/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java +++ b/server/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java @@ -387,6 +387,16 @@ SegmentPublishResult commitAppendSegmentsAndMetadata( @Nullable SegmentSchemaMapping segmentSchemaMapping ); + /** + * Inserts entries into the upgrade_segments table for segments that need to be upgraded + * to a new version when a REPLACE task finishes using {@link #commitReplaceSegments}. + * + * @param segmentToReplaceLock map from segment to its corresponding REPLACE lock, identifying + * which segments should be upgraded when the lock's task completes + * @return number of entries successfully inserted into the upgrade_segments table + */ + int insertIntoUpgradeSegmentsTable(Map segmentToReplaceLock); + /** * Commits segments and corresponding schema created by a REPLACE task. * This method also handles the segment upgrade scenarios that may result diff --git a/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java b/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java index dc0db5b56839..fbe30a31e88e 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,15 @@ public SegmentPublishResult commitReplaceSegments( final SegmentPublishResult result = inReadWriteDatasourceTransaction( dataSource, transaction -> { - final Set segmentsToInsert = new HashSet<>(replaceSegments); - - Set upgradedSegments = createNewIdsOfAppendSegmentsAfterReplace( + final Set newSegments = createNewSegmentsAfterReplace( dataSource, transaction, replaceSegments, locksHeldByReplaceTask ); - Map upgradeSegmentMetadata = new HashMap<>(); final Map upgradedFromSegmentIdMap = new HashMap<>(); - for (DataSegmentPlus dataSegmentPlus : upgradedSegments) { - segmentsToInsert.add(dataSegmentPlus.getDataSegment()); + for (DataSegmentPlus dataSegmentPlus : newSegments) { if (dataSegmentPlus.getSchemaFingerprint() != null && dataSegmentPlus.getNumRows() != null) { upgradeSegmentMetadata.put( dataSegmentPlus.getDataSegment().getId(), @@ -518,6 +517,9 @@ public SegmentPublishResult commitReplaceSegments( ); } } + final Set segmentsToInsert = newSegments.stream() + .map(DataSegmentPlus::getDataSegment) + .collect(Collectors.toSet()); return SegmentPublishResult.ok( insertSegments( transaction, @@ -1852,9 +1854,14 @@ protected Set insertSegments( } /** - * Creates new versions of segments appended while a "REPLACE" task was in progress. + * Retrieves segments from the upgrade segments table and creates upgraded versions with new intervals, + * versions, and partition numbers. Combines upgraded segments with replace segments and updates shard + * specs with correct core partition counts. + * + * @return segments to insert into segment table + * @throws DruidException if a replace interval partially overlaps a segment being upgraded */ - private Set createNewIdsOfAppendSegmentsAfterReplace( + private Set createNewSegmentsAfterReplace( final String dataSource, final SegmentMetadataTransaction transaction, final Set replaceSegments, @@ -1867,17 +1874,19 @@ private Set createNewIdsOfAppendSegmentsAfterReplace( return 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().canCreateNumberedPartitionChunk()) { + numChunkNotSupported.add(segment.getInterval()); + } } // Find the segments that need to be upgraded @@ -1885,18 +1894,21 @@ private Set createNewIdsOfAppendSegmentsAfterReplace( .map(ReplaceTaskLock::getSupervisorTaskId) .findFirst().orElse(null); final Map upgradeSegmentToLockVersion - = getAppendSegmentsCommittedDuringTask(transaction, taskId); + = getSegmentsCoveredByReplaceTaskLock(transaction, taskId); final List segmentsToUpgrade = retrieveSegmentsById(dataSource, transaction, upgradeSegmentToLockVersion.keySet()); + // If there is nothing to upgrade, return the replaceSegments unchanged if (segmentsToUpgrade.isEmpty()) { - return Collections.emptySet(); + return replaceSegments.stream() + .map(s -> new DataSegmentPlus(s, null, null, null, null, null, null, null)) + .collect(Collectors.toSet()); } - final Set replaceIntervals = intervalToNumCorePartitions.keySet(); - - final Set upgradedSegments = new HashSet<>(); + final Set replaceIntervals = intervalToCurrentPartitionNum.keySet(); + final Map upgradedSegments = new HashMap<>(); + 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().canCreateNumberedPartitionChunk()) { + 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) @@ -1947,7 +1960,8 @@ private Set createNewIdsOfAppendSegmentsAfterReplace( ? oldSegmentMetadata.getDataSegment().getId().toString() : oldSegmentMetadata.getUpgradedFromSegmentId(); - upgradedSegments.add( + upgradedSegments.put( + dataSegment, new DataSegmentPlus( dataSegment, null, @@ -1959,9 +1973,35 @@ private Set createNewIdsOfAppendSegmentsAfterReplace( oldSegmentMetadata.getIndexingStateFingerprint() ) ); + segmentsToInsert.add(dataSegment); } - return upgradedSegments; + return segmentsToInsert.stream().map(segment -> { + // update corePartitions in shard spec + Integer partitionNum = intervalToCurrentPartitionNum.get(segment.getInterval()); + if (!segment.isTombstone() && !numChunkNotSupported.contains(segment.getInterval()) && partitionNum != null) { + return segment.withShardSpec(segment.getShardSpec().withCorePartitions(partitionNum + 1)); + } else { + return segment; + } + }).map(s -> { + // wrap with DataSegmentPlus + if (upgradedSegments.containsKey(s)) { + DataSegmentPlus upgraded = upgradedSegments.get(s); + return new DataSegmentPlus( + s, + upgraded.getCreatedDate(), + upgraded.getUsedStatusLastUpdatedDate(), + upgraded.getUsed(), + upgraded.getSchemaFingerprint(), + upgraded.getNumRows(), + upgraded.getUpgradedFromSegmentId(), + upgraded.getIndexingStateFingerprint() + ); + } else { + return new DataSegmentPlus(s, null, null, null, null, null, null, null); + } + }).collect(Collectors.toSet()); } /** @@ -2097,17 +2137,27 @@ private SegmentMetadata getSegmentMetadataFromSchemaMappingOrUpgradeMetadata( return segmentMetadata; } + @Override + public int insertIntoUpgradeSegmentsTable(Map segmentToReplaceLock) + { + final String dataSource = verifySegmentsToCommit(segmentToReplaceLock.keySet()); + return inReadWriteDatasourceTransaction( + dataSource, + transaction -> insertIntoUpgradeSegmentsTable(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 insertIntoUpgradeSegmentsTable( SegmentMetadataTransaction transaction, Map segmentToReplaceLock ) { if (segmentToReplaceLock.isEmpty()) { - return; + return 0; } final PreparedBatch batch = transaction.getHandle().prepareBatch( @@ -2118,6 +2168,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 +2188,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 +2199,7 @@ private void insertIntoUpgradeSegmentsTable( ); } } + return inserted; } private List retrieveSegmentsById( @@ -2169,13 +2223,13 @@ private List retrieveSegmentsById( } /** - * Finds the append segments that were covered by the given task REPLACE locks. - * These append segments must now be upgraded to the same version as the segments + * Finds segments that were covered by the given task REPLACE locks. + * These segments must now be upgraded to the same version as the segments * being committed by this replace task. * * @return Map from append Segment ID to REPLACE lock version */ - private Map getAppendSegmentsCommittedDuringTask( + private Map getSegmentsCoveredByReplaceTaskLock( SegmentMetadataTransaction transaction, String taskId ) @@ -2185,13 +2239,12 @@ private Map getAppendSegmentsCommittedDuringTask( dbTables.getUpgradeSegmentsTable() ); - ResultIterator> resultIterator = transaction.getHandle() - .createQuery(sql) - .bind("task_id", taskId) - .map( - (index, r, ctx) -> Pair.of(r.getString("segment_id"), r.getString("lock_version")) - ) - .iterator(); + ResultIterator> resultIterator = + transaction.getHandle() + .createQuery(sql) + .bind("task_id", taskId) + .map((index, r, ctx) -> Pair.of(r.getString("segment_id"), r.getString("lock_version"))) + .iterator(); final Map segmentIdToLockVersion = new HashMap<>(); while (resultIterator.hasNext()) { @@ -2359,8 +2412,8 @@ protected SegmentPublishResult updateDataSourceMetadataInTransaction( .execute(); publishResult = numRows == 1 - ? SegmentPublishResult.ok(Set.of()) - : SegmentPublishResult.retryableFailure("Insert failed"); + ? SegmentPublishResult.ok(Set.of()) + : SegmentPublishResult.retryableFailure("Insert failed"); } else { // Expecting a particular old metadata; use the SHA1 in a compare-and-swap UPDATE final String updateSql = StringUtils.format( @@ -2378,8 +2431,8 @@ protected SegmentPublishResult updateDataSourceMetadataInTransaction( .execute(); publishResult = numRows == 1 - ? SegmentPublishResult.ok(Set.of()) - : SegmentPublishResult.retryableFailure("Compare-and-swap update failed"); + ? SegmentPublishResult.ok(Set.of()) + : SegmentPublishResult.retryableFailure("Compare-and-swap update failed"); } if (publishResult.isSuccess()) { @@ -2462,8 +2515,8 @@ public int deleteSegments(final Set segments) final String dataSource = verifySegmentsToCommit(segments); final Set idsToDelete = segments.stream() - .map(DataSegment::getId) - .collect(Collectors.toSet()); + .map(DataSegment::getId) + .collect(Collectors.toSet()); int numDeletedSegments = inReadWriteDatasourceTransaction( dataSource, transaction -> transaction.deleteSegments(idsToDelete) @@ -2735,10 +2788,10 @@ private void markIndexingStateFingerprintsAsActive(Set segments) // Collect unique non-null indexing state fingerprints final List fingerprints = segments.stream() - .map(DataSegment::getIndexingStateFingerprint) - .filter(fp -> fp != null && !fp.isEmpty()) - .distinct() - .collect(Collectors.toList()); + .map(DataSegment::getIndexingStateFingerprint) + .filter(fp -> fp != null && !fp.isEmpty()) + .distinct() + .collect(Collectors.toList()); try { int rowsUpdated = indexingStateStorage.markIndexingStatesAsActive(fingerprints); @@ -2749,7 +2802,11 @@ private void markIndexingStateFingerprintsAsActive(Set segments) catch (Exception e) { // Log but don't fail the overall operation - the fingerprint will stay pending // and be cleaned up by the pending grace period - log.warn(e, "Failed to mark indexing states for the following fingerprints as active (Future segments publishes may remediate): %s", fingerprints); + log.warn( + e, + "Failed to mark indexing states for the following fingerprints as active (Future segments publishes may remediate): %s", + fingerprints + ); } } diff --git a/server/src/main/java/org/apache/druid/server/compaction/CompactionCandidate.java b/server/src/main/java/org/apache/druid/server/compaction/CompactionCandidate.java index 4d53d8e9cf57..d3653ccd992a 100644 --- a/server/src/main/java/org/apache/druid/server/compaction/CompactionCandidate.java +++ b/server/src/main/java/org/apache/druid/server/compaction/CompactionCandidate.java @@ -167,6 +167,12 @@ public CompactionStatistics getUncompactedStats() return currentStatus.getUncompactedStats(); } + @Nullable + public List getUncompactedSegments() + { + return currentStatus.getUncompactedSegments(); + } + /** * Non-null current compaction status of the time chunk corresponding to this candidate. */ diff --git a/server/src/main/java/org/apache/druid/server/compaction/CompactionCandidateSearchPolicy.java b/server/src/main/java/org/apache/druid/server/compaction/CompactionCandidateSearchPolicy.java index eeb5d612b0e4..44889fb7e10c 100644 --- a/server/src/main/java/org/apache/druid/server/compaction/CompactionCandidateSearchPolicy.java +++ b/server/src/main/java/org/apache/druid/server/compaction/CompactionCandidateSearchPolicy.java @@ -65,7 +65,7 @@ Eligibility checkEligibilityForCompaction( */ class Eligibility { - public static final Eligibility OK = new Eligibility(true, null, CompactionMode.FULL_COMPACTION); + public static final Eligibility OK = new Eligibility(true, null, CompactionMode.ALL_SEGMENTS); private final boolean eligible; private final String reason; @@ -104,6 +104,11 @@ public static Eligibility fail(String messageFormat, Object... args) return new Eligibility(false, StringUtils.format(messageFormat, args), null); } + public static Eligibility incremental(String messageFormat, Object... args) + { + return new Eligibility(true, StringUtils.format(messageFormat, args), CompactionMode.UNCOMPACTED_SEGMENTS_ONLY); + } + @Override public boolean equals(Object object) { diff --git a/server/src/main/java/org/apache/druid/server/compaction/CompactionMode.java b/server/src/main/java/org/apache/druid/server/compaction/CompactionMode.java index c4ccfb7be28f..83ba94ce80ca 100644 --- a/server/src/main/java/org/apache/druid/server/compaction/CompactionMode.java +++ b/server/src/main/java/org/apache/druid/server/compaction/CompactionMode.java @@ -27,5 +27,9 @@ public enum CompactionMode /** * Indicates that all existing segments of the interval will be picked for compaction. */ - FULL_COMPACTION; + ALL_SEGMENTS, + /** + * Indicates that only uncompacted segments of the interval will be picked for compaction. + */ + UNCOMPACTED_SEGMENTS_ONLY; } 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 bf738aa5e887..cdf1f1832b3e 100644 --- a/server/src/main/java/org/apache/druid/server/compaction/CompactionStatus.java +++ b/server/src/main/java/org/apache/druid/server/compaction/CompactionStatus.java @@ -59,7 +59,7 @@ public class CompactionStatus { private static final Logger log = new Logger(CompactionStatus.class); - private static final CompactionStatus COMPLETE = new CompactionStatus(State.COMPLETE, null, null, null); + private static final CompactionStatus COMPLETE = new CompactionStatus(State.COMPLETE, null, null, null, null); public static final String NEVER_COMPACTED_REASON = "not compacted yet"; public enum State @@ -93,20 +93,28 @@ public enum State private final State state; private final String reason; + + @Nullable private final CompactionStatistics compactedStats; + @Nullable private final CompactionStatistics uncompactedStats; + @Nullable + private final List uncompactedSegments; + private CompactionStatus( State state, String reason, - CompactionStatistics compactedStats, - CompactionStatistics uncompactedStats + @Nullable CompactionStatistics compactedStats, + @Nullable CompactionStatistics uncompactedStats, + @Nullable List uncompactedSegments ) { this.state = state; this.reason = reason; this.compactedStats = compactedStats; this.uncompactedStats = uncompactedStats; + this.uncompactedSegments = uncompactedSegments; } public boolean isComplete() @@ -139,6 +147,11 @@ public CompactionStatistics getUncompactedStats() return uncompactedStats; } + public List getUncompactedSegments() + { + return uncompactedSegments; + } + @Override public String toString() { @@ -152,12 +165,13 @@ public String toString() public static CompactionStatus pending(String reasonFormat, Object... args) { - return new CompactionStatus(State.PENDING, StringUtils.format(reasonFormat, args), null, null); + return new CompactionStatus(State.PENDING, StringUtils.format(reasonFormat, args), null, null, null); } public static CompactionStatus pending( CompactionStatistics compactedStats, CompactionStatistics uncompactedStats, + List uncompactedSegments, String reasonFormat, Object... args ) @@ -166,7 +180,8 @@ public static CompactionStatus pending( State.PENDING, StringUtils.format(reasonFormat, args), compactedStats, - uncompactedStats + uncompactedStats, + uncompactedSegments ); } @@ -244,12 +259,12 @@ private static String asString(PartitionsSpec partitionsSpec) public static CompactionStatus skipped(String reasonFormat, Object... args) { - return new CompactionStatus(State.SKIPPED, StringUtils.format(reasonFormat, args), null, null); + return new CompactionStatus(State.SKIPPED, StringUtils.format(reasonFormat, args), null, null, null); } public static CompactionStatus running(String message) { - return new CompactionStatus(State.RUNNING, message, null, null); + return new CompactionStatus(State.RUNNING, message, null, null, null); } /** @@ -428,7 +443,8 @@ private CompactionStatus evaluate() } else { return CompactionStatus.pending( createStats(this.compactedSegments), - createStats(uncompactedSegments), + createStats(this.uncompactedSegments), + this.uncompactedSegments, reasonsForCompaction.get(0) ); } diff --git a/server/src/main/java/org/apache/druid/server/compaction/CompactionStatusTracker.java b/server/src/main/java/org/apache/druid/server/compaction/CompactionStatusTracker.java index e98e3575f47b..cdd52a4f917c 100644 --- a/server/src/main/java/org/apache/druid/server/compaction/CompactionStatusTracker.java +++ b/server/src/main/java/org/apache/druid/server/compaction/CompactionStatusTracker.java @@ -91,8 +91,25 @@ public CompactionStatus computeCompactionStatus( CompactionCandidateSearchPolicy searchPolicy ) { - // Skip intervals that already have a running task final CompactionTaskStatus lastTaskStatus = getLatestTaskStatus(candidate); + CompactionStatus status = deriveCompactionStatus(lastTaskStatus); + if (!CompactionStatus.State.PENDING.equals(status.getState())) { + return status; + } + + // Skip intervals that have been filtered out by the policy + final CompactionCandidateSearchPolicy.Eligibility eligibility + = searchPolicy.checkEligibilityForCompaction(candidate, lastTaskStatus); + if (eligibility.isEligible()) { + return CompactionStatus.pending("Not compacted yet"); + } else { + return CompactionStatus.skipped("Rejected by search policy: %s", eligibility.getReason()); + } + } + + public CompactionStatus deriveCompactionStatus(CompactionTaskStatus lastTaskStatus) + { + // Skip intervals that already have a running task if (lastTaskStatus != null && lastTaskStatus.getState() == TaskState.RUNNING) { return CompactionStatus.running("Task for interval is already running"); } @@ -106,15 +123,7 @@ public CompactionStatus computeCompactionStatus( "Segment timeline not updated since last compaction task succeeded" ); } - - // Skip intervals that have been filtered out by the policy - final CompactionCandidateSearchPolicy.Eligibility eligibility - = searchPolicy.checkEligibilityForCompaction(candidate, lastTaskStatus); - if (eligibility.isEligible()) { - return CompactionStatus.pending("Not compacted yet"); - } else { - return CompactionStatus.skipped("Rejected by search policy: %s", eligibility.getReason()); - } + return CompactionStatus.pending("Not compacted yet"); } /** 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..90d3ce40471e 100644 --- a/server/src/main/java/org/apache/druid/server/compaction/MostFragmentedIntervalFirstPolicy.java +++ b/server/src/main/java/org/apache/druid/server/compaction/MostFragmentedIntervalFirstPolicy.java @@ -47,6 +47,7 @@ public class MostFragmentedIntervalFirstPolicy extends BaseCandidateSearchPolicy private final int minUncompactedCount; private final HumanReadableBytes minUncompactedBytes; private final HumanReadableBytes maxAverageUncompactedBytesPerSegment; + private final int minUncompactedBytesPercentForFullCompaction; @JsonCreator public MostFragmentedIntervalFirstPolicy( @@ -54,6 +55,8 @@ public MostFragmentedIntervalFirstPolicy( @JsonProperty("minUncompactedBytes") @Nullable HumanReadableBytes minUncompactedBytes, @JsonProperty("maxAverageUncompactedBytesPerSegment") @Nullable HumanReadableBytes maxAverageUncompactedBytesPerSegment, + @JsonProperty("minUncompactedBytesPercentForFullCompaction") @Nullable + Integer minUncompactedBytesPercentForFullCompaction, @JsonProperty("priorityDatasource") @Nullable String priorityDatasource ) { @@ -69,11 +72,20 @@ public MostFragmentedIntervalFirstPolicy( "'minUncompactedCount'[%s] must be greater than 0", maxAverageUncompactedBytesPerSegment ); + InvalidInput.conditionalException( + minUncompactedBytesPercentForFullCompaction == null + || (minUncompactedBytesPercentForFullCompaction >= 0 + && minUncompactedBytesPercentForFullCompaction < 100), + "'minUncompactedBytesPercentForFullCompaction'[%s] must be between 0 and 100", + minUncompactedBytesPercentForFullCompaction + ); this.minUncompactedCount = Configs.valueOrDefault(minUncompactedCount, 100); this.minUncompactedBytes = Configs.valueOrDefault(minUncompactedBytes, SIZE_10_MB); this.maxAverageUncompactedBytesPerSegment = Configs.valueOrDefault(maxAverageUncompactedBytesPerSegment, SIZE_2_GB); + this.minUncompactedBytesPercentForFullCompaction = + Configs.valueOrDefault(minUncompactedBytesPercentForFullCompaction, 0); } /** @@ -106,6 +118,17 @@ public HumanReadableBytes getMaxAverageUncompactedBytesPerSegment() return maxAverageUncompactedBytesPerSegment; } + /** + * Threshold percentage of uncompacted bytes to total bytes below which + * minor compaction is eligible instead of full compaction. + * Default value is 0. + */ + @JsonProperty + public int minUncompactedBytesPercentForFullCompaction() + { + return minUncompactedBytesPercentForFullCompaction; + } + @Override protected Comparator getSegmentComparator() { @@ -124,7 +147,8 @@ 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) + && minUncompactedBytesPercentForFullCompaction == policy.minUncompactedBytesPercentForFullCompaction; } @Override @@ -134,19 +158,22 @@ public int hashCode() super.hashCode(), minUncompactedCount, minUncompactedBytes, - maxAverageUncompactedBytesPerSegment + maxAverageUncompactedBytesPerSegment, + minUncompactedBytesPercentForFullCompaction ); } @Override public String toString() { - return "MostFragmentedIntervalFirstPolicy{" + - "minUncompactedCount=" + minUncompactedCount + - ", minUncompactedBytes=" + minUncompactedBytes + - ", maxAverageUncompactedBytesPerSegment=" + maxAverageUncompactedBytesPerSegment + - ", priorityDataSource='" + getPriorityDatasource() + '\'' + - '}'; + return + "MostFragmentedIntervalFirstPolicy{" + + "minUncompactedCount=" + minUncompactedCount + + ", minUncompactedBytes=" + minUncompactedBytes + + ", maxAverageUncompactedBytesPerSegment=" + maxAverageUncompactedBytesPerSegment + + ", minUncompactedBytesPercentForFullCompaction=" + minUncompactedBytesPercentForFullCompaction + + ", priorityDataSource='" + getPriorityDatasource() + '\'' + + '}'; } private int compare(CompactionCandidate candidateA, CompactionCandidate candidateB) @@ -185,6 +212,17 @@ public Eligibility checkEligibilityForCompaction( "Average size[%,d] of uncompacted segments in interval must be at most [%,d]", avgSegmentSize, maxAverageUncompactedBytesPerSegment.getBytes() ); + } + + final double uncompactedBytesRatio = (double) uncompacted.getTotalBytes() / + (uncompacted.getTotalBytes() + candidate.getCompactedStats().getTotalBytes()) + * 100; + if (uncompactedBytesRatio < minUncompactedBytesPercentForFullCompaction) { + return Eligibility.incremental( + "Uncompacted bytes ratio[%.2f] is below threshold[%d]", + uncompactedBytesRatio, + minUncompactedBytesPercentForFullCompaction + ); } else { return Eligibility.OK; } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/UserCompactionTaskQueryTuningConfig.java b/server/src/main/java/org/apache/druid/server/coordinator/UserCompactionTaskQueryTuningConfig.java index 9c1bfb200648..ca93ef4b180e 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/UserCompactionTaskQueryTuningConfig.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/UserCompactionTaskQueryTuningConfig.java @@ -88,4 +88,171 @@ public Integer getMaxRowsPerSegment() { throw new UnsupportedOperationException(); } + + public static Builder builder() + { + return new Builder(); + } + + public static class Builder + { + private Integer maxRowsInMemory; + private AppendableIndexSpec appendableIndexSpec; + private Long maxBytesInMemory; + private Long maxTotalRows; + private SplitHintSpec splitHintSpec; + private PartitionsSpec partitionsSpec; + private IndexSpec indexSpec; + private IndexSpec indexSpecForIntermediatePersists; + private Integer maxPendingPersists; + private Long pushTimeout; + private SegmentWriteOutMediumFactory segmentWriteOutMediumFactory; + private Integer maxNumConcurrentSubTasks; + private Integer maxRetry; + private Long taskStatusCheckPeriodMs; + private Duration chatHandlerTimeout; + private Integer chatHandlerNumRetries; + private Integer maxNumSegmentsToMerge; + private Integer totalNumMergeTasks; + private Integer maxColumnsToMerge; + + public Builder maxRowsInMemory(Integer maxRowsInMemory) + { + this.maxRowsInMemory = maxRowsInMemory; + return this; + } + + public Builder appendableIndexSpec(AppendableIndexSpec appendableIndexSpec) + { + this.appendableIndexSpec = appendableIndexSpec; + return this; + } + + public Builder maxBytesInMemory(Long maxBytesInMemory) + { + this.maxBytesInMemory = maxBytesInMemory; + return this; + } + + public Builder maxTotalRows(Long maxTotalRows) + { + this.maxTotalRows = maxTotalRows; + return this; + } + + public Builder splitHintSpec(SplitHintSpec splitHintSpec) + { + this.splitHintSpec = splitHintSpec; + return this; + } + + public Builder partitionsSpec(PartitionsSpec partitionsSpec) + { + this.partitionsSpec = partitionsSpec; + return this; + } + + public Builder indexSpec(IndexSpec indexSpec) + { + this.indexSpec = indexSpec; + return this; + } + + public Builder indexSpecForIntermediatePersists(IndexSpec indexSpecForIntermediatePersists) + { + this.indexSpecForIntermediatePersists = indexSpecForIntermediatePersists; + return this; + } + + public Builder maxPendingPersists(Integer maxPendingPersists) + { + this.maxPendingPersists = maxPendingPersists; + return this; + } + + public Builder pushTimeout(Long pushTimeout) + { + this.pushTimeout = pushTimeout; + return this; + } + + public Builder segmentWriteOutMediumFactory(SegmentWriteOutMediumFactory segmentWriteOutMediumFactory) + { + this.segmentWriteOutMediumFactory = segmentWriteOutMediumFactory; + return this; + } + + public Builder maxNumConcurrentSubTasks(Integer maxNumConcurrentSubTasks) + { + this.maxNumConcurrentSubTasks = maxNumConcurrentSubTasks; + return this; + } + + public Builder maxRetry(Integer maxRetry) + { + this.maxRetry = maxRetry; + return this; + } + + public Builder taskStatusCheckPeriodMs(Long taskStatusCheckPeriodMs) + { + this.taskStatusCheckPeriodMs = taskStatusCheckPeriodMs; + return this; + } + + public Builder chatHandlerTimeout(Duration chatHandlerTimeout) + { + this.chatHandlerTimeout = chatHandlerTimeout; + return this; + } + + public Builder chatHandlerNumRetries(Integer chatHandlerNumRetries) + { + this.chatHandlerNumRetries = chatHandlerNumRetries; + return this; + } + + public Builder maxNumSegmentsToMerge(Integer maxNumSegmentsToMerge) + { + this.maxNumSegmentsToMerge = maxNumSegmentsToMerge; + return this; + } + + public Builder totalNumMergeTasks(Integer totalNumMergeTasks) + { + this.totalNumMergeTasks = totalNumMergeTasks; + return this; + } + + public Builder maxColumnsToMerge(Integer maxColumnsToMerge) + { + this.maxColumnsToMerge = maxColumnsToMerge; + return this; + } + + public UserCompactionTaskQueryTuningConfig build() + { + return new UserCompactionTaskQueryTuningConfig( + maxRowsInMemory, + appendableIndexSpec, + maxBytesInMemory, + maxTotalRows, + splitHintSpec, + partitionsSpec, + indexSpec, + indexSpecForIntermediatePersists, + maxPendingPersists, + pushTimeout, + segmentWriteOutMediumFactory, + maxNumConcurrentSubTasks, + maxRetry, + taskStatusCheckPeriodMs, + chatHandlerTimeout, + chatHandlerNumRetries, + maxNumSegmentsToMerge, + totalNumMergeTasks, + maxColumnsToMerge + ); + } + } } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java b/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java index c376aac34406..108e74b1921b 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java @@ -34,15 +34,18 @@ import org.apache.druid.common.guava.FutureUtils; import org.apache.druid.common.utils.IdUtils; import org.apache.druid.data.input.impl.AggregateProjectionSpec; +import org.apache.druid.error.DruidException; import org.apache.druid.indexer.CompactionEngine; import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.common.granularity.GranularityType; import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.query.SegmentDescriptor; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.rpc.indexing.OverlordClient; import org.apache.druid.segment.transform.CompactionTransformSpec; import org.apache.druid.server.compaction.CompactionCandidate; import org.apache.druid.server.compaction.CompactionCandidateSearchPolicy; +import org.apache.druid.server.compaction.CompactionMode; import org.apache.druid.server.compaction.CompactionSegmentIterator; import org.apache.druid.server.compaction.CompactionSlotManager; import org.apache.druid.server.compaction.CompactionSnapshotBuilder; @@ -262,6 +265,7 @@ private int submitCompactionTasks( final ClientCompactionTaskQuery taskPayload = createCompactionTask( entry, + CompactionMode.ALL_SEGMENTS, config, defaultEngine, null, @@ -291,6 +295,7 @@ private int submitCompactionTasks( */ public static ClientCompactionTaskQuery createCompactionTask( CompactionCandidate candidate, + CompactionMode compactionMode, DataSourceCompactionConfig config, CompactionEngine defaultEngine, String indexingStateFingerprint, @@ -380,6 +385,7 @@ public static ClientCompactionTaskQuery createCompactionTask( return compactSegments( candidate, + compactionMode, config.getTaskPriority(), ClientCompactionTaskQueryTuningConfig.from( config.getTuningConfig(), @@ -439,6 +445,7 @@ public Map getAutoCompactionSnapshot() private static ClientCompactionTaskQuery compactSegments( CompactionCandidate entry, + CompactionMode compactionMode, int compactionTaskPriority, ClientCompactionTaskQueryTuningConfig tuningConfig, ClientCompactionTaskGranularitySpec granularitySpec, @@ -463,14 +470,26 @@ private static ClientCompactionTaskQuery compactSegments( context.put("priority", compactionTaskPriority); final String taskId = IdUtils.newTaskId(TASK_ID_PREFIX, ClientCompactionTaskQuery.TYPE, dataSource, null); + final ClientCompactionIntervalSpec inputSpec; + switch (compactionMode) { + case ALL_SEGMENTS: + inputSpec = new ClientCompactionIntervalSpec(entry.getCompactionInterval(), null, null); + break; + case UNCOMPACTED_SEGMENTS_ONLY: + List uncompacted = entry.getUncompactedSegments() + .stream() + .map(DataSegment::toDescriptor) + .toList(); + inputSpec = new ClientCompactionIntervalSpec(entry.getCompactionInterval(), uncompacted, null); + break; + default: + throw DruidException.defensive("unexpected compaction mode[%s]", compactionMode); + } return new ClientCompactionTaskQuery( taskId, dataSource, - new ClientCompactionIOConfig( - new ClientCompactionIntervalSpec(entry.getCompactionInterval(), null), - dropExisting - ), + new ClientCompactionIOConfig(inputSpec, dropExisting), tuningConfig, granularitySpec, dimensionsSpec, diff --git a/server/src/main/java/org/apache/druid/server/http/DataSegmentPlus.java b/server/src/main/java/org/apache/druid/server/http/DataSegmentPlus.java index e06241f6aae7..56d9155c6881 100644 --- a/server/src/main/java/org/apache/druid/server/http/DataSegmentPlus.java +++ b/server/src/main/java/org/apache/druid/server/http/DataSegmentPlus.java @@ -82,7 +82,7 @@ public DataSegmentPlus( this.upgradedFromSegmentId = upgradedFromSegmentId; this.indexingStateFingerprint = indexingStateFingerprint; } - + @Nullable @JsonProperty public DateTime getCreatedDate() 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 ee5ed58f1c61..0ab3e0593612 100644 --- a/server/src/test/java/org/apache/druid/client/indexing/ClientCompactionIntervalSpecTest.java +++ b/server/src/test/java/org/apache/druid/client/indexing/ClientCompactionIntervalSpecTest.java @@ -19,20 +19,26 @@ package org.apache.druid.client.indexing; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableList; +import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.granularity.Granularities; +import org.apache.druid.query.SegmentDescriptor; import org.apache.druid.segment.IndexIO; import org.apache.druid.server.compaction.CompactionCandidate; import org.apache.druid.server.compaction.CompactionStatus; 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 { @@ -113,4 +119,40 @@ public void testFromSegmentWithFinerSegmentGranularityAndUmbrellaIntervalNotAlig // Hence the compaction interval is modified to aling with the segmentGranularity Assert.assertEquals(Intervals.of("2015-02-09/2015-04-20"), actual.getCompactionInterval()); } + + @Test + public void testClientCompactionIntervalSpec_throwsException_whenEmptySegmentsList() + { + Interval interval = Intervals.of("2015-04-11/2015-04-12"); + List emptySegments = List.of(); + + Assert.assertThrows( + IAE.class, + () -> new ClientCompactionIntervalSpec(interval, emptySegments, null) + ); + } + + @Test + public void testClientCompactionIntervalSpec_serde() throws Exception + { + ObjectMapper mapper = new DefaultObjectMapper(); + Interval interval = Intervals.of("2015-04-11/2015-04-12"); + List segments = List.of( + new SegmentDescriptor(Intervals.of("2015-04-11/2015-04-12"), "v1", 0) + ); + + // Test with uncompactedSegments (minor compaction) + ClientCompactionIntervalSpec withSegments = new ClientCompactionIntervalSpec(interval, segments, "sha256hash"); + String json1 = mapper.writeValueAsString(withSegments); + ClientCompactionIntervalSpec deserialized1 = mapper.readValue(json1, ClientCompactionIntervalSpec.class); + Assert.assertEquals(withSegments, deserialized1); + Assert.assertEquals(segments, deserialized1.getUncompactedSegments()); + + // Test without uncompactedSegments (full compaction) + ClientCompactionIntervalSpec withoutSegments = new ClientCompactionIntervalSpec(interval, null, null); + String json2 = mapper.writeValueAsString(withoutSegments); + ClientCompactionIntervalSpec deserialized2 = mapper.readValue(json2, ClientCompactionIntervalSpec.class); + Assert.assertEquals(withoutSegments, deserialized2); + Assert.assertNull(deserialized2.getUncompactedSegments()); + } } diff --git a/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java b/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java index 9add89949c83..8e166dbfcf91 100644 --- a/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java +++ b/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java @@ -74,6 +74,7 @@ import org.apache.druid.timeline.partition.NumberedShardSpec; import org.apache.druid.timeline.partition.PartialShardSpec; import org.apache.druid.timeline.partition.PartitionIds; +import org.apache.druid.timeline.partition.ShardSpec; import org.apache.druid.timeline.partition.SingleDimensionShardSpec; import org.apache.druid.timeline.partition.TombstoneShardSpec; import org.assertj.core.api.Assertions; @@ -559,14 +560,149 @@ public void testCommitReplaceSegments() ImmutableMap.of("path", "b-" + i), ImmutableList.of("dim1"), ImmutableList.of("m1"), - new NumberedShardSpec(i, 9), + new NumberedShardSpec(i - 1, 8), + 9, + 100 + ); + replacingSegments.add(segment); + } + + Assert.assertTrue(coordinator.commitReplaceSegments(replacingSegments, Set.of(replaceLock), null).isSuccess()); + + Assert.assertEquals( + 2L * segmentsAppendedWithReplaceLock.size() + replacingSegments.size(), + retrieveUsedSegmentIds(derbyConnectorRule.metadataTablesConfigSupplier().get()).size() + ); + + final Set usedSegments + = new HashSet<>(retrieveUsedSegments(derbyConnectorRule.metadataTablesConfigSupplier().get())); + + final Map upgradedFromSegmentIdMap = coordinator.retrieveUpgradedFromSegmentIds( + "foo", + usedSegments.stream().map(DataSegment::getId).map(SegmentId::toString).collect(Collectors.toSet()) + ); + + Assert.assertTrue(usedSegments.containsAll(segmentsAppendedWithReplaceLock)); + for (DataSegment appendSegment : segmentsAppendedWithReplaceLock) { + Assert.assertNull(upgradedFromSegmentIdMap.get(appendSegment.getId().toString())); + } + usedSegments.removeAll(segmentsAppendedWithReplaceLock); + Assert.assertEquals(usedSegments, coordinator.retrieveAllUsedSegments("foo", Segments.ONLY_VISIBLE)); + + Assert.assertTrue(usedSegments.containsAll(replacingSegments)); + for (DataSegment replaceSegment : replacingSegments) { + Assert.assertNull(upgradedFromSegmentIdMap.get(replaceSegment.getId().toString())); + } + usedSegments.removeAll(replacingSegments); + + Assert.assertEquals(segmentsAppendedWithReplaceLock.size(), usedSegments.size()); + for (DataSegment segmentReplicaWithNewVersion : usedSegments) { + boolean hasBeenCarriedForward = false; + for (DataSegment appendedSegment : segmentsAppendedWithReplaceLock) { + if (appendedSegment.getLoadSpec().equals(segmentReplicaWithNewVersion.getLoadSpec())) { + Assert.assertEquals( + appendedSegment.getId().toString(), + upgradedFromSegmentIdMap.get(segmentReplicaWithNewVersion.getId().toString()) + ); + hasBeenCarriedForward = true; + break; + } + } + Assert.assertTrue(hasBeenCarriedForward); + } + + List pendingSegmentsInInterval = + coordinator.getPendingSegments("foo", Intervals.of("2023-01-01/2023-02-01")); + Assert.assertEquals(2, pendingSegmentsInInterval.size()); + final SegmentId rootPendingSegmentId = pendingSegmentInInterval.getId().asSegmentId(); + if (pendingSegmentsInInterval.get(0).getUpgradedFromSegmentId() == null) { + Assert.assertEquals(rootPendingSegmentId, pendingSegmentsInInterval.get(0).getId().asSegmentId()); + Assert.assertEquals(rootPendingSegmentId.toString(), pendingSegmentsInInterval.get(1).getUpgradedFromSegmentId()); + } else { + Assert.assertEquals(rootPendingSegmentId, pendingSegmentsInInterval.get(1).getId().asSegmentId()); + Assert.assertEquals(rootPendingSegmentId.toString(), pendingSegmentsInInterval.get(0).getUpgradedFromSegmentId()); + } + + List pendingSegmentsOutsideInterval = + coordinator.getPendingSegments("foo", Intervals.of("2023-04-01/2023-05-01")); + Assert.assertEquals(1, pendingSegmentsOutsideInterval.size()); + Assert.assertEquals( + pendingSegmentOutsideInterval.getId().asSegmentId(), pendingSegmentsOutsideInterval.get(0).getId().asSegmentId() + ); + } + + @Test + public void testCommitReplaceSegmentsWithUpdatedCorePartitions() + { + // this test is very similar to testCommitReplaceSegments, except both append/replace segments use DimensionRangeShardSpec + final ReplaceTaskLock replaceLock = new ReplaceTaskLock("g1", Intervals.of("2023-01-01/2023-02-01"), "2023-02-01"); + final Set segmentsAppendedWithReplaceLock = new HashSet<>(); + final Map appendedSegmentToReplaceLockMap = new HashMap<>(); + final PendingSegmentRecord pendingSegmentInInterval = PendingSegmentRecord.create( + new SegmentIdWithShardSpec( + "foo", + Intervals.of("2023-01-01/2023-01-02"), + "2023-01-02", + new NumberedShardSpec(100, 0) + ), + "", + "", + null, + "append" + ); + final PendingSegmentRecord pendingSegmentOutsideInterval = PendingSegmentRecord.create( + new SegmentIdWithShardSpec( + "foo", + Intervals.of("2023-04-01/2023-04-02"), + "2023-01-02", + new NumberedShardSpec(100, 0) + ), + "", + "", + null, + "append" + ); + for (int i = 1; i < 9; i++) { + final DataSegment segment = new DataSegment( + "foo", + Intervals.of("2023-01-0" + i + "/2023-01-0" + (i + 1)), + "2023-01-0" + i, + ImmutableMap.of("path", "a-" + i), + ImmutableList.of("dim1"), + ImmutableList.of("m1"), + new DimensionRangeShardSpec(List.of("dim1"), null, null, i - 1, 8), + 9, + 100 + ); + segmentsAppendedWithReplaceLock.add(segment); + appendedSegmentToReplaceLockMap.put(segment, replaceLock); + } + + segmentSchemaTestUtils.insertUsedSegments(segmentsAppendedWithReplaceLock, Collections.emptyMap()); + insertPendingSegments( + "foo", + List.of(pendingSegmentInInterval, pendingSegmentOutsideInterval), + true + ); + insertIntoUpgradeSegmentsTable(appendedSegmentToReplaceLockMap, derbyConnectorRule.metadataTablesConfigSupplier().get()); + + final Set replacingSegments = new HashSet<>(); + for (int i = 1; i < 9; i++) { + final DataSegment segment = new DataSegment( + "foo", + Intervals.of("2023-01-01/2023-02-01"), + "2023-02-01", + ImmutableMap.of("path", "b-" + i), + ImmutableList.of("dim1"), + ImmutableList.of("m1"), + new DimensionRangeShardSpec(List.of("dim1"), null, null, i - 1, 8), 9, 100 ); replacingSegments.add(segment); } - coordinator.commitReplaceSegments(replacingSegments, ImmutableSet.of(replaceLock), null); + Assert.assertTrue(coordinator.commitReplaceSegments(replacingSegments, Set.of(replaceLock), null).isSuccess()); Assert.assertEquals( 2L * segmentsAppendedWithReplaceLock.size() + replacingSegments.size(), @@ -587,6 +723,12 @@ public void testCommitReplaceSegments() } usedSegments.removeAll(segmentsAppendedWithReplaceLock); + Set fetched = coordinator.retrieveAllUsedSegments("foo", Segments.ONLY_VISIBLE); + Assert.assertEquals(usedSegments, fetched); + // all segments have the same corePartitions, exactly the size of replaced + appended + List shardSpecs = fetched.stream().map(DataSegment::getShardSpec).toList(); + Assert.assertTrue(shardSpecs.stream().allMatch(s -> s.getNumCorePartitions() == usedSegments.size())); + Assert.assertTrue(shardSpecs.stream().allMatch(s -> s instanceof DimensionRangeShardSpec)); Assert.assertTrue(usedSegments.containsAll(replacingSegments)); for (DataSegment replaceSegment : replacingSegments) { Assert.assertNull(upgradedFromSegmentIdMap.get(replaceSegment.getId().toString())); diff --git a/server/src/test/java/org/apache/druid/server/compaction/MostFragmentedIntervalFirstPolicyTest.java b/server/src/test/java/org/apache/druid/server/compaction/MostFragmentedIntervalFirstPolicyTest.java index fd5c21e0a2c5..f242cec16805 100644 --- a/server/src/test/java/org/apache/druid/server/compaction/MostFragmentedIntervalFirstPolicyTest.java +++ b/server/src/test/java/org/apache/druid/server/compaction/MostFragmentedIntervalFirstPolicyTest.java @@ -39,7 +39,8 @@ public class MostFragmentedIntervalFirstPolicyTest @Test public void test_thresholdValues_ofDefaultPolicy() { - final MostFragmentedIntervalFirstPolicy policy = new MostFragmentedIntervalFirstPolicy(null, null, null, null); + final MostFragmentedIntervalFirstPolicy policy = + new MostFragmentedIntervalFirstPolicy(null, null, null, null, null); Assertions.assertEquals(100, policy.getMinUncompactedCount()); Assertions.assertEquals(new HumanReadableBytes("10MiB"), policy.getMinUncompactedBytes()); Assertions.assertEquals(new HumanReadableBytes("2GiB"), policy.getMaxAverageUncompactedBytesPerSegment()); @@ -54,6 +55,7 @@ public void test_checkEligibilityForCompaction_fails_ifUncompactedCountLessThanC minUncompactedCount, HumanReadableBytes.valueOf(1), HumanReadableBytes.valueOf(10_000), + null, null ); @@ -77,6 +79,7 @@ public void test_checkEligibilityForCompaction_fails_ifUncompactedBytesLessThanC 1, minUncompactedBytes, HumanReadableBytes.valueOf(10_000), + null, null ); @@ -100,6 +103,7 @@ public void test_checkEligibilityForCompaction_fails_ifAvgSegmentSizeGreaterThan 1, HumanReadableBytes.valueOf(100), maxAvgSegmentSize, + null, null ); @@ -122,6 +126,7 @@ public void test_policy_favorsIntervalWithMoreUncompactedSegments_ifTotalBytesIs 1, HumanReadableBytes.valueOf(1), HumanReadableBytes.valueOf(10_000), + null, null ); @@ -142,6 +147,7 @@ public void test_policy_favorsIntervalWithMoreUncompactedSegments_ifAverageSizeI 1, HumanReadableBytes.valueOf(1), HumanReadableBytes.valueOf(10_000), + null, null ); @@ -162,6 +168,7 @@ public void test_policy_favorsIntervalWithSmallerSegments_ifCountIsEqual() 1, HumanReadableBytes.valueOf(1), HumanReadableBytes.valueOf(10_000), + null, null ); @@ -182,6 +189,7 @@ public void test_compareCandidates_returnsZeroIfSegmentCountAndAvgSizeScaleEquiv 100, HumanReadableBytes.valueOf(1), HumanReadableBytes.valueOf(100), + null, null ); @@ -211,6 +219,7 @@ public void test_serde_allFieldsSet() throws IOException 1, HumanReadableBytes.valueOf(2), HumanReadableBytes.valueOf(3), + 50, "foo" ); final DefaultObjectMapper mapper = new DefaultObjectMapper(); @@ -222,13 +231,93 @@ public void test_serde_allFieldsSet() throws IOException @Test public void test_serde_noFieldsSet() throws IOException { - final MostFragmentedIntervalFirstPolicy policy = new MostFragmentedIntervalFirstPolicy(null, null, null, null); + final MostFragmentedIntervalFirstPolicy policy = + new MostFragmentedIntervalFirstPolicy(null, null, null, null, null); final DefaultObjectMapper mapper = new DefaultObjectMapper(); final CompactionCandidateSearchPolicy policy2 = mapper.readValue(mapper.writeValueAsString(policy), CompactionCandidateSearchPolicy.class); Assertions.assertEquals(policy, policy2); } + @Test + public void test_compactionMode_returnsMinorCompactionMode_whenPercentageBelowThreshold() + { + // Set threshold to 0.5 (50%) + final MostFragmentedIntervalFirstPolicy policy = new MostFragmentedIntervalFirstPolicy( + 1, + HumanReadableBytes.valueOf(1), + HumanReadableBytes.valueOf(10_000), + 50, + null + ); + + final CompactionStatistics compacted = CompactionStatistics.create(1200L, 10, 1L); + final CompactionStatistics uncompacted = CompactionStatistics.create(400L, 100, 1L); + final CompactionStatus status = CompactionStatus.pending(compacted, uncompacted, List.of(SEGMENT), ""); + + final CompactionCandidate candidate = CompactionCandidate.from(List.of(SEGMENT), null, status); + final CompactionCandidateSearchPolicy.Eligibility eligibility = + policy.checkEligibilityForCompaction(candidate, null); + + Assertions.assertEquals(CompactionMode.UNCOMPACTED_SEGMENTS_ONLY, eligibility.getMode()); + Assertions.assertTrue(eligibility.isEligible()); + } + + @Test + public void test_compactionMode_returnsFullCompaction_whenPercentageAboveThreshold() + { + // Set threshold to 0.5 (50%) + final MostFragmentedIntervalFirstPolicy policy = new MostFragmentedIntervalFirstPolicy( + 1, + HumanReadableBytes.valueOf(1), + HumanReadableBytes.valueOf(10_000), + 50, + null + ); + + final CompactionStatus status = + CompactionStatus.pending( + CompactionStatistics.create(500L, 5, 1), + CompactionStatistics.create(600L, 100, 1), + List.of(), + "" + ); + final CompactionCandidate candidate = CompactionCandidate.from(List.of(SEGMENT), null, status); + final CompactionCandidateSearchPolicy.Eligibility eligibility = + policy.checkEligibilityForCompaction(candidate, null); + + Assertions.assertEquals(CompactionMode.ALL_SEGMENTS, eligibility.getMode()); + Assertions.assertTrue(eligibility.isEligible()); + } + + @Test + public void test_compactionMode_returnsFullCompaction_whenThresholdIsDefault() + { + // Default threshold is 0 + final MostFragmentedIntervalFirstPolicy policy = new MostFragmentedIntervalFirstPolicy( + 1, + HumanReadableBytes.valueOf(1), + HumanReadableBytes.valueOf(10_000), + null, + null + ); + + // With default threshold 0, any positive percentage >= 0, so always ALL_SEGMENTS_ELIGIBLE + final CompactionStatus status = + CompactionStatus.pending( + CompactionStatistics.create(1_000L, 10, 1), + CompactionStatistics.create(100L, 100, 1), + List.of(), + "" + ); + final CompactionCandidate candidate = CompactionCandidate.from(List.of(SEGMENT), null, status); + final CompactionCandidateSearchPolicy.Eligibility eligibility = + policy.checkEligibilityForCompaction(candidate, null); + + Assertions.assertEquals(CompactionMode.ALL_SEGMENTS, eligibility.getMode()); + Assertions.assertTrue(eligibility.isEligible()); + } + private CompactionCandidate createCandidate(int numSegments, long avgSizeBytes) { final CompactionStatistics dummyCompactedStats = CompactionStatistics.create(1L, 1L, 1L); @@ -240,7 +329,7 @@ private CompactionCandidate createCandidate(int numSegments, long avgSizeBytes) return CompactionCandidate.from( List.of(SEGMENT), null, - CompactionStatus.pending(dummyCompactedStats, uncompactedStats, "") + CompactionStatus.pending(dummyCompactedStats, uncompactedStats, 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 b7f01cf57d2e..1e8a8f6ea0ac 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 @@ -1106,6 +1106,7 @@ public void testCompactWithGranularitySpecConflictWithActiveCompactionTask() new ClientCompactionIOConfig( new ClientCompactionIntervalSpec( Intervals.of("2000/2099"), + null, "testSha256OfSortedSegmentIds" ), null