From 7a3f46ddf1fcf4a107fe740751d58a08dc728f2a Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Thu, 14 Aug 2025 11:55:39 +0530 Subject: [PATCH 01/25] Add catalog templates for cascading compaction --- .../NewestSegmentFirstPolicyBenchmark.java | 5 +- embedded-tests/pom.xml | 5 + .../compact/CompactionSupervisorTest.java | 316 ++++++++++ .../compact/CatalogCompactionTest.java | 4 +- .../apache/druid/guice/SupervisorModule.java | 10 +- .../indexing/common/task/CompactionTask.java | 9 +- .../druid/indexing/common/task/Tasks.java | 1 - .../compact/CascadingCompactionTemplate.java | 238 +++++++ .../compact/CatalogCompactionJobTemplate.java | 96 +++ .../CompactionConfigBasedJobTemplate.java | 127 ++++ .../druid/indexing/compact/CompactionJob.java | 93 +++ .../indexing/compact/CompactionJobParams.java | 68 ++ .../indexing/compact/CompactionJobQueue.java | 297 +++++++++ .../compact/CompactionJobTemplate.java | 87 +++ .../indexing/compact/CompactionRule.java | 52 ++ .../indexing/compact/CompactionScheduler.java | 2 +- .../compact/CompactionSupervisor.java | 38 +- .../compact/CompactionSupervisorSpec.java | 14 +- .../compact/InlineCompactionJobTemplate.java | 117 ++++ .../indexing/compact/LocalOverlordClient.java | 8 +- .../compact/OverlordCompactionScheduler.java | 102 ++- .../input/DruidDatasourceDestination.java | 42 ++ .../indexing/input/DruidInputSource.java | 19 + .../supervisor/BatchIndexingSupervisor.java | 48 ++ .../BatchIndexingSupervisorSpec.java | 41 ++ .../compact/CompactionSupervisorSpecTest.java | 4 +- .../OverlordCompactionSchedulerTest.java | 78 ++- .../druid/data/output/OutputDestination.java | 33 + .../druid/java/util/common/Intervals.java | 13 + .../catalog/model/SchemaRegistryImpl.java | 15 +- .../catalog/model/TableDefnRegistry.java | 4 +- .../apache/druid/catalog/model/TableId.java | 5 + .../model/table/IndexingTemplateDefn.java | 65 ++ .../indexing/template/BatchIndexingJob.java | 90 +++ .../template/BatchIndexingJobTemplate.java | 50 ++ .../druid/indexing/template/JobParams.java | 30 + .../compaction/CompactionRunSimulator.java | 14 +- .../compaction/CompactionSlotManager.java | 358 +++++++++++ .../compaction/CompactionSnapshotBuilder.java | 94 +++ .../server/compaction/CompactionStatus.java | 56 +- .../compaction/CompactionStatusTracker.java | 57 +- .../DataSourceCompactibleSegmentIterator.java | 13 +- ...riorityBasedCompactionSegmentIterator.java | 6 +- .../coordinator/duty/CompactSegments.java | 582 +++++------------- .../stats/CoordinatorRunStats.java | 14 + .../druid/server/coordinator/stats/Stats.java | 2 + .../CompactionRunSimulatorTest.java | 8 +- .../compaction/CompactionStatusTest.java | 12 +- .../CompactionStatusTrackerTest.java | 47 +- .../NewestSegmentFirstPolicyTest.java | 23 +- .../coordinator/DruidCoordinatorTest.java | 13 +- .../coordinator/duty/CompactSegmentsTest.java | 23 +- .../CoordinatorSimulationBuilder.java | 2 +- 53 files changed, 2873 insertions(+), 677 deletions(-) create mode 100644 embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionSupervisorTest.java create mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/compact/CascadingCompactionTemplate.java create mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/compact/CatalogCompactionJobTemplate.java create mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionConfigBasedJobTemplate.java create mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionJob.java create mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionJobParams.java create mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionJobQueue.java create mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionJobTemplate.java create mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionRule.java create mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/compact/InlineCompactionJobTemplate.java create mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/input/DruidDatasourceDestination.java create mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/BatchIndexingSupervisor.java create mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/BatchIndexingSupervisorSpec.java create mode 100644 processing/src/main/java/org/apache/druid/data/output/OutputDestination.java create mode 100644 server/src/main/java/org/apache/druid/catalog/model/table/IndexingTemplateDefn.java create mode 100644 server/src/main/java/org/apache/druid/indexing/template/BatchIndexingJob.java create mode 100644 server/src/main/java/org/apache/druid/indexing/template/BatchIndexingJobTemplate.java create mode 100644 server/src/main/java/org/apache/druid/indexing/template/JobParams.java create mode 100644 server/src/main/java/org/apache/druid/server/compaction/CompactionSlotManager.java create mode 100644 server/src/main/java/org/apache/druid/server/compaction/CompactionSnapshotBuilder.java diff --git a/benchmarks/src/test/java/org/apache/druid/server/coordinator/NewestSegmentFirstPolicyBenchmark.java b/benchmarks/src/test/java/org/apache/druid/server/coordinator/NewestSegmentFirstPolicyBenchmark.java index a0a1da9369a0..c9c4599fad76 100644 --- a/benchmarks/src/test/java/org/apache/druid/server/coordinator/NewestSegmentFirstPolicyBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/server/coordinator/NewestSegmentFirstPolicyBenchmark.java @@ -21,11 +21,9 @@ import com.google.common.collect.ImmutableList; import org.apache.druid.client.DataSourcesSnapshot; -import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.server.compaction.CompactionCandidateSearchPolicy; import org.apache.druid.server.compaction.CompactionSegmentIterator; -import org.apache.druid.server.compaction.CompactionStatusTracker; import org.apache.druid.server.compaction.NewestSegmentFirstPolicy; import org.apache.druid.server.compaction.PriorityBasedCompactionSegmentIterator; import org.apache.druid.timeline.DataSegment; @@ -137,8 +135,7 @@ public void measureNewestSegmentFirstPolicy(Blackhole blackhole) policy, compactionConfigs, dataSources, - Collections.emptyMap(), - new CompactionStatusTracker(new DefaultObjectMapper()) + Collections.emptyMap() ); for (int i = 0; i < numCompactionTaskSlots && iterator.hasNext(); i++) { blackhole.consume(iterator.next()); diff --git a/embedded-tests/pom.xml b/embedded-tests/pom.xml index 6d4ad968fd8c..995743d90c5a 100644 --- a/embedded-tests/pom.xml +++ b/embedded-tests/pom.xml @@ -64,6 +64,11 @@ druid-multi-stage-query ${project.parent.version} + + org.apache.druid.extensions + druid-catalog + ${project.parent.version} + org.apache.druid druid-server 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 new file mode 100644 index 000000000000..82b32e67a785 --- /dev/null +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionSupervisorTest.java @@ -0,0 +1,316 @@ +/* + * 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.testing.embedded.compact; + +import org.apache.druid.catalog.guice.CatalogClientModule; +import org.apache.druid.catalog.guice.CatalogCoordinatorModule; +import org.apache.druid.catalog.model.ResolvedTable; +import org.apache.druid.catalog.model.TableId; +import org.apache.druid.catalog.model.TableSpec; +import org.apache.druid.catalog.model.table.IndexingTemplateDefn; +import org.apache.druid.catalog.sync.CatalogClient; +import org.apache.druid.common.utils.IdUtils; +import org.apache.druid.indexing.common.task.IndexTask; +import org.apache.druid.indexing.compact.CascadingCompactionTemplate; +import org.apache.druid.indexing.compact.CatalogCompactionJobTemplate; +import org.apache.druid.indexing.compact.CompactionJobTemplate; +import org.apache.druid.indexing.compact.CompactionRule; +import org.apache.druid.indexing.compact.CompactionSupervisorSpec; +import org.apache.druid.indexing.compact.InlineCompactionJobTemplate; +import org.apache.druid.indexing.overlord.Segments; +import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.java.util.common.granularity.Granularities; +import org.apache.druid.java.util.common.granularity.Granularity; +import org.apache.druid.query.DruidMetrics; +import org.apache.druid.rpc.UpdateResponse; +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.UserCompactionTaskGranularityConfig; +import org.apache.druid.testing.embedded.EmbeddedBroker; +import org.apache.druid.testing.embedded.EmbeddedCoordinator; +import org.apache.druid.testing.embedded.EmbeddedDruidCluster; +import org.apache.druid.testing.embedded.EmbeddedHistorical; +import org.apache.druid.testing.embedded.EmbeddedIndexer; +import org.apache.druid.testing.embedded.EmbeddedOverlord; +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.timeline.DataSegment; +import org.joda.time.DateTime; +import org.joda.time.Period; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; + +import java.util.List; +import java.util.Map; +import java.util.Set; + +public class CompactionSupervisorTest extends EmbeddedClusterTestBase +{ + protected final EmbeddedBroker broker = new EmbeddedBroker(); + protected final EmbeddedIndexer indexer = new EmbeddedIndexer() + .addProperty("druid.worker.capacity", "8"); + protected final EmbeddedOverlord overlord = new EmbeddedOverlord() + .addProperty("druid.manager.segments.pollDuration", "PT1s") + .addProperty("druid.manager.segments.useIncrementalCache", "always"); + protected final EmbeddedHistorical historical = new EmbeddedHistorical(); + protected final EmbeddedCoordinator coordinator = new EmbeddedCoordinator() + .addProperty("druid.manager.segments.useIncrementalCache", "always"); + + @Override + public EmbeddedDruidCluster createCluster() + { + return EmbeddedDruidCluster.withEmbeddedDerbyAndZookeeper() + .useLatchableEmitter() + .addExtensions(CatalogClientModule.class, CatalogCoordinatorModule.class) + .addServer(coordinator) + .addServer(overlord) + .addServer(indexer) + .addServer(historical) + .addServer(broker) + .addServer(new EmbeddedRouter()); + } + + @BeforeAll + public void enableCompactionSupervisors() + { + final UpdateResponse updateResponse = cluster.callApi().onLeaderOverlord( + o -> o.updateClusterCompactionConfig(new ClusterCompactionConfig(1.0, 10, null, true, null)) + ); + Assertions.assertTrue(updateResponse.isSuccess()); + } + + @Test + public void test_ingestDayGranularity_andCompactToMonthGranularity() + { + // Ingest data at DAY granularity and verify + runIngestionAtGranularity( + "DAY", + "2025-06-01T00:00:00.000Z,shirt,105" + + "\n2025-06-02T00:00:00.000Z,trousers,210" + + "\n2025-06-03T00:00:00.000Z,jeans,150" + ); + Set segments = cluster.callApi().getVisibleUsedSegments(dataSource, overlord); + Assertions.assertEquals(3, segments.size()); + segments.forEach( + segment -> Assertions.assertTrue(Granularities.DAY.isAligned(segment.getInterval())) + ); + + // Create a compaction config with MONTH granularity + InlineSchemaDataSourceCompactionConfig compactionConfig = + InlineSchemaDataSourceCompactionConfig + .builder() + .forDataSource(dataSource) + .withSkipOffsetFromLatest(Period.seconds(0)) + .withGranularitySpec( + new UserCompactionTaskGranularityConfig(Granularities.MONTH, null, null) + ) + .build(); + + runCompactionWithSpec(compactionConfig); + + // Verify that segments are now compacted to MONTH granularity + segments = cluster.callApi().getVisibleUsedSegments(dataSource, overlord); + Assertions.assertEquals(1, segments.size()); + Assertions.assertTrue( + Granularities.MONTH.isAligned(segments.iterator().next().getInterval()) + ); + } + + @Test + public void test_ingestHourGranularity_andCompactToDayAndMonth_withInlineTemplates() + { + // Create a cascading template with DAY and MONTH granularity + CascadingCompactionTemplate cascadingTemplate = new CascadingCompactionTemplate( + dataSource, + List.of( + new CompactionRule(Period.days(2), new InlineCompactionJobTemplate(null, Granularities.DAY)), + new CompactionRule(Period.days(100), new InlineCompactionJobTemplate(null, Granularities.MONTH)) + ) + ); + + final CompactionSupervisorSpec compactionSupervisor + = new CompactionSupervisorSpec(cascadingTemplate, false, null); + cluster.callApi().postSupervisor(compactionSupervisor); + + ingestRecordsAtGranularity(2400, "HOUR"); + runCompactionWithSpec(cascadingTemplate); + verifyDayAndMonth(); + } + + @Test + public void test_ingestHourGranularity_andCompactToDayAndMonth_withCatalogTemplates() + { + ingestRecordsAtGranularity(2400, "HOUR"); + + // Add compaction templates to catalog + final String dayGranularityTemplateId = saveTemplateToCatalog( + new InlineCompactionJobTemplate(null, Granularities.DAY) + ); + final String monthGranularityTemplateId = saveTemplateToCatalog( + new InlineCompactionJobTemplate(null, Granularities.MONTH) + ); + + // Create a cascading template with DAY and MONTH granularity + CascadingCompactionTemplate cascadingTemplate = new CascadingCompactionTemplate( + dataSource, + List.of( + new CompactionRule(Period.days(2), new CatalogCompactionJobTemplate(dayGranularityTemplateId, null)), + new CompactionRule(Period.days(100), new CatalogCompactionJobTemplate(monthGranularityTemplateId, null)) + ) + ); + + runCompactionWithSpec(cascadingTemplate); + verifyDayAndMonth(); + } + + private void ingestRecordsAtGranularity(int numRecords, String granularityName) + { + // Ingest data at HOUR granularity and verify + Granularity granularity = Granularity.fromString(granularityName); + runIngestionAtGranularity( + granularityName, + createHourlyInlineDataCsv(DateTimes.nowUtc(), numRecords) + ); + List segments = List.copyOf( + overlord.bindings() + .segmentsMetadataStorage() + .retrieveAllUsedSegments(dataSource, Segments.ONLY_VISIBLE) + ); + Assertions.assertEquals(numRecords, segments.size()); + segments.forEach( + segment -> Assertions.assertTrue(granularity.isAligned(segment.getInterval())) + ); + } + + private void runCompactionWithSpec(DataSourceCompactionConfig config) + { + final CompactionSupervisorSpec compactionSupervisor + = new CompactionSupervisorSpec(config, false, null); + cluster.callApi().postSupervisor(compactionSupervisor); + + // Wait for compaction tasks to be submitted + final int numCompactionTasks = overlord.latchableEmitter().waitForEvent( + event -> event.hasMetricName("compact/task/count") + .hasDimension(DruidMetrics.DATASOURCE, dataSource) + .hasValueAtLeast(1L) + ).getValue().intValue(); + + // Wait for the submitted tasks to finish + overlord.latchableEmitter().waitForEventAggregate( + event -> event.hasMetricName("task/run/time") + .hasDimension(DruidMetrics.TASK_TYPE, "compact") + .hasDimension(DruidMetrics.DATASOURCE, dataSource), + agg -> agg.hasCountAtLeast(numCompactionTasks) + ); + + } + + private void verifyDayAndMonth() + { + // Verify that segments are now compacted to MONTH and DAY granularity + List segments = List.copyOf( + overlord.bindings() + .segmentsMetadataStorage() + .retrieveAllUsedSegments(dataSource, Segments.ONLY_VISIBLE) + ); + Assertions.assertTrue(segments.size() < 2400); + + int numMonthSegments = 0; + int numDaySegments = 0; + int numHourSegments = 0; + + for (DataSegment segment : segments) { + if (Granularities.HOUR.isAligned(segment.getInterval())) { + ++numHourSegments; + } else if (Granularities.DAY.isAligned(segment.getInterval())) { + ++numDaySegments; + } else if (Granularities.MONTH.isAligned(segment.getInterval())) { + ++numMonthSegments; + } + } + + // Verify that atleast 2 days are fully compacted to DAY + Assertions.assertTrue(numDaySegments >= 2); + + // Verify that atleast 2 months are fully compacted to MONTH + Assertions.assertTrue(numMonthSegments >= 2); + + // Verify that number of uncompacted days is between 5 and 38 + Assertions.assertTrue(5 * 24 <= numHourSegments && numHourSegments <= 38 * 24); + } + + private String saveTemplateToCatalog(CompactionJobTemplate template) + { + final String templateId = IdUtils.getRandomId(); + final CatalogClient catalogClient = overlord.bindings().getInstance(CatalogClient.class); + + final TableId tableId = TableId.of(TableId.INDEXING_TEMPLATE_SCHEMA, templateId); + catalogClient.createTable( + tableId, + new TableSpec( + IndexingTemplateDefn.TYPE, + Map.of(IndexingTemplateDefn.PROPERTY_PAYLOAD, template), + null + ) + ); + + ResolvedTable table = catalogClient.resolveTable(tableId); + Assertions.assertNotNull(table); + + return templateId; + } + + private void runIngestionAtGranularity( + String granularity, + String inlineDataCsv + ) + { + final String taskId = IdUtils.getRandomId(); + final IndexTask task = createIndexTaskForInlineData(taskId, granularity, inlineDataCsv); + + cluster.callApi().runTask(task, overlord); + } + + private String createHourlyInlineDataCsv(DateTime latestRecordTimestamp, int numRecords) + { + final StringBuilder builder = new StringBuilder(); + for (int i = 0; i < numRecords; ++i) { + builder.append(latestRecordTimestamp.minusHours(i)) + .append(",").append("item_").append(IdUtils.getRandomId()) + .append(",").append(0) + .append("\n"); + } + + return builder.toString(); + } + + private IndexTask createIndexTaskForInlineData(String taskId, String granularity, String inlineDataCsv) + { + return MoreResources.Task.BASIC_INDEX + .get() + .segmentGranularity(granularity) + .inlineInputSourceWithData(inlineDataCsv) + .dataSource(dataSource) + .withId(taskId); + } +} diff --git a/extensions-core/druid-catalog/src/test/java/org/apache/druid/catalog/compact/CatalogCompactionTest.java b/extensions-core/druid-catalog/src/test/java/org/apache/druid/catalog/compact/CatalogCompactionTest.java index 9f3106b88441..798c5f2183f9 100644 --- a/extensions-core/druid-catalog/src/test/java/org/apache/druid/catalog/compact/CatalogCompactionTest.java +++ b/extensions-core/druid-catalog/src/test/java/org/apache/druid/catalog/compact/CatalogCompactionTest.java @@ -32,6 +32,7 @@ import org.apache.druid.indexing.overlord.Segments; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.Granularities; +import org.apache.druid.query.DruidMetrics; import org.apache.druid.rpc.UpdateResponse; import org.apache.druid.server.coordinator.CatalogDataSourceCompactionConfig; import org.apache.druid.server.coordinator.ClusterCompactionConfig; @@ -119,7 +120,8 @@ public void test_ingestDayGranularity_andCompactToMonthGranularity() // Wait for compaction to finish overlord.latchableEmitter().waitForEvent( event -> event.hasMetricName("task/run/time") - .hasDimension("taskType", "compact") + .hasDimension(DruidMetrics.TASK_TYPE, "compact") + .hasDimension(DruidMetrics.DATASOURCE, dataSource) ); // Verify that segments are now compacted to MONTH granularity diff --git a/indexing-service/src/main/java/org/apache/druid/guice/SupervisorModule.java b/indexing-service/src/main/java/org/apache/druid/guice/SupervisorModule.java index 73e8e06e8964..d334267b3d32 100644 --- a/indexing-service/src/main/java/org/apache/druid/guice/SupervisorModule.java +++ b/indexing-service/src/main/java/org/apache/druid/guice/SupervisorModule.java @@ -25,7 +25,11 @@ import com.fasterxml.jackson.databind.module.SimpleModule; import com.google.common.collect.ImmutableList; import com.google.inject.Binder; +import org.apache.druid.indexing.compact.CascadingCompactionTemplate; +import org.apache.druid.indexing.compact.CatalogCompactionJobTemplate; import org.apache.druid.indexing.compact.CompactionSupervisorSpec; +import org.apache.druid.indexing.compact.InlineCompactionJobTemplate; +import org.apache.druid.indexing.input.DruidDatasourceDestination; import org.apache.druid.indexing.overlord.supervisor.SupervisorStateManagerConfig; import org.apache.druid.indexing.scheduledbatch.ScheduledBatchSupervisorSpec; import org.apache.druid.initialization.DruidModule; @@ -46,8 +50,12 @@ public List getJacksonModules() return ImmutableList.of( new SimpleModule(getClass().getSimpleName()) .registerSubtypes( + new NamedType(InlineCompactionJobTemplate.class, InlineCompactionJobTemplate.TYPE), + new NamedType(CatalogCompactionJobTemplate.class, CatalogCompactionJobTemplate.TYPE), + new NamedType(CascadingCompactionTemplate.class, CascadingCompactionTemplate.TYPE), new NamedType(CompactionSupervisorSpec.class, CompactionSupervisorSpec.TYPE), - new NamedType(ScheduledBatchSupervisorSpec.class, ScheduledBatchSupervisorSpec.TYPE) + new NamedType(ScheduledBatchSupervisorSpec.class, ScheduledBatchSupervisorSpec.TYPE), + new NamedType(DruidDatasourceDestination.class, DruidDatasourceDestination.TYPE) ) ); } 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 2a6457e36349..508a466cd8d7 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java @@ -27,7 +27,6 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; -import com.google.common.base.Verify; import com.google.common.collect.BiMap; import com.google.common.collect.HashBiMap; import com.google.common.collect.ImmutableList; @@ -95,8 +94,8 @@ import org.apache.druid.segment.transform.CompactionTransformSpec; import org.apache.druid.segment.transform.TransformSpec; import org.apache.druid.segment.writeout.SegmentWriteOutMediumFactory; +import org.apache.druid.server.compaction.CompactionSlotManager; import org.apache.druid.server.coordinator.CompactionConfigValidationResult; -import org.apache.druid.server.coordinator.duty.CompactSegments; import org.apache.druid.server.lookup.cache.LookupLoadingSpec; import org.apache.druid.server.security.ResourceAction; import org.apache.druid.timeline.DataSegment; @@ -133,7 +132,7 @@ */ public class CompactionTask extends AbstractBatchIndexTask implements PendingSegmentAllocatingTask { - public static final String TYPE = "compact"; + public static final String TYPE = CompactionSlotManager.COMPACTION_TASK_TYPE; private static final Logger log = new Logger(CompactionTask.class); /** @@ -148,10 +147,6 @@ public class CompactionTask extends AbstractBatchIndexTask implements PendingSeg */ public static final String CTX_KEY_APPENDERATOR_TRACKING_TASK_ID = "appenderatorTrackingTaskId"; - static { - Verify.verify(TYPE.equals(CompactSegments.COMPACTION_TASK_TYPE)); - } - private final CompactionIOConfig ioConfig; @Nullable private final DimensionsSpec dimensionsSpec; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/Tasks.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/Tasks.java index 90fb67116b9e..b45eb45dc041 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/Tasks.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/Tasks.java @@ -47,7 +47,6 @@ public class Tasks public static final long DEFAULT_SUB_TASK_TIMEOUT_MILLIS = 0; public static final boolean DEFAULT_FORCE_TIME_CHUNK_LOCK = true; public static final boolean DEFAULT_STORE_COMPACTION_STATE = false; - public static final boolean DEFAULT_USE_MAX_MEMORY_ESTIMATES = false; public static final TaskLockType DEFAULT_TASK_LOCK_TYPE = TaskLockType.EXCLUSIVE; public static final boolean DEFAULT_USE_CONCURRENT_LOCKS = false; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CascadingCompactionTemplate.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CascadingCompactionTemplate.java new file mode 100644 index 000000000000..3b54743c766d --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CascadingCompactionTemplate.java @@ -0,0 +1,238 @@ +/* + * 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.compact; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.druid.data.input.InputSource; +import org.apache.druid.data.input.impl.AggregateProjectionSpec; +import org.apache.druid.data.output.OutputDestination; +import org.apache.druid.indexer.CompactionEngine; +import org.apache.druid.indexing.input.DruidInputSource; +import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.java.util.common.granularity.Granularity; +import org.apache.druid.query.aggregation.AggregatorFactory; +import org.apache.druid.segment.transform.CompactionTransformSpec; +import org.apache.druid.server.coordinator.DataSourceCompactionConfig; +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; +import org.joda.time.DateTime; +import org.joda.time.Interval; +import org.joda.time.Period; + +import javax.annotation.Nullable; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.stream.Collectors; + +/** + * This template never needs to be deserialized as a {@code BatchIndexingJobTemplate}, + * only as a {@link DataSourceCompactionConfig} in {@link CompactionSupervisorSpec}. + */ +public class CascadingCompactionTemplate extends CompactionJobTemplate implements DataSourceCompactionConfig +{ + public static final String TYPE = "compactCascade"; + + private final String dataSource; + private final List rules; + + @JsonCreator + public CascadingCompactionTemplate( + @JsonProperty("dataSource") String dataSource, + @JsonProperty("rules") List rules + ) + { + this.rules = rules; + this.dataSource = Objects.requireNonNull(dataSource, "'dataSource' cannot be null"); + } + + @Override + @JsonProperty + public String getDataSource() + { + return dataSource; + } + + @JsonProperty + public List getRules() + { + return rules; + } + + @Override + public List createCompactionJobs( + InputSource source, + OutputDestination destination, + CompactionJobParams jobParams + ) + { + final List allJobs = new ArrayList<>(); + + final DruidInputSource druidInputSource = ensureDruidInputSource(source); + + // Include future dates in the first rule + final DateTime currentTime = jobParams.getScheduleStartTime(); + DateTime previousRuleStartTime = DateTimes.MAX; + for (int i = 0; i < rules.size() - 1; ++i) { + final CompactionRule rule = rules.get(i); + final DateTime ruleStartTime = currentTime.minus(rule.getPeriod()); + final Interval ruleInterval = new Interval(ruleStartTime, previousRuleStartTime); + + allJobs.addAll( + createJobsUsingTemplate(rule.getTemplate(), ruleInterval, druidInputSource, destination, jobParams) + ); + + previousRuleStartTime = ruleStartTime; + } + + // Include past dates in the last rule + final CompactionRule lastRule = rules.get(rules.size() - 1); + final Interval lastRuleInterval = new Interval(DateTimes.MIN, previousRuleStartTime); + allJobs.addAll( + createJobsUsingTemplate(lastRule.getTemplate(), lastRuleInterval, druidInputSource, destination, jobParams) + ); + + return allJobs; + } + + private List createJobsUsingTemplate( + CompactionJobTemplate template, + Interval searchInterval, + DruidInputSource inputSource, + OutputDestination destination, + CompactionJobParams jobParams + ) + { + // Skip jobs if they exceed the upper bound of the search interval as the + // corresponding candidate segments fall in the purview of a prior rule + return template + .createCompactionJobs(inputSource.withInterval(searchInterval), destination, jobParams) + .stream() + .filter(job -> !job.getCompactionInterval().getEnd().isAfter(searchInterval.getEnd())) + .collect(Collectors.toList()); + } + + @Override + public String getType() + { + return TYPE; + } + + // Legacy fields from DataSourceCompactionConfig that are not used by this template + + @Nullable + @Override + public CompactionEngine getEngine() + { + return null; + } + + @Override + public int getTaskPriority() + { + return 0; + } + + @Override + public long getInputSegmentSizeBytes() + { + return 0; + } + + @Nullable + @Override + public Integer getMaxRowsPerSegment() + { + return 0; + } + + @Override + public Period getSkipOffsetFromLatest() + { + return null; + } + + @Nullable + @Override + public UserCompactionTaskQueryTuningConfig getTuningConfig() + { + return null; + } + + @Nullable + @Override + public UserCompactionTaskIOConfig getIoConfig() + { + return null; + } + + @Nullable + @Override + public Map getTaskContext() + { + return Map.of(); + } + + @Nullable + @Override + public Granularity getSegmentGranularity() + { + return null; + } + + @Nullable + @Override + public UserCompactionTaskGranularityConfig getGranularitySpec() + { + return null; + } + + @Nullable + @Override + public List getProjections() + { + return List.of(); + } + + @Nullable + @Override + public CompactionTransformSpec getTransformSpec() + { + return null; + } + + @Nullable + @Override + public UserCompactionTaskDimensionsConfig getDimensionsSpec() + { + return null; + } + + @Nullable + @Override + public AggregatorFactory[] getMetricsSpec() + { + return new AggregatorFactory[0]; + } +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CatalogCompactionJobTemplate.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CatalogCompactionJobTemplate.java new file mode 100644 index 000000000000..663faf34d98d --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CatalogCompactionJobTemplate.java @@ -0,0 +1,96 @@ +/* + * 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.compact; + +import com.fasterxml.jackson.annotation.JacksonInject; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.druid.catalog.MetadataCatalog; +import org.apache.druid.catalog.model.ResolvedTable; +import org.apache.druid.catalog.model.TableId; +import org.apache.druid.catalog.model.table.IndexingTemplateDefn; +import org.apache.druid.data.input.InputSource; +import org.apache.druid.data.output.OutputDestination; +import org.apache.druid.error.InvalidInput; +import org.apache.druid.indexing.template.BatchIndexingJobTemplate; + +import java.util.List; + +/** + * Compaction template that delegates job creation to a template stored in the + * Druid catalog. + */ +public class CatalogCompactionJobTemplate extends CompactionJobTemplate +{ + public static final String TYPE = "compactCatalog"; + + private final String templateId; + + private final TableId tableId; + private final MetadataCatalog catalog; + + @JsonCreator + public CatalogCompactionJobTemplate( + @JsonProperty("templateId") String templateId, + @JacksonInject MetadataCatalog catalog + ) + { + this.templateId = templateId; + this.catalog = catalog; + this.tableId = TableId.of(TableId.INDEXING_TEMPLATE_SCHEMA, templateId); + } + + @JsonProperty + public String getTemplateId() + { + return templateId; + } + + @Override + public List createCompactionJobs( + InputSource source, + OutputDestination target, + CompactionJobParams params + ) + { + final ResolvedTable resolvedTable = catalog.resolveTable(tableId); + if (resolvedTable == null) { + return List.of(); + } + + // Create jobs using the catalog template + final BatchIndexingJobTemplate delegate + = resolvedTable.decodeProperty(IndexingTemplateDefn.PROPERTY_PAYLOAD); + if (delegate instanceof CompactionJobTemplate) { + return ((CompactionJobTemplate) delegate).createCompactionJobs(source, target, params); + } else { + throw InvalidInput.exception( + "Template[%s] of type[%s] cannot be used for creating compaction tasks", + templateId, delegate == null ? null : delegate.getType() + ); + } + } + + @Override + public String getType() + { + return TYPE; + } +} 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 new file mode 100644 index 000000000000..76549f9cf7b5 --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionConfigBasedJobTemplate.java @@ -0,0 +1,127 @@ +/* + * 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.compact; + +import org.apache.druid.client.indexing.ClientCompactionTaskQuery; +import org.apache.druid.data.input.InputSource; +import org.apache.druid.data.output.OutputDestination; +import org.apache.druid.error.InvalidInput; +import org.apache.druid.indexer.CompactionEngine; +import org.apache.druid.indexing.input.DruidDatasourceDestination; +import org.apache.druid.indexing.input.DruidInputSource; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.server.compaction.CompactionCandidate; +import org.apache.druid.server.compaction.CompactionSlotManager; +import org.apache.druid.server.compaction.DataSourceCompactibleSegmentIterator; +import org.apache.druid.server.compaction.NewestSegmentFirstPolicy; +import org.apache.druid.server.coordinator.DataSourceCompactionConfig; +import org.apache.druid.server.coordinator.duty.CompactSegments; +import org.apache.druid.timeline.SegmentTimeline; +import org.joda.time.Interval; + +import java.util.ArrayList; +import java.util.List; +import java.util.Objects; + +/** + * This template never needs to be deserialized as a {@code BatchIndexingJobTemplate}. + * It is just a delegating template that uses a {@link DataSourceCompactionConfig} + * to create compaction jobs. + */ +public class CompactionConfigBasedJobTemplate extends CompactionJobTemplate +{ + private final DataSourceCompactionConfig config; + + public CompactionConfigBasedJobTemplate(DataSourceCompactionConfig config) + { + this.config = config; + } + + @Override + public List createCompactionJobs( + InputSource source, + OutputDestination destination, + CompactionJobParams params + ) + { + validateInput(source); + validateOutput(destination); + + final Interval searchInterval = Objects.requireNonNull(ensureDruidInputSource(source).getInterval()); + + final SegmentTimeline timeline = params.getTimeline(config.getDataSource()); + final DataSourceCompactibleSegmentIterator segmentIterator = new DataSourceCompactibleSegmentIterator( + config, + timeline, + Intervals.complementOf(searchInterval), + new NewestSegmentFirstPolicy(null) + ); + + final List jobs = new ArrayList<>(); + + // Create a job for each CompactionCandidate + while (segmentIterator.hasNext()) { + final CompactionCandidate candidate = segmentIterator.next(); + + // TODO: choose the right engine here + ClientCompactionTaskQuery taskPayload + = CompactSegments.createCompactionTask(candidate, config, CompactionEngine.NATIVE); + final Interval compactionInterval = taskPayload.getIoConfig().getInputSpec().getInterval(); + jobs.add( + new CompactionJob( + taskPayload, + candidate, + compactionInterval, + CompactionSlotManager.getMaxTaskSlotsForNativeCompactionTask(taskPayload.getTuningConfig()) + ) + ); + } + + return jobs; + } + + @Override + public String getType() + { + throw new UnsupportedOperationException("This template type cannot be serialized"); + } + + private void validateInput(InputSource source) + { + final DruidInputSource druidInputSource = ensureDruidInputSource(source); + if (!druidInputSource.getDataSource().equals(config.getDataSource())) { + throw InvalidInput.exception( + "Datasource[%s] in compaction config does not match datasource[%s] in input source", + config.getDataSource(), druidInputSource.getDataSource() + ); + } + } + + private void validateOutput(OutputDestination destination) + { + final DruidDatasourceDestination druidDestination = ensureDruidDataSourceDestination(destination); + if (!druidDestination.getDataSource().equals(config.getDataSource())) { + throw InvalidInput.exception( + "Datasource[%s] in compaction config does not match datasource[%s] in output destination", + config.getDataSource(), druidDestination.getDataSource() + ); + } + } +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionJob.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionJob.java new file mode 100644 index 000000000000..99b560157e47 --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionJob.java @@ -0,0 +1,93 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.compact; + +import org.apache.druid.client.indexing.ClientCompactionTaskQuery; +import org.apache.druid.indexing.template.BatchIndexingJob; +import org.apache.druid.query.http.ClientSqlQuery; +import org.apache.druid.server.compaction.CompactionCandidate; +import org.joda.time.Interval; + +/** + * {@link BatchIndexingJob} to compact an interval of a datasource. + */ +public class CompactionJob extends BatchIndexingJob +{ + private final CompactionCandidate candidate; + private final Interval compactionInterval; + private final int maxRequiredTaskSlots; + + public CompactionJob( + ClientCompactionTaskQuery task, + CompactionCandidate candidate, + Interval compactionInterval, + int maxRequiredTaskSlots + ) + { + super(task, null); + this.candidate = candidate; + this.compactionInterval = compactionInterval; + this.maxRequiredTaskSlots = maxRequiredTaskSlots; + } + + public CompactionJob( + ClientSqlQuery msqQuery, + CompactionCandidate candidate, + Interval compactionInterval, + int maxRequiredTaskSlots + ) + { + super(null, msqQuery); + this.candidate = candidate; + this.compactionInterval = compactionInterval; + this.maxRequiredTaskSlots = maxRequiredTaskSlots; + } + + public String getDataSource() + { + return candidate.getDataSource(); + } + + public CompactionCandidate getCandidate() + { + return candidate; + } + + public Interval getCompactionInterval() + { + return compactionInterval; + } + + public int getMaxRequiredTaskSlots() + { + return maxRequiredTaskSlots; + } + + @Override + public String toString() + { + return "CompactionJob{" + + super.toString() + + ", candidate=" + candidate + + ", compactionInterval=" + compactionInterval + + ", maxRequiredTaskSlots=" + maxRequiredTaskSlots + + '}'; + } +} 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 new file mode 100644 index 000000000000..907e841094e4 --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionJobParams.java @@ -0,0 +1,68 @@ +/* + * 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.compact; + +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.indexing.template.JobParams; +import org.apache.druid.timeline.SegmentTimeline; +import org.joda.time.DateTime; + +/** + * Parameters used while creating a {@link CompactionJob} using a {@link CompactionJobTemplate}. + */ +public class CompactionJobParams implements JobParams +{ + private final DateTime scheduleStartTime; + private final ObjectMapper mapper; + private final TimelineProvider timelineProvider; + + public CompactionJobParams( + DateTime scheduleStartTime, + ObjectMapper mapper, + TimelineProvider timelineProvider + ) + { + this.mapper = mapper; + this.scheduleStartTime = scheduleStartTime; + this.timelineProvider = timelineProvider; + } + + @Override + public DateTime getScheduleStartTime() + { + return scheduleStartTime; + } + + public ObjectMapper getMapper() + { + return mapper; + } + + public SegmentTimeline getTimeline(String dataSource) + { + return timelineProvider.getTimelineForDataSource(dataSource); + } + + @FunctionalInterface + public interface TimelineProvider + { + SegmentTimeline getTimelineForDataSource(String dataSource); + } +} 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 new file mode 100644 index 000000000000..1675829109bd --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionJobQueue.java @@ -0,0 +1,297 @@ +/* + * 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.compact; + +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.client.DataSourcesSnapshot; +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.indexing.common.actions.TaskActionClientFactory; +import org.apache.druid.indexing.common.task.Task; +import org.apache.druid.indexing.overlord.GlobalTaskLockbox; +import org.apache.druid.indexing.template.BatchIndexingJob; +import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.rpc.indexing.OverlordClient; +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.CompactionSnapshotBuilder; +import org.apache.druid.server.compaction.CompactionStatus; +import org.apache.druid.server.compaction.CompactionStatusTracker; +import org.apache.druid.server.coordinator.AutoCompactionSnapshot; +import org.apache.druid.server.coordinator.ClusterCompactionConfig; +import org.apache.druid.server.coordinator.CompactionConfigValidationResult; +import org.apache.druid.server.coordinator.stats.CoordinatorRunStats; +import org.apache.druid.server.coordinator.stats.Dimension; +import org.apache.druid.server.coordinator.stats.RowKey; +import org.apache.druid.server.coordinator.stats.Stats; + +import java.util.Map; +import java.util.Objects; +import java.util.PriorityQueue; + +/** + * Iterates over all eligible compaction jobs in order of their priority. + * A fresh instance of this class must be used in every run of the + * {@link CompactionScheduler}. + * + * TODO: Remaining items: + * - fill timeline gaps, support realiging intervals + * - cancel mismatching task + * - pass in the engine to the template + * - MSQ template + * - invoke onTimelineUpdated - timeline will now get updated very frequently, + * - we don't want to recompact intervals, try to find the right thing to do. + * - we might have to do it via the policy + * - maybe use searchInterval instead of skipIntervals + * - how does this whole thing affect queuedIntervals + * - for duty, it doesn't matter + * - for supervisors, intervals will always be mutually exclusive + */ +public class CompactionJobQueue +{ + private static final Logger log = new Logger(CompactionJobQueue.class); + + private final CompactionJobParams jobParams; + private final CompactionCandidateSearchPolicy searchPolicy; + + private final ObjectMapper objectMapper; + private final CompactionStatusTracker statusTracker; + private final TaskActionClientFactory taskActionClientFactory; + private final OverlordClient overlordClient; + private final GlobalTaskLockbox taskLockbox; + + private final CompactionSnapshotBuilder snapshotBuilder; + private final PriorityQueue queue; + private final CoordinatorRunStats runStats; + + private final CompactionSlotManager slotManager; + + public CompactionJobQueue( + DataSourcesSnapshot dataSourcesSnapshot, + ClusterCompactionConfig clusterCompactionConfig, + CompactionStatusTracker statusTracker, + TaskActionClientFactory taskActionClientFactory, + GlobalTaskLockbox taskLockbox, + OverlordClient overlordClient, + ObjectMapper objectMapper + ) + { + this.searchPolicy = clusterCompactionConfig.getCompactionPolicy(); + this.queue = new PriorityQueue<>( + (o1, o2) -> searchPolicy.compareCandidates(o1.getCandidate(), o2.getCandidate()) + ); + this.jobParams = new CompactionJobParams( + DateTimes.nowUtc(), + objectMapper, + dataSourcesSnapshot.getUsedSegmentsTimelinesPerDataSource()::get + ); + this.slotManager = new CompactionSlotManager( + overlordClient, + statusTracker, + clusterCompactionConfig + ); + + this.runStats = new CoordinatorRunStats(); + this.snapshotBuilder = new CompactionSnapshotBuilder(runStats); + this.taskActionClientFactory = taskActionClientFactory; + this.overlordClient = overlordClient; + this.statusTracker = statusTracker; + this.objectMapper = objectMapper; + this.taskLockbox = taskLockbox; + + computeAvailableTaskSlots(); + } + + /** + * Adds a job to this queue. + */ + public void add(CompactionJob job) + { + queue.add(job); + } + + /** + * Creates jobs for the given {@link CompactionSupervisor} and adds them to + * the job queue. + */ + public void createAndEnqueueJobs(CompactionSupervisor supervisor) + { + final String supervisorId = supervisor.getSpec().getId(); + try { + if (supervisor.shouldCreateJobs(jobParams)) { + queue.addAll(supervisor.createJobs(jobParams)); + } else { + log.debug("Skipping job creation for supervisor[%s]", supervisorId); + } + } + catch (Exception e) { + log.error(e, "Error while creating jobs for supervisor[%s]", supervisorId); + } + } + + /** + * Submits jobs which are ready to either the Overlord or a Broker (if it is + * an MSQ SQL job). + */ + public void runReadyJobs() + { + while (!queue.isEmpty()) { + final CompactionJob job = queue.poll(); + final ClientTaskQuery task = Objects.requireNonNull(job.getNonNullTask()); + + if (startJobIfPendingAndReady(job, searchPolicy)) { + statusTracker.onTaskSubmitted(task.getId(), job.getCandidate()); + runStats.add(Stats.Compaction.SUBMITTED_TASKS, RowKey.of(Dimension.DATASOURCE, task.getDataSource()), 1); + } + } + + // TODO: Add the skipped and the already compacted stuff determined by the DatasourceCompactibleSegmentIterator + // to the stats + } + + /** + * Builds and returns the compaction snapshots for all the datasources being + * tracked in this queue. Must be called after {@link #runReadyJobs()}. + */ + public Map getCompactionSnapshots() + { + return snapshotBuilder.build(); + } + + public CoordinatorRunStats getRunStats() + { + return runStats; + } + + private void computeAvailableTaskSlots() + { + // Do not cancel any currently running compaction tasks to be valid + // Future iterations can cancel a job if it doesn't match the given template + for (ClientCompactionTaskQuery task : slotManager.fetchRunningCompactionTasks()) { + slotManager.reserveTaskSlots(task); + } + } + + /** + * Starts a job if it is ready and is not already in progress. + * + * @return true if the job was submitted successfully for execution + */ + private boolean startJobIfPendingAndReady(CompactionJob job, CompactionCandidateSearchPolicy policy) + { + // Check if the job is a valid compaction job + final CompactionCandidate candidate = job.getCandidate(); + final CompactionConfigValidationResult validationResult = validateCompactionJob(job); + if (!validationResult.isValid()) { + log.error("Compaction job[%s] is invalid due to reason[%s].", job, validationResult.getReason()); + snapshotBuilder.addToSkipped(candidate); + return false; + } + + // Check if the job is already running, completed or skipped + final CompactionStatus compactionStatus = getCurrentStatusForJob(job, policy); + switch (compactionStatus.getState()) { + case RUNNING: + case COMPLETE: + snapshotBuilder.addToComplete(candidate); + return false; + case SKIPPED: + snapshotBuilder.addToSkipped(candidate); + return false; + } + + // Check if enough compaction task slots are available + if (job.getMaxRequiredTaskSlots() > slotManager.getNumAvailableTaskSlots()) { + snapshotBuilder.addToPending(candidate); + return false; + } + + // Reserve task slots and try to start the task + slotManager.reserveTaskSlots(job.getMaxRequiredTaskSlots()); + if(startTaskIfReady(job)) { + snapshotBuilder.addToComplete(candidate); + return true; + } else { + snapshotBuilder.addToPending(candidate); + return false; + } + } + + /** + * Starts the given job if the underlying Task is able to acquire locks. + * + * @return true if the Task was submitted successfully. + */ + private boolean startTaskIfReady(CompactionJob job) + { + // Assume MSQ jobs to be always ready + if (job.isMsq()) { + // TODO: submit the MSQ job to Broker here + return true; + } + + final ClientTaskQuery taskQuery = job.getNonNullTask(); + final Task task = objectMapper.convertValue(taskQuery, Task.class); + + log.info("Checking readiness of task[%s] with interval[%s]", task.getId(), job.getCompactionInterval()); + try { + taskLockbox.add(task); + if (task.isReady(taskActionClientFactory.create(task))) { + // Hold the locks acquired by task.isReady() as we will reacquire them anyway + FutureUtils.getUnchecked(overlordClient.runTask(task.getId(), task), true); + return true; + } else { + taskLockbox.unlockAll(task); + return false; + } + } + catch (Exception e) { + log.error(e, "Error while checking readiness of task[%s]", task.getId()); + taskLockbox.unlockAll(task); + return false; + } + } + + public CompactionStatus getCurrentStatusForJob(CompactionJob job, CompactionCandidateSearchPolicy policy) + { + final CompactionStatus compactionStatus = statusTracker.computeCompactionStatus(job.getCandidate(), policy); + final CompactionCandidate candidatesWithStatus = job.getCandidate().withCurrentStatus(null); + statusTracker.onCompactionStatusComputed(candidatesWithStatus, null); + return compactionStatus; + } + + public static CompactionConfigValidationResult validateCompactionJob(BatchIndexingJob job) + { + // For MSQ jobs, do not perform any validation + if (job.isMsq()) { + return CompactionConfigValidationResult.success(); + } + + final ClientTaskQuery task = job.getNonNullTask(); + if (!(task instanceof ClientCompactionTaskQuery)) { + return CompactionConfigValidationResult.failure("Invalid task type[%s]", task.getType()); + } + + return CompactionConfigValidationResult.success(); + } +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionJobTemplate.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionJobTemplate.java new file mode 100644 index 000000000000..f6f000db81a2 --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionJobTemplate.java @@ -0,0 +1,87 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.compact; + +import org.apache.druid.data.input.InputSource; +import org.apache.druid.data.output.OutputDestination; +import org.apache.druid.error.InvalidInput; +import org.apache.druid.indexing.input.DruidDatasourceDestination; +import org.apache.druid.indexing.input.DruidInputSource; +import org.apache.druid.indexing.template.BatchIndexingJob; +import org.apache.druid.indexing.template.BatchIndexingJobTemplate; +import org.apache.druid.indexing.template.JobParams; + +import java.util.List; +import java.util.stream.Collectors; + +/** + * Base indexing template for creating {@link CompactionJob}. + */ +public abstract class CompactionJobTemplate implements BatchIndexingJobTemplate +{ + abstract List createCompactionJobs( + InputSource source, + OutputDestination destination, + CompactionJobParams jobParams + ); + + @Override + public final List createJobs( + InputSource source, + OutputDestination destination, + JobParams jobParams + ) + { + if (!(jobParams instanceof CompactionJobParams)) { + throw InvalidInput.exception( + "Job params[%s] for compaction template must be of type CompactionJobParams.", + jobParams + ); + } + return createCompactionJobs(source, destination, (CompactionJobParams) jobParams) + .stream() + .map(job -> (BatchIndexingJob) job) + .collect(Collectors.toList()); + } + + /** + * Verifies that the input source is of type {@link DruidInputSource}. + */ + public final DruidInputSource ensureDruidInputSource(InputSource inputSource) + { + if (inputSource instanceof DruidInputSource) { + return (DruidInputSource) inputSource; + } else { + throw InvalidInput.exception("Invalid input source[%s] for compaction", inputSource); + } + } + + /** + * Verifies that the output destination is of type {@link DruidDatasourceDestination}. + */ + public final DruidDatasourceDestination ensureDruidDataSourceDestination(OutputDestination destination) + { + if (destination instanceof DruidDatasourceDestination) { + return (DruidDatasourceDestination) destination; + } else { + throw InvalidInput.exception("Invalid output destination[%s] for compaction", destination); + } + } +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionRule.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionRule.java new file mode 100644 index 000000000000..bbb877237b06 --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionRule.java @@ -0,0 +1,52 @@ +/* + * 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.compact; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import org.joda.time.Period; + +public class CompactionRule +{ + private final Period period; + private final CompactionJobTemplate template; + + @JsonCreator + public CompactionRule( + @JsonProperty("period") Period period, + @JsonProperty("template") CompactionJobTemplate template + ) + { + this.period = period; + this.template = template; + } + + @JsonProperty + public CompactionJobTemplate getTemplate() + { + return template; + } + + @JsonProperty + public Period getPeriod() + { + return period; + } +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionScheduler.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionScheduler.java index 5f0aa6e3ea2b..6f5ed1a7a6ef 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionScheduler.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionScheduler.java @@ -65,7 +65,7 @@ public interface CompactionScheduler /** * Starts compaction for a datasource if not already running. */ - void startCompaction(String dataSourceName, DataSourceCompactionConfig compactionConfig); + void startCompaction(String dataSourceName, CompactionSupervisor supervisor); /** * Stops compaction for a datasource if currently running. diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionSupervisor.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionSupervisor.java index 851b3920b1a1..a20620b2845f 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionSupervisor.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionSupervisor.java @@ -19,21 +19,27 @@ package org.apache.druid.indexing.compact; +import org.apache.druid.indexing.input.DruidDatasourceDestination; +import org.apache.druid.indexing.input.DruidInputSource; import org.apache.druid.indexing.overlord.DataSourceMetadata; -import org.apache.druid.indexing.overlord.supervisor.Supervisor; +import org.apache.druid.indexing.overlord.supervisor.BatchIndexingSupervisor; import org.apache.druid.indexing.overlord.supervisor.SupervisorReport; import org.apache.druid.indexing.overlord.supervisor.SupervisorStateManager; import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.server.coordinator.AutoCompactionSnapshot; +import org.joda.time.Interval; import javax.annotation.Nullable; +import java.util.List; +import java.util.Map; /** * Supervisor for compaction of a single datasource. */ -public class CompactionSupervisor implements Supervisor +public class CompactionSupervisor implements BatchIndexingSupervisor { private static final Logger log = new Logger(CompactionSupervisor.class); @@ -51,6 +57,11 @@ public CompactionSupervisor( this.dataSource = supervisorSpec.getSpec().getDataSource(); } + public CompactionSupervisorSpec getSpec() + { + return supervisorSpec; + } + @Override public void start() { @@ -66,7 +77,7 @@ public void start() ); } else { log.info("Starting compaction for dataSource[%s].", dataSource); - scheduler.startCompaction(dataSource, supervisorSpec.getSpec()); + scheduler.startCompaction(dataSource, this); } } @@ -119,6 +130,27 @@ public void reset(@Nullable DataSourceMetadata dataSourceMetadata) // do nothing } + @Override + public boolean shouldCreateJobs(CompactionJobParams jobParams) + { + return !supervisorSpec.isSuspended(); + } + + @Override + public List createJobs(CompactionJobParams jobParams) + { + final Interval interval = Intervals.ETERNITY; + return supervisorSpec.getTemplate().createCompactionJobs( + // Create a DruidInputSource for this datasource + jobParams.getMapper().convertValue( + Map.of("type", "druid", "dataSource", dataSource, "interval", interval), + DruidInputSource.class + ), + new DruidDatasourceDestination(dataSource), + jobParams + ); + } + public enum State implements SupervisorStateManager.State { SCHEDULER_STOPPED(true), diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionSupervisorSpec.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionSupervisorSpec.java index ca6008cc9d79..cf5fab87d651 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionSupervisorSpec.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionSupervisorSpec.java @@ -23,7 +23,7 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import org.apache.druid.common.config.Configs; -import org.apache.druid.indexing.overlord.supervisor.SupervisorSpec; +import org.apache.druid.indexing.overlord.supervisor.BatchIndexingSupervisorSpec; import org.apache.druid.server.coordinator.CompactionConfigValidationResult; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; import org.apache.druid.server.security.ResourceAction; @@ -35,7 +35,7 @@ import java.util.Objects; import java.util.Set; -public class CompactionSupervisorSpec implements SupervisorSpec +public class CompactionSupervisorSpec implements BatchIndexingSupervisorSpec { public static final String TYPE = "autocompact"; public static final String ID_PREFIX = "autocompact__"; @@ -93,6 +93,16 @@ public CompactionSupervisor createSupervisor() return new CompactionSupervisor(this, scheduler); } + @Override + public CompactionJobTemplate getTemplate() + { + if (spec instanceof CascadingCompactionTemplate) { + return (CascadingCompactionTemplate) spec; + } else { + return new CompactionConfigBasedJobTemplate(spec); + } + } + @Override public List getDataSources() { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/InlineCompactionJobTemplate.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/InlineCompactionJobTemplate.java new file mode 100644 index 000000000000..c147fde4f9f2 --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/InlineCompactionJobTemplate.java @@ -0,0 +1,117 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.compact; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.druid.data.input.InputSource; +import org.apache.druid.data.output.OutputDestination; +import org.apache.druid.java.util.common.granularity.Granularity; +import org.apache.druid.server.coordinator.InlineSchemaDataSourceCompactionConfig; +import org.apache.druid.server.coordinator.UserCompactionTaskGranularityConfig; +import org.apache.druid.server.coordinator.UserCompactionTaskQueryTuningConfig; +import org.joda.time.Period; + +import java.util.List; +import java.util.Objects; + +/** + * Template to create compaction jobs using inline specifications. This template + * does not fetch any information from the Druid catalog while creating jobs. + *

+ * This template does not contain all the fields supported by + * {@link InlineSchemaDataSourceCompactionConfig} since some of those fields may + * change the data itself (and not just its layout) and are thus not considered + * compaction-compatible. + */ +public class InlineCompactionJobTemplate extends CompactionJobTemplate +{ + public static final String TYPE = "compactInline"; + + private final UserCompactionTaskQueryTuningConfig tuningConfig; + private final Granularity segmentGranularity; + + @JsonCreator + public InlineCompactionJobTemplate( + @JsonProperty("tuningConfig") UserCompactionTaskQueryTuningConfig tuningConfig, + @JsonProperty("segmentGranularity") Granularity segmentGranularity + ) + { + this.tuningConfig = tuningConfig; + this.segmentGranularity = segmentGranularity; + } + + @JsonProperty + public Granularity getSegmentGranularity() + { + return segmentGranularity; + } + + @JsonProperty + public UserCompactionTaskQueryTuningConfig getTuningConfig() + { + return tuningConfig; + } + + @Override + public List createCompactionJobs( + InputSource source, + OutputDestination destination, + CompactionJobParams jobParams + ) + { + final String dataSource = ensureDruidInputSource(source).getDataSource(); + return new CompactionConfigBasedJobTemplate( + InlineSchemaDataSourceCompactionConfig + .builder() + .forDataSource(dataSource) + .withSkipOffsetFromLatest(Period.ZERO) + .withTuningConfig(tuningConfig) + .withGranularitySpec(new UserCompactionTaskGranularityConfig(segmentGranularity, null, null)) + .build() + ).createCompactionJobs(source, destination, jobParams); + } + + @Override + public boolean equals(Object object) + { + if (this == object) { + return true; + } + if (object == null || getClass() != object.getClass()) { + return false; + } + InlineCompactionJobTemplate that = (InlineCompactionJobTemplate) object; + return Objects.equals(this.tuningConfig, that.tuningConfig) + && Objects.equals(this.segmentGranularity, that.segmentGranularity); + } + + @Override + public int hashCode() + { + return Objects.hash(tuningConfig, segmentGranularity); + } + + @Override + public String getType() + { + return TYPE; + } +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/LocalOverlordClient.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/LocalOverlordClient.java index 3f1427c7c34e..eb200577bb8b 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/compact/LocalOverlordClient.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/LocalOverlordClient.java @@ -72,10 +72,12 @@ class LocalOverlordClient extends NoopOverlordClient @Override public ListenableFuture runTask(String taskId, Object clientTaskQuery) { + final CompactionTask task = + clientTaskQuery instanceof CompactionTask + ? (CompactionTask) clientTaskQuery + : convertTask(clientTaskQuery, ClientCompactionTaskQuery.class, CompactionTask.class); return futureOf(() -> { - getValidTaskQueue().add( - convertTask(clientTaskQuery, ClientCompactionTaskQuery.class, CompactionTask.class) - ); + getValidTaskQueue().add(task); return null; }); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/OverlordCompactionScheduler.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/OverlordCompactionScheduler.java index 95ea3d116939..9896db489264 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/compact/OverlordCompactionScheduler.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/OverlordCompactionScheduler.java @@ -27,6 +27,8 @@ import org.apache.druid.client.indexing.ClientCompactionRunnerInfo; import org.apache.druid.indexer.TaskLocation; import org.apache.druid.indexer.TaskStatus; +import org.apache.druid.indexing.common.actions.TaskActionClientFactory; +import org.apache.druid.indexing.overlord.GlobalTaskLockbox; import org.apache.druid.indexing.overlord.TaskMaster; import org.apache.druid.indexing.overlord.TaskQueryTool; import org.apache.druid.indexing.overlord.TaskRunner; @@ -52,17 +54,19 @@ import org.apache.druid.server.coordinator.duty.CompactSegments; import org.apache.druid.server.coordinator.stats.CoordinatorRunStats; import org.apache.druid.server.coordinator.stats.CoordinatorStat; -import org.apache.druid.server.coordinator.stats.Dimension; +import org.apache.druid.server.coordinator.stats.RowKey; import org.apache.druid.server.coordinator.stats.Stats; import org.joda.time.Duration; -import java.util.ArrayList; import java.util.Collections; +import java.util.List; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicReference; +import java.util.stream.Collectors; /** * Implementation of {@link CompactionScheduler}. @@ -87,10 +91,13 @@ public class OverlordCompactionScheduler implements CompactionScheduler private final SegmentsMetadataManager segmentManager; private final LocalOverlordClient overlordClient; private final ServiceEmitter emitter; + private final ObjectMapper objectMapper; private final TaskMaster taskMaster; private final Supplier compactionConfigSupplier; - private final ConcurrentHashMap activeDatasourceConfigs; + private final ConcurrentHashMap activeSupervisors; + + private final AtomicReference> datasourceToCompactionSnapshot; /** * Single-threaded executor to process the compaction queue. @@ -98,6 +105,8 @@ public class OverlordCompactionScheduler implements CompactionScheduler private final ScheduledExecutorService executor; private final CompactionStatusTracker statusTracker; + private final TaskActionClientFactory taskActionClientFactory; + private final GlobalTaskLockbox taskLockbox; /** * Listener to watch task completion events and update CompactionStatusTracker. @@ -107,7 +116,6 @@ public class OverlordCompactionScheduler implements CompactionScheduler private final AtomicBoolean isLeader = new AtomicBoolean(false); private final AtomicBoolean started = new AtomicBoolean(false); - private final CompactSegments duty; /** * The scheduler should enable/disable polling of segments only if the Overlord @@ -121,11 +129,13 @@ public class OverlordCompactionScheduler implements CompactionScheduler @Inject public OverlordCompactionScheduler( TaskMaster taskMaster, + GlobalTaskLockbox taskLockbox, TaskQueryTool taskQueryTool, SegmentsMetadataManager segmentManager, Supplier compactionConfigSupplier, CompactionStatusTracker statusTracker, CoordinatorOverlordServiceConfig coordinatorOverlordServiceConfig, + TaskActionClientFactory taskActionClientFactory, ScheduledExecutorFactory executorFactory, ServiceEmitter emitter, ObjectMapper objectMapper @@ -133,7 +143,9 @@ public OverlordCompactionScheduler( { this.segmentManager = segmentManager; this.emitter = emitter; + this.objectMapper = objectMapper; this.taskMaster = taskMaster; + this.taskLockbox = taskLockbox; this.compactionConfigSupplier = compactionConfigSupplier; this.executor = executorFactory.create(1, "CompactionScheduler-%s"); @@ -141,9 +153,10 @@ public OverlordCompactionScheduler( this.shouldPollSegments = segmentManager != null && !coordinatorOverlordServiceConfig.isEnabled(); this.overlordClient = new LocalOverlordClient(taskMaster, taskQueryTool, objectMapper); - this.duty = new CompactSegments(this.statusTracker, overlordClient); - this.activeDatasourceConfigs = new ConcurrentHashMap<>(); + this.activeSupervisors = new ConcurrentHashMap<>(); + this.datasourceToCompactionSnapshot = new AtomicReference<>(); + this.taskActionClientFactory = taskActionClientFactory; this.taskRunnerListener = new TaskRunnerListener() { @Override @@ -208,25 +221,25 @@ public CompactionConfigValidationResult validateCompactionConfig(DataSourceCompa } else { return ClientCompactionRunnerInfo.validateCompactionConfig( compactionConfig, - getLatestConfig().getEngine() + getLatestClusterConfig().getEngine() ); } } @Override - public void startCompaction(String dataSourceName, DataSourceCompactionConfig config) + public void startCompaction(String dataSourceName, CompactionSupervisor supervisor) { // Track active datasources even if scheduler has not started yet because // SupervisorManager is started before the scheduler if (isEnabled()) { - activeDatasourceConfigs.put(dataSourceName, config); + activeSupervisors.put(dataSourceName, supervisor); } } @Override public void stopCompaction(String dataSourceName) { - activeDatasourceConfigs.remove(dataSourceName); + activeSupervisors.remove(dataSourceName); statusTracker.removeDatasource(dataSourceName); } @@ -264,7 +277,7 @@ private synchronized void cleanupState() taskRunnerOptional.get().unregisterListener(taskRunnerListener.getListenerId()); } statusTracker.stop(); - activeDatasourceConfigs.clear(); + activeSupervisors.clear(); if (shouldPollSegments) { segmentManager.stopPollingDatabasePeriodically(); @@ -304,41 +317,52 @@ private synchronized void scheduledRun() } /** - * Runs the compaction duty and emits stats if {@link #METRIC_EMISSION_PERIOD} - * has elapsed. + * Creates and runs eligible compaction jobs. */ private synchronized void runCompactionDuty() { - final CoordinatorRunStats stats = new CoordinatorRunStats(); - duty.run(getLatestConfig(), getDatasourceSnapshot(), getLatestConfig().getEngine(), stats); + final CompactionJobQueue queue = new CompactionJobQueue( + getDatasourceSnapshot(), + getLatestClusterConfig(), + statusTracker, + taskActionClientFactory, + taskLockbox, + overlordClient, + objectMapper + ); + statusTracker.resetActiveDatasources(activeSupervisors.keySet()); + activeSupervisors.forEach((datasource, supervisor) -> queue.createAndEnqueueJobs(supervisor)); + queue.runReadyJobs(); + + datasourceToCompactionSnapshot.set(queue.getCompactionSnapshots()); + emitStatsIfPeriodHasElapsed(queue.getRunStats()); + } + /** + * Emits stats if {@link #METRIC_EMISSION_PERIOD} has elapsed. + */ + private void emitStatsIfPeriodHasElapsed(CoordinatorRunStats stats) + { // Emit stats only if emission period has elapsed if (!sinceStatsEmitted.isRunning() || sinceStatsEmitted.hasElapsed(METRIC_EMISSION_PERIOD)) { - stats.forEachStat( - (stat, dimensions, value) -> { - if (stat.shouldEmit()) { - emitStat(stat, dimensions.getValues(), value); - } - } - ); + stats.forEachStat(this::emitStat); sinceStatsEmitted.restart(); } else { // Always emit number of submitted tasks - long numSubmittedTasks = stats.get(Stats.Compaction.SUBMITTED_TASKS); - emitStat(Stats.Compaction.SUBMITTED_TASKS, Collections.emptyMap(), numSubmittedTasks); + stats.forEachEntry(Stats.Compaction.SUBMITTED_TASKS, this::emitStat); } } @Override public AutoCompactionSnapshot getCompactionSnapshot(String dataSource) { - if (!activeDatasourceConfigs.containsKey(dataSource)) { + if (!activeSupervisors.containsKey(dataSource)) { return AutoCompactionSnapshot.builder(dataSource) .withStatus(AutoCompactionSnapshot.ScheduleStatus.NOT_ENABLED) .build(); } - final AutoCompactionSnapshot snapshot = duty.getAutoCompactionSnapshot(dataSource); + final AutoCompactionSnapshot snapshot = datasourceToCompactionSnapshot.get().get(dataSource); if (snapshot == null) { final AutoCompactionSnapshot.ScheduleStatus status = isEnabled() @@ -353,7 +377,7 @@ public AutoCompactionSnapshot getCompactionSnapshot(String dataSource) @Override public Map getAllCompactionSnapshots() { - return duty.getAutoCompactionSnapshot(); + return Map.copyOf(datasourceToCompactionSnapshot.get()); } @Override @@ -363,17 +387,21 @@ public CompactionSimulateResult simulateRunWithConfigUpdate(ClusterCompactionCon return new CompactionRunSimulator(statusTracker, overlordClient).simulateRunWithConfig( getLatestConfig().withClusterConfig(updateRequest), getDatasourceSnapshot(), - getLatestConfig().getEngine() + updateRequest.getEngine() ); } else { return new CompactionSimulateResult(Collections.emptyMap()); } } - private void emitStat(CoordinatorStat stat, Map dimensionValues, long value) + private void emitStat(CoordinatorStat stat, RowKey rowKey, long value) { + if (!stat.shouldEmit()) { + return; + } + ServiceMetricEvent.Builder eventBuilder = new ServiceMetricEvent.Builder(); - dimensionValues.forEach( + rowKey.getValues().forEach( (dim, dimValue) -> eventBuilder.setDimension(dim.reportedName(), dimValue) ); emitter.emit(eventBuilder.setMetric(stat.getMetricName(), value)); @@ -381,10 +409,20 @@ private void emitStat(CoordinatorStat stat, Map dimensionValu private DruidCompactionConfig getLatestConfig() { + final List configs = activeSupervisors + .values() + .stream() + .map(s -> s.getSpec().getSpec()) + .collect(Collectors.toList()); return DruidCompactionConfig .empty() - .withClusterConfig(compactionConfigSupplier.get().clusterConfig()) - .withDatasourceConfigs(new ArrayList<>(activeDatasourceConfigs.values())); + .withClusterConfig(getLatestClusterConfig()) + .withDatasourceConfigs(configs); + } + + private ClusterCompactionConfig getLatestClusterConfig() + { + return compactionConfigSupplier.get().clusterConfig(); } private DataSourcesSnapshot getDatasourceSnapshot() diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/input/DruidDatasourceDestination.java b/indexing-service/src/main/java/org/apache/druid/indexing/input/DruidDatasourceDestination.java new file mode 100644 index 000000000000..7d10ce86bd75 --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/input/DruidDatasourceDestination.java @@ -0,0 +1,42 @@ +/* + * 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.input; + +import org.apache.druid.data.output.OutputDestination; + +/** + * {@link OutputDestination} for writing out data into a Druid datasource. + */ +public class DruidDatasourceDestination implements OutputDestination +{ + public static final String TYPE = "druid"; + + private final String dataSource; + + public DruidDatasourceDestination(String dataSource) + { + this.dataSource = dataSource; + } + + public String getDataSource() + { + return dataSource; + } +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/input/DruidInputSource.java b/indexing-service/src/main/java/org/apache/druid/indexing/input/DruidInputSource.java index 84c17f1a8fa0..fea6f30ef444 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/input/DruidInputSource.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/input/DruidInputSource.java @@ -291,6 +291,25 @@ public InputSource withTaskToolbox(TaskToolbox toolbox) ); } + /** + * Creates a new {@link DruidInputSource} with the given interval. + */ + public DruidInputSource withInterval(Interval interval) + { + return new DruidInputSource( + this.dataSource, + interval, + null, + this.dimFilter, + this.dimensions, + this.metrics, + this.indexIO, + this.coordinatorClient, + this.segmentCacheManagerFactory, + this.taskConfig + ); + } + @Override protected InputSourceReader fixedFormatReader(InputRowSchema inputRowSchema, @Nullable File temporaryDirectory) { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/BatchIndexingSupervisor.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/BatchIndexingSupervisor.java new file mode 100644 index 000000000000..c26c27fc1038 --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/BatchIndexingSupervisor.java @@ -0,0 +1,48 @@ +/* + * 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.overlord.supervisor; + +import org.apache.druid.indexing.template.BatchIndexingJob; +import org.apache.druid.indexing.template.JobParams; + +import java.util.List; + +/** + * Supervisor to perform batch ingestion using {@link BatchIndexingJob}. + */ +public interface BatchIndexingSupervisor + extends Supervisor +{ + /** + * Checks if this supervisor is ready to create jobs in the current run. + * + * @param jobParams Parameters for the current run of the scheduler. + */ + boolean shouldCreateJobs(P jobParams); + + /** + * Creates jobs to be launched in the current run of the scheduler. + * + * @param jobParams Parameters for the current run of the scheduler. + * @return Empty iterator if no tasks are to be submitted in the current run + * of the scheduler. + */ + List createJobs(P jobParams); +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/BatchIndexingSupervisorSpec.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/BatchIndexingSupervisorSpec.java new file mode 100644 index 000000000000..30e2f2b5109c --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/BatchIndexingSupervisorSpec.java @@ -0,0 +1,41 @@ +/* + * 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.overlord.supervisor; + +import org.apache.druid.indexing.template.BatchIndexingJob; +import org.apache.druid.indexing.template.BatchIndexingJobTemplate; +import org.apache.druid.indexing.template.JobParams; + +/** + * Spec for {@link BatchIndexingSupervisor}. Provides a template to create + * {@link BatchIndexingJob}. + */ +public interface BatchIndexingSupervisorSpec + + extends SupervisorSpec +{ + @Override + BatchIndexingSupervisor createSupervisor(); + + /** + * Template used by the corresponding supervisor to create {@link BatchIndexingJob}s. + */ + BatchIndexingJobTemplate getTemplate(); +} diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/compact/CompactionSupervisorSpecTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/compact/CompactionSupervisorSpecTest.java index b1732acabbbc..b008d74829ae 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/compact/CompactionSupervisorSpecTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/compact/CompactionSupervisorSpecTest.java @@ -137,7 +137,7 @@ public void testStartStopSupervisorForActiveSpec() supervisor.start(); supervisor.stop(false); - Mockito.verify(scheduler, Mockito.times(1)).startCompaction(TestDataSource.WIKI, spec); + Mockito.verify(scheduler, Mockito.times(1)).startCompaction(TestDataSource.WIKI, supervisor); Mockito.verify(scheduler, Mockito.times(1)).stopCompaction(TestDataSource.WIKI); } @@ -157,7 +157,7 @@ public void testStartStopSupervisorWhenSchedulerStopped() supervisor.start(); supervisor.stop(false); - Mockito.verify(scheduler, Mockito.times(1)).startCompaction(TestDataSource.WIKI, spec); + Mockito.verify(scheduler, Mockito.times(1)).startCompaction(TestDataSource.WIKI, supervisor); Mockito.verify(scheduler, Mockito.times(1)).stopCompaction(TestDataSource.WIKI); } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/compact/OverlordCompactionSchedulerTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/compact/OverlordCompactionSchedulerTest.java index 9353953fc2af..86bbf3dff614 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/compact/OverlordCompactionSchedulerTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/compact/OverlordCompactionSchedulerTest.java @@ -21,10 +21,20 @@ import com.fasterxml.jackson.databind.InjectableValues; import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.client.coordinator.CoordinatorClient; +import org.apache.druid.client.coordinator.NoopCoordinatorClient; import org.apache.druid.client.indexing.ClientMSQContext; +import org.apache.druid.guice.IndexingServiceInputSourceModule; import org.apache.druid.guice.IndexingServiceTuningConfigModule; import org.apache.druid.indexer.CompactionEngine; import org.apache.druid.indexing.common.SegmentCacheManagerFactory; +import org.apache.druid.indexing.common.TimeChunkLock; +import org.apache.druid.indexing.common.actions.RetrieveUsedSegmentsAction; +import org.apache.druid.indexing.common.actions.TaskAction; +import org.apache.druid.indexing.common.actions.TaskActionClient; +import org.apache.druid.indexing.common.actions.TaskActionClientFactory; +import org.apache.druid.indexing.common.actions.TimeChunkLockTryAcquireAction; +import org.apache.druid.indexing.common.config.TaskConfig; import org.apache.druid.indexing.common.config.TaskStorageConfig; import org.apache.druid.indexing.common.task.CompactionTask; import org.apache.druid.indexing.common.task.Task; @@ -38,9 +48,11 @@ import org.apache.druid.indexing.overlord.setup.WorkerBehaviorConfig; import org.apache.druid.indexing.test.TestIndexerMetadataStorageCoordinator; import org.apache.druid.jackson.DefaultObjectMapper; +import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.metrics.StubServiceEmitter; +import org.apache.druid.segment.IndexIO; import org.apache.druid.segment.TestDataSource; import org.apache.druid.segment.TestIndex; import org.apache.druid.server.compaction.CompactionSimulateResult; @@ -81,9 +93,13 @@ public class OverlordCompactionSchedulerTest static { OBJECT_MAPPER = new DefaultObjectMapper(); OBJECT_MAPPER.registerModules(new IndexingServiceTuningConfigModule().getJacksonModules()); + OBJECT_MAPPER.registerModules(new IndexingServiceInputSourceModule().getJacksonModules()); OBJECT_MAPPER.setInjectableValues( new InjectableValues .Std() + .addValue(IndexIO.class, TestIndex.INDEX_IO) + .addValue(TaskConfig.class, Mockito.mock(TaskConfig.class)) + .addValue(CoordinatorClient.class, new NoopCoordinatorClient()) .addValue( SegmentCacheManagerFactory.class, new SegmentCacheManagerFactory(TestIndex.INDEX_IO, OBJECT_MAPPER) @@ -96,6 +112,7 @@ public class OverlordCompactionSchedulerTest private TaskMaster taskMaster; private TaskQueue taskQueue; + private TaskActionClientFactory taskActionClientFactory; private BlockingExecutorService executor; private HeapMemoryTaskStorage taskStorage; @@ -131,6 +148,30 @@ public void setUp() compactionConfig = new AtomicReference<>(new ClusterCompactionConfig(null, null, null, true, null)); coordinatorOverlordServiceConfig = new CoordinatorOverlordServiceConfig(false, null); + taskActionClientFactory = task -> new TaskActionClient() + { + @Override + @SuppressWarnings("unchecked") + public RetType submit(TaskAction taskAction) + { + if (taskAction instanceof RetrieveUsedSegmentsAction) { + return (RetType) segmentsMetadataManager.getAllSegments(); + } else if (taskAction instanceof TimeChunkLockTryAcquireAction) { + final TimeChunkLockTryAcquireAction lockAcquireAction = (TimeChunkLockTryAcquireAction) taskAction; + return (RetType) new TimeChunkLock( + null, + task.getGroupId(), + task.getDataSource(), + lockAcquireAction.getInterval(), + DateTimes.nowUtc().toString(), + 1 + ); + } else { + return null; + } + } + }; + initScheduler(); } @@ -142,11 +183,13 @@ private void initScheduler() = new DefaultWorkerBehaviorConfig(WorkerBehaviorConfig.DEFAULT_STRATEGY, null); scheduler = new OverlordCompactionScheduler( taskMaster, + taskLockbox, new TaskQueryTool(taskStorage, taskLockbox, taskMaster, null, () -> defaultWorkerConfig), segmentsMetadataManager, () -> DruidCompactionConfig.empty().withClusterConfig(compactionConfig.get()), - new CompactionStatusTracker(OBJECT_MAPPER), + new CompactionStatusTracker(), coordinatorOverlordServiceConfig, + taskActionClientFactory, (nameFormat, numThreads) -> new WrappingScheduledExecutorService("test", executor, false), serviceEmitter, OBJECT_MAPPER @@ -304,13 +347,7 @@ public void testStartCompaction() wikiSegments.forEach(segmentsMetadataManager::addSegment); scheduler.becomeLeader(); - scheduler.startCompaction( - TestDataSource.WIKI, - InlineSchemaDataSourceCompactionConfig.builder() - .forDataSource(TestDataSource.WIKI) - .withSkipOffsetFromLatest(Period.seconds(0)) - .build() - ); + scheduler.startCompaction(TestDataSource.WIKI, createSupervisor()); executor.finishNextPendingTask(); @@ -351,10 +388,7 @@ public void testStopCompaction() scheduler.becomeLeader(); scheduler.startCompaction( TestDataSource.WIKI, - InlineSchemaDataSourceCompactionConfig.builder() - .forDataSource(TestDataSource.WIKI) - .withSkipOffsetFromLatest(Period.seconds(0)) - .build() + createSupervisor() ); scheduler.stopCompaction(TestDataSource.WIKI); @@ -390,13 +424,7 @@ public void testSimulateRun() scheduler.becomeLeader(); runScheduledJob(); - scheduler.startCompaction( - TestDataSource.WIKI, - InlineSchemaDataSourceCompactionConfig.builder() - .forDataSource(TestDataSource.WIKI) - .withSkipOffsetFromLatest(Period.seconds(0)) - .build() - ); + scheduler.startCompaction(TestDataSource.WIKI, createSupervisor()); final CompactionSimulateResult simulateResult = scheduler.simulateRunWithConfigUpdate( new ClusterCompactionConfig(null, null, null, null, null) @@ -446,4 +474,16 @@ private void runScheduledJob() executor.finishNextPendingTask(); } + private CompactionSupervisor createSupervisor() + { + return new CompactionSupervisorSpec( + InlineSchemaDataSourceCompactionConfig + .builder() + .forDataSource(TestDataSource.WIKI) + .withSkipOffsetFromLatest(Period.seconds(0)) + .build(), + false, + scheduler + ).createSupervisor(); + } } diff --git a/processing/src/main/java/org/apache/druid/data/output/OutputDestination.java b/processing/src/main/java/org/apache/druid/data/output/OutputDestination.java new file mode 100644 index 000000000000..39721de67500 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/data/output/OutputDestination.java @@ -0,0 +1,33 @@ +/* + * 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.data.output; + +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; +import org.apache.druid.data.input.InputSource; + +/** + * Destination where data is written out. + */ +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = InputSource.TYPE_PROPERTY) +@JsonSubTypes(value = {}) +public interface OutputDestination +{ +} diff --git a/processing/src/main/java/org/apache/druid/java/util/common/Intervals.java b/processing/src/main/java/org/apache/druid/java/util/common/Intervals.java index 623f546349f9..4234a34090ee 100644 --- a/processing/src/main/java/org/apache/druid/java/util/common/Intervals.java +++ b/processing/src/main/java/org/apache/druid/java/util/common/Intervals.java @@ -27,6 +27,7 @@ import org.joda.time.chrono.ISOChronology; import javax.annotation.Nullable; +import java.util.List; public final class Intervals { @@ -98,6 +99,18 @@ public static Interval findOverlappingInterval(Interval searchInterval, Interval return null; } + public static List complementOf(Interval interval) + { + if (interval.equals(Intervals.ETERNITY)) { + return List.of(); + } else { + return List.of( + new Interval(DateTimes.MIN, interval.getStart()), + new Interval(interval.getEnd(), DateTimes.MAX) + ); + } + } + private Intervals() { } diff --git a/server/src/main/java/org/apache/druid/catalog/model/SchemaRegistryImpl.java b/server/src/main/java/org/apache/druid/catalog/model/SchemaRegistryImpl.java index ff3b25b45e7b..21de49dc73c3 100644 --- a/server/src/main/java/org/apache/druid/catalog/model/SchemaRegistryImpl.java +++ b/server/src/main/java/org/apache/druid/catalog/model/SchemaRegistryImpl.java @@ -19,13 +19,13 @@ package org.apache.druid.catalog.model; -import com.google.common.collect.ImmutableSet; import com.google.common.collect.Lists; import org.apache.druid.catalog.model.table.DatasourceDefn; import org.apache.druid.catalog.model.table.ExternalTableDefn; +import org.apache.druid.catalog.model.table.IndexingTemplateDefn; import org.apache.druid.server.security.ResourceType; -import java.util.Collections; +import java.util.Comparator; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -96,7 +96,7 @@ public SchemaRegistryImpl() register(new SchemaDefnImpl( TableId.DRUID_SCHEMA, ResourceType.DATASOURCE, - ImmutableSet.of(DatasourceDefn.TABLE_TYPE) + Set.of(DatasourceDefn.TABLE_TYPE) )); register(new SchemaDefnImpl( TableId.LOOKUP_SCHEMA, @@ -116,13 +116,18 @@ public SchemaRegistryImpl() register(new SchemaDefnImpl( TableId.EXTERNAL_SCHEMA, EXTERNAL_RESOURCE, - ImmutableSet.of(ExternalTableDefn.TABLE_TYPE) + Set.of(ExternalTableDefn.TABLE_TYPE) )); register(new SchemaDefnImpl( TableId.VIEW_SCHEMA, ResourceType.VIEW, null // TODO )); + register(new SchemaDefnImpl( + TableId.INDEXING_TEMPLATE_SCHEMA, + ResourceType.CONFIG, + Set.of(IndexingTemplateDefn.TYPE) + )); } private void register(SchemaSpec schemaDefn) @@ -148,7 +153,7 @@ public List schemas() // No real need to sort every time. However, this is used infrequently, // so OK for now. List schemas = Lists.newArrayList(builtIns.values()); - Collections.sort(schemas, (s1, s2) -> s1.name().compareTo(s2.name())); + schemas.sort(Comparator.comparing(SchemaSpec::name)); return schemas; } } diff --git a/server/src/main/java/org/apache/druid/catalog/model/TableDefnRegistry.java b/server/src/main/java/org/apache/druid/catalog/model/TableDefnRegistry.java index 1a14d7403937..ea442d7266cc 100644 --- a/server/src/main/java/org/apache/druid/catalog/model/TableDefnRegistry.java +++ b/server/src/main/java/org/apache/druid/catalog/model/TableDefnRegistry.java @@ -25,6 +25,7 @@ import org.apache.druid.catalog.model.table.DatasourceDefn; import org.apache.druid.catalog.model.table.ExternalTableDefn; import org.apache.druid.catalog.model.table.HttpInputSourceDefn; +import org.apache.druid.catalog.model.table.IndexingTemplateDefn; import org.apache.druid.catalog.model.table.InlineInputSourceDefn; import org.apache.druid.catalog.model.table.InputFormatDefn; import org.apache.druid.catalog.model.table.InputFormats; @@ -63,7 +64,8 @@ public class TableDefnRegistry // Guice later to allow extensions to define table types. private static final List BUILTIN_TABLE_DEFNS = Arrays.asList( new DatasourceDefn(), - new ExternalTableDefn() + new ExternalTableDefn(), + new IndexingTemplateDefn() ); private static final List BUILTIN_INPUT_SOURCE_DEFNS = Arrays.asList( new InlineInputSourceDefn(), diff --git a/server/src/main/java/org/apache/druid/catalog/model/TableId.java b/server/src/main/java/org/apache/druid/catalog/model/TableId.java index 55fcc797561b..77265c5d7d0b 100644 --- a/server/src/main/java/org/apache/druid/catalog/model/TableId.java +++ b/server/src/main/java/org/apache/druid/catalog/model/TableId.java @@ -41,6 +41,11 @@ public class TableId // Extra for views public static final String VIEW_SCHEMA = "view"; + /** + * Schema for indexing templates. + */ + public static final String INDEXING_TEMPLATE_SCHEMA = "index_template"; + private final String schema; private final String name; diff --git a/server/src/main/java/org/apache/druid/catalog/model/table/IndexingTemplateDefn.java b/server/src/main/java/org/apache/druid/catalog/model/table/IndexingTemplateDefn.java new file mode 100644 index 000000000000..f11423ddb60e --- /dev/null +++ b/server/src/main/java/org/apache/druid/catalog/model/table/IndexingTemplateDefn.java @@ -0,0 +1,65 @@ +/* + * 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.catalog.model.table; + +import com.fasterxml.jackson.core.type.TypeReference; +import org.apache.druid.catalog.model.ModelProperties; +import org.apache.druid.catalog.model.TableDefn; +import org.apache.druid.indexing.template.BatchIndexingJobTemplate; + +import java.util.List; + +/** + * Definition for indexing templates. + */ +public class IndexingTemplateDefn extends TableDefn +{ + public static final String TYPE = "index_template"; + + /** + * Property to contain template payload of type {@link BatchIndexingJobTemplate}. + * + * @see PayloadProperty#TYPE_REF + */ + public static final String PROPERTY_PAYLOAD = "payload"; + + public IndexingTemplateDefn() + { + super( + "Ingestion Template", + TYPE, + List.of(new PayloadProperty()), + null + ); + } + + /** + * Template payload property. + */ + public static class PayloadProperty extends ModelProperties.TypeRefPropertyDefn + { + public static final TypeReference TYPE_REF = new TypeReference<>() {}; + + public PayloadProperty() + { + super(PROPERTY_PAYLOAD, "Payload of the batch indexing template", TYPE_REF); + } + } +} diff --git a/server/src/main/java/org/apache/druid/indexing/template/BatchIndexingJob.java b/server/src/main/java/org/apache/druid/indexing/template/BatchIndexingJob.java new file mode 100644 index 000000000000..509226da108d --- /dev/null +++ b/server/src/main/java/org/apache/druid/indexing/template/BatchIndexingJob.java @@ -0,0 +1,90 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.template; + +import org.apache.druid.client.indexing.ClientTaskQuery; +import org.apache.druid.error.InvalidInput; +import org.apache.druid.query.http.ClientSqlQuery; + +import javax.annotation.Nullable; +import java.util.Objects; + +/** + * A batch indexing job that can be launched by the Overlord as a task. + * A job may contain the {@link ClientTaskQuery} itself or an MSQ query that gets converted + * by the Broker to a {@code ControllerTask} and is then submitted to the Overlord. + */ +public class BatchIndexingJob +{ + private final boolean isMsq; + private final ClientSqlQuery msqQuery; + private final ClientTaskQuery task; + + protected BatchIndexingJob( + @Nullable ClientTaskQuery task, + @Nullable ClientSqlQuery msqQuery + ) + { + this.isMsq = task == null; + this.msqQuery = msqQuery; + this.task = task; + + InvalidInput.conditionalException( + (task == null || msqQuery == null) && (task != null || msqQuery != null), + "Exactly one of 'task' or 'msqQuery' must be non-null" + ); + } + + /** + * @return MSQ query to be run in this job, if any. + * @throws NullPointerException if this not an MSQ job. + */ + public ClientSqlQuery getNonNullMsqQuery() + { + return Objects.requireNonNull(msqQuery); + } + + /** + * @return Task to be run in this job, if any. + * @throws NullPointerException if this is an MSQ job. + */ + public ClientTaskQuery getNonNullTask() + { + return Objects.requireNonNull(task); + } + + /** + * @return true if this is an MSQ job. + */ + public boolean isMsq() + { + return isMsq; + } + + @Override + public String toString() + { + return "BatchIndexingJob{" + + "isMsq=" + isMsq + + ", msqQuery=" + msqQuery + + ", task=" + task + + '}'; + } +} diff --git a/server/src/main/java/org/apache/druid/indexing/template/BatchIndexingJobTemplate.java b/server/src/main/java/org/apache/druid/indexing/template/BatchIndexingJobTemplate.java new file mode 100644 index 000000000000..d8e9251dfdf0 --- /dev/null +++ b/server/src/main/java/org/apache/druid/indexing/template/BatchIndexingJobTemplate.java @@ -0,0 +1,50 @@ +/* + * 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.template; + +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeInfo; +import org.apache.druid.data.input.InputSource; +import org.apache.druid.data.output.OutputDestination; + +import java.util.List; + +/** + * ETL template to create a {@link BatchIndexingJob} that indexes data from an + * {@link InputSource} into an {@link OutputDestination}. + */ +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type") +public interface BatchIndexingJobTemplate +{ + /** + * Creates jobs with this template for the given input and output. + */ + List createJobs( + InputSource source, + OutputDestination destination, + JobParams jobParams + ); + + /** + * Unique type name of this template used for JSON serialization. + */ + @JsonProperty + String getType(); +} diff --git a/server/src/main/java/org/apache/druid/indexing/template/JobParams.java b/server/src/main/java/org/apache/druid/indexing/template/JobParams.java new file mode 100644 index 000000000000..775b63b04c6b --- /dev/null +++ b/server/src/main/java/org/apache/druid/indexing/template/JobParams.java @@ -0,0 +1,30 @@ +/* + * 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.template; + +import org.joda.time.DateTime; + +/** + * Provides parameters required to create a {@link BatchIndexingJob}. + */ +public interface JobParams +{ + DateTime getScheduleStartTime(); +} diff --git a/server/src/main/java/org/apache/druid/server/compaction/CompactionRunSimulator.java b/server/src/main/java/org/apache/druid/server/compaction/CompactionRunSimulator.java index 646319eec3b2..b6db09501967 100644 --- a/server/src/main/java/org/apache/druid/server/compaction/CompactionRunSimulator.java +++ b/server/src/main/java/org/apache/druid/server/compaction/CompactionRunSimulator.java @@ -22,7 +22,6 @@ import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; import org.apache.druid.client.DataSourcesSnapshot; -import org.apache.druid.client.indexing.ClientCompactionTaskQuery; import org.apache.druid.client.indexing.ClientCompactionTaskQueryTuningConfig; import org.apache.druid.client.indexing.IndexingTotalWorkerCapacityInfo; import org.apache.druid.client.indexing.TaskPayloadResponse; @@ -86,16 +85,15 @@ public CompactionSimulateResult simulateRunWithConfig( // Add a read-only wrapper over the actual status tracker so that we can // account for the active tasks - final CompactionStatusTracker simulationStatusTracker = new CompactionStatusTracker(null) + final CompactionStatusTracker simulationStatusTracker = new CompactionStatusTracker() { @Override public CompactionStatus computeCompactionStatus( CompactionCandidate candidate, - DataSourceCompactionConfig config, CompactionCandidateSearchPolicy searchPolicy ) { - return statusTracker.computeCompactionStatus(candidate, config, searchPolicy); + return statusTracker.computeCompactionStatus(candidate, searchPolicy); } @Override @@ -123,12 +121,12 @@ public void onCompactionStatusComputed( } @Override - public void onTaskSubmitted(ClientCompactionTaskQuery taskPayload, CompactionCandidate candidateSegments) + public void onTaskSubmitted(String taskId, CompactionCandidate candidateSegments) { // Add a row for each task in order of submission final CompactionStatus status = candidateSegments.getCurrentStatus(); queuedIntervals.addRow( - createRow(candidateSegments, taskPayload.getTuningConfig(), status == null ? "" : status.getReason()) + createRow(candidateSegments, null, status == null ? "" : status.getReason()) ); } }; @@ -174,9 +172,7 @@ private Object[] createRow( row.add(candidate.getUmbrellaInterval()); row.add(candidate.numSegments()); row.add(candidate.getTotalBytes()); - if (tuningConfig != null) { - row.add(CompactSegments.findMaxNumTaskSlotsUsedByOneNativeCompactionTask(tuningConfig)); - } + row.add(CompactionSlotManager.getMaxTaskSlotsForNativeCompactionTask(tuningConfig)); if (reason != null) { row.add(reason); } diff --git a/server/src/main/java/org/apache/druid/server/compaction/CompactionSlotManager.java b/server/src/main/java/org/apache/druid/server/compaction/CompactionSlotManager.java new file mode 100644 index 000000000000..efe1c8bdba28 --- /dev/null +++ b/server/src/main/java/org/apache/druid/server/compaction/CompactionSlotManager.java @@ -0,0 +1,358 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.server.compaction; + +import com.google.common.annotations.VisibleForTesting; +import org.apache.druid.client.indexing.ClientCompactionTaskQuery; +import org.apache.druid.client.indexing.ClientCompactionTaskQueryTuningConfig; +import org.apache.druid.client.indexing.ClientMSQContext; +import org.apache.druid.client.indexing.TaskPayloadResponse; +import org.apache.druid.common.guava.FutureUtils; +import org.apache.druid.indexer.CompactionEngine; +import org.apache.druid.indexer.TaskStatus; +import org.apache.druid.indexer.TaskStatusPlus; +import org.apache.druid.indexer.partitions.DimensionRangePartitionsSpec; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.granularity.Granularity; +import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.metadata.LockFilterPolicy; +import org.apache.druid.rpc.indexing.OverlordClient; +import org.apache.druid.server.coordinator.ClusterCompactionConfig; +import org.apache.druid.server.coordinator.DataSourceCompactionConfig; +import org.apache.druid.server.coordinator.duty.CoordinatorDutyUtils; +import org.joda.time.Interval; + +import javax.annotation.Nullable; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; + +/** + * Fetches running compaction tasks from the Overlord and tracks their compaction + * intervals and task slots. + */ +public class CompactionSlotManager +{ + /** + * Task type for native compaction tasks. + */ + public static final String COMPACTION_TASK_TYPE = "compact"; + + private static final Logger log = new Logger(CompactionSlotManager.class); + + private final OverlordClient overlordClient; + private final CompactionStatusTracker statusTracker; + + private final Map> intervalsToSkipCompaction; + + private int numAvailableTaskSlots; + + public CompactionSlotManager( + OverlordClient overlordClient, + CompactionStatusTracker statusTracker, + ClusterCompactionConfig clusterCompactionConfig + ) + { + this.overlordClient = overlordClient; + this.statusTracker = statusTracker; + this.numAvailableTaskSlots = getCompactionTaskCapacity(clusterCompactionConfig); + this.intervalsToSkipCompaction = new HashMap<>(); + } + + public int getNumAvailableTaskSlots() + { + return numAvailableTaskSlots; + } + + public Map> getDatasourceIntervalsToSkipCompaction() + { + return intervalsToSkipCompaction; + } + + public void reserveTaskSlots(int numSlotsToReserve) + { + numAvailableTaskSlots -= numSlotsToReserve; + } + + /** + * Reserves task slots for the given task from the overall compaction task capacity. + */ + public void reserveTaskSlots(ClientCompactionTaskQuery compactionTaskQuery) + { + // Note: The default compactionRunnerType used here should match the default runner used in CompactionTask when + // no runner is provided there. + CompactionEngine compactionRunnerType = compactionTaskQuery.getCompactionRunner() == null + ? CompactionEngine.NATIVE + : compactionTaskQuery.getCompactionRunner().getType(); + if (compactionRunnerType == CompactionEngine.NATIVE) { + numAvailableTaskSlots -= + getMaxTaskSlotsForNativeCompactionTask(compactionTaskQuery.getTuningConfig()); + } else { + numAvailableTaskSlots -= + getMaxTaskSlotsForMSQCompactionTask(compactionTaskQuery.getContext()); + } + } + + /** + * Retrieves currently running tasks of type {@link #COMPACTION_TASK_TYPE} from + * the Overlord. + *

+ * Also queries the Overlord for the status of all tasks that were submitted + * recently but are not active anymore. The statuses are then updated in the + * {@link CompactionStatusTracker}. + */ + public List fetchRunningCompactionTasks() + { + // Fetch currently running compaction tasks + final List compactionTasks = CoordinatorDutyUtils.getStatusOfActiveTasks( + overlordClient, + status -> status != null && COMPACTION_TASK_TYPE.equals(status.getType()) + ); + + final Set activeTaskIds + = compactionTasks.stream().map(TaskStatusPlus::getId).collect(Collectors.toSet()); + trackStatusOfCompletedTasks(activeTaskIds); + + final List runningCompactTasks = new ArrayList<>(); + for (TaskStatusPlus status : compactionTasks) { + final TaskPayloadResponse response = + FutureUtils.getUnchecked(overlordClient.taskPayload(status.getId()), true); + if (response == null) { + throw new ISE("Could not find payload for active compaction task[%s]", status.getId()); + } else if (!COMPACTION_TASK_TYPE.equals(response.getPayload().getType())) { + throw new ISE( + "Payload of active compaction task[%s] is of invalid type[%s]", + status.getId(), response.getPayload().getType() + ); + } + + runningCompactTasks.add((ClientCompactionTaskQuery) response.getPayload()); + } + + return runningCompactTasks; + } + + /** + * Cancels a currently running compaction task only if the segment granularity + * has changed in the datasource compaction config. Otherwise, the task is + * retained and its intervals are skipped from the current round of compaction. + * + * @return true if the task was canceled, false otherwise. + */ + public boolean cancelTaskOnlyIfGranularityChanged( + ClientCompactionTaskQuery compactionTaskQuery, + DataSourceCompactionConfig dataSourceCompactionConfig + ) + { + if (dataSourceCompactionConfig == null + || dataSourceCompactionConfig.getGranularitySpec() == null + || compactionTaskQuery.getGranularitySpec() == null) { + skipTaskInterval(compactionTaskQuery); + reserveTaskSlots(compactionTaskQuery); + return false; + } + + Granularity configuredSegmentGranularity = dataSourceCompactionConfig.getGranularitySpec() + .getSegmentGranularity(); + Granularity taskSegmentGranularity = compactionTaskQuery.getGranularitySpec().getSegmentGranularity(); + if (configuredSegmentGranularity == null || configuredSegmentGranularity.equals(taskSegmentGranularity)) { + skipTaskInterval(compactionTaskQuery); + reserveTaskSlots(compactionTaskQuery); + return false; + } + + log.info( + "Cancelling task[%s] as task segmentGranularity[%s] differs from compaction config segmentGranularity[%s].", + compactionTaskQuery.getId(), taskSegmentGranularity, configuredSegmentGranularity + ); + overlordClient.cancelTask(compactionTaskQuery.getId()); + return true; + } + + /** + * Retrieves the list of intervals locked by higher priority tasks for each datasource. + * Since compaction tasks submitted for these Intervals would have to wait anyway, + * we skip these Intervals until the next compaction run by adding them to + * {@link #intervalsToSkipCompaction}. + *

+ */ + public void skipLockedIntervals(List compactionConfigs) + { + final List lockFilterPolicies = compactionConfigs + .stream() + .map(config -> + new LockFilterPolicy(config.getDataSource(), config.getTaskPriority(), null, config.getTaskContext())) + .collect(Collectors.toList()); + final Map> datasourceToLockedIntervals = + new HashMap<>(FutureUtils.getUnchecked(overlordClient.findLockedIntervals(lockFilterPolicies), true)); + log.debug( + "Skipping the following intervals for Compaction as they are currently locked: %s", + datasourceToLockedIntervals + ); + + // Skip all the intervals locked by higher priority tasks for each datasource + // This must be done after the invalid compaction tasks are cancelled + // in the loop above so that their intervals are not considered locked + datasourceToLockedIntervals.forEach( + (dataSource, intervals) -> + intervalsToSkipCompaction + .computeIfAbsent(dataSource, ds -> new ArrayList<>()) + .addAll(intervals) + ); + } + + /** + * Adds the compaction interval of this task to {@link #intervalsToSkipCompaction} + */ + private void skipTaskInterval(ClientCompactionTaskQuery compactionTask) + { + final Interval interval = compactionTask.getIoConfig().getInputSpec().getInterval(); + intervalsToSkipCompaction.computeIfAbsent(compactionTask.getDataSource(), k -> new ArrayList<>()) + .add(interval); + } + + /** + * Computes overall compaction task capacity for the cluster. + * + * @return A value >= 1. + */ + private int getCompactionTaskCapacity(ClusterCompactionConfig clusterConfig) + { + int totalWorkerCapacity = CoordinatorDutyUtils.getTotalWorkerCapacity(overlordClient); + + int compactionTaskCapacity = Math.min( + (int) (totalWorkerCapacity * clusterConfig.getCompactionTaskSlotRatio()), + clusterConfig.getMaxCompactionTaskSlots() + ); + + // Always consider atleast one slot available for compaction + return Math.max(1, compactionTaskCapacity); + } + + /** + * Queries the Overlord for the status of all tasks that were submitted + * recently but are not active anymore. The statuses are then updated in the + * {@link #statusTracker}. + */ + private void trackStatusOfCompletedTasks(Set activeTaskIds) + { + final Set finishedTaskIds = new HashSet<>(statusTracker.getSubmittedTaskIds()); + finishedTaskIds.removeAll(activeTaskIds); + + if (finishedTaskIds.isEmpty()) { + return; + } + + final Map taskStatusMap + = FutureUtils.getUnchecked(overlordClient.taskStatuses(finishedTaskIds), true); + for (String taskId : finishedTaskIds) { + // Assume unknown task to have finished successfully + final TaskStatus taskStatus = taskStatusMap.getOrDefault(taskId, TaskStatus.success(taskId)); + if (taskStatus.isComplete()) { + statusTracker.onTaskFinished(taskId, taskStatus); + } + } + } + + /** + * @return Maximum number of task slots used by a native compaction task at + * any time when the task is run with the given tuning config. + */ + public static int getMaxTaskSlotsForNativeCompactionTask( + @Nullable ClientCompactionTaskQueryTuningConfig tuningConfig + ) + { + if (isParallelMode(tuningConfig)) { + Integer maxNumConcurrentSubTasks = tuningConfig.getMaxNumConcurrentSubTasks(); + // Max number of task slots used in parallel mode = maxNumConcurrentSubTasks + 1 (supervisor task) + return (maxNumConcurrentSubTasks == null ? 1 : maxNumConcurrentSubTasks) + 1; + } else { + return 1; + } + } + + /** + * @return Maximum number of task slots used by an MSQ compaction task at any + * time when the task is run with the given context. + */ + public static int getMaxTaskSlotsForMSQCompactionTask(@Nullable Map context) + { + return context == null + ? ClientMSQContext.DEFAULT_MAX_NUM_TASKS + : (int) context.getOrDefault(ClientMSQContext.CTX_MAX_NUM_TASKS, ClientMSQContext.DEFAULT_MAX_NUM_TASKS); + } + + + /** + * Returns true if the compaction task can run in the parallel mode with the given tuningConfig. + * This method should be synchronized with ParallelIndexSupervisorTask.isParallelMode(InputSource, ParallelIndexTuningConfig). + */ + @VisibleForTesting + public static boolean isParallelMode(@Nullable ClientCompactionTaskQueryTuningConfig tuningConfig) + { + if (null == tuningConfig) { + return false; + } + boolean useRangePartitions = useRangePartitions(tuningConfig); + int minRequiredNumConcurrentSubTasks = useRangePartitions ? 1 : 2; + return tuningConfig.getMaxNumConcurrentSubTasks() != null + && tuningConfig.getMaxNumConcurrentSubTasks() >= minRequiredNumConcurrentSubTasks; + } + + private static boolean useRangePartitions(ClientCompactionTaskQueryTuningConfig tuningConfig) + { + // dynamic partitionsSpec will be used if getPartitionsSpec() returns null + return tuningConfig.getPartitionsSpec() instanceof DimensionRangePartitionsSpec; + } + + public int computeSlotsRequiredForTask( + ClientCompactionTaskQuery task, + DataSourceCompactionConfig config + ) + { + if (task.getCompactionRunner().getType() == CompactionEngine.MSQ) { + final Map autoCompactionContext = task.getContext(); + if (autoCompactionContext.containsKey(ClientMSQContext.CTX_MAX_NUM_TASKS)) { + return (int) autoCompactionContext.get(ClientMSQContext.CTX_MAX_NUM_TASKS); + } else { + // Since MSQ needs all task slots for the calculated #tasks to be available upfront, allot all available + // compaction slots (upto a max of MAX_TASK_SLOTS_FOR_MSQ_COMPACTION) to current compaction task to avoid + // stalling. Setting "taskAssignment" to "auto" has the problem of not being able to determine the actual + // count, which is required for subsequent tasks. + final int slotsRequiredForCurrentTask = Math.min( + // Update the slots to 2 (min required for MSQ) if only 1 slot is available. + numAvailableTaskSlots == 1 ? 2 : numAvailableTaskSlots, + ClientMSQContext.MAX_TASK_SLOTS_FOR_MSQ_COMPACTION_TASK + ); + autoCompactionContext.put(ClientMSQContext.CTX_MAX_NUM_TASKS, slotsRequiredForCurrentTask); + + return slotsRequiredForCurrentTask; + } + } else { + return CompactionSlotManager.getMaxTaskSlotsForNativeCompactionTask( + config.getTuningConfig() + ); + } + } +} diff --git a/server/src/main/java/org/apache/druid/server/compaction/CompactionSnapshotBuilder.java b/server/src/main/java/org/apache/druid/server/compaction/CompactionSnapshotBuilder.java new file mode 100644 index 000000000000..e52d24edbf01 --- /dev/null +++ b/server/src/main/java/org/apache/druid/server/compaction/CompactionSnapshotBuilder.java @@ -0,0 +1,94 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.server.compaction; + +import org.apache.druid.server.coordinator.AutoCompactionSnapshot; +import org.apache.druid.server.coordinator.stats.CoordinatorRunStats; +import org.apache.druid.server.coordinator.stats.Dimension; +import org.apache.druid.server.coordinator.stats.RowKey; +import org.apache.druid.server.coordinator.stats.Stats; + +import java.util.HashMap; +import java.util.Map; + +/** + * Builds {@link AutoCompactionSnapshot} for multiple datasources using the + * identified {@link CompactionCandidate} list. + */ +public class CompactionSnapshotBuilder +{ + private final CoordinatorRunStats stats; + private final Map datasourceToBuilder = new HashMap<>(); + + public CompactionSnapshotBuilder(CoordinatorRunStats runStats) + { + this.stats = runStats; + } + + public void addToComplete(CompactionCandidate candidate) + { + getBuilderForDatasource(candidate.getDataSource()) + .incrementCompactedStats(candidate.getStats()); + } + + public void addToPending(CompactionCandidate candidate) + { + getBuilderForDatasource(candidate.getDataSource()) + .incrementWaitingStats(candidate.getStats()); + } + + public void addToSkipped(CompactionCandidate candidate) + { + getBuilderForDatasource(candidate.getDataSource()) + .incrementSkippedStats(candidate.getStats()); + } + + public Map build() + { + final Map datasourceToSnapshot = new HashMap<>(); + datasourceToBuilder.forEach((dataSource, builder) -> { + final AutoCompactionSnapshot autoCompactionSnapshot = builder.build(); + datasourceToSnapshot.put(dataSource, autoCompactionSnapshot); + collectSnapshotStats(autoCompactionSnapshot); + }); + + return datasourceToSnapshot; + } + + private AutoCompactionSnapshot.Builder getBuilderForDatasource(String dataSource) + { + return datasourceToBuilder.computeIfAbsent(dataSource, AutoCompactionSnapshot::builder); + } + + private void collectSnapshotStats(AutoCompactionSnapshot autoCompactionSnapshot) + { + final RowKey rowKey = RowKey.of(Dimension.DATASOURCE, autoCompactionSnapshot.getDataSource()); + + stats.add(Stats.Compaction.PENDING_BYTES, rowKey, autoCompactionSnapshot.getBytesAwaitingCompaction()); + stats.add(Stats.Compaction.PENDING_SEGMENTS, rowKey, autoCompactionSnapshot.getSegmentCountAwaitingCompaction()); + stats.add(Stats.Compaction.PENDING_INTERVALS, rowKey, autoCompactionSnapshot.getIntervalCountAwaitingCompaction()); + stats.add(Stats.Compaction.COMPACTED_BYTES, rowKey, autoCompactionSnapshot.getBytesCompacted()); + stats.add(Stats.Compaction.COMPACTED_SEGMENTS, rowKey, autoCompactionSnapshot.getSegmentCountCompacted()); + stats.add(Stats.Compaction.COMPACTED_INTERVALS, rowKey, autoCompactionSnapshot.getIntervalCountCompacted()); + stats.add(Stats.Compaction.SKIPPED_BYTES, rowKey, autoCompactionSnapshot.getBytesSkipped()); + stats.add(Stats.Compaction.SKIPPED_SEGMENTS, rowKey, autoCompactionSnapshot.getSegmentCountSkipped()); + stats.add(Stats.Compaction.SKIPPED_INTERVALS, rowKey, autoCompactionSnapshot.getIntervalCountSkipped()); + } +} 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 98194eefdbd5..97b645388218 100644 --- a/server/src/main/java/org/apache/druid/server/compaction/CompactionStatus.java +++ b/server/src/main/java/org/apache/druid/server/compaction/CompactionStatus.java @@ -19,12 +19,12 @@ package org.apache.druid.server.compaction; -import com.fasterxml.jackson.databind.ObjectMapper; import org.apache.commons.lang3.ArrayUtils; import org.apache.druid.client.indexing.ClientCompactionTaskGranularitySpec; import org.apache.druid.client.indexing.ClientCompactionTaskQueryTuningConfig; import org.apache.druid.common.config.Configs; import org.apache.druid.data.input.impl.DimensionSchema; +import org.apache.druid.indexer.granularity.GranularitySpec; import org.apache.druid.indexer.partitions.DimensionRangePartitionsSpec; import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; import org.apache.druid.indexer.partitions.HashedPartitionsSpec; @@ -64,6 +64,7 @@ public enum State * The order of the checks must be honored while evaluating them. */ private static final List> CHECKS = Arrays.asList( + Evaluator::inputBytesAreWithinLimit, Evaluator::segmentsHaveBeenCompactedAtLeastOnce, Evaluator::allCandidatesHaveSameCompactionState, Evaluator::partitionsSpecIsUpToDate, @@ -115,7 +116,7 @@ public String toString() '}'; } - private static CompactionStatus incomplete(String reasonFormat, Object... args) + public static CompactionStatus pending(String reasonFormat, Object... args) { return new CompactionStatus(State.PENDING, StringUtils.format(reasonFormat, args)); } @@ -141,7 +142,7 @@ private static CompactionStatus configChanged( Function stringFunction ) { - return CompactionStatus.incomplete( + return CompactionStatus.pending( "'%s' mismatch: required[%s], current[%s]", field, target == null ? null : stringFunction.apply(target), @@ -204,11 +205,10 @@ static CompactionStatus running(String reasonForCompaction) */ static CompactionStatus compute( CompactionCandidate candidateSegments, - DataSourceCompactionConfig config, - ObjectMapper objectMapper + DataSourceCompactionConfig config ) { - final Evaluator evaluator = new Evaluator(candidateSegments, config, objectMapper); + final Evaluator evaluator = new Evaluator(candidateSegments, config); return CHECKS.stream().map(f -> f.apply(evaluator)) .filter(status -> !status.isComplete()) .findFirst().orElse(COMPLETE); @@ -266,7 +266,6 @@ static DimensionRangePartitionsSpec getEffectiveRangePartitionsSpec(DimensionRan */ private static class Evaluator { - private final ObjectMapper objectMapper; private final DataSourceCompactionConfig compactionConfig; private final CompactionCandidate candidateSegments; private final CompactionState lastCompactionState; @@ -276,12 +275,10 @@ private static class Evaluator private Evaluator( CompactionCandidate candidateSegments, - DataSourceCompactionConfig compactionConfig, - ObjectMapper objectMapper + DataSourceCompactionConfig compactionConfig ) { this.candidateSegments = candidateSegments; - this.objectMapper = objectMapper; this.lastCompactionState = candidateSegments.getSegments().get(0).getLastCompactionState(); this.compactionConfig = compactionConfig; this.tuningConfig = ClientCompactionTaskQueryTuningConfig.from(compactionConfig); @@ -290,8 +287,7 @@ private Evaluator( this.existingGranularitySpec = null; } else { this.existingGranularitySpec = convertIfNotNull( - lastCompactionState.getGranularitySpec(), - ClientCompactionTaskGranularitySpec.class + lastCompactionState.getGranularitySpec() ); } } @@ -299,7 +295,7 @@ private Evaluator( private CompactionStatus segmentsHaveBeenCompactedAtLeastOnce() { if (lastCompactionState == null) { - return CompactionStatus.incomplete("not compacted yet"); + return CompactionStatus.pending("not compacted yet"); } else { return COMPLETE; } @@ -313,7 +309,7 @@ private CompactionStatus allCandidatesHaveSameCompactionState() if (allHaveSameCompactionState) { return COMPLETE; } else { - return CompactionStatus.incomplete("segments have different last compaction states"); + return CompactionStatus.pending("segments have different last compaction states"); } } @@ -336,7 +332,7 @@ private CompactionStatus indexSpecIsUpToDate() return CompactionStatus.completeIfEqual( "indexSpec", Configs.valueOrDefault(tuningConfig.getIndexSpec(), IndexSpec.DEFAULT), - objectMapper.convertValue(lastCompactionState.getIndexSpec(), IndexSpec.class), + lastCompactionState.getIndexSpec(), String::valueOf ); } @@ -351,6 +347,19 @@ private CompactionStatus projectionsAreUpToDate() ); } + private CompactionStatus inputBytesAreWithinLimit() + { + final long inputSegmentSize = compactionConfig.getInputSegmentSizeBytes(); + if (candidateSegments.getTotalBytes() > inputSegmentSize) { + return CompactionStatus.skipped( + "'inputSegmentSize' exceeded: Total segment size[%d] is larger than allowed inputSegmentSize[%d]", + candidateSegments.getTotalBytes(), inputSegmentSize + ); + } else { + return COMPLETE; + } + } + private CompactionStatus segmentGranularityIsUpToDate() { if (configuredGranularitySpec == null @@ -371,7 +380,7 @@ private CompactionStatus segmentGranularityIsUpToDate() segment -> !configuredSegmentGranularity.isAligned(segment.getInterval()) ); if (needsCompaction) { - return CompactionStatus.incomplete( + return CompactionStatus.pending( "segmentGranularity: segments do not align with target[%s]", asString(configuredSegmentGranularity) ); @@ -477,10 +486,7 @@ private CompactionStatus transformSpecFilterIsUpToDate() return COMPLETE; } - CompactionTransformSpec existingTransformSpec = convertIfNotNull( - lastCompactionState.getTransformSpec(), - CompactionTransformSpec.class - ); + CompactionTransformSpec existingTransformSpec = lastCompactionState.getTransformSpec(); return CompactionStatus.completeIfEqual( "transformSpec filter", compactionConfig.getTransformSpec().getFilter(), @@ -490,12 +496,16 @@ private CompactionStatus transformSpecFilterIsUpToDate() } @Nullable - private T convertIfNotNull(Object object, Class clazz) + private ClientCompactionTaskGranularitySpec convertIfNotNull(GranularitySpec granularitySpec) { - if (object == null) { + if (granularitySpec == null) { return null; } else { - return objectMapper.convertValue(object, clazz); + return new ClientCompactionTaskGranularitySpec( + granularitySpec.getSegmentGranularity(), + granularitySpec.getQueryGranularity(), + granularitySpec.isRollup() + ); } } } 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 cbf5f25f9d7b..5c3daf2a26d9 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 @@ -19,14 +19,10 @@ package org.apache.druid.server.compaction; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.inject.Inject; -import org.apache.druid.client.indexing.ClientCompactionTaskQuery; import org.apache.druid.indexer.TaskState; import org.apache.druid.indexer.TaskStatus; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; -import org.apache.druid.server.coordinator.DruidCompactionConfig; import org.joda.time.DateTime; import org.joda.time.Duration; import org.joda.time.Interval; @@ -45,7 +41,6 @@ public class CompactionStatusTracker { private static final Duration MAX_STATUS_RETAIN_DURATION = Duration.standardHours(12); - private final ObjectMapper objectMapper; private final ConcurrentHashMap datasourceStatuses = new ConcurrentHashMap<>(); private final ConcurrentHashMap submittedTaskIdToSegments @@ -53,10 +48,8 @@ public class CompactionStatusTracker private final AtomicReference segmentSnapshotTime = new AtomicReference<>(); - @Inject - public CompactionStatusTracker(ObjectMapper objectMapper) + public CompactionStatusTracker() { - this.objectMapper = objectMapper; } public void stop() @@ -86,25 +79,17 @@ public Set getSubmittedTaskIds() return submittedTaskIdToSegments.keySet(); } + /** + * Checks if compaction can be started for the given {@link CompactionCandidate}. + * This method assumes that the given candidate is eligible for compaction + * based on the current compaction config/supervisor of the datasource. + */ public CompactionStatus computeCompactionStatus( CompactionCandidate candidate, - DataSourceCompactionConfig config, CompactionCandidateSearchPolicy searchPolicy ) { - final CompactionStatus compactionStatus = CompactionStatus.compute(candidate, config, objectMapper); - if (compactionStatus.isComplete()) { - return compactionStatus; - } - - // Skip intervals that violate max allowed input segment size - final long inputSegmentSize = config.getInputSegmentSizeBytes(); - if (candidate.getTotalBytes() > inputSegmentSize) { - return CompactionStatus.skipped( - "'inputSegmentSize' exceeded: Total segment size[%d] is larger than allowed inputSegmentSize[%d]", - candidate.getTotalBytes(), inputSegmentSize - ); - } + final CompactionStatus pendingStatus = CompactionStatus.pending("not compacted yet"); // Skip intervals that already have a running task final CompactionTaskStatus lastTaskStatus = getLatestTaskStatus(candidate); @@ -123,11 +108,11 @@ public CompactionStatus computeCompactionStatus( } // Skip intervals that have been filtered out by the policy - if (!searchPolicy.isEligibleForCompaction(candidate, compactionStatus, lastTaskStatus)) { + if (!searchPolicy.isEligibleForCompaction(candidate, pendingStatus, lastTaskStatus)) { return CompactionStatus.skipped("Rejected by search policy"); } - return compactionStatus; + return pendingStatus; } public void onCompactionStatusComputed( @@ -143,17 +128,15 @@ public void onSegmentTimelineUpdated(DateTime snapshotTime) this.segmentSnapshotTime.set(snapshotTime); } - public void onCompactionConfigUpdated(DruidCompactionConfig compactionConfig) + /** + * Updates the set of datasources that have compaction enabled and cleans up + * stale task statuses. + */ + public void resetActiveDatasources(Set compactionEnabledDatasources) { - final Set compactionEnabledDatasources = new HashSet<>(); - if (compactionConfig.getCompactionConfigs() != null) { - compactionConfig.getCompactionConfigs().forEach(config -> { - getOrComputeDatasourceStatus(config.getDataSource()) - .cleanupStaleTaskStatuses(); - - compactionEnabledDatasources.add(config.getDataSource()); - }); - } + compactionEnabledDatasources.forEach( + dataSource -> getOrComputeDatasourceStatus(dataSource).cleanupStaleTaskStatuses() + ); // Clean up state for datasources where compaction has been disabled final Set allDatasources = new HashSet<>(datasourceStatuses.keySet()); @@ -165,12 +148,12 @@ public void onCompactionConfigUpdated(DruidCompactionConfig compactionConfig) } public void onTaskSubmitted( - ClientCompactionTaskQuery taskPayload, + String taskId, CompactionCandidate candidateSegments ) { - submittedTaskIdToSegments.put(taskPayload.getId(), candidateSegments); - getOrComputeDatasourceStatus(taskPayload.getDataSource()) + submittedTaskIdToSegments.put(taskId, candidateSegments); + getOrComputeDatasourceStatus(candidateSegments.getDataSource()) .handleSubmittedTask(candidateSegments); } diff --git a/server/src/main/java/org/apache/druid/server/compaction/DataSourceCompactibleSegmentIterator.java b/server/src/main/java/org/apache/druid/server/compaction/DataSourceCompactibleSegmentIterator.java index e15d310b33db..588e4ba77d41 100644 --- a/server/src/main/java/org/apache/druid/server/compaction/DataSourceCompactibleSegmentIterator.java +++ b/server/src/main/java/org/apache/druid/server/compaction/DataSourceCompactibleSegmentIterator.java @@ -65,8 +65,6 @@ public class DataSourceCompactibleSegmentIterator implements CompactionSegmentIt private final String dataSource; private final DataSourceCompactionConfig config; - private final CompactionStatusTracker statusTracker; - private final CompactionCandidateSearchPolicy searchPolicy; private final List compactedSegments = new ArrayList<>(); private final List skippedSegments = new ArrayList<>(); @@ -83,14 +81,11 @@ public DataSourceCompactibleSegmentIterator( DataSourceCompactionConfig config, SegmentTimeline timeline, List skipIntervals, - CompactionCandidateSearchPolicy searchPolicy, - CompactionStatusTracker statusTracker + CompactionCandidateSearchPolicy searchPolicy ) { - this.statusTracker = statusTracker; this.config = config; this.dataSource = config.getDataSource(); - this.searchPolicy = searchPolicy; this.queue = new PriorityQueue<>(searchPolicy::compareCandidates); populateQueue(timeline, skipIntervals); @@ -121,7 +116,6 @@ private void populateQueue(SegmentTimeline timeline, List skipInterval CompactionStatus.skipped("Segments have partial-eternity intervals") ); skippedSegments.add(candidatesWithStatus); - statusTracker.onCompactionStatusComputed(candidatesWithStatus, config); return; } @@ -316,10 +310,8 @@ private void findAndEnqueueSegmentsToCompact(CompactibleSegmentIterator compacti } final CompactionCandidate candidates = CompactionCandidate.from(segments); - final CompactionStatus compactionStatus - = statusTracker.computeCompactionStatus(candidates, config, searchPolicy); + final CompactionStatus compactionStatus = CompactionStatus.compute(candidates, config); final CompactionCandidate candidatesWithStatus = candidates.withCurrentStatus(compactionStatus); - statusTracker.onCompactionStatusComputed(candidatesWithStatus, config); if (compactionStatus.isComplete()) { compactedSegments.add(candidatesWithStatus); @@ -371,7 +363,6 @@ private List findInitialSearchInterval( final CompactionCandidate candidatesWithStatus = candidates.withCurrentStatus(reason); skippedSegments.add(candidatesWithStatus); - statusTracker.onCompactionStatusComputed(candidatesWithStatus, config); } } diff --git a/server/src/main/java/org/apache/druid/server/compaction/PriorityBasedCompactionSegmentIterator.java b/server/src/main/java/org/apache/druid/server/compaction/PriorityBasedCompactionSegmentIterator.java index 1e91df7e38a0..49d936fda0ac 100644 --- a/server/src/main/java/org/apache/druid/server/compaction/PriorityBasedCompactionSegmentIterator.java +++ b/server/src/main/java/org/apache/druid/server/compaction/PriorityBasedCompactionSegmentIterator.java @@ -48,8 +48,7 @@ public PriorityBasedCompactionSegmentIterator( CompactionCandidateSearchPolicy searchPolicy, Map compactionConfigs, Map datasourceToTimeline, - Map> skipIntervals, - CompactionStatusTracker statusTracker + Map> skipIntervals ) { this.queue = new PriorityQueue<>(searchPolicy::compareCandidates); @@ -70,8 +69,7 @@ public PriorityBasedCompactionSegmentIterator( compactionConfigs.get(datasource), timeline, skipIntervals.getOrDefault(datasource, Collections.emptyList()), - searchPolicy, - statusTracker + searchPolicy ) ); addNextItemForDatasourceToQueue(datasource); 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 543bc2b82139..cf166c8f3204 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 @@ -23,7 +23,6 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; -import com.google.common.base.Predicate; import org.apache.druid.client.DataSourcesSnapshot; import org.apache.druid.client.indexing.ClientCompactionIOConfig; import org.apache.druid.client.indexing.ClientCompactionIntervalSpec; @@ -32,26 +31,22 @@ import org.apache.druid.client.indexing.ClientCompactionTaskGranularitySpec; import org.apache.druid.client.indexing.ClientCompactionTaskQuery; import org.apache.druid.client.indexing.ClientCompactionTaskQueryTuningConfig; -import org.apache.druid.client.indexing.ClientMSQContext; -import org.apache.druid.client.indexing.TaskPayloadResponse; 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.indexer.CompactionEngine; -import org.apache.druid.indexer.TaskStatus; -import org.apache.druid.indexer.TaskStatusPlus; -import org.apache.druid.indexer.partitions.DimensionRangePartitionsSpec; -import org.apache.druid.java.util.common.ISE; 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.metadata.LockFilterPolicy; 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.CompactionSegmentIterator; +import org.apache.druid.server.compaction.CompactionSlotManager; +import org.apache.druid.server.compaction.CompactionSnapshotBuilder; +import org.apache.druid.server.compaction.CompactionStatus; import org.apache.druid.server.compaction.CompactionStatusTracker; import org.apache.druid.server.compaction.PriorityBasedCompactionSegmentIterator; import org.apache.druid.server.coordinator.AutoCompactionSnapshot; @@ -66,10 +61,8 @@ import org.joda.time.Interval; import javax.annotation.Nullable; -import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; -import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; @@ -79,10 +72,6 @@ public class CompactSegments implements CoordinatorCustomDuty { - /** - * Must be the same as org.apache.druid.indexing.common.task.CompactionTask.TYPE. - */ - public static final String COMPACTION_TASK_TYPE = "compact"; /** * Must be the same as org.apache.druid.indexing.common.task.Tasks.STORE_COMPACTION_STATE_KEY */ @@ -92,8 +81,6 @@ public class CompactSegments implements CoordinatorCustomDuty private static final Logger LOG = new Logger(CompactSegments.class); private static final String TASK_ID_PREFIX = "coordinator-issued"; - private static final Predicate IS_COMPACTION_TASK = - status -> null != status && COMPACTION_TASK_TYPE.equals(status.getType()); private final CompactionStatusTracker statusTracker; private final OverlordClient overlordClient; @@ -152,75 +139,36 @@ public void run( } statusTracker.onSegmentTimelineUpdated(dataSources.getSnapshotTime()); - statusTracker.onCompactionConfigUpdated(dynamicConfig); List compactionConfigList = dynamicConfig.getCompactionConfigs(); if (compactionConfigList == null || compactionConfigList.isEmpty()) { resetCompactionSnapshot(); + statusTracker.resetActiveDatasources(Set.of()); return; } Map compactionConfigs = compactionConfigList .stream() .collect(Collectors.toMap(DataSourceCompactionConfig::getDataSource, Function.identity())); + statusTracker.resetActiveDatasources(compactionConfigs.keySet()); - // Map from dataSource to list of intervals for which compaction will be skipped in this run - final Map> intervalsToSkipCompaction = new HashMap<>(); - - // Fetch currently running compaction tasks - int busyCompactionTaskSlots = 0; - final List compactionTasks = CoordinatorDutyUtils.getStatusOfActiveTasks( + final CompactionSlotManager slotManager = new CompactionSlotManager( overlordClient, - IS_COMPACTION_TASK + statusTracker, + dynamicConfig.clusterConfig() ); + stats.add(Stats.Compaction.MAX_SLOTS, slotManager.getNumAvailableTaskSlots()); - final Set activeTaskIds - = compactionTasks.stream().map(TaskStatusPlus::getId).collect(Collectors.toSet()); - trackStatusOfCompletedTasks(activeTaskIds); - - for (TaskStatusPlus status : compactionTasks) { - final TaskPayloadResponse response = - FutureUtils.getUnchecked(overlordClient.taskPayload(status.getId()), true); - if (response == null) { - throw new ISE("Could not find payload for active compaction task[%s]", status.getId()); - } else if (!COMPACTION_TASK_TYPE.equals(response.getPayload().getType())) { - throw new ISE( - "Payload of active compaction task[%s] is of invalid type[%s]", - status.getId(), response.getPayload().getType() - ); - } - - final ClientCompactionTaskQuery compactionTaskQuery = (ClientCompactionTaskQuery) response.getPayload(); - DataSourceCompactionConfig dataSourceCompactionConfig = compactionConfigs.get(status.getDataSource()); - if (cancelTaskIfGranularityChanged(compactionTaskQuery, dataSourceCompactionConfig)) { - continue; - } - - // Skip this interval as the current active compaction task is good - final Interval interval = compactionTaskQuery.getIoConfig().getInputSpec().getInterval(); - intervalsToSkipCompaction.computeIfAbsent(status.getDataSource(), k -> new ArrayList<>()) - .add(interval); - // Note: The default compactionRunnerType used here should match the default runner used in CompactionTask when - // no runner is provided there. - CompactionEngine compactionRunnerType = compactionTaskQuery.getCompactionRunner() == null - ? CompactionEngine.NATIVE - : compactionTaskQuery.getCompactionRunner().getType(); - if (compactionRunnerType == CompactionEngine.NATIVE) { - busyCompactionTaskSlots += - findMaxNumTaskSlotsUsedByOneNativeCompactionTask(compactionTaskQuery.getTuningConfig()); - } else { - busyCompactionTaskSlots += findMaxNumTaskSlotsUsedByOneMsqCompactionTask(compactionTaskQuery.getContext()); + // Fetch currently running compaction tasks + for (ClientCompactionTaskQuery compactionTaskQuery : slotManager.fetchRunningCompactionTasks()) { + final String dataSource = compactionTaskQuery.getDataSource(); + DataSourceCompactionConfig dataSourceCompactionConfig = compactionConfigs.get(dataSource); + if (slotManager.cancelTaskOnlyIfGranularityChanged(compactionTaskQuery, dataSourceCompactionConfig)) { + stats.add(Stats.Compaction.CANCELLED_TASKS, RowKey.of(Dimension.DATASOURCE, dataSource), 1L); } } + stats.add(Stats.Compaction.AVAILABLE_SLOTS, slotManager.getNumAvailableTaskSlots()); - // Skip all the intervals locked by higher priority tasks for each datasource - // This must be done after the invalid compaction tasks are cancelled - // in the loop above so that their intervals are not considered locked - getLockedIntervals(compactionConfigList).forEach( - (dataSource, intervals) -> - intervalsToSkipCompaction - .computeIfAbsent(dataSource, ds -> new ArrayList<>()) - .addAll(intervals) - ); + slotManager.skipLockedIntervals(compactionConfigList); // Get iterator over segments to compact and submit compaction tasks final CompactionCandidateSearchPolicy policy = dynamicConfig.getCompactionPolicy(); @@ -228,27 +176,21 @@ public void run( policy, compactionConfigs, dataSources.getUsedSegmentsTimelinesPerDataSource(), - intervalsToSkipCompaction, - statusTracker + slotManager.getDatasourceIntervalsToSkipCompaction() ); - final int compactionTaskCapacity = getCompactionTaskCapacity(dynamicConfig); - final int availableCompactionTaskSlots - = getAvailableCompactionTaskSlots(compactionTaskCapacity, busyCompactionTaskSlots); - - final Map currentRunAutoCompactionSnapshotBuilders = new HashMap<>(); + final CompactionSnapshotBuilder compactionSnapshotBuilder = new CompactionSnapshotBuilder(stats); final int numSubmittedCompactionTasks = submitCompactionTasks( compactionConfigs, - currentRunAutoCompactionSnapshotBuilders, - availableCompactionTaskSlots, + compactionSnapshotBuilder, + slotManager, iterator, + policy, defaultEngine ); - stats.add(Stats.Compaction.MAX_SLOTS, compactionTaskCapacity); - stats.add(Stats.Compaction.AVAILABLE_SLOTS, availableCompactionTaskSlots); stats.add(Stats.Compaction.SUBMITTED_TASKS, numSubmittedCompactionTasks); - updateCompactionSnapshotStats(currentRunAutoCompactionSnapshotBuilders, iterator, stats); + updateCompactionSnapshotStats(compactionSnapshotBuilder, iterator, compactionConfigs); } private void resetCompactionSnapshot() @@ -271,333 +213,168 @@ private boolean isCompactionSupervisorEnabled() } } - /** - * Queries the Overlord for the status of all tasks that were submitted - * recently but are not active anymore. The statuses are then updated in the - * {@link #statusTracker}. - */ - private void trackStatusOfCompletedTasks(Set activeTaskIds) - { - final Set finishedTaskIds = new HashSet<>(statusTracker.getSubmittedTaskIds()); - finishedTaskIds.removeAll(activeTaskIds); - - if (finishedTaskIds.isEmpty()) { - return; - } - - final Map taskStatusMap - = FutureUtils.getUnchecked(overlordClient.taskStatuses(finishedTaskIds), true); - for (String taskId : finishedTaskIds) { - // Assume unknown task to have finished successfully - final TaskStatus taskStatus = taskStatusMap.getOrDefault(taskId, TaskStatus.success(taskId)); - if (taskStatus.isComplete()) { - statusTracker.onTaskFinished(taskId, taskStatus); - } - } - } - - /** - * Cancels a currently running compaction task if the segment granularity - * for this datasource has changed in the compaction config. - * - * @return true if the task was canceled, false otherwise. - */ - private boolean cancelTaskIfGranularityChanged( - ClientCompactionTaskQuery compactionTaskQuery, - DataSourceCompactionConfig dataSourceCompactionConfig - ) - { - if (dataSourceCompactionConfig == null - || dataSourceCompactionConfig.getGranularitySpec() == null - || compactionTaskQuery.getGranularitySpec() == null) { - return false; - } - - Granularity configuredSegmentGranularity = dataSourceCompactionConfig.getGranularitySpec() - .getSegmentGranularity(); - Granularity taskSegmentGranularity = compactionTaskQuery.getGranularitySpec().getSegmentGranularity(); - if (configuredSegmentGranularity == null || configuredSegmentGranularity.equals(taskSegmentGranularity)) { - return false; - } - - LOG.info( - "Cancelling task[%s] as task segmentGranularity[%s] differs from compaction config segmentGranularity[%s].", - compactionTaskQuery.getId(), taskSegmentGranularity, configuredSegmentGranularity - ); - overlordClient.cancelTask(compactionTaskQuery.getId()); - return true; - } - - /** - * Gets a List of Intervals locked by higher priority tasks for each datasource. - * However, when using a REPLACE lock for compaction, intervals locked with any APPEND lock will not be returned - * Since compaction tasks submitted for these Intervals would have to wait anyway, - * we skip these Intervals until the next compaction run. - *

- * For now, Segment Locks are being treated the same as Time Chunk Locks even - * though they lock only a Segment and not the entire Interval. Thus, - * a compaction task will not be submitted for an Interval if - *

    - *
  • either the whole Interval is locked by a higher priority Task with an incompatible lock type
  • - *
  • or there is atleast one Segment in the Interval that is locked by a - * higher priority Task
  • - *
- */ - private Map> getLockedIntervals( - List compactionConfigs - ) - { - final List lockFilterPolicies = compactionConfigs - .stream() - .map(config -> - new LockFilterPolicy(config.getDataSource(), config.getTaskPriority(), null, config.getTaskContext())) - .collect(Collectors.toList()); - final Map> datasourceToLockedIntervals = - new HashMap<>(FutureUtils.getUnchecked(overlordClient.findLockedIntervals(lockFilterPolicies), true)); - LOG.debug( - "Skipping the following intervals for Compaction as they are currently locked: %s", - datasourceToLockedIntervals - ); - - return datasourceToLockedIntervals; - } - - /** - * Returns the maximum number of task slots used by one native compaction task at any time when the task is - * issued with the given tuningConfig. - */ - public static int findMaxNumTaskSlotsUsedByOneNativeCompactionTask( - @Nullable ClientCompactionTaskQueryTuningConfig tuningConfig - ) - { - if (isParallelMode(tuningConfig)) { - @Nullable - Integer maxNumConcurrentSubTasks = tuningConfig.getMaxNumConcurrentSubTasks(); - // Max number of task slots used in parallel mode = maxNumConcurrentSubTasks + 1 (supervisor task) - return (maxNumConcurrentSubTasks == null ? 1 : maxNumConcurrentSubTasks) + 1; - } else { - return 1; - } - } - - /** - * Returns the maximum number of task slots used by one MSQ compaction task at any time when the task is - * issued with the given context. - */ - static int findMaxNumTaskSlotsUsedByOneMsqCompactionTask(@Nullable Map context) - { - return context == null - ? ClientMSQContext.DEFAULT_MAX_NUM_TASKS - : (int) context.getOrDefault(ClientMSQContext.CTX_MAX_NUM_TASKS, ClientMSQContext.DEFAULT_MAX_NUM_TASKS); - } - - - /** - * Returns true if the compaction task can run in the parallel mode with the given tuningConfig. - * This method should be synchronized with ParallelIndexSupervisorTask.isParallelMode(InputSource, ParallelIndexTuningConfig). - */ - @VisibleForTesting - static boolean isParallelMode(@Nullable ClientCompactionTaskQueryTuningConfig tuningConfig) - { - if (null == tuningConfig) { - return false; - } - boolean useRangePartitions = useRangePartitions(tuningConfig); - int minRequiredNumConcurrentSubTasks = useRangePartitions ? 1 : 2; - return tuningConfig.getMaxNumConcurrentSubTasks() != null - && tuningConfig.getMaxNumConcurrentSubTasks() >= minRequiredNumConcurrentSubTasks; - } - - private static boolean useRangePartitions(ClientCompactionTaskQueryTuningConfig tuningConfig) - { - // dynamic partitionsSpec will be used if getPartitionsSpec() returns null - return tuningConfig.getPartitionsSpec() instanceof DimensionRangePartitionsSpec; - } - - private int getCompactionTaskCapacity(DruidCompactionConfig dynamicConfig) - { - int totalWorkerCapacity = CoordinatorDutyUtils.getTotalWorkerCapacity(overlordClient); - - return Math.min( - (int) (totalWorkerCapacity * dynamicConfig.getCompactionTaskSlotRatio()), - dynamicConfig.getMaxCompactionTaskSlots() - ); - } - - private int getAvailableCompactionTaskSlots(int compactionTaskCapacity, int busyCompactionTaskSlots) - { - final int availableCompactionTaskSlots; - if (busyCompactionTaskSlots > 0) { - availableCompactionTaskSlots = Math.max(0, compactionTaskCapacity - busyCompactionTaskSlots); - } else { - // compactionTaskCapacity might be 0 if totalWorkerCapacity is low. - // This guarantees that at least one slot is available if - // compaction is enabled and estimatedIncompleteCompactionTasks is 0. - availableCompactionTaskSlots = Math.max(1, compactionTaskCapacity); - } - LOG.debug( - "Found [%d] available task slots for compaction out of max compaction task capacity [%d]", - availableCompactionTaskSlots, compactionTaskCapacity - ); - - return availableCompactionTaskSlots; - } - /** * Submits compaction tasks to the Overlord. Returns total number of tasks submitted. */ private int submitCompactionTasks( Map compactionConfigs, - Map currentRunAutoCompactionSnapshotBuilders, - int numAvailableCompactionTaskSlots, + CompactionSnapshotBuilder snapshotBuilder, + CompactionSlotManager slotManager, CompactionSegmentIterator iterator, + CompactionCandidateSearchPolicy policy, CompactionEngine defaultEngine ) { - if (numAvailableCompactionTaskSlots <= 0) { + if (slotManager.getNumAvailableTaskSlots() <= 0) { return 0; } int numSubmittedTasks = 0; int totalTaskSlotsAssigned = 0; - while (iterator.hasNext() && totalTaskSlotsAssigned < numAvailableCompactionTaskSlots) { + while (iterator.hasNext() && totalTaskSlotsAssigned < slotManager.getNumAvailableTaskSlots()) { final CompactionCandidate entry = iterator.next(); final String dataSourceName = entry.getDataSource(); + final DataSourceCompactionConfig config = compactionConfigs.get(dataSourceName); - // As these segments will be compacted, we will aggregate the statistic to the Compacted statistics - currentRunAutoCompactionSnapshotBuilders - .computeIfAbsent(dataSourceName, AutoCompactionSnapshot::builder) - .incrementCompactedStats(entry.getStats()); + final CompactionStatus compactionStatus = + statusTracker.computeCompactionStatus(entry, policy); + final CompactionCandidate candidatesWithStatus = entry.withCurrentStatus(compactionStatus); + statusTracker.onCompactionStatusComputed(candidatesWithStatus, config); - final DataSourceCompactionConfig config = compactionConfigs.get(dataSourceName); - final List segmentsToCompact = entry.getSegments(); - - // Create granularitySpec to send to compaction task - Granularity segmentGranularityToUse = null; - if (config.getGranularitySpec() == null || config.getGranularitySpec().getSegmentGranularity() == null) { - // Determines segmentGranularity from the segmentsToCompact - // Each batch of segmentToCompact from CompactionSegmentIterator will contain the same interval as - // segmentGranularity is not set in the compaction config - Interval interval = segmentsToCompact.get(0).getInterval(); - if (segmentsToCompact.stream().allMatch(segment -> interval.overlaps(segment.getInterval()))) { - try { - segmentGranularityToUse = GranularityType.fromPeriod(interval.toPeriod()).getDefaultGranularity(); - } - catch (IllegalArgumentException iae) { - // This case can happen if the existing segment interval result in complicated periods. - // Fall back to setting segmentGranularity as null - LOG.warn("Cannot determine segmentGranularity from interval[%s].", interval); - } - } else { - LOG.warn( - "Not setting 'segmentGranularity' for auto-compaction task as" - + " the segments to compact do not have the same interval." - ); - } + if (compactionStatus.isComplete()) { + snapshotBuilder.addToComplete(candidatesWithStatus); + } else if (compactionStatus.isSkipped()) { + snapshotBuilder.addToSkipped(candidatesWithStatus); } else { - segmentGranularityToUse = config.getGranularitySpec().getSegmentGranularity(); + // As these segments will be compacted, we will aggregate the statistic to the Compacted statistics + snapshotBuilder.addToComplete(entry); } - final ClientCompactionTaskGranularitySpec granularitySpec = new ClientCompactionTaskGranularitySpec( - segmentGranularityToUse, - config.getGranularitySpec() != null ? config.getGranularitySpec().getQueryGranularity() : null, - config.getGranularitySpec() != null ? config.getGranularitySpec().isRollup() : null - ); - // Create dimensionsSpec to send to compaction task - final ClientCompactionTaskDimensionsSpec dimensionsSpec; - if (config.getDimensionsSpec() != null) { - dimensionsSpec = new ClientCompactionTaskDimensionsSpec( - config.getDimensionsSpec().getDimensions() - ); - } else { - dimensionsSpec = null; - } + final ClientCompactionTaskQuery taskPayload = createCompactionTask(entry, config, defaultEngine); - Boolean dropExisting = null; - if (config.getIoConfig() != null) { - dropExisting = config.getIoConfig().isDropExisting(); - } + final String taskId = taskPayload.getId(); + FutureUtils.getUnchecked(overlordClient.runTask(taskId, taskPayload), true); + statusTracker.onTaskSubmitted(taskId, entry); - // If all the segments found to be compacted are tombstones then dropExisting - // needs to be forced to true. This forcing needs to happen in the case that - // the flag is null, or it is false. It is needed when it is null to avoid the - // possibility of the code deciding to default it to false later. - // Forcing the flag to true will enable the task ingestion code to generate new, compacted, tombstones to - // cover the tombstones found to be compacted as well as to mark them - // as compacted (update their lastCompactionState). If we don't force the - // flag then every time this compact duty runs it will find the same tombstones - // in the interval since their lastCompactionState - // was not set repeating this over and over and the duty will not make progress; it - // will become stuck on this set of tombstones. - // This forcing code should be revised - // when/if the autocompaction code policy to decide which segments to compact changes - if (dropExisting == null || !dropExisting) { - if (segmentsToCompact.stream().allMatch(DataSegment::isTombstone)) { - dropExisting = true; - LOG.info("Forcing dropExisting to true since all segments to compact are tombstones."); - } - } + LOG.debug( + "Submitted a compaction task[%s] for [%d] segments in datasource[%s], umbrella interval[%s].", + taskId, entry.numSegments(), dataSourceName, entry.getUmbrellaInterval() + ); + LOG.debugSegments(entry.getSegments(), "Compacting segments"); + numSubmittedTasks++; + totalTaskSlotsAssigned += slotManager.computeSlotsRequiredForTask(taskPayload, config); + } - final CompactionEngine compactionEngine = config.getEngine() == null ? defaultEngine : config.getEngine(); - final Map autoCompactionContext = newAutoCompactionContext(config.getTaskContext()); - int slotsRequiredForCurrentTask; - - if (compactionEngine == CompactionEngine.MSQ) { - if (autoCompactionContext.containsKey(ClientMSQContext.CTX_MAX_NUM_TASKS)) { - slotsRequiredForCurrentTask = (int) autoCompactionContext.get(ClientMSQContext.CTX_MAX_NUM_TASKS); - } else { - // Since MSQ needs all task slots for the calculated #tasks to be available upfront, allot all available - // compaction slots (upto a max of MAX_TASK_SLOTS_FOR_MSQ_COMPACTION) to current compaction task to avoid - // stalling. Setting "taskAssignment" to "auto" has the problem of not being able to determine the actual - // count, which is required for subsequent tasks. - slotsRequiredForCurrentTask = Math.min( - // Update the slots to 2 (min required for MSQ) if only 1 slot is available. - numAvailableCompactionTaskSlots == 1 ? 2 : numAvailableCompactionTaskSlots, - ClientMSQContext.MAX_TASK_SLOTS_FOR_MSQ_COMPACTION_TASK - ); - autoCompactionContext.put(ClientMSQContext.CTX_MAX_NUM_TASKS, slotsRequiredForCurrentTask); + LOG.info("Submitted a total of [%d] compaction tasks.", numSubmittedTasks); + return numSubmittedTasks; + } + + /** + * Creates a {@link ClientCompactionTaskQuery} which can be submitted to an + * {@link OverlordClient} to start a compaction task. + */ + public static ClientCompactionTaskQuery createCompactionTask( + CompactionCandidate candidate, + DataSourceCompactionConfig config, + CompactionEngine defaultEngine + ) + { + final List segmentsToCompact = candidate.getSegments(); + + // Create granularitySpec to send to compaction task + Granularity segmentGranularityToUse = null; + if (config.getGranularitySpec() == null || config.getGranularitySpec().getSegmentGranularity() == null) { + // Determines segmentGranularity from the segmentsToCompact + // Each batch of segmentToCompact from CompactionSegmentIterator will contain the same interval as + // segmentGranularity is not set in the compaction config + Interval interval = segmentsToCompact.get(0).getInterval(); + if (segmentsToCompact.stream().allMatch(segment -> interval.overlaps(segment.getInterval()))) { + try { + segmentGranularityToUse = GranularityType.fromPeriod(interval.toPeriod()).getDefaultGranularity(); + } + catch (IllegalArgumentException iae) { + // This case can happen if the existing segment interval result in complicated periods. + // Fall back to setting segmentGranularity as null + LOG.warn("Cannot determine segmentGranularity from interval[%s].", interval); } } else { - slotsRequiredForCurrentTask = findMaxNumTaskSlotsUsedByOneNativeCompactionTask(config.getTuningConfig()); + LOG.warn( + "Not setting 'segmentGranularity' for auto-compaction task as" + + " the segments to compact do not have the same interval." + ); } + } else { + segmentGranularityToUse = config.getGranularitySpec().getSegmentGranularity(); + } + final ClientCompactionTaskGranularitySpec granularitySpec = new ClientCompactionTaskGranularitySpec( + segmentGranularityToUse, + config.getGranularitySpec() != null ? config.getGranularitySpec().getQueryGranularity() : null, + config.getGranularitySpec() != null ? config.getGranularitySpec().isRollup() : null + ); + + // Create dimensionsSpec to send to compaction task + final ClientCompactionTaskDimensionsSpec dimensionsSpec; + if (config.getDimensionsSpec() != null) { + dimensionsSpec = new ClientCompactionTaskDimensionsSpec( + config.getDimensionsSpec().getDimensions() + ); + } else { + dimensionsSpec = null; + } - if (entry.getCurrentStatus() != null) { - autoCompactionContext.put(COMPACTION_REASON_KEY, entry.getCurrentStatus().getReason()); + Boolean dropExisting = null; + if (config.getIoConfig() != null) { + dropExisting = config.getIoConfig().isDropExisting(); + } + + // If all the segments found to be compacted are tombstones then dropExisting + // needs to be forced to true. This forcing needs to happen in the case that + // the flag is null, or it is false. It is needed when it is null to avoid the + // possibility of the code deciding to default it to false later. + // Forcing the flag to true will enable the task ingestion code to generate new, compacted, tombstones to + // cover the tombstones found to be compacted as well as to mark them + // as compacted (update their lastCompactionState). If we don't force the + // flag then every time this compact duty runs it will find the same tombstones + // in the interval since their lastCompactionState + // was not set repeating this over and over and the duty will not make progress; it + // will become stuck on this set of tombstones. + // This forcing code should be revised + // when/if the autocompaction code policy to decide which segments to compact changes + if (dropExisting == null || !dropExisting) { + if (segmentsToCompact.stream().allMatch(DataSegment::isTombstone)) { + dropExisting = true; + LOG.info("Forcing dropExisting to true since all segments to compact are tombstones."); } + } - final String taskId = compactSegments( - entry, - config.getTaskPriority(), - ClientCompactionTaskQueryTuningConfig.from( - config.getTuningConfig(), - config.getMaxRowsPerSegment(), - config.getMetricsSpec() != null - ), - granularitySpec, - dimensionsSpec, - config.getMetricsSpec(), - config.getTransformSpec(), - config.getProjections(), - dropExisting, - autoCompactionContext, - new ClientCompactionRunnerInfo(compactionEngine) - ); + final CompactionEngine compactionEngine = config.getEngine() == null ? defaultEngine : config.getEngine(); + final Map autoCompactionContext = newAutoCompactionContext(config.getTaskContext()); - LOG.debug( - "Submitted a compaction task[%s] for [%d] segments in datasource[%s], umbrella interval[%s].", - taskId, segmentsToCompact.size(), dataSourceName, entry.getUmbrellaInterval() - ); - LOG.debugSegments(segmentsToCompact, "Compacting segments"); - numSubmittedTasks++; - totalTaskSlotsAssigned += slotsRequiredForCurrentTask; + if (candidate.getCurrentStatus() != null) { + autoCompactionContext.put(COMPACTION_REASON_KEY, candidate.getCurrentStatus().getReason()); } - LOG.info("Submitted a total of [%d] compaction tasks.", numSubmittedTasks); - return numSubmittedTasks; + return compactSegments( + candidate, + config.getTaskPriority(), + ClientCompactionTaskQueryTuningConfig.from( + config.getTuningConfig(), + config.getMaxRowsPerSegment(), + config.getMetricsSpec() != null + ), + granularitySpec, + dimensionsSpec, + config.getMetricsSpec(), + config.getTransformSpec(), + config.getProjections(), + dropExisting, + autoCompactionContext, + new ClientCompactionRunnerInfo(compactionEngine) + ); } - private Map newAutoCompactionContext(@Nullable Map configuredContext) + private static Map newAutoCompactionContext(@Nullable Map configuredContext) { final Map newContext = configuredContext == null ? new HashMap<>() @@ -607,60 +384,23 @@ private Map newAutoCompactionContext(@Nullable Map currentRunAutoCompactionSnapshotBuilders, + CompactionSnapshotBuilder snapshotBuilder, CompactionSegmentIterator iterator, - CoordinatorRunStats stats + Map datasourceToConfig ) { // Mark all the segments remaining in the iterator as "awaiting compaction" while (iterator.hasNext()) { - final CompactionCandidate entry = iterator.next(); - currentRunAutoCompactionSnapshotBuilders - .computeIfAbsent(entry.getDataSource(), AutoCompactionSnapshot::builder) - .incrementWaitingStats(entry.getStats()); + snapshotBuilder.addToPending(iterator.next()); } - - // Statistics of all segments considered compacted after this run - iterator.getCompactedSegments().forEach( - candidateSegments -> currentRunAutoCompactionSnapshotBuilders - .computeIfAbsent(candidateSegments.getDataSource(), AutoCompactionSnapshot::builder) - .incrementCompactedStats(candidateSegments.getStats()) - ); - - // Statistics of all segments considered skipped after this run - iterator.getSkippedSegments().forEach( - candidateSegments -> currentRunAutoCompactionSnapshotBuilders - .computeIfAbsent(candidateSegments.getDataSource(), AutoCompactionSnapshot::builder) - .incrementSkippedStats(candidateSegments.getStats()) - ); - - final Map currentAutoCompactionSnapshotPerDataSource = new HashMap<>(); - currentRunAutoCompactionSnapshotBuilders.forEach((dataSource, builder) -> { - final AutoCompactionSnapshot autoCompactionSnapshot = builder.build(); - currentAutoCompactionSnapshotPerDataSource.put(dataSource, autoCompactionSnapshot); - collectSnapshotStats(autoCompactionSnapshot, stats); + iterator.getCompactedSegments().forEach(snapshotBuilder::addToComplete); + iterator.getSkippedSegments().forEach(entry -> { + statusTracker.onCompactionStatusComputed(entry, datasourceToConfig.get(entry.getDataSource())); + snapshotBuilder.addToSkipped(entry); }); // Atomic update of autoCompactionSnapshotPerDataSource with the latest from this coordinator run - autoCompactionSnapshotPerDataSource.set(currentAutoCompactionSnapshotPerDataSource); - } - - private void collectSnapshotStats( - AutoCompactionSnapshot autoCompactionSnapshot, - CoordinatorRunStats stats - ) - { - final RowKey rowKey = RowKey.of(Dimension.DATASOURCE, autoCompactionSnapshot.getDataSource()); - - stats.add(Stats.Compaction.PENDING_BYTES, rowKey, autoCompactionSnapshot.getBytesAwaitingCompaction()); - stats.add(Stats.Compaction.PENDING_SEGMENTS, rowKey, autoCompactionSnapshot.getSegmentCountAwaitingCompaction()); - stats.add(Stats.Compaction.PENDING_INTERVALS, rowKey, autoCompactionSnapshot.getIntervalCountAwaitingCompaction()); - stats.add(Stats.Compaction.COMPACTED_BYTES, rowKey, autoCompactionSnapshot.getBytesCompacted()); - stats.add(Stats.Compaction.COMPACTED_SEGMENTS, rowKey, autoCompactionSnapshot.getSegmentCountCompacted()); - stats.add(Stats.Compaction.COMPACTED_INTERVALS, rowKey, autoCompactionSnapshot.getIntervalCountCompacted()); - stats.add(Stats.Compaction.SKIPPED_BYTES, rowKey, autoCompactionSnapshot.getBytesSkipped()); - stats.add(Stats.Compaction.SKIPPED_SEGMENTS, rowKey, autoCompactionSnapshot.getSegmentCountSkipped()); - stats.add(Stats.Compaction.SKIPPED_INTERVALS, rowKey, autoCompactionSnapshot.getIntervalCountSkipped()); + autoCompactionSnapshotPerDataSource.set(snapshotBuilder.build()); } @Nullable @@ -674,7 +414,7 @@ public Map getAutoCompactionSnapshot() return autoCompactionSnapshotPerDataSource.get(); } - private String compactSegments( + private static ClientCompactionTaskQuery compactSegments( CompactionCandidate entry, int compactionTaskPriority, ClientCompactionTaskQueryTuningConfig tuningConfig, @@ -703,7 +443,7 @@ private String compactSegments( final String taskId = IdUtils.newTaskId(TASK_ID_PREFIX, ClientCompactionTaskQuery.TYPE, dataSource, null); final Granularity segmentGranularity = granularitySpec == null ? null : granularitySpec.getSegmentGranularity(); - final ClientCompactionTaskQuery taskPayload = new ClientCompactionTaskQuery( + return new ClientCompactionTaskQuery( taskId, dataSource, new ClientCompactionIOConfig( @@ -719,9 +459,5 @@ private String compactSegments( context, compactionRunner ); - FutureUtils.getUnchecked(overlordClient.runTask(taskId, taskPayload), true); - statusTracker.onTaskSubmitted(taskPayload, entry); - - return taskId; } } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/stats/CoordinatorRunStats.java b/server/src/main/java/org/apache/druid/server/coordinator/stats/CoordinatorRunStats.java index 76d788cb4306..a9473b298cc9 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/stats/CoordinatorRunStats.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/stats/CoordinatorRunStats.java @@ -95,6 +95,20 @@ public long get(CoordinatorStat stat, RowKey rowKey) return statValues == null ? 0 : statValues.getLong(stat); } + /** + * Invokes the {@link StatHandler} for each value of the specified stat. + */ + public void forEachEntry(CoordinatorStat stat, StatHandler handler) + { + allStats.forEach( + (rowKey, stats) -> stats.object2LongEntrySet().fastForEach(entry -> { + if (entry.getKey().equals(stat)) { + handler.handle(stat, rowKey, entry.getLongValue()); + } + }) + ); + } + public void forEachStat(StatHandler handler) { allStats.forEach( diff --git a/server/src/main/java/org/apache/druid/server/coordinator/stats/Stats.java b/server/src/main/java/org/apache/druid/server/coordinator/stats/Stats.java index f4ae1b5f8a9e..32d4af8806ab 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/stats/Stats.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/stats/Stats.java @@ -112,6 +112,8 @@ public static class Compaction { public static final CoordinatorStat SUBMITTED_TASKS = CoordinatorStat.toDebugAndEmit("compactTasks", "compact/task/count"); + public static final CoordinatorStat CANCELLED_TASKS + = CoordinatorStat.toDebugAndEmit("compactCancelled", "compactTask/cancelled/count"); public static final CoordinatorStat MAX_SLOTS = CoordinatorStat.toDebugAndEmit("compactMaxSlots", "compactTask/maxSlot/count"); public static final CoordinatorStat AVAILABLE_SLOTS diff --git a/server/src/test/java/org/apache/druid/server/compaction/CompactionRunSimulatorTest.java b/server/src/test/java/org/apache/druid/server/compaction/CompactionRunSimulatorTest.java index fd491ef930f8..bab2e351df11 100644 --- a/server/src/test/java/org/apache/druid/server/compaction/CompactionRunSimulatorTest.java +++ b/server/src/test/java/org/apache/druid/server/compaction/CompactionRunSimulatorTest.java @@ -19,14 +19,12 @@ package org.apache.druid.server.compaction; -import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; import org.apache.druid.client.indexing.TaskPayloadResponse; import org.apache.druid.indexer.CompactionEngine; import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexer.TaskStatusPlus; -import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.CloseableIterators; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.granularity.Granularities; @@ -51,10 +49,8 @@ public class CompactionRunSimulatorTest { - private static final ObjectMapper OBJECT_MAPPER = new DefaultObjectMapper(); - private final CompactionRunSimulator simulator = new CompactionRunSimulator( - new CompactionStatusTracker(OBJECT_MAPPER), + new CompactionStatusTracker(), new TestOverlordClient() ); @@ -115,7 +111,7 @@ public void testSimulateClusterCompactionConfigUpdate() ); Assert.assertEquals( Collections.singletonList( - Arrays.asList("wiki", Intervals.of("2013-01-10/P1D"), 10, 1_000_000_000L, "skip offset from latest[P1D]") + Arrays.asList("wiki", Intervals.of("2013-01-10/P1D"), 10, 1_000_000_000L, 1, "skip offset from latest[P1D]") ), skippedTable.getRows() ); diff --git a/server/src/test/java/org/apache/druid/server/compaction/CompactionStatusTest.java b/server/src/test/java/org/apache/druid/server/compaction/CompactionStatusTest.java index a8e6977dc42c..9af152a5cc46 100644 --- a/server/src/test/java/org/apache/druid/server/compaction/CompactionStatusTest.java +++ b/server/src/test/java/org/apache/druid/server/compaction/CompactionStatusTest.java @@ -320,8 +320,7 @@ public void testStatusWhenLastCompactionStateSameAsRequired() final DataSegment segment = DataSegment.builder(WIKI_SEGMENT).lastCompactionState(lastCompactionState).build(); final CompactionStatus status = CompactionStatus.compute( CompactionCandidate.from(Collections.singletonList(segment)), - compactionConfig, - OBJECT_MAPPER + compactionConfig ); Assert.assertTrue(status.isComplete()); } @@ -370,8 +369,7 @@ public void testStatusWhenProjectionsMatch() final DataSegment segment = DataSegment.builder(WIKI_SEGMENT).lastCompactionState(lastCompactionState).build(); final CompactionStatus status = CompactionStatus.compute( CompactionCandidate.from(Collections.singletonList(segment)), - compactionConfig, - OBJECT_MAPPER + compactionConfig ); Assert.assertTrue(status.isComplete()); } @@ -425,8 +423,7 @@ public void testStatusWhenProjectionsMismatch() final DataSegment segment = DataSegment.builder(WIKI_SEGMENT).lastCompactionState(lastCompactionState).build(); final CompactionStatus status = CompactionStatus.compute( CompactionCandidate.from(Collections.singletonList(segment)), - compactionConfig, - OBJECT_MAPPER + compactionConfig ); Assert.assertFalse(status.isComplete()); } @@ -443,8 +440,7 @@ private void verifyCompactionStatusIsPendingBecause( .build(); final CompactionStatus status = CompactionStatus.compute( CompactionCandidate.from(Collections.singletonList(segment)), - compactionConfig, - OBJECT_MAPPER + compactionConfig ); Assert.assertFalse(status.isComplete()); diff --git a/server/src/test/java/org/apache/druid/server/compaction/CompactionStatusTrackerTest.java b/server/src/test/java/org/apache/druid/server/compaction/CompactionStatusTrackerTest.java index 1878aab8c0c3..40a2c501ca02 100644 --- a/server/src/test/java/org/apache/druid/server/compaction/CompactionStatusTrackerTest.java +++ b/server/src/test/java/org/apache/druid/server/compaction/CompactionStatusTrackerTest.java @@ -19,16 +19,11 @@ package org.apache.druid.server.compaction; -import com.fasterxml.jackson.databind.ObjectMapper; -import org.apache.druid.client.indexing.ClientCompactionTaskQuery; import org.apache.druid.indexer.TaskState; import org.apache.druid.indexer.TaskStatus; -import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.segment.TestDataSource; import org.apache.druid.server.coordinator.CreateDataSegments; -import org.apache.druid.server.coordinator.DataSourceCompactionConfig; -import org.apache.druid.server.coordinator.InlineSchemaDataSourceCompactionConfig; import org.apache.druid.timeline.DataSegment; import org.junit.Assert; import org.junit.Before; @@ -38,7 +33,6 @@ public class CompactionStatusTrackerTest { - private static final ObjectMapper MAPPER = new DefaultObjectMapper(); private static final DataSegment WIKI_SEGMENT = CreateDataSegments.ofDatasource(TestDataSource.WIKI).eachOfSizeInMb(100).get(0); @@ -47,7 +41,7 @@ public class CompactionStatusTrackerTest @Before public void setup() { - statusTracker = new CompactionStatusTracker(MAPPER); + statusTracker = new CompactionStatusTracker(); } @Test @@ -55,7 +49,7 @@ public void testGetLatestTaskStatusForSubmittedTask() { final CompactionCandidate candidateSegments = CompactionCandidate.from(Collections.singletonList(WIKI_SEGMENT)); - statusTracker.onTaskSubmitted(createCompactionTask("task1"), candidateSegments); + statusTracker.onTaskSubmitted("task1", candidateSegments); CompactionTaskStatus status = statusTracker.getLatestTaskStatus(candidateSegments); Assert.assertEquals(TaskState.RUNNING, status.getState()); @@ -66,7 +60,7 @@ public void testGetLatestTaskStatusForSuccessfulTask() { final CompactionCandidate candidateSegments = CompactionCandidate.from(Collections.singletonList(WIKI_SEGMENT)); - statusTracker.onTaskSubmitted(createCompactionTask("task1"), candidateSegments); + statusTracker.onTaskSubmitted("task1", candidateSegments); statusTracker.onTaskFinished("task1", TaskStatus.success("task1")); CompactionTaskStatus status = statusTracker.getLatestTaskStatus(candidateSegments); @@ -78,7 +72,7 @@ public void testGetLatestTaskStatusForFailedTask() { final CompactionCandidate candidateSegments = CompactionCandidate.from(Collections.singletonList(WIKI_SEGMENT)); - statusTracker.onTaskSubmitted(createCompactionTask("task1"), candidateSegments); + statusTracker.onTaskSubmitted("task1", candidateSegments); statusTracker.onTaskFinished("task1", TaskStatus.failure("task1", "some failure")); CompactionTaskStatus status = statusTracker.getLatestTaskStatus(candidateSegments); @@ -92,10 +86,10 @@ public void testGetLatestTaskStatusForRepeatedlyFailingTask() final CompactionCandidate candidateSegments = CompactionCandidate.from(Collections.singletonList(WIKI_SEGMENT)); - statusTracker.onTaskSubmitted(createCompactionTask("task1"), candidateSegments); + statusTracker.onTaskSubmitted("task1", candidateSegments); statusTracker.onTaskFinished("task1", TaskStatus.failure("task1", "some failure")); - statusTracker.onTaskSubmitted(createCompactionTask("task2"), candidateSegments); + statusTracker.onTaskSubmitted("task2", candidateSegments); CompactionTaskStatus status = statusTracker.getLatestTaskStatus(candidateSegments); Assert.assertEquals(TaskState.RUNNING, status.getState()); Assert.assertEquals(1, status.getNumConsecutiveFailures()); @@ -110,24 +104,22 @@ public void testGetLatestTaskStatusForRepeatedlyFailingTask() @Test public void testComputeCompactionStatusForSuccessfulTask() { - final DataSourceCompactionConfig compactionConfig - = InlineSchemaDataSourceCompactionConfig.builder().forDataSource(TestDataSource.WIKI).build(); final NewestSegmentFirstPolicy policy = new NewestSegmentFirstPolicy(null); final CompactionCandidate candidateSegments = CompactionCandidate.from(Collections.singletonList(WIKI_SEGMENT)); // Verify that interval is originally eligible for compaction CompactionStatus status - = statusTracker.computeCompactionStatus(candidateSegments, compactionConfig, policy); + = statusTracker.computeCompactionStatus(candidateSegments, policy); Assert.assertEquals(CompactionStatus.State.PENDING, status.getState()); Assert.assertEquals("not compacted yet", status.getReason()); // Verify that interval is skipped for compaction after task has finished statusTracker.onSegmentTimelineUpdated(DateTimes.nowUtc().minusMinutes(1)); - statusTracker.onTaskSubmitted(createCompactionTask("task1"), candidateSegments); + statusTracker.onTaskSubmitted("task1", candidateSegments); statusTracker.onTaskFinished("task1", TaskStatus.success("task1")); - status = statusTracker.computeCompactionStatus(candidateSegments, compactionConfig, policy); + status = statusTracker.computeCompactionStatus(candidateSegments, policy); Assert.assertEquals(CompactionStatus.State.SKIPPED, status.getState()); Assert.assertEquals( "Segment timeline not updated since last compaction task succeeded", @@ -136,26 +128,7 @@ public void testComputeCompactionStatusForSuccessfulTask() // Verify that interval becomes eligible again after timeline has been updated statusTracker.onSegmentTimelineUpdated(DateTimes.nowUtc()); - status = statusTracker.computeCompactionStatus(candidateSegments, compactionConfig, policy); + status = statusTracker.computeCompactionStatus(candidateSegments, policy); Assert.assertEquals(CompactionStatus.State.PENDING, status.getState()); } - - private ClientCompactionTaskQuery createCompactionTask( - String taskId - ) - { - return new ClientCompactionTaskQuery( - taskId, - TestDataSource.WIKI, - null, - null, - null, - null, - null, - null, - null, - null, - null - ); - } } diff --git a/server/src/test/java/org/apache/druid/server/compaction/NewestSegmentFirstPolicyTest.java b/server/src/test/java/org/apache/druid/server/compaction/NewestSegmentFirstPolicyTest.java index 5cba8cc17c22..dd48bd2aba19 100644 --- a/server/src/test/java/org/apache/druid/server/compaction/NewestSegmentFirstPolicyTest.java +++ b/server/src/test/java/org/apache/druid/server/compaction/NewestSegmentFirstPolicyTest.java @@ -65,7 +65,6 @@ import org.joda.time.Interval; import org.joda.time.Period; import org.junit.Assert; -import org.junit.Before; import org.junit.Test; import java.util.ArrayList; @@ -82,13 +81,6 @@ public class NewestSegmentFirstPolicyTest private static final int DEFAULT_NUM_SEGMENTS_PER_SHARD = 4; private final ObjectMapper mapper = new DefaultObjectMapper(); private final NewestSegmentFirstPolicy policy = new NewestSegmentFirstPolicy(null); - private CompactionStatusTracker statusTracker; - - @Before - public void setup() - { - statusTracker = new CompactionStatusTracker(mapper); - } @Test public void testLargeOffsetAndSmallSegmentInterval() @@ -284,8 +276,7 @@ public void testSkipDataSourceWithNoSegments() .withNumPartitions(4) ) ), - Collections.emptyMap(), - statusTracker + Collections.emptyMap() ); assertCompactSegmentIntervals( @@ -517,8 +508,7 @@ public void testWithSkipIntervals() Intervals.of("2017-11-15T00:00:00/2017-11-15T20:00:00"), Intervals.of("2017-11-13T00:00:00/2017-11-14T01:00:00") ) - ), - statusTracker + ) ); assertCompactSegmentIntervals( @@ -557,8 +547,7 @@ public void testHoleInSearchInterval() Intervals.of("2017-11-16T04:00:00/2017-11-16T10:00:00"), Intervals.of("2017-11-16T14:00:00/2017-11-16T20:00:00") ) - ), - statusTracker + ) ); assertCompactSegmentIntervals( @@ -2063,8 +2052,7 @@ TestDataSource.KOALA, configBuilder().forDataSource(TestDataSource.KOALA).build( TestDataSource.WIKI, SegmentTimeline.forSegments(wikiSegments), TestDataSource.KOALA, SegmentTimeline.forSegments(koalaSegments) ), - Collections.emptyMap(), - statusTracker + Collections.emptyMap() ); // Verify that the segments of WIKI are preferred even though they are older @@ -2085,8 +2073,7 @@ private CompactionSegmentIterator createIterator(DataSourceCompactionConfig conf policy, Collections.singletonMap(TestDataSource.WIKI, config), Collections.singletonMap(TestDataSource.WIKI, timeline), - Collections.emptyMap(), - statusTracker + Collections.emptyMap() ); } diff --git a/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorTest.java b/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorTest.java index f77da3140a4b..39213ff8616d 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorTest.java @@ -139,8 +139,7 @@ public void setUp() throws Exception ) ).andReturn(new AtomicReference<>(DruidCompactionConfig.empty())).anyTimes(); EasyMock.replay(configManager); - final ObjectMapper objectMapper = new DefaultObjectMapper(); - statusTracker = new CompactionStatusTracker(objectMapper); + statusTracker = new CompactionStatusTracker(); druidCoordinatorConfig = new DruidCoordinatorConfig( new CoordinatorRunConfig(new Duration(COORDINATOR_START_DELAY), new Duration(COORDINATOR_PERIOD)), new CoordinatorPeriodConfig(null, null), @@ -169,7 +168,7 @@ public void setUp() throws Exception new TestDruidLeaderSelector(), null, CentralizedDatasourceSchemaConfig.create(), - new CompactionStatusTracker(OBJECT_MAPPER), + new CompactionStatusTracker(), EasyMock.niceMock(CoordinatorDynamicConfigSyncer.class), EasyMock.niceMock(CloneStatusManager.class) ); @@ -481,7 +480,7 @@ public void testCompactSegmentsDutyWhenCustomDutyGroupEmpty() new TestDruidLeaderSelector(), null, CentralizedDatasourceSchemaConfig.create(), - new CompactionStatusTracker(OBJECT_MAPPER), + new CompactionStatusTracker(), EasyMock.niceMock(CoordinatorDynamicConfigSyncer.class), EasyMock.niceMock(CloneStatusManager.class) ); @@ -533,7 +532,7 @@ public void testInitializeCompactSegmentsDutyWhenCustomDutyGroupDoesNotContainsC new TestDruidLeaderSelector(), null, CentralizedDatasourceSchemaConfig.create(), - new CompactionStatusTracker(OBJECT_MAPPER), + new CompactionStatusTracker(), EasyMock.niceMock(CoordinatorDynamicConfigSyncer.class), EasyMock.niceMock(CloneStatusManager.class) ); @@ -585,7 +584,7 @@ public void testInitializeCompactSegmentsDutyWhenCustomDutyGroupContainsCompactS new TestDruidLeaderSelector(), null, CentralizedDatasourceSchemaConfig.create(), - new CompactionStatusTracker(OBJECT_MAPPER), + new CompactionStatusTracker(), EasyMock.niceMock(CoordinatorDynamicConfigSyncer.class), EasyMock.niceMock(CloneStatusManager.class) ); @@ -695,7 +694,7 @@ public void testCoordinatorCustomDutyGroupsRunAsExpected() throws Exception new TestDruidLeaderSelector(), null, CentralizedDatasourceSchemaConfig.create(), - new CompactionStatusTracker(OBJECT_MAPPER), + new CompactionStatusTracker(), EasyMock.niceMock(CoordinatorDynamicConfigSyncer.class), EasyMock.niceMock(CloneStatusManager.class) ); 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 0d0022896066..d7bbb947d9dc 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 @@ -81,6 +81,7 @@ import org.apache.druid.segment.indexing.BatchIOConfig; import org.apache.druid.segment.transform.CompactionTransformSpec; import org.apache.druid.server.compaction.CompactionCandidateSearchPolicy; +import org.apache.druid.server.compaction.CompactionSlotManager; import org.apache.druid.server.compaction.CompactionStatusTracker; import org.apache.druid.server.compaction.FixedIntervalOrderPolicy; import org.apache.druid.server.compaction.NewestSegmentFirstPolicy; @@ -226,7 +227,7 @@ public void setup() } } dataSources = DataSourcesSnapshot.fromUsedSegments(allSegments); - statusTracker = new CompactionStatusTracker(JSON_MAPPER); + statusTracker = new CompactionStatusTracker(); policy = new NewestSegmentFirstPolicy(null); } @@ -2004,7 +2005,7 @@ public static class StaticUtilsTest @Test public void testIsParalleModeNullTuningConfigReturnFalse() { - Assert.assertFalse(CompactSegments.isParallelMode(null)); + Assert.assertFalse(CompactionSlotManager.isParallelMode(null)); } @Test @@ -2012,7 +2013,7 @@ public void testIsParallelModeNullPartitionsSpecReturnFalse() { ClientCompactionTaskQueryTuningConfig tuningConfig = Mockito.mock(ClientCompactionTaskQueryTuningConfig.class); Mockito.when(tuningConfig.getPartitionsSpec()).thenReturn(null); - Assert.assertFalse(CompactSegments.isParallelMode(tuningConfig)); + Assert.assertFalse(CompactionSlotManager.isParallelMode(tuningConfig)); } @Test @@ -2022,13 +2023,13 @@ public void testIsParallelModeNonRangePartitionVaryingMaxNumConcurrentSubTasks() Mockito.when(tuningConfig.getPartitionsSpec()).thenReturn(Mockito.mock(PartitionsSpec.class)); Mockito.when(tuningConfig.getMaxNumConcurrentSubTasks()).thenReturn(null); - Assert.assertFalse(CompactSegments.isParallelMode(tuningConfig)); + Assert.assertFalse(CompactionSlotManager.isParallelMode(tuningConfig)); Mockito.when(tuningConfig.getMaxNumConcurrentSubTasks()).thenReturn(1); - Assert.assertFalse(CompactSegments.isParallelMode(tuningConfig)); + Assert.assertFalse(CompactionSlotManager.isParallelMode(tuningConfig)); Mockito.when(tuningConfig.getMaxNumConcurrentSubTasks()).thenReturn(2); - Assert.assertTrue(CompactSegments.isParallelMode(tuningConfig)); + Assert.assertTrue(CompactionSlotManager.isParallelMode(tuningConfig)); } @Test @@ -2038,13 +2039,13 @@ public void testIsParallelModeRangePartitionVaryingMaxNumConcurrentSubTasks() Mockito.when(tuningConfig.getPartitionsSpec()).thenReturn(Mockito.mock(SingleDimensionPartitionsSpec.class)); Mockito.when(tuningConfig.getMaxNumConcurrentSubTasks()).thenReturn(null); - Assert.assertFalse(CompactSegments.isParallelMode(tuningConfig)); + Assert.assertFalse(CompactionSlotManager.isParallelMode(tuningConfig)); Mockito.when(tuningConfig.getMaxNumConcurrentSubTasks()).thenReturn(1); - Assert.assertTrue(CompactSegments.isParallelMode(tuningConfig)); + Assert.assertTrue(CompactionSlotManager.isParallelMode(tuningConfig)); Mockito.when(tuningConfig.getMaxNumConcurrentSubTasks()).thenReturn(2); - Assert.assertTrue(CompactSegments.isParallelMode(tuningConfig)); + Assert.assertTrue(CompactionSlotManager.isParallelMode(tuningConfig)); } @Test @@ -2053,7 +2054,7 @@ public void testFindMaxNumTaskSlotsUsedByOneCompactionTaskWhenIsParallelMode() ClientCompactionTaskQueryTuningConfig tuningConfig = Mockito.mock(ClientCompactionTaskQueryTuningConfig.class); Mockito.when(tuningConfig.getPartitionsSpec()).thenReturn(Mockito.mock(PartitionsSpec.class)); Mockito.when(tuningConfig.getMaxNumConcurrentSubTasks()).thenReturn(2); - Assert.assertEquals(3, CompactSegments.findMaxNumTaskSlotsUsedByOneNativeCompactionTask(tuningConfig)); + Assert.assertEquals(3, CompactionSlotManager.getMaxTaskSlotsForNativeCompactionTask(tuningConfig)); } @Test @@ -2062,7 +2063,7 @@ public void testFindMaxNumTaskSlotsUsedByOneCompactionTaskWhenIsSequentialMode() ClientCompactionTaskQueryTuningConfig tuningConfig = Mockito.mock(ClientCompactionTaskQueryTuningConfig.class); Mockito.when(tuningConfig.getPartitionsSpec()).thenReturn(Mockito.mock(PartitionsSpec.class)); Mockito.when(tuningConfig.getMaxNumConcurrentSubTasks()).thenReturn(1); - Assert.assertEquals(1, CompactSegments.findMaxNumTaskSlotsUsedByOneNativeCompactionTask(tuningConfig)); + Assert.assertEquals(1, CompactionSlotManager.getMaxTaskSlotsForNativeCompactionTask(tuningConfig)); } } diff --git a/server/src/test/java/org/apache/druid/server/coordinator/simulate/CoordinatorSimulationBuilder.java b/server/src/test/java/org/apache/druid/server/coordinator/simulate/CoordinatorSimulationBuilder.java index 8d6349237531..0838b41402d5 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/simulate/CoordinatorSimulationBuilder.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/simulate/CoordinatorSimulationBuilder.java @@ -223,7 +223,7 @@ public CoordinatorSimulation build() env.leaderSelector, null, CentralizedDatasourceSchemaConfig.create(), - new CompactionStatusTracker(OBJECT_MAPPER), + new CompactionStatusTracker(), env.configSyncer, env.cloneStatusManager ); From ce0b28fdeaacfe996bab8e4d7f589df1c43a2fe2 Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Thu, 14 Aug 2025 12:25:43 +0530 Subject: [PATCH 02/25] Remove TODOs --- .../CompactionConfigBasedJobTemplate.java | 1 - .../indexing/compact/CompactionJobQueue.java | 16 ---------------- 2 files changed, 17 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionConfigBasedJobTemplate.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionConfigBasedJobTemplate.java index 76549f9cf7b5..fccc22c1ae65 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 @@ -80,7 +80,6 @@ public List createCompactionJobs( while (segmentIterator.hasNext()) { final CompactionCandidate candidate = segmentIterator.next(); - // TODO: choose the right engine here ClientCompactionTaskQuery taskPayload = CompactSegments.createCompactionTask(candidate, config, CompactionEngine.NATIVE); final Interval compactionInterval = taskPayload.getIoConfig().getInputSpec().getInterval(); 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 1675829109bd..80b1438369be 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 @@ -53,19 +53,6 @@ * Iterates over all eligible compaction jobs in order of their priority. * A fresh instance of this class must be used in every run of the * {@link CompactionScheduler}. - * - * TODO: Remaining items: - * - fill timeline gaps, support realiging intervals - * - cancel mismatching task - * - pass in the engine to the template - * - MSQ template - * - invoke onTimelineUpdated - timeline will now get updated very frequently, - * - we don't want to recompact intervals, try to find the right thing to do. - * - we might have to do it via the policy - * - maybe use searchInterval instead of skipIntervals - * - how does this whole thing affect queuedIntervals - * - for duty, it doesn't matter - * - for supervisors, intervals will always be mutually exclusive */ public class CompactionJobQueue { @@ -164,9 +151,6 @@ public void runReadyJobs() runStats.add(Stats.Compaction.SUBMITTED_TASKS, RowKey.of(Dimension.DATASOURCE, task.getDataSource()), 1); } } - - // TODO: Add the skipped and the already compacted stuff determined by the DatasourceCompactibleSegmentIterator - // to the stats } /** From fa566ce1e373fa6303dcd5591e267d269f87dcf2 Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Thu, 14 Aug 2025 13:47:16 +0530 Subject: [PATCH 03/25] Fix up tests --- .../CompactionConfigBasedJobTemplate.java | 3 +- .../indexing/compact/CompactionJobParams.java | 12 +-- .../indexing/compact/CompactionJobQueue.java | 21 ++++-- .../compact/CompactionSupervisor.java | 49 ++++++------ .../compact/CompactionSupervisorSpec.java | 8 +- .../compact/DruidInputSourceFactory.java | 74 +++++++++++++++++++ .../compact/OverlordCompactionScheduler.java | 17 ++++- .../supervisor/BatchIndexingSupervisor.java | 48 ------------ .../BatchIndexingSupervisorSpec.java | 41 ---------- .../OverlordCompactionSchedulerTest.java | 13 ++-- .../druid/java/util/common/Intervals.java | 6 +- .../druid/java/util/common/IntervalsTest.java | 33 +++++++++ .../org/apache/druid/cli/CliOverlord.java | 2 + 13 files changed, 185 insertions(+), 142 deletions(-) create mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/compact/DruidInputSourceFactory.java delete mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/BatchIndexingSupervisor.java delete mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/BatchIndexingSupervisorSpec.java 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 fccc22c1ae65..913960832b61 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 @@ -23,7 +23,6 @@ import org.apache.druid.data.input.InputSource; import org.apache.druid.data.output.OutputDestination; import org.apache.druid.error.InvalidInput; -import org.apache.druid.indexer.CompactionEngine; import org.apache.druid.indexing.input.DruidDatasourceDestination; import org.apache.druid.indexing.input.DruidInputSource; import org.apache.druid.java.util.common.Intervals; @@ -81,7 +80,7 @@ public List createCompactionJobs( final CompactionCandidate candidate = segmentIterator.next(); ClientCompactionTaskQuery taskPayload - = CompactSegments.createCompactionTask(candidate, config, CompactionEngine.NATIVE); + = CompactSegments.createCompactionTask(candidate, config, params.getClusterCompactionConfig().getEngine()); final Interval compactionInterval = taskPayload.getIoConfig().getInputSpec().getInterval(); jobs.add( new CompactionJob( 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 907e841094e4..588ed5aded06 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 @@ -19,8 +19,8 @@ package org.apache.druid.indexing.compact; -import com.fasterxml.jackson.databind.ObjectMapper; import org.apache.druid.indexing.template.JobParams; +import org.apache.druid.server.coordinator.ClusterCompactionConfig; import org.apache.druid.timeline.SegmentTimeline; import org.joda.time.DateTime; @@ -30,17 +30,17 @@ public class CompactionJobParams implements JobParams { private final DateTime scheduleStartTime; - private final ObjectMapper mapper; private final TimelineProvider timelineProvider; + private final ClusterCompactionConfig clusterCompactionConfig; public CompactionJobParams( DateTime scheduleStartTime, - ObjectMapper mapper, + ClusterCompactionConfig clusterCompactionConfig, TimelineProvider timelineProvider ) { - this.mapper = mapper; this.scheduleStartTime = scheduleStartTime; + this.clusterCompactionConfig = clusterCompactionConfig; this.timelineProvider = timelineProvider; } @@ -50,9 +50,9 @@ public DateTime getScheduleStartTime() return scheduleStartTime; } - public ObjectMapper getMapper() + public ClusterCompactionConfig getClusterCompactionConfig() { - return mapper; + return clusterCompactionConfig; } public SegmentTimeline getTimeline(String dataSource) 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 80b1438369be..de2b88984b5a 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 @@ -26,6 +26,8 @@ import org.apache.druid.common.guava.FutureUtils; import org.apache.druid.indexing.common.actions.TaskActionClientFactory; import org.apache.druid.indexing.common.task.Task; +import org.apache.druid.indexing.input.DruidDatasourceDestination; +import org.apache.druid.indexing.input.DruidInputSource; import org.apache.druid.indexing.overlord.GlobalTaskLockbox; import org.apache.druid.indexing.template.BatchIndexingJob; import org.apache.druid.java.util.common.DateTimes; @@ -89,7 +91,7 @@ public CompactionJobQueue( ); this.jobParams = new CompactionJobParams( DateTimes.nowUtc(), - objectMapper, + clusterCompactionConfig, dataSourcesSnapshot.getUsedSegmentsTimelinesPerDataSource()::get ); this.slotManager = new CompactionSlotManager( @@ -121,12 +123,16 @@ public void add(CompactionJob job) * Creates jobs for the given {@link CompactionSupervisor} and adds them to * the job queue. */ - public void createAndEnqueueJobs(CompactionSupervisor supervisor) + public void createAndEnqueueJobs( + CompactionSupervisor supervisor, + DruidInputSource source, + DruidDatasourceDestination destination + ) { final String supervisorId = supervisor.getSpec().getId(); try { - if (supervisor.shouldCreateJobs(jobParams)) { - queue.addAll(supervisor.createJobs(jobParams)); + if (supervisor.shouldCreateJobs()) { + queue.addAll(supervisor.createJobs(source, destination, jobParams)); } else { log.debug("Skipping job creation for supervisor[%s]", supervisorId); } @@ -202,6 +208,8 @@ private boolean startJobIfPendingAndReady(CompactionJob job, CompactionCandidate case SKIPPED: snapshotBuilder.addToSkipped(candidate); return false; + default: + break; } // Check if enough compaction task slots are available @@ -212,11 +220,12 @@ private boolean startJobIfPendingAndReady(CompactionJob job, CompactionCandidate // Reserve task slots and try to start the task slotManager.reserveTaskSlots(job.getMaxRequiredTaskSlots()); - if(startTaskIfReady(job)) { + if (startTaskIfReady(job)) { snapshotBuilder.addToComplete(candidate); return true; } else { - snapshotBuilder.addToPending(candidate); + // Mark the job as skipped for now as the intervals might be locked by other tasks + snapshotBuilder.addToSkipped(candidate); return false; } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionSupervisor.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionSupervisor.java index a20620b2845f..afe9341196b2 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionSupervisor.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionSupervisor.java @@ -22,24 +22,21 @@ import org.apache.druid.indexing.input.DruidDatasourceDestination; import org.apache.druid.indexing.input.DruidInputSource; import org.apache.druid.indexing.overlord.DataSourceMetadata; -import org.apache.druid.indexing.overlord.supervisor.BatchIndexingSupervisor; +import org.apache.druid.indexing.overlord.supervisor.Supervisor; import org.apache.druid.indexing.overlord.supervisor.SupervisorReport; import org.apache.druid.indexing.overlord.supervisor.SupervisorStateManager; import org.apache.druid.java.util.common.DateTimes; -import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.server.coordinator.AutoCompactionSnapshot; -import org.joda.time.Interval; import javax.annotation.Nullable; import java.util.List; -import java.util.Map; /** * Supervisor for compaction of a single datasource. */ -public class CompactionSupervisor implements BatchIndexingSupervisor +public class CompactionSupervisor implements Supervisor { private static final Logger log = new Logger(CompactionSupervisor.class); @@ -62,6 +59,27 @@ public CompactionSupervisorSpec getSpec() return supervisorSpec; } + /** + * Checks if this supervisor is ready to create jobs in the current run of the + * scheduler. + */ + public boolean shouldCreateJobs() + { + return !supervisorSpec.isSuspended(); + } + + /** + * Creates compaction jobs for this supervisor. + */ + public List createJobs( + DruidInputSource inputSource, + DruidDatasourceDestination destination, + CompactionJobParams jobParams + ) + { + return supervisorSpec.getTemplate().createCompactionJobs(inputSource, destination, jobParams); + } + @Override public void start() { @@ -130,27 +148,6 @@ public void reset(@Nullable DataSourceMetadata dataSourceMetadata) // do nothing } - @Override - public boolean shouldCreateJobs(CompactionJobParams jobParams) - { - return !supervisorSpec.isSuspended(); - } - - @Override - public List createJobs(CompactionJobParams jobParams) - { - final Interval interval = Intervals.ETERNITY; - return supervisorSpec.getTemplate().createCompactionJobs( - // Create a DruidInputSource for this datasource - jobParams.getMapper().convertValue( - Map.of("type", "druid", "dataSource", dataSource, "interval", interval), - DruidInputSource.class - ), - new DruidDatasourceDestination(dataSource), - jobParams - ); - } - public enum State implements SupervisorStateManager.State { SCHEDULER_STOPPED(true), diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionSupervisorSpec.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionSupervisorSpec.java index cf5fab87d651..998d7e0c7e67 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionSupervisorSpec.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionSupervisorSpec.java @@ -23,7 +23,7 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import org.apache.druid.common.config.Configs; -import org.apache.druid.indexing.overlord.supervisor.BatchIndexingSupervisorSpec; +import org.apache.druid.indexing.overlord.supervisor.SupervisorSpec; import org.apache.druid.server.coordinator.CompactionConfigValidationResult; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; import org.apache.druid.server.security.ResourceAction; @@ -35,7 +35,7 @@ import java.util.Objects; import java.util.Set; -public class CompactionSupervisorSpec implements BatchIndexingSupervisorSpec +public class CompactionSupervisorSpec implements SupervisorSpec { public static final String TYPE = "autocompact"; public static final String ID_PREFIX = "autocompact__"; @@ -93,7 +93,9 @@ public CompactionSupervisor createSupervisor() return new CompactionSupervisor(this, scheduler); } - @Override + /** + * @return {@link CompactionJobTemplate} used to create jobs for the supervisor. + */ public CompactionJobTemplate getTemplate() { if (spec instanceof CascadingCompactionTemplate) { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/DruidInputSourceFactory.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/DruidInputSourceFactory.java new file mode 100644 index 000000000000..408914640447 --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/DruidInputSourceFactory.java @@ -0,0 +1,74 @@ +/* + * 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.compact; + +import com.google.inject.Inject; +import org.apache.druid.client.coordinator.CoordinatorClient; +import org.apache.druid.indexing.common.SegmentCacheManagerFactory; +import org.apache.druid.indexing.common.config.TaskConfig; +import org.apache.druid.indexing.input.DruidInputSource; +import org.apache.druid.segment.IndexIO; +import org.joda.time.Interval; + +/** + * Factory for creating {@link DruidInputSource} for a given datasource and + * interval used by {@link CompactionJobTemplate}. + */ +public class DruidInputSourceFactory +{ + private final IndexIO indexIO; + private final TaskConfig taskConfig; + private final CoordinatorClient coordinatorClient; + private final SegmentCacheManagerFactory segmentCacheManagerFactory; + + @Inject + public DruidInputSourceFactory( + IndexIO indexIO, + TaskConfig taskConfig, + CoordinatorClient coordinatorClient, + SegmentCacheManagerFactory segmentCacheManagerFactory + ) + { + this.indexIO = indexIO; + this.coordinatorClient = coordinatorClient; + this.segmentCacheManagerFactory = segmentCacheManagerFactory; + this.taskConfig = taskConfig; + } + + /** + * Creates a new {@link DruidInputSource} for the given {@code dataSource} and + * {@code interval}. + */ + public DruidInputSource create(String dataSource, Interval interval) + { + return new DruidInputSource( + dataSource, + interval, + null, + null, + null, + null, + indexIO, + coordinatorClient, + segmentCacheManagerFactory, + taskConfig + ); + } +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/OverlordCompactionScheduler.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/OverlordCompactionScheduler.java index 9896db489264..6860214a6729 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/compact/OverlordCompactionScheduler.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/OverlordCompactionScheduler.java @@ -28,11 +28,13 @@ import org.apache.druid.indexer.TaskLocation; import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexing.common.actions.TaskActionClientFactory; +import org.apache.druid.indexing.input.DruidDatasourceDestination; import org.apache.druid.indexing.overlord.GlobalTaskLockbox; import org.apache.druid.indexing.overlord.TaskMaster; import org.apache.druid.indexing.overlord.TaskQueryTool; import org.apache.druid.indexing.overlord.TaskRunner; import org.apache.druid.indexing.overlord.TaskRunnerListener; +import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.Stopwatch; import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.common.concurrent.ScheduledExecutorFactory; @@ -106,6 +108,7 @@ public class OverlordCompactionScheduler implements CompactionScheduler private final CompactionStatusTracker statusTracker; private final TaskActionClientFactory taskActionClientFactory; + private final DruidInputSourceFactory druidInputSourceFactory; private final GlobalTaskLockbox taskLockbox; /** @@ -136,6 +139,7 @@ public OverlordCompactionScheduler( CompactionStatusTracker statusTracker, CoordinatorOverlordServiceConfig coordinatorOverlordServiceConfig, TaskActionClientFactory taskActionClientFactory, + DruidInputSourceFactory druidInputSourceFactory, ScheduledExecutorFactory executorFactory, ServiceEmitter emitter, ObjectMapper objectMapper @@ -157,6 +161,7 @@ public OverlordCompactionScheduler( this.datasourceToCompactionSnapshot = new AtomicReference<>(); this.taskActionClientFactory = taskActionClientFactory; + this.druidInputSourceFactory = druidInputSourceFactory; this.taskRunnerListener = new TaskRunnerListener() { @Override @@ -321,8 +326,9 @@ private synchronized void scheduledRun() */ private synchronized void runCompactionDuty() { + final DataSourcesSnapshot dataSourcesSnapshot = getDatasourceSnapshot(); final CompactionJobQueue queue = new CompactionJobQueue( - getDatasourceSnapshot(), + dataSourcesSnapshot, getLatestClusterConfig(), statusTracker, taskActionClientFactory, @@ -331,7 +337,14 @@ private synchronized void runCompactionDuty() objectMapper ); statusTracker.resetActiveDatasources(activeSupervisors.keySet()); - activeSupervisors.forEach((datasource, supervisor) -> queue.createAndEnqueueJobs(supervisor)); + statusTracker.onSegmentTimelineUpdated(dataSourcesSnapshot.getSnapshotTime()); + activeSupervisors.forEach( + (datasource, supervisor) -> queue.createAndEnqueueJobs( + supervisor, + druidInputSourceFactory.create(datasource, Intervals.ETERNITY), + new DruidDatasourceDestination(datasource) + ) + ); queue.runReadyJobs(); datasourceToCompactionSnapshot.set(queue.getCompactionSnapshots()); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/BatchIndexingSupervisor.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/BatchIndexingSupervisor.java deleted file mode 100644 index c26c27fc1038..000000000000 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/BatchIndexingSupervisor.java +++ /dev/null @@ -1,48 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.druid.indexing.overlord.supervisor; - -import org.apache.druid.indexing.template.BatchIndexingJob; -import org.apache.druid.indexing.template.JobParams; - -import java.util.List; - -/** - * Supervisor to perform batch ingestion using {@link BatchIndexingJob}. - */ -public interface BatchIndexingSupervisor - extends Supervisor -{ - /** - * Checks if this supervisor is ready to create jobs in the current run. - * - * @param jobParams Parameters for the current run of the scheduler. - */ - boolean shouldCreateJobs(P jobParams); - - /** - * Creates jobs to be launched in the current run of the scheduler. - * - * @param jobParams Parameters for the current run of the scheduler. - * @return Empty iterator if no tasks are to be submitted in the current run - * of the scheduler. - */ - List createJobs(P jobParams); -} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/BatchIndexingSupervisorSpec.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/BatchIndexingSupervisorSpec.java deleted file mode 100644 index 30e2f2b5109c..000000000000 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/BatchIndexingSupervisorSpec.java +++ /dev/null @@ -1,41 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.druid.indexing.overlord.supervisor; - -import org.apache.druid.indexing.template.BatchIndexingJob; -import org.apache.druid.indexing.template.BatchIndexingJobTemplate; -import org.apache.druid.indexing.template.JobParams; - -/** - * Spec for {@link BatchIndexingSupervisor}. Provides a template to create - * {@link BatchIndexingJob}. - */ -public interface BatchIndexingSupervisorSpec - - extends SupervisorSpec -{ - @Override - BatchIndexingSupervisor createSupervisor(); - - /** - * Template used by the corresponding supervisor to create {@link BatchIndexingJob}s. - */ - BatchIndexingJobTemplate getTemplate(); -} diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/compact/OverlordCompactionSchedulerTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/compact/OverlordCompactionSchedulerTest.java index 86bbf3dff614..7ab01ced15e1 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/compact/OverlordCompactionSchedulerTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/compact/OverlordCompactionSchedulerTest.java @@ -21,10 +21,8 @@ import com.fasterxml.jackson.databind.InjectableValues; import com.fasterxml.jackson.databind.ObjectMapper; -import org.apache.druid.client.coordinator.CoordinatorClient; import org.apache.druid.client.coordinator.NoopCoordinatorClient; import org.apache.druid.client.indexing.ClientMSQContext; -import org.apache.druid.guice.IndexingServiceInputSourceModule; import org.apache.druid.guice.IndexingServiceTuningConfigModule; import org.apache.druid.indexer.CompactionEngine; import org.apache.druid.indexing.common.SegmentCacheManagerFactory; @@ -52,7 +50,6 @@ import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.metrics.StubServiceEmitter; -import org.apache.druid.segment.IndexIO; import org.apache.druid.segment.TestDataSource; import org.apache.druid.segment.TestIndex; import org.apache.druid.server.compaction.CompactionSimulateResult; @@ -93,13 +90,9 @@ public class OverlordCompactionSchedulerTest static { OBJECT_MAPPER = new DefaultObjectMapper(); OBJECT_MAPPER.registerModules(new IndexingServiceTuningConfigModule().getJacksonModules()); - OBJECT_MAPPER.registerModules(new IndexingServiceInputSourceModule().getJacksonModules()); OBJECT_MAPPER.setInjectableValues( new InjectableValues .Std() - .addValue(IndexIO.class, TestIndex.INDEX_IO) - .addValue(TaskConfig.class, Mockito.mock(TaskConfig.class)) - .addValue(CoordinatorClient.class, new NoopCoordinatorClient()) .addValue( SegmentCacheManagerFactory.class, new SegmentCacheManagerFactory(TestIndex.INDEX_IO, OBJECT_MAPPER) @@ -190,6 +183,12 @@ private void initScheduler() new CompactionStatusTracker(), coordinatorOverlordServiceConfig, taskActionClientFactory, + new DruidInputSourceFactory( + TestIndex.INDEX_IO, + Mockito.mock(TaskConfig.class), + new NoopCoordinatorClient(), + new SegmentCacheManagerFactory(TestIndex.INDEX_IO, OBJECT_MAPPER) + ), (nameFormat, numThreads) -> new WrappingScheduledExecutorService("test", executor, false), serviceEmitter, OBJECT_MAPPER diff --git a/processing/src/main/java/org/apache/druid/java/util/common/Intervals.java b/processing/src/main/java/org/apache/druid/java/util/common/Intervals.java index 4234a34090ee..0d6acbfe6451 100644 --- a/processing/src/main/java/org/apache/druid/java/util/common/Intervals.java +++ b/processing/src/main/java/org/apache/druid/java/util/common/Intervals.java @@ -99,9 +99,13 @@ public static Interval findOverlappingInterval(Interval searchInterval, Interval return null; } + /** + * @return List of intervals which when added to the given interval create + * {@link Intervals#ETERNITY}. + */ public static List complementOf(Interval interval) { - if (interval.equals(Intervals.ETERNITY)) { + if (isEternity(interval)) { return List.of(); } else { return List.of( diff --git a/processing/src/test/java/org/apache/druid/java/util/common/IntervalsTest.java b/processing/src/test/java/org/apache/druid/java/util/common/IntervalsTest.java index a8703b0ec70e..c4c635367e9b 100644 --- a/processing/src/test/java/org/apache/druid/java/util/common/IntervalsTest.java +++ b/processing/src/test/java/org/apache/druid/java/util/common/IntervalsTest.java @@ -26,6 +26,7 @@ import org.junit.Test; import java.util.Arrays; +import java.util.List; public class IntervalsTest { @@ -86,4 +87,36 @@ public void testInvalidInterval() () -> Intervals.of("invalid string") ); } + + @Test + public void testComplementOf() + { + Assert.assertEquals( + List.of(), + Intervals.complementOf(Intervals.ETERNITY) + ); + + testComplementOf("2020/P1Y"); + testComplementOf("2001/2001-01"); + testComplementOf("2001-01-02/2001-02"); + } + + private void testComplementOf(String interval) + { + final Interval testInterval = Intervals.of(interval); + final List complement = List.of( + new Interval(DateTimes.MIN, testInterval.getStart()), + new Interval(testInterval.getEnd(), DateTimes.MAX) + ); + Assert.assertEquals( + complement, + Intervals.complementOf(testInterval) + ); + Assert.assertEquals( + Intervals.ONLY_ETERNITY, + JodaUtils.condenseIntervals( + List.of(complement.get(0), complement.get(1), testInterval) + ) + ); + } } diff --git a/services/src/main/java/org/apache/druid/cli/CliOverlord.java b/services/src/main/java/org/apache/druid/cli/CliOverlord.java index c627f84174ea..8913586b9748 100644 --- a/services/src/main/java/org/apache/druid/cli/CliOverlord.java +++ b/services/src/main/java/org/apache/druid/cli/CliOverlord.java @@ -69,6 +69,7 @@ import org.apache.druid.indexing.common.tasklogs.SwitchingTaskLogStreamer; import org.apache.druid.indexing.common.tasklogs.TaskRunnerTaskLogStreamer; import org.apache.druid.indexing.compact.CompactionScheduler; +import org.apache.druid.indexing.compact.DruidInputSourceFactory; import org.apache.druid.indexing.compact.OverlordCompactionScheduler; import org.apache.druid.indexing.overlord.DruidOverlord; import org.apache.druid.indexing.overlord.ForkingTaskRunnerFactory; @@ -243,6 +244,7 @@ public void configure(Binder binder) binder.bind(TaskQueryTool.class).in(LazySingleton.class); binder.bind(IndexerMetadataStorageAdapter.class).in(LazySingleton.class); binder.bind(CompactionScheduler.class).to(OverlordCompactionScheduler.class).in(ManageLifecycle.class); + binder.bind(DruidInputSourceFactory.class).in(LazySingleton.class); binder.bind(ScheduledBatchTaskManager.class).in(LazySingleton.class); binder.bind(SupervisorManager.class).in(LazySingleton.class); From f3e19d269cfe9f9d47c9d047b05b3585f32dea13 Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Thu, 14 Aug 2025 22:01:04 +0530 Subject: [PATCH 04/25] Add template for creating MSQ compaction jobs --- .../compact/CompactionSupervisorTest.java | 147 ++++++++++----- .../apache/druid/guice/SupervisorModule.java | 2 + .../CompactionConfigBasedJobTemplate.java | 54 ++++-- .../indexing/compact/CompactionJobQueue.java | 43 +++-- .../compact/CompactionStateMatcher.java | 171 ++++++++++++++++++ .../compact/InlineCompactionJobTemplate.java | 40 +--- .../compact/MSQCompactionJobTemplate.java | 171 ++++++++++++++++++ .../compact/OverlordCompactionScheduler.java | 5 + .../OverlordCompactionSchedulerTest.java | 2 + .../druid/query/http/ClientSqlQuery.java | 12 ++ .../compaction/CompactionCandidate.java | 14 ++ .../server/compaction/CompactionStatus.java | 20 +- .../UserCompactionTaskGranularityConfig.java | 15 ++ 13 files changed, 571 insertions(+), 125 deletions(-) create mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionStateMatcher.java create mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/compact/MSQCompactionJobTemplate.java diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionSupervisorTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionSupervisorTest.java index 82b32e67a785..496da4b5f557 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 @@ -32,13 +32,21 @@ import org.apache.druid.indexing.compact.CatalogCompactionJobTemplate; import org.apache.druid.indexing.compact.CompactionJobTemplate; import org.apache.druid.indexing.compact.CompactionRule; +import org.apache.druid.indexing.compact.CompactionStateMatcher; import org.apache.druid.indexing.compact.CompactionSupervisorSpec; import org.apache.druid.indexing.compact.InlineCompactionJobTemplate; +import org.apache.druid.indexing.compact.MSQCompactionJobTemplate; import org.apache.druid.indexing.overlord.Segments; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.granularity.Granularity; +import org.apache.druid.msq.guice.IndexerMemoryManagementModule; +import org.apache.druid.msq.guice.MSQDurableStorageModule; +import org.apache.druid.msq.guice.MSQIndexingModule; +import org.apache.druid.msq.guice.MSQSqlModule; +import org.apache.druid.msq.guice.SqlTaskModule; import org.apache.druid.query.DruidMetrics; +import org.apache.druid.query.http.ClientSqlQuery; import org.apache.druid.rpc.UpdateResponse; import org.apache.druid.server.coordinator.ClusterCompactionConfig; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; @@ -62,12 +70,12 @@ import java.util.List; import java.util.Map; -import java.util.Set; public class CompactionSupervisorTest extends EmbeddedClusterTestBase { protected final EmbeddedBroker broker = new EmbeddedBroker(); protected final EmbeddedIndexer indexer = new EmbeddedIndexer() + .setServerMemory(4_000_000_000L) .addProperty("druid.worker.capacity", "8"); protected final EmbeddedOverlord overlord = new EmbeddedOverlord() .addProperty("druid.manager.segments.pollDuration", "PT1s") @@ -81,7 +89,15 @@ public EmbeddedDruidCluster createCluster() { return EmbeddedDruidCluster.withEmbeddedDerbyAndZookeeper() .useLatchableEmitter() - .addExtensions(CatalogClientModule.class, CatalogCoordinatorModule.class) + .addExtensions( + CatalogClientModule.class, + CatalogCoordinatorModule.class, + IndexerMemoryManagementModule.class, + MSQDurableStorageModule.class, + MSQIndexingModule.class, + MSQSqlModule.class, + SqlTaskModule.class + ) .addServer(coordinator) .addServer(overlord) .addServer(indexer) @@ -109,11 +125,7 @@ public void test_ingestDayGranularity_andCompactToMonthGranularity() + "\n2025-06-02T00:00:00.000Z,trousers,210" + "\n2025-06-03T00:00:00.000Z,jeans,150" ); - Set segments = cluster.callApi().getVisibleUsedSegments(dataSource, overlord); - Assertions.assertEquals(3, segments.size()); - segments.forEach( - segment -> Assertions.assertTrue(Granularities.DAY.isAligned(segment.getInterval())) - ); + verifyDayAndMonthSegments(3, 0); // Create a compaction config with MONTH granularity InlineSchemaDataSourceCompactionConfig compactionConfig = @@ -127,13 +139,9 @@ public void test_ingestDayGranularity_andCompactToMonthGranularity() .build(); runCompactionWithSpec(compactionConfig); + waitForCompactionTasksToFinish("compact", 1); - // Verify that segments are now compacted to MONTH granularity - segments = cluster.callApi().getVisibleUsedSegments(dataSource, overlord); - Assertions.assertEquals(1, segments.size()); - Assertions.assertTrue( - Granularities.MONTH.isAligned(segments.iterator().next().getInterval()) - ); + verifyDayAndMonthSegments(0, 1); } @Test @@ -143,44 +151,87 @@ public void test_ingestHourGranularity_andCompactToDayAndMonth_withInlineTemplat CascadingCompactionTemplate cascadingTemplate = new CascadingCompactionTemplate( dataSource, List.of( - new CompactionRule(Period.days(2), new InlineCompactionJobTemplate(null, Granularities.DAY)), - new CompactionRule(Period.days(100), new InlineCompactionJobTemplate(null, Granularities.MONTH)) + new CompactionRule(Period.days(1), new InlineCompactionJobTemplate(createMatcher(Granularities.DAY))), + new CompactionRule(Period.days(50), new InlineCompactionJobTemplate(createMatcher(Granularities.MONTH))) ) ); - final CompactionSupervisorSpec compactionSupervisor - = new CompactionSupervisorSpec(cascadingTemplate, false, null); - cluster.callApi().postSupervisor(compactionSupervisor); - - ingestRecordsAtGranularity(2400, "HOUR"); + ingestRecordsAtGranularity(1200, "HOUR"); runCompactionWithSpec(cascadingTemplate); - verifyDayAndMonth(); + waitForCompactionTasksToFinish("compact", 2); + verifyDayAndMonthSegments(1, 1); } @Test public void test_ingestHourGranularity_andCompactToDayAndMonth_withCatalogTemplates() { - ingestRecordsAtGranularity(2400, "HOUR"); + ingestRecordsAtGranularity(1200, "HOUR"); // Add compaction templates to catalog final String dayGranularityTemplateId = saveTemplateToCatalog( - new InlineCompactionJobTemplate(null, Granularities.DAY) + new InlineCompactionJobTemplate(createMatcher(Granularities.DAY)) ); final String monthGranularityTemplateId = saveTemplateToCatalog( - new InlineCompactionJobTemplate(null, Granularities.MONTH) + new InlineCompactionJobTemplate(createMatcher(Granularities.MONTH)) ); // Create a cascading template with DAY and MONTH granularity CascadingCompactionTemplate cascadingTemplate = new CascadingCompactionTemplate( dataSource, List.of( - new CompactionRule(Period.days(2), new CatalogCompactionJobTemplate(dayGranularityTemplateId, null)), - new CompactionRule(Period.days(100), new CatalogCompactionJobTemplate(monthGranularityTemplateId, null)) + new CompactionRule(Period.days(1), new CatalogCompactionJobTemplate(dayGranularityTemplateId, null)), + new CompactionRule(Period.days(50), new CatalogCompactionJobTemplate(monthGranularityTemplateId, null)) ) ); runCompactionWithSpec(cascadingTemplate); - verifyDayAndMonth(); + waitForCompactionTasksToFinish("compact", 2); + verifyDayAndMonthSegments(1, 1); + } + + @Test + public void test_ingestHourGranularity_andCompactToDayAndMonth_withCatalogMSQTemplates() + { + ingestRecordsAtGranularity(1200, "HOUR"); + + // Add compaction templates to catalog + final String sqlDayGranularity = + "REPLACE INTO ${dataSource}" + + " OVERWRITE WHERE __time >= TIMESTAMP '${startDate}' AND __time < TIMESTAMP '${endDate}'" + + " SELECT * FROM ${dataSource}" + + " WHERE __time BETWEEN '${startDate}' AND '${endDate}'" + + " PARTITIONED BY DAY"; + final String dayGranularityTemplateId = saveTemplateToCatalog( + new MSQCompactionJobTemplate( + new ClientSqlQuery(sqlDayGranularity, null, false, false, false, null, null), + createMatcher(Granularities.DAY) + ) + ); + final String sqlMonthGranularity = + "REPLACE INTO ${dataSource}" + + " OVERWRITE WHERE __time >= TIMESTAMP '${startDate}' AND __time < TIMESTAMP '${endDate}'" + + " SELECT * FROM ${dataSource}" + + " WHERE __time >= TIMESTAMP '${startDate}' AND __time < TIMESTAMP '${endDate}'" + + " PARTITIONED BY MONTH"; + final String monthGranularityTemplateId = saveTemplateToCatalog( + new MSQCompactionJobTemplate( + new ClientSqlQuery(sqlMonthGranularity, null, false, false, false, null, null), + createMatcher(Granularities.MONTH) + ) + ); + + // Create a cascading template with DAY and MONTH granularity + CascadingCompactionTemplate cascadingTemplate = new CascadingCompactionTemplate( + dataSource, + List.of( + new CompactionRule(Period.days(1), new CatalogCompactionJobTemplate(dayGranularityTemplateId, null)), + new CompactionRule(Period.days(50), new CatalogCompactionJobTemplate(monthGranularityTemplateId, null)) + ) + ); + + runCompactionWithSpec(cascadingTemplate); + waitForCompactionTasksToFinish("query_controller", 2); + verifyDayAndMonthSegments(1, 1); } private void ingestRecordsAtGranularity(int numRecords, String granularityName) @@ -207,25 +258,19 @@ private void runCompactionWithSpec(DataSourceCompactionConfig config) final CompactionSupervisorSpec compactionSupervisor = new CompactionSupervisorSpec(config, false, null); cluster.callApi().postSupervisor(compactionSupervisor); + } - // Wait for compaction tasks to be submitted - final int numCompactionTasks = overlord.latchableEmitter().waitForEvent( - event -> event.hasMetricName("compact/task/count") - .hasDimension(DruidMetrics.DATASOURCE, dataSource) - .hasValueAtLeast(1L) - ).getValue().intValue(); - - // Wait for the submitted tasks to finish + private void waitForCompactionTasksToFinish(String taskType, int expectedCount) + { overlord.latchableEmitter().waitForEventAggregate( event -> event.hasMetricName("task/run/time") - .hasDimension(DruidMetrics.TASK_TYPE, "compact") + .hasDimension(DruidMetrics.TASK_TYPE, taskType) .hasDimension(DruidMetrics.DATASOURCE, dataSource), - agg -> agg.hasCountAtLeast(numCompactionTasks) + agg -> agg.hasCountAtLeast(expectedCount) ); - } - private void verifyDayAndMonth() + private void verifyDayAndMonthSegments(int expectedDaySegments, int expectedMonthSegments) { // Verify that segments are now compacted to MONTH and DAY granularity List segments = List.copyOf( @@ -233,7 +278,6 @@ private void verifyDayAndMonth() .segmentsMetadataStorage() .retrieveAllUsedSegments(dataSource, Segments.ONLY_VISIBLE) ); - Assertions.assertTrue(segments.size() < 2400); int numMonthSegments = 0; int numDaySegments = 0; @@ -249,14 +293,8 @@ private void verifyDayAndMonth() } } - // Verify that atleast 2 days are fully compacted to DAY - Assertions.assertTrue(numDaySegments >= 2); - - // Verify that atleast 2 months are fully compacted to MONTH - Assertions.assertTrue(numMonthSegments >= 2); - - // Verify that number of uncompacted days is between 5 and 38 - Assertions.assertTrue(5 * 24 <= numHourSegments && numHourSegments <= 38 * 24); + Assertions.assertTrue(numDaySegments >= expectedDaySegments); + Assertions.assertTrue(numMonthSegments >= expectedMonthSegments); } private String saveTemplateToCatalog(CompactionJobTemplate template) @@ -313,4 +351,17 @@ private IndexTask createIndexTaskForInlineData(String taskId, String granularity .dataSource(dataSource) .withId(taskId); } + + private static CompactionStateMatcher createMatcher(Granularity segmentGranularity) + { + return new CompactionStateMatcher( + null, + null, + null, + null, + null, + new UserCompactionTaskGranularityConfig(segmentGranularity, null, null), + null + ); + } } diff --git a/indexing-service/src/main/java/org/apache/druid/guice/SupervisorModule.java b/indexing-service/src/main/java/org/apache/druid/guice/SupervisorModule.java index d334267b3d32..eb3ea599e30e 100644 --- a/indexing-service/src/main/java/org/apache/druid/guice/SupervisorModule.java +++ b/indexing-service/src/main/java/org/apache/druid/guice/SupervisorModule.java @@ -29,6 +29,7 @@ import org.apache.druid.indexing.compact.CatalogCompactionJobTemplate; import org.apache.druid.indexing.compact.CompactionSupervisorSpec; import org.apache.druid.indexing.compact.InlineCompactionJobTemplate; +import org.apache.druid.indexing.compact.MSQCompactionJobTemplate; import org.apache.druid.indexing.input.DruidDatasourceDestination; import org.apache.druid.indexing.overlord.supervisor.SupervisorStateManagerConfig; import org.apache.druid.indexing.scheduledbatch.ScheduledBatchSupervisorSpec; @@ -52,6 +53,7 @@ public List getJacksonModules() .registerSubtypes( new NamedType(InlineCompactionJobTemplate.class, InlineCompactionJobTemplate.TYPE), new NamedType(CatalogCompactionJobTemplate.class, CatalogCompactionJobTemplate.TYPE), + new NamedType(MSQCompactionJobTemplate.class, MSQCompactionJobTemplate.TYPE), new NamedType(CascadingCompactionTemplate.class, CascadingCompactionTemplate.TYPE), new NamedType(CompactionSupervisorSpec.class, CompactionSupervisorSpec.TYPE), new NamedType(ScheduledBatchSupervisorSpec.class, ScheduledBatchSupervisorSpec.TYPE), 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 913960832b61..c3571f55b1ac 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 @@ -31,9 +31,11 @@ import org.apache.druid.server.compaction.DataSourceCompactibleSegmentIterator; import org.apache.druid.server.compaction.NewestSegmentFirstPolicy; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; +import org.apache.druid.server.coordinator.InlineSchemaDataSourceCompactionConfig; import org.apache.druid.server.coordinator.duty.CompactSegments; import org.apache.druid.timeline.SegmentTimeline; import org.joda.time.Interval; +import org.joda.time.Period; import java.util.ArrayList; import java.util.List; @@ -53,6 +55,21 @@ public CompactionConfigBasedJobTemplate(DataSourceCompactionConfig config) this.config = config; } + public static CompactionConfigBasedJobTemplate create(String dataSource, CompactionStateMatcher stateMatcher) + { + return new CompactionConfigBasedJobTemplate( + InlineSchemaDataSourceCompactionConfig + .builder() + .forDataSource(dataSource) + .withSkipOffsetFromLatest(Period.ZERO) + .withTransformSpec(stateMatcher.getTransformSpec()) + .withProjections(stateMatcher.getProjections()) + .withMetricsSpec(stateMatcher.getMetricsSpec()) + .withGranularitySpec(stateMatcher.getGranularitySpec()) + .build() + ); + } + @Override public List createCompactionJobs( InputSource source, @@ -60,18 +77,7 @@ public List createCompactionJobs( CompactionJobParams params ) { - validateInput(source); - validateOutput(destination); - - final Interval searchInterval = Objects.requireNonNull(ensureDruidInputSource(source).getInterval()); - - final SegmentTimeline timeline = params.getTimeline(config.getDataSource()); - final DataSourceCompactibleSegmentIterator segmentIterator = new DataSourceCompactibleSegmentIterator( - config, - timeline, - Intervals.complementOf(searchInterval), - new NewestSegmentFirstPolicy(null) - ); + final DataSourceCompactibleSegmentIterator segmentIterator = getCompactibleCandidates(source, destination, params); final List jobs = new ArrayList<>(); @@ -101,6 +107,30 @@ public String getType() throw new UnsupportedOperationException("This template type cannot be serialized"); } + /** + * Creates an iterator over the compactible candidate segments for the given + * params. + */ + DataSourceCompactibleSegmentIterator getCompactibleCandidates( + InputSource source, + OutputDestination destination, + CompactionJobParams params + ) + { + validateInput(source); + validateOutput(destination); + + final Interval searchInterval = Objects.requireNonNull(ensureDruidInputSource(source).getInterval()); + + final SegmentTimeline timeline = params.getTimeline(config.getDataSource()); + return new DataSourceCompactibleSegmentIterator( + config, + timeline, + Intervals.complementOf(searchInterval), + new NewestSegmentFirstPolicy(null) + ); + } + private void validateInput(InputSource source) { final DruidInputSource druidInputSource = ensureDruidInputSource(source); 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 de2b88984b5a..da64839cb2bf 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 @@ -21,6 +21,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import org.apache.druid.client.DataSourcesSnapshot; +import org.apache.druid.client.broker.BrokerClient; import org.apache.druid.client.indexing.ClientCompactionTaskQuery; import org.apache.druid.client.indexing.ClientTaskQuery; import org.apache.druid.common.guava.FutureUtils; @@ -47,8 +48,8 @@ import org.apache.druid.server.coordinator.stats.RowKey; import org.apache.druid.server.coordinator.stats.Stats; +import javax.annotation.Nullable; import java.util.Map; -import java.util.Objects; import java.util.PriorityQueue; /** @@ -68,6 +69,7 @@ public class CompactionJobQueue private final TaskActionClientFactory taskActionClientFactory; private final OverlordClient overlordClient; private final GlobalTaskLockbox taskLockbox; + private final BrokerClient brokerClient; private final CompactionSnapshotBuilder snapshotBuilder; private final PriorityQueue queue; @@ -82,6 +84,7 @@ public CompactionJobQueue( TaskActionClientFactory taskActionClientFactory, GlobalTaskLockbox taskLockbox, OverlordClient overlordClient, + BrokerClient brokerClient, ObjectMapper objectMapper ) { @@ -104,6 +107,7 @@ public CompactionJobQueue( this.snapshotBuilder = new CompactionSnapshotBuilder(runStats); this.taskActionClientFactory = taskActionClientFactory; this.overlordClient = overlordClient; + this.brokerClient = brokerClient; this.statusTracker = statusTracker; this.objectMapper = objectMapper; this.taskLockbox = taskLockbox; @@ -150,11 +154,8 @@ public void runReadyJobs() { while (!queue.isEmpty()) { final CompactionJob job = queue.poll(); - final ClientTaskQuery task = Objects.requireNonNull(job.getNonNullTask()); - if (startJobIfPendingAndReady(job, searchPolicy)) { - statusTracker.onTaskSubmitted(task.getId(), job.getCandidate()); - runStats.add(Stats.Compaction.SUBMITTED_TASKS, RowKey.of(Dimension.DATASOURCE, task.getDataSource()), 1); + runStats.add(Stats.Compaction.SUBMITTED_TASKS, RowKey.of(Dimension.DATASOURCE, job.getDataSource()), 1); } } } @@ -220,27 +221,35 @@ private boolean startJobIfPendingAndReady(CompactionJob job, CompactionCandidate // Reserve task slots and try to start the task slotManager.reserveTaskSlots(job.getMaxRequiredTaskSlots()); - if (startTaskIfReady(job)) { - snapshotBuilder.addToComplete(candidate); - return true; - } else { + final String taskId = startTaskIfReady(job); + if (taskId == null) { // Mark the job as skipped for now as the intervals might be locked by other tasks snapshotBuilder.addToSkipped(candidate); return false; + } else { + snapshotBuilder.addToComplete(candidate); + statusTracker.onTaskSubmitted(taskId, job.getCandidate()); + return true; } } /** * Starts the given job if the underlying Task is able to acquire locks. * - * @return true if the Task was submitted successfully. + * @return Non-null taskId if the Task was submitted successfully. */ - private boolean startTaskIfReady(CompactionJob job) + @Nullable + private String startTaskIfReady(CompactionJob job) { // Assume MSQ jobs to be always ready if (job.isMsq()) { - // TODO: submit the MSQ job to Broker here - return true; + try { + return FutureUtils.getUnchecked(brokerClient.submitSqlTask(job.getNonNullMsqQuery()), true) + .getTaskId(); + } + catch (Exception e) { + log.error(e, "Error while submitting query[%s] to Broker", job.getNonNullMsqQuery()); + } } final ClientTaskQuery taskQuery = job.getNonNullTask(); @@ -252,16 +261,16 @@ private boolean startTaskIfReady(CompactionJob job) if (task.isReady(taskActionClientFactory.create(task))) { // Hold the locks acquired by task.isReady() as we will reacquire them anyway FutureUtils.getUnchecked(overlordClient.runTask(task.getId(), task), true); - return true; + return task.getId(); } else { taskLockbox.unlockAll(task); - return false; + return null; } } catch (Exception e) { - log.error(e, "Error while checking readiness of task[%s]", task.getId()); + log.error(e, "Error while submitting task[%s] to Overlord", task.getId()); taskLockbox.unlockAll(task); - return false; + return null; } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionStateMatcher.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionStateMatcher.java new file mode 100644 index 000000000000..e43beedfe4b4 --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionStateMatcher.java @@ -0,0 +1,171 @@ +/* + * 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.compact; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonInclude; +import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.druid.data.input.impl.AggregateProjectionSpec; +import org.apache.druid.data.input.impl.DimensionsSpec; +import org.apache.druid.indexer.partitions.PartitionsSpec; +import org.apache.druid.query.aggregation.AggregatorFactory; +import org.apache.druid.segment.IndexSpec; +import org.apache.druid.segment.transform.CompactionTransformSpec; +import org.apache.druid.server.coordinator.UserCompactionTaskGranularityConfig; + +import javax.annotation.Nullable; +import java.util.Arrays; +import java.util.List; +import java.util.Objects; + +/** + * Instances of this class are used to determine if an interval needs to be + * compacted based on the current compaction state of the underlying segments. + *

+ * This class is mostly a duplicate of {@code CompactionState} but allows null + * values for all fields so that only non-null fields are used for matching. + */ +public class CompactionStateMatcher +{ + private final PartitionsSpec partitionsSpec; + private final DimensionsSpec dimensionsSpec; + private final CompactionTransformSpec transformSpec; + private final IndexSpec indexSpec; + private final UserCompactionTaskGranularityConfig granularitySpec; + private final AggregatorFactory[] metricsSpec; + private final List projections; + + @JsonCreator + public CompactionStateMatcher( + @JsonProperty("partitionsSpec") @Nullable PartitionsSpec partitionsSpec, + @JsonProperty("dimensionsSpec") @Nullable DimensionsSpec dimensionsSpec, + @JsonProperty("metricsSpec") @Nullable AggregatorFactory[] metricsSpec, + @JsonProperty("transformSpec") @Nullable CompactionTransformSpec transformSpec, + @JsonProperty("indexSpec") @Nullable IndexSpec indexSpec, + @JsonProperty("granularitySpec") @Nullable UserCompactionTaskGranularityConfig granularitySpec, + @JsonProperty("projections") @Nullable List projections + ) + { + this.partitionsSpec = partitionsSpec; + this.dimensionsSpec = dimensionsSpec; + this.metricsSpec = metricsSpec; + this.transformSpec = transformSpec; + this.indexSpec = indexSpec; + this.granularitySpec = granularitySpec; + this.projections = projections; + } + + @Nullable + @JsonProperty + public PartitionsSpec getPartitionsSpec() + { + return partitionsSpec; + } + + @Nullable + @JsonProperty + public DimensionsSpec getDimensionsSpec() + { + return dimensionsSpec; + } + + @Nullable + @JsonProperty + public AggregatorFactory[] getMetricsSpec() + { + return metricsSpec; + } + + @Nullable + @JsonProperty + public CompactionTransformSpec getTransformSpec() + { + return transformSpec; + } + + @Nullable + @JsonProperty + public IndexSpec getIndexSpec() + { + return indexSpec; + } + + @Nullable + @JsonProperty + public UserCompactionTaskGranularityConfig getGranularitySpec() + { + return granularitySpec; + } + + @JsonProperty + @JsonInclude(JsonInclude.Include.NON_NULL) + @Nullable + public List getProjections() + { + return projections; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + CompactionStateMatcher that = (CompactionStateMatcher) o; + return Objects.equals(partitionsSpec, that.partitionsSpec) && + Objects.equals(dimensionsSpec, that.dimensionsSpec) && + Objects.equals(transformSpec, that.transformSpec) && + Objects.equals(indexSpec, that.indexSpec) && + Objects.equals(granularitySpec, that.granularitySpec) && + Arrays.equals(metricsSpec, that.metricsSpec) && + Objects.equals(projections, that.projections); + } + + @Override + public int hashCode() + { + return Objects.hash( + partitionsSpec, + dimensionsSpec, + transformSpec, + indexSpec, + granularitySpec, + Arrays.hashCode(metricsSpec), + projections + ); + } + + @Override + public String toString() + { + return "CompactionState{" + + "partitionsSpec=" + partitionsSpec + + ", dimensionsSpec=" + dimensionsSpec + + ", transformSpec=" + transformSpec + + ", indexSpec=" + indexSpec + + ", granularitySpec=" + granularitySpec + + ", metricsSpec=" + Arrays.toString(metricsSpec) + + ", projections=" + projections + + '}'; + } +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/InlineCompactionJobTemplate.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/InlineCompactionJobTemplate.java index c147fde4f9f2..73b5afff9873 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/compact/InlineCompactionJobTemplate.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/InlineCompactionJobTemplate.java @@ -23,11 +23,7 @@ import com.fasterxml.jackson.annotation.JsonProperty; import org.apache.druid.data.input.InputSource; import org.apache.druid.data.output.OutputDestination; -import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.server.coordinator.InlineSchemaDataSourceCompactionConfig; -import org.apache.druid.server.coordinator.UserCompactionTaskGranularityConfig; -import org.apache.druid.server.coordinator.UserCompactionTaskQueryTuningConfig; -import org.joda.time.Period; import java.util.List; import java.util.Objects; @@ -45,29 +41,20 @@ public class InlineCompactionJobTemplate extends CompactionJobTemplate { public static final String TYPE = "compactInline"; - private final UserCompactionTaskQueryTuningConfig tuningConfig; - private final Granularity segmentGranularity; + private final CompactionStateMatcher stateMatcher; @JsonCreator public InlineCompactionJobTemplate( - @JsonProperty("tuningConfig") UserCompactionTaskQueryTuningConfig tuningConfig, - @JsonProperty("segmentGranularity") Granularity segmentGranularity + @JsonProperty("stateMatcher") CompactionStateMatcher stateMatcher ) { - this.tuningConfig = tuningConfig; - this.segmentGranularity = segmentGranularity; + this.stateMatcher = stateMatcher; } @JsonProperty - public Granularity getSegmentGranularity() + public CompactionStateMatcher getStateMatcher() { - return segmentGranularity; - } - - @JsonProperty - public UserCompactionTaskQueryTuningConfig getTuningConfig() - { - return tuningConfig; + return stateMatcher; } @Override @@ -78,15 +65,9 @@ public List createCompactionJobs( ) { final String dataSource = ensureDruidInputSource(source).getDataSource(); - return new CompactionConfigBasedJobTemplate( - InlineSchemaDataSourceCompactionConfig - .builder() - .forDataSource(dataSource) - .withSkipOffsetFromLatest(Period.ZERO) - .withTuningConfig(tuningConfig) - .withGranularitySpec(new UserCompactionTaskGranularityConfig(segmentGranularity, null, null)) - .build() - ).createCompactionJobs(source, destination, jobParams); + return CompactionConfigBasedJobTemplate + .create(dataSource, stateMatcher) + .createCompactionJobs(source, destination, jobParams); } @Override @@ -99,14 +80,13 @@ public boolean equals(Object object) return false; } InlineCompactionJobTemplate that = (InlineCompactionJobTemplate) object; - return Objects.equals(this.tuningConfig, that.tuningConfig) - && Objects.equals(this.segmentGranularity, that.segmentGranularity); + return Objects.equals(this.stateMatcher, that.stateMatcher); } @Override public int hashCode() { - return Objects.hash(tuningConfig, segmentGranularity); + return Objects.hash(stateMatcher); } @Override diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/MSQCompactionJobTemplate.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/MSQCompactionJobTemplate.java new file mode 100644 index 000000000000..6453c70fe248 --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/MSQCompactionJobTemplate.java @@ -0,0 +1,171 @@ +/* + * 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.compact; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.druid.data.input.InputSource; +import org.apache.druid.data.output.OutputDestination; +import org.apache.druid.indexing.input.DruidInputSource; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.query.http.ClientSqlQuery; +import org.apache.druid.server.compaction.CompactionCandidate; +import org.apache.druid.server.compaction.CompactionSlotManager; +import org.apache.druid.server.compaction.DataSourceCompactibleSegmentIterator; +import org.joda.time.Interval; +import org.joda.time.format.DateTimeFormat; +import org.joda.time.format.DateTimeFormatter; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.Objects; + +/** + * Compaction template that creates MSQ SQL jobs using a SQL template. The + * template provided by can contain template variables of the format + * {@code ${variableName}} for fields such as datasource name and interval start + * and end. + *

+ * Compaction is triggered for an interval only if the current compaction state + * of the underlying segments does not match with the {@link #stateMatcher}. + */ +public class MSQCompactionJobTemplate extends CompactionJobTemplate +{ + public static final String TYPE = "compactMsq"; + + public static final String VAR_DATASOURCE = "${dataSource}"; + public static final String VAR_START_DATE = "${startDate}"; + public static final String VAR_END_DATE = "${endDate}"; + + private static final DateTimeFormatter TIMESTAMP_FORMATTER = + DateTimeFormat.forPattern("yyyy-MM-dd HH:mm:ss.SSS"); + + private final ClientSqlQuery sqlTemplate; + private final CompactionStateMatcher stateMatcher; + + @JsonCreator + public MSQCompactionJobTemplate( + @JsonProperty("sqlTemplate") ClientSqlQuery sqlTemplate, + @JsonProperty("stateMatcher") CompactionStateMatcher stateMatcher + ) + { + this.sqlTemplate = sqlTemplate; + this.stateMatcher = stateMatcher; + } + + @JsonProperty + public ClientSqlQuery getSqlTemplate() + { + return sqlTemplate; + } + + @JsonProperty + public CompactionStateMatcher getStateMatcher() + { + return stateMatcher; + } + + @Override + List createCompactionJobs( + InputSource source, + OutputDestination destination, + CompactionJobParams jobParams + ) + { + final DruidInputSource druidInputSource = ensureDruidInputSource(source); + final String dataSource = druidInputSource.getDataSource(); + + // Identify the compactible candidate segments + final CompactionConfigBasedJobTemplate delegate = + CompactionConfigBasedJobTemplate.create(dataSource, stateMatcher); + final DataSourceCompactibleSegmentIterator candidateIterator = + delegate.getCompactibleCandidates(source, destination, jobParams); + + // Create MSQ jobs for each candidate by interpolating the template variables + final List jobs = new ArrayList<>(); + while (candidateIterator.hasNext()) { + final CompactionCandidate candidate = candidateIterator.next(); + final Interval compactionInterval = candidate.getCompactionInterval( + stateMatcher.getGranularitySpec() == null + ? null : stateMatcher.getGranularitySpec().getSegmentGranularity() + ); + + final String formattedSql = formatSql( + sqlTemplate.getQuery(), + Map.of( + VAR_DATASOURCE, dataSource, + VAR_START_DATE, compactionInterval.getStart().toString(TIMESTAMP_FORMATTER), + VAR_END_DATE, compactionInterval.getEnd().toString(TIMESTAMP_FORMATTER) + ) + ); + + jobs.add( + new CompactionJob( + sqlTemplate.withSql(formattedSql), + candidate, + compactionInterval, + CompactionSlotManager.getMaxTaskSlotsForMSQCompactionTask(sqlTemplate.getContext()) + ) + ); + } + + return jobs; + } + + /** + * Formats the given SQL by replacing the template variables. + */ + private static String formatSql(String sqlTemplate, Map templateVariables) + { + String sql = sqlTemplate; + for (Map.Entry variable : templateVariables.entrySet()) { + sql = StringUtils.replace(sql, variable.getKey(), variable.getValue()); + } + + return sql; + } + + @Override + public boolean equals(Object object) + { + if (this == object) { + return true; + } + if (object == null || getClass() != object.getClass()) { + return false; + } + MSQCompactionJobTemplate that = (MSQCompactionJobTemplate) object; + return Objects.equals(sqlTemplate, that.sqlTemplate) + && Objects.equals(stateMatcher, that.stateMatcher); + } + + @Override + public int hashCode() + { + return Objects.hash(sqlTemplate, stateMatcher); + } + + @Override + public String getType() + { + return TYPE; + } +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/OverlordCompactionScheduler.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/OverlordCompactionScheduler.java index 6860214a6729..a3190a23f9f7 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/compact/OverlordCompactionScheduler.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/OverlordCompactionScheduler.java @@ -24,6 +24,7 @@ import com.google.common.base.Supplier; import com.google.inject.Inject; import org.apache.druid.client.DataSourcesSnapshot; +import org.apache.druid.client.broker.BrokerClient; import org.apache.druid.client.indexing.ClientCompactionRunnerInfo; import org.apache.druid.indexer.TaskLocation; import org.apache.druid.indexer.TaskStatus; @@ -92,6 +93,7 @@ public class OverlordCompactionScheduler implements CompactionScheduler private final SegmentsMetadataManager segmentManager; private final LocalOverlordClient overlordClient; + private final BrokerClient brokerClient; private final ServiceEmitter emitter; private final ObjectMapper objectMapper; private final TaskMaster taskMaster; @@ -141,6 +143,7 @@ public OverlordCompactionScheduler( TaskActionClientFactory taskActionClientFactory, DruidInputSourceFactory druidInputSourceFactory, ScheduledExecutorFactory executorFactory, + BrokerClient brokerClient, ServiceEmitter emitter, ObjectMapper objectMapper ) @@ -157,6 +160,7 @@ public OverlordCompactionScheduler( this.shouldPollSegments = segmentManager != null && !coordinatorOverlordServiceConfig.isEnabled(); this.overlordClient = new LocalOverlordClient(taskMaster, taskQueryTool, objectMapper); + this.brokerClient = brokerClient; this.activeSupervisors = new ConcurrentHashMap<>(); this.datasourceToCompactionSnapshot = new AtomicReference<>(); @@ -334,6 +338,7 @@ private synchronized void runCompactionDuty() taskActionClientFactory, taskLockbox, overlordClient, + brokerClient, objectMapper ); statusTracker.resetActiveDatasources(activeSupervisors.keySet()); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/compact/OverlordCompactionSchedulerTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/compact/OverlordCompactionSchedulerTest.java index 7ab01ced15e1..4c8a3fe90991 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/compact/OverlordCompactionSchedulerTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/compact/OverlordCompactionSchedulerTest.java @@ -21,6 +21,7 @@ import com.fasterxml.jackson.databind.InjectableValues; import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.client.broker.BrokerClient; import org.apache.druid.client.coordinator.NoopCoordinatorClient; import org.apache.druid.client.indexing.ClientMSQContext; import org.apache.druid.guice.IndexingServiceTuningConfigModule; @@ -190,6 +191,7 @@ private void initScheduler() new SegmentCacheManagerFactory(TestIndex.INDEX_IO, OBJECT_MAPPER) ), (nameFormat, numThreads) -> new WrappingScheduledExecutorService("test", executor, false), + Mockito.mock(BrokerClient.class), serviceEmitter, OBJECT_MAPPER ); diff --git a/processing/src/main/java/org/apache/druid/query/http/ClientSqlQuery.java b/processing/src/main/java/org/apache/druid/query/http/ClientSqlQuery.java index 3b3a2a239806..b619f76d43c0 100644 --- a/processing/src/main/java/org/apache/druid/query/http/ClientSqlQuery.java +++ b/processing/src/main/java/org/apache/druid/query/http/ClientSqlQuery.java @@ -109,6 +109,18 @@ public List getParameters() return parameters; } + public ClientSqlQuery withSql(String sql) + { + return new ClientSqlQuery( + sql, + resultFormat, + header, + typesHeader, + sqlTypesHeader, + context, + parameters + ); + } @Override public boolean equals(final Object o) 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 4cd9b22df812..b5a8a3b35b0a 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 @@ -21,6 +21,7 @@ import org.apache.druid.error.InvalidInput; import org.apache.druid.java.util.common.JodaUtils; +import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.segment.SegmentUtils; import org.apache.druid.timeline.DataSegment; import org.joda.time.Interval; @@ -91,6 +92,19 @@ public Interval getUmbrellaInterval() return umbrellaInterval; } + /** + * Interval used for the compaction task, aligned to the + * {@code targetSegmentGranularity}, if specified. + */ + public Interval getCompactionInterval(@Nullable Granularity targetSegmentGranularity) + { + if (targetSegmentGranularity == null) { + return umbrellaInterval; + } else { + return JodaUtils.umbrellaInterval(targetSegmentGranularity.getIterable(umbrellaInterval)); + } + } + public String getDataSource() { return dataSource; 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 97b645388218..d1da394e290c 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 @@ -20,11 +20,9 @@ package org.apache.druid.server.compaction; import org.apache.commons.lang3.ArrayUtils; -import org.apache.druid.client.indexing.ClientCompactionTaskGranularitySpec; import org.apache.druid.client.indexing.ClientCompactionTaskQueryTuningConfig; import org.apache.druid.common.config.Configs; import org.apache.druid.data.input.impl.DimensionSchema; -import org.apache.druid.indexer.granularity.GranularitySpec; import org.apache.druid.indexer.partitions.DimensionRangePartitionsSpec; import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; import org.apache.druid.indexer.partitions.HashedPartitionsSpec; @@ -270,7 +268,7 @@ private static class Evaluator private final CompactionCandidate candidateSegments; private final CompactionState lastCompactionState; private final ClientCompactionTaskQueryTuningConfig tuningConfig; - private final ClientCompactionTaskGranularitySpec existingGranularitySpec; + private final UserCompactionTaskGranularityConfig existingGranularitySpec; private final UserCompactionTaskGranularityConfig configuredGranularitySpec; private Evaluator( @@ -286,7 +284,7 @@ private Evaluator( if (lastCompactionState == null) { this.existingGranularitySpec = null; } else { - this.existingGranularitySpec = convertIfNotNull( + this.existingGranularitySpec = UserCompactionTaskGranularityConfig.from( lastCompactionState.getGranularitySpec() ); } @@ -494,19 +492,5 @@ private CompactionStatus transformSpecFilterIsUpToDate() String::valueOf ); } - - @Nullable - private ClientCompactionTaskGranularitySpec convertIfNotNull(GranularitySpec granularitySpec) - { - if (granularitySpec == null) { - return null; - } else { - return new ClientCompactionTaskGranularitySpec( - granularitySpec.getSegmentGranularity(), - granularitySpec.getQueryGranularity(), - granularitySpec.isRollup() - ); - } - } } } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/UserCompactionTaskGranularityConfig.java b/server/src/main/java/org/apache/druid/server/coordinator/UserCompactionTaskGranularityConfig.java index be3c7e1af965..1db44319e562 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/UserCompactionTaskGranularityConfig.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/UserCompactionTaskGranularityConfig.java @@ -24,6 +24,7 @@ import org.apache.druid.indexer.granularity.GranularitySpec; import org.apache.druid.java.util.common.granularity.Granularity; +import javax.annotation.Nullable; import java.util.Objects; /** @@ -71,6 +72,20 @@ public Boolean isRollup() return rollup; } + @Nullable + public static UserCompactionTaskGranularityConfig from(GranularitySpec granularitySpec) + { + if (granularitySpec == null) { + return null; + } else { + return new UserCompactionTaskGranularityConfig( + granularitySpec.getSegmentGranularity(), + granularitySpec.getQueryGranularity(), + granularitySpec.isRollup() + ); + } + } + @Override public boolean equals(Object o) { From 939e96abc4e1a083947189d4baeb0ac552932d0f Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Fri, 15 Aug 2025 12:50:05 +0530 Subject: [PATCH 05/25] Add field CompactionCandidate.compactionInterval --- .../compact/CompactionSupervisorTest.java | 17 +++--- .../compact/CascadingCompactionTemplate.java | 15 ++--- .../CompactionConfigBasedJobTemplate.java | 2 - .../druid/indexing/compact/CompactionJob.java | 12 ---- .../indexing/compact/CompactionJobQueue.java | 5 +- .../compact/MSQCompactionJobTemplate.java | 47 ++++++++++------ .../druid/query/http/ClientSqlQuery.java | 13 ----- .../ClientCompactionIntervalSpec.java | 45 +-------------- .../compaction/CompactionCandidate.java | 56 +++++++++++++------ .../compaction/CompactionRunSimulator.java | 2 +- .../compaction/CompactionStatusTracker.java | 6 +- .../DataSourceCompactibleSegmentIterator.java | 14 +++-- .../coordinator/duty/CompactSegments.java | 6 +- .../ClientCompactionIntervalSpecTest.java | 21 +++---- .../compaction/CompactionStatusTest.java | 12 ++-- .../CompactionStatusTrackerTest.java | 12 ++-- .../coordinator/duty/CompactSegmentsTest.java | 21 +++---- 17 files changed, 132 insertions(+), 174 deletions(-) diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionSupervisorTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionSupervisorTest.java index 496da4b5f557..b69281b35d8e 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 @@ -158,8 +158,8 @@ public void test_ingestHourGranularity_andCompactToDayAndMonth_withInlineTemplat ingestRecordsAtGranularity(1200, "HOUR"); runCompactionWithSpec(cascadingTemplate); - waitForCompactionTasksToFinish("compact", 2); - verifyDayAndMonthSegments(1, 1); + waitForCompactionTasksToFinish("compact", 3); + verifyDayAndMonthSegments(1, 2); } @Test @@ -185,8 +185,8 @@ public void test_ingestHourGranularity_andCompactToDayAndMonth_withCatalogTempla ); runCompactionWithSpec(cascadingTemplate); - waitForCompactionTasksToFinish("compact", 2); - verifyDayAndMonthSegments(1, 1); + waitForCompactionTasksToFinish("compact", 3); + verifyDayAndMonthSegments(1, 2); } @Test @@ -230,8 +230,8 @@ public void test_ingestHourGranularity_andCompactToDayAndMonth_withCatalogMSQTem ); runCompactionWithSpec(cascadingTemplate); - waitForCompactionTasksToFinish("query_controller", 2); - verifyDayAndMonthSegments(1, 1); + waitForCompactionTasksToFinish("query_controller", 4); + verifyDayAndMonthSegments(1, 2); } private void ingestRecordsAtGranularity(int numRecords, String granularityName) @@ -281,12 +281,9 @@ private void verifyDayAndMonthSegments(int expectedDaySegments, int expectedMont int numMonthSegments = 0; int numDaySegments = 0; - int numHourSegments = 0; for (DataSegment segment : segments) { - if (Granularities.HOUR.isAligned(segment.getInterval())) { - ++numHourSegments; - } else if (Granularities.DAY.isAligned(segment.getInterval())) { + if (Granularities.DAY.isAligned(segment.getInterval())) { ++numDaySegments; } else if (Granularities.MONTH.isAligned(segment.getInterval())) { ++numMonthSegments; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CascadingCompactionTemplate.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CascadingCompactionTemplate.java index 3b54743c766d..394aeebff85f 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CascadingCompactionTemplate.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CascadingCompactionTemplate.java @@ -44,7 +44,6 @@ import java.util.List; import java.util.Map; import java.util.Objects; -import java.util.stream.Collectors; /** * This template never needs to be deserialized as a {@code BatchIndexingJobTemplate}, @@ -100,7 +99,7 @@ public List createCompactionJobs( final Interval ruleInterval = new Interval(ruleStartTime, previousRuleStartTime); allJobs.addAll( - createJobsUsingTemplate(rule.getTemplate(), ruleInterval, druidInputSource, destination, jobParams) + createJobs(rule.getTemplate(), ruleInterval, druidInputSource, destination, jobParams) ); previousRuleStartTime = ruleStartTime; @@ -110,13 +109,13 @@ public List createCompactionJobs( final CompactionRule lastRule = rules.get(rules.size() - 1); final Interval lastRuleInterval = new Interval(DateTimes.MIN, previousRuleStartTime); allJobs.addAll( - createJobsUsingTemplate(lastRule.getTemplate(), lastRuleInterval, druidInputSource, destination, jobParams) + createJobs(lastRule.getTemplate(), lastRuleInterval, druidInputSource, destination, jobParams) ); return allJobs; } - private List createJobsUsingTemplate( + private List createJobs( CompactionJobTemplate template, Interval searchInterval, DruidInputSource inputSource, @@ -124,13 +123,7 @@ private List createJobsUsingTemplate( CompactionJobParams jobParams ) { - // Skip jobs if they exceed the upper bound of the search interval as the - // corresponding candidate segments fall in the purview of a prior rule - return template - .createCompactionJobs(inputSource.withInterval(searchInterval), destination, jobParams) - .stream() - .filter(job -> !job.getCompactionInterval().getEnd().isAfter(searchInterval.getEnd())) - .collect(Collectors.toList()); + return template.createCompactionJobs(inputSource.withInterval(searchInterval), destination, jobParams); } @Override 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 c3571f55b1ac..c431a43f8a86 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 @@ -87,12 +87,10 @@ public List createCompactionJobs( ClientCompactionTaskQuery taskPayload = CompactSegments.createCompactionTask(candidate, config, params.getClusterCompactionConfig().getEngine()); - final Interval compactionInterval = taskPayload.getIoConfig().getInputSpec().getInterval(); jobs.add( new CompactionJob( taskPayload, candidate, - compactionInterval, CompactionSlotManager.getMaxTaskSlotsForNativeCompactionTask(taskPayload.getTuningConfig()) ) ); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionJob.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionJob.java index 99b560157e47..7a7e7fdc1eab 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionJob.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionJob.java @@ -23,7 +23,6 @@ import org.apache.druid.indexing.template.BatchIndexingJob; import org.apache.druid.query.http.ClientSqlQuery; import org.apache.druid.server.compaction.CompactionCandidate; -import org.joda.time.Interval; /** * {@link BatchIndexingJob} to compact an interval of a datasource. @@ -31,32 +30,27 @@ public class CompactionJob extends BatchIndexingJob { private final CompactionCandidate candidate; - private final Interval compactionInterval; private final int maxRequiredTaskSlots; public CompactionJob( ClientCompactionTaskQuery task, CompactionCandidate candidate, - Interval compactionInterval, int maxRequiredTaskSlots ) { super(task, null); this.candidate = candidate; - this.compactionInterval = compactionInterval; this.maxRequiredTaskSlots = maxRequiredTaskSlots; } public CompactionJob( ClientSqlQuery msqQuery, CompactionCandidate candidate, - Interval compactionInterval, int maxRequiredTaskSlots ) { super(null, msqQuery); this.candidate = candidate; - this.compactionInterval = compactionInterval; this.maxRequiredTaskSlots = maxRequiredTaskSlots; } @@ -70,11 +64,6 @@ public CompactionCandidate getCandidate() return candidate; } - public Interval getCompactionInterval() - { - return compactionInterval; - } - public int getMaxRequiredTaskSlots() { return maxRequiredTaskSlots; @@ -86,7 +75,6 @@ public String toString() return "CompactionJob{" + super.toString() + ", candidate=" + candidate + - ", compactionInterval=" + compactionInterval + ", maxRequiredTaskSlots=" + maxRequiredTaskSlots + '}'; } 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 da64839cb2bf..bb93e940add8 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 @@ -255,7 +255,10 @@ private String startTaskIfReady(CompactionJob job) final ClientTaskQuery taskQuery = job.getNonNullTask(); final Task task = objectMapper.convertValue(taskQuery, Task.class); - log.info("Checking readiness of task[%s] with interval[%s]", task.getId(), job.getCompactionInterval()); + log.debug( + "Checking readiness of task[%s] with interval[%s]", + task.getId(), job.getCandidate().getCompactionInterval() + ); try { taskLockbox.add(task); if (task.isReady(taskActionClientFactory.create(task))) { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/MSQCompactionJobTemplate.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/MSQCompactionJobTemplate.java index 6453c70fe248..e6b88c60b036 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/compact/MSQCompactionJobTemplate.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/MSQCompactionJobTemplate.java @@ -29,11 +29,13 @@ import org.apache.druid.server.compaction.CompactionCandidate; import org.apache.druid.server.compaction.CompactionSlotManager; import org.apache.druid.server.compaction.DataSourceCompactibleSegmentIterator; +import org.apache.druid.server.coordinator.duty.CompactSegments; import org.joda.time.Interval; import org.joda.time.format.DateTimeFormat; import org.joda.time.format.DateTimeFormatter; import java.util.ArrayList; +import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Objects; @@ -103,25 +105,10 @@ List createCompactionJobs( final List jobs = new ArrayList<>(); while (candidateIterator.hasNext()) { final CompactionCandidate candidate = candidateIterator.next(); - final Interval compactionInterval = candidate.getCompactionInterval( - stateMatcher.getGranularitySpec() == null - ? null : stateMatcher.getGranularitySpec().getSegmentGranularity() - ); - - final String formattedSql = formatSql( - sqlTemplate.getQuery(), - Map.of( - VAR_DATASOURCE, dataSource, - VAR_START_DATE, compactionInterval.getStart().toString(TIMESTAMP_FORMATTER), - VAR_END_DATE, compactionInterval.getEnd().toString(TIMESTAMP_FORMATTER) - ) - ); - jobs.add( new CompactionJob( - sqlTemplate.withSql(formattedSql), + createQueryForJob(dataSource, candidate.getCompactionInterval()), candidate, - compactionInterval, CompactionSlotManager.getMaxTaskSlotsForMSQCompactionTask(sqlTemplate.getContext()) ) ); @@ -130,6 +117,34 @@ List createCompactionJobs( return jobs; } + private ClientSqlQuery createQueryForJob(String dataSource, Interval compactionInterval) + { + final String formattedSql = formatSql( + sqlTemplate.getQuery(), + Map.of( + VAR_DATASOURCE, dataSource, + VAR_START_DATE, compactionInterval.getStart().toString(TIMESTAMP_FORMATTER), + VAR_END_DATE, compactionInterval.getEnd().toString(TIMESTAMP_FORMATTER) + ) + ); + + final Map context = new HashMap<>(); + if (sqlTemplate.getContext() != null) { + context.putAll(sqlTemplate.getContext()); + } + context.put(CompactSegments.STORE_COMPACTION_STATE_KEY, true); + + return new ClientSqlQuery( + formattedSql, + sqlTemplate.getResultFormat(), + sqlTemplate.isHeader(), + sqlTemplate.isTypesHeader(), + sqlTemplate.isSqlTypesHeader(), + context, + sqlTemplate.getParameters() + ); + } + /** * Formats the given SQL by replacing the template variables. */ diff --git a/processing/src/main/java/org/apache/druid/query/http/ClientSqlQuery.java b/processing/src/main/java/org/apache/druid/query/http/ClientSqlQuery.java index b619f76d43c0..663714b4d90d 100644 --- a/processing/src/main/java/org/apache/druid/query/http/ClientSqlQuery.java +++ b/processing/src/main/java/org/apache/druid/query/http/ClientSqlQuery.java @@ -109,19 +109,6 @@ public List getParameters() return parameters; } - public ClientSqlQuery withSql(String sql) - { - return new ClientSqlQuery( - sql, - resultFormat, - header, - typesHeader, - sqlTypesHeader, - context, - parameters - ); - } - @Override public boolean equals(final Object o) { 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 7ff9ff424fd1..7a7f65572319 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 @@ -22,67 +22,24 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import org.apache.druid.java.util.common.IAE; -import org.apache.druid.java.util.common.JodaUtils; -import org.apache.druid.java.util.common.granularity.Granularity; -import org.apache.druid.java.util.common.logger.Logger; -import org.apache.druid.timeline.DataSegment; 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. */ public class ClientCompactionIntervalSpec { - private static final Logger LOGGER = new Logger(ClientCompactionIntervalSpec.class); - private static final String TYPE = "interval"; private final Interval interval; @Nullable private final String sha256OfSortedSegmentIds; - public static ClientCompactionIntervalSpec fromSegments(List segments, @Nullable Granularity segmentGranularity) - { - Interval interval = JodaUtils.umbrellaInterval(segments.stream().map(DataSegment::getInterval).collect(Collectors.toList())); - LOGGER.info("Original umbrella interval %s in compaction task for datasource %s", interval, segments.get(0).getDataSource()); - if (segmentGranularity != null) { - // If segmentGranularity is set, then the segmentGranularity of the segments may not align with the configured segmentGranularity - // We must adjust the interval of the compaction task to fully cover and align with the segmentGranularity - // For example, - // - The umbrella interval of the segments is 2015-04-11/2015-04-12 but configured segmentGranularity is YEAR, - // if the compaction task's interval is 2015-04-11/2015-04-12 then we can run into race condition where after submitting - // the compaction task, a new segment outside of the interval (i.e. 2015-02-11/2015-02-12) got created will be lost as it is - // overshadowed by the compacted segment (compacted segment has interval 2015-01-01/2016-01-01. - // Hence, in this case, we must adjust the compaction task interval to 2015-01-01/2016-01-01. - // - The segment to be compacted has MONTH segmentGranularity with the interval 2015-02-01/2015-03-01 but configured - // segmentGranularity is WEEK. If the compaction task's interval is 2015-02-01/2015-03-01 then compacted segments created will be - // 2015-01-26/2015-02-02, 2015-02-02/2015-02-09, 2015-02-09/2015-02-16, 2015-02-16/2015-02-23, 2015-02-23/2015-03-02. - // This is because Druid's WEEK segments alway start and end on Monday. In the above example, 2015-01-26 and 2015-03-02 - // are Mondays but 2015-02-01 and 2015-03-01 are not. Hence, the WEEK segments have to start and end on 2015-01-26 and 2015-03-02. - // If the compaction task's interval is 2015-02-01/2015-03-01, then the compacted segment would cause existing data - // from 2015-01-26 to 2015-02-01 and 2015-03-01 to 2015-03-02 to be lost. Hence, in this case, - // we must adjust the compaction task interval to 2015-01-26/2015-03-02 - interval = JodaUtils.umbrellaInterval(segmentGranularity.getIterable(interval)); - LOGGER.info( - "Interval adjusted to %s in compaction task for datasource %s with configured segmentGranularity %s", - interval, - segments.get(0).getDataSource(), - segmentGranularity - ); - } - return new ClientCompactionIntervalSpec( - interval, - null - ); - } - @JsonCreator public ClientCompactionIntervalSpec( @JsonProperty("interval") Interval interval, 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 b5a8a3b35b0a..f936f3d49a91 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 @@ -28,6 +28,7 @@ import javax.annotation.Nullable; import java.util.List; +import java.util.Set; import java.util.stream.Collectors; /** @@ -38,29 +39,54 @@ public class CompactionCandidate { private final List segments; private final Interval umbrellaInterval; + private final Interval compactionInterval; private final String dataSource; private final long totalBytes; private final int numIntervals; private final CompactionStatus currentStatus; - public static CompactionCandidate from(List segments) + public static CompactionCandidate from( + List segments, + @Nullable Granularity targetSegmentGranularity + ) { if (segments == null || segments.isEmpty()) { throw InvalidInput.exception("Segments to compact must be non-empty"); - } else { - return new CompactionCandidate(segments, null); } + + final Set segmentIntervals = + segments.stream().map(DataSegment::getInterval).collect(Collectors.toSet()); + final Interval umbrellaInterval = JodaUtils.umbrellaInterval(segmentIntervals); + final Interval compactionInterval = + targetSegmentGranularity == null + ? umbrellaInterval + : JodaUtils.umbrellaInterval(targetSegmentGranularity.getIterable(umbrellaInterval)); + + return new CompactionCandidate( + segments, + umbrellaInterval, + compactionInterval, + segmentIntervals.size(), + null + ); } - private CompactionCandidate(List segments, @Nullable CompactionStatus currentStatus) + private CompactionCandidate( + List segments, + Interval umbrellaInterval, + Interval compactionInterval, + int numDistinctSegmentIntervals, + @Nullable CompactionStatus currentStatus + ) { this.segments = segments; this.totalBytes = segments.stream().mapToLong(DataSegment::getSize).sum(); - this.umbrellaInterval = JodaUtils.umbrellaInterval( - segments.stream().map(DataSegment::getInterval).collect(Collectors.toList()) - ); - this.numIntervals = (int) segments.stream().map(DataSegment::getInterval).distinct().count(); + + this.umbrellaInterval = umbrellaInterval; + this.compactionInterval = compactionInterval; + + this.numIntervals = numDistinctSegmentIntervals; this.dataSource = segments.get(0).getDataSource(); this.currentStatus = currentStatus; } @@ -93,16 +119,12 @@ public Interval getUmbrellaInterval() } /** - * Interval used for the compaction task, aligned to the - * {@code targetSegmentGranularity}, if specified. + * Interval aligned to the target segment granularity used for the compaction + * task. This interval completely contains the {@link #umbrellaInterval}. */ - public Interval getCompactionInterval(@Nullable Granularity targetSegmentGranularity) + public Interval getCompactionInterval() { - if (targetSegmentGranularity == null) { - return umbrellaInterval; - } else { - return JodaUtils.umbrellaInterval(targetSegmentGranularity.getIterable(umbrellaInterval)); - } + return compactionInterval; } public String getDataSource() @@ -129,7 +151,7 @@ public CompactionStatus getCurrentStatus() */ public CompactionCandidate withCurrentStatus(CompactionStatus status) { - return new CompactionCandidate(this.segments, status); + return new CompactionCandidate(segments, umbrellaInterval, compactionInterval, numIntervals, status); } @Override diff --git a/server/src/main/java/org/apache/druid/server/compaction/CompactionRunSimulator.java b/server/src/main/java/org/apache/druid/server/compaction/CompactionRunSimulator.java index b6db09501967..781019e12f7d 100644 --- a/server/src/main/java/org/apache/druid/server/compaction/CompactionRunSimulator.java +++ b/server/src/main/java/org/apache/druid/server/compaction/CompactionRunSimulator.java @@ -169,7 +169,7 @@ private Object[] createRow( { final List row = new ArrayList<>(); row.add(candidate.getDataSource()); - row.add(candidate.getUmbrellaInterval()); + row.add(candidate.getCompactionInterval()); row.add(candidate.numSegments()); row.add(candidate.getTotalBytes()); row.add(CompactionSlotManager.getMaxTaskSlotsForNativeCompactionTask(tuningConfig)); 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 5c3daf2a26d9..94bfd41a2efd 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 @@ -67,7 +67,7 @@ public CompactionTaskStatus getLatestTaskStatus(CompactionCandidate candidates) return datasourceStatuses .getOrDefault(candidates.getDataSource(), DatasourceStatus.EMPTY) .intervalToTaskStatus - .get(candidates.getUmbrellaInterval()); + .get(candidates.getCompactionInterval()); } /** @@ -169,7 +169,7 @@ public void onTaskFinished(String taskId, TaskStatus taskStatus) return; } - final Interval compactionInterval = candidateSegments.getUmbrellaInterval(); + final Interval compactionInterval = candidateSegments.getCompactionInterval(); getOrComputeDatasourceStatus(candidateSegments.getDataSource()) .handleCompletedTask(compactionInterval, taskStatus); } @@ -212,7 +212,7 @@ void handleCompletedTask(Interval compactionInterval, TaskStatus taskStatus) void handleSubmittedTask(CompactionCandidate candidateSegments) { - final Interval interval = candidateSegments.getUmbrellaInterval(); + final Interval interval = candidateSegments.getCompactionInterval(); final CompactionTaskStatus lastStatus = intervalToTaskStatus.get(interval); final DateTime now = DateTimes.nowUtc(); diff --git a/server/src/main/java/org/apache/druid/server/compaction/DataSourceCompactibleSegmentIterator.java b/server/src/main/java/org/apache/druid/server/compaction/DataSourceCompactibleSegmentIterator.java index 588e4ba77d41..40867eb5c0c2 100644 --- a/server/src/main/java/org/apache/druid/server/compaction/DataSourceCompactibleSegmentIterator.java +++ b/server/src/main/java/org/apache/druid/server/compaction/DataSourceCompactibleSegmentIterator.java @@ -112,9 +112,11 @@ private void populateQueue(SegmentTimeline timeline, List skipInterval } } if (!partialEternitySegments.isEmpty()) { - CompactionCandidate candidatesWithStatus = CompactionCandidate.from(partialEternitySegments).withCurrentStatus( - CompactionStatus.skipped("Segments have partial-eternity intervals") - ); + // Do not use the target segment granularity in the CompactionCandidate + // as Granularities.getIterable() will cause OOM due to the above issue + CompactionCandidate candidatesWithStatus = CompactionCandidate + .from(partialEternitySegments, null) + .withCurrentStatus(CompactionStatus.skipped("Segments have partial-eternity intervals")); skippedSegments.add(candidatesWithStatus); return; } @@ -309,7 +311,7 @@ private void findAndEnqueueSegmentsToCompact(CompactibleSegmentIterator compacti continue; } - final CompactionCandidate candidates = CompactionCandidate.from(segments); + final CompactionCandidate candidates = CompactionCandidate.from(segments, config.getSegmentGranularity()); final CompactionStatus compactionStatus = CompactionStatus.compute(candidates, config); final CompactionCandidate candidatesWithStatus = candidates.withCurrentStatus(compactionStatus); @@ -352,10 +354,10 @@ private List findInitialSearchInterval( timeline.findNonOvershadowedObjectsInInterval(skipInterval, Partitions.ONLY_COMPLETE) ); if (!CollectionUtils.isNullOrEmpty(segments)) { - final CompactionCandidate candidates = CompactionCandidate.from(segments); + final CompactionCandidate candidates = CompactionCandidate.from(segments, config.getSegmentGranularity()); final CompactionStatus reason; - if (candidates.getUmbrellaInterval().overlaps(latestSkipInterval)) { + if (candidates.getCompactionInterval().overlaps(latestSkipInterval)) { reason = CompactionStatus.skipped("skip offset from latest[%s]", skipOffset); } else { reason = CompactionStatus.skipped("interval locked by another task"); 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 cf166c8f3204..0803fb871641 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 @@ -424,7 +424,7 @@ private static ClientCompactionTaskQuery compactSegments( @Nullable CompactionTransformSpec transformSpec, @Nullable List projectionSpecs, @Nullable Boolean dropExisting, - @Nullable Map context, + Map context, ClientCompactionRunnerInfo compactionRunner ) { @@ -437,17 +437,15 @@ private static ClientCompactionTaskQuery compactSegments( "Segments must have the same dataSource" ); - context = context == null ? new HashMap<>() : context; context.put("priority", compactionTaskPriority); final String taskId = IdUtils.newTaskId(TASK_ID_PREFIX, ClientCompactionTaskQuery.TYPE, dataSource, null); - final Granularity segmentGranularity = granularitySpec == null ? null : granularitySpec.getSegmentGranularity(); return new ClientCompactionTaskQuery( taskId, dataSource, new ClientCompactionIOConfig( - ClientCompactionIntervalSpec.fromSegments(segments, segmentGranularity), + new ClientCompactionIntervalSpec(entry.getCompactionInterval(), null), dropExisting ), tuningConfig, 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 646e798aa666..46ecc64d72d1 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 @@ -24,6 +24,7 @@ import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.segment.IndexIO; +import org.apache.druid.server.compaction.CompactionCandidate; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.partition.NoneShardSpec; import org.junit.Assert; @@ -72,43 +73,43 @@ public class ClientCompactionIntervalSpecTest public void testFromSegmentWithNoSegmentGranularity() { // The umbrella interval of segments is 2015-02-12/2015-04-14 - ClientCompactionIntervalSpec actual = ClientCompactionIntervalSpec.fromSegments(ImmutableList.of(dataSegment1, dataSegment2, dataSegment3), null); - Assert.assertEquals(Intervals.of("2015-02-12/2015-04-14"), actual.getInterval()); + CompactionCandidate actual = CompactionCandidate.from(ImmutableList.of(dataSegment1, dataSegment2, dataSegment3), null); + Assert.assertEquals(Intervals.of("2015-02-12/2015-04-14"), actual.getCompactionInterval()); } @Test public void testFromSegmentWitSegmentGranularitySameAsSegment() { // The umbrella interval of segments is 2015-04-11/2015-04-12 - ClientCompactionIntervalSpec actual = ClientCompactionIntervalSpec.fromSegments(ImmutableList.of(dataSegment1), Granularities.DAY); - Assert.assertEquals(Intervals.of("2015-04-11/2015-04-12"), actual.getInterval()); + CompactionCandidate actual = CompactionCandidate.from(ImmutableList.of(dataSegment1), Granularities.DAY); + Assert.assertEquals(Intervals.of("2015-04-11/2015-04-12"), actual.getCompactionInterval()); } @Test public void testFromSegmentWithCoarserSegmentGranularity() { // The umbrella interval of segments is 2015-02-12/2015-04-14 - ClientCompactionIntervalSpec actual = ClientCompactionIntervalSpec.fromSegments(ImmutableList.of(dataSegment1, dataSegment2, dataSegment3), Granularities.YEAR); + CompactionCandidate actual = CompactionCandidate.from(ImmutableList.of(dataSegment1, dataSegment2, dataSegment3), Granularities.YEAR); // The compaction interval should be expanded to start of the year and end of the year to cover the segmentGranularity - Assert.assertEquals(Intervals.of("2015-01-01/2016-01-01"), actual.getInterval()); + Assert.assertEquals(Intervals.of("2015-01-01/2016-01-01"), actual.getCompactionInterval()); } @Test public void testFromSegmentWithFinerSegmentGranularityAndUmbrellaIntervalAlign() { // The umbrella interval of segments is 2015-02-12/2015-04-14 - ClientCompactionIntervalSpec actual = ClientCompactionIntervalSpec.fromSegments(ImmutableList.of(dataSegment1, dataSegment2, dataSegment3), Granularities.DAY); + CompactionCandidate actual = CompactionCandidate.from(ImmutableList.of(dataSegment1, dataSegment2, dataSegment3), Granularities.DAY); // The segmentGranularity of DAY align with the umbrella interval (umbrella interval can be evenly divide into the segmentGranularity) - Assert.assertEquals(Intervals.of("2015-02-12/2015-04-14"), actual.getInterval()); + Assert.assertEquals(Intervals.of("2015-02-12/2015-04-14"), actual.getCompactionInterval()); } @Test public void testFromSegmentWithFinerSegmentGranularityAndUmbrellaIntervalNotAlign() { // The umbrella interval of segments is 2015-02-12/2015-04-14 - ClientCompactionIntervalSpec actual = ClientCompactionIntervalSpec.fromSegments(ImmutableList.of(dataSegment1, dataSegment2, dataSegment3), Granularities.WEEK); + CompactionCandidate actual = CompactionCandidate.from(ImmutableList.of(dataSegment1, dataSegment2, dataSegment3), Granularities.WEEK); // The segmentGranularity of WEEK does not align with the umbrella interval (umbrella interval cannot be evenly divide into the segmentGranularity) // Hence the compaction interval is modified to aling with the segmentGranularity - Assert.assertEquals(Intervals.of("2015-02-09/2015-04-20"), actual.getInterval()); + Assert.assertEquals(Intervals.of("2015-02-09/2015-04-20"), actual.getCompactionInterval()); } } diff --git a/server/src/test/java/org/apache/druid/server/compaction/CompactionStatusTest.java b/server/src/test/java/org/apache/druid/server/compaction/CompactionStatusTest.java index 9af152a5cc46..ff9d0e5af7c5 100644 --- a/server/src/test/java/org/apache/druid/server/compaction/CompactionStatusTest.java +++ b/server/src/test/java/org/apache/druid/server/compaction/CompactionStatusTest.java @@ -19,7 +19,6 @@ package org.apache.druid.server.compaction; -import com.fasterxml.jackson.databind.ObjectMapper; import org.apache.druid.client.indexing.ClientCompactionTaskQueryTuningConfig; import org.apache.druid.data.input.impl.AggregateProjectionSpec; import org.apache.druid.data.input.impl.LongDimensionSchema; @@ -30,7 +29,6 @@ import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; import org.apache.druid.indexer.partitions.HashedPartitionsSpec; import org.apache.druid.indexer.partitions.PartitionsSpec; -import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.query.aggregation.LongSumAggregatorFactory; @@ -51,8 +49,6 @@ public class CompactionStatusTest { - private static final ObjectMapper OBJECT_MAPPER = new DefaultObjectMapper(); - private static final DataSegment WIKI_SEGMENT = DataSegment.builder() .dataSource(TestDataSource.WIKI) @@ -319,7 +315,7 @@ public void testStatusWhenLastCompactionStateSameAsRequired() final DataSegment segment = DataSegment.builder(WIKI_SEGMENT).lastCompactionState(lastCompactionState).build(); final CompactionStatus status = CompactionStatus.compute( - CompactionCandidate.from(Collections.singletonList(segment)), + CompactionCandidate.from(List.of(segment), Granularities.HOUR), compactionConfig ); Assert.assertTrue(status.isComplete()); @@ -368,7 +364,7 @@ public void testStatusWhenProjectionsMatch() final DataSegment segment = DataSegment.builder(WIKI_SEGMENT).lastCompactionState(lastCompactionState).build(); final CompactionStatus status = CompactionStatus.compute( - CompactionCandidate.from(Collections.singletonList(segment)), + CompactionCandidate.from(List.of(segment), Granularities.HOUR), compactionConfig ); Assert.assertTrue(status.isComplete()); @@ -422,7 +418,7 @@ public void testStatusWhenProjectionsMismatch() final DataSegment segment = DataSegment.builder(WIKI_SEGMENT).lastCompactionState(lastCompactionState).build(); final CompactionStatus status = CompactionStatus.compute( - CompactionCandidate.from(Collections.singletonList(segment)), + CompactionCandidate.from(List.of(segment), Granularities.HOUR), compactionConfig ); Assert.assertFalse(status.isComplete()); @@ -439,7 +435,7 @@ private void verifyCompactionStatusIsPendingBecause( .lastCompactionState(lastCompactionState) .build(); final CompactionStatus status = CompactionStatus.compute( - CompactionCandidate.from(Collections.singletonList(segment)), + CompactionCandidate.from(List.of(segment), null), compactionConfig ); diff --git a/server/src/test/java/org/apache/druid/server/compaction/CompactionStatusTrackerTest.java b/server/src/test/java/org/apache/druid/server/compaction/CompactionStatusTrackerTest.java index 40a2c501ca02..14154d268217 100644 --- a/server/src/test/java/org/apache/druid/server/compaction/CompactionStatusTrackerTest.java +++ b/server/src/test/java/org/apache/druid/server/compaction/CompactionStatusTrackerTest.java @@ -29,7 +29,7 @@ import org.junit.Before; import org.junit.Test; -import java.util.Collections; +import java.util.List; public class CompactionStatusTrackerTest { @@ -48,7 +48,7 @@ public void setup() public void testGetLatestTaskStatusForSubmittedTask() { final CompactionCandidate candidateSegments - = CompactionCandidate.from(Collections.singletonList(WIKI_SEGMENT)); + = CompactionCandidate.from(List.of(WIKI_SEGMENT), null); statusTracker.onTaskSubmitted("task1", candidateSegments); CompactionTaskStatus status = statusTracker.getLatestTaskStatus(candidateSegments); @@ -59,7 +59,7 @@ public void testGetLatestTaskStatusForSubmittedTask() public void testGetLatestTaskStatusForSuccessfulTask() { final CompactionCandidate candidateSegments - = CompactionCandidate.from(Collections.singletonList(WIKI_SEGMENT)); + = CompactionCandidate.from(List.of(WIKI_SEGMENT), null); statusTracker.onTaskSubmitted("task1", candidateSegments); statusTracker.onTaskFinished("task1", TaskStatus.success("task1")); @@ -71,7 +71,7 @@ public void testGetLatestTaskStatusForSuccessfulTask() public void testGetLatestTaskStatusForFailedTask() { final CompactionCandidate candidateSegments - = CompactionCandidate.from(Collections.singletonList(WIKI_SEGMENT)); + = CompactionCandidate.from(List.of(WIKI_SEGMENT), null); statusTracker.onTaskSubmitted("task1", candidateSegments); statusTracker.onTaskFinished("task1", TaskStatus.failure("task1", "some failure")); @@ -84,7 +84,7 @@ public void testGetLatestTaskStatusForFailedTask() public void testGetLatestTaskStatusForRepeatedlyFailingTask() { final CompactionCandidate candidateSegments - = CompactionCandidate.from(Collections.singletonList(WIKI_SEGMENT)); + = CompactionCandidate.from(List.of(WIKI_SEGMENT), null); statusTracker.onTaskSubmitted("task1", candidateSegments); statusTracker.onTaskFinished("task1", TaskStatus.failure("task1", "some failure")); @@ -106,7 +106,7 @@ public void testComputeCompactionStatusForSuccessfulTask() { final NewestSegmentFirstPolicy policy = new NewestSegmentFirstPolicy(null); final CompactionCandidate candidateSegments - = CompactionCandidate.from(Collections.singletonList(WIKI_SEGMENT)); + = CompactionCandidate.from(List.of(WIKI_SEGMENT), null); // Verify that interval is originally eligible for compaction CompactionStatus status 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 d7bbb947d9dc..301dd77493c2 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/duty/CompactSegmentsTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/duty/CompactSegmentsTest.java @@ -80,6 +80,7 @@ import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.segment.indexing.BatchIOConfig; import org.apache.druid.segment.transform.CompactionTransformSpec; +import org.apache.druid.server.compaction.CompactionCandidate; import org.apache.druid.server.compaction.CompactionCandidateSearchPolicy; import org.apache.druid.server.compaction.CompactionSlotManager; import org.apache.druid.server.compaction.CompactionStatusTracker; @@ -871,8 +872,8 @@ public void testCompactWithGranularitySpec() // All segments is compact at the same time since we changed the segment granularity to YEAR and all segment // are within the same year Assert.assertEquals( - ClientCompactionIntervalSpec.fromSegments(datasourceToSegments.get(dataSource), Granularities.YEAR), - taskPayload.getIoConfig().getInputSpec() + CompactionCandidate.from(datasourceToSegments.get(dataSource), Granularities.YEAR).getCompactionInterval(), + taskPayload.getIoConfig().getInputSpec().getInterval() ); ClientCompactionTaskGranularitySpec expectedGranularitySpec = @@ -1062,8 +1063,8 @@ public void testCompactWithRollupInGranularitySpec() // All segments is compact at the same time since we changed the segment granularity to YEAR and all segment // are within the same year Assert.assertEquals( - ClientCompactionIntervalSpec.fromSegments(datasourceToSegments.get(dataSource), Granularities.YEAR), - taskPayload.getIoConfig().getInputSpec() + CompactionCandidate.from(datasourceToSegments.get(dataSource), Granularities.YEAR).getCompactionInterval(), + taskPayload.getIoConfig().getInputSpec().getInterval() ); ClientCompactionTaskGranularitySpec expectedGranularitySpec = @@ -1157,8 +1158,8 @@ public void testCompactWithGranularitySpecConflictWithActiveCompactionTask() // All segments is compact at the same time since we changed the segment granularity to YEAR and all segment // are within the same year Assert.assertEquals( - ClientCompactionIntervalSpec.fromSegments(datasourceToSegments.get(dataSource), Granularities.YEAR), - taskPayload.getIoConfig().getInputSpec() + CompactionCandidate.from(datasourceToSegments.get(dataSource), Granularities.YEAR).getCompactionInterval(), + taskPayload.getIoConfig().getInputSpec().getInterval() ); ClientCompactionTaskGranularitySpec expectedGranularitySpec = @@ -1398,8 +1399,8 @@ public void testDetermineSegmentGranularityFromSegmentsToCompact() ClientCompactionTaskQuery taskPayload = (ClientCompactionTaskQuery) payloadCaptor.getValue(); Assert.assertEquals( - ClientCompactionIntervalSpec.fromSegments(segments, Granularities.DAY), - taskPayload.getIoConfig().getInputSpec() + CompactionCandidate.from(segments, Granularities.DAY).getCompactionInterval(), + taskPayload.getIoConfig().getInputSpec().getInterval() ); ClientCompactionTaskGranularitySpec expectedGranularitySpec = @@ -1461,8 +1462,8 @@ public void testDetermineSegmentGranularityFromSegmentGranularityInCompactionCon ClientCompactionTaskQuery taskPayload = (ClientCompactionTaskQuery) payloadCaptor.getValue(); Assert.assertEquals( - ClientCompactionIntervalSpec.fromSegments(segments, Granularities.YEAR), - taskPayload.getIoConfig().getInputSpec() + CompactionCandidate.from(segments, Granularities.YEAR).getCompactionInterval(), + taskPayload.getIoConfig().getInputSpec().getInterval() ); ClientCompactionTaskGranularitySpec expectedGranularitySpec = From 0f96366cf3b035603d732c205bb73487cc6326b3 Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Sat, 16 Aug 2025 13:15:20 +0530 Subject: [PATCH 06/25] Add more test coverage for templates --- .../compact/CompactionSupervisorTest.java | 60 ++- .../compact/CascadingCompactionTemplate.java | 17 +- .../compact/MSQCompactionJobTemplate.java | 10 +- .../OverlordCompactionSchedulerTest.java | 393 +++++++++++++++--- ...TestIndexerMetadataStorageCoordinator.java | 6 + .../druid/java/util/common/Intervals.java | 4 + .../server/compaction/CompactionStatus.java | 33 +- .../compaction/CompactionStatusTest.java | 24 +- .../coordinator/CreateDataSegments.java | 4 + 9 files changed, 433 insertions(+), 118 deletions(-) diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionSupervisorTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionSupervisorTest.java index b69281b35d8e..64679400aae3 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 @@ -38,6 +38,7 @@ import org.apache.druid.indexing.compact.MSQCompactionJobTemplate; import org.apache.druid.indexing.overlord.Segments; import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.msq.guice.IndexerMemoryManagementModule; @@ -125,7 +126,7 @@ public void test_ingestDayGranularity_andCompactToMonthGranularity() + "\n2025-06-02T00:00:00.000Z,trousers,210" + "\n2025-06-03T00:00:00.000Z,jeans,150" ); - verifyDayAndMonthSegments(3, 0); + verifyNumSegmentsWith(Granularities.DAY, 3); // Create a compaction config with MONTH granularity InlineSchemaDataSourceCompactionConfig compactionConfig = @@ -141,7 +142,8 @@ public void test_ingestDayGranularity_andCompactToMonthGranularity() runCompactionWithSpec(compactionConfig); waitForCompactionTasksToFinish("compact", 1); - verifyDayAndMonthSegments(0, 1); + verifyNumSegmentsWith(Granularities.DAY, 0); + verifyNumSegmentsWith(Granularities.MONTH, 1); } @Test @@ -159,7 +161,8 @@ public void test_ingestHourGranularity_andCompactToDayAndMonth_withInlineTemplat ingestRecordsAtGranularity(1200, "HOUR"); runCompactionWithSpec(cascadingTemplate); waitForCompactionTasksToFinish("compact", 3); - verifyDayAndMonthSegments(1, 2); + verifyNumSegmentsWith(Granularities.DAY, 1); + verifyNumSegmentsWith(Granularities.MONTH, 2); } @Test @@ -186,7 +189,8 @@ public void test_ingestHourGranularity_andCompactToDayAndMonth_withCatalogTempla runCompactionWithSpec(cascadingTemplate); waitForCompactionTasksToFinish("compact", 3); - verifyDayAndMonthSegments(1, 2); + verifyNumSegmentsWith(Granularities.DAY, 1); + verifyNumSegmentsWith(Granularities.MONTH, 2); } @Test @@ -197,9 +201,9 @@ public void test_ingestHourGranularity_andCompactToDayAndMonth_withCatalogMSQTem // Add compaction templates to catalog final String sqlDayGranularity = "REPLACE INTO ${dataSource}" - + " OVERWRITE WHERE __time >= TIMESTAMP '${startDate}' AND __time < TIMESTAMP '${endDate}'" + + " OVERWRITE WHERE __time >= TIMESTAMP '${startTimestamp}' AND __time < TIMESTAMP '${endTimestamp}'" + " SELECT * FROM ${dataSource}" - + " WHERE __time BETWEEN '${startDate}' AND '${endDate}'" + + " WHERE __time BETWEEN '${startTimestamp}' AND '${endTimestamp}'" + " PARTITIONED BY DAY"; final String dayGranularityTemplateId = saveTemplateToCatalog( new MSQCompactionJobTemplate( @@ -209,9 +213,9 @@ public void test_ingestHourGranularity_andCompactToDayAndMonth_withCatalogMSQTem ); final String sqlMonthGranularity = "REPLACE INTO ${dataSource}" - + " OVERWRITE WHERE __time >= TIMESTAMP '${startDate}' AND __time < TIMESTAMP '${endDate}'" + + " OVERWRITE WHERE __time >= TIMESTAMP '${startTimestamp}' AND __time < TIMESTAMP '${endTimestamp}'" + " SELECT * FROM ${dataSource}" - + " WHERE __time >= TIMESTAMP '${startDate}' AND __time < TIMESTAMP '${endDate}'" + + " WHERE __time >= TIMESTAMP '${startTimestamp}' AND __time < TIMESTAMP '${endTimestamp}'" + " PARTITIONED BY MONTH"; final String monthGranularityTemplateId = saveTemplateToCatalog( new MSQCompactionJobTemplate( @@ -230,13 +234,14 @@ public void test_ingestHourGranularity_andCompactToDayAndMonth_withCatalogMSQTem ); runCompactionWithSpec(cascadingTemplate); - waitForCompactionTasksToFinish("query_controller", 4); - verifyDayAndMonthSegments(1, 2); + waitForCompactionTasksToFinish("query_controller", 3); + verifyNumSegmentsWith(Granularities.DAY, 1); + verifyNumSegmentsWith(Granularities.MONTH, 2); } private void ingestRecordsAtGranularity(int numRecords, String granularityName) { - // Ingest data at HOUR granularity and verify + // Ingest data at specified granularity and verify Granularity granularity = Granularity.fromString(granularityName); runIngestionAtGranularity( granularityName, @@ -270,28 +275,21 @@ private void waitForCompactionTasksToFinish(String taskType, int expectedCount) ); } - private void verifyDayAndMonthSegments(int expectedDaySegments, int expectedMonthSegments) + private void verifyNumSegmentsWith(Granularity granularity, int numExpectedSegments) { - // Verify that segments are now compacted to MONTH and DAY granularity - List segments = List.copyOf( - overlord.bindings() - .segmentsMetadataStorage() - .retrieveAllUsedSegments(dataSource, Segments.ONLY_VISIBLE) + long numMatchingSegments = overlord + .bindings() + .segmentsMetadataStorage() + .retrieveAllUsedSegments(dataSource, Segments.ONLY_VISIBLE) + .stream() + .filter(segment -> granularity.isAligned(segment.getInterval())) + .count(); + + Assertions.assertEquals( + numExpectedSegments, + (int) numMatchingSegments, + StringUtils.format("Segment with granularity[%s]", granularity) ); - - int numMonthSegments = 0; - int numDaySegments = 0; - - for (DataSegment segment : segments) { - if (Granularities.DAY.isAligned(segment.getInterval())) { - ++numDaySegments; - } else if (Granularities.MONTH.isAligned(segment.getInterval())) { - ++numMonthSegments; - } - } - - Assertions.assertTrue(numDaySegments >= expectedDaySegments); - Assertions.assertTrue(numMonthSegments >= expectedMonthSegments); } private String saveTemplateToCatalog(CompactionJobTemplate template) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CascadingCompactionTemplate.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CascadingCompactionTemplate.java index 394aeebff85f..a474047c6f0c 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CascadingCompactionTemplate.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CascadingCompactionTemplate.java @@ -123,7 +123,22 @@ private List createJobs( CompactionJobParams jobParams ) { - return template.createCompactionJobs(inputSource.withInterval(searchInterval), destination, jobParams); + final List allJobs = template.createCompactionJobs( + inputSource.withInterval(searchInterval), + destination, + jobParams + ); + + // Filter out jobs if they are outside the search interval + final List validJobs = new ArrayList<>(); + for (CompactionJob job : allJobs) { + final Interval compactionInterval = job.getCandidate().getCompactionInterval(); + if (searchInterval.contains(compactionInterval)) { + validJobs.add(job); + } + } + + return validJobs; } @Override diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/MSQCompactionJobTemplate.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/MSQCompactionJobTemplate.java index e6b88c60b036..522421853cf3 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/compact/MSQCompactionJobTemplate.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/MSQCompactionJobTemplate.java @@ -54,8 +54,8 @@ public class MSQCompactionJobTemplate extends CompactionJobTemplate public static final String TYPE = "compactMsq"; public static final String VAR_DATASOURCE = "${dataSource}"; - public static final String VAR_START_DATE = "${startDate}"; - public static final String VAR_END_DATE = "${endDate}"; + public static final String VAR_START_TIMESTAMP = "${startTimestamp}"; + public static final String VAR_END_TIMESTAMP = "${endTimestamp}"; private static final DateTimeFormatter TIMESTAMP_FORMATTER = DateTimeFormat.forPattern("yyyy-MM-dd HH:mm:ss.SSS"); @@ -123,8 +123,8 @@ private ClientSqlQuery createQueryForJob(String dataSource, Interval compactionI sqlTemplate.getQuery(), Map.of( VAR_DATASOURCE, dataSource, - VAR_START_DATE, compactionInterval.getStart().toString(TIMESTAMP_FORMATTER), - VAR_END_DATE, compactionInterval.getEnd().toString(TIMESTAMP_FORMATTER) + VAR_START_TIMESTAMP, compactionInterval.getStart().toString(TIMESTAMP_FORMATTER), + VAR_END_TIMESTAMP, compactionInterval.getEnd().toString(TIMESTAMP_FORMATTER) ) ); @@ -148,7 +148,7 @@ private ClientSqlQuery createQueryForJob(String dataSource, Interval compactionI /** * Formats the given SQL by replacing the template variables. */ - private static String formatSql(String sqlTemplate, Map templateVariables) + public static String formatSql(String sqlTemplate, Map templateVariables) { String sql = sqlTemplate; for (Map.Entry variable : templateVariables.entrySet()) { diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/compact/OverlordCompactionSchedulerTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/compact/OverlordCompactionSchedulerTest.java index 4c8a3fe90991..f83daf7bb5bd 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/compact/OverlordCompactionSchedulerTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/compact/OverlordCompactionSchedulerTest.java @@ -21,11 +21,20 @@ import com.fasterxml.jackson.databind.InjectableValues; import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.util.concurrent.Futures; +import org.apache.druid.catalog.MapMetadataCatalog; +import org.apache.druid.catalog.model.ResolvedTable; +import org.apache.druid.catalog.model.TableId; +import org.apache.druid.catalog.model.TableSpec; +import org.apache.druid.catalog.model.table.IndexingTemplateDefn; import org.apache.druid.client.broker.BrokerClient; import org.apache.druid.client.coordinator.NoopCoordinatorClient; import org.apache.druid.client.indexing.ClientMSQContext; +import org.apache.druid.common.utils.IdUtils; import org.apache.druid.guice.IndexingServiceTuningConfigModule; +import org.apache.druid.guice.SupervisorModule; import org.apache.druid.indexer.CompactionEngine; +import org.apache.druid.indexer.TaskState; import org.apache.druid.indexing.common.SegmentCacheManagerFactory; import org.apache.druid.indexing.common.TimeChunkLock; import org.apache.druid.indexing.common.actions.RetrieveUsedSegmentsAction; @@ -39,6 +48,7 @@ import org.apache.druid.indexing.common.task.Task; import org.apache.druid.indexing.overlord.GlobalTaskLockbox; import org.apache.druid.indexing.overlord.HeapMemoryTaskStorage; +import org.apache.druid.indexing.overlord.Segments; import org.apache.druid.indexing.overlord.TaskMaster; import org.apache.druid.indexing.overlord.TaskQueryTool; import org.apache.druid.indexing.overlord.TaskQueue; @@ -49,8 +59,13 @@ import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.Granularities; +import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.metrics.StubServiceEmitter; +import org.apache.druid.metadata.SegmentsMetadataManager; +import org.apache.druid.query.http.ClientSqlQuery; +import org.apache.druid.query.http.SqlTaskStatus; import org.apache.druid.segment.TestDataSource; import org.apache.druid.segment.TestIndex; import org.apache.druid.server.compaction.CompactionSimulateResult; @@ -66,11 +81,15 @@ import org.apache.druid.server.coordinator.DataSourceCompactionConfig; import org.apache.druid.server.coordinator.DruidCompactionConfig; import org.apache.druid.server.coordinator.InlineSchemaDataSourceCompactionConfig; +import org.apache.druid.server.coordinator.UserCompactionTaskGranularityConfig; import org.apache.druid.server.coordinator.simulate.BlockingExecutorService; -import org.apache.druid.server.coordinator.simulate.TestSegmentsMetadataManager; import org.apache.druid.server.coordinator.simulate.WrappingScheduledExecutorService; import org.apache.druid.server.coordinator.stats.Stats; import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.Partitions; +import org.joda.time.DateTime; +import org.joda.time.Duration; +import org.joda.time.Interval; import org.joda.time.Period; import org.junit.Assert; import org.junit.Before; @@ -82,6 +101,8 @@ import java.util.Arrays; import java.util.Collections; import java.util.List; +import java.util.Map; +import java.util.Set; import java.util.concurrent.atomic.AtomicReference; public class OverlordCompactionSchedulerTest @@ -91,6 +112,7 @@ public class OverlordCompactionSchedulerTest static { OBJECT_MAPPER = new DefaultObjectMapper(); OBJECT_MAPPER.registerModules(new IndexingServiceTuningConfigModule().getJacksonModules()); + OBJECT_MAPPER.registerModules(new SupervisorModule().getJacksonModules()); OBJECT_MAPPER.setInjectableValues( new InjectableValues .Std() @@ -101,25 +123,42 @@ public class OverlordCompactionSchedulerTest ); } + private static final DateTime JAN_20 = DateTimes.of("2025-01-20"); + private static final DateTime MAR_11 = DateTimes.of("2025-03-11"); + private AtomicReference compactionConfig; private CoordinatorOverlordServiceConfig coordinatorOverlordServiceConfig; private TaskMaster taskMaster; private TaskQueue taskQueue; + private BrokerClient brokerClient; private TaskActionClientFactory taskActionClientFactory; private BlockingExecutorService executor; private HeapMemoryTaskStorage taskStorage; - private TestSegmentsMetadataManager segmentsMetadataManager; + private TestIndexerMetadataStorageCoordinator segmentStorage; + private SegmentsMetadataManager segmentsMetadataManager; private StubServiceEmitter serviceEmitter; + private String dataSource; + private MapMetadataCatalog catalog; private OverlordCompactionScheduler scheduler; @Before public void setUp() { + dataSource = "wiki_" + IdUtils.getRandomId(); + final TaskRunner taskRunner = Mockito.mock(TaskRunner.class); + Mockito.when(taskRunner.getTotalCapacity()).thenReturn(10); + Mockito.when(taskRunner.getMaximumCapacityWithAutoscale()).thenReturn(10); + taskQueue = Mockito.mock(TaskQueue.class); + catalog = new MapMetadataCatalog(OBJECT_MAPPER); + + brokerClient = Mockito.mock(BrokerClient.class); + Mockito.when(brokerClient.submitSqlTask(ArgumentMatchers.any(ClientSqlQuery.class))) + .thenReturn(Futures.immediateFuture(new SqlTaskStatus(IdUtils.getRandomId(), TaskState.RUNNING, null))); taskMaster = new TaskMaster(null, null); Assert.assertFalse(taskMaster.isHalfOrFullLeader()); @@ -137,9 +176,10 @@ public void setUp() executor = new BlockingExecutorService("test"); serviceEmitter = new StubServiceEmitter(); - segmentsMetadataManager = new TestSegmentsMetadataManager(); + segmentStorage = new TestIndexerMetadataStorageCoordinator(); + segmentsMetadataManager = segmentStorage.getManager(); - compactionConfig = new AtomicReference<>(new ClusterCompactionConfig(null, null, null, true, null)); + compactionConfig = new AtomicReference<>(new ClusterCompactionConfig(1.0, 10, null, true, null)); coordinatorOverlordServiceConfig = new CoordinatorOverlordServiceConfig(false, null); taskActionClientFactory = task -> new TaskActionClient() @@ -149,7 +189,10 @@ public void setUp() public RetType submit(TaskAction taskAction) { if (taskAction instanceof RetrieveUsedSegmentsAction) { - return (RetType) segmentsMetadataManager.getAllSegments(); + return (RetType) segmentStorage.retrieveAllUsedSegments( + ((RetrieveUsedSegmentsAction) taskAction).getDataSource(), + Segments.ONLY_VISIBLE + ); } else if (taskAction instanceof TimeChunkLockTryAcquireAction) { final TimeChunkLockTryAcquireAction lockAcquireAction = (TimeChunkLockTryAcquireAction) taskAction; return (RetType) new TimeChunkLock( @@ -191,14 +234,14 @@ private void initScheduler() new SegmentCacheManagerFactory(TestIndex.INDEX_IO, OBJECT_MAPPER) ), (nameFormat, numThreads) -> new WrappingScheduledExecutorService("test", executor, false), - Mockito.mock(BrokerClient.class), + brokerClient, serviceEmitter, OBJECT_MAPPER ); } @Test - public void testBecomeLeader_triggersStart_ifEnabled() + public void test_becomeLeader_triggersStart_ifEnabled() { Assert.assertTrue(scheduler.isEnabled()); @@ -212,7 +255,7 @@ public void testBecomeLeader_triggersStart_ifEnabled() } @Test - public void testBecomeLeader_doesNotTriggerStart_ifDisabled() + public void test_becomeLeader_doesNotTriggerStart_ifDisabled() { disableScheduler(); Assert.assertFalse(scheduler.isEnabled()); @@ -226,7 +269,7 @@ public void testBecomeLeader_doesNotTriggerStart_ifDisabled() } @Test - public void testStopBeingLeader_triggersStop() + public void test_stopBeingLeader_triggersStop() { Assert.assertFalse(scheduler.isRunning()); @@ -242,7 +285,7 @@ public void testStopBeingLeader_triggersStop() } @Test - public void testDisablingScheduler_triggersStop() + public void test_disableSupervisors_triggersStop() { // Start scheduler scheduler.becomeLeader(); @@ -260,7 +303,7 @@ public void testDisablingScheduler_triggersStop() } @Test - public void testEnablingScheduler_triggersStart() + public void test_enableSupervisors_triggersStart() { disableScheduler(); @@ -279,7 +322,7 @@ public void testEnablingScheduler_triggersStart() } @Test - public void testSegmentsAreNotPolled_ifSupervisorsAreDisabled() + public void test_disableSupervisors_disablesSegmentPolling() { disableScheduler(); @@ -287,7 +330,7 @@ public void testSegmentsAreNotPolled_ifSupervisorsAreDisabled() } @Test - public void testSegmentsArePolled_whenRunningInStandaloneMode() + public void test_enableSupervisors_inStandaloneMode_enablesSegmentPolling() { coordinatorOverlordServiceConfig = new CoordinatorOverlordServiceConfig(false, null); initScheduler(); @@ -296,7 +339,7 @@ public void testSegmentsArePolled_whenRunningInStandaloneMode() } @Test - public void testSegmentsAreNotPolled_whenRunningInCoordinatorMode() + public void test_enableSupervisors_inCoordinatorMode_disablesSegmentPolling() { coordinatorOverlordServiceConfig = new CoordinatorOverlordServiceConfig(true, "overlord"); initScheduler(); @@ -316,7 +359,7 @@ private void verifySegmentPolling(boolean enabled) } @Test - public void testNullCompactionConfigIsInvalid() + public void test_validateCompactionConfig_returnsInvalid_forNullConfig() { final CompactionConfigValidationResult result = scheduler.validateCompactionConfig(null); Assert.assertFalse(result.isValid()); @@ -324,11 +367,11 @@ public void testNullCompactionConfigIsInvalid() } @Test - public void testMsqCompactionConfigWithOneMaxTasksIsInvalid() + public void test_validateCompactionConfig_returnsInvalid_forMSQConfigWithOneMaxTasks() { final DataSourceCompactionConfig datasourceConfig = InlineSchemaDataSourceCompactionConfig .builder() - .forDataSource(TestDataSource.WIKI) + .forDataSource(dataSource) .withEngine(CompactionEngine.MSQ) .withTaskContext(Collections.singletonMap(ClientMSQContext.CTX_MAX_NUM_TASKS, 1)) .build(); @@ -342,35 +385,24 @@ public void testMsqCompactionConfigWithOneMaxTasksIsInvalid() } @Test - public void testStartCompaction() + public void test_startCompaction_enablesTaskSubmission_forDatasource() { - final List wikiSegments = CreateDataSegments.ofDatasource(TestDataSource.WIKI).eachOfSizeInMb(100); - wikiSegments.forEach(segmentsMetadataManager::addSegment); + createSegments(1, Granularities.DAY, JAN_20); scheduler.becomeLeader(); - scheduler.startCompaction(TestDataSource.WIKI, createSupervisor()); + scheduler.startCompaction(dataSource, createSupervisorWithInlineSpec()); - executor.finishNextPendingTask(); + runCompactionTasks(1); - ArgumentCaptor taskArgumentCaptor = ArgumentCaptor.forClass(Task.class); - Mockito.verify(taskQueue, Mockito.times(1)).add(taskArgumentCaptor.capture()); - - Task submittedTask = taskArgumentCaptor.getValue(); - Assert.assertNotNull(submittedTask); - Assert.assertTrue(submittedTask instanceof CompactionTask); - - final CompactionTask compactionTask = (CompactionTask) submittedTask; - Assert.assertEquals(TestDataSource.WIKI, compactionTask.getDataSource()); - - final AutoCompactionSnapshot.Builder expectedSnapshot = AutoCompactionSnapshot.builder(TestDataSource.WIKI); + final AutoCompactionSnapshot.Builder expectedSnapshot = AutoCompactionSnapshot.builder(dataSource); expectedSnapshot.incrementCompactedStats(CompactionStatistics.create(100_000_000, 1, 1)); Assert.assertEquals( expectedSnapshot.build(), - scheduler.getCompactionSnapshot(TestDataSource.WIKI) + scheduler.getCompactionSnapshot(dataSource) ); Assert.assertEquals( - Collections.singletonMap(TestDataSource.WIKI, expectedSnapshot.build()), + Map.of(dataSource, expectedSnapshot.build()), scheduler.getAllCompactionSnapshots() ); @@ -381,27 +413,22 @@ public void testStartCompaction() } @Test - public void testStopCompaction() + public void test_stopCompaction_disablesTaskSubmission_forDatasource() { - final List wikiSegments = CreateDataSegments.ofDatasource(TestDataSource.WIKI).eachOfSizeInMb(100); - wikiSegments.forEach(segmentsMetadataManager::addSegment); + createSegments(1, Granularities.DAY, JAN_20); scheduler.becomeLeader(); - scheduler.startCompaction( - TestDataSource.WIKI, - createSupervisor() - ); - scheduler.stopCompaction(TestDataSource.WIKI); - - executor.finishNextPendingTask(); + scheduler.startCompaction(dataSource, createSupervisorWithInlineSpec()); + scheduler.stopCompaction(dataSource); + runScheduledJob(); Mockito.verify(taskQueue, Mockito.never()).add(ArgumentMatchers.any()); Assert.assertEquals( - AutoCompactionSnapshot.builder(TestDataSource.WIKI) + AutoCompactionSnapshot.builder(dataSource) .withStatus(AutoCompactionSnapshot.ScheduleStatus.NOT_ENABLED) .build(), - scheduler.getCompactionSnapshot(TestDataSource.WIKI) + scheduler.getCompactionSnapshot(dataSource) ); Assert.assertTrue(scheduler.getAllCompactionSnapshots().isEmpty()); @@ -412,20 +439,14 @@ public void testStopCompaction() } @Test - public void testSimulateRun() + public void test_simulateRunWithConfigUpdate() { - final List wikiSegments = CreateDataSegments - .ofDatasource(TestDataSource.WIKI) - .forIntervals(1, Granularities.DAY) - .startingAt("2013-01-01") - .withNumPartitions(10) - .eachOfSizeInMb(100); - wikiSegments.forEach(segmentsMetadataManager::addSegment); + createSegments(1, Granularities.DAY, DateTimes.of("2013-01-01")); scheduler.becomeLeader(); runScheduledJob(); - scheduler.startCompaction(TestDataSource.WIKI, createSupervisor()); + scheduler.startCompaction(dataSource, createSupervisorWithInlineSpec()); final CompactionSimulateResult simulateResult = scheduler.simulateRunWithConfigUpdate( new ClusterCompactionConfig(null, null, null, null, null) @@ -439,10 +460,10 @@ public void testSimulateRun() Assert.assertEquals( Collections.singletonList( Arrays.asList( - TestDataSource.WIKI, + dataSource, Intervals.of("2013-01-01/P1D"), - 10, - 1_000_000_000L, + 1, + 100_000_000L, 1, "not compacted yet" ) @@ -450,7 +471,7 @@ public void testSimulateRun() pendingCompactionTable.getRows() ); - scheduler.stopCompaction(TestDataSource.WIKI); + scheduler.stopCompaction(dataSource); final CompactionSimulateResult simulateResultWhenDisabled = scheduler.simulateRunWithConfigUpdate( new ClusterCompactionConfig(null, null, null, null, null) @@ -460,6 +481,254 @@ public void testSimulateRun() scheduler.stopBeingLeader(); } + @Test + public void test_ingestHourGranularity_andCompactToDayAndMonth_withInlineTemplates() + { + final int numDays = (int) new Duration(MAR_11.getMillis() - JAN_20.getMillis()).getStandardDays(); + createSegments(24 * numDays, Granularities.HOUR, JAN_20); + verifyNumSegmentsWith(Granularities.HOUR, 24 * numDays); + + // Compact everything going back to Mar 10 to DAY granularity, rest to MONTH + final DateTime now = DateTimes.nowUtc(); + final Period dayRulePeriod = new Period(now.getMillis() - MAR_11.minusDays(1).minusMinutes(1).getMillis()); + CascadingCompactionTemplate cascadingTemplate = new CascadingCompactionTemplate( + dataSource, + List.of( + new CompactionRule(dayRulePeriod, new InlineCompactionJobTemplate(createMatcher(Granularities.DAY))), + new CompactionRule(Period.ZERO, new InlineCompactionJobTemplate(createMatcher(Granularities.MONTH))) + ) + ); + + startCompactionWithSpec(cascadingTemplate); + runCompactionTasks(3); + verifyNumSegmentsWith(Granularities.DAY, 1); + verifyNumSegmentsWith(Granularities.MONTH, 2); + } + + @Test + public void test_ingestHourGranularity_andCompactToDayAndMonth_withCatalogTemplates() + { + final int numDays = (int) new Duration(MAR_11.getMillis() - JAN_20.getMillis()).getStandardDays(); + createSegments(24 * numDays, Granularities.HOUR, JAN_20); + verifyNumSegmentsWith(Granularities.HOUR, 24 * numDays); + + // Add compaction templates to catalog + final String dayGranularityTemplateId = saveTemplateToCatalog( + new InlineCompactionJobTemplate(createMatcher(Granularities.DAY)) + ); + final String monthGranularityTemplateId = saveTemplateToCatalog( + new InlineCompactionJobTemplate(createMatcher(Granularities.MONTH)) + ); + + // Compact everything going back to Mar 10 to DAY granularity, rest to MONTH + final DateTime now = DateTimes.nowUtc(); + final Period dayRulePeriod = new Period(now.getMillis() - MAR_11.minusDays(1).minusMinutes(1).getMillis()); + CascadingCompactionTemplate cascadingTemplate = new CascadingCompactionTemplate( + dataSource, + List.of( + new CompactionRule(dayRulePeriod, new CatalogCompactionJobTemplate(dayGranularityTemplateId, catalog)), + new CompactionRule(Period.ZERO, new CatalogCompactionJobTemplate(monthGranularityTemplateId, catalog)) + ) + ); + + startCompactionWithSpec(cascadingTemplate); + runCompactionTasks(3); + verifyNumSegmentsWith(Granularities.MONTH, 2); + verifyNumSegmentsWith(Granularities.DAY, 1); + verifyNumSegmentsWith(Granularities.HOUR, 24 * (numDays - 41)); + } + + @Test + public void test_ingestHourGranularity_andCompactToDayAndMonth_withCatalogMSQTemplates() + { + dataSource = TestDataSource.WIKI; + + final int numDays = (int) new Duration(MAR_11.getMillis() - JAN_20.getMillis()).getStandardDays(); + createSegments(24 * numDays, Granularities.HOUR, JAN_20); + verifyNumSegmentsWith(Granularities.HOUR, 24 * numDays); + + // Add compaction templates to catalog + final String sqlDayGranularity = + "REPLACE INTO ${dataSource}" + + " OVERWRITE WHERE __time >= TIMESTAMP '${startTimestamp}' AND __time < TIMESTAMP '${endTimestamp}'" + + " SELECT * FROM ${dataSource}" + + " WHERE __time BETWEEN '${startTimestamp}' AND '${endTimestamp}'" + + " PARTITIONED BY DAY"; + final String dayGranularityTemplateId = saveTemplateToCatalog( + new MSQCompactionJobTemplate( + new ClientSqlQuery(sqlDayGranularity, null, false, false, false, null, null), + createMatcher(Granularities.DAY) + ) + ); + final String sqlMonthGranularity = + "REPLACE INTO ${dataSource}" + + " OVERWRITE WHERE __time >= TIMESTAMP '${startTimestamp}' AND __time < TIMESTAMP '${endTimestamp}'" + + " SELECT * FROM ${dataSource}" + + " WHERE __time BETWEEN '${startTimestamp}' AND '${endTimestamp}'" + + " PARTITIONED BY MONTH"; + final String monthGranularityTemplateId = saveTemplateToCatalog( + new MSQCompactionJobTemplate( + new ClientSqlQuery(sqlMonthGranularity, null, false, false, false, null, null), + createMatcher(Granularities.MONTH) + ) + ); + + // Compact everything going back to Mar 10 to DAY granularity, rest to MONTH + final DateTime now = DateTimes.nowUtc(); + final Period dayRulePeriod = new Period(now.getMillis() - MAR_11.minusDays(1).minusMinutes(1).getMillis()); + CascadingCompactionTemplate cascadingTemplate = new CascadingCompactionTemplate( + dataSource, + List.of( + new CompactionRule(dayRulePeriod, new CatalogCompactionJobTemplate(dayGranularityTemplateId, catalog)), + new CompactionRule(Period.ZERO, new CatalogCompactionJobTemplate(monthGranularityTemplateId, catalog)) + ) + ); + + startCompactionWithSpec(cascadingTemplate); + + // Verify that 3 MSQ compaction jobs are submitted to the Broker + executor.finishNextPendingTask(); + + ArgumentCaptor queryArgumentCaptor = ArgumentCaptor.forClass(ClientSqlQuery.class); + Mockito.verify(brokerClient, Mockito.times(3)).submitSqlTask(queryArgumentCaptor.capture()); + + final List submittedJobs = queryArgumentCaptor.getAllValues(); + Assert.assertEquals(3, submittedJobs.size()); + + Assert.assertEquals( + "REPLACE INTO wiki" + + " OVERWRITE WHERE __time >= TIMESTAMP '2025-03-10 00:00:00.000' AND __time < TIMESTAMP '2025-03-11 00:00:00.000'" + + " SELECT * FROM wiki" + + " WHERE __time BETWEEN '2025-03-10 00:00:00.000' AND '2025-03-11 00:00:00.000'" + + " PARTITIONED BY DAY", + submittedJobs.get(0).getQuery() + ); + Assert.assertEquals( + "REPLACE INTO wiki" + + " OVERWRITE WHERE __time >= TIMESTAMP '2025-02-01 00:00:00.000' AND __time < TIMESTAMP '2025-03-01 00:00:00.000'" + + " SELECT * FROM wiki" + + " WHERE __time BETWEEN '2025-02-01 00:00:00.000' AND '2025-03-01 00:00:00.000'" + + " PARTITIONED BY MONTH", + submittedJobs.get(1).getQuery() + ); + Assert.assertEquals( + "REPLACE INTO wiki" + + " OVERWRITE WHERE __time >= TIMESTAMP '2025-01-01 00:00:00.000' AND __time < TIMESTAMP '2025-02-01 00:00:00.000'" + + " SELECT * FROM wiki" + + " WHERE __time BETWEEN '2025-01-01 00:00:00.000' AND '2025-02-01 00:00:00.000'" + + " PARTITIONED BY MONTH", + submittedJobs.get(2).getQuery() + ); + } + + private void verifyNumSegmentsWith(Granularity granularity, int numExpectedSegments) + { + long numMatchingSegments = segmentsMetadataManager + .getRecentDataSourcesSnapshot() + .getUsedSegmentsTimelinesPerDataSource() + .get(dataSource) + .findNonOvershadowedObjectsInInterval(Intervals.ETERNITY, Partitions.ONLY_COMPLETE) + .stream() + .filter(segment -> granularity.isAligned(segment.getInterval())) + .count(); + + Assert.assertEquals( + StringUtils.format("Segment with granularity[%s]", granularity), + numExpectedSegments, + (int) numMatchingSegments + ); + } + + private void createSegments(int numSegments, Granularity granularity, DateTime firstSegmentStart) + { + final List segments = CreateDataSegments + .ofDatasource(dataSource) + .forIntervals(numSegments, granularity) + .startingAt(firstSegmentStart) + .eachOfSizeInMb(100); + segmentStorage.commitSegments(Set.copyOf(segments), null); + } + + private String saveTemplateToCatalog(CompactionJobTemplate template) + { + final String templateId = IdUtils.getRandomId(); + final TableId tableId = TableId.of(TableId.INDEXING_TEMPLATE_SCHEMA, templateId); + + catalog.addSpec( + tableId, + new TableSpec( + IndexingTemplateDefn.TYPE, + Map.of(IndexingTemplateDefn.PROPERTY_PAYLOAD, template), + null + ) + ); + + ResolvedTable table = catalog.resolveTable(tableId); + Assert.assertNotNull(table); + + return templateId; + } + + private void startCompactionWithSpec(DataSourceCompactionConfig config) + { + scheduler.becomeLeader(); + final CompactionSupervisorSpec compactionSupervisor + = new CompactionSupervisorSpec(config, false, scheduler); + scheduler.startCompaction(config.getDataSource(), compactionSupervisor.createSupervisor()); + } + + private void runCompactionTasks(int expectedCount) + { + executor.finishNextPendingTask(); + + ArgumentCaptor taskArgumentCaptor = ArgumentCaptor.forClass(Task.class); + Mockito.verify(taskQueue, Mockito.times(expectedCount)).add(taskArgumentCaptor.capture()); + + for (Task task : taskArgumentCaptor.getAllValues()) { + Assert.assertTrue(task instanceof CompactionTask); + Assert.assertEquals(dataSource, task.getDataSource()); + runCompactionTask((CompactionTask) task); + } + + segmentStorage.getManager().forceUpdateDataSourcesSnapshot(); + } + + private void runCompactionTask(CompactionTask task) + { + // Determine interval and granularity and apply it to the timeline + final Interval compactionInterval = task.getIoConfig().getInputSpec().findInterval(dataSource); + final Granularity segmentGranularity = task.getSegmentGranularity(); + if (segmentGranularity == null) { + // Nothing to do + return; + } + + for (Interval replaceInterval : segmentGranularity.getIterable(compactionInterval)) { + // Create a single segment in this interval + DataSegment replaceSegment = CreateDataSegments + .ofDatasource(dataSource) + .forIntervals(1, segmentGranularity) + .startingAt(replaceInterval.getStart()) + .withVersion("2") + .eachOfSizeInMb(100) + .get(0); + segmentStorage.commitSegments(Set.of(replaceSegment), null); + } + } + + private static CompactionStateMatcher createMatcher(Granularity segmentGranularity) + { + return new CompactionStateMatcher( + null, + null, + null, + null, + null, + new UserCompactionTaskGranularityConfig(segmentGranularity, null, null), + null + ); + } + private void disableScheduler() { compactionConfig.set(new ClusterCompactionConfig(null, null, null, false, null)); @@ -475,12 +744,12 @@ private void runScheduledJob() executor.finishNextPendingTask(); } - private CompactionSupervisor createSupervisor() + private CompactionSupervisor createSupervisorWithInlineSpec() { return new CompactionSupervisorSpec( InlineSchemaDataSourceCompactionConfig .builder() - .forDataSource(TestDataSource.WIKI) + .forDataSource(dataSource) .withSkipOffsetFromLatest(Period.seconds(0)) .build(), false, 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 e7ecc49df8ba..80bb6919098c 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 @@ -30,6 +30,7 @@ import org.apache.druid.java.util.common.Pair; import org.apache.druid.metadata.PendingSegmentRecord; import org.apache.druid.metadata.ReplaceTaskLock; +import org.apache.druid.metadata.SegmentsMetadataManager; import org.apache.druid.metadata.SortOrder; import org.apache.druid.segment.SegmentSchemaMapping; import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; @@ -56,6 +57,11 @@ public class TestIndexerMetadataStorageCoordinator implements IndexerMetadataSto private int deleteSegmentsCount = 0; + public SegmentsMetadataManager getManager() + { + return segmentsMetadataManager; + } + @Override public Set retrieveAllDatasourceNames() { diff --git a/processing/src/main/java/org/apache/druid/java/util/common/Intervals.java b/processing/src/main/java/org/apache/druid/java/util/common/Intervals.java index 0d6acbfe6451..6ab3e6259a71 100644 --- a/processing/src/main/java/org/apache/druid/java/util/common/Intervals.java +++ b/processing/src/main/java/org/apache/druid/java/util/common/Intervals.java @@ -107,6 +107,10 @@ public static List complementOf(Interval interval) { if (isEternity(interval)) { return List.of(); + } else if (DateTimes.MIN.equals(interval.getStart())) { + return List.of(new Interval(interval.getEnd(), DateTimes.MAX)); + } else if (DateTimes.MAX.equals(interval.getEnd())) { + return List.of(new Interval(DateTimes.MIN, interval.getStart())); } else { return List.of( new Interval(DateTimes.MIN, interval.getStart()), 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 d1da394e290c..ebf151978447 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 @@ -119,7 +119,12 @@ public static CompactionStatus pending(String reasonFormat, Object... args) return new CompactionStatus(State.PENDING, StringUtils.format(reasonFormat, args)); } - private static CompactionStatus completeIfEqual( + /** + * Computes compaction status for the given field. The status is assumed to be + * COMPLETE (i.e. no further compaction is required) if the configured value + * of the field is null or equal to the current value. + */ + private static CompactionStatus completeIfNullOrEqual( String field, T configured, T current, @@ -212,12 +217,20 @@ static CompactionStatus compute( .findFirst().orElse(COMPLETE); } + @Nullable static PartitionsSpec findPartitionsSpecFromConfig(ClientCompactionTaskQueryTuningConfig tuningConfig) { final PartitionsSpec partitionsSpecFromTuningConfig = tuningConfig.getPartitionsSpec(); if (partitionsSpecFromTuningConfig == null) { - final long maxTotalRows = Configs.valueOrDefault(tuningConfig.getMaxTotalRows(), Long.MAX_VALUE); - return new DynamicPartitionsSpec(tuningConfig.getMaxRowsPerSegment(), maxTotalRows); + final Long maxTotalRows = tuningConfig.getMaxTotalRows(); + final Integer maxRowsPerSegment = tuningConfig.getMaxRowsPerSegment(); + + if (maxTotalRows == null && maxRowsPerSegment == null) { + // If not specified, return null so that partitionsSpec is not compared + return null; + } else { + return new DynamicPartitionsSpec(maxRowsPerSegment, maxTotalRows); + } } else if (partitionsSpecFromTuningConfig instanceof DynamicPartitionsSpec) { return new DynamicPartitionsSpec( partitionsSpecFromTuningConfig.getMaxRowsPerSegment(), @@ -317,7 +330,7 @@ private CompactionStatus partitionsSpecIsUpToDate() if (existingPartionsSpec instanceof DimensionRangePartitionsSpec) { existingPartionsSpec = getEffectiveRangePartitionsSpec((DimensionRangePartitionsSpec) existingPartionsSpec); } - return CompactionStatus.completeIfEqual( + return CompactionStatus.completeIfNullOrEqual( "partitionsSpec", findPartitionsSpecFromConfig(tuningConfig), existingPartionsSpec, @@ -327,7 +340,7 @@ private CompactionStatus partitionsSpecIsUpToDate() private CompactionStatus indexSpecIsUpToDate() { - return CompactionStatus.completeIfEqual( + return CompactionStatus.completeIfNullOrEqual( "indexSpec", Configs.valueOrDefault(tuningConfig.getIndexSpec(), IndexSpec.DEFAULT), lastCompactionState.getIndexSpec(), @@ -337,7 +350,7 @@ private CompactionStatus indexSpecIsUpToDate() private CompactionStatus projectionsAreUpToDate() { - return CompactionStatus.completeIfEqual( + return CompactionStatus.completeIfNullOrEqual( "projections", compactionConfig.getProjections(), lastCompactionState.getProjections(), @@ -400,7 +413,7 @@ private CompactionStatus rollupIsUpToDate() if (configuredGranularitySpec == null) { return COMPLETE; } else { - return CompactionStatus.completeIfEqual( + return CompactionStatus.completeIfNullOrEqual( "rollup", configuredGranularitySpec.isRollup(), existingGranularitySpec == null ? null : existingGranularitySpec.isRollup(), @@ -414,7 +427,7 @@ private CompactionStatus queryGranularityIsUpToDate() if (configuredGranularitySpec == null) { return COMPLETE; } else { - return CompactionStatus.completeIfEqual( + return CompactionStatus.completeIfNullOrEqual( "queryGranularity", configuredGranularitySpec.getQueryGranularity(), existingGranularitySpec == null ? null : existingGranularitySpec.getQueryGranularity(), @@ -444,7 +457,7 @@ private CompactionStatus dimensionsSpecIsUpToDate() compactionConfig.getTuningConfig() == null ? null : compactionConfig.getTuningConfig().getPartitionsSpec() ); { - return CompactionStatus.completeIfEqual( + return CompactionStatus.completeIfNullOrEqual( "dimensionsSpec", configuredDimensions, existingDimensions, @@ -485,7 +498,7 @@ private CompactionStatus transformSpecFilterIsUpToDate() } CompactionTransformSpec existingTransformSpec = lastCompactionState.getTransformSpec(); - return CompactionStatus.completeIfEqual( + return CompactionStatus.completeIfNullOrEqual( "transformSpec filter", compactionConfig.getTransformSpec().getFilter(), existingTransformSpec == null ? null : existingTransformSpec.getFilter(), diff --git a/server/src/test/java/org/apache/druid/server/compaction/CompactionStatusTest.java b/server/src/test/java/org/apache/druid/server/compaction/CompactionStatusTest.java index ff9d0e5af7c5..3694cce2121a 100644 --- a/server/src/test/java/org/apache/druid/server/compaction/CompactionStatusTest.java +++ b/server/src/test/java/org/apache/druid/server/compaction/CompactionStatusTest.java @@ -41,6 +41,7 @@ import org.apache.druid.server.coordinator.UserCompactionTaskQueryTuningConfig; import org.apache.druid.timeline.CompactionState; import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.SegmentId; import org.junit.Assert; import org.junit.Test; @@ -50,11 +51,8 @@ public class CompactionStatusTest { private static final DataSegment WIKI_SEGMENT - = DataSegment.builder() - .dataSource(TestDataSource.WIKI) - .interval(Intervals.of("2013-01-01/PT1H")) + = DataSegment.builder(SegmentId.of(TestDataSource.WIKI, Intervals.of("2013-01-01/PT1H"), "v1", 0)) .size(100_000_000L) - .version("v1") .build(); @Test @@ -62,8 +60,7 @@ public void testFindPartitionsSpecWhenGivenIsNull() { final ClientCompactionTaskQueryTuningConfig tuningConfig = ClientCompactionTaskQueryTuningConfig.from(null); - Assert.assertEquals( - new DynamicPartitionsSpec(null, Long.MAX_VALUE), + Assert.assertNull( CompactionStatus.findPartitionsSpecFromConfig(tuningConfig) ); } @@ -195,9 +192,14 @@ public void testStatusWhenLastCompactionStateIsNull() @Test public void testStatusWhenLastCompactionStateIsEmpty() { + final PartitionsSpec requiredPartitionsSpec = new DynamicPartitionsSpec(5_000_000, null); verifyCompactionStatusIsPendingBecause( new CompactionState(null, null, null, null, null, null, null), - InlineSchemaDataSourceCompactionConfig.builder().forDataSource(TestDataSource.WIKI).build(), + InlineSchemaDataSourceCompactionConfig + .builder() + .withTuningConfig(createTuningConfig(requiredPartitionsSpec, null)) + .forDataSource(TestDataSource.WIKI) + .build(), "'partitionsSpec' mismatch: required['dynamic' with 5,000,000 rows], current[null]" ); } @@ -205,12 +207,16 @@ public void testStatusWhenLastCompactionStateIsEmpty() @Test public void testStatusOnPartitionsSpecMismatch() { + final PartitionsSpec requiredPartitionsSpec = new DynamicPartitionsSpec(5_000_000, 0L); final PartitionsSpec currentPartitionsSpec = new DynamicPartitionsSpec(100, 0L); final CompactionState lastCompactionState = new CompactionState(currentPartitionsSpec, null, null, null, null, null, null); - final DataSourceCompactionConfig compactionConfig - = InlineSchemaDataSourceCompactionConfig.builder().forDataSource(TestDataSource.WIKI).build(); + final DataSourceCompactionConfig compactionConfig = InlineSchemaDataSourceCompactionConfig + .builder() + .withTuningConfig(createTuningConfig(requiredPartitionsSpec, null)) + .forDataSource(TestDataSource.WIKI) + .build(); verifyCompactionStatusIsPendingBecause( lastCompactionState, diff --git a/server/src/test/java/org/apache/druid/server/coordinator/CreateDataSegments.java b/server/src/test/java/org/apache/druid/server/coordinator/CreateDataSegments.java index 371d251b06af..380533c72905 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/CreateDataSegments.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/CreateDataSegments.java @@ -105,6 +105,10 @@ public CreateDataSegments withCompactionState(CompactionState compactionState) return this; } + /** + * Specifies the version to use for creating the segments. Default version is + * {@code "1"}. + */ public CreateDataSegments withVersion(String version) { this.version = version; From 032729e845eb9650861183b1dd722d2625b33573 Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Sun, 17 Aug 2025 10:19:58 +0530 Subject: [PATCH 07/25] Adjust rule boundaries to ensure maximum compaction --- .../compact/CompactionSupervisorTest.java | 102 ++++++------ .../compact/CascadingCompactionTemplate.java | 7 +- .../compact/CatalogCompactionJobTemplate.java | 28 +++- .../CompactionConfigBasedJobTemplate.java | 21 ++- .../indexing/compact/CompactionJobParams.java | 11 +- .../indexing/compact/CompactionJobQueue.java | 9 +- .../compact/CompactionJobTemplate.java | 15 +- .../indexing/compact/CompactionRule.java | 36 +++++ .../compact/CompactionStateMatcher.java | 7 + .../compact/InlineCompactionJobTemplate.java | 11 +- .../compact/MSQCompactionJobTemplate.java | 14 +- .../compact/OverlordCompactionScheduler.java | 34 ++-- .../indexing/overlord/TaskQueryTool.java | 2 +- .../druid/indexing/overlord/TaskQueue.java | 12 ++ .../OverlordCompactionSchedulerTest.java | 152 +++++++++++------- ...TestIndexerMetadataStorageCoordinator.java | 13 +- .../server/compaction/CompactionStatus.java | 11 +- .../compaction/CompactionStatusTracker.java | 12 +- .../coordinator/duty/CompactSegments.java | 1 + .../CompactionStatusTrackerTest.java | 2 +- .../NewestSegmentFirstPolicyTest.java | 4 +- .../server/metrics/LatchableEmitter.java | 16 +- 22 files changed, 362 insertions(+), 158 deletions(-) diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionSupervisorTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionSupervisorTest.java index 64679400aae3..6df35a499962 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 @@ -38,7 +38,6 @@ import org.apache.druid.indexing.compact.MSQCompactionJobTemplate; import org.apache.druid.indexing.overlord.Segments; import org.apache.druid.java.util.common.DateTimes; -import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.msq.guice.IndexerMemoryManagementModule; @@ -62,7 +61,6 @@ 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.timeline.DataSegment; import org.joda.time.DateTime; import org.joda.time.Period; import org.junit.jupiter.api.Assertions; @@ -74,15 +72,15 @@ public class CompactionSupervisorTest extends EmbeddedClusterTestBase { - protected final EmbeddedBroker broker = new EmbeddedBroker(); - protected final EmbeddedIndexer indexer = new EmbeddedIndexer() + private final EmbeddedBroker broker = new EmbeddedBroker(); + private final EmbeddedIndexer indexer = new EmbeddedIndexer() .setServerMemory(4_000_000_000L) .addProperty("druid.worker.capacity", "8"); - protected final EmbeddedOverlord overlord = new EmbeddedOverlord() + private final EmbeddedOverlord overlord = new EmbeddedOverlord() .addProperty("druid.manager.segments.pollDuration", "PT1s") .addProperty("druid.manager.segments.useIncrementalCache", "always"); - protected final EmbeddedHistorical historical = new EmbeddedHistorical(); - protected final EmbeddedCoordinator coordinator = new EmbeddedCoordinator() + private final EmbeddedHistorical historical = new EmbeddedHistorical(); + private final EmbeddedCoordinator coordinator = new EmbeddedCoordinator() .addProperty("druid.manager.segments.useIncrementalCache", "always"); @Override @@ -117,7 +115,7 @@ public void enableCompactionSupervisors() } @Test - public void test_ingestDayGranularity_andCompactToMonthGranularity() + public void test_ingestDayGranularity_andCompactToMonthGranularity_withInlineConfig() { // Ingest data at DAY granularity and verify runIngestionAtGranularity( @@ -126,7 +124,7 @@ public void test_ingestDayGranularity_andCompactToMonthGranularity() + "\n2025-06-02T00:00:00.000Z,trousers,210" + "\n2025-06-03T00:00:00.000Z,jeans,150" ); - verifyNumSegmentsWith(Granularities.DAY, 3); + Assertions.assertEquals(3, getNumSegmentsWith(Granularities.DAY)); // Create a compaction config with MONTH granularity InlineSchemaDataSourceCompactionConfig compactionConfig = @@ -142,8 +140,8 @@ public void test_ingestDayGranularity_andCompactToMonthGranularity() runCompactionWithSpec(compactionConfig); waitForCompactionTasksToFinish("compact", 1); - verifyNumSegmentsWith(Granularities.DAY, 0); - verifyNumSegmentsWith(Granularities.MONTH, 1); + Assertions.assertEquals(0, getNumSegmentsWith(Granularities.DAY)); + Assertions.assertEquals(1, getNumSegmentsWith(Granularities.MONTH)); } @Test @@ -158,17 +156,19 @@ public void test_ingestHourGranularity_andCompactToDayAndMonth_withInlineTemplat ) ); - ingestRecordsAtGranularity(1200, "HOUR"); + ingestSegments(1200, "HOUR"); runCompactionWithSpec(cascadingTemplate); waitForCompactionTasksToFinish("compact", 3); - verifyNumSegmentsWith(Granularities.DAY, 1); - verifyNumSegmentsWith(Granularities.MONTH, 2); + + Assertions.assertEquals(0, getNumSegmentsWith(Granularities.HOUR)); + Assertions.assertTrue(getNumSegmentsWith(Granularities.DAY) >= 1); + Assertions.assertTrue(getNumSegmentsWith(Granularities.MONTH) >= 1); } @Test public void test_ingestHourGranularity_andCompactToDayAndMonth_withCatalogTemplates() { - ingestRecordsAtGranularity(1200, "HOUR"); + ingestSegments(1200, "HOUR"); // Add compaction templates to catalog final String dayGranularityTemplateId = saveTemplateToCatalog( @@ -189,14 +189,16 @@ public void test_ingestHourGranularity_andCompactToDayAndMonth_withCatalogTempla runCompactionWithSpec(cascadingTemplate); waitForCompactionTasksToFinish("compact", 3); - verifyNumSegmentsWith(Granularities.DAY, 1); - verifyNumSegmentsWith(Granularities.MONTH, 2); + + Assertions.assertEquals(0, getNumSegmentsWith(Granularities.HOUR)); + Assertions.assertTrue(getNumSegmentsWith(Granularities.DAY) >= 1); + Assertions.assertTrue(getNumSegmentsWith(Granularities.MONTH) >= 1); } @Test public void test_ingestHourGranularity_andCompactToDayAndMonth_withCatalogMSQTemplates() { - ingestRecordsAtGranularity(1200, "HOUR"); + ingestSegments(1200, "HOUR"); // Add compaction templates to catalog final String sqlDayGranularity = @@ -235,26 +237,17 @@ public void test_ingestHourGranularity_andCompactToDayAndMonth_withCatalogMSQTem runCompactionWithSpec(cascadingTemplate); waitForCompactionTasksToFinish("query_controller", 3); - verifyNumSegmentsWith(Granularities.DAY, 1); - verifyNumSegmentsWith(Granularities.MONTH, 2); + + Assertions.assertEquals(0, getNumSegmentsWith(Granularities.HOUR)); + Assertions.assertTrue(getNumSegmentsWith(Granularities.DAY) >= 1); + Assertions.assertTrue(getNumSegmentsWith(Granularities.MONTH) >= 1); } - private void ingestRecordsAtGranularity(int numRecords, String granularityName) + private void ingestSegments(int numSegments, String granularityName) { - // Ingest data at specified granularity and verify - Granularity granularity = Granularity.fromString(granularityName); runIngestionAtGranularity( granularityName, - createHourlyInlineDataCsv(DateTimes.nowUtc(), numRecords) - ); - List segments = List.copyOf( - overlord.bindings() - .segmentsMetadataStorage() - .retrieveAllUsedSegments(dataSource, Segments.ONLY_VISIBLE) - ); - Assertions.assertEquals(numRecords, segments.size()); - segments.forEach( - segment -> Assertions.assertTrue(granularity.isAligned(segment.getInterval())) + createHourlyInlineDataCsv(DateTimes.nowUtc(), numSegments) ); } @@ -267,29 +260,35 @@ private void runCompactionWithSpec(DataSourceCompactionConfig config) private void waitForCompactionTasksToFinish(String taskType, int expectedCount) { + // Wait for all intervals to be compacted + overlord.latchableEmitter().waitForEvent( + event -> event.hasMetricName("interval/waitCompact/count") + .hasDimension(DruidMetrics.DATASOURCE, dataSource) + .hasValue(0) + ); + + // Wait for all submitted compaction jobs to finish + int numSubmittedTasks = overlord.latchableEmitter().getMetricValues( + "compact/task/count", + Map.of(DruidMetrics.DATASOURCE, dataSource) + ).stream().mapToInt(Number::intValue).sum(); overlord.latchableEmitter().waitForEventAggregate( event -> event.hasMetricName("task/run/time") .hasDimension(DruidMetrics.TASK_TYPE, taskType) .hasDimension(DruidMetrics.DATASOURCE, dataSource), - agg -> agg.hasCountAtLeast(expectedCount) + agg -> agg.hasCountAtLeast(numSubmittedTasks) ); } - private void verifyNumSegmentsWith(Granularity granularity, int numExpectedSegments) + private int getNumSegmentsWith(Granularity granularity) { - long numMatchingSegments = overlord + return (int) overlord .bindings() .segmentsMetadataStorage() .retrieveAllUsedSegments(dataSource, Segments.ONLY_VISIBLE) .stream() .filter(segment -> granularity.isAligned(segment.getInterval())) .count(); - - Assertions.assertEquals( - numExpectedSegments, - (int) numMatchingSegments, - StringUtils.format("Segment with granularity[%s]", granularity) - ); } private String saveTemplateToCatalog(CompactionJobTemplate template) @@ -318,9 +317,12 @@ private void runIngestionAtGranularity( String inlineDataCsv ) { - final String taskId = IdUtils.getRandomId(); - final IndexTask task = createIndexTaskForInlineData(taskId, granularity, inlineDataCsv); - + final IndexTask task = MoreResources.Task.BASIC_INDEX + .get() + .segmentGranularity(granularity) + .inlineInputSourceWithData(inlineDataCsv) + .dataSource(dataSource) + .withId(IdUtils.getRandomId()); cluster.callApi().runTask(task, overlord); } @@ -337,16 +339,6 @@ private String createHourlyInlineDataCsv(DateTime latestRecordTimestamp, int num return builder.toString(); } - private IndexTask createIndexTaskForInlineData(String taskId, String granularity, String inlineDataCsv) - { - return MoreResources.Task.BASIC_INDEX - .get() - .segmentGranularity(granularity) - .inlineInputSourceWithData(inlineDataCsv) - .dataSource(dataSource) - .withId(taskId); - } - private static CompactionStateMatcher createMatcher(Granularity segmentGranularity) { return new CompactionStateMatcher( diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CascadingCompactionTemplate.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CascadingCompactionTemplate.java index a474047c6f0c..0758c22f1e4d 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CascadingCompactionTemplate.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CascadingCompactionTemplate.java @@ -24,6 +24,7 @@ import org.apache.druid.data.input.InputSource; import org.apache.druid.data.input.impl.AggregateProjectionSpec; import org.apache.druid.data.output.OutputDestination; +import org.apache.druid.error.InvalidInput; import org.apache.druid.indexer.CompactionEngine; import org.apache.druid.indexing.input.DruidInputSource; import org.apache.druid.java.util.common.DateTimes; @@ -49,7 +50,7 @@ * This template never needs to be deserialized as a {@code BatchIndexingJobTemplate}, * only as a {@link DataSourceCompactionConfig} in {@link CompactionSupervisorSpec}. */ -public class CascadingCompactionTemplate extends CompactionJobTemplate implements DataSourceCompactionConfig +public class CascadingCompactionTemplate implements CompactionJobTemplate, DataSourceCompactionConfig { public static final String TYPE = "compactCascade"; @@ -64,6 +65,8 @@ public CascadingCompactionTemplate( { this.rules = rules; this.dataSource = Objects.requireNonNull(dataSource, "'dataSource' cannot be null"); + + InvalidInput.conditionalException(rules != null && !rules.isEmpty(), "'rules' cannot be empty"); } @Override @@ -95,7 +98,7 @@ public List createCompactionJobs( DateTime previousRuleStartTime = DateTimes.MAX; for (int i = 0; i < rules.size() - 1; ++i) { final CompactionRule rule = rules.get(i); - final DateTime ruleStartTime = currentTime.minus(rule.getPeriod()); + final DateTime ruleStartTime = rule.computeStartTime(currentTime, rules.get(i + 1)); final Interval ruleInterval = new Interval(ruleStartTime, previousRuleStartTime); allJobs.addAll( diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CatalogCompactionJobTemplate.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CatalogCompactionJobTemplate.java index 663faf34d98d..f97e738b8577 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CatalogCompactionJobTemplate.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CatalogCompactionJobTemplate.java @@ -30,14 +30,16 @@ import org.apache.druid.data.output.OutputDestination; import org.apache.druid.error.InvalidInput; import org.apache.druid.indexing.template.BatchIndexingJobTemplate; +import org.apache.druid.java.util.common.granularity.Granularity; +import javax.annotation.Nullable; import java.util.List; /** * Compaction template that delegates job creation to a template stored in the * Druid catalog. */ -public class CatalogCompactionJobTemplate extends CompactionJobTemplate +public class CatalogCompactionJobTemplate implements CompactionJobTemplate { public static final String TYPE = "compactCatalog"; @@ -63,23 +65,41 @@ public String getTemplateId() return templateId; } + @Nullable + @Override + public Granularity getSegmentGranularity() + { + final CompactionJobTemplate delegate = getDelegate(); + return delegate == null ? null : delegate.getSegmentGranularity(); + } + @Override public List createCompactionJobs( InputSource source, OutputDestination target, CompactionJobParams params ) + { + final CompactionJobTemplate delegate = getDelegate(); + if (delegate == null) { + return List.of(); + } else { + return delegate.createCompactionJobs(source, target, params); + } + } + + @Nullable + private CompactionJobTemplate getDelegate() { final ResolvedTable resolvedTable = catalog.resolveTable(tableId); if (resolvedTable == null) { - return List.of(); + return null; } - // Create jobs using the catalog template final BatchIndexingJobTemplate delegate = resolvedTable.decodeProperty(IndexingTemplateDefn.PROPERTY_PAYLOAD); if (delegate instanceof CompactionJobTemplate) { - return ((CompactionJobTemplate) delegate).createCompactionJobs(source, target, params); + return (CompactionJobTemplate) delegate; } else { throw InvalidInput.exception( "Template[%s] of type[%s] cannot be used for creating compaction tasks", 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 c431a43f8a86..4dd7f8cb4d4c 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.indexing.input.DruidDatasourceDestination; import org.apache.druid.indexing.input.DruidInputSource; 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.CompactionSlotManager; import org.apache.druid.server.compaction.DataSourceCompactibleSegmentIterator; @@ -37,6 +38,7 @@ import org.joda.time.Interval; import org.joda.time.Period; +import javax.annotation.Nullable; import java.util.ArrayList; import java.util.List; import java.util.Objects; @@ -46,7 +48,7 @@ * It is just a delegating template that uses a {@link DataSourceCompactionConfig} * to create compaction jobs. */ -public class CompactionConfigBasedJobTemplate extends CompactionJobTemplate +public class CompactionConfigBasedJobTemplate implements CompactionJobTemplate { private final DataSourceCompactionConfig config; @@ -70,6 +72,13 @@ public static CompactionConfigBasedJobTemplate create(String dataSource, Compact ); } + @Nullable + @Override + public Granularity getSegmentGranularity() + { + return config.getSegmentGranularity(); + } + @Override public List createCompactionJobs( InputSource source, @@ -107,7 +116,8 @@ public String getType() /** * Creates an iterator over the compactible candidate segments for the given - * params. + * params. Adds stats for segments that are already compacted to the + * {@link CompactionJobParams#getSnapshotBuilder()}. */ DataSourceCompactibleSegmentIterator getCompactibleCandidates( InputSource source, @@ -121,12 +131,17 @@ DataSourceCompactibleSegmentIterator getCompactibleCandidates( final Interval searchInterval = Objects.requireNonNull(ensureDruidInputSource(source).getInterval()); final SegmentTimeline timeline = params.getTimeline(config.getDataSource()); - return new DataSourceCompactibleSegmentIterator( + final DataSourceCompactibleSegmentIterator iterator = new DataSourceCompactibleSegmentIterator( config, timeline, Intervals.complementOf(searchInterval), new NewestSegmentFirstPolicy(null) ); + + // Collect stats for segments that are already compacted + iterator.getCompactedSegments().forEach(entry -> params.getSnapshotBuilder().addToComplete(entry)); + + return iterator; } private void validateInput(InputSource source) 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 588ed5aded06..3f4a867ff3e2 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,6 +20,7 @@ package org.apache.druid.indexing.compact; import org.apache.druid.indexing.template.JobParams; +import org.apache.druid.server.compaction.CompactionSnapshotBuilder; import org.apache.druid.server.coordinator.ClusterCompactionConfig; import org.apache.druid.timeline.SegmentTimeline; import org.joda.time.DateTime; @@ -32,16 +33,19 @@ public class CompactionJobParams implements JobParams private final DateTime scheduleStartTime; private final TimelineProvider timelineProvider; private final ClusterCompactionConfig clusterCompactionConfig; + private final CompactionSnapshotBuilder snapshotBuilder; public CompactionJobParams( DateTime scheduleStartTime, ClusterCompactionConfig clusterCompactionConfig, - TimelineProvider timelineProvider + TimelineProvider timelineProvider, + CompactionSnapshotBuilder snapshotBuilder ) { this.scheduleStartTime = scheduleStartTime; this.clusterCompactionConfig = clusterCompactionConfig; this.timelineProvider = timelineProvider; + this.snapshotBuilder = snapshotBuilder; } @Override @@ -60,6 +64,11 @@ public SegmentTimeline getTimeline(String dataSource) return timelineProvider.getTimelineForDataSource(dataSource); } + public CompactionSnapshotBuilder getSnapshotBuilder() + { + return snapshotBuilder; + } + @FunctionalInterface public interface TimelineProvider { 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 bb93e940add8..bc08583f97a2 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 @@ -88,6 +88,8 @@ public CompactionJobQueue( ObjectMapper objectMapper ) { + this.runStats = new CoordinatorRunStats(); + this.snapshotBuilder = new CompactionSnapshotBuilder(runStats); this.searchPolicy = clusterCompactionConfig.getCompactionPolicy(); this.queue = new PriorityQueue<>( (o1, o2) -> searchPolicy.compareCandidates(o1.getCandidate(), o2.getCandidate()) @@ -95,7 +97,8 @@ public CompactionJobQueue( this.jobParams = new CompactionJobParams( DateTimes.nowUtc(), clusterCompactionConfig, - dataSourcesSnapshot.getUsedSegmentsTimelinesPerDataSource()::get + dataSourcesSnapshot.getUsedSegmentsTimelinesPerDataSource()::get, + snapshotBuilder ); this.slotManager = new CompactionSlotManager( overlordClient, @@ -103,8 +106,6 @@ public CompactionJobQueue( clusterCompactionConfig ); - this.runStats = new CoordinatorRunStats(); - this.snapshotBuilder = new CompactionSnapshotBuilder(runStats); this.taskActionClientFactory = taskActionClientFactory; this.overlordClient = overlordClient; this.brokerClient = brokerClient; @@ -194,7 +195,7 @@ private boolean startJobIfPendingAndReady(CompactionJob job, CompactionCandidate final CompactionCandidate candidate = job.getCandidate(); final CompactionConfigValidationResult validationResult = validateCompactionJob(job); if (!validationResult.isValid()) { - log.error("Compaction job[%s] is invalid due to reason[%s].", job, validationResult.getReason()); + log.error("Skipping invalid compaction job[%s] due to reason[%s].", job, validationResult.getReason()); snapshotBuilder.addToSkipped(candidate); return false; } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionJobTemplate.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionJobTemplate.java index f6f000db81a2..0108adbf2d55 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionJobTemplate.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionJobTemplate.java @@ -27,23 +27,28 @@ import org.apache.druid.indexing.template.BatchIndexingJob; import org.apache.druid.indexing.template.BatchIndexingJobTemplate; import org.apache.druid.indexing.template.JobParams; +import org.apache.druid.java.util.common.granularity.Granularity; +import javax.annotation.Nullable; import java.util.List; import java.util.stream.Collectors; /** * Base indexing template for creating {@link CompactionJob}. */ -public abstract class CompactionJobTemplate implements BatchIndexingJobTemplate +public interface CompactionJobTemplate extends BatchIndexingJobTemplate { - abstract List createCompactionJobs( + List createCompactionJobs( InputSource source, OutputDestination destination, CompactionJobParams jobParams ); + @Nullable + Granularity getSegmentGranularity(); + @Override - public final List createJobs( + default List createJobs( InputSource source, OutputDestination destination, JobParams jobParams @@ -64,7 +69,7 @@ public final List createJobs( /** * Verifies that the input source is of type {@link DruidInputSource}. */ - public final DruidInputSource ensureDruidInputSource(InputSource inputSource) + default DruidInputSource ensureDruidInputSource(InputSource inputSource) { if (inputSource instanceof DruidInputSource) { return (DruidInputSource) inputSource; @@ -76,7 +81,7 @@ public final DruidInputSource ensureDruidInputSource(InputSource inputSource) /** * Verifies that the output destination is of type {@link DruidDatasourceDestination}. */ - public final DruidDatasourceDestination ensureDruidDataSourceDestination(OutputDestination destination) + default DruidDatasourceDestination ensureDruidDataSourceDestination(OutputDestination destination) { if (destination instanceof DruidDatasourceDestination) { return (DruidDatasourceDestination) destination; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionRule.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionRule.java index bbb877237b06..10099566ea71 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionRule.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionRule.java @@ -21,8 +21,13 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.druid.java.util.common.granularity.Granularity; +import org.joda.time.DateTime; import org.joda.time.Period; +/** + * A single rule used inside {@link CascadingCompactionTemplate}. + */ public class CompactionRule { private final Period period; @@ -49,4 +54,35 @@ public Period getPeriod() { return period; } + + /** + * Computes the start time of this rule by subtracting its period from the + * reference timestamp. + *

+ * If both this rule and the {@code beforeRule} explicitly specify a target + * segment granularity, the start time may be adjusted to ensure that there + * are no uncompacted gaps left in the timeline. + * + * @param referenceTime Current time when the rules are being evaluated + * @param beforeRule The rule before this one in chronological order + */ + public DateTime computeStartTime(DateTime referenceTime, CompactionRule beforeRule) + { + final Granularity granularity = template.getSegmentGranularity(); + final Granularity beforeGranularity = beforeRule.template.getSegmentGranularity(); + + final DateTime calculatedStartTime = referenceTime.minus(period); + + if (granularity == null || beforeGranularity == null) { + return calculatedStartTime; + } else { + // The gap can be filled only if it is bigger than the granularity of this rule. + // If beforeGranularity > granularity, gap would always be smaller than both + final DateTime beforeRuleEffectiveEnd = beforeGranularity.bucketStart(calculatedStartTime); + final DateTime possibleStartTime = granularity.bucketStart(beforeRuleEffectiveEnd); + return possibleStartTime.isBefore(beforeRuleEffectiveEnd) + ? granularity.increment(possibleStartTime) + : possibleStartTime; + } + } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionStateMatcher.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionStateMatcher.java index e43beedfe4b4..c8460be50025 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionStateMatcher.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionStateMatcher.java @@ -25,6 +25,7 @@ import org.apache.druid.data.input.impl.AggregateProjectionSpec; import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.indexer.partitions.PartitionsSpec; +import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.transform.CompactionTransformSpec; @@ -122,6 +123,12 @@ public List getProjections() return projections; } + @Nullable + public Granularity getSegmentGranularity() + { + return granularitySpec == null ? null : granularitySpec.getSegmentGranularity(); + } + @Override public boolean equals(Object o) { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/InlineCompactionJobTemplate.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/InlineCompactionJobTemplate.java index 73b5afff9873..e1bd3db9cc10 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/compact/InlineCompactionJobTemplate.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/InlineCompactionJobTemplate.java @@ -23,8 +23,10 @@ import com.fasterxml.jackson.annotation.JsonProperty; import org.apache.druid.data.input.InputSource; import org.apache.druid.data.output.OutputDestination; +import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.server.coordinator.InlineSchemaDataSourceCompactionConfig; +import javax.annotation.Nullable; import java.util.List; import java.util.Objects; @@ -37,7 +39,7 @@ * change the data itself (and not just its layout) and are thus not considered * compaction-compatible. */ -public class InlineCompactionJobTemplate extends CompactionJobTemplate +public class InlineCompactionJobTemplate implements CompactionJobTemplate { public static final String TYPE = "compactInline"; @@ -57,6 +59,13 @@ public CompactionStateMatcher getStateMatcher() return stateMatcher; } + @Nullable + @Override + public Granularity getSegmentGranularity() + { + return stateMatcher.getSegmentGranularity(); + } + @Override public List createCompactionJobs( InputSource source, diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/MSQCompactionJobTemplate.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/MSQCompactionJobTemplate.java index 522421853cf3..cab5c8480616 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/compact/MSQCompactionJobTemplate.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/MSQCompactionJobTemplate.java @@ -25,6 +25,7 @@ import org.apache.druid.data.output.OutputDestination; import org.apache.druid.indexing.input.DruidInputSource; import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.query.http.ClientSqlQuery; import org.apache.druid.server.compaction.CompactionCandidate; import org.apache.druid.server.compaction.CompactionSlotManager; @@ -34,6 +35,7 @@ import org.joda.time.format.DateTimeFormat; import org.joda.time.format.DateTimeFormatter; +import javax.annotation.Nullable; import java.util.ArrayList; import java.util.HashMap; import java.util.List; @@ -49,7 +51,7 @@ * Compaction is triggered for an interval only if the current compaction state * of the underlying segments does not match with the {@link #stateMatcher}. */ -public class MSQCompactionJobTemplate extends CompactionJobTemplate +public class MSQCompactionJobTemplate implements CompactionJobTemplate { public static final String TYPE = "compactMsq"; @@ -85,8 +87,15 @@ public CompactionStateMatcher getStateMatcher() return stateMatcher; } + @Nullable @Override - List createCompactionJobs( + public Granularity getSegmentGranularity() + { + return stateMatcher.getSegmentGranularity(); + } + + @Override + public List createCompactionJobs( InputSource source, OutputDestination destination, CompactionJobParams jobParams @@ -133,6 +142,7 @@ private ClientSqlQuery createQueryForJob(String dataSource, Interval compactionI context.putAll(sqlTemplate.getContext()); } context.put(CompactSegments.STORE_COMPACTION_STATE_KEY, true); + context.put(CompactSegments.COMPACTION_INTERVAL_KEY, compactionInterval); return new ClientSqlQuery( formattedSql, diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/OverlordCompactionScheduler.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/OverlordCompactionScheduler.java index a3190a23f9f7..a3f456e074ca 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/compact/OverlordCompactionScheduler.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/OverlordCompactionScheduler.java @@ -45,6 +45,7 @@ import org.apache.druid.java.util.emitter.service.ServiceEmitter; import org.apache.druid.java.util.emitter.service.ServiceMetricEvent; import org.apache.druid.metadata.SegmentsMetadataManager; +import org.apache.druid.metadata.SegmentsMetadataManagerConfig; import org.apache.druid.server.compaction.CompactionRunSimulator; import org.apache.druid.server.compaction.CompactionSimulateResult; import org.apache.druid.server.compaction.CompactionStatusTracker; @@ -88,7 +89,6 @@ public class OverlordCompactionScheduler implements CompactionScheduler { private static final Logger log = new Logger(OverlordCompactionScheduler.class); - private static final long SCHEDULE_PERIOD_SECONDS = 5; private static final Duration METRIC_EMISSION_PERIOD = Duration.standardMinutes(5); private final SegmentsMetadataManager segmentManager; @@ -128,6 +128,7 @@ public class OverlordCompactionScheduler implements CompactionScheduler * class itself. */ private final boolean shouldPollSegments; + private final long schedulePeriodMillis; private final Stopwatch sinceStatsEmitted = Stopwatch.createUnstarted(); @@ -137,6 +138,7 @@ public OverlordCompactionScheduler( GlobalTaskLockbox taskLockbox, TaskQueryTool taskQueryTool, SegmentsMetadataManager segmentManager, + SegmentsMetadataManagerConfig segmentManagerConfig, Supplier compactionConfigSupplier, CompactionStatusTracker statusTracker, CoordinatorOverlordServiceConfig coordinatorOverlordServiceConfig, @@ -148,6 +150,10 @@ public OverlordCompactionScheduler( ObjectMapper objectMapper ) { + final long segmentPollPeriodSeconds = + segmentManagerConfig.getPollDuration().toStandardDuration().getMillis(); + this.schedulePeriodMillis = Math.min(5_000, segmentPollPeriodSeconds); + this.segmentManager = segmentManager; this.emitter = emitter; this.objectMapper = objectMapper; @@ -206,7 +212,7 @@ public synchronized void stop() public void becomeLeader() { if (isLeader.compareAndSet(false, true)) { - scheduleOnExecutor(this::scheduledRun, SCHEDULE_PERIOD_SECONDS); + scheduleOnExecutor(this::scheduledRun, schedulePeriodMillis); } } @@ -318,10 +324,10 @@ private synchronized void scheduledRun() catch (Exception e) { log.error(e, "Error processing compaction queue. Continuing schedule."); } - scheduleOnExecutor(this::scheduledRun, SCHEDULE_PERIOD_SECONDS); + scheduleOnExecutor(this::scheduledRun, schedulePeriodMillis); } else { cleanupState(); - scheduleOnExecutor(this::scheduledRun, SCHEDULE_PERIOD_SECONDS * 4); + scheduleOnExecutor(this::scheduledRun, schedulePeriodMillis * 4); } } @@ -366,8 +372,11 @@ private void emitStatsIfPeriodHasElapsed(CoordinatorRunStats stats) stats.forEachStat(this::emitStat); sinceStatsEmitted.restart(); } else { - // Always emit number of submitted tasks - stats.forEachEntry(Stats.Compaction.SUBMITTED_TASKS, this::emitStat); + // Always emit number of submitted tasks and interval statuses + stats.forEachEntry(Stats.Compaction.SUBMITTED_TASKS, this::emitNonZeroStat); + stats.forEachEntry(Stats.Compaction.COMPACTED_INTERVALS, this::emitNonZeroStat); + stats.forEachEntry(Stats.Compaction.SKIPPED_INTERVALS, this::emitNonZeroStat); + stats.forEachEntry(Stats.Compaction.PENDING_INTERVALS, this::emitStat); } } @@ -412,6 +421,13 @@ public CompactionSimulateResult simulateRunWithConfigUpdate(ClusterCompactionCon } } + private void emitNonZeroStat(CoordinatorStat stat, RowKey rowKey, long value) + { + if (value > 0) { + emitStat(stat, rowKey, value); + } + } + private void emitStat(CoordinatorStat stat, RowKey rowKey, long value) { if (!stat.shouldEmit()) { @@ -448,7 +464,7 @@ private DataSourcesSnapshot getDatasourceSnapshot() return segmentManager.getRecentDataSourcesSnapshot(); } - private void scheduleOnExecutor(Runnable runnable, long delaySeconds) + private void scheduleOnExecutor(Runnable runnable, long delayMillis) { executor.schedule( () -> { @@ -459,8 +475,8 @@ private void scheduleOnExecutor(Runnable runnable, long delaySeconds) log.error(t, "Error while executing runnable"); } }, - delaySeconds, - TimeUnit.SECONDS + delayMillis, + TimeUnit.MILLISECONDS ); } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskQueryTool.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskQueryTool.java index 8e4830611bee..15a04e2d5877 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskQueryTool.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskQueryTool.java @@ -160,7 +160,7 @@ public List getAllActiveTasks() // Use a dummy created time as this is not used by the caller, just needs to be non-null final DateTime createdTime = DateTimes.nowUtc(); - final List activeTasks = taskQueue.get().getTasks(); + final List activeTasks = taskQueue.get().getActiveTasks(); for (Task task : activeTasks) { final Optional statusOptional = taskQueue.get().getTaskStatus(task.getId()); if (statusOptional.isPresent()) { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskQueue.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskQueue.java index f18c3b4465e5..1c9bea7ef72b 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskQueue.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskQueue.java @@ -1028,6 +1028,18 @@ public List getTasks() return activeTasks.values().stream().map(entry -> entry.task).collect(Collectors.toList()); } + /** + * List of all active tasks currently being managed by this TaskQueue. + */ + public List getActiveTasks() + { + return activeTasks.values() + .stream() + .filter(entry -> !entry.isComplete) + .map(entry -> entry.task) + .collect(Collectors.toList()); + } + /** * Returns the list of currently active tasks for the given datasource. */ diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/compact/OverlordCompactionSchedulerTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/compact/OverlordCompactionSchedulerTest.java index f83daf7bb5bd..59a8228bc3b1 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/compact/OverlordCompactionSchedulerTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/compact/OverlordCompactionSchedulerTest.java @@ -21,6 +21,7 @@ import com.fasterxml.jackson.databind.InjectableValues; import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Optional; import com.google.common.util.concurrent.Futures; import org.apache.druid.catalog.MapMetadataCatalog; import org.apache.druid.catalog.model.ResolvedTable; @@ -35,6 +36,7 @@ import org.apache.druid.guice.SupervisorModule; import org.apache.druid.indexer.CompactionEngine; import org.apache.druid.indexer.TaskState; +import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexing.common.SegmentCacheManagerFactory; import org.apache.druid.indexing.common.TimeChunkLock; import org.apache.druid.indexing.common.actions.RetrieveUsedSegmentsAction; @@ -64,6 +66,7 @@ import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.metrics.StubServiceEmitter; import org.apache.druid.metadata.SegmentsMetadataManager; +import org.apache.druid.metadata.SegmentsMetadataManagerConfig; import org.apache.druid.query.http.ClientSqlQuery; import org.apache.druid.query.http.SqlTaskStatus; import org.apache.druid.segment.TestDataSource; @@ -82,11 +85,11 @@ import org.apache.druid.server.coordinator.DruidCompactionConfig; import org.apache.druid.server.coordinator.InlineSchemaDataSourceCompactionConfig; import org.apache.druid.server.coordinator.UserCompactionTaskGranularityConfig; +import org.apache.druid.server.coordinator.duty.CompactSegments; import org.apache.druid.server.coordinator.simulate.BlockingExecutorService; import org.apache.druid.server.coordinator.simulate.WrappingScheduledExecutorService; import org.apache.druid.server.coordinator.stats.Stats; import org.apache.druid.timeline.DataSegment; -import org.apache.druid.timeline.Partitions; import org.joda.time.DateTime; import org.joda.time.Duration; import org.joda.time.Interval; @@ -100,6 +103,7 @@ import java.util.Arrays; import java.util.Collections; +import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Set; @@ -144,21 +148,33 @@ public class OverlordCompactionSchedulerTest private MapMetadataCatalog catalog; private OverlordCompactionScheduler scheduler; + private Map submittedMsqTaskIds; + @Before public void setUp() { dataSource = "wiki_" + IdUtils.getRandomId(); final TaskRunner taskRunner = Mockito.mock(TaskRunner.class); - Mockito.when(taskRunner.getTotalCapacity()).thenReturn(10); - Mockito.when(taskRunner.getMaximumCapacityWithAutoscale()).thenReturn(10); + Mockito.when(taskRunner.getTotalCapacity()).thenReturn(100); + Mockito.when(taskRunner.getMaximumCapacityWithAutoscale()).thenReturn(100); taskQueue = Mockito.mock(TaskQueue.class); catalog = new MapMetadataCatalog(OBJECT_MAPPER); + submittedMsqTaskIds = new HashMap<>(); brokerClient = Mockito.mock(BrokerClient.class); - Mockito.when(brokerClient.submitSqlTask(ArgumentMatchers.any(ClientSqlQuery.class))) - .thenReturn(Futures.immediateFuture(new SqlTaskStatus(IdUtils.getRandomId(), TaskState.RUNNING, null))); + Mockito.when(brokerClient.submitSqlTask(ArgumentMatchers.any(ClientSqlQuery.class))).thenAnswer( + arg -> { + final ClientSqlQuery query = arg.getArgument(0); + final Interval compactionInterval = + (Interval) query.getContext().get(CompactSegments.COMPACTION_INTERVAL_KEY); + + final String taskId = IdUtils.getRandomId(); + submittedMsqTaskIds.put(compactionInterval, taskId); + return Futures.immediateFuture(new SqlTaskStatus(taskId, TaskState.RUNNING, null)); + } + ); taskMaster = new TaskMaster(null, null); Assert.assertFalse(taskMaster.isHalfOrFullLeader()); @@ -179,7 +195,7 @@ public void setUp() segmentStorage = new TestIndexerMetadataStorageCoordinator(); segmentsMetadataManager = segmentStorage.getManager(); - compactionConfig = new AtomicReference<>(new ClusterCompactionConfig(1.0, 10, null, true, null)); + compactionConfig = new AtomicReference<>(new ClusterCompactionConfig(1.0, 100, null, true, null)); coordinatorOverlordServiceConfig = new CoordinatorOverlordServiceConfig(false, null); taskActionClientFactory = task -> new TaskActionClient() @@ -223,6 +239,7 @@ private void initScheduler() taskLockbox, new TaskQueryTool(taskStorage, taskLockbox, taskMaster, null, () -> defaultWorkerConfig), segmentsMetadataManager, + new SegmentsMetadataManagerConfig(null, null, null), () -> DruidCompactionConfig.empty().withClusterConfig(compactionConfig.get()), new CompactionStatusTracker(), coordinatorOverlordServiceConfig, @@ -500,8 +517,11 @@ public void test_ingestHourGranularity_andCompactToDayAndMonth_withInlineTemplat ); startCompactionWithSpec(cascadingTemplate); - runCompactionTasks(3); - verifyNumSegmentsWith(Granularities.DAY, 1); + runCompactionTasks(12); + + verifyFullyCompacted(); + verifyNumSegmentsWith(Granularities.HOUR, 0); + verifyNumSegmentsWith(Granularities.DAY, 10); verifyNumSegmentsWith(Granularities.MONTH, 2); } @@ -532,10 +552,12 @@ public void test_ingestHourGranularity_andCompactToDayAndMonth_withCatalogTempla ); startCompactionWithSpec(cascadingTemplate); - runCompactionTasks(3); + runCompactionTasks(12); + + verifyFullyCompacted(); + verifyNumSegmentsWith(Granularities.HOUR, 0); + verifyNumSegmentsWith(Granularities.DAY, 10); verifyNumSegmentsWith(Granularities.MONTH, 2); - verifyNumSegmentsWith(Granularities.DAY, 1); - verifyNumSegmentsWith(Granularities.HOUR, 24 * (numDays - 41)); } @Test @@ -585,49 +607,18 @@ public void test_ingestHourGranularity_andCompactToDayAndMonth_withCatalogMSQTem ); startCompactionWithSpec(cascadingTemplate); + runMSQCompactionJobs(12); - // Verify that 3 MSQ compaction jobs are submitted to the Broker - executor.finishNextPendingTask(); - - ArgumentCaptor queryArgumentCaptor = ArgumentCaptor.forClass(ClientSqlQuery.class); - Mockito.verify(brokerClient, Mockito.times(3)).submitSqlTask(queryArgumentCaptor.capture()); - - final List submittedJobs = queryArgumentCaptor.getAllValues(); - Assert.assertEquals(3, submittedJobs.size()); - - Assert.assertEquals( - "REPLACE INTO wiki" - + " OVERWRITE WHERE __time >= TIMESTAMP '2025-03-10 00:00:00.000' AND __time < TIMESTAMP '2025-03-11 00:00:00.000'" - + " SELECT * FROM wiki" - + " WHERE __time BETWEEN '2025-03-10 00:00:00.000' AND '2025-03-11 00:00:00.000'" - + " PARTITIONED BY DAY", - submittedJobs.get(0).getQuery() - ); - Assert.assertEquals( - "REPLACE INTO wiki" - + " OVERWRITE WHERE __time >= TIMESTAMP '2025-02-01 00:00:00.000' AND __time < TIMESTAMP '2025-03-01 00:00:00.000'" - + " SELECT * FROM wiki" - + " WHERE __time BETWEEN '2025-02-01 00:00:00.000' AND '2025-03-01 00:00:00.000'" - + " PARTITIONED BY MONTH", - submittedJobs.get(1).getQuery() - ); - Assert.assertEquals( - "REPLACE INTO wiki" - + " OVERWRITE WHERE __time >= TIMESTAMP '2025-01-01 00:00:00.000' AND __time < TIMESTAMP '2025-02-01 00:00:00.000'" - + " SELECT * FROM wiki" - + " WHERE __time BETWEEN '2025-01-01 00:00:00.000' AND '2025-02-01 00:00:00.000'" - + " PARTITIONED BY MONTH", - submittedJobs.get(2).getQuery() - ); + verifyFullyCompacted(); + verifyNumSegmentsWith(Granularities.HOUR, 0); + verifyNumSegmentsWith(Granularities.DAY, 10); + verifyNumSegmentsWith(Granularities.MONTH, 2); } private void verifyNumSegmentsWith(Granularity granularity, int numExpectedSegments) { - long numMatchingSegments = segmentsMetadataManager - .getRecentDataSourcesSnapshot() - .getUsedSegmentsTimelinesPerDataSource() - .get(dataSource) - .findNonOvershadowedObjectsInInterval(Intervals.ETERNITY, Partitions.ONLY_COMPLETE) + long numMatchingSegments = segmentStorage + .retrieveAllUsedSegments(dataSource, Segments.ONLY_VISIBLE) .stream() .filter(segment -> granularity.isAligned(segment.getInterval())) .count(); @@ -639,6 +630,17 @@ private void verifyNumSegmentsWith(Granularity granularity, int numExpectedSegme ); } + private void verifyFullyCompacted() + { + runScheduledJob(); + int numSegments = segmentStorage.retrieveAllUsedSegments(dataSource, Segments.ONLY_VISIBLE).size(); + + final AutoCompactionSnapshot snapshot = scheduler.getCompactionSnapshot(dataSource); + Assert.assertEquals(0, snapshot.getSegmentCountAwaitingCompaction()); + Assert.assertEquals(0, snapshot.getSegmentCountSkipped()); + Assert.assertEquals(numSegments, snapshot.getSegmentCountCompacted()); + } + private void createSegments(int numSegments, Granularity granularity, DateTime firstSegmentStart) { final List segments = CreateDataSegments @@ -679,7 +681,8 @@ private void startCompactionWithSpec(DataSourceCompactionConfig config) private void runCompactionTasks(int expectedCount) { - executor.finishNextPendingTask(); + runScheduledJob(); + serviceEmitter.verifySum("compact/task/count", expectedCount); ArgumentCaptor taskArgumentCaptor = ArgumentCaptor.forClass(Task.class); Mockito.verify(taskQueue, Mockito.times(expectedCount)).add(taskArgumentCaptor.capture()); @@ -687,17 +690,25 @@ private void runCompactionTasks(int expectedCount) for (Task task : taskArgumentCaptor.getAllValues()) { Assert.assertTrue(task instanceof CompactionTask); Assert.assertEquals(dataSource, task.getDataSource()); - runCompactionTask((CompactionTask) task); + + final CompactionTask compactionTask = (CompactionTask) task; + runCompactionTask( + compactionTask.getId(), + compactionTask.getIoConfig().getInputSpec().findInterval(dataSource), + compactionTask.getSegmentGranularity() + ); } segmentStorage.getManager().forceUpdateDataSourcesSnapshot(); } - private void runCompactionTask(CompactionTask task) + private void runCompactionTask(String taskId, Interval compactionInterval, Granularity segmentGranularity) { + // Update status of task in TaskQueue + Mockito.when(taskQueue.getTaskStatus(taskId)) + .thenReturn(Optional.of(TaskStatus.success(taskId))); + // Determine interval and granularity and apply it to the timeline - final Interval compactionInterval = task.getIoConfig().getInputSpec().findInterval(dataSource); - final Granularity segmentGranularity = task.getSegmentGranularity(); if (segmentGranularity == null) { // Nothing to do return; @@ -716,6 +727,39 @@ private void runCompactionTask(CompactionTask task) } } + private void runMSQCompactionJobs(int numExpectedJobs) + { + runScheduledJob(); + serviceEmitter.verifySum("compact/task/count", numExpectedJobs); + + ArgumentCaptor queryArgumentCaptor = ArgumentCaptor.forClass(ClientSqlQuery.class); + Mockito.verify(brokerClient, Mockito.times(numExpectedJobs)) + .submitSqlTask(queryArgumentCaptor.capture()); + + for (ClientSqlQuery job : queryArgumentCaptor.getAllValues()) { + final String query = job.getQuery(); + + final Granularity segmentGranularity; + if (query.contains("PARTITIONED BY DAY")) { + segmentGranularity = Granularities.DAY; + } else if (query.contains("PARTITIONED BY MONTH")) { + segmentGranularity = Granularities.MONTH; + } else { + segmentGranularity = Granularities.HOUR; + } + + final Interval compactionInterval = + (Interval) job.getContext().get(CompactSegments.COMPACTION_INTERVAL_KEY); + runCompactionTask( + submittedMsqTaskIds.get(compactionInterval), + compactionInterval, + segmentGranularity + ); + } + + segmentStorage.getManager().forceUpdateDataSourcesSnapshot(); + } + private static CompactionStateMatcher createMatcher(Granularity segmentGranularity) { return new CompactionStateMatcher( 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 80bb6919098c..ee9e85204e07 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 @@ -27,6 +27,7 @@ import org.apache.druid.indexing.overlord.SegmentPublishResult; import org.apache.druid.indexing.overlord.Segments; import org.apache.druid.jackson.DefaultObjectMapper; +import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.Pair; import org.apache.druid.metadata.PendingSegmentRecord; import org.apache.druid.metadata.ReplaceTaskLock; @@ -112,9 +113,15 @@ public boolean insertDataSourceMetadata(String supervisorId, DataSourceMetadata @Override public Set retrieveAllUsedSegments(String dataSource, Segments visibility) { - return Set.copyOf( - segmentsMetadataManager.getRecentDataSourcesSnapshot().getDataSource(dataSource).getSegments() - ); + if (visibility == Segments.ONLY_VISIBLE) { + return segmentsMetadataManager + .getRecentDataSourcesSnapshot() + .getAllUsedNonOvershadowedSegments(dataSource, Intervals.ETERNITY); + } else { + return Set.copyOf( + segmentsMetadataManager.getRecentDataSourcesSnapshot().getDataSource(dataSource).getSegments() + ); + } } @Override 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 ebf151978447..f1cf548e92dd 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 @@ -191,14 +191,19 @@ private static String asString(PartitionsSpec partitionsSpec) } } - static CompactionStatus skipped(String reasonFormat, Object... args) + public static CompactionStatus skipped(String reasonFormat, Object... args) { return new CompactionStatus(State.SKIPPED, StringUtils.format(reasonFormat, args)); } - static CompactionStatus running(String reasonForCompaction) + public static CompactionStatus running(String message) { - return new CompactionStatus(State.RUNNING, reasonForCompaction); + return new CompactionStatus(State.RUNNING, message); + } + + public static CompactionStatus complete(String message) + { + return new CompactionStatus(State.COMPLETE, message); } /** 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 94bfd41a2efd..db844d580b63 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 @@ -48,10 +48,6 @@ public class CompactionStatusTracker private final AtomicReference segmentSnapshotTime = new AtomicReference<>(); - public CompactionStatusTracker() - { - } - public void stop() { datasourceStatuses.clear(); @@ -94,7 +90,7 @@ public CompactionStatus computeCompactionStatus( // Skip intervals that already have a running task final CompactionTaskStatus lastTaskStatus = getLatestTaskStatus(candidate); if (lastTaskStatus != null && lastTaskStatus.getState() == TaskState.RUNNING) { - return CompactionStatus.skipped("Task for interval is already running"); + return CompactionStatus.running("Task for interval is already running"); } // Skip intervals that have been recently compacted if segment timeline is not updated yet @@ -102,7 +98,7 @@ public CompactionStatus computeCompactionStatus( if (lastTaskStatus != null && lastTaskStatus.getState() == TaskState.SUCCESS && snapshotTime != null && snapshotTime.isBefore(lastTaskStatus.getUpdatedTime())) { - return CompactionStatus.skipped( + return CompactionStatus.complete( "Segment timeline not updated since last compaction task succeeded" ); } @@ -115,6 +111,10 @@ public CompactionStatus computeCompactionStatus( return pendingStatus; } + /** + * Tracks the latest compaction status of the given compaction candidates. + * Used only by the {@link CompactionRunSimulator}. + */ public void onCompactionStatusComputed( CompactionCandidate candidateSegments, DataSourceCompactionConfig config 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 0803fb871641..1dca23090c26 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 @@ -76,6 +76,7 @@ public class CompactSegments implements CoordinatorCustomDuty * Must be the same as org.apache.druid.indexing.common.task.Tasks.STORE_COMPACTION_STATE_KEY */ public static final String STORE_COMPACTION_STATE_KEY = "storeCompactionState"; + public static final String COMPACTION_INTERVAL_KEY = "compactionInterval"; private static final String COMPACTION_REASON_KEY = "compactionReason"; private static final Logger LOG = new Logger(CompactSegments.class); diff --git a/server/src/test/java/org/apache/druid/server/compaction/CompactionStatusTrackerTest.java b/server/src/test/java/org/apache/druid/server/compaction/CompactionStatusTrackerTest.java index 14154d268217..4f82f9376fac 100644 --- a/server/src/test/java/org/apache/druid/server/compaction/CompactionStatusTrackerTest.java +++ b/server/src/test/java/org/apache/druid/server/compaction/CompactionStatusTrackerTest.java @@ -120,7 +120,7 @@ public void testComputeCompactionStatusForSuccessfulTask() statusTracker.onTaskFinished("task1", TaskStatus.success("task1")); status = statusTracker.computeCompactionStatus(candidateSegments, policy); - Assert.assertEquals(CompactionStatus.State.SKIPPED, status.getState()); + Assert.assertEquals(CompactionStatus.State.COMPLETE, status.getState()); Assert.assertEquals( "Segment timeline not updated since last compaction task succeeded", status.getReason() diff --git a/server/src/test/java/org/apache/druid/server/compaction/NewestSegmentFirstPolicyTest.java b/server/src/test/java/org/apache/druid/server/compaction/NewestSegmentFirstPolicyTest.java index dd48bd2aba19..739c28da4c29 100644 --- a/server/src/test/java/org/apache/druid/server/compaction/NewestSegmentFirstPolicyTest.java +++ b/server/src/test/java/org/apache/druid/server/compaction/NewestSegmentFirstPolicyTest.java @@ -1745,7 +1745,7 @@ public void testIteratorDoesNotReturnSegmentWithChangingAppendableIndexSpec() null, new OnheapIncrementalIndex.Spec(true), null, - 1000L, + null, null, partitionsSpec, IndexSpec.DEFAULT, @@ -1773,7 +1773,7 @@ public void testIteratorDoesNotReturnSegmentWithChangingAppendableIndexSpec() null, new OnheapIncrementalIndex.Spec(false), null, - 1000L, + null, null, partitionsSpec, IndexSpec.DEFAULT, diff --git a/server/src/test/java/org/apache/druid/server/metrics/LatchableEmitter.java b/server/src/test/java/org/apache/druid/server/metrics/LatchableEmitter.java index bd0b486f6dac..03fddf8a0126 100644 --- a/server/src/test/java/org/apache/druid/server/metrics/LatchableEmitter.java +++ b/server/src/test/java/org/apache/druid/server/metrics/LatchableEmitter.java @@ -238,6 +238,7 @@ public static class EventMatcher implements Predicate private String host; private String service; private String metricName; + private Long minMetricValue; private Long metricValue; private final Map dimensions = new HashMap<>(); @@ -256,7 +257,16 @@ public EventMatcher hasMetricName(String metricName) * Matches an event only if it has a metric value equal to or greater than * the given value. */ - public EventMatcher hasValueAtLeast(long metricValue) + public EventMatcher hasValueAtLeast(long minMetricValue) + { + this.minMetricValue = minMetricValue; + return this; + } + + /** + * Matches an event only if it has a metric value equal to the given value. + */ + public EventMatcher hasValue(long metricValue) { this.metricValue = metricValue; return this; @@ -294,7 +304,9 @@ public boolean test(ServiceMetricEvent event) { if (metricName != null && !event.getMetric().equals(metricName)) { return false; - } else if (metricValue != null && event.getValue().longValue() < metricValue) { + } else if (minMetricValue != null && event.getValue().longValue() < minMetricValue) { + return false; + } else if (metricValue != null && event.getValue().longValue() != metricValue) { return false; } else if (service != null && !service.equals(event.getService())) { return false; From 6dea5640ddc20a1eb78ffc8a137ae6af054fb318 Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Sun, 17 Aug 2025 11:27:08 +0530 Subject: [PATCH 08/25] Fix tests, comments --- .../compact/CompactionSupervisorTest.java | 10 +++---- .../indexing/KafkaClusterMetricsTest.java | 2 +- .../compact/CascadingCompactionTemplate.java | 21 ++++++++++++--- .../compact/CompactionStateMatcher.java | 15 +++++++---- .../compact/InlineCompactionJobTemplate.java | 24 +++++++---------- .../compact/MSQCompactionJobTemplate.java | 27 +++++++++---------- .../UserCompactionTaskGranularityConfig.java | 2 ++ 7 files changed, 58 insertions(+), 43 deletions(-) diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionSupervisorTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionSupervisorTest.java index 6df35a499962..c7532269fe22 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 @@ -138,7 +138,7 @@ public void test_ingestDayGranularity_andCompactToMonthGranularity_withInlineCon .build(); runCompactionWithSpec(compactionConfig); - waitForCompactionTasksToFinish("compact", 1); + waitForAllCompactionTasksToFinish("compact"); Assertions.assertEquals(0, getNumSegmentsWith(Granularities.DAY)); Assertions.assertEquals(1, getNumSegmentsWith(Granularities.MONTH)); @@ -158,7 +158,7 @@ public void test_ingestHourGranularity_andCompactToDayAndMonth_withInlineTemplat ingestSegments(1200, "HOUR"); runCompactionWithSpec(cascadingTemplate); - waitForCompactionTasksToFinish("compact", 3); + waitForAllCompactionTasksToFinish("compact"); Assertions.assertEquals(0, getNumSegmentsWith(Granularities.HOUR)); Assertions.assertTrue(getNumSegmentsWith(Granularities.DAY) >= 1); @@ -188,7 +188,7 @@ public void test_ingestHourGranularity_andCompactToDayAndMonth_withCatalogTempla ); runCompactionWithSpec(cascadingTemplate); - waitForCompactionTasksToFinish("compact", 3); + waitForAllCompactionTasksToFinish("compact"); Assertions.assertEquals(0, getNumSegmentsWith(Granularities.HOUR)); Assertions.assertTrue(getNumSegmentsWith(Granularities.DAY) >= 1); @@ -236,7 +236,7 @@ public void test_ingestHourGranularity_andCompactToDayAndMonth_withCatalogMSQTem ); runCompactionWithSpec(cascadingTemplate); - waitForCompactionTasksToFinish("query_controller", 3); + waitForAllCompactionTasksToFinish("query_controller"); Assertions.assertEquals(0, getNumSegmentsWith(Granularities.HOUR)); Assertions.assertTrue(getNumSegmentsWith(Granularities.DAY) >= 1); @@ -258,7 +258,7 @@ private void runCompactionWithSpec(DataSourceCompactionConfig config) cluster.callApi().postSupervisor(compactionSupervisor); } - private void waitForCompactionTasksToFinish(String taskType, int expectedCount) + private void waitForAllCompactionTasksToFinish(String taskType) { // Wait for all intervals to be compacted overlord.latchableEmitter().waitForEvent( diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaClusterMetricsTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaClusterMetricsTest.java index aa3667405bd0..090c7d0ccbe2 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaClusterMetricsTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaClusterMetricsTest.java @@ -251,7 +251,7 @@ public void test_ingestClusterMetrics_withConcurrentCompactionSupervisor_andSkip event -> event.hasMetricName("task/run/time") .hasDimension(DruidMetrics.TASK_TYPE, "compact") .hasDimension(DruidMetrics.TASK_STATUS, "SUCCESS"), - agg -> agg.hasCountAtLeast(2) + agg -> agg.hasCountAtLeast(10) ); // Verify that some segments have been upgraded due to Concurrent Append and Replace diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CascadingCompactionTemplate.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CascadingCompactionTemplate.java index 0758c22f1e4d..a63b5a0d1e40 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CascadingCompactionTemplate.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CascadingCompactionTemplate.java @@ -47,6 +47,21 @@ import java.util.Objects; /** + * Template to perform period-based cascading compaction. Contains a list of + * {@link CompactionRule} which divide the segment timeline into compactible + * intervals. Each rule specifies a period relative to the current time which is + * used to determine its applicable interval: + *

    + *
  • Rule 1: range = [now - p1, +inf)
  • + *
  • Rule 2: range = [now - p2, now - p1)
  • + *
  • ...
  • + *
  • Rule n: range = (-inf, now - p(n - 1))
  • + *
+ * + * If two adjacent rules explicitly specify a segment granularity, the boundary + * between them may be {@linkplain CompactionRule#computeStartTime adjusted} + * to ensure that there are no uncompacted gaps in the timeline. + *

* This template never needs to be deserialized as a {@code BatchIndexingJobTemplate}, * only as a {@link DataSourceCompactionConfig} in {@link CompactionSupervisorSpec}. */ @@ -102,7 +117,7 @@ public List createCompactionJobs( final Interval ruleInterval = new Interval(ruleStartTime, previousRuleStartTime); allJobs.addAll( - createJobs(rule.getTemplate(), ruleInterval, druidInputSource, destination, jobParams) + createJobsForSearchInterval(rule.getTemplate(), ruleInterval, druidInputSource, destination, jobParams) ); previousRuleStartTime = ruleStartTime; @@ -112,13 +127,13 @@ public List createCompactionJobs( final CompactionRule lastRule = rules.get(rules.size() - 1); final Interval lastRuleInterval = new Interval(DateTimes.MIN, previousRuleStartTime); allJobs.addAll( - createJobs(lastRule.getTemplate(), lastRuleInterval, druidInputSource, destination, jobParams) + createJobsForSearchInterval(lastRule.getTemplate(), lastRuleInterval, druidInputSource, destination, jobParams) ); return allJobs; } - private List createJobs( + private List createJobsForSearchInterval( CompactionJobTemplate template, Interval searchInterval, DruidInputSource inputSource, diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionStateMatcher.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionStateMatcher.java index c8460be50025..0ef1c898af4c 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionStateMatcher.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionStateMatcher.java @@ -37,12 +37,18 @@ import java.util.Objects; /** - * Instances of this class are used to determine if an interval needs to be - * compacted based on the current compaction state of the underlying segments. + * Target compacted state of segments used to determine if compaction is needed + * for an interval. An explicitly defined target state helps avoid superfluous + * compaction when only the job definition has changed. *

- * This class is mostly a duplicate of {@code CompactionState} but allows null - * values for all fields so that only non-null fields are used for matching. + * This class is mostly a duplicate of {@code CompactionState} but is kept + * separate to allow: + *

    + *
  • fields to be nullable so that only non-null fields are used for matching
  • + *
  • legacy "compaction-incompatible" fields to be removed
  • + *
*/ +@JsonInclude(JsonInclude.Include.NON_NULL) public class CompactionStateMatcher { private final PartitionsSpec partitionsSpec; @@ -116,7 +122,6 @@ public UserCompactionTaskGranularityConfig getGranularitySpec() } @JsonProperty - @JsonInclude(JsonInclude.Include.NON_NULL) @Nullable public List getProjections() { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/InlineCompactionJobTemplate.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/InlineCompactionJobTemplate.java index e1bd3db9cc10..16a74fbea74a 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/compact/InlineCompactionJobTemplate.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/InlineCompactionJobTemplate.java @@ -24,7 +24,6 @@ import org.apache.druid.data.input.InputSource; import org.apache.druid.data.output.OutputDestination; import org.apache.druid.java.util.common.granularity.Granularity; -import org.apache.druid.server.coordinator.InlineSchemaDataSourceCompactionConfig; import javax.annotation.Nullable; import java.util.List; @@ -33,37 +32,32 @@ /** * Template to create compaction jobs using inline specifications. This template * does not fetch any information from the Druid catalog while creating jobs. - *

- * This template does not contain all the fields supported by - * {@link InlineSchemaDataSourceCompactionConfig} since some of those fields may - * change the data itself (and not just its layout) and are thus not considered - * compaction-compatible. */ public class InlineCompactionJobTemplate implements CompactionJobTemplate { public static final String TYPE = "compactInline"; - private final CompactionStateMatcher stateMatcher; + private final CompactionStateMatcher targetState; @JsonCreator public InlineCompactionJobTemplate( - @JsonProperty("stateMatcher") CompactionStateMatcher stateMatcher + @JsonProperty("targetState") CompactionStateMatcher targetState ) { - this.stateMatcher = stateMatcher; + this.targetState = targetState; } @JsonProperty - public CompactionStateMatcher getStateMatcher() + public CompactionStateMatcher getTargetState() { - return stateMatcher; + return targetState; } @Nullable @Override public Granularity getSegmentGranularity() { - return stateMatcher.getSegmentGranularity(); + return targetState.getSegmentGranularity(); } @Override @@ -75,7 +69,7 @@ public List createCompactionJobs( { final String dataSource = ensureDruidInputSource(source).getDataSource(); return CompactionConfigBasedJobTemplate - .create(dataSource, stateMatcher) + .create(dataSource, targetState) .createCompactionJobs(source, destination, jobParams); } @@ -89,13 +83,13 @@ public boolean equals(Object object) return false; } InlineCompactionJobTemplate that = (InlineCompactionJobTemplate) object; - return Objects.equals(this.stateMatcher, that.stateMatcher); + return Objects.equals(this.targetState, that.targetState); } @Override public int hashCode() { - return Objects.hash(stateMatcher); + return Objects.hash(targetState); } @Override diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/MSQCompactionJobTemplate.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/MSQCompactionJobTemplate.java index cab5c8480616..80d97f36d8a1 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/compact/MSQCompactionJobTemplate.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/MSQCompactionJobTemplate.java @@ -43,13 +43,12 @@ import java.util.Objects; /** - * Compaction template that creates MSQ SQL jobs using a SQL template. The - * template provided by can contain template variables of the format - * {@code ${variableName}} for fields such as datasource name and interval start - * and end. + * Compaction template that creates MSQ SQL jobs using a templatized SQL with + * variables of the format {@code ${variableName}} for fields such as datasource + * name and start timestamp. *

* Compaction is triggered for an interval only if the current compaction state - * of the underlying segments does not match with the {@link #stateMatcher}. + * of the underlying segments DOES NOT match with the {@link #targetState}. */ public class MSQCompactionJobTemplate implements CompactionJobTemplate { @@ -63,16 +62,16 @@ public class MSQCompactionJobTemplate implements CompactionJobTemplate DateTimeFormat.forPattern("yyyy-MM-dd HH:mm:ss.SSS"); private final ClientSqlQuery sqlTemplate; - private final CompactionStateMatcher stateMatcher; + private final CompactionStateMatcher targetState; @JsonCreator public MSQCompactionJobTemplate( @JsonProperty("sqlTemplate") ClientSqlQuery sqlTemplate, - @JsonProperty("stateMatcher") CompactionStateMatcher stateMatcher + @JsonProperty("targetState") CompactionStateMatcher targetState ) { this.sqlTemplate = sqlTemplate; - this.stateMatcher = stateMatcher; + this.targetState = targetState; } @JsonProperty @@ -82,16 +81,16 @@ public ClientSqlQuery getSqlTemplate() } @JsonProperty - public CompactionStateMatcher getStateMatcher() + public CompactionStateMatcher getTargetState() { - return stateMatcher; + return targetState; } @Nullable @Override public Granularity getSegmentGranularity() { - return stateMatcher.getSegmentGranularity(); + return targetState.getSegmentGranularity(); } @Override @@ -106,7 +105,7 @@ public List createCompactionJobs( // Identify the compactible candidate segments final CompactionConfigBasedJobTemplate delegate = - CompactionConfigBasedJobTemplate.create(dataSource, stateMatcher); + CompactionConfigBasedJobTemplate.create(dataSource, targetState); final DataSourceCompactibleSegmentIterator candidateIterator = delegate.getCompactibleCandidates(source, destination, jobParams); @@ -179,13 +178,13 @@ public boolean equals(Object object) } MSQCompactionJobTemplate that = (MSQCompactionJobTemplate) object; return Objects.equals(sqlTemplate, that.sqlTemplate) - && Objects.equals(stateMatcher, that.stateMatcher); + && Objects.equals(targetState, that.targetState); } @Override public int hashCode() { - return Objects.hash(sqlTemplate, stateMatcher); + return Objects.hash(sqlTemplate, targetState); } @Override diff --git a/server/src/main/java/org/apache/druid/server/coordinator/UserCompactionTaskGranularityConfig.java b/server/src/main/java/org/apache/druid/server/coordinator/UserCompactionTaskGranularityConfig.java index 1db44319e562..4ad57f3adf02 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/UserCompactionTaskGranularityConfig.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/UserCompactionTaskGranularityConfig.java @@ -20,6 +20,7 @@ package org.apache.druid.server.coordinator; import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonInclude; import com.fasterxml.jackson.annotation.JsonProperty; import org.apache.druid.indexer.granularity.GranularitySpec; import org.apache.druid.java.util.common.granularity.Granularity; @@ -36,6 +37,7 @@ * Note that this class is not the same as {@link GranularitySpec}. This class simply holds Granularity configs * and pass it to compaction task spec. This class does not do bucketing, group events or knows how to partition data. */ +@JsonInclude(JsonInclude.Include.NON_NULL) public class UserCompactionTaskGranularityConfig { private final Granularity segmentGranularity; From 69a5425797f6e6833c2856b1f570b3cc2b2c3821 Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Sun, 17 Aug 2025 12:47:04 +0530 Subject: [PATCH 09/25] Add test with mixed templates --- .../compact/CompactionSupervisorTest.java | 72 ++++++++++++++++--- .../server/metrics/LatchableEmitter.java | 19 +++-- 2 files changed, 75 insertions(+), 16 deletions(-) diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionSupervisorTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionSupervisorTest.java index c7532269fe22..e7fc2dc8a9bd 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 @@ -61,6 +61,8 @@ 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.hamcrest.Matcher; +import org.hamcrest.Matchers; import org.joda.time.DateTime; import org.joda.time.Period; import org.junit.jupiter.api.Assertions; @@ -138,7 +140,7 @@ public void test_ingestDayGranularity_andCompactToMonthGranularity_withInlineCon .build(); runCompactionWithSpec(compactionConfig); - waitForAllCompactionTasksToFinish("compact"); + waitForAllCompactionTasksToFinish(); Assertions.assertEquals(0, getNumSegmentsWith(Granularities.DAY)); Assertions.assertEquals(1, getNumSegmentsWith(Granularities.MONTH)); @@ -156,9 +158,9 @@ public void test_ingestHourGranularity_andCompactToDayAndMonth_withInlineTemplat ) ); - ingestSegments(1200, "HOUR"); + ingestHourSegments(1000); runCompactionWithSpec(cascadingTemplate); - waitForAllCompactionTasksToFinish("compact"); + waitForAllCompactionTasksToFinish(); Assertions.assertEquals(0, getNumSegmentsWith(Granularities.HOUR)); Assertions.assertTrue(getNumSegmentsWith(Granularities.DAY) >= 1); @@ -168,7 +170,7 @@ public void test_ingestHourGranularity_andCompactToDayAndMonth_withInlineTemplat @Test public void test_ingestHourGranularity_andCompactToDayAndMonth_withCatalogTemplates() { - ingestSegments(1200, "HOUR"); + ingestHourSegments(1200); // Add compaction templates to catalog final String dayGranularityTemplateId = saveTemplateToCatalog( @@ -188,7 +190,7 @@ public void test_ingestHourGranularity_andCompactToDayAndMonth_withCatalogTempla ); runCompactionWithSpec(cascadingTemplate); - waitForAllCompactionTasksToFinish("compact"); + waitForAllCompactionTasksToFinish(); Assertions.assertEquals(0, getNumSegmentsWith(Granularities.HOUR)); Assertions.assertTrue(getNumSegmentsWith(Granularities.DAY) >= 1); @@ -198,7 +200,7 @@ public void test_ingestHourGranularity_andCompactToDayAndMonth_withCatalogTempla @Test public void test_ingestHourGranularity_andCompactToDayAndMonth_withCatalogMSQTemplates() { - ingestSegments(1200, "HOUR"); + ingestHourSegments(1200); // Add compaction templates to catalog final String sqlDayGranularity = @@ -236,17 +238,60 @@ public void test_ingestHourGranularity_andCompactToDayAndMonth_withCatalogMSQTem ); runCompactionWithSpec(cascadingTemplate); - waitForAllCompactionTasksToFinish("query_controller"); + waitForAllCompactionTasksToFinish(); Assertions.assertEquals(0, getNumSegmentsWith(Granularities.HOUR)); Assertions.assertTrue(getNumSegmentsWith(Granularities.DAY) >= 1); Assertions.assertTrue(getNumSegmentsWith(Granularities.MONTH) >= 1); } - private void ingestSegments(int numSegments, String granularityName) + @Test + public void test_ingestHourGranularity_andCompactToDayAndMonth_withMixedTemplates() + { + ingestHourSegments(1200); + + // Add compaction templates to catalog + final String sqlDayGranularity = + "REPLACE INTO ${dataSource}" + + " OVERWRITE WHERE __time >= TIMESTAMP '${startTimestamp}' AND __time < TIMESTAMP '${endTimestamp}'" + + " SELECT * FROM ${dataSource}" + + " WHERE __time BETWEEN '${startTimestamp}' AND '${endTimestamp}'" + + " PARTITIONED BY DAY"; + final MSQCompactionJobTemplate dayTemplate = new MSQCompactionJobTemplate( + new ClientSqlQuery(sqlDayGranularity, null, false, false, false, null, null), + createMatcher(Granularities.DAY) + ); + final String dayTemplateId = saveTemplateToCatalog(dayTemplate); + final String weekTemplateId = saveTemplateToCatalog( + new InlineCompactionJobTemplate(createMatcher(Granularities.WEEK)) + ); + final InlineCompactionJobTemplate monthTemplate = + new InlineCompactionJobTemplate(createMatcher(Granularities.MONTH)); + + // Compact last 1 day to DAY, next 14 days to WEEK, then 1 more DAY, rest to MONTH + CascadingCompactionTemplate cascadingTemplate = new CascadingCompactionTemplate( + dataSource, + List.of( + new CompactionRule(Period.days(1), new CatalogCompactionJobTemplate(dayTemplateId, null)), + new CompactionRule(Period.days(15), new CatalogCompactionJobTemplate(weekTemplateId, null)), + new CompactionRule(Period.days(16), dayTemplate), + new CompactionRule(Period.ZERO, monthTemplate) + ) + ); + + runCompactionWithSpec(cascadingTemplate); + waitForAllCompactionTasksToFinish(); + + Assertions.assertEquals(0, getNumSegmentsWith(Granularities.HOUR)); + Assertions.assertTrue(getNumSegmentsWith(Granularities.DAY) >= 1); + Assertions.assertTrue(getNumSegmentsWith(Granularities.WEEK) >= 1); + Assertions.assertTrue(getNumSegmentsWith(Granularities.MONTH) >= 1); + } + + private void ingestHourSegments(int numSegments) { runIngestionAtGranularity( - granularityName, + "HOUR", createHourlyInlineDataCsv(DateTimes.nowUtc(), numSegments) ); } @@ -258,7 +303,7 @@ private void runCompactionWithSpec(DataSourceCompactionConfig config) cluster.callApi().postSupervisor(compactionSupervisor); } - private void waitForAllCompactionTasksToFinish(String taskType) + private void waitForAllCompactionTasksToFinish() { // Wait for all intervals to be compacted overlord.latchableEmitter().waitForEvent( @@ -272,9 +317,14 @@ private void waitForAllCompactionTasksToFinish(String taskType) "compact/task/count", Map.of(DruidMetrics.DATASOURCE, dataSource) ).stream().mapToInt(Number::intValue).sum(); + + final Matcher taskTypeMatcher = Matchers.anyOf( + Matchers.equalTo("query_controller"), + Matchers.equalTo("compact") + ); overlord.latchableEmitter().waitForEventAggregate( event -> event.hasMetricName("task/run/time") - .hasDimension(DruidMetrics.TASK_TYPE, taskType) + .hasDimension(DruidMetrics.TASK_TYPE, taskTypeMatcher) .hasDimension(DruidMetrics.DATASOURCE, dataSource), agg -> agg.hasCountAtLeast(numSubmittedTasks) ); diff --git a/server/src/test/java/org/apache/druid/server/metrics/LatchableEmitter.java b/server/src/test/java/org/apache/druid/server/metrics/LatchableEmitter.java index 03fddf8a0126..01313e671b22 100644 --- a/server/src/test/java/org/apache/druid/server/metrics/LatchableEmitter.java +++ b/server/src/test/java/org/apache/druid/server/metrics/LatchableEmitter.java @@ -24,6 +24,8 @@ import org.apache.druid.java.util.emitter.core.Event; import org.apache.druid.java.util.emitter.service.ServiceMetricEvent; import org.apache.druid.java.util.metrics.StubServiceEmitter; +import org.hamcrest.Matcher; +import org.hamcrest.Matchers; import org.junit.jupiter.api.Timeout; import java.util.ArrayList; @@ -240,7 +242,7 @@ public static class EventMatcher implements Predicate private String metricName; private Long minMetricValue; private Long metricValue; - private final Map dimensions = new HashMap<>(); + private final Map> dimensions = new HashMap<>(); private final AtomicReference matchingEvent = new AtomicReference<>(); @@ -277,7 +279,16 @@ public EventMatcher hasValue(long metricValue) */ public EventMatcher hasDimension(String dimension, Object value) { - dimensions.put(dimension, value); + dimensions.put(dimension, Matchers.equalTo(value)); + return this; + } + + /** + * Matches an event if the value of the given dimension satisfies the matcher. + */ + public EventMatcher hasDimension(String dimension, Matcher matcher) + { + dimensions.put(dimension, matcher); return this; } @@ -315,9 +326,7 @@ public boolean test(ServiceMetricEvent event) } final boolean matches = dimensions.entrySet().stream().allMatch( - dimValue -> event.getUserDims() - .getOrDefault(dimValue.getKey(), "") - .equals(dimValue.getValue()) + dimValue -> dimValue.getValue().matches(event.getUserDims().get(dimValue.getKey())) ); if (matches) { From db2aebd54fa7a5571e89ce9927036b5e0467696a Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Sun, 17 Aug 2025 16:43:58 +0530 Subject: [PATCH 10/25] Clean up API for CompactionJobTemplate, fix test --- .../compact/CompactionSupervisorTest.java | 2 +- .../embedded/server/HighAvailabilityTest.java | 3 +- extensions-core/druid-catalog/pom.xml | 5 +++ .../compact/CascadingCompactionTemplate.java | 13 ++------ .../compact/CatalogCompactionJobTemplate.java | 8 ++--- .../CompactionConfigBasedJobTemplate.java | 19 ++++-------- .../compact/CompactionJobTemplate.java | 28 ++++++++++++++--- .../compact/CompactionSupervisor.java | 2 +- .../compact/InlineCompactionJobTemplate.java | 10 +++--- .../compact/MSQCompactionJobTemplate.java | 10 ++---- .../server/metrics/LatchableEmitter.java | 31 ++++++------------- 11 files changed, 60 insertions(+), 71 deletions(-) diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionSupervisorTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionSupervisorTest.java index e7fc2dc8a9bd..4a51c933204f 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 @@ -309,7 +309,7 @@ private void waitForAllCompactionTasksToFinish() overlord.latchableEmitter().waitForEvent( event -> event.hasMetricName("interval/waitCompact/count") .hasDimension(DruidMetrics.DATASOURCE, dataSource) - .hasValue(0) + .hasValue(Matchers.equalTo(0L)) ); // Wait for all submitted compaction jobs to finish diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/server/HighAvailabilityTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/server/HighAvailabilityTest.java index 303ad16afadf..50d8e16779e1 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/server/HighAvailabilityTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/server/HighAvailabilityTest.java @@ -43,6 +43,7 @@ import org.apache.druid.testing.embedded.EmbeddedRouter; import org.apache.druid.testing.embedded.indexing.Resources; import org.apache.druid.testing.embedded.junit5.EmbeddedClusterTestBase; +import org.hamcrest.Matchers; import org.jboss.netty.handler.codec.http.HttpMethod; import org.jboss.netty.handler.codec.http.HttpResponseStatus; import org.junit.jupiter.api.Assertions; @@ -128,7 +129,7 @@ public void test_switchLeader_andVerifyUsingSysTables() coordinator1.latchableEmitter().waitForEvent( event -> event.hasMetricName("segment/metadataCache/used/count") .hasDimension(DruidMetrics.DATASOURCE, dataSource) - .hasValueAtLeast(10) + .hasValue(Matchers.greaterThanOrEqualTo(10L)) ); // Run sys queries, switch leaders, repeat diff --git a/extensions-core/druid-catalog/pom.xml b/extensions-core/druid-catalog/pom.xml index 80d64d7b46e9..309ca004534b 100644 --- a/extensions-core/druid-catalog/pom.xml +++ b/extensions-core/druid-catalog/pom.xml @@ -187,6 +187,11 @@ easymock test + + org.hamcrest + hamcrest-all + test + org.apache.curator curator-test diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CascadingCompactionTemplate.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CascadingCompactionTemplate.java index a63b5a0d1e40..6465375e4eba 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CascadingCompactionTemplate.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CascadingCompactionTemplate.java @@ -21,9 +21,7 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import org.apache.druid.data.input.InputSource; import org.apache.druid.data.input.impl.AggregateProjectionSpec; -import org.apache.druid.data.output.OutputDestination; import org.apache.druid.error.InvalidInput; import org.apache.druid.indexer.CompactionEngine; import org.apache.druid.indexing.input.DruidInputSource; @@ -99,15 +97,12 @@ public List getRules() @Override public List createCompactionJobs( - InputSource source, - OutputDestination destination, + DruidInputSource source, CompactionJobParams jobParams ) { final List allJobs = new ArrayList<>(); - final DruidInputSource druidInputSource = ensureDruidInputSource(source); - // Include future dates in the first rule final DateTime currentTime = jobParams.getScheduleStartTime(); DateTime previousRuleStartTime = DateTimes.MAX; @@ -117,7 +112,7 @@ public List createCompactionJobs( final Interval ruleInterval = new Interval(ruleStartTime, previousRuleStartTime); allJobs.addAll( - createJobsForSearchInterval(rule.getTemplate(), ruleInterval, druidInputSource, destination, jobParams) + createJobsForSearchInterval(rule.getTemplate(), ruleInterval, source, jobParams) ); previousRuleStartTime = ruleStartTime; @@ -127,7 +122,7 @@ public List createCompactionJobs( final CompactionRule lastRule = rules.get(rules.size() - 1); final Interval lastRuleInterval = new Interval(DateTimes.MIN, previousRuleStartTime); allJobs.addAll( - createJobsForSearchInterval(lastRule.getTemplate(), lastRuleInterval, druidInputSource, destination, jobParams) + createJobsForSearchInterval(lastRule.getTemplate(), lastRuleInterval, source, jobParams) ); return allJobs; @@ -137,13 +132,11 @@ private List createJobsForSearchInterval( CompactionJobTemplate template, Interval searchInterval, DruidInputSource inputSource, - OutputDestination destination, CompactionJobParams jobParams ) { final List allJobs = template.createCompactionJobs( inputSource.withInterval(searchInterval), - destination, jobParams ); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CatalogCompactionJobTemplate.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CatalogCompactionJobTemplate.java index f97e738b8577..572ec60f4cc5 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CatalogCompactionJobTemplate.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CatalogCompactionJobTemplate.java @@ -26,9 +26,8 @@ import org.apache.druid.catalog.model.ResolvedTable; import org.apache.druid.catalog.model.TableId; import org.apache.druid.catalog.model.table.IndexingTemplateDefn; -import org.apache.druid.data.input.InputSource; -import org.apache.druid.data.output.OutputDestination; import org.apache.druid.error.InvalidInput; +import org.apache.druid.indexing.input.DruidInputSource; import org.apache.druid.indexing.template.BatchIndexingJobTemplate; import org.apache.druid.java.util.common.granularity.Granularity; @@ -75,8 +74,7 @@ public Granularity getSegmentGranularity() @Override public List createCompactionJobs( - InputSource source, - OutputDestination target, + DruidInputSource source, CompactionJobParams params ) { @@ -84,7 +82,7 @@ public List createCompactionJobs( if (delegate == null) { return List.of(); } else { - return delegate.createCompactionJobs(source, target, params); + return delegate.createCompactionJobs(source, params); } } 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 4dd7f8cb4d4c..8dfb857e9671 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 @@ -20,8 +20,6 @@ package org.apache.druid.indexing.compact; import org.apache.druid.client.indexing.ClientCompactionTaskQuery; -import org.apache.druid.data.input.InputSource; -import org.apache.druid.data.output.OutputDestination; import org.apache.druid.error.InvalidInput; import org.apache.druid.indexing.input.DruidDatasourceDestination; import org.apache.druid.indexing.input.DruidInputSource; @@ -81,12 +79,11 @@ public Granularity getSegmentGranularity() @Override public List createCompactionJobs( - InputSource source, - OutputDestination destination, + DruidInputSource source, CompactionJobParams params ) { - final DataSourceCompactibleSegmentIterator segmentIterator = getCompactibleCandidates(source, destination, params); + final DataSourceCompactibleSegmentIterator segmentIterator = getCompactibleCandidates(source, params); final List jobs = new ArrayList<>(); @@ -120,15 +117,13 @@ public String getType() * {@link CompactionJobParams#getSnapshotBuilder()}. */ DataSourceCompactibleSegmentIterator getCompactibleCandidates( - InputSource source, - OutputDestination destination, + DruidInputSource source, CompactionJobParams params ) { validateInput(source); - validateOutput(destination); - final Interval searchInterval = Objects.requireNonNull(ensureDruidInputSource(source).getInterval()); + final Interval searchInterval = Objects.requireNonNull(source.getInterval()); final SegmentTimeline timeline = params.getTimeline(config.getDataSource()); final DataSourceCompactibleSegmentIterator iterator = new DataSourceCompactibleSegmentIterator( @@ -144,9 +139,8 @@ DataSourceCompactibleSegmentIterator getCompactibleCandidates( return iterator; } - private void validateInput(InputSource source) + private void validateInput(DruidInputSource druidInputSource) { - final DruidInputSource druidInputSource = ensureDruidInputSource(source); if (!druidInputSource.getDataSource().equals(config.getDataSource())) { throw InvalidInput.exception( "Datasource[%s] in compaction config does not match datasource[%s] in input source", @@ -155,9 +149,8 @@ private void validateInput(InputSource source) } } - private void validateOutput(OutputDestination destination) + private void validateOutput(DruidDatasourceDestination druidDestination) { - final DruidDatasourceDestination druidDestination = ensureDruidDataSourceDestination(destination); if (!druidDestination.getDataSource().equals(config.getDataSource())) { throw InvalidInput.exception( "Datasource[%s] in compaction config does not match datasource[%s] in output destination", diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionJobTemplate.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionJobTemplate.java index 0108adbf2d55..5872fc8f66ab 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionJobTemplate.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionJobTemplate.java @@ -38,12 +38,20 @@ */ public interface CompactionJobTemplate extends BatchIndexingJobTemplate { + /** + * Creates compaction jobs with this template for the given datasource. + */ List createCompactionJobs( - InputSource source, - OutputDestination destination, + DruidInputSource source, CompactionJobParams jobParams ); + /** + * Granularity of segments created upon successful compaction. + * + * @return null only if this template does not change segment granularity upon + * successful compaction. + */ @Nullable Granularity getSegmentGranularity(); @@ -60,7 +68,17 @@ default List createJobs( jobParams ); } - return createCompactionJobs(source, destination, (CompactionJobParams) jobParams) + + final DruidInputSource druidInputSource = ensureDruidInputSource(source); + final DruidDatasourceDestination druidDestination = ensureDruidDataSourceDestination(destination); + if (!druidInputSource.getDataSource().equals(druidDestination.getDataSource())) { + throw InvalidInput.exception( + "Input datasource[%s] does not match output datasource[%s]", + druidInputSource.getDataSource(), druidDestination.getDataSource() + ); + } + + return createCompactionJobs(druidInputSource, (CompactionJobParams) jobParams) .stream() .map(job -> (BatchIndexingJob) job) .collect(Collectors.toList()); @@ -69,7 +87,7 @@ default List createJobs( /** * Verifies that the input source is of type {@link DruidInputSource}. */ - default DruidInputSource ensureDruidInputSource(InputSource inputSource) + static DruidInputSource ensureDruidInputSource(InputSource inputSource) { if (inputSource instanceof DruidInputSource) { return (DruidInputSource) inputSource; @@ -81,7 +99,7 @@ default DruidInputSource ensureDruidInputSource(InputSource inputSource) /** * Verifies that the output destination is of type {@link DruidDatasourceDestination}. */ - default DruidDatasourceDestination ensureDruidDataSourceDestination(OutputDestination destination) + static DruidDatasourceDestination ensureDruidDataSourceDestination(OutputDestination destination) { if (destination instanceof DruidDatasourceDestination) { return (DruidDatasourceDestination) destination; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionSupervisor.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionSupervisor.java index afe9341196b2..857916e6ec52 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionSupervisor.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionSupervisor.java @@ -77,7 +77,7 @@ public List createJobs( CompactionJobParams jobParams ) { - return supervisorSpec.getTemplate().createCompactionJobs(inputSource, destination, jobParams); + return supervisorSpec.getTemplate().createCompactionJobs(inputSource, jobParams); } @Override diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/InlineCompactionJobTemplate.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/InlineCompactionJobTemplate.java index 16a74fbea74a..43e4c6f8c267 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/compact/InlineCompactionJobTemplate.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/InlineCompactionJobTemplate.java @@ -21,8 +21,7 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import org.apache.druid.data.input.InputSource; -import org.apache.druid.data.output.OutputDestination; +import org.apache.druid.indexing.input.DruidInputSource; import org.apache.druid.java.util.common.granularity.Granularity; import javax.annotation.Nullable; @@ -62,15 +61,14 @@ public Granularity getSegmentGranularity() @Override public List createCompactionJobs( - InputSource source, - OutputDestination destination, + DruidInputSource source, CompactionJobParams jobParams ) { - final String dataSource = ensureDruidInputSource(source).getDataSource(); + final String dataSource = source.getDataSource(); return CompactionConfigBasedJobTemplate .create(dataSource, targetState) - .createCompactionJobs(source, destination, jobParams); + .createCompactionJobs(source, jobParams); } @Override diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/MSQCompactionJobTemplate.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/MSQCompactionJobTemplate.java index 80d97f36d8a1..ea7193e443be 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/compact/MSQCompactionJobTemplate.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/MSQCompactionJobTemplate.java @@ -21,8 +21,6 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import org.apache.druid.data.input.InputSource; -import org.apache.druid.data.output.OutputDestination; import org.apache.druid.indexing.input.DruidInputSource; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.Granularity; @@ -95,19 +93,17 @@ public Granularity getSegmentGranularity() @Override public List createCompactionJobs( - InputSource source, - OutputDestination destination, + DruidInputSource source, CompactionJobParams jobParams ) { - final DruidInputSource druidInputSource = ensureDruidInputSource(source); - final String dataSource = druidInputSource.getDataSource(); + final String dataSource = source.getDataSource(); // Identify the compactible candidate segments final CompactionConfigBasedJobTemplate delegate = CompactionConfigBasedJobTemplate.create(dataSource, targetState); final DataSourceCompactibleSegmentIterator candidateIterator = - delegate.getCompactibleCandidates(source, destination, jobParams); + delegate.getCompactibleCandidates(source, jobParams); // Create MSQ jobs for each candidate by interpolating the template variables final List jobs = new ArrayList<>(); diff --git a/server/src/test/java/org/apache/druid/server/metrics/LatchableEmitter.java b/server/src/test/java/org/apache/druid/server/metrics/LatchableEmitter.java index 01313e671b22..933710093935 100644 --- a/server/src/test/java/org/apache/druid/server/metrics/LatchableEmitter.java +++ b/server/src/test/java/org/apache/druid/server/metrics/LatchableEmitter.java @@ -240,9 +240,8 @@ public static class EventMatcher implements Predicate private String host; private String service; private String metricName; - private Long minMetricValue; - private Long metricValue; - private final Map> dimensions = new HashMap<>(); + private Matcher valueMatcher; + private final Map> dimensionMatchers = new HashMap<>(); private final AtomicReference matchingEvent = new AtomicReference<>(); @@ -256,21 +255,11 @@ public EventMatcher hasMetricName(String metricName) } /** - * Matches an event only if it has a metric value equal to or greater than - * the given value. + * Matches an event only if the metric value satisfies the given matcher. */ - public EventMatcher hasValueAtLeast(long minMetricValue) + public EventMatcher hasValue(Matcher valueMatcher) { - this.minMetricValue = minMetricValue; - return this; - } - - /** - * Matches an event only if it has a metric value equal to the given value. - */ - public EventMatcher hasValue(long metricValue) - { - this.metricValue = metricValue; + this.valueMatcher = valueMatcher; return this; } @@ -279,7 +268,7 @@ public EventMatcher hasValue(long metricValue) */ public EventMatcher hasDimension(String dimension, Object value) { - dimensions.put(dimension, Matchers.equalTo(value)); + dimensionMatchers.put(dimension, Matchers.equalTo(value)); return this; } @@ -288,7 +277,7 @@ public EventMatcher hasDimension(String dimension, Object value) */ public EventMatcher hasDimension(String dimension, Matcher matcher) { - dimensions.put(dimension, matcher); + dimensionMatchers.put(dimension, matcher); return this; } @@ -315,9 +304,7 @@ public boolean test(ServiceMetricEvent event) { if (metricName != null && !event.getMetric().equals(metricName)) { return false; - } else if (minMetricValue != null && event.getValue().longValue() < minMetricValue) { - return false; - } else if (metricValue != null && event.getValue().longValue() != metricValue) { + } else if (valueMatcher != null && !valueMatcher.matches(event.getValue())) { return false; } else if (service != null && !service.equals(event.getService())) { return false; @@ -325,7 +312,7 @@ public boolean test(ServiceMetricEvent event) return false; } - final boolean matches = dimensions.entrySet().stream().allMatch( + final boolean matches = dimensionMatchers.entrySet().stream().allMatch( dimValue -> dimValue.getValue().matches(event.getUserDims().get(dimValue.getKey())) ); From 3e084bf640b777cd2fff04f76c6a2ffd6c68991e Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Sun, 17 Aug 2025 21:32:35 +0530 Subject: [PATCH 11/25] Use SQL to query template definitions --- .../compact/CompactionSupervisorTest.java | 38 +++++- .../schema/DruidCalciteSchemaModule.java | 2 + .../schema/IndexingTemplateSchema.java | 84 +++++++++++++ .../schema/NamedIndexingTemplateSchema.java | 50 ++++++++ .../schema/DruidCalciteSchemaModuleTest.java | 65 ++++++---- .../schema/IndexingTemplateSchemaTest.java | 116 ++++++++++++++++++ 6 files changed, 329 insertions(+), 26 deletions(-) create mode 100644 sql/src/main/java/org/apache/druid/sql/calcite/schema/IndexingTemplateSchema.java create mode 100644 sql/src/main/java/org/apache/druid/sql/calcite/schema/NamedIndexingTemplateSchema.java create mode 100644 sql/src/test/java/org/apache/druid/sql/calcite/schema/IndexingTemplateSchemaTest.java diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionSupervisorTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionSupervisorTest.java index 4a51c933204f..537846facbf6 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,6 +19,7 @@ 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.catalog.model.ResolvedTable; @@ -38,6 +39,7 @@ import org.apache.druid.indexing.compact.MSQCompactionJobTemplate; import org.apache.druid.indexing.overlord.Segments; import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.msq.guice.IndexerMemoryManagementModule; @@ -48,6 +50,7 @@ import org.apache.druid.query.DruidMetrics; import org.apache.druid.query.http.ClientSqlQuery; import org.apache.druid.rpc.UpdateResponse; +import org.apache.druid.segment.TestHelper; import org.apache.druid.server.coordinator.ClusterCompactionConfig; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; import org.apache.druid.server.coordinator.InlineSchemaDataSourceCompactionConfig; @@ -76,8 +79,8 @@ public class CompactionSupervisorTest extends EmbeddedClusterTestBase { private final EmbeddedBroker broker = new EmbeddedBroker(); private final EmbeddedIndexer indexer = new EmbeddedIndexer() - .setServerMemory(4_000_000_000L) - .addProperty("druid.worker.capacity", "8"); + .setServerMemory(2_000_000_000L) + .addProperty("druid.worker.capacity", "4"); private final EmbeddedOverlord overlord = new EmbeddedOverlord() .addProperty("druid.manager.segments.pollDuration", "PT1s") .addProperty("druid.manager.segments.useIncrementalCache", "always"); @@ -116,6 +119,37 @@ public void enableCompactionSupervisors() Assertions.assertTrue(updateResponse.isSuccess()); } + @Test + public void test_saveCompactionTemplateToCatalog_andQueryDefinition() throws Exception + { + final String templateId = saveTemplateToCatalog( + new InlineCompactionJobTemplate(createMatcher(Granularities.DAY)) + ); + + final ClientSqlQuery query = new ClientSqlQuery( + StringUtils.format("SELECT * FROM index_template.%s", templateId), + "array", + true, + true, + true, + null, + null + ); + + final String result = cluster.callApi().onAnyBroker(b -> b.submitSqlQuery(query)); + final List> rows = TestHelper.JSON_MAPPER.readValue(result, new TypeReference<>() {}); + + Assertions.assertEquals( + List.of( + List.of("type", "payload"), + List.of("STRING", "COMPLEX"), + List.of("VARCHAR", "OTHER"), + List.of("compactInline", "{\"targetState\":{\"granularitySpec\":{\"segmentGranularity\":\"DAY\"}},\"type\":\"compactInline\"}") + ), + rows + ); + } + @Test public void test_ingestDayGranularity_andCompactToMonthGranularity_withInlineConfig() { diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/schema/DruidCalciteSchemaModule.java b/sql/src/main/java/org/apache/druid/sql/calcite/schema/DruidCalciteSchemaModule.java index 14e1fc068824..016972000c2a 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/schema/DruidCalciteSchemaModule.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/schema/DruidCalciteSchemaModule.java @@ -55,12 +55,14 @@ public void configure(Binder binder) binder.bind(SystemSchema.class).in(LazySingleton.class); binder.bind(InformationSchema.class).in(LazySingleton.class); binder.bind(LookupSchema.class).in(LazySingleton.class); + binder.bind(IndexingTemplateSchema.class).in(LazySingleton.class); // Binder to inject different schema to Calcite SqlBindings.addSchema(binder, NamedDruidSchema.class); SqlBindings.addSchema(binder, NamedSystemSchema.class); SqlBindings.addSchema(binder, NamedLookupSchema.class); SqlBindings.addSchema(binder, NamedViewSchema.class); + SqlBindings.addSchema(binder, NamedIndexingTemplateSchema.class); } @Provides diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/schema/IndexingTemplateSchema.java b/sql/src/main/java/org/apache/druid/sql/calcite/schema/IndexingTemplateSchema.java new file mode 100644 index 000000000000..86c471264aa7 --- /dev/null +++ b/sql/src/main/java/org/apache/druid/sql/calcite/schema/IndexingTemplateSchema.java @@ -0,0 +1,84 @@ +/* + * 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.sql.calcite.schema; + +import com.google.inject.Inject; +import org.apache.calcite.schema.Table; +import org.apache.druid.catalog.MetadataCatalog; +import org.apache.druid.catalog.model.ResolvedTable; +import org.apache.druid.catalog.model.TableId; +import org.apache.druid.catalog.model.table.IndexingTemplateDefn; +import org.apache.druid.query.InlineDataSource; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.column.RowSignature; +import org.apache.druid.sql.calcite.table.InlineTable; + +import javax.annotation.Nullable; +import java.util.Collections; +import java.util.Map; +import java.util.Set; + +/** + * Creates the {@link TableId#INDEXING_TEMPLATE_SCHEMA indexing template schema} + * for Druid SQL. + */ +public class IndexingTemplateSchema extends AbstractTableSchema +{ + private static final RowSignature ROW_SIGNATURE = + RowSignature.builder() + .add("type", ColumnType.STRING) + .add("payload", ColumnType.UNKNOWN_COMPLEX) + .build(); + + private final MetadataCatalog catalog; + + @Inject + public IndexingTemplateSchema(MetadataCatalog catalog) + { + this.catalog = catalog; + } + + @Override + @Nullable + public Table getTable(String name) + { + final TableId tableId = TableId.of(TableId.INDEXING_TEMPLATE_SCHEMA, name); + final ResolvedTable resolvedTable = catalog.resolveTable(tableId); + if (resolvedTable == null) { + return null; + } + + final Map template + = resolvedTable.mapProperty(IndexingTemplateDefn.PROPERTY_PAYLOAD); + final InlineDataSource dataSource = InlineDataSource.fromIterable( + Collections.singletonList( + new Object[]{template.get("type"), template} + ), + ROW_SIGNATURE + ); + return new InlineTable(dataSource); + } + + @Override + public Set getTableNames() + { + return catalog.tableNames(TableId.INDEXING_TEMPLATE_SCHEMA); + } +} diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/schema/NamedIndexingTemplateSchema.java b/sql/src/main/java/org/apache/druid/sql/calcite/schema/NamedIndexingTemplateSchema.java new file mode 100644 index 000000000000..230006085105 --- /dev/null +++ b/sql/src/main/java/org/apache/druid/sql/calcite/schema/NamedIndexingTemplateSchema.java @@ -0,0 +1,50 @@ +/* + * 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.sql.calcite.schema; + +import com.google.inject.Inject; +import org.apache.calcite.schema.Schema; +import org.apache.druid.catalog.model.TableId; + +/** + * Makes indexing template definitions accessible via SQL. + */ +public class NamedIndexingTemplateSchema implements NamedSchema +{ + private final IndexingTemplateSchema schema; + + @Inject + public NamedIndexingTemplateSchema(IndexingTemplateSchema schema) + { + this.schema = schema; + } + + @Override + public String getSchemaName() + { + return TableId.INDEXING_TEMPLATE_SCHEMA; + } + + @Override + public Schema getSchema() + { + return schema; + } +} diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/schema/DruidCalciteSchemaModuleTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/schema/DruidCalciteSchemaModuleTest.java index 23335615409f..50cdde4a5bd4 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/schema/DruidCalciteSchemaModuleTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/schema/DruidCalciteSchemaModuleTest.java @@ -28,6 +28,8 @@ import com.google.inject.Scopes; import com.google.inject.TypeLiteral; import com.google.inject.name.Names; +import org.apache.druid.catalog.MapMetadataCatalog; +import org.apache.druid.catalog.MetadataCatalog; import org.apache.druid.client.FilteredServerInventoryView; import org.apache.druid.client.TimelineServerView; import org.apache.druid.client.coordinator.CoordinatorClient; @@ -56,7 +58,7 @@ import org.easymock.EasyMock; import org.easymock.EasyMockExtension; import org.easymock.Mock; -import org.junit.Assert; +import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.ExtendWith; @@ -120,6 +122,7 @@ public void setUp() binder.bindScope(LazySingleton.class, Scopes.SINGLETON); binder.bind(LookupExtractorFactoryContainerProvider.class).toInstance(lookupReferencesManager); binder.bind(CatalogResolver.class).toInstance(CatalogResolver.NULL_RESOLVER); + binder.bind(MetadataCatalog.class).toInstance(new MapMetadataCatalog(objectMapper)); binder.bind(ServiceEmitter.class).toInstance(new ServiceEmitter("", "", null)); binder.bind(OverlordClient.class).to(NoopOverlordClient.class); binder.bind(CoordinatorClient.class).to(NoopCoordinatorClient.class); @@ -134,35 +137,40 @@ public void setUp() public void testDruidSchemaNameIsInjected() { String schemaName = injector.getInstance(Key.get(String.class, DruidSchemaName.class)); - Assert.assertEquals(DRUID_SCHEMA_NAME, schemaName); + Assertions.assertEquals(DRUID_SCHEMA_NAME, schemaName); } @Test public void testDruidSqlSchemaIsInjectedAsSingleton() { NamedDruidSchema namedDruidSchema = injector.getInstance(NamedDruidSchema.class); - Assert.assertNotNull(namedDruidSchema); + Assertions.assertNotNull(namedDruidSchema); NamedDruidSchema other = injector.getInstance(NamedDruidSchema.class); - Assert.assertSame(other, namedDruidSchema); + Assertions.assertSame(other, namedDruidSchema); } @Test public void testSystemSqlSchemaIsInjectedAsSingleton() { NamedSystemSchema namedSystemSchema = injector.getInstance(NamedSystemSchema.class); - Assert.assertNotNull(namedSystemSchema); + Assertions.assertNotNull(namedSystemSchema); NamedSystemSchema other = injector.getInstance(NamedSystemSchema.class); - Assert.assertSame(other, namedSystemSchema); + Assertions.assertSame(other, namedSystemSchema); } @Test public void testDruidCalciteSchemasAreInjected() { Set sqlSchemas = injector.getInstance(Key.get(new TypeLiteral<>() {})); - Set> expectedSchemas = - ImmutableSet.of(NamedSystemSchema.class, NamedDruidSchema.class, NamedLookupSchema.class, NamedViewSchema.class); - Assert.assertEquals(expectedSchemas.size(), sqlSchemas.size()); - Assert.assertEquals( + Set> expectedSchemas = Set.of( + NamedSystemSchema.class, + NamedDruidSchema.class, + NamedLookupSchema.class, + NamedViewSchema.class, + NamedIndexingTemplateSchema.class + ); + Assertions.assertEquals(expectedSchemas.size(), sqlSchemas.size()); + Assertions.assertEquals( expectedSchemas, sqlSchemas.stream().map(NamedSchema::getClass).collect(Collectors.toSet())); } @@ -171,36 +179,45 @@ public void testDruidCalciteSchemasAreInjected() public void testDruidSchemaIsInjectedAsSingleton() { DruidSchema schema = injector.getInstance(DruidSchema.class); - Assert.assertNotNull(schema); + Assertions.assertNotNull(schema); DruidSchema other = injector.getInstance(DruidSchema.class); - Assert.assertSame(other, schema); + Assertions.assertSame(other, schema); } @Test public void testSystemSchemaIsInjectedAsSingleton() { SystemSchema schema = injector.getInstance(SystemSchema.class); - Assert.assertNotNull(schema); + Assertions.assertNotNull(schema); SystemSchema other = injector.getInstance(SystemSchema.class); - Assert.assertSame(other, schema); + Assertions.assertSame(other, schema); } @Test public void testInformationSchemaIsInjectedAsSingleton() { InformationSchema schema = injector.getInstance(InformationSchema.class); - Assert.assertNotNull(schema); + Assertions.assertNotNull(schema); InformationSchema other = injector.getInstance(InformationSchema.class); - Assert.assertSame(other, schema); + Assertions.assertSame(other, schema); } @Test public void testLookupSchemaIsInjectedAsSingleton() { LookupSchema schema = injector.getInstance(LookupSchema.class); - Assert.assertNotNull(schema); + Assertions.assertNotNull(schema); LookupSchema other = injector.getInstance(LookupSchema.class); - Assert.assertSame(other, schema); + Assertions.assertSame(other, schema); + } + + @Test + public void testIndexingTemplateSchemaIsInjectedAsSingleton() + { + IndexingTemplateSchema schema = injector.getInstance(IndexingTemplateSchema.class); + Assertions.assertNotNull(schema); + IndexingTemplateSchema other = injector.getInstance(IndexingTemplateSchema.class); + Assertions.assertSame(other, schema); } @Test @@ -209,22 +226,22 @@ public void testRootSchemaAnnotatedIsInjectedAsSingleton() DruidSchemaCatalog rootSchema = injector.getInstance( Key.get(DruidSchemaCatalog.class, Names.named(DruidCalciteSchemaModule.INCOMPLETE_SCHEMA)) ); - Assert.assertNotNull(rootSchema); + Assertions.assertNotNull(rootSchema); DruidSchemaCatalog other = injector.getInstance( Key.get(DruidSchemaCatalog.class, Names.named(DruidCalciteSchemaModule.INCOMPLETE_SCHEMA)) ); - Assert.assertSame(other, rootSchema); + Assertions.assertSame(other, rootSchema); } @Test public void testRootSchemaIsInjectedAsSingleton() { DruidSchemaCatalog rootSchema = injector.getInstance(Key.get(DruidSchemaCatalog.class)); - Assert.assertNotNull(rootSchema); + Assertions.assertNotNull(rootSchema); DruidSchemaCatalog other = injector.getInstance( Key.get(DruidSchemaCatalog.class, Names.named(DruidCalciteSchemaModule.INCOMPLETE_SCHEMA)) ); - Assert.assertSame(other, rootSchema); + Assertions.assertSame(other, rootSchema); } @Test @@ -232,7 +249,7 @@ public void testRootSchemaIsInjectedAndHasInformationSchema() { DruidSchemaCatalog rootSchema = injector.getInstance(Key.get(DruidSchemaCatalog.class)); InformationSchema expectedSchema = injector.getInstance(InformationSchema.class); - Assert.assertNotNull(rootSchema); - Assert.assertSame(expectedSchema, rootSchema.getSubSchema("INFORMATION_SCHEMA").unwrap(InformationSchema.class)); + Assertions.assertNotNull(rootSchema); + Assertions.assertSame(expectedSchema, rootSchema.getSubSchema("INFORMATION_SCHEMA").unwrap(InformationSchema.class)); } } diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/schema/IndexingTemplateSchemaTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/schema/IndexingTemplateSchemaTest.java new file mode 100644 index 000000000000..915f4b6bb80c --- /dev/null +++ b/sql/src/test/java/org/apache/druid/sql/calcite/schema/IndexingTemplateSchemaTest.java @@ -0,0 +1,116 @@ +/* + * 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.sql.calcite.schema; + +import org.apache.druid.catalog.MapMetadataCatalog; +import org.apache.druid.catalog.model.TableId; +import org.apache.druid.catalog.model.TableSpec; +import org.apache.druid.catalog.model.table.IndexingTemplateDefn; +import org.apache.druid.query.InlineDataSource; +import org.apache.druid.segment.TestHelper; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.sql.calcite.table.InlineTable; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +import java.util.List; +import java.util.Map; +import java.util.Set; + +public class IndexingTemplateSchemaTest +{ + private IndexingTemplateSchema schema; + private MapMetadataCatalog catalog; + + @BeforeEach + public void setUp() + { + catalog = new MapMetadataCatalog(TestHelper.JSON_MAPPER); + schema = new IndexingTemplateSchema(catalog); + } + + @Test + public void test_getTableNames_returnsAllKnownTemplateIds() + { + Assertions.assertTrue(schema.getTableNames().isEmpty()); + + final String msqTemplate = "msq_1"; + final String compactTemplate = "compact_2"; + catalog.addSpec( + TableId.of(TableId.INDEXING_TEMPLATE_SCHEMA, msqTemplate), + new TableSpec(IndexingTemplateDefn.TYPE, null, null) + ); + catalog.addSpec( + TableId.of(TableId.INDEXING_TEMPLATE_SCHEMA, compactTemplate), + new TableSpec(IndexingTemplateDefn.TYPE, null, null) + ); + + final Set tableNames = schema.getTableNames(); + Assertions.assertEquals(Set.of(msqTemplate, compactTemplate), tableNames); + } + + @Test + public void test_getTable_returnsNull_forUnknownTable() + { + Assertions.assertNull(schema.getTable("msq")); + } + + @Test + public void test_getTable_returnsTable() + { + // Add a template to the catalog + final String msqTemplate = "msq_1"; + catalog.addSpec( + TableId.of(TableId.INDEXING_TEMPLATE_SCHEMA, msqTemplate), + new TableSpec( + IndexingTemplateDefn.TYPE, + Map.of("payload", Map.of("type", "msq", "granularity", "DAY")), + null + ) + ); + + final InlineTable inlineTable = Assertions.assertInstanceOf( + InlineTable.class, + schema.getTable(msqTemplate) + ); + Assertions.assertNotNull(inlineTable); + + final InlineDataSource dataSource = Assertions.assertInstanceOf( + InlineDataSource.class, + inlineTable.getDataSource() + ); + Assertions.assertEquals( + List.of("type", "payload"), + dataSource.getColumnNames() + ); + Assertions.assertEquals( + List.of(ColumnType.STRING, ColumnType.UNKNOWN_COMPLEX), + dataSource.getColumnTypes() + ); + + final List rows = dataSource.getRowsAsList(); + Assertions.assertEquals(1, rows.size()); + Assertions.assertArrayEquals( + rows.get(0), + new Object[]{"msq", Map.of("type", "msq", "granularity", "DAY")} + ); + } +} From 45fe7cb6ef57c34022cbfbe120800c93bb97516b Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Mon, 18 Aug 2025 10:00:09 +0530 Subject: [PATCH 12/25] Add CatalogCoreModule to quidem test setup --- .../org/apache/druid/sql/calcite/util/SqlTestFramework.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/util/SqlTestFramework.java b/sql/src/test/java/org/apache/druid/sql/calcite/util/SqlTestFramework.java index 4cd08014bb33..7d1b627b9f0f 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/util/SqlTestFramework.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/util/SqlTestFramework.java @@ -35,6 +35,7 @@ import org.apache.druid.collections.BlockingPool; import org.apache.druid.collections.NonBlockingPool; import org.apache.druid.guice.BuiltInTypesModule; +import org.apache.druid.guice.CatalogCoreModule; import org.apache.druid.guice.DruidInjectorBuilder; import org.apache.druid.guice.ExpressionModule; import org.apache.druid.guice.LazySingleton; @@ -409,6 +410,7 @@ public DruidModule getCoreModule() new LookylooModule(), new SegmentWranglerModule(), new ExpressionModule(), + new CatalogCoreModule(), DruidModule.override( new QueryRunnerFactoryModule(), new Module() From ec51aecb3d092047107d4869c24db6aa9b03c32b Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Mon, 18 Aug 2025 10:36:29 +0530 Subject: [PATCH 13/25] Fix SqlModuleTest --- sql/src/test/java/org/apache/druid/sql/guice/SqlModuleTest.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/sql/src/test/java/org/apache/druid/sql/guice/SqlModuleTest.java b/sql/src/test/java/org/apache/druid/sql/guice/SqlModuleTest.java index 28e8923f634e..f0057eb9c5d9 100644 --- a/sql/src/test/java/org/apache/druid/sql/guice/SqlModuleTest.java +++ b/sql/src/test/java/org/apache/druid/sql/guice/SqlModuleTest.java @@ -32,6 +32,7 @@ import org.apache.druid.client.coordinator.CoordinatorClient; import org.apache.druid.client.coordinator.NoopCoordinatorClient; import org.apache.druid.discovery.DruidNodeDiscoveryProvider; +import org.apache.druid.guice.CatalogCoreModule; import org.apache.druid.guice.DruidGuiceExtensions; import org.apache.druid.guice.JsonConfigurator; import org.apache.druid.guice.LazySingleton; @@ -182,6 +183,7 @@ private Injector makeInjectorWithProperties(final Properties props) new JacksonModule(), new PolicyModule(), new AuthenticatorMapperModule(), + new CatalogCoreModule(), binder -> { binder.bind(Validator.class).toInstance(Validation.buildDefaultValidatorFactory().getValidator()); binder.bind(JsonConfigurator.class).in(LazySingleton.class); From fcac67585bf9c645911c97143e61f073a18928db Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Wed, 1 Oct 2025 13:10:27 +0530 Subject: [PATCH 14/25] Add javadocs, simplify JobParams --- .../compact/CatalogCompactionJobTemplate.java | 13 ++------- .../CompactionConfigBasedJobTemplate.java | 8 +++++- .../indexing/compact/CompactionJobParams.java | 28 +++++++++++++------ .../druid/indexing/template/JobParams.java | 18 ++++++++++-- 4 files changed, 45 insertions(+), 22 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CatalogCompactionJobTemplate.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CatalogCompactionJobTemplate.java index 572ec60f4cc5..15fb761ffae5 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CatalogCompactionJobTemplate.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CatalogCompactionJobTemplate.java @@ -68,8 +68,7 @@ public String getTemplateId() @Override public Granularity getSegmentGranularity() { - final CompactionJobTemplate delegate = getDelegate(); - return delegate == null ? null : delegate.getSegmentGranularity(); + return getDelegate().getSegmentGranularity(); } @Override @@ -78,20 +77,14 @@ public List createCompactionJobs( CompactionJobParams params ) { - final CompactionJobTemplate delegate = getDelegate(); - if (delegate == null) { - return List.of(); - } else { - return delegate.createCompactionJobs(source, params); - } + return getDelegate().createCompactionJobs(source, params); } - @Nullable private CompactionJobTemplate getDelegate() { final ResolvedTable resolvedTable = catalog.resolveTable(tableId); if (resolvedTable == null) { - return null; + throw InvalidInput.exception("Could not find table[%s] in the catalog", tableId); } final BatchIndexingJobTemplate delegate 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 8dfb857e9671..1d1599848dad 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 @@ -20,6 +20,7 @@ package org.apache.druid.indexing.compact; import org.apache.druid.client.indexing.ClientCompactionTaskQuery; +import org.apache.druid.error.DruidException; import org.apache.druid.error.InvalidInput; import org.apache.druid.indexing.input.DruidDatasourceDestination; import org.apache.druid.indexing.input.DruidInputSource; @@ -108,7 +109,12 @@ public List createCompactionJobs( @Override public String getType() { - throw new UnsupportedOperationException("This template type cannot be serialized"); + throw DruidException.defensive( + "This template cannot be serialized. It is an adapter used to create jobs" + + " using a legacy DataSourceCompactionConfig. Do not use this template" + + " in a supervisor spec directly. Use types [compactCatalog], [compactMsq]" + + " or [compactInline] instead." + ); } /** 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 3f4a867ff3e2..551c26f7742f 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 @@ -28,9 +28,8 @@ /** * Parameters used while creating a {@link CompactionJob} using a {@link CompactionJobTemplate}. */ -public class CompactionJobParams implements JobParams +public class CompactionJobParams extends JobParams { - private final DateTime scheduleStartTime; private final TimelineProvider timelineProvider; private final ClusterCompactionConfig clusterCompactionConfig; private final CompactionSnapshotBuilder snapshotBuilder; @@ -42,28 +41,39 @@ public CompactionJobParams( CompactionSnapshotBuilder snapshotBuilder ) { - this.scheduleStartTime = scheduleStartTime; + super(scheduleStartTime); this.clusterCompactionConfig = clusterCompactionConfig; this.timelineProvider = timelineProvider; this.snapshotBuilder = snapshotBuilder; } - @Override - public DateTime getScheduleStartTime() - { - return scheduleStartTime; - } - + /** + * Cluster-level compaction config containing details such as the engine, + * compaction search policy, etc. to use while creating {@link CompactionJob}. + */ public ClusterCompactionConfig getClusterCompactionConfig() { return clusterCompactionConfig; } + /** + * Provides the full {@link SegmentTimeline} of used segments for the given + * datasource. This timeline is used to identify eligible intervals for which + * compaction jobs should be created. + */ public SegmentTimeline getTimeline(String dataSource) { return timelineProvider.getTimelineForDataSource(dataSource); } + /** + * 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 + * candidate intervals are identified as compacted, skipped or pending, they + * should be updated in this snapshot builder by invoking + * {@link CompactionSnapshotBuilder#addToComplete}, {@link CompactionSnapshotBuilder#addToSkipped} + * and {@link CompactionSnapshotBuilder#addToPending} respectively. + */ public CompactionSnapshotBuilder getSnapshotBuilder() { return snapshotBuilder; diff --git a/server/src/main/java/org/apache/druid/indexing/template/JobParams.java b/server/src/main/java/org/apache/druid/indexing/template/JobParams.java index 775b63b04c6b..e365ad757979 100644 --- a/server/src/main/java/org/apache/druid/indexing/template/JobParams.java +++ b/server/src/main/java/org/apache/druid/indexing/template/JobParams.java @@ -24,7 +24,21 @@ /** * Provides parameters required to create a {@link BatchIndexingJob}. */ -public interface JobParams +public class JobParams { - DateTime getScheduleStartTime(); + private final DateTime scheduleStartTime; + + public JobParams(DateTime scheduleStartTime) + { + this.scheduleStartTime = scheduleStartTime; + } + + /** + * Timestamp denoting the start of the current run of the scheduler which has + * triggered creation of jobs using these {@link JobParams}. + */ + public DateTime getScheduleStartTime() + { + return scheduleStartTime; + } } From bd490e1bc450d8568f35f110891380dfd29cf67e Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Mon, 27 Oct 2025 12:26:04 +0530 Subject: [PATCH 15/25] Minor fixes --- .../indexing/compact/CatalogCompactionJobTemplate.java | 3 ++- .../druid/server/compaction/CompactionStatusTest.java | 10 ++++------ 2 files changed, 6 insertions(+), 7 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CatalogCompactionJobTemplate.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CatalogCompactionJobTemplate.java index 15fb761ffae5..e34dbb2b1d88 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CatalogCompactionJobTemplate.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CatalogCompactionJobTemplate.java @@ -27,6 +27,7 @@ import org.apache.druid.catalog.model.TableId; import org.apache.druid.catalog.model.table.IndexingTemplateDefn; import org.apache.druid.error.InvalidInput; +import org.apache.druid.error.NotFound; import org.apache.druid.indexing.input.DruidInputSource; import org.apache.druid.indexing.template.BatchIndexingJobTemplate; import org.apache.druid.java.util.common.granularity.Granularity; @@ -84,7 +85,7 @@ private CompactionJobTemplate getDelegate() { final ResolvedTable resolvedTable = catalog.resolveTable(tableId); if (resolvedTable == null) { - throw InvalidInput.exception("Could not find table[%s] in the catalog", tableId); + throw NotFound.exception("Could not find table[%s] in the catalog", tableId); } final BatchIndexingJobTemplate delegate diff --git a/server/src/test/java/org/apache/druid/server/compaction/CompactionStatusTest.java b/server/src/test/java/org/apache/druid/server/compaction/CompactionStatusTest.java index 01b42c006508..dc948b9d13bb 100644 --- a/server/src/test/java/org/apache/druid/server/compaction/CompactionStatusTest.java +++ b/server/src/test/java/org/apache/druid/server/compaction/CompactionStatusTest.java @@ -481,9 +481,8 @@ public void testStatusWhenAutoSchemaMatch() final DataSegment segment = DataSegment.builder(WIKI_SEGMENT).lastCompactionState(lastCompactionState).build(); final CompactionStatus status = CompactionStatus.compute( - CompactionCandidate.from(Collections.singletonList(segment)), - compactionConfig, - OBJECT_MAPPER + CompactionCandidate.from(List.of(segment), null), + compactionConfig ); Assert.assertTrue(status.isComplete()); } @@ -535,9 +534,8 @@ public void testStatusWhenAutoSchemaMismatch() final DataSegment segment = DataSegment.builder(WIKI_SEGMENT).lastCompactionState(lastCompactionState).build(); final CompactionStatus status = CompactionStatus.compute( - CompactionCandidate.from(Collections.singletonList(segment)), - compactionConfig, - OBJECT_MAPPER + CompactionCandidate.from(List.of(segment), null), + compactionConfig ); Assert.assertFalse(status.isComplete()); } From ae17f67845d5eb03c7cbc40885dae509579d2b6a Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Mon, 27 Oct 2025 18:17:29 +0530 Subject: [PATCH 16/25] Minor fixes --- .../embedded/compact/CompactionSupervisorTest.java | 9 +++++---- .../testing/embedded/server/HighAvailabilityTest.java | 2 +- .../indexing/compact/CascadingCompactionTemplate.java | 4 +++- .../druid/indexing/compact/CompactionJobQueue.java | 9 +++++++++ .../indexing/compact/OverlordCompactionScheduler.java | 3 +++ .../org/apache/druid/server/coordinator/stats/Stats.java | 5 +++++ .../apache/druid/server/metrics/LatchableEmitter.java | 4 ++-- 7 files changed, 28 insertions(+), 8 deletions(-) diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionSupervisorTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionSupervisorTest.java index 537846facbf6..b65155d5924b 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 @@ -80,7 +80,7 @@ public class CompactionSupervisorTest extends EmbeddedClusterTestBase private final EmbeddedBroker broker = new EmbeddedBroker(); private final EmbeddedIndexer indexer = new EmbeddedIndexer() .setServerMemory(2_000_000_000L) - .addProperty("druid.worker.capacity", "4"); + .addProperty("druid.worker.capacity", "20"); private final EmbeddedOverlord overlord = new EmbeddedOverlord() .addProperty("druid.manager.segments.pollDuration", "PT1s") .addProperty("druid.manager.segments.useIncrementalCache", "always"); @@ -93,6 +93,7 @@ public EmbeddedDruidCluster createCluster() { return EmbeddedDruidCluster.withEmbeddedDerbyAndZookeeper() .useLatchableEmitter() + .useDefaultTimeoutForLatchableEmitter(600) .addExtensions( CatalogClientModule.class, CatalogCoordinatorModule.class, @@ -114,7 +115,7 @@ public EmbeddedDruidCluster createCluster() public void enableCompactionSupervisors() { final UpdateResponse updateResponse = cluster.callApi().onLeaderOverlord( - o -> o.updateClusterCompactionConfig(new ClusterCompactionConfig(1.0, 10, null, true, null)) + o -> o.updateClusterCompactionConfig(new ClusterCompactionConfig(1.0, 100, null, true, null)) ); Assertions.assertTrue(updateResponse.isSuccess()); } @@ -343,7 +344,7 @@ private void waitForAllCompactionTasksToFinish() overlord.latchableEmitter().waitForEvent( event -> event.hasMetricName("interval/waitCompact/count") .hasDimension(DruidMetrics.DATASOURCE, dataSource) - .hasValue(Matchers.equalTo(0L)) + .hasValueMatching(Matchers.equalTo(0L)) ); // Wait for all submitted compaction jobs to finish @@ -358,7 +359,7 @@ private void waitForAllCompactionTasksToFinish() ); overlord.latchableEmitter().waitForEventAggregate( event -> event.hasMetricName("task/run/time") - .hasDimension(DruidMetrics.TASK_TYPE, taskTypeMatcher) + .hasDimensionMatching(DruidMetrics.TASK_TYPE, taskTypeMatcher) .hasDimension(DruidMetrics.DATASOURCE, dataSource), agg -> agg.hasCountAtLeast(numSubmittedTasks) ); diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/server/HighAvailabilityTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/server/HighAvailabilityTest.java index 50d8e16779e1..2a0f9d3cbef6 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/server/HighAvailabilityTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/server/HighAvailabilityTest.java @@ -129,7 +129,7 @@ public void test_switchLeader_andVerifyUsingSysTables() coordinator1.latchableEmitter().waitForEvent( event -> event.hasMetricName("segment/metadataCache/used/count") .hasDimension(DruidMetrics.DATASOURCE, dataSource) - .hasValue(Matchers.greaterThanOrEqualTo(10L)) + .hasValueMatching(Matchers.greaterThanOrEqualTo(10L)) ); // Run sys queries, switch leaders, repeat diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CascadingCompactionTemplate.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CascadingCompactionTemplate.java index 6465375e4eba..afdde14fde71 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CascadingCompactionTemplate.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CascadingCompactionTemplate.java @@ -140,7 +140,9 @@ private List createJobsForSearchInterval( jobParams ); - // Filter out jobs if they are outside the search interval + // Some jobs may have a compaction interval that goes beyond the search interval + // in order to be able to align with the target segment granularity. + // Filter out such jobs to avoid submission of overlapping compaction tasks. final List validJobs = new ArrayList<>(); for (CompactionJob job : allJobs) { final Interval compactionInterval = job.getCandidate().getCompactionInterval(); 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 bc08583f97a2..cce0198b8a85 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 @@ -32,6 +32,7 @@ import org.apache.druid.indexing.overlord.GlobalTaskLockbox; import org.apache.druid.indexing.template.BatchIndexingJob; import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.java.util.common.Stopwatch; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.rpc.indexing.OverlordClient; import org.apache.druid.server.compaction.CompactionCandidate; @@ -134,6 +135,7 @@ public void createAndEnqueueJobs( DruidDatasourceDestination destination ) { + final Stopwatch jobCreationTime = Stopwatch.createStarted(); final String supervisorId = supervisor.getSpec().getId(); try { if (supervisor.shouldCreateJobs()) { @@ -145,6 +147,13 @@ public void createAndEnqueueJobs( catch (Exception e) { log.error(e, "Error while creating jobs for supervisor[%s]", supervisorId); } + finally { + runStats.add( + Stats.Compaction.JOB_CREATION_TIME, + RowKey.of(Dimension.DATASOURCE, source.getDataSource()), + jobCreationTime.millisElapsed() + ); + } } /** diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/OverlordCompactionScheduler.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/OverlordCompactionScheduler.java index a3f456e074ca..48d828109b81 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/compact/OverlordCompactionScheduler.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/OverlordCompactionScheduler.java @@ -212,6 +212,7 @@ public synchronized void stop() public void becomeLeader() { if (isLeader.compareAndSet(false, true)) { + log.info("Running compaction scheduler with period [%d] millis.", schedulePeriodMillis); scheduleOnExecutor(this::scheduledRun, schedulePeriodMillis); } } @@ -336,6 +337,7 @@ private synchronized void scheduledRun() */ private synchronized void runCompactionDuty() { + final Stopwatch runDuration = Stopwatch.createStarted(); final DataSourcesSnapshot dataSourcesSnapshot = getDatasourceSnapshot(); final CompactionJobQueue queue = new CompactionJobQueue( dataSourcesSnapshot, @@ -360,6 +362,7 @@ private synchronized void runCompactionDuty() datasourceToCompactionSnapshot.set(queue.getCompactionSnapshots()); emitStatsIfPeriodHasElapsed(queue.getRunStats()); + emitStat(Stats.Compaction.SCHEDULER_RUN_TIME, RowKey.empty(), runDuration.millisElapsed()); } /** diff --git a/server/src/main/java/org/apache/druid/server/coordinator/stats/Stats.java b/server/src/main/java/org/apache/druid/server/coordinator/stats/Stats.java index 32d4af8806ab..a41a1696c89d 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/stats/Stats.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/stats/Stats.java @@ -110,6 +110,11 @@ public static class Tier public static class Compaction { + public static final CoordinatorStat JOB_CREATION_TIME + = CoordinatorStat.toDebugAndEmit("jobCreationTime", "compact/createJobs/time"); + public static final CoordinatorStat SCHEDULER_RUN_TIME + = CoordinatorStat.toDebugAndEmit("schedulerRunTime", "compact/runScheduler/time"); + public static final CoordinatorStat SUBMITTED_TASKS = CoordinatorStat.toDebugAndEmit("compactTasks", "compact/task/count"); public static final CoordinatorStat CANCELLED_TASKS diff --git a/server/src/test/java/org/apache/druid/server/metrics/LatchableEmitter.java b/server/src/test/java/org/apache/druid/server/metrics/LatchableEmitter.java index 7b0d6ce86cce..5d8b110e17ac 100644 --- a/server/src/test/java/org/apache/druid/server/metrics/LatchableEmitter.java +++ b/server/src/test/java/org/apache/druid/server/metrics/LatchableEmitter.java @@ -264,7 +264,7 @@ public EventMatcher hasMetricName(String metricName) /** * Matches an event only if the metric value satisfies the given matcher. */ - public EventMatcher hasValue(Matcher valueMatcher) + public EventMatcher hasValueMatching(Matcher valueMatcher) { this.valueMatcher = valueMatcher; return this; @@ -282,7 +282,7 @@ public EventMatcher hasDimension(String dimension, Object value) /** * Matches an event if the value of the given dimension satisfies the matcher. */ - public EventMatcher hasDimension(String dimension, Matcher matcher) + public EventMatcher hasDimensionMatching(String dimension, Matcher matcher) { dimensionMatchers.put(dimension, matcher); return this; From b6423f8bf4ae6791f2a909239534caf3976b1dec Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Mon, 27 Oct 2025 19:52:11 +0530 Subject: [PATCH 17/25] Add metric, remove redundant checks --- .../compact/CascadingCompactionTemplate.java | 15 +-------------- .../indexing/compact/CompactionJobQueue.java | 10 +++++++++- .../druid/server/coordinator/stats/Stats.java | 2 ++ 3 files changed, 12 insertions(+), 15 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CascadingCompactionTemplate.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CascadingCompactionTemplate.java index afdde14fde71..89d389743667 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CascadingCompactionTemplate.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CascadingCompactionTemplate.java @@ -135,23 +135,10 @@ private List createJobsForSearchInterval( CompactionJobParams jobParams ) { - final List allJobs = template.createCompactionJobs( + return template.createCompactionJobs( inputSource.withInterval(searchInterval), jobParams ); - - // Some jobs may have a compaction interval that goes beyond the search interval - // in order to be able to align with the target segment granularity. - // Filter out such jobs to avoid submission of overlapping compaction tasks. - final List validJobs = new ArrayList<>(); - for (CompactionJob job : allJobs) { - final Interval compactionInterval = job.getCandidate().getCompactionInterval(); - if (searchInterval.contains(compactionInterval)) { - validJobs.add(job); - } - } - - return validJobs; } @Override 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 cce0198b8a85..ed51cb5bddaf 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 @@ -50,6 +50,7 @@ import org.apache.druid.server.coordinator.stats.Stats; import javax.annotation.Nullable; +import java.util.List; import java.util.Map; import java.util.PriorityQueue; @@ -139,7 +140,14 @@ public void createAndEnqueueJobs( final String supervisorId = supervisor.getSpec().getId(); try { if (supervisor.shouldCreateJobs()) { - queue.addAll(supervisor.createJobs(source, destination, jobParams)); + final List jobs = supervisor.createJobs(source, destination, jobParams); + queue.addAll(jobs); + + runStats.add( + Stats.Compaction.CREATED_JOBS, + RowKey.of(Dimension.DATASOURCE, source.getDataSource()), + jobs.size() + ); } else { log.debug("Skipping job creation for supervisor[%s]", supervisorId); } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/stats/Stats.java b/server/src/main/java/org/apache/druid/server/coordinator/stats/Stats.java index a41a1696c89d..3851b38fc91d 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/stats/Stats.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/stats/Stats.java @@ -112,6 +112,8 @@ public static class Compaction { public static final CoordinatorStat JOB_CREATION_TIME = CoordinatorStat.toDebugAndEmit("jobCreationTime", "compact/createJobs/time"); + public static final CoordinatorStat CREATED_JOBS + = CoordinatorStat.toDebugAndEmit("jobQueueSize", "compact/createJobs/count"); public static final CoordinatorStat SCHEDULER_RUN_TIME = CoordinatorStat.toDebugAndEmit("schedulerRunTime", "compact/runScheduler/time"); From c830cedf5cd496bdbbd47a7866cec94200410cf1 Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Thu, 30 Oct 2025 11:38:29 +0530 Subject: [PATCH 18/25] Update compaction status when job finishes --- .../compact/CompactionSupervisorTest.java | 3 + .../indexing/compact/CompactionJobQueue.java | 105 +++++++++++------- .../compact/CompactionStateMatcher.java | 5 +- .../compact/CompactionSupervisor.java | 1 - .../compact/OverlordCompactionScheduler.java | 71 +++++++++--- .../compaction/CompactionSlotManager.java | 10 ++ .../compaction/CompactionSnapshotBuilder.java | 14 +++ .../compaction/CompactionStatistics.java | 7 ++ .../compaction/CompactionStatusTracker.java | 6 +- .../coordinator/AutoCompactionSnapshot.java | 5 + 10 files changed, 168 insertions(+), 59 deletions(-) diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionSupervisorTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionSupervisorTest.java index b65155d5924b..edd4e5249368 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 @@ -75,6 +75,9 @@ import java.util.List; import java.util.Map; +/** + * Embedded test that runs compaction supervisors of various types. + */ public class CompactionSupervisorTest extends EmbeddedClusterTestBase { private final EmbeddedBroker broker = new EmbeddedBroker(); 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 ed51cb5bddaf..39a000bfe53e 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,9 +25,10 @@ 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.indexer.TaskState; +import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexing.common.actions.TaskActionClientFactory; import org.apache.druid.indexing.common.task.Task; -import org.apache.druid.indexing.input.DruidDatasourceDestination; import org.apache.druid.indexing.input.DruidInputSource; import org.apache.druid.indexing.overlord.GlobalTaskLockbox; import org.apache.druid.indexing.template.BatchIndexingJob; @@ -50,6 +51,8 @@ import org.apache.druid.server.coordinator.stats.Stats; import javax.annotation.Nullable; +import java.util.ArrayList; +import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.PriorityQueue; @@ -58,6 +61,13 @@ * Iterates over all eligible compaction jobs in order of their priority. * A fresh instance of this class must be used in every run of the * {@link CompactionScheduler}. + *

+ * Unlike the Coordinator duty {@code CompactSegments}, the job queue currently + * does not cancel running compaction tasks even if their target segment + * granularity has changed. This has not been done here for simplicity since the + * {@code CompactionJobQueue} uses compaction templates and may have a different + * target segment granulariy for different intervals of the same datasource. + * The cancellation of invalid tasks has been left as a future enhancement. */ public class CompactionJobQueue { @@ -65,6 +75,7 @@ public class CompactionJobQueue private final CompactionJobParams jobParams; private final CompactionCandidateSearchPolicy searchPolicy; + private final ClusterCompactionConfig clusterCompactionConfig; private final ObjectMapper objectMapper; private final CompactionStatusTracker statusTracker; @@ -75,10 +86,9 @@ public class CompactionJobQueue private final CompactionSnapshotBuilder snapshotBuilder; private final PriorityQueue queue; + private final Map submittedTaskIdToJob; private final CoordinatorRunStats runStats; - private final CompactionSlotManager slotManager; - public CompactionJobQueue( DataSourcesSnapshot dataSourcesSnapshot, ClusterCompactionConfig clusterCompactionConfig, @@ -92,21 +102,18 @@ public CompactionJobQueue( { this.runStats = new CoordinatorRunStats(); this.snapshotBuilder = new CompactionSnapshotBuilder(runStats); + this.clusterCompactionConfig = clusterCompactionConfig; this.searchPolicy = clusterCompactionConfig.getCompactionPolicy(); this.queue = new PriorityQueue<>( (o1, o2) -> searchPolicy.compareCandidates(o1.getCandidate(), o2.getCandidate()) ); + this.submittedTaskIdToJob = new HashMap<>(); this.jobParams = new CompactionJobParams( DateTimes.nowUtc(), clusterCompactionConfig, dataSourcesSnapshot.getUsedSegmentsTimelinesPerDataSource()::get, snapshotBuilder ); - this.slotManager = new CompactionSlotManager( - overlordClient, - statusTracker, - clusterCompactionConfig - ); this.taskActionClientFactory = taskActionClientFactory; this.overlordClient = overlordClient; @@ -114,16 +121,6 @@ public CompactionJobQueue( this.statusTracker = statusTracker; this.objectMapper = objectMapper; this.taskLockbox = taskLockbox; - - computeAvailableTaskSlots(); - } - - /** - * Adds a job to this queue. - */ - public void add(CompactionJob job) - { - queue.add(job); } /** @@ -132,15 +129,16 @@ public void add(CompactionJob job) */ public void createAndEnqueueJobs( CompactionSupervisor supervisor, - DruidInputSource source, - DruidDatasourceDestination destination + DruidInputSource source ) { final Stopwatch jobCreationTime = Stopwatch.createStarted(); final String supervisorId = supervisor.getSpec().getId(); try { if (supervisor.shouldCreateJobs()) { - final List jobs = supervisor.createJobs(source, destination, jobParams); + final List jobs = supervisor.createJobs(source, jobParams); + jobs.forEach(job -> snapshotBuilder.addToPending(job.getCandidate())); + queue.addAll(jobs); runStats.add( @@ -170,21 +168,43 @@ public void createAndEnqueueJobs( */ public void runReadyJobs() { + final CompactionSlotManager slotManager = new CompactionSlotManager( + overlordClient, + statusTracker, + clusterCompactionConfig + ); + slotManager.reserveTaskSlotsForRunningCompactionTasks(); + + final List pendingJobs = new ArrayList<>(); while (!queue.isEmpty()) { final CompactionJob job = queue.poll(); - if (startJobIfPendingAndReady(job, searchPolicy)) { + if (startJobIfPendingAndReady(job, searchPolicy, pendingJobs, slotManager)) { runStats.add(Stats.Compaction.SUBMITTED_TASKS, RowKey.of(Dimension.DATASOURCE, job.getDataSource()), 1); } } + + // Requeue pending jobs so that they can be launched when slots become available + queue.addAll(pendingJobs); } /** - * Builds and returns the compaction snapshots for all the datasources being - * tracked in this queue. Must be called after {@link #runReadyJobs()}. + * Notifies completion of the given so that the compaction snapshots may be + * updated. */ - public Map getCompactionSnapshots() + public void onTaskFinished(String taskId, TaskStatus taskStatus) { - return snapshotBuilder.build(); + final CompactionJob job = submittedTaskIdToJob.remove(taskId); + if (job == null || !taskStatus.getStatusCode().isComplete()) { + // This is an unknown task ID + return; + } + + if (taskStatus.getStatusCode() == TaskState.FAILED) { + // Add this job back to the queue + queue.add(job); + } else { + snapshotBuilder.moveFromPendingToCompleted(job.getCandidate()); + } } public CoordinatorRunStats getRunStats() @@ -192,13 +212,14 @@ public CoordinatorRunStats getRunStats() return runStats; } - private void computeAvailableTaskSlots() + /** + * Builds compaction snapshots for all the datasources being tracked by this + * queue. + */ + public Map getSnapshots() { - // Do not cancel any currently running compaction tasks to be valid - // Future iterations can cancel a job if it doesn't match the given template - for (ClientCompactionTaskQuery task : slotManager.fetchRunningCompactionTasks()) { - slotManager.reserveTaskSlots(task); - } + // TODO: fix the stats problem + return snapshotBuilder.build(); } /** @@ -206,14 +227,19 @@ private void computeAvailableTaskSlots() * * @return true if the job was submitted successfully for execution */ - private boolean startJobIfPendingAndReady(CompactionJob job, CompactionCandidateSearchPolicy policy) + private boolean startJobIfPendingAndReady( + CompactionJob job, + CompactionCandidateSearchPolicy policy, + List pendingJobs, + CompactionSlotManager slotManager + ) { // Check if the job is a valid compaction job final CompactionCandidate candidate = job.getCandidate(); final CompactionConfigValidationResult validationResult = validateCompactionJob(job); if (!validationResult.isValid()) { log.error("Skipping invalid compaction job[%s] due to reason[%s].", job, validationResult.getReason()); - snapshotBuilder.addToSkipped(candidate); + snapshotBuilder.moveFromPendingToSkipped(candidate); return false; } @@ -221,11 +247,12 @@ private boolean startJobIfPendingAndReady(CompactionJob job, CompactionCandidate final CompactionStatus compactionStatus = getCurrentStatusForJob(job, policy); switch (compactionStatus.getState()) { case RUNNING: + return false; case COMPLETE: - snapshotBuilder.addToComplete(candidate); + snapshotBuilder.moveFromPendingToCompleted(candidate); return false; case SKIPPED: - snapshotBuilder.addToSkipped(candidate); + snapshotBuilder.moveFromPendingToSkipped(candidate); return false; default: break; @@ -233,7 +260,7 @@ private boolean startJobIfPendingAndReady(CompactionJob job, CompactionCandidate // Check if enough compaction task slots are available if (job.getMaxRequiredTaskSlots() > slotManager.getNumAvailableTaskSlots()) { - snapshotBuilder.addToPending(candidate); + pendingJobs.add(job); return false; } @@ -242,11 +269,11 @@ private boolean startJobIfPendingAndReady(CompactionJob job, CompactionCandidate final String taskId = startTaskIfReady(job); if (taskId == null) { // Mark the job as skipped for now as the intervals might be locked by other tasks - snapshotBuilder.addToSkipped(candidate); + snapshotBuilder.moveFromPendingToSkipped(candidate); return false; } else { - snapshotBuilder.addToComplete(candidate); statusTracker.onTaskSubmitted(taskId, job.getCandidate()); + submittedTaskIdToJob.put(taskId, job); return true; } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionStateMatcher.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionStateMatcher.java index 0ef1c898af4c..84a95e2f6e97 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionStateMatcher.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionStateMatcher.java @@ -45,7 +45,10 @@ * separate to allow: *

    *
  • fields to be nullable so that only non-null fields are used for matching
  • - *
  • legacy "compaction-incompatible" fields to be removed
  • + *
  • legacy "compaction-incompatible" fields such as {@link #transformSpec} to + * be removed in the future. These fields do not just change the layout/partitioning + * of the data but may also alter its meaning which does not fall in the purview + * of compaction.
  • *
*/ @JsonInclude(JsonInclude.Include.NON_NULL) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionSupervisor.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionSupervisor.java index 857916e6ec52..b87cb150e387 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionSupervisor.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionSupervisor.java @@ -73,7 +73,6 @@ public boolean shouldCreateJobs() */ public List createJobs( DruidInputSource inputSource, - DruidDatasourceDestination destination, CompactionJobParams jobParams ) { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/OverlordCompactionScheduler.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/OverlordCompactionScheduler.java index 48d828109b81..6dbb43b4ae2c 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/compact/OverlordCompactionScheduler.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/OverlordCompactionScheduler.java @@ -29,7 +29,6 @@ import org.apache.druid.indexer.TaskLocation; import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexing.common.actions.TaskActionClientFactory; -import org.apache.druid.indexing.input.DruidDatasourceDestination; import org.apache.druid.indexing.overlord.GlobalTaskLockbox; import org.apache.druid.indexing.overlord.TaskMaster; import org.apache.druid.indexing.overlord.TaskQueryTool; @@ -37,7 +36,6 @@ import org.apache.druid.indexing.overlord.TaskRunnerListener; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.Stopwatch; -import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.common.concurrent.ScheduledExecutorFactory; import org.apache.druid.java.util.common.lifecycle.LifecycleStart; import org.apache.druid.java.util.common.lifecycle.LifecycleStop; @@ -89,6 +87,7 @@ public class OverlordCompactionScheduler implements CompactionScheduler { private static final Logger log = new Logger(OverlordCompactionScheduler.class); + private static final long DEFAULT_SCHEDULE_PERIOD_MILLIS = 15 * 60_000; private static final Duration METRIC_EMISSION_PERIOD = Duration.standardMinutes(5); private final SegmentsMetadataManager segmentManager; @@ -103,6 +102,11 @@ public class OverlordCompactionScheduler implements CompactionScheduler private final AtomicReference> datasourceToCompactionSnapshot; + /** + * Compaction job queue built in the last invocation of {@link #resetCompactionJobQueue()}. + */ + private final AtomicReference latestJobQueue; + /** * Single-threaded executor to process the compaction queue. */ @@ -150,9 +154,9 @@ public OverlordCompactionScheduler( ObjectMapper objectMapper ) { - final long segmentPollPeriodSeconds = + final long segmentPollPeriodMillis = segmentManagerConfig.getPollDuration().toStandardDuration().getMillis(); - this.schedulePeriodMillis = Math.min(5_000, segmentPollPeriodSeconds); + this.schedulePeriodMillis = Math.min(DEFAULT_SCHEDULE_PERIOD_MILLIS, segmentPollPeriodMillis); this.segmentManager = segmentManager; this.emitter = emitter; @@ -169,6 +173,7 @@ public OverlordCompactionScheduler( this.brokerClient = brokerClient; this.activeSupervisors = new ConcurrentHashMap<>(); this.datasourceToCompactionSnapshot = new AtomicReference<>(); + this.latestJobQueue = new AtomicReference<>(); this.taskActionClientFactory = taskActionClientFactory; this.druidInputSourceFactory = druidInputSourceFactory; @@ -190,7 +195,8 @@ public void locationChanged(String taskId, TaskLocation newLocation) public void statusChanged(String taskId, TaskStatus status) { if (status.isComplete()) { - statusTracker.onTaskFinished(taskId, status); + onTaskFinished(taskId, status); + launchPendingJobs(); } } }; @@ -271,7 +277,7 @@ private synchronized void initState() log.info("Starting compaction scheduler."); final Optional taskRunnerOptional = taskMaster.getTaskRunner(); if (taskRunnerOptional.isPresent()) { - taskRunnerOptional.get().registerListener(taskRunnerListener, Execs.directExecutor()); + taskRunnerOptional.get().registerListener(taskRunnerListener, executor); } if (shouldPollSegments) { segmentManager.startPollingDatabasePeriodically(); @@ -320,7 +326,7 @@ private synchronized void scheduledRun() if (isEnabled()) { initState(); try { - runCompactionDuty(); + resetCompactionJobQueue(); } catch (Exception e) { log.error(e, "Error processing compaction queue. Continuing schedule."); @@ -333,9 +339,9 @@ private synchronized void scheduledRun() } /** - * Creates and runs eligible compaction jobs. + * Creates and launches eligible compaction jobs. */ - private synchronized void runCompactionDuty() + private synchronized void resetCompactionJobQueue() { final Stopwatch runDuration = Stopwatch.createStarted(); final DataSourcesSnapshot dataSourcesSnapshot = getDatasourceSnapshot(); @@ -349,22 +355,43 @@ private synchronized void runCompactionDuty() brokerClient, objectMapper ); + latestJobQueue.set(queue); + statusTracker.resetActiveDatasources(activeSupervisors.keySet()); statusTracker.onSegmentTimelineUpdated(dataSourcesSnapshot.getSnapshotTime()); activeSupervisors.forEach( (datasource, supervisor) -> queue.createAndEnqueueJobs( supervisor, - druidInputSourceFactory.create(datasource, Intervals.ETERNITY), - new DruidDatasourceDestination(datasource) + druidInputSourceFactory.create(datasource, Intervals.ETERNITY) ) ); - queue.runReadyJobs(); - - datasourceToCompactionSnapshot.set(queue.getCompactionSnapshots()); + launchPendingJobs(); emitStatsIfPeriodHasElapsed(queue.getRunStats()); emitStat(Stats.Compaction.SCHEDULER_RUN_TIME, RowKey.empty(), runDuration.millisElapsed()); } + /** + * Launches pending compaction jobs if compaction task slots become available. + * This method uses the jobs created by the last invocation of {@link #resetCompactionJobQueue()}. + */ + private synchronized void launchPendingJobs() + { + final CompactionJobQueue queue = latestJobQueue.get(); + if (queue == null) { + // Job queue has not been built yet + return; + } + + queue.runReadyJobs(); + updateCompactionSnapshots(queue); + + // TODO: Try to trigger a recompute in these cases: + // supervisor spec updated - schedule recompute atleast for that supervisor? + // dynamic config updated - schedule full recompute? + // + // supervisor reset? not needed right now? we can probably do it later + } + /** * Emits stats if {@link #METRIC_EMISSION_PERIOD} has elapsed. */ @@ -383,6 +410,22 @@ private void emitStatsIfPeriodHasElapsed(CoordinatorRunStats stats) } } + private void onTaskFinished(String taskId, TaskStatus taskStatus) + { + statusTracker.onTaskFinished(taskId, taskStatus); + + final CompactionJobQueue queue = latestJobQueue.get(); + if (queue != null) { + queue.onTaskFinished(taskId, taskStatus); + updateCompactionSnapshots(queue); + } + } + + private void updateCompactionSnapshots(CompactionJobQueue queue) + { + datasourceToCompactionSnapshot.set(queue.getSnapshots()); + } + @Override public AutoCompactionSnapshot getCompactionSnapshot(String dataSource) { diff --git a/server/src/main/java/org/apache/druid/server/compaction/CompactionSlotManager.java b/server/src/main/java/org/apache/druid/server/compaction/CompactionSlotManager.java index efe1c8bdba28..5bdab3009cfa 100644 --- a/server/src/main/java/org/apache/druid/server/compaction/CompactionSlotManager.java +++ b/server/src/main/java/org/apache/druid/server/compaction/CompactionSlotManager.java @@ -114,6 +114,16 @@ public void reserveTaskSlots(ClientCompactionTaskQuery compactionTaskQuery) } } + /** + * Reserves task slots for all running compaction tasks. + */ + public void reserveTaskSlotsForRunningCompactionTasks() + { + for (ClientCompactionTaskQuery task : fetchRunningCompactionTasks()) { + reserveTaskSlots(task); + } + } + /** * Retrieves currently running tasks of type {@link #COMPACTION_TASK_TYPE} from * the Overlord. diff --git a/server/src/main/java/org/apache/druid/server/compaction/CompactionSnapshotBuilder.java b/server/src/main/java/org/apache/druid/server/compaction/CompactionSnapshotBuilder.java index e52d24edbf01..d186587cb870 100644 --- a/server/src/main/java/org/apache/druid/server/compaction/CompactionSnapshotBuilder.java +++ b/server/src/main/java/org/apache/druid/server/compaction/CompactionSnapshotBuilder.java @@ -60,6 +60,20 @@ public void addToSkipped(CompactionCandidate candidate) .incrementSkippedStats(candidate.getStats()); } + public void moveFromPendingToSkipped(CompactionCandidate candidate) + { + getBuilderForDatasource(candidate.getDataSource()) + .decrementWaitingStats(candidate.getStats()); + addToSkipped(candidate); + } + + public void moveFromPendingToCompleted(CompactionCandidate candidate) + { + getBuilderForDatasource(candidate.getDataSource()) + .decrementWaitingStats(candidate.getStats()); + addToComplete(candidate); + } + public Map build() { final Map datasourceToSnapshot = new HashMap<>(); diff --git a/server/src/main/java/org/apache/druid/server/compaction/CompactionStatistics.java b/server/src/main/java/org/apache/druid/server/compaction/CompactionStatistics.java index 23f1b7fe9ef5..d7e51655861b 100644 --- a/server/src/main/java/org/apache/druid/server/compaction/CompactionStatistics.java +++ b/server/src/main/java/org/apache/druid/server/compaction/CompactionStatistics.java @@ -58,4 +58,11 @@ public void increment(CompactionStatistics other) numIntervals += other.getNumIntervals(); numSegments += other.getNumSegments(); } + + public void decrement(CompactionStatistics other) + { + totalBytes -= other.getTotalBytes(); + numIntervals -= other.getNumIntervals(); + numSegments -= other.getNumSegments(); + } } 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 db844d580b63..401f413e7fd8 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 @@ -85,8 +85,6 @@ public CompactionStatus computeCompactionStatus( CompactionCandidateSearchPolicy searchPolicy ) { - final CompactionStatus pendingStatus = CompactionStatus.pending("not compacted yet"); - // Skip intervals that already have a running task final CompactionTaskStatus lastTaskStatus = getLatestTaskStatus(candidate); if (lastTaskStatus != null && lastTaskStatus.getState() == TaskState.RUNNING) { @@ -104,11 +102,11 @@ public CompactionStatus computeCompactionStatus( } // Skip intervals that have been filtered out by the policy - if (!searchPolicy.isEligibleForCompaction(candidate, pendingStatus, lastTaskStatus)) { + if (!searchPolicy.isEligibleForCompaction(candidate, CompactionStatus.pending(""), lastTaskStatus)) { return CompactionStatus.skipped("Rejected by search policy"); } - return pendingStatus; + return CompactionStatus.pending("Not compacted yet"); } /** diff --git a/server/src/main/java/org/apache/druid/server/coordinator/AutoCompactionSnapshot.java b/server/src/main/java/org/apache/druid/server/coordinator/AutoCompactionSnapshot.java index e31a7919f24f..29891cfdde44 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/AutoCompactionSnapshot.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/AutoCompactionSnapshot.java @@ -241,6 +241,11 @@ public void incrementWaitingStats(CompactionStatistics entry) waitingStats.increment(entry); } + public void decrementWaitingStats(CompactionStatistics entry) + { + waitingStats.decrement(entry); + } + public void incrementCompactedStats(CompactionStatistics entry) { compactedStats.increment(entry); From e6c68b3bbdbae50fcde4fb1ceaf3c2d32c653378 Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Thu, 30 Oct 2025 11:52:40 +0530 Subject: [PATCH 19/25] Remove catalog changes --- .../compact/CompactionSupervisorTest.java | 105 +++------------- .../apache/druid/guice/SupervisorModule.java | 2 - .../compact/CatalogCompactionJobTemplate.java | 108 ---------------- .../OverlordCompactionSchedulerTest.java | 58 +++------ .../catalog/model/SchemaRegistryImpl.java | 6 - .../catalog/model/TableDefnRegistry.java | 4 +- .../model/table/IndexingTemplateDefn.java | 65 ---------- .../coordinator/AutoCompactionSnapshot.java | 19 +++ .../schema/DruidCalciteSchemaModule.java | 2 - .../schema/IndexingTemplateSchema.java | 84 ------------- .../schema/NamedIndexingTemplateSchema.java | 50 -------- .../schema/DruidCalciteSchemaModuleTest.java | 12 +- .../schema/IndexingTemplateSchemaTest.java | 116 ------------------ 13 files changed, 53 insertions(+), 578 deletions(-) delete mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/compact/CatalogCompactionJobTemplate.java delete mode 100644 server/src/main/java/org/apache/druid/catalog/model/table/IndexingTemplateDefn.java delete mode 100644 sql/src/main/java/org/apache/druid/sql/calcite/schema/IndexingTemplateSchema.java delete mode 100644 sql/src/main/java/org/apache/druid/sql/calcite/schema/NamedIndexingTemplateSchema.java delete mode 100644 sql/src/test/java/org/apache/druid/sql/calcite/schema/IndexingTemplateSchemaTest.java diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionSupervisorTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionSupervisorTest.java index edd4e5249368..0dba6f709b09 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,18 +19,11 @@ 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.catalog.model.ResolvedTable; -import org.apache.druid.catalog.model.TableId; -import org.apache.druid.catalog.model.TableSpec; -import org.apache.druid.catalog.model.table.IndexingTemplateDefn; -import org.apache.druid.catalog.sync.CatalogClient; import org.apache.druid.common.utils.IdUtils; import org.apache.druid.indexing.common.task.IndexTask; import org.apache.druid.indexing.compact.CascadingCompactionTemplate; -import org.apache.druid.indexing.compact.CatalogCompactionJobTemplate; import org.apache.druid.indexing.compact.CompactionJobTemplate; import org.apache.druid.indexing.compact.CompactionRule; import org.apache.druid.indexing.compact.CompactionStateMatcher; @@ -39,7 +32,6 @@ import org.apache.druid.indexing.compact.MSQCompactionJobTemplate; import org.apache.druid.indexing.overlord.Segments; import org.apache.druid.java.util.common.DateTimes; -import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.msq.guice.IndexerMemoryManagementModule; @@ -50,7 +42,6 @@ import org.apache.druid.query.DruidMetrics; import org.apache.druid.query.http.ClientSqlQuery; import org.apache.druid.rpc.UpdateResponse; -import org.apache.druid.segment.TestHelper; import org.apache.druid.server.coordinator.ClusterCompactionConfig; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; import org.apache.druid.server.coordinator.InlineSchemaDataSourceCompactionConfig; @@ -123,37 +114,6 @@ public void enableCompactionSupervisors() Assertions.assertTrue(updateResponse.isSuccess()); } - @Test - public void test_saveCompactionTemplateToCatalog_andQueryDefinition() throws Exception - { - final String templateId = saveTemplateToCatalog( - new InlineCompactionJobTemplate(createMatcher(Granularities.DAY)) - ); - - final ClientSqlQuery query = new ClientSqlQuery( - StringUtils.format("SELECT * FROM index_template.%s", templateId), - "array", - true, - true, - true, - null, - null - ); - - final String result = cluster.callApi().onAnyBroker(b -> b.submitSqlQuery(query)); - final List> rows = TestHelper.JSON_MAPPER.readValue(result, new TypeReference<>() {}); - - Assertions.assertEquals( - List.of( - List.of("type", "payload"), - List.of("STRING", "COMPLEX"), - List.of("VARCHAR", "OTHER"), - List.of("compactInline", "{\"targetState\":{\"granularitySpec\":{\"segmentGranularity\":\"DAY\"}},\"type\":\"compactInline\"}") - ), - rows - ); - } - @Test public void test_ingestDayGranularity_andCompactToMonthGranularity_withInlineConfig() { @@ -211,19 +171,17 @@ public void test_ingestHourGranularity_andCompactToDayAndMonth_withCatalogTempla ingestHourSegments(1200); // Add compaction templates to catalog - final String dayGranularityTemplateId = saveTemplateToCatalog( - new InlineCompactionJobTemplate(createMatcher(Granularities.DAY)) - ); - final String monthGranularityTemplateId = saveTemplateToCatalog( - new InlineCompactionJobTemplate(createMatcher(Granularities.MONTH)) - ); + final CompactionJobTemplate dayGranularityTemplate = + new InlineCompactionJobTemplate(createMatcher(Granularities.DAY)); + final CompactionJobTemplate monthGranularityTemplate = + new InlineCompactionJobTemplate(createMatcher(Granularities.MONTH)); // Create a cascading template with DAY and MONTH granularity CascadingCompactionTemplate cascadingTemplate = new CascadingCompactionTemplate( dataSource, List.of( - new CompactionRule(Period.days(1), new CatalogCompactionJobTemplate(dayGranularityTemplateId, null)), - new CompactionRule(Period.days(50), new CatalogCompactionJobTemplate(monthGranularityTemplateId, null)) + new CompactionRule(Period.days(1), dayGranularityTemplate), + new CompactionRule(Period.days(50), monthGranularityTemplate) ) ); @@ -247,11 +205,9 @@ public void test_ingestHourGranularity_andCompactToDayAndMonth_withCatalogMSQTem + " SELECT * FROM ${dataSource}" + " WHERE __time BETWEEN '${startTimestamp}' AND '${endTimestamp}'" + " PARTITIONED BY DAY"; - final String dayGranularityTemplateId = saveTemplateToCatalog( - new MSQCompactionJobTemplate( - new ClientSqlQuery(sqlDayGranularity, null, false, false, false, null, null), - createMatcher(Granularities.DAY) - ) + final CompactionJobTemplate dayGranularityTemplate = new MSQCompactionJobTemplate( + new ClientSqlQuery(sqlDayGranularity, null, false, false, false, null, null), + createMatcher(Granularities.DAY) ); final String sqlMonthGranularity = "REPLACE INTO ${dataSource}" @@ -259,19 +215,17 @@ public void test_ingestHourGranularity_andCompactToDayAndMonth_withCatalogMSQTem + " SELECT * FROM ${dataSource}" + " WHERE __time >= TIMESTAMP '${startTimestamp}' AND __time < TIMESTAMP '${endTimestamp}'" + " PARTITIONED BY MONTH"; - final String monthGranularityTemplateId = saveTemplateToCatalog( - new MSQCompactionJobTemplate( - new ClientSqlQuery(sqlMonthGranularity, null, false, false, false, null, null), - createMatcher(Granularities.MONTH) - ) + final CompactionJobTemplate monthGranularityTemplate = new MSQCompactionJobTemplate( + new ClientSqlQuery(sqlMonthGranularity, null, false, false, false, null, null), + createMatcher(Granularities.MONTH) ); // Create a cascading template with DAY and MONTH granularity CascadingCompactionTemplate cascadingTemplate = new CascadingCompactionTemplate( dataSource, List.of( - new CompactionRule(Period.days(1), new CatalogCompactionJobTemplate(dayGranularityTemplateId, null)), - new CompactionRule(Period.days(50), new CatalogCompactionJobTemplate(monthGranularityTemplateId, null)) + new CompactionRule(Period.days(1), dayGranularityTemplate), + new CompactionRule(Period.days(50), monthGranularityTemplate) ) ); @@ -299,10 +253,8 @@ public void test_ingestHourGranularity_andCompactToDayAndMonth_withMixedTemplate new ClientSqlQuery(sqlDayGranularity, null, false, false, false, null, null), createMatcher(Granularities.DAY) ); - final String dayTemplateId = saveTemplateToCatalog(dayTemplate); - final String weekTemplateId = saveTemplateToCatalog( - new InlineCompactionJobTemplate(createMatcher(Granularities.WEEK)) - ); + final CompactionJobTemplate weekTemplate = + new InlineCompactionJobTemplate(createMatcher(Granularities.WEEK)); final InlineCompactionJobTemplate monthTemplate = new InlineCompactionJobTemplate(createMatcher(Granularities.MONTH)); @@ -310,8 +262,8 @@ public void test_ingestHourGranularity_andCompactToDayAndMonth_withMixedTemplate CascadingCompactionTemplate cascadingTemplate = new CascadingCompactionTemplate( dataSource, List.of( - new CompactionRule(Period.days(1), new CatalogCompactionJobTemplate(dayTemplateId, null)), - new CompactionRule(Period.days(15), new CatalogCompactionJobTemplate(weekTemplateId, null)), + new CompactionRule(Period.days(1), dayTemplate), + new CompactionRule(Period.days(15), weekTemplate), new CompactionRule(Period.days(16), dayTemplate), new CompactionRule(Period.ZERO, monthTemplate) ) @@ -379,27 +331,6 @@ private int getNumSegmentsWith(Granularity granularity) .count(); } - private String saveTemplateToCatalog(CompactionJobTemplate template) - { - final String templateId = IdUtils.getRandomId(); - final CatalogClient catalogClient = overlord.bindings().getInstance(CatalogClient.class); - - final TableId tableId = TableId.of(TableId.INDEXING_TEMPLATE_SCHEMA, templateId); - catalogClient.createTable( - tableId, - new TableSpec( - IndexingTemplateDefn.TYPE, - Map.of(IndexingTemplateDefn.PROPERTY_PAYLOAD, template), - null - ) - ); - - ResolvedTable table = catalogClient.resolveTable(tableId); - Assertions.assertNotNull(table); - - return templateId; - } - private void runIngestionAtGranularity( String granularity, String inlineDataCsv diff --git a/indexing-service/src/main/java/org/apache/druid/guice/SupervisorModule.java b/indexing-service/src/main/java/org/apache/druid/guice/SupervisorModule.java index eb3ea599e30e..a5fc5fb4847a 100644 --- a/indexing-service/src/main/java/org/apache/druid/guice/SupervisorModule.java +++ b/indexing-service/src/main/java/org/apache/druid/guice/SupervisorModule.java @@ -26,7 +26,6 @@ import com.google.common.collect.ImmutableList; import com.google.inject.Binder; import org.apache.druid.indexing.compact.CascadingCompactionTemplate; -import org.apache.druid.indexing.compact.CatalogCompactionJobTemplate; import org.apache.druid.indexing.compact.CompactionSupervisorSpec; import org.apache.druid.indexing.compact.InlineCompactionJobTemplate; import org.apache.druid.indexing.compact.MSQCompactionJobTemplate; @@ -52,7 +51,6 @@ public List getJacksonModules() new SimpleModule(getClass().getSimpleName()) .registerSubtypes( new NamedType(InlineCompactionJobTemplate.class, InlineCompactionJobTemplate.TYPE), - new NamedType(CatalogCompactionJobTemplate.class, CatalogCompactionJobTemplate.TYPE), new NamedType(MSQCompactionJobTemplate.class, MSQCompactionJobTemplate.TYPE), new NamedType(CascadingCompactionTemplate.class, CascadingCompactionTemplate.TYPE), new NamedType(CompactionSupervisorSpec.class, CompactionSupervisorSpec.TYPE), diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CatalogCompactionJobTemplate.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CatalogCompactionJobTemplate.java deleted file mode 100644 index e34dbb2b1d88..000000000000 --- a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CatalogCompactionJobTemplate.java +++ /dev/null @@ -1,108 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.druid.indexing.compact; - -import com.fasterxml.jackson.annotation.JacksonInject; -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; -import org.apache.druid.catalog.MetadataCatalog; -import org.apache.druid.catalog.model.ResolvedTable; -import org.apache.druid.catalog.model.TableId; -import org.apache.druid.catalog.model.table.IndexingTemplateDefn; -import org.apache.druid.error.InvalidInput; -import org.apache.druid.error.NotFound; -import org.apache.druid.indexing.input.DruidInputSource; -import org.apache.druid.indexing.template.BatchIndexingJobTemplate; -import org.apache.druid.java.util.common.granularity.Granularity; - -import javax.annotation.Nullable; -import java.util.List; - -/** - * Compaction template that delegates job creation to a template stored in the - * Druid catalog. - */ -public class CatalogCompactionJobTemplate implements CompactionJobTemplate -{ - public static final String TYPE = "compactCatalog"; - - private final String templateId; - - private final TableId tableId; - private final MetadataCatalog catalog; - - @JsonCreator - public CatalogCompactionJobTemplate( - @JsonProperty("templateId") String templateId, - @JacksonInject MetadataCatalog catalog - ) - { - this.templateId = templateId; - this.catalog = catalog; - this.tableId = TableId.of(TableId.INDEXING_TEMPLATE_SCHEMA, templateId); - } - - @JsonProperty - public String getTemplateId() - { - return templateId; - } - - @Nullable - @Override - public Granularity getSegmentGranularity() - { - return getDelegate().getSegmentGranularity(); - } - - @Override - public List createCompactionJobs( - DruidInputSource source, - CompactionJobParams params - ) - { - return getDelegate().createCompactionJobs(source, params); - } - - private CompactionJobTemplate getDelegate() - { - final ResolvedTable resolvedTable = catalog.resolveTable(tableId); - if (resolvedTable == null) { - throw NotFound.exception("Could not find table[%s] in the catalog", tableId); - } - - final BatchIndexingJobTemplate delegate - = resolvedTable.decodeProperty(IndexingTemplateDefn.PROPERTY_PAYLOAD); - if (delegate instanceof CompactionJobTemplate) { - return (CompactionJobTemplate) delegate; - } else { - throw InvalidInput.exception( - "Template[%s] of type[%s] cannot be used for creating compaction tasks", - templateId, delegate == null ? null : delegate.getType() - ); - } - } - - @Override - public String getType() - { - return TYPE; - } -} diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/compact/OverlordCompactionSchedulerTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/compact/OverlordCompactionSchedulerTest.java index 59a8228bc3b1..16ebefc20f83 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/compact/OverlordCompactionSchedulerTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/compact/OverlordCompactionSchedulerTest.java @@ -24,10 +24,6 @@ import com.google.common.base.Optional; import com.google.common.util.concurrent.Futures; import org.apache.druid.catalog.MapMetadataCatalog; -import org.apache.druid.catalog.model.ResolvedTable; -import org.apache.druid.catalog.model.TableId; -import org.apache.druid.catalog.model.TableSpec; -import org.apache.druid.catalog.model.table.IndexingTemplateDefn; import org.apache.druid.client.broker.BrokerClient; import org.apache.druid.client.coordinator.NoopCoordinatorClient; import org.apache.druid.client.indexing.ClientMSQContext; @@ -533,12 +529,10 @@ public void test_ingestHourGranularity_andCompactToDayAndMonth_withCatalogTempla verifyNumSegmentsWith(Granularities.HOUR, 24 * numDays); // Add compaction templates to catalog - final String dayGranularityTemplateId = saveTemplateToCatalog( - new InlineCompactionJobTemplate(createMatcher(Granularities.DAY)) - ); - final String monthGranularityTemplateId = saveTemplateToCatalog( - new InlineCompactionJobTemplate(createMatcher(Granularities.MONTH)) - ); + final CompactionJobTemplate dayGranularityTemplate = + new InlineCompactionJobTemplate(createMatcher(Granularities.DAY)); + final CompactionJobTemplate monthGranularityTemplate = + new InlineCompactionJobTemplate(createMatcher(Granularities.MONTH)); // Compact everything going back to Mar 10 to DAY granularity, rest to MONTH final DateTime now = DateTimes.nowUtc(); @@ -546,8 +540,8 @@ public void test_ingestHourGranularity_andCompactToDayAndMonth_withCatalogTempla CascadingCompactionTemplate cascadingTemplate = new CascadingCompactionTemplate( dataSource, List.of( - new CompactionRule(dayRulePeriod, new CatalogCompactionJobTemplate(dayGranularityTemplateId, catalog)), - new CompactionRule(Period.ZERO, new CatalogCompactionJobTemplate(monthGranularityTemplateId, catalog)) + new CompactionRule(dayRulePeriod, dayGranularityTemplate), + new CompactionRule(Period.ZERO, monthGranularityTemplate) ) ); @@ -576,11 +570,9 @@ public void test_ingestHourGranularity_andCompactToDayAndMonth_withCatalogMSQTem + " SELECT * FROM ${dataSource}" + " WHERE __time BETWEEN '${startTimestamp}' AND '${endTimestamp}'" + " PARTITIONED BY DAY"; - final String dayGranularityTemplateId = saveTemplateToCatalog( - new MSQCompactionJobTemplate( - new ClientSqlQuery(sqlDayGranularity, null, false, false, false, null, null), - createMatcher(Granularities.DAY) - ) + final CompactionJobTemplate dayGranularityTemplate = new MSQCompactionJobTemplate( + new ClientSqlQuery(sqlDayGranularity, null, false, false, false, null, null), + createMatcher(Granularities.DAY) ); final String sqlMonthGranularity = "REPLACE INTO ${dataSource}" @@ -588,11 +580,9 @@ public void test_ingestHourGranularity_andCompactToDayAndMonth_withCatalogMSQTem + " SELECT * FROM ${dataSource}" + " WHERE __time BETWEEN '${startTimestamp}' AND '${endTimestamp}'" + " PARTITIONED BY MONTH"; - final String monthGranularityTemplateId = saveTemplateToCatalog( - new MSQCompactionJobTemplate( - new ClientSqlQuery(sqlMonthGranularity, null, false, false, false, null, null), - createMatcher(Granularities.MONTH) - ) + final CompactionJobTemplate monthGranularityTemplate = new MSQCompactionJobTemplate( + new ClientSqlQuery(sqlMonthGranularity, null, false, false, false, null, null), + createMatcher(Granularities.MONTH) ); // Compact everything going back to Mar 10 to DAY granularity, rest to MONTH @@ -601,8 +591,8 @@ public void test_ingestHourGranularity_andCompactToDayAndMonth_withCatalogMSQTem CascadingCompactionTemplate cascadingTemplate = new CascadingCompactionTemplate( dataSource, List.of( - new CompactionRule(dayRulePeriod, new CatalogCompactionJobTemplate(dayGranularityTemplateId, catalog)), - new CompactionRule(Period.ZERO, new CatalogCompactionJobTemplate(monthGranularityTemplateId, catalog)) + new CompactionRule(dayRulePeriod, dayGranularityTemplate), + new CompactionRule(Period.ZERO, monthGranularityTemplate) ) ); @@ -651,26 +641,6 @@ private void createSegments(int numSegments, Granularity granularity, DateTime f segmentStorage.commitSegments(Set.copyOf(segments), null); } - private String saveTemplateToCatalog(CompactionJobTemplate template) - { - final String templateId = IdUtils.getRandomId(); - final TableId tableId = TableId.of(TableId.INDEXING_TEMPLATE_SCHEMA, templateId); - - catalog.addSpec( - tableId, - new TableSpec( - IndexingTemplateDefn.TYPE, - Map.of(IndexingTemplateDefn.PROPERTY_PAYLOAD, template), - null - ) - ); - - ResolvedTable table = catalog.resolveTable(tableId); - Assert.assertNotNull(table); - - return templateId; - } - private void startCompactionWithSpec(DataSourceCompactionConfig config) { scheduler.becomeLeader(); diff --git a/server/src/main/java/org/apache/druid/catalog/model/SchemaRegistryImpl.java b/server/src/main/java/org/apache/druid/catalog/model/SchemaRegistryImpl.java index 21de49dc73c3..68922cf3ed5e 100644 --- a/server/src/main/java/org/apache/druid/catalog/model/SchemaRegistryImpl.java +++ b/server/src/main/java/org/apache/druid/catalog/model/SchemaRegistryImpl.java @@ -22,7 +22,6 @@ import com.google.common.collect.Lists; import org.apache.druid.catalog.model.table.DatasourceDefn; import org.apache.druid.catalog.model.table.ExternalTableDefn; -import org.apache.druid.catalog.model.table.IndexingTemplateDefn; import org.apache.druid.server.security.ResourceType; import java.util.Comparator; @@ -123,11 +122,6 @@ public SchemaRegistryImpl() ResourceType.VIEW, null // TODO )); - register(new SchemaDefnImpl( - TableId.INDEXING_TEMPLATE_SCHEMA, - ResourceType.CONFIG, - Set.of(IndexingTemplateDefn.TYPE) - )); } private void register(SchemaSpec schemaDefn) diff --git a/server/src/main/java/org/apache/druid/catalog/model/TableDefnRegistry.java b/server/src/main/java/org/apache/druid/catalog/model/TableDefnRegistry.java index ea442d7266cc..1a14d7403937 100644 --- a/server/src/main/java/org/apache/druid/catalog/model/TableDefnRegistry.java +++ b/server/src/main/java/org/apache/druid/catalog/model/TableDefnRegistry.java @@ -25,7 +25,6 @@ import org.apache.druid.catalog.model.table.DatasourceDefn; import org.apache.druid.catalog.model.table.ExternalTableDefn; import org.apache.druid.catalog.model.table.HttpInputSourceDefn; -import org.apache.druid.catalog.model.table.IndexingTemplateDefn; import org.apache.druid.catalog.model.table.InlineInputSourceDefn; import org.apache.druid.catalog.model.table.InputFormatDefn; import org.apache.druid.catalog.model.table.InputFormats; @@ -64,8 +63,7 @@ public class TableDefnRegistry // Guice later to allow extensions to define table types. private static final List BUILTIN_TABLE_DEFNS = Arrays.asList( new DatasourceDefn(), - new ExternalTableDefn(), - new IndexingTemplateDefn() + new ExternalTableDefn() ); private static final List BUILTIN_INPUT_SOURCE_DEFNS = Arrays.asList( new InlineInputSourceDefn(), diff --git a/server/src/main/java/org/apache/druid/catalog/model/table/IndexingTemplateDefn.java b/server/src/main/java/org/apache/druid/catalog/model/table/IndexingTemplateDefn.java deleted file mode 100644 index f11423ddb60e..000000000000 --- a/server/src/main/java/org/apache/druid/catalog/model/table/IndexingTemplateDefn.java +++ /dev/null @@ -1,65 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.druid.catalog.model.table; - -import com.fasterxml.jackson.core.type.TypeReference; -import org.apache.druid.catalog.model.ModelProperties; -import org.apache.druid.catalog.model.TableDefn; -import org.apache.druid.indexing.template.BatchIndexingJobTemplate; - -import java.util.List; - -/** - * Definition for indexing templates. - */ -public class IndexingTemplateDefn extends TableDefn -{ - public static final String TYPE = "index_template"; - - /** - * Property to contain template payload of type {@link BatchIndexingJobTemplate}. - * - * @see PayloadProperty#TYPE_REF - */ - public static final String PROPERTY_PAYLOAD = "payload"; - - public IndexingTemplateDefn() - { - super( - "Ingestion Template", - TYPE, - List.of(new PayloadProperty()), - null - ); - } - - /** - * Template payload property. - */ - public static class PayloadProperty extends ModelProperties.TypeRefPropertyDefn - { - public static final TypeReference TYPE_REF = new TypeReference<>() {}; - - public PayloadProperty() - { - super(PROPERTY_PAYLOAD, "Payload of the batch indexing template", TYPE_REF); - } - } -} diff --git a/server/src/main/java/org/apache/druid/server/coordinator/AutoCompactionSnapshot.java b/server/src/main/java/org/apache/druid/server/coordinator/AutoCompactionSnapshot.java index 29891cfdde44..b450dd5a5042 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/AutoCompactionSnapshot.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/AutoCompactionSnapshot.java @@ -204,6 +204,25 @@ public int hashCode() ); } + @Override + public String toString() + { + return "AutoCompactionSnapshot{" + + "dataSource='" + dataSource + '\'' + + ", scheduleStatus=" + scheduleStatus + + ", message='" + message + '\'' + + ", bytesAwaitingCompaction=" + bytesAwaitingCompaction + + ", bytesCompacted=" + bytesCompacted + + ", bytesSkipped=" + bytesSkipped + + ", segmentCountAwaitingCompaction=" + segmentCountAwaitingCompaction + + ", segmentCountCompacted=" + segmentCountCompacted + + ", segmentCountSkipped=" + segmentCountSkipped + + ", intervalCountAwaitingCompaction=" + intervalCountAwaitingCompaction + + ", intervalCountCompacted=" + intervalCountCompacted + + ", intervalCountSkipped=" + intervalCountSkipped + + '}'; + } + public static class Builder { private final String dataSource; diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/schema/DruidCalciteSchemaModule.java b/sql/src/main/java/org/apache/druid/sql/calcite/schema/DruidCalciteSchemaModule.java index 016972000c2a..14e1fc068824 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/schema/DruidCalciteSchemaModule.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/schema/DruidCalciteSchemaModule.java @@ -55,14 +55,12 @@ public void configure(Binder binder) binder.bind(SystemSchema.class).in(LazySingleton.class); binder.bind(InformationSchema.class).in(LazySingleton.class); binder.bind(LookupSchema.class).in(LazySingleton.class); - binder.bind(IndexingTemplateSchema.class).in(LazySingleton.class); // Binder to inject different schema to Calcite SqlBindings.addSchema(binder, NamedDruidSchema.class); SqlBindings.addSchema(binder, NamedSystemSchema.class); SqlBindings.addSchema(binder, NamedLookupSchema.class); SqlBindings.addSchema(binder, NamedViewSchema.class); - SqlBindings.addSchema(binder, NamedIndexingTemplateSchema.class); } @Provides diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/schema/IndexingTemplateSchema.java b/sql/src/main/java/org/apache/druid/sql/calcite/schema/IndexingTemplateSchema.java deleted file mode 100644 index 86c471264aa7..000000000000 --- a/sql/src/main/java/org/apache/druid/sql/calcite/schema/IndexingTemplateSchema.java +++ /dev/null @@ -1,84 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.druid.sql.calcite.schema; - -import com.google.inject.Inject; -import org.apache.calcite.schema.Table; -import org.apache.druid.catalog.MetadataCatalog; -import org.apache.druid.catalog.model.ResolvedTable; -import org.apache.druid.catalog.model.TableId; -import org.apache.druid.catalog.model.table.IndexingTemplateDefn; -import org.apache.druid.query.InlineDataSource; -import org.apache.druid.segment.column.ColumnType; -import org.apache.druid.segment.column.RowSignature; -import org.apache.druid.sql.calcite.table.InlineTable; - -import javax.annotation.Nullable; -import java.util.Collections; -import java.util.Map; -import java.util.Set; - -/** - * Creates the {@link TableId#INDEXING_TEMPLATE_SCHEMA indexing template schema} - * for Druid SQL. - */ -public class IndexingTemplateSchema extends AbstractTableSchema -{ - private static final RowSignature ROW_SIGNATURE = - RowSignature.builder() - .add("type", ColumnType.STRING) - .add("payload", ColumnType.UNKNOWN_COMPLEX) - .build(); - - private final MetadataCatalog catalog; - - @Inject - public IndexingTemplateSchema(MetadataCatalog catalog) - { - this.catalog = catalog; - } - - @Override - @Nullable - public Table getTable(String name) - { - final TableId tableId = TableId.of(TableId.INDEXING_TEMPLATE_SCHEMA, name); - final ResolvedTable resolvedTable = catalog.resolveTable(tableId); - if (resolvedTable == null) { - return null; - } - - final Map template - = resolvedTable.mapProperty(IndexingTemplateDefn.PROPERTY_PAYLOAD); - final InlineDataSource dataSource = InlineDataSource.fromIterable( - Collections.singletonList( - new Object[]{template.get("type"), template} - ), - ROW_SIGNATURE - ); - return new InlineTable(dataSource); - } - - @Override - public Set getTableNames() - { - return catalog.tableNames(TableId.INDEXING_TEMPLATE_SCHEMA); - } -} diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/schema/NamedIndexingTemplateSchema.java b/sql/src/main/java/org/apache/druid/sql/calcite/schema/NamedIndexingTemplateSchema.java deleted file mode 100644 index 230006085105..000000000000 --- a/sql/src/main/java/org/apache/druid/sql/calcite/schema/NamedIndexingTemplateSchema.java +++ /dev/null @@ -1,50 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.druid.sql.calcite.schema; - -import com.google.inject.Inject; -import org.apache.calcite.schema.Schema; -import org.apache.druid.catalog.model.TableId; - -/** - * Makes indexing template definitions accessible via SQL. - */ -public class NamedIndexingTemplateSchema implements NamedSchema -{ - private final IndexingTemplateSchema schema; - - @Inject - public NamedIndexingTemplateSchema(IndexingTemplateSchema schema) - { - this.schema = schema; - } - - @Override - public String getSchemaName() - { - return TableId.INDEXING_TEMPLATE_SCHEMA; - } - - @Override - public Schema getSchema() - { - return schema; - } -} diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/schema/DruidCalciteSchemaModuleTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/schema/DruidCalciteSchemaModuleTest.java index 50cdde4a5bd4..eb0ab607e82d 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/schema/DruidCalciteSchemaModuleTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/schema/DruidCalciteSchemaModuleTest.java @@ -166,8 +166,7 @@ public void testDruidCalciteSchemasAreInjected() NamedSystemSchema.class, NamedDruidSchema.class, NamedLookupSchema.class, - NamedViewSchema.class, - NamedIndexingTemplateSchema.class + NamedViewSchema.class ); Assertions.assertEquals(expectedSchemas.size(), sqlSchemas.size()); Assertions.assertEquals( @@ -211,15 +210,6 @@ public void testLookupSchemaIsInjectedAsSingleton() Assertions.assertSame(other, schema); } - @Test - public void testIndexingTemplateSchemaIsInjectedAsSingleton() - { - IndexingTemplateSchema schema = injector.getInstance(IndexingTemplateSchema.class); - Assertions.assertNotNull(schema); - IndexingTemplateSchema other = injector.getInstance(IndexingTemplateSchema.class); - Assertions.assertSame(other, schema); - } - @Test public void testRootSchemaAnnotatedIsInjectedAsSingleton() { diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/schema/IndexingTemplateSchemaTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/schema/IndexingTemplateSchemaTest.java deleted file mode 100644 index 915f4b6bb80c..000000000000 --- a/sql/src/test/java/org/apache/druid/sql/calcite/schema/IndexingTemplateSchemaTest.java +++ /dev/null @@ -1,116 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.druid.sql.calcite.schema; - -import org.apache.druid.catalog.MapMetadataCatalog; -import org.apache.druid.catalog.model.TableId; -import org.apache.druid.catalog.model.TableSpec; -import org.apache.druid.catalog.model.table.IndexingTemplateDefn; -import org.apache.druid.query.InlineDataSource; -import org.apache.druid.segment.TestHelper; -import org.apache.druid.segment.column.ColumnType; -import org.apache.druid.sql.calcite.table.InlineTable; -import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Test; - -import java.util.List; -import java.util.Map; -import java.util.Set; - -public class IndexingTemplateSchemaTest -{ - private IndexingTemplateSchema schema; - private MapMetadataCatalog catalog; - - @BeforeEach - public void setUp() - { - catalog = new MapMetadataCatalog(TestHelper.JSON_MAPPER); - schema = new IndexingTemplateSchema(catalog); - } - - @Test - public void test_getTableNames_returnsAllKnownTemplateIds() - { - Assertions.assertTrue(schema.getTableNames().isEmpty()); - - final String msqTemplate = "msq_1"; - final String compactTemplate = "compact_2"; - catalog.addSpec( - TableId.of(TableId.INDEXING_TEMPLATE_SCHEMA, msqTemplate), - new TableSpec(IndexingTemplateDefn.TYPE, null, null) - ); - catalog.addSpec( - TableId.of(TableId.INDEXING_TEMPLATE_SCHEMA, compactTemplate), - new TableSpec(IndexingTemplateDefn.TYPE, null, null) - ); - - final Set tableNames = schema.getTableNames(); - Assertions.assertEquals(Set.of(msqTemplate, compactTemplate), tableNames); - } - - @Test - public void test_getTable_returnsNull_forUnknownTable() - { - Assertions.assertNull(schema.getTable("msq")); - } - - @Test - public void test_getTable_returnsTable() - { - // Add a template to the catalog - final String msqTemplate = "msq_1"; - catalog.addSpec( - TableId.of(TableId.INDEXING_TEMPLATE_SCHEMA, msqTemplate), - new TableSpec( - IndexingTemplateDefn.TYPE, - Map.of("payload", Map.of("type", "msq", "granularity", "DAY")), - null - ) - ); - - final InlineTable inlineTable = Assertions.assertInstanceOf( - InlineTable.class, - schema.getTable(msqTemplate) - ); - Assertions.assertNotNull(inlineTable); - - final InlineDataSource dataSource = Assertions.assertInstanceOf( - InlineDataSource.class, - inlineTable.getDataSource() - ); - Assertions.assertEquals( - List.of("type", "payload"), - dataSource.getColumnNames() - ); - Assertions.assertEquals( - List.of(ColumnType.STRING, ColumnType.UNKNOWN_COMPLEX), - dataSource.getColumnTypes() - ); - - final List rows = dataSource.getRowsAsList(); - Assertions.assertEquals(1, rows.size()); - Assertions.assertArrayEquals( - rows.get(0), - new Object[]{"msq", Map.of("type", "msq", "granularity", "DAY")} - ); - } -} From 558e0b5385fcaad8082be65924904009fa60fe61 Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Thu, 30 Oct 2025 11:55:29 +0530 Subject: [PATCH 20/25] Remove extra changes --- .../apache/druid/catalog/model/SchemaRegistryImpl.java | 9 +++++---- .../java/org/apache/druid/catalog/model/TableId.java | 5 ----- 2 files changed, 5 insertions(+), 9 deletions(-) diff --git a/server/src/main/java/org/apache/druid/catalog/model/SchemaRegistryImpl.java b/server/src/main/java/org/apache/druid/catalog/model/SchemaRegistryImpl.java index 68922cf3ed5e..ff3b25b45e7b 100644 --- a/server/src/main/java/org/apache/druid/catalog/model/SchemaRegistryImpl.java +++ b/server/src/main/java/org/apache/druid/catalog/model/SchemaRegistryImpl.java @@ -19,12 +19,13 @@ package org.apache.druid.catalog.model; +import com.google.common.collect.ImmutableSet; import com.google.common.collect.Lists; import org.apache.druid.catalog.model.table.DatasourceDefn; import org.apache.druid.catalog.model.table.ExternalTableDefn; import org.apache.druid.server.security.ResourceType; -import java.util.Comparator; +import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -95,7 +96,7 @@ public SchemaRegistryImpl() register(new SchemaDefnImpl( TableId.DRUID_SCHEMA, ResourceType.DATASOURCE, - Set.of(DatasourceDefn.TABLE_TYPE) + ImmutableSet.of(DatasourceDefn.TABLE_TYPE) )); register(new SchemaDefnImpl( TableId.LOOKUP_SCHEMA, @@ -115,7 +116,7 @@ public SchemaRegistryImpl() register(new SchemaDefnImpl( TableId.EXTERNAL_SCHEMA, EXTERNAL_RESOURCE, - Set.of(ExternalTableDefn.TABLE_TYPE) + ImmutableSet.of(ExternalTableDefn.TABLE_TYPE) )); register(new SchemaDefnImpl( TableId.VIEW_SCHEMA, @@ -147,7 +148,7 @@ public List schemas() // No real need to sort every time. However, this is used infrequently, // so OK for now. List schemas = Lists.newArrayList(builtIns.values()); - schemas.sort(Comparator.comparing(SchemaSpec::name)); + Collections.sort(schemas, (s1, s2) -> s1.name().compareTo(s2.name())); return schemas; } } diff --git a/server/src/main/java/org/apache/druid/catalog/model/TableId.java b/server/src/main/java/org/apache/druid/catalog/model/TableId.java index 77265c5d7d0b..55fcc797561b 100644 --- a/server/src/main/java/org/apache/druid/catalog/model/TableId.java +++ b/server/src/main/java/org/apache/druid/catalog/model/TableId.java @@ -41,11 +41,6 @@ public class TableId // Extra for views public static final String VIEW_SCHEMA = "view"; - /** - * Schema for indexing templates. - */ - public static final String INDEXING_TEMPLATE_SCHEMA = "index_template"; - private final String schema; private final String name; From d8ce2a7933f652ed1c0bfc6b3b917b65d5328bfc Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Thu, 30 Oct 2025 12:21:38 +0530 Subject: [PATCH 21/25] Fix up test, remove extra files --- .../apache/druid/guice/SupervisorModule.java | 4 +- .../CompactionConfigBasedJobTemplate.java | 11 ---- .../indexing/compact/CompactionJobParams.java | 15 ++++- .../compact/CompactionJobTemplate.java | 60 ------------------- .../compact/CompactionSupervisor.java | 1 - .../input/DruidDatasourceDestination.java | 42 ------------- .../OverlordCompactionSchedulerTest.java | 7 +-- .../druid/data/output/OutputDestination.java | 33 ---------- .../template/BatchIndexingJobTemplate.java | 17 ++---- .../druid/indexing/template/JobParams.java | 44 -------------- 10 files changed, 19 insertions(+), 215 deletions(-) delete mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/input/DruidDatasourceDestination.java delete mode 100644 processing/src/main/java/org/apache/druid/data/output/OutputDestination.java delete mode 100644 server/src/main/java/org/apache/druid/indexing/template/JobParams.java diff --git a/indexing-service/src/main/java/org/apache/druid/guice/SupervisorModule.java b/indexing-service/src/main/java/org/apache/druid/guice/SupervisorModule.java index a5fc5fb4847a..5642fe1126d6 100644 --- a/indexing-service/src/main/java/org/apache/druid/guice/SupervisorModule.java +++ b/indexing-service/src/main/java/org/apache/druid/guice/SupervisorModule.java @@ -29,7 +29,6 @@ import org.apache.druid.indexing.compact.CompactionSupervisorSpec; import org.apache.druid.indexing.compact.InlineCompactionJobTemplate; import org.apache.druid.indexing.compact.MSQCompactionJobTemplate; -import org.apache.druid.indexing.input.DruidDatasourceDestination; import org.apache.druid.indexing.overlord.supervisor.SupervisorStateManagerConfig; import org.apache.druid.indexing.scheduledbatch.ScheduledBatchSupervisorSpec; import org.apache.druid.initialization.DruidModule; @@ -54,8 +53,7 @@ public List getJacksonModules() new NamedType(MSQCompactionJobTemplate.class, MSQCompactionJobTemplate.TYPE), new NamedType(CascadingCompactionTemplate.class, CascadingCompactionTemplate.TYPE), new NamedType(CompactionSupervisorSpec.class, CompactionSupervisorSpec.TYPE), - new NamedType(ScheduledBatchSupervisorSpec.class, ScheduledBatchSupervisorSpec.TYPE), - new NamedType(DruidDatasourceDestination.class, DruidDatasourceDestination.TYPE) + new NamedType(ScheduledBatchSupervisorSpec.class, ScheduledBatchSupervisorSpec.TYPE) ) ); } 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 1d1599848dad..c8563290a66d 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 @@ -22,7 +22,6 @@ import org.apache.druid.client.indexing.ClientCompactionTaskQuery; import org.apache.druid.error.DruidException; import org.apache.druid.error.InvalidInput; -import org.apache.druid.indexing.input.DruidDatasourceDestination; import org.apache.druid.indexing.input.DruidInputSource; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.granularity.Granularity; @@ -154,14 +153,4 @@ private void validateInput(DruidInputSource druidInputSource) ); } } - - private void validateOutput(DruidDatasourceDestination druidDestination) - { - if (!druidDestination.getDataSource().equals(config.getDataSource())) { - throw InvalidInput.exception( - "Datasource[%s] in compaction config does not match datasource[%s] in output destination", - config.getDataSource(), druidDestination.getDataSource() - ); - } - } } 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 551c26f7742f..0113f1b78bac 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 @@ -19,7 +19,6 @@ package org.apache.druid.indexing.compact; -import org.apache.druid.indexing.template.JobParams; import org.apache.druid.server.compaction.CompactionSnapshotBuilder; import org.apache.druid.server.coordinator.ClusterCompactionConfig; import org.apache.druid.timeline.SegmentTimeline; @@ -28,8 +27,9 @@ /** * Parameters used while creating a {@link CompactionJob} using a {@link CompactionJobTemplate}. */ -public class CompactionJobParams extends JobParams +public class CompactionJobParams { + private final DateTime scheduleStartTime; private final TimelineProvider timelineProvider; private final ClusterCompactionConfig clusterCompactionConfig; private final CompactionSnapshotBuilder snapshotBuilder; @@ -41,12 +41,21 @@ public CompactionJobParams( CompactionSnapshotBuilder snapshotBuilder ) { - super(scheduleStartTime); + this.scheduleStartTime = scheduleStartTime; this.clusterCompactionConfig = clusterCompactionConfig; this.timelineProvider = timelineProvider; this.snapshotBuilder = snapshotBuilder; } + /** + * Timestamp denoting the start of the current run of the scheduler which has + * triggered creation of jobs using these {@link CompactionJobParams}. + */ + public DateTime getScheduleStartTime() + { + return scheduleStartTime; + } + /** * Cluster-level compaction config containing details such as the engine, * compaction search policy, etc. to use while creating {@link CompactionJob}. diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionJobTemplate.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionJobTemplate.java index 5872fc8f66ab..cd5096a8cf8e 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionJobTemplate.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionJobTemplate.java @@ -19,19 +19,12 @@ package org.apache.druid.indexing.compact; -import org.apache.druid.data.input.InputSource; -import org.apache.druid.data.output.OutputDestination; -import org.apache.druid.error.InvalidInput; -import org.apache.druid.indexing.input.DruidDatasourceDestination; import org.apache.druid.indexing.input.DruidInputSource; -import org.apache.druid.indexing.template.BatchIndexingJob; import org.apache.druid.indexing.template.BatchIndexingJobTemplate; -import org.apache.druid.indexing.template.JobParams; import org.apache.druid.java.util.common.granularity.Granularity; import javax.annotation.Nullable; import java.util.List; -import java.util.stream.Collectors; /** * Base indexing template for creating {@link CompactionJob}. @@ -54,57 +47,4 @@ List createCompactionJobs( */ @Nullable Granularity getSegmentGranularity(); - - @Override - default List createJobs( - InputSource source, - OutputDestination destination, - JobParams jobParams - ) - { - if (!(jobParams instanceof CompactionJobParams)) { - throw InvalidInput.exception( - "Job params[%s] for compaction template must be of type CompactionJobParams.", - jobParams - ); - } - - final DruidInputSource druidInputSource = ensureDruidInputSource(source); - final DruidDatasourceDestination druidDestination = ensureDruidDataSourceDestination(destination); - if (!druidInputSource.getDataSource().equals(druidDestination.getDataSource())) { - throw InvalidInput.exception( - "Input datasource[%s] does not match output datasource[%s]", - druidInputSource.getDataSource(), druidDestination.getDataSource() - ); - } - - return createCompactionJobs(druidInputSource, (CompactionJobParams) jobParams) - .stream() - .map(job -> (BatchIndexingJob) job) - .collect(Collectors.toList()); - } - - /** - * Verifies that the input source is of type {@link DruidInputSource}. - */ - static DruidInputSource ensureDruidInputSource(InputSource inputSource) - { - if (inputSource instanceof DruidInputSource) { - return (DruidInputSource) inputSource; - } else { - throw InvalidInput.exception("Invalid input source[%s] for compaction", inputSource); - } - } - - /** - * Verifies that the output destination is of type {@link DruidDatasourceDestination}. - */ - static DruidDatasourceDestination ensureDruidDataSourceDestination(OutputDestination destination) - { - if (destination instanceof DruidDatasourceDestination) { - return (DruidDatasourceDestination) destination; - } else { - throw InvalidInput.exception("Invalid output destination[%s] for compaction", destination); - } - } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionSupervisor.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionSupervisor.java index b87cb150e387..f9ba0eee6db7 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionSupervisor.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionSupervisor.java @@ -19,7 +19,6 @@ package org.apache.druid.indexing.compact; -import org.apache.druid.indexing.input.DruidDatasourceDestination; import org.apache.druid.indexing.input.DruidInputSource; import org.apache.druid.indexing.overlord.DataSourceMetadata; import org.apache.druid.indexing.overlord.supervisor.Supervisor; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/input/DruidDatasourceDestination.java b/indexing-service/src/main/java/org/apache/druid/indexing/input/DruidDatasourceDestination.java deleted file mode 100644 index 7d10ce86bd75..000000000000 --- a/indexing-service/src/main/java/org/apache/druid/indexing/input/DruidDatasourceDestination.java +++ /dev/null @@ -1,42 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.druid.indexing.input; - -import org.apache.druid.data.output.OutputDestination; - -/** - * {@link OutputDestination} for writing out data into a Druid datasource. - */ -public class DruidDatasourceDestination implements OutputDestination -{ - public static final String TYPE = "druid"; - - private final String dataSource; - - public DruidDatasourceDestination(String dataSource) - { - this.dataSource = dataSource; - } - - public String getDataSource() - { - return dataSource; - } -} diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/compact/OverlordCompactionSchedulerTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/compact/OverlordCompactionSchedulerTest.java index 16ebefc20f83..1484a719db10 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/compact/OverlordCompactionSchedulerTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/compact/OverlordCompactionSchedulerTest.java @@ -23,7 +23,6 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Optional; import com.google.common.util.concurrent.Futures; -import org.apache.druid.catalog.MapMetadataCatalog; import org.apache.druid.client.broker.BrokerClient; import org.apache.druid.client.coordinator.NoopCoordinatorClient; import org.apache.druid.client.indexing.ClientMSQContext; @@ -141,7 +140,6 @@ public class OverlordCompactionSchedulerTest private StubServiceEmitter serviceEmitter; private String dataSource; - private MapMetadataCatalog catalog; private OverlordCompactionScheduler scheduler; private Map submittedMsqTaskIds; @@ -156,7 +154,6 @@ public void setUp() Mockito.when(taskRunner.getMaximumCapacityWithAutoscale()).thenReturn(100); taskQueue = Mockito.mock(TaskQueue.class); - catalog = new MapMetadataCatalog(OBJECT_MAPPER); submittedMsqTaskIds = new HashMap<>(); brokerClient = Mockito.mock(BrokerClient.class); @@ -408,7 +405,7 @@ public void test_startCompaction_enablesTaskSubmission_forDatasource() runCompactionTasks(1); final AutoCompactionSnapshot.Builder expectedSnapshot = AutoCompactionSnapshot.builder(dataSource); - expectedSnapshot.incrementCompactedStats(CompactionStatistics.create(100_000_000, 1, 1)); + expectedSnapshot.incrementWaitingStats(CompactionStatistics.create(100_000_000, 1, 1)); Assert.assertEquals( expectedSnapshot.build(), @@ -420,7 +417,7 @@ public void test_startCompaction_enablesTaskSubmission_forDatasource() ); serviceEmitter.verifyValue(Stats.Compaction.SUBMITTED_TASKS.getMetricName(), 1L); - serviceEmitter.verifyValue(Stats.Compaction.COMPACTED_BYTES.getMetricName(), 100_000_000L); + serviceEmitter.verifyValue(Stats.Compaction.PENDING_BYTES.getMetricName(), 100_000_000L); scheduler.stopBeingLeader(); } diff --git a/processing/src/main/java/org/apache/druid/data/output/OutputDestination.java b/processing/src/main/java/org/apache/druid/data/output/OutputDestination.java deleted file mode 100644 index 39721de67500..000000000000 --- a/processing/src/main/java/org/apache/druid/data/output/OutputDestination.java +++ /dev/null @@ -1,33 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.druid.data.output; - -import com.fasterxml.jackson.annotation.JsonSubTypes; -import com.fasterxml.jackson.annotation.JsonTypeInfo; -import org.apache.druid.data.input.InputSource; - -/** - * Destination where data is written out. - */ -@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = InputSource.TYPE_PROPERTY) -@JsonSubTypes(value = {}) -public interface OutputDestination -{ -} diff --git a/server/src/main/java/org/apache/druid/indexing/template/BatchIndexingJobTemplate.java b/server/src/main/java/org/apache/druid/indexing/template/BatchIndexingJobTemplate.java index d8e9251dfdf0..e8a09a6bb152 100644 --- a/server/src/main/java/org/apache/druid/indexing/template/BatchIndexingJobTemplate.java +++ b/server/src/main/java/org/apache/druid/indexing/template/BatchIndexingJobTemplate.java @@ -22,26 +22,17 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonTypeInfo; import org.apache.druid.data.input.InputSource; -import org.apache.druid.data.output.OutputDestination; - -import java.util.List; /** * ETL template to create a {@link BatchIndexingJob} that indexes data from an - * {@link InputSource} into an {@link OutputDestination}. + * {@link InputSource} into an output destination. + *

+ * This interface currently defines no methods and is used only as the root type + * for serialization of compaction template objects. */ @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type") public interface BatchIndexingJobTemplate { - /** - * Creates jobs with this template for the given input and output. - */ - List createJobs( - InputSource source, - OutputDestination destination, - JobParams jobParams - ); - /** * Unique type name of this template used for JSON serialization. */ diff --git a/server/src/main/java/org/apache/druid/indexing/template/JobParams.java b/server/src/main/java/org/apache/druid/indexing/template/JobParams.java deleted file mode 100644 index e365ad757979..000000000000 --- a/server/src/main/java/org/apache/druid/indexing/template/JobParams.java +++ /dev/null @@ -1,44 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.druid.indexing.template; - -import org.joda.time.DateTime; - -/** - * Provides parameters required to create a {@link BatchIndexingJob}. - */ -public class JobParams -{ - private final DateTime scheduleStartTime; - - public JobParams(DateTime scheduleStartTime) - { - this.scheduleStartTime = scheduleStartTime; - } - - /** - * Timestamp denoting the start of the current run of the scheduler which has - * triggered creation of jobs using these {@link JobParams}. - */ - public DateTime getScheduleStartTime() - { - return scheduleStartTime; - } -} From a67f4086cb1f514dde529def4c15ac5d9fbe7338 Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Thu, 30 Oct 2025 12:30:17 +0530 Subject: [PATCH 22/25] Update javadocs --- .../compaction/CompactionSlotManager.java | 20 +++++++++++++------ 1 file changed, 14 insertions(+), 6 deletions(-) diff --git a/server/src/main/java/org/apache/druid/server/compaction/CompactionSlotManager.java b/server/src/main/java/org/apache/druid/server/compaction/CompactionSlotManager.java index 5bdab3009cfa..428367db8ea0 100644 --- a/server/src/main/java/org/apache/druid/server/compaction/CompactionSlotManager.java +++ b/server/src/main/java/org/apache/druid/server/compaction/CompactionSlotManager.java @@ -206,24 +206,32 @@ public boolean cancelTaskOnlyIfGranularityChanged( * we skip these Intervals until the next compaction run by adding them to * {@link #intervalsToSkipCompaction}. *

+ * This method must be called after invalid compaction tasks have already been + * cancelled using {@link #cancelTaskOnlyIfGranularityChanged} so that their + * intervals are not considered locked. */ public void skipLockedIntervals(List compactionConfigs) { final List lockFilterPolicies = compactionConfigs .stream() - .map(config -> - new LockFilterPolicy(config.getDataSource(), config.getTaskPriority(), null, config.getTaskContext())) + .map( + config -> new LockFilterPolicy( + config.getDataSource(), + config.getTaskPriority(), + null, + config.getTaskContext() + ) + ) .collect(Collectors.toList()); - final Map> datasourceToLockedIntervals = - new HashMap<>(FutureUtils.getUnchecked(overlordClient.findLockedIntervals(lockFilterPolicies), true)); + final Map> datasourceToLockedIntervals = new HashMap<>( + FutureUtils.getUnchecked(overlordClient.findLockedIntervals(lockFilterPolicies), true) + ); log.debug( "Skipping the following intervals for Compaction as they are currently locked: %s", datasourceToLockedIntervals ); // Skip all the intervals locked by higher priority tasks for each datasource - // This must be done after the invalid compaction tasks are cancelled - // in the loop above so that their intervals are not considered locked datasourceToLockedIntervals.forEach( (dataSource, intervals) -> intervalsToSkipCompaction From 738f4f8632c7cbf62261a4098c2689909e9dd6f4 Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Thu, 30 Oct 2025 17:33:56 +0530 Subject: [PATCH 23/25] Fix up tests --- .../compact/CompactionSupervisorTest.java | 30 +---- .../compact/CatalogCompactionTest.java | 3 +- .../indexing/compact/CompactionJobQueue.java | 32 +++++- .../compact/OverlordCompactionScheduler.java | 108 +++++++++--------- .../CompactionStatusTrackerTest.java | 2 +- 5 files changed, 90 insertions(+), 85 deletions(-) diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionSupervisorTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionSupervisorTest.java index 0dba6f709b09..35ba77274f8b 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 @@ -166,35 +166,7 @@ public void test_ingestHourGranularity_andCompactToDayAndMonth_withInlineTemplat } @Test - public void test_ingestHourGranularity_andCompactToDayAndMonth_withCatalogTemplates() - { - ingestHourSegments(1200); - - // Add compaction templates to catalog - final CompactionJobTemplate dayGranularityTemplate = - new InlineCompactionJobTemplate(createMatcher(Granularities.DAY)); - final CompactionJobTemplate monthGranularityTemplate = - new InlineCompactionJobTemplate(createMatcher(Granularities.MONTH)); - - // Create a cascading template with DAY and MONTH granularity - CascadingCompactionTemplate cascadingTemplate = new CascadingCompactionTemplate( - dataSource, - List.of( - new CompactionRule(Period.days(1), dayGranularityTemplate), - new CompactionRule(Period.days(50), monthGranularityTemplate) - ) - ); - - runCompactionWithSpec(cascadingTemplate); - waitForAllCompactionTasksToFinish(); - - Assertions.assertEquals(0, getNumSegmentsWith(Granularities.HOUR)); - Assertions.assertTrue(getNumSegmentsWith(Granularities.DAY) >= 1); - Assertions.assertTrue(getNumSegmentsWith(Granularities.MONTH) >= 1); - } - - @Test - public void test_ingestHourGranularity_andCompactToDayAndMonth_withCatalogMSQTemplates() + public void test_ingestHourGranularity_andCompactToDayAndMonth_withMSQTemplates() { ingestHourSegments(1200); diff --git a/extensions-core/druid-catalog/src/test/java/org/apache/druid/catalog/compact/CatalogCompactionTest.java b/extensions-core/druid-catalog/src/test/java/org/apache/druid/catalog/compact/CatalogCompactionTest.java index 798c5f2183f9..ece6a9ef0573 100644 --- a/extensions-core/druid-catalog/src/test/java/org/apache/druid/catalog/compact/CatalogCompactionTest.java +++ b/extensions-core/druid-catalog/src/test/java/org/apache/druid/catalog/compact/CatalogCompactionTest.java @@ -53,7 +53,8 @@ public class CatalogCompactionTest extends EmbeddedClusterTestBase { private final EmbeddedOverlord overlord = new EmbeddedOverlord() - .addProperty("druid.catalog.client.maxSyncRetries", "0"); + .addProperty("druid.catalog.client.maxSyncRetries", "0") + .addProperty("druid.manager.segments.pollDuration", "PT1s"); private final EmbeddedCoordinator coordinator = new EmbeddedCoordinator() .addProperty("druid.manager.segments.useIncrementalCache", "always"); private final EmbeddedBroker broker = new EmbeddedBroker() 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 39a000bfe53e..77886af1a017 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 @@ -51,11 +51,15 @@ import org.apache.druid.server.coordinator.stats.Stats; import javax.annotation.Nullable; +import javax.annotation.concurrent.NotThreadSafe; import java.util.ArrayList; import java.util.HashMap; +import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.PriorityQueue; +import java.util.Set; +import java.util.stream.Collectors; /** * Iterates over all eligible compaction jobs in order of their priority. @@ -69,6 +73,7 @@ * target segment granulariy for different intervals of the same datasource. * The cancellation of invalid tasks has been left as a future enhancement. */ +@NotThreadSafe public class CompactionJobQueue { private static final Logger log = new Logger(CompactionJobQueue.class); @@ -86,9 +91,11 @@ public class CompactionJobQueue private final CompactionSnapshotBuilder snapshotBuilder; private final PriorityQueue queue; - private final Map submittedTaskIdToJob; private final CoordinatorRunStats runStats; + private final Set activeSupervisors; + private final Map submittedTaskIdToJob; + public CompactionJobQueue( DataSourcesSnapshot dataSourcesSnapshot, ClusterCompactionConfig clusterCompactionConfig, @@ -108,6 +115,7 @@ public CompactionJobQueue( (o1, o2) -> searchPolicy.compareCandidates(o1.getCandidate(), o2.getCandidate()) ); this.submittedTaskIdToJob = new HashMap<>(); + this.activeSupervisors = new HashSet<>(); this.jobParams = new CompactionJobParams( DateTimes.nowUtc(), clusterCompactionConfig, @@ -126,6 +134,9 @@ public CompactionJobQueue( /** * Creates jobs for the given {@link CompactionSupervisor} and adds them to * the job queue. + *

+ * This method is idempotent. If jobs for the given supervisor already exist + * in the queue, the method does nothing. */ public void createAndEnqueueJobs( CompactionSupervisor supervisor, @@ -135,11 +146,13 @@ public void createAndEnqueueJobs( final Stopwatch jobCreationTime = Stopwatch.createStarted(); final String supervisorId = supervisor.getSpec().getId(); try { - if (supervisor.shouldCreateJobs()) { + if (supervisor.shouldCreateJobs() && !activeSupervisors.contains(supervisorId)) { + // Queue fresh jobs final List jobs = supervisor.createJobs(source, jobParams); jobs.forEach(job -> snapshotBuilder.addToPending(job.getCandidate())); queue.addAll(jobs); + activeSupervisors.add(supervisorId); runStats.add( Stats.Compaction.CREATED_JOBS, @@ -162,6 +175,20 @@ public void createAndEnqueueJobs( } } + /** + * Removes all existing jobs for the given datasource from the queue. + */ + public void removeJobs(String dataSource) + { + final List jobsToRemove = queue + .stream() + .filter(job -> job.getDataSource().equals(dataSource)) + .collect(Collectors.toList()); + + queue.removeAll(jobsToRemove); + log.info("Removed [%d] jobs for datasource[%s] from queue.", jobsToRemove.size(), dataSource); + } + /** * Submits jobs which are ready to either the Overlord or a Broker (if it is * an MSQ SQL job). @@ -218,7 +245,6 @@ public CoordinatorRunStats getRunStats() */ public Map getSnapshots() { - // TODO: fix the stats problem return snapshotBuilder.build(); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/OverlordCompactionScheduler.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/OverlordCompactionScheduler.java index 6dbb43b4ae2c..c3bce6a09de1 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/compact/OverlordCompactionScheduler.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/OverlordCompactionScheduler.java @@ -54,11 +54,9 @@ import org.apache.druid.server.coordinator.DataSourceCompactionConfig; import org.apache.druid.server.coordinator.DruidCompactionConfig; import org.apache.druid.server.coordinator.duty.CompactSegments; -import org.apache.druid.server.coordinator.stats.CoordinatorRunStats; import org.apache.druid.server.coordinator.stats.CoordinatorStat; import org.apache.druid.server.coordinator.stats.RowKey; import org.apache.druid.server.coordinator.stats.Stats; -import org.joda.time.Duration; import java.util.Collections; import java.util.List; @@ -68,6 +66,7 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Consumer; import java.util.stream.Collectors; /** @@ -87,8 +86,13 @@ public class OverlordCompactionScheduler implements CompactionScheduler { private static final Logger log = new Logger(OverlordCompactionScheduler.class); + /** + * Scheduler run period is 15 minutes. It has been kept high to avoid eager + * recomputation of the queue as it may be a very compute-intensive operation + * taking upto several minutes on clusters with a large number of used segments. + * Jobs for a single supervisor may still be recomputed when the supervisor is updated. + */ private static final long DEFAULT_SCHEDULE_PERIOD_MILLIS = 15 * 60_000; - private static final Duration METRIC_EMISSION_PERIOD = Duration.standardMinutes(5); private final SegmentsMetadataManager segmentManager; private final LocalOverlordClient overlordClient; @@ -101,6 +105,7 @@ public class OverlordCompactionScheduler implements CompactionScheduler private final ConcurrentHashMap activeSupervisors; private final AtomicReference> datasourceToCompactionSnapshot; + private final AtomicBoolean shouldRecomputeJobsForAnyDatasource = new AtomicBoolean(false); /** * Compaction job queue built in the last invocation of {@link #resetCompactionJobQueue()}. @@ -134,8 +139,6 @@ public class OverlordCompactionScheduler implements CompactionScheduler private final boolean shouldPollSegments; private final long schedulePeriodMillis; - private final Stopwatch sinceStatsEmitted = Stopwatch.createUnstarted(); - @Inject public OverlordCompactionScheduler( TaskMaster taskMaster, @@ -218,8 +221,8 @@ public synchronized void stop() public void becomeLeader() { if (isLeader.compareAndSet(false, true)) { - log.info("Running compaction scheduler with period [%d] millis.", schedulePeriodMillis); - scheduleOnExecutor(this::scheduledRun, schedulePeriodMillis); + // Schedule first run after a small delay + scheduleOnExecutor(this::scheduledRun, 1_000L); } } @@ -251,10 +254,15 @@ public CompactionConfigValidationResult validateCompactionConfig(DataSourceCompa @Override public void startCompaction(String dataSourceName, CompactionSupervisor supervisor) { - // Track active datasources even if scheduler has not started yet because + // Track active supervisors even if scheduler has not started yet because // SupervisorManager is started before the scheduler if (isEnabled()) { activeSupervisors.put(dataSourceName, supervisor); + + if (started.get()) { + shouldRecomputeJobsForAnyDatasource.set(true); + scheduleOnExecutor(() -> recreateJobs(dataSourceName, supervisor), 0L); + } } } @@ -262,6 +270,7 @@ public void startCompaction(String dataSourceName, CompactionSupervisor supervis public void stopCompaction(String dataSourceName) { activeSupervisors.remove(dataSourceName); + updateQueueIfComputed(queue -> queue.removeJobs(dataSourceName)); statusTracker.removeDatasource(dataSourceName); } @@ -274,7 +283,7 @@ private synchronized void initState() return; } - log.info("Starting compaction scheduler."); + log.info("Starting compaction scheduler with period [%d] millis.", schedulePeriodMillis); final Optional taskRunnerOptional = taskMaster.getTaskRunner(); if (taskRunnerOptional.isPresent()) { taskRunnerOptional.get().registerListener(taskRunnerListener, executor); @@ -300,6 +309,7 @@ private synchronized void cleanupState() } statusTracker.stop(); activeSupervisors.clear(); + latestJobQueue.set(null); if (shouldPollSegments) { segmentManager.stopPollingDatabasePeriodically(); @@ -334,7 +344,7 @@ private synchronized void scheduledRun() scheduleOnExecutor(this::scheduledRun, schedulePeriodMillis); } else { cleanupState(); - scheduleOnExecutor(this::scheduledRun, schedulePeriodMillis * 4); + scheduleOnExecutor(this::scheduledRun, schedulePeriodMillis); } } @@ -343,6 +353,9 @@ private synchronized void scheduledRun() */ private synchronized void resetCompactionJobQueue() { + // Remove the old queue so that no more jobs are added to it + latestJobQueue.set(null); + final Stopwatch runDuration = Stopwatch.createStarted(); final DataSourcesSnapshot dataSourcesSnapshot = getDatasourceSnapshot(); final CompactionJobQueue queue = new CompactionJobQueue( @@ -359,14 +372,14 @@ private synchronized void resetCompactionJobQueue() statusTracker.resetActiveDatasources(activeSupervisors.keySet()); statusTracker.onSegmentTimelineUpdated(dataSourcesSnapshot.getSnapshotTime()); - activeSupervisors.forEach( - (datasource, supervisor) -> queue.createAndEnqueueJobs( - supervisor, - druidInputSourceFactory.create(datasource, Intervals.ETERNITY) - ) - ); + + // Jobs for all active supervisors are being freshly created + // recomputation will not be needed + shouldRecomputeJobsForAnyDatasource.set(false); + activeSupervisors.forEach(this::createAndEnqueueJobs); + launchPendingJobs(); - emitStatsIfPeriodHasElapsed(queue.getRunStats()); + queue.getRunStats().forEachStat(this::emitStat); emitStat(Stats.Compaction.SCHEDULER_RUN_TIME, RowKey.empty(), runDuration.millisElapsed()); } @@ -376,37 +389,38 @@ private synchronized void resetCompactionJobQueue() */ private synchronized void launchPendingJobs() { - final CompactionJobQueue queue = latestJobQueue.get(); - if (queue == null) { - // Job queue has not been built yet - return; - } + updateQueueIfComputed(queue -> { + queue.runReadyJobs(); + updateCompactionSnapshots(queue); + }); + } - queue.runReadyJobs(); - updateCompactionSnapshots(queue); + private synchronized void recreateJobs(String dataSource, CompactionSupervisor supervisor) + { + if (shouldRecomputeJobsForAnyDatasource.get()) { + createAndEnqueueJobs(dataSource, supervisor); + } + } - // TODO: Try to trigger a recompute in these cases: - // supervisor spec updated - schedule recompute atleast for that supervisor? - // dynamic config updated - schedule full recompute? - // - // supervisor reset? not needed right now? we can probably do it later + private synchronized void createAndEnqueueJobs(String dataSource, CompactionSupervisor supervisor) + { + updateQueueIfComputed( + queue -> queue.createAndEnqueueJobs( + supervisor, + druidInputSourceFactory.create(dataSource, Intervals.ETERNITY) + ) + ); } /** - * Emits stats if {@link #METRIC_EMISSION_PERIOD} has elapsed. + * Performs an operation on the {@link #latestJobQueue} if it has been already + * computed. */ - private void emitStatsIfPeriodHasElapsed(CoordinatorRunStats stats) + private void updateQueueIfComputed(Consumer operation) { - // Emit stats only if emission period has elapsed - if (!sinceStatsEmitted.isRunning() || sinceStatsEmitted.hasElapsed(METRIC_EMISSION_PERIOD)) { - stats.forEachStat(this::emitStat); - sinceStatsEmitted.restart(); - } else { - // Always emit number of submitted tasks and interval statuses - stats.forEachEntry(Stats.Compaction.SUBMITTED_TASKS, this::emitNonZeroStat); - stats.forEachEntry(Stats.Compaction.COMPACTED_INTERVALS, this::emitNonZeroStat); - stats.forEachEntry(Stats.Compaction.SKIPPED_INTERVALS, this::emitNonZeroStat); - stats.forEachEntry(Stats.Compaction.PENDING_INTERVALS, this::emitStat); + final CompactionJobQueue queue = latestJobQueue.get(); + if (queue != null) { + operation.accept(queue); } } @@ -414,11 +428,10 @@ private void onTaskFinished(String taskId, TaskStatus taskStatus) { statusTracker.onTaskFinished(taskId, taskStatus); - final CompactionJobQueue queue = latestJobQueue.get(); - if (queue != null) { + updateQueueIfComputed(queue -> { queue.onTaskFinished(taskId, taskStatus); updateCompactionSnapshots(queue); - } + }); } private void updateCompactionSnapshots(CompactionJobQueue queue) @@ -467,13 +480,6 @@ public CompactionSimulateResult simulateRunWithConfigUpdate(ClusterCompactionCon } } - private void emitNonZeroStat(CoordinatorStat stat, RowKey rowKey, long value) - { - if (value > 0) { - emitStat(stat, rowKey, value); - } - } - private void emitStat(CoordinatorStat stat, RowKey rowKey, long value) { if (!stat.shouldEmit()) { diff --git a/server/src/test/java/org/apache/druid/server/compaction/CompactionStatusTrackerTest.java b/server/src/test/java/org/apache/druid/server/compaction/CompactionStatusTrackerTest.java index 4f82f9376fac..c0496b0705c2 100644 --- a/server/src/test/java/org/apache/druid/server/compaction/CompactionStatusTrackerTest.java +++ b/server/src/test/java/org/apache/druid/server/compaction/CompactionStatusTrackerTest.java @@ -112,7 +112,7 @@ public void testComputeCompactionStatusForSuccessfulTask() CompactionStatus status = statusTracker.computeCompactionStatus(candidateSegments, policy); Assert.assertEquals(CompactionStatus.State.PENDING, status.getState()); - Assert.assertEquals("not compacted yet", status.getReason()); + Assert.assertEquals("Not compacted yet", status.getReason()); // Verify that interval is skipped for compaction after task has finished statusTracker.onSegmentTimelineUpdated(DateTimes.nowUtc().minusMinutes(1)); From 1be4e042e32b3862b499d0d2986c26607d9e4b49 Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Thu, 30 Oct 2025 20:55:07 +0530 Subject: [PATCH 24/25] Remove new template implementations --- .../compact/CompactionSupervisorTest.java | 150 - .../apache/druid/guice/SupervisorModule.java | 6 - .../compact/CascadingCompactionTemplate.java | 246 - .../CompactionConfigBasedJobTemplate.java | 17 - .../compact/CompactionStateMatcher.java | 186 - .../compact/CompactionSupervisorSpec.java | 6 +- .../compact/InlineCompactionJobTemplate.java | 98 - .../compact/MSQCompactionJobTemplate.java | 191 - .../OverlordCompactionSchedulerTest.java | 206 - stuff.diff | 7184 +++++++++++++++++ 10 files changed, 7185 insertions(+), 1105 deletions(-) delete mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/compact/CascadingCompactionTemplate.java delete mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionStateMatcher.java delete mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/compact/InlineCompactionJobTemplate.java delete mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/compact/MSQCompactionJobTemplate.java create mode 100644 stuff.diff 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 35ba77274f8b..e11563c03897 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionSupervisorTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionSupervisorTest.java @@ -23,15 +23,8 @@ import org.apache.druid.catalog.guice.CatalogCoordinatorModule; import org.apache.druid.common.utils.IdUtils; import org.apache.druid.indexing.common.task.IndexTask; -import org.apache.druid.indexing.compact.CascadingCompactionTemplate; -import org.apache.druid.indexing.compact.CompactionJobTemplate; -import org.apache.druid.indexing.compact.CompactionRule; -import org.apache.druid.indexing.compact.CompactionStateMatcher; import org.apache.druid.indexing.compact.CompactionSupervisorSpec; -import org.apache.druid.indexing.compact.InlineCompactionJobTemplate; -import org.apache.druid.indexing.compact.MSQCompactionJobTemplate; import org.apache.druid.indexing.overlord.Segments; -import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.msq.guice.IndexerMemoryManagementModule; @@ -40,7 +33,6 @@ import org.apache.druid.msq.guice.MSQSqlModule; import org.apache.druid.msq.guice.SqlTaskModule; import org.apache.druid.query.DruidMetrics; -import org.apache.druid.query.http.ClientSqlQuery; import org.apache.druid.rpc.UpdateResponse; import org.apache.druid.server.coordinator.ClusterCompactionConfig; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; @@ -57,13 +49,11 @@ import org.apache.druid.testing.embedded.junit5.EmbeddedClusterTestBase; import org.hamcrest.Matcher; import org.hamcrest.Matchers; -import org.joda.time.DateTime; import org.joda.time.Period; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; -import java.util.List; import java.util.Map; /** @@ -144,120 +134,6 @@ public void test_ingestDayGranularity_andCompactToMonthGranularity_withInlineCon Assertions.assertEquals(1, getNumSegmentsWith(Granularities.MONTH)); } - @Test - public void test_ingestHourGranularity_andCompactToDayAndMonth_withInlineTemplates() - { - // Create a cascading template with DAY and MONTH granularity - CascadingCompactionTemplate cascadingTemplate = new CascadingCompactionTemplate( - dataSource, - List.of( - new CompactionRule(Period.days(1), new InlineCompactionJobTemplate(createMatcher(Granularities.DAY))), - new CompactionRule(Period.days(50), new InlineCompactionJobTemplate(createMatcher(Granularities.MONTH))) - ) - ); - - ingestHourSegments(1000); - runCompactionWithSpec(cascadingTemplate); - waitForAllCompactionTasksToFinish(); - - Assertions.assertEquals(0, getNumSegmentsWith(Granularities.HOUR)); - Assertions.assertTrue(getNumSegmentsWith(Granularities.DAY) >= 1); - Assertions.assertTrue(getNumSegmentsWith(Granularities.MONTH) >= 1); - } - - @Test - public void test_ingestHourGranularity_andCompactToDayAndMonth_withMSQTemplates() - { - ingestHourSegments(1200); - - // Add compaction templates to catalog - final String sqlDayGranularity = - "REPLACE INTO ${dataSource}" - + " OVERWRITE WHERE __time >= TIMESTAMP '${startTimestamp}' AND __time < TIMESTAMP '${endTimestamp}'" - + " SELECT * FROM ${dataSource}" - + " WHERE __time BETWEEN '${startTimestamp}' AND '${endTimestamp}'" - + " PARTITIONED BY DAY"; - final CompactionJobTemplate dayGranularityTemplate = new MSQCompactionJobTemplate( - new ClientSqlQuery(sqlDayGranularity, null, false, false, false, null, null), - createMatcher(Granularities.DAY) - ); - final String sqlMonthGranularity = - "REPLACE INTO ${dataSource}" - + " OVERWRITE WHERE __time >= TIMESTAMP '${startTimestamp}' AND __time < TIMESTAMP '${endTimestamp}'" - + " SELECT * FROM ${dataSource}" - + " WHERE __time >= TIMESTAMP '${startTimestamp}' AND __time < TIMESTAMP '${endTimestamp}'" - + " PARTITIONED BY MONTH"; - final CompactionJobTemplate monthGranularityTemplate = new MSQCompactionJobTemplate( - new ClientSqlQuery(sqlMonthGranularity, null, false, false, false, null, null), - createMatcher(Granularities.MONTH) - ); - - // Create a cascading template with DAY and MONTH granularity - CascadingCompactionTemplate cascadingTemplate = new CascadingCompactionTemplate( - dataSource, - List.of( - new CompactionRule(Period.days(1), dayGranularityTemplate), - new CompactionRule(Period.days(50), monthGranularityTemplate) - ) - ); - - runCompactionWithSpec(cascadingTemplate); - waitForAllCompactionTasksToFinish(); - - Assertions.assertEquals(0, getNumSegmentsWith(Granularities.HOUR)); - Assertions.assertTrue(getNumSegmentsWith(Granularities.DAY) >= 1); - Assertions.assertTrue(getNumSegmentsWith(Granularities.MONTH) >= 1); - } - - @Test - public void test_ingestHourGranularity_andCompactToDayAndMonth_withMixedTemplates() - { - ingestHourSegments(1200); - - // Add compaction templates to catalog - final String sqlDayGranularity = - "REPLACE INTO ${dataSource}" - + " OVERWRITE WHERE __time >= TIMESTAMP '${startTimestamp}' AND __time < TIMESTAMP '${endTimestamp}'" - + " SELECT * FROM ${dataSource}" - + " WHERE __time BETWEEN '${startTimestamp}' AND '${endTimestamp}'" - + " PARTITIONED BY DAY"; - final MSQCompactionJobTemplate dayTemplate = new MSQCompactionJobTemplate( - new ClientSqlQuery(sqlDayGranularity, null, false, false, false, null, null), - createMatcher(Granularities.DAY) - ); - final CompactionJobTemplate weekTemplate = - new InlineCompactionJobTemplate(createMatcher(Granularities.WEEK)); - final InlineCompactionJobTemplate monthTemplate = - new InlineCompactionJobTemplate(createMatcher(Granularities.MONTH)); - - // Compact last 1 day to DAY, next 14 days to WEEK, then 1 more DAY, rest to MONTH - CascadingCompactionTemplate cascadingTemplate = new CascadingCompactionTemplate( - dataSource, - List.of( - new CompactionRule(Period.days(1), dayTemplate), - new CompactionRule(Period.days(15), weekTemplate), - new CompactionRule(Period.days(16), dayTemplate), - new CompactionRule(Period.ZERO, monthTemplate) - ) - ); - - runCompactionWithSpec(cascadingTemplate); - waitForAllCompactionTasksToFinish(); - - Assertions.assertEquals(0, getNumSegmentsWith(Granularities.HOUR)); - Assertions.assertTrue(getNumSegmentsWith(Granularities.DAY) >= 1); - Assertions.assertTrue(getNumSegmentsWith(Granularities.WEEK) >= 1); - Assertions.assertTrue(getNumSegmentsWith(Granularities.MONTH) >= 1); - } - - private void ingestHourSegments(int numSegments) - { - runIngestionAtGranularity( - "HOUR", - createHourlyInlineDataCsv(DateTimes.nowUtc(), numSegments) - ); - } - private void runCompactionWithSpec(DataSourceCompactionConfig config) { final CompactionSupervisorSpec compactionSupervisor @@ -316,30 +192,4 @@ private void runIngestionAtGranularity( .withId(IdUtils.getRandomId()); cluster.callApi().runTask(task, overlord); } - - private String createHourlyInlineDataCsv(DateTime latestRecordTimestamp, int numRecords) - { - final StringBuilder builder = new StringBuilder(); - for (int i = 0; i < numRecords; ++i) { - builder.append(latestRecordTimestamp.minusHours(i)) - .append(",").append("item_").append(IdUtils.getRandomId()) - .append(",").append(0) - .append("\n"); - } - - return builder.toString(); - } - - private static CompactionStateMatcher createMatcher(Granularity segmentGranularity) - { - return new CompactionStateMatcher( - null, - null, - null, - null, - null, - new UserCompactionTaskGranularityConfig(segmentGranularity, null, null), - null - ); - } } diff --git a/indexing-service/src/main/java/org/apache/druid/guice/SupervisorModule.java b/indexing-service/src/main/java/org/apache/druid/guice/SupervisorModule.java index 5642fe1126d6..73e8e06e8964 100644 --- a/indexing-service/src/main/java/org/apache/druid/guice/SupervisorModule.java +++ b/indexing-service/src/main/java/org/apache/druid/guice/SupervisorModule.java @@ -25,10 +25,7 @@ import com.fasterxml.jackson.databind.module.SimpleModule; import com.google.common.collect.ImmutableList; import com.google.inject.Binder; -import org.apache.druid.indexing.compact.CascadingCompactionTemplate; import org.apache.druid.indexing.compact.CompactionSupervisorSpec; -import org.apache.druid.indexing.compact.InlineCompactionJobTemplate; -import org.apache.druid.indexing.compact.MSQCompactionJobTemplate; import org.apache.druid.indexing.overlord.supervisor.SupervisorStateManagerConfig; import org.apache.druid.indexing.scheduledbatch.ScheduledBatchSupervisorSpec; import org.apache.druid.initialization.DruidModule; @@ -49,9 +46,6 @@ public List getJacksonModules() return ImmutableList.of( new SimpleModule(getClass().getSimpleName()) .registerSubtypes( - new NamedType(InlineCompactionJobTemplate.class, InlineCompactionJobTemplate.TYPE), - new NamedType(MSQCompactionJobTemplate.class, MSQCompactionJobTemplate.TYPE), - new NamedType(CascadingCompactionTemplate.class, CascadingCompactionTemplate.TYPE), new NamedType(CompactionSupervisorSpec.class, CompactionSupervisorSpec.TYPE), new NamedType(ScheduledBatchSupervisorSpec.class, ScheduledBatchSupervisorSpec.TYPE) ) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CascadingCompactionTemplate.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CascadingCompactionTemplate.java deleted file mode 100644 index 89d389743667..000000000000 --- a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CascadingCompactionTemplate.java +++ /dev/null @@ -1,246 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.druid.indexing.compact; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; -import org.apache.druid.data.input.impl.AggregateProjectionSpec; -import org.apache.druid.error.InvalidInput; -import org.apache.druid.indexer.CompactionEngine; -import org.apache.druid.indexing.input.DruidInputSource; -import org.apache.druid.java.util.common.DateTimes; -import org.apache.druid.java.util.common.granularity.Granularity; -import org.apache.druid.query.aggregation.AggregatorFactory; -import org.apache.druid.segment.transform.CompactionTransformSpec; -import org.apache.druid.server.coordinator.DataSourceCompactionConfig; -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; -import org.joda.time.DateTime; -import org.joda.time.Interval; -import org.joda.time.Period; - -import javax.annotation.Nullable; -import java.util.ArrayList; -import java.util.List; -import java.util.Map; -import java.util.Objects; - -/** - * Template to perform period-based cascading compaction. Contains a list of - * {@link CompactionRule} which divide the segment timeline into compactible - * intervals. Each rule specifies a period relative to the current time which is - * used to determine its applicable interval: - *

    - *
  • Rule 1: range = [now - p1, +inf)
  • - *
  • Rule 2: range = [now - p2, now - p1)
  • - *
  • ...
  • - *
  • Rule n: range = (-inf, now - p(n - 1))
  • - *
- * - * If two adjacent rules explicitly specify a segment granularity, the boundary - * between them may be {@linkplain CompactionRule#computeStartTime adjusted} - * to ensure that there are no uncompacted gaps in the timeline. - *

- * This template never needs to be deserialized as a {@code BatchIndexingJobTemplate}, - * only as a {@link DataSourceCompactionConfig} in {@link CompactionSupervisorSpec}. - */ -public class CascadingCompactionTemplate implements CompactionJobTemplate, DataSourceCompactionConfig -{ - public static final String TYPE = "compactCascade"; - - private final String dataSource; - private final List rules; - - @JsonCreator - public CascadingCompactionTemplate( - @JsonProperty("dataSource") String dataSource, - @JsonProperty("rules") List rules - ) - { - this.rules = rules; - this.dataSource = Objects.requireNonNull(dataSource, "'dataSource' cannot be null"); - - InvalidInput.conditionalException(rules != null && !rules.isEmpty(), "'rules' cannot be empty"); - } - - @Override - @JsonProperty - public String getDataSource() - { - return dataSource; - } - - @JsonProperty - public List getRules() - { - return rules; - } - - @Override - public List createCompactionJobs( - DruidInputSource source, - CompactionJobParams jobParams - ) - { - final List allJobs = new ArrayList<>(); - - // Include future dates in the first rule - final DateTime currentTime = jobParams.getScheduleStartTime(); - DateTime previousRuleStartTime = DateTimes.MAX; - for (int i = 0; i < rules.size() - 1; ++i) { - final CompactionRule rule = rules.get(i); - final DateTime ruleStartTime = rule.computeStartTime(currentTime, rules.get(i + 1)); - final Interval ruleInterval = new Interval(ruleStartTime, previousRuleStartTime); - - allJobs.addAll( - createJobsForSearchInterval(rule.getTemplate(), ruleInterval, source, jobParams) - ); - - previousRuleStartTime = ruleStartTime; - } - - // Include past dates in the last rule - final CompactionRule lastRule = rules.get(rules.size() - 1); - final Interval lastRuleInterval = new Interval(DateTimes.MIN, previousRuleStartTime); - allJobs.addAll( - createJobsForSearchInterval(lastRule.getTemplate(), lastRuleInterval, source, jobParams) - ); - - return allJobs; - } - - private List createJobsForSearchInterval( - CompactionJobTemplate template, - Interval searchInterval, - DruidInputSource inputSource, - CompactionJobParams jobParams - ) - { - return template.createCompactionJobs( - inputSource.withInterval(searchInterval), - jobParams - ); - } - - @Override - public String getType() - { - return TYPE; - } - - // Legacy fields from DataSourceCompactionConfig that are not used by this template - - @Nullable - @Override - public CompactionEngine getEngine() - { - return null; - } - - @Override - public int getTaskPriority() - { - return 0; - } - - @Override - public long getInputSegmentSizeBytes() - { - return 0; - } - - @Nullable - @Override - public Integer getMaxRowsPerSegment() - { - return 0; - } - - @Override - public Period getSkipOffsetFromLatest() - { - return null; - } - - @Nullable - @Override - public UserCompactionTaskQueryTuningConfig getTuningConfig() - { - return null; - } - - @Nullable - @Override - public UserCompactionTaskIOConfig getIoConfig() - { - return null; - } - - @Nullable - @Override - public Map getTaskContext() - { - return Map.of(); - } - - @Nullable - @Override - public Granularity getSegmentGranularity() - { - return null; - } - - @Nullable - @Override - public UserCompactionTaskGranularityConfig getGranularitySpec() - { - return null; - } - - @Nullable - @Override - public List getProjections() - { - return List.of(); - } - - @Nullable - @Override - public CompactionTransformSpec getTransformSpec() - { - return null; - } - - @Nullable - @Override - public UserCompactionTaskDimensionsConfig getDimensionsSpec() - { - return null; - } - - @Nullable - @Override - public AggregatorFactory[] getMetricsSpec() - { - return new AggregatorFactory[0]; - } -} 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 c8563290a66d..fd89714dadb6 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 @@ -30,11 +30,9 @@ import org.apache.druid.server.compaction.DataSourceCompactibleSegmentIterator; import org.apache.druid.server.compaction.NewestSegmentFirstPolicy; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; -import org.apache.druid.server.coordinator.InlineSchemaDataSourceCompactionConfig; import org.apache.druid.server.coordinator.duty.CompactSegments; import org.apache.druid.timeline.SegmentTimeline; import org.joda.time.Interval; -import org.joda.time.Period; import javax.annotation.Nullable; import java.util.ArrayList; @@ -55,21 +53,6 @@ public CompactionConfigBasedJobTemplate(DataSourceCompactionConfig config) this.config = config; } - public static CompactionConfigBasedJobTemplate create(String dataSource, CompactionStateMatcher stateMatcher) - { - return new CompactionConfigBasedJobTemplate( - InlineSchemaDataSourceCompactionConfig - .builder() - .forDataSource(dataSource) - .withSkipOffsetFromLatest(Period.ZERO) - .withTransformSpec(stateMatcher.getTransformSpec()) - .withProjections(stateMatcher.getProjections()) - .withMetricsSpec(stateMatcher.getMetricsSpec()) - .withGranularitySpec(stateMatcher.getGranularitySpec()) - .build() - ); - } - @Nullable @Override public Granularity getSegmentGranularity() diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionStateMatcher.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionStateMatcher.java deleted file mode 100644 index 84a95e2f6e97..000000000000 --- a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionStateMatcher.java +++ /dev/null @@ -1,186 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.druid.indexing.compact; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonInclude; -import com.fasterxml.jackson.annotation.JsonProperty; -import org.apache.druid.data.input.impl.AggregateProjectionSpec; -import org.apache.druid.data.input.impl.DimensionsSpec; -import org.apache.druid.indexer.partitions.PartitionsSpec; -import org.apache.druid.java.util.common.granularity.Granularity; -import org.apache.druid.query.aggregation.AggregatorFactory; -import org.apache.druid.segment.IndexSpec; -import org.apache.druid.segment.transform.CompactionTransformSpec; -import org.apache.druid.server.coordinator.UserCompactionTaskGranularityConfig; - -import javax.annotation.Nullable; -import java.util.Arrays; -import java.util.List; -import java.util.Objects; - -/** - * Target compacted state of segments used to determine if compaction is needed - * for an interval. An explicitly defined target state helps avoid superfluous - * compaction when only the job definition has changed. - *

- * This class is mostly a duplicate of {@code CompactionState} but is kept - * separate to allow: - *

    - *
  • fields to be nullable so that only non-null fields are used for matching
  • - *
  • legacy "compaction-incompatible" fields such as {@link #transformSpec} to - * be removed in the future. These fields do not just change the layout/partitioning - * of the data but may also alter its meaning which does not fall in the purview - * of compaction.
  • - *
- */ -@JsonInclude(JsonInclude.Include.NON_NULL) -public class CompactionStateMatcher -{ - private final PartitionsSpec partitionsSpec; - private final DimensionsSpec dimensionsSpec; - private final CompactionTransformSpec transformSpec; - private final IndexSpec indexSpec; - private final UserCompactionTaskGranularityConfig granularitySpec; - private final AggregatorFactory[] metricsSpec; - private final List projections; - - @JsonCreator - public CompactionStateMatcher( - @JsonProperty("partitionsSpec") @Nullable PartitionsSpec partitionsSpec, - @JsonProperty("dimensionsSpec") @Nullable DimensionsSpec dimensionsSpec, - @JsonProperty("metricsSpec") @Nullable AggregatorFactory[] metricsSpec, - @JsonProperty("transformSpec") @Nullable CompactionTransformSpec transformSpec, - @JsonProperty("indexSpec") @Nullable IndexSpec indexSpec, - @JsonProperty("granularitySpec") @Nullable UserCompactionTaskGranularityConfig granularitySpec, - @JsonProperty("projections") @Nullable List projections - ) - { - this.partitionsSpec = partitionsSpec; - this.dimensionsSpec = dimensionsSpec; - this.metricsSpec = metricsSpec; - this.transformSpec = transformSpec; - this.indexSpec = indexSpec; - this.granularitySpec = granularitySpec; - this.projections = projections; - } - - @Nullable - @JsonProperty - public PartitionsSpec getPartitionsSpec() - { - return partitionsSpec; - } - - @Nullable - @JsonProperty - public DimensionsSpec getDimensionsSpec() - { - return dimensionsSpec; - } - - @Nullable - @JsonProperty - public AggregatorFactory[] getMetricsSpec() - { - return metricsSpec; - } - - @Nullable - @JsonProperty - public CompactionTransformSpec getTransformSpec() - { - return transformSpec; - } - - @Nullable - @JsonProperty - public IndexSpec getIndexSpec() - { - return indexSpec; - } - - @Nullable - @JsonProperty - public UserCompactionTaskGranularityConfig getGranularitySpec() - { - return granularitySpec; - } - - @JsonProperty - @Nullable - public List getProjections() - { - return projections; - } - - @Nullable - public Granularity getSegmentGranularity() - { - return granularitySpec == null ? null : granularitySpec.getSegmentGranularity(); - } - - @Override - public boolean equals(Object o) - { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - CompactionStateMatcher that = (CompactionStateMatcher) o; - return Objects.equals(partitionsSpec, that.partitionsSpec) && - Objects.equals(dimensionsSpec, that.dimensionsSpec) && - Objects.equals(transformSpec, that.transformSpec) && - Objects.equals(indexSpec, that.indexSpec) && - Objects.equals(granularitySpec, that.granularitySpec) && - Arrays.equals(metricsSpec, that.metricsSpec) && - Objects.equals(projections, that.projections); - } - - @Override - public int hashCode() - { - return Objects.hash( - partitionsSpec, - dimensionsSpec, - transformSpec, - indexSpec, - granularitySpec, - Arrays.hashCode(metricsSpec), - projections - ); - } - - @Override - public String toString() - { - return "CompactionState{" + - "partitionsSpec=" + partitionsSpec + - ", dimensionsSpec=" + dimensionsSpec + - ", transformSpec=" + transformSpec + - ", indexSpec=" + indexSpec + - ", granularitySpec=" + granularitySpec + - ", metricsSpec=" + Arrays.toString(metricsSpec) + - ", projections=" + projections + - '}'; - } -} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionSupervisorSpec.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionSupervisorSpec.java index 998d7e0c7e67..cf12be8ce90e 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionSupervisorSpec.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionSupervisorSpec.java @@ -98,11 +98,7 @@ public CompactionSupervisor createSupervisor() */ public CompactionJobTemplate getTemplate() { - if (spec instanceof CascadingCompactionTemplate) { - return (CascadingCompactionTemplate) spec; - } else { - return new CompactionConfigBasedJobTemplate(spec); - } + return new CompactionConfigBasedJobTemplate(spec); } @Override diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/InlineCompactionJobTemplate.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/InlineCompactionJobTemplate.java deleted file mode 100644 index 43e4c6f8c267..000000000000 --- a/indexing-service/src/main/java/org/apache/druid/indexing/compact/InlineCompactionJobTemplate.java +++ /dev/null @@ -1,98 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.druid.indexing.compact; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; -import org.apache.druid.indexing.input.DruidInputSource; -import org.apache.druid.java.util.common.granularity.Granularity; - -import javax.annotation.Nullable; -import java.util.List; -import java.util.Objects; - -/** - * Template to create compaction jobs using inline specifications. This template - * does not fetch any information from the Druid catalog while creating jobs. - */ -public class InlineCompactionJobTemplate implements CompactionJobTemplate -{ - public static final String TYPE = "compactInline"; - - private final CompactionStateMatcher targetState; - - @JsonCreator - public InlineCompactionJobTemplate( - @JsonProperty("targetState") CompactionStateMatcher targetState - ) - { - this.targetState = targetState; - } - - @JsonProperty - public CompactionStateMatcher getTargetState() - { - return targetState; - } - - @Nullable - @Override - public Granularity getSegmentGranularity() - { - return targetState.getSegmentGranularity(); - } - - @Override - public List createCompactionJobs( - DruidInputSource source, - CompactionJobParams jobParams - ) - { - final String dataSource = source.getDataSource(); - return CompactionConfigBasedJobTemplate - .create(dataSource, targetState) - .createCompactionJobs(source, jobParams); - } - - @Override - public boolean equals(Object object) - { - if (this == object) { - return true; - } - if (object == null || getClass() != object.getClass()) { - return false; - } - InlineCompactionJobTemplate that = (InlineCompactionJobTemplate) object; - return Objects.equals(this.targetState, that.targetState); - } - - @Override - public int hashCode() - { - return Objects.hash(targetState); - } - - @Override - public String getType() - { - return TYPE; - } -} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/MSQCompactionJobTemplate.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/MSQCompactionJobTemplate.java deleted file mode 100644 index ea7193e443be..000000000000 --- a/indexing-service/src/main/java/org/apache/druid/indexing/compact/MSQCompactionJobTemplate.java +++ /dev/null @@ -1,191 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.druid.indexing.compact; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; -import org.apache.druid.indexing.input.DruidInputSource; -import org.apache.druid.java.util.common.StringUtils; -import org.apache.druid.java.util.common.granularity.Granularity; -import org.apache.druid.query.http.ClientSqlQuery; -import org.apache.druid.server.compaction.CompactionCandidate; -import org.apache.druid.server.compaction.CompactionSlotManager; -import org.apache.druid.server.compaction.DataSourceCompactibleSegmentIterator; -import org.apache.druid.server.coordinator.duty.CompactSegments; -import org.joda.time.Interval; -import org.joda.time.format.DateTimeFormat; -import org.joda.time.format.DateTimeFormatter; - -import javax.annotation.Nullable; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Objects; - -/** - * Compaction template that creates MSQ SQL jobs using a templatized SQL with - * variables of the format {@code ${variableName}} for fields such as datasource - * name and start timestamp. - *

- * Compaction is triggered for an interval only if the current compaction state - * of the underlying segments DOES NOT match with the {@link #targetState}. - */ -public class MSQCompactionJobTemplate implements CompactionJobTemplate -{ - public static final String TYPE = "compactMsq"; - - public static final String VAR_DATASOURCE = "${dataSource}"; - public static final String VAR_START_TIMESTAMP = "${startTimestamp}"; - public static final String VAR_END_TIMESTAMP = "${endTimestamp}"; - - private static final DateTimeFormatter TIMESTAMP_FORMATTER = - DateTimeFormat.forPattern("yyyy-MM-dd HH:mm:ss.SSS"); - - private final ClientSqlQuery sqlTemplate; - private final CompactionStateMatcher targetState; - - @JsonCreator - public MSQCompactionJobTemplate( - @JsonProperty("sqlTemplate") ClientSqlQuery sqlTemplate, - @JsonProperty("targetState") CompactionStateMatcher targetState - ) - { - this.sqlTemplate = sqlTemplate; - this.targetState = targetState; - } - - @JsonProperty - public ClientSqlQuery getSqlTemplate() - { - return sqlTemplate; - } - - @JsonProperty - public CompactionStateMatcher getTargetState() - { - return targetState; - } - - @Nullable - @Override - public Granularity getSegmentGranularity() - { - return targetState.getSegmentGranularity(); - } - - @Override - public List createCompactionJobs( - DruidInputSource source, - CompactionJobParams jobParams - ) - { - final String dataSource = source.getDataSource(); - - // Identify the compactible candidate segments - final CompactionConfigBasedJobTemplate delegate = - CompactionConfigBasedJobTemplate.create(dataSource, targetState); - final DataSourceCompactibleSegmentIterator candidateIterator = - delegate.getCompactibleCandidates(source, jobParams); - - // Create MSQ jobs for each candidate by interpolating the template variables - final List jobs = new ArrayList<>(); - while (candidateIterator.hasNext()) { - final CompactionCandidate candidate = candidateIterator.next(); - jobs.add( - new CompactionJob( - createQueryForJob(dataSource, candidate.getCompactionInterval()), - candidate, - CompactionSlotManager.getMaxTaskSlotsForMSQCompactionTask(sqlTemplate.getContext()) - ) - ); - } - - return jobs; - } - - private ClientSqlQuery createQueryForJob(String dataSource, Interval compactionInterval) - { - final String formattedSql = formatSql( - sqlTemplate.getQuery(), - Map.of( - VAR_DATASOURCE, dataSource, - VAR_START_TIMESTAMP, compactionInterval.getStart().toString(TIMESTAMP_FORMATTER), - VAR_END_TIMESTAMP, compactionInterval.getEnd().toString(TIMESTAMP_FORMATTER) - ) - ); - - final Map context = new HashMap<>(); - if (sqlTemplate.getContext() != null) { - context.putAll(sqlTemplate.getContext()); - } - context.put(CompactSegments.STORE_COMPACTION_STATE_KEY, true); - context.put(CompactSegments.COMPACTION_INTERVAL_KEY, compactionInterval); - - return new ClientSqlQuery( - formattedSql, - sqlTemplate.getResultFormat(), - sqlTemplate.isHeader(), - sqlTemplate.isTypesHeader(), - sqlTemplate.isSqlTypesHeader(), - context, - sqlTemplate.getParameters() - ); - } - - /** - * Formats the given SQL by replacing the template variables. - */ - public static String formatSql(String sqlTemplate, Map templateVariables) - { - String sql = sqlTemplate; - for (Map.Entry variable : templateVariables.entrySet()) { - sql = StringUtils.replace(sql, variable.getKey(), variable.getValue()); - } - - return sql; - } - - @Override - public boolean equals(Object object) - { - if (this == object) { - return true; - } - if (object == null || getClass() != object.getClass()) { - return false; - } - MSQCompactionJobTemplate that = (MSQCompactionJobTemplate) object; - return Objects.equals(sqlTemplate, that.sqlTemplate) - && Objects.equals(targetState, that.targetState); - } - - @Override - public int hashCode() - { - return Objects.hash(sqlTemplate, targetState); - } - - @Override - public String getType() - { - return TYPE; - } -} diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/compact/OverlordCompactionSchedulerTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/compact/OverlordCompactionSchedulerTest.java index 1484a719db10..d154c32ce3eb 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/compact/OverlordCompactionSchedulerTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/compact/OverlordCompactionSchedulerTest.java @@ -56,7 +56,6 @@ import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.Intervals; -import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.metrics.StubServiceEmitter; @@ -64,7 +63,6 @@ import org.apache.druid.metadata.SegmentsMetadataManagerConfig; import org.apache.druid.query.http.ClientSqlQuery; import org.apache.druid.query.http.SqlTaskStatus; -import org.apache.druid.segment.TestDataSource; import org.apache.druid.segment.TestIndex; import org.apache.druid.server.compaction.CompactionSimulateResult; import org.apache.druid.server.compaction.CompactionStatistics; @@ -79,14 +77,11 @@ import org.apache.druid.server.coordinator.DataSourceCompactionConfig; import org.apache.druid.server.coordinator.DruidCompactionConfig; import org.apache.druid.server.coordinator.InlineSchemaDataSourceCompactionConfig; -import org.apache.druid.server.coordinator.UserCompactionTaskGranularityConfig; -import org.apache.druid.server.coordinator.duty.CompactSegments; import org.apache.druid.server.coordinator.simulate.BlockingExecutorService; import org.apache.druid.server.coordinator.simulate.WrappingScheduledExecutorService; import org.apache.druid.server.coordinator.stats.Stats; import org.apache.druid.timeline.DataSegment; import org.joda.time.DateTime; -import org.joda.time.Duration; import org.joda.time.Interval; import org.joda.time.Period; import org.junit.Assert; @@ -98,7 +93,6 @@ import java.util.Arrays; import java.util.Collections; -import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Set; @@ -123,7 +117,6 @@ public class OverlordCompactionSchedulerTest } private static final DateTime JAN_20 = DateTimes.of("2025-01-20"); - private static final DateTime MAR_11 = DateTimes.of("2025-03-11"); private AtomicReference compactionConfig; private CoordinatorOverlordServiceConfig coordinatorOverlordServiceConfig; @@ -142,8 +135,6 @@ public class OverlordCompactionSchedulerTest private String dataSource; private OverlordCompactionScheduler scheduler; - private Map submittedMsqTaskIds; - @Before public void setUp() { @@ -155,16 +146,10 @@ public void setUp() taskQueue = Mockito.mock(TaskQueue.class); - submittedMsqTaskIds = new HashMap<>(); brokerClient = Mockito.mock(BrokerClient.class); Mockito.when(brokerClient.submitSqlTask(ArgumentMatchers.any(ClientSqlQuery.class))).thenAnswer( arg -> { - final ClientSqlQuery query = arg.getArgument(0); - final Interval compactionInterval = - (Interval) query.getContext().get(CompactSegments.COMPACTION_INTERVAL_KEY); - final String taskId = IdUtils.getRandomId(); - submittedMsqTaskIds.put(compactionInterval, taskId); return Futures.immediateFuture(new SqlTaskStatus(taskId, TaskState.RUNNING, null)); } ); @@ -491,143 +476,6 @@ public void test_simulateRunWithConfigUpdate() scheduler.stopBeingLeader(); } - @Test - public void test_ingestHourGranularity_andCompactToDayAndMonth_withInlineTemplates() - { - final int numDays = (int) new Duration(MAR_11.getMillis() - JAN_20.getMillis()).getStandardDays(); - createSegments(24 * numDays, Granularities.HOUR, JAN_20); - verifyNumSegmentsWith(Granularities.HOUR, 24 * numDays); - - // Compact everything going back to Mar 10 to DAY granularity, rest to MONTH - final DateTime now = DateTimes.nowUtc(); - final Period dayRulePeriod = new Period(now.getMillis() - MAR_11.minusDays(1).minusMinutes(1).getMillis()); - CascadingCompactionTemplate cascadingTemplate = new CascadingCompactionTemplate( - dataSource, - List.of( - new CompactionRule(dayRulePeriod, new InlineCompactionJobTemplate(createMatcher(Granularities.DAY))), - new CompactionRule(Period.ZERO, new InlineCompactionJobTemplate(createMatcher(Granularities.MONTH))) - ) - ); - - startCompactionWithSpec(cascadingTemplate); - runCompactionTasks(12); - - verifyFullyCompacted(); - verifyNumSegmentsWith(Granularities.HOUR, 0); - verifyNumSegmentsWith(Granularities.DAY, 10); - verifyNumSegmentsWith(Granularities.MONTH, 2); - } - - @Test - public void test_ingestHourGranularity_andCompactToDayAndMonth_withCatalogTemplates() - { - final int numDays = (int) new Duration(MAR_11.getMillis() - JAN_20.getMillis()).getStandardDays(); - createSegments(24 * numDays, Granularities.HOUR, JAN_20); - verifyNumSegmentsWith(Granularities.HOUR, 24 * numDays); - - // Add compaction templates to catalog - final CompactionJobTemplate dayGranularityTemplate = - new InlineCompactionJobTemplate(createMatcher(Granularities.DAY)); - final CompactionJobTemplate monthGranularityTemplate = - new InlineCompactionJobTemplate(createMatcher(Granularities.MONTH)); - - // Compact everything going back to Mar 10 to DAY granularity, rest to MONTH - final DateTime now = DateTimes.nowUtc(); - final Period dayRulePeriod = new Period(now.getMillis() - MAR_11.minusDays(1).minusMinutes(1).getMillis()); - CascadingCompactionTemplate cascadingTemplate = new CascadingCompactionTemplate( - dataSource, - List.of( - new CompactionRule(dayRulePeriod, dayGranularityTemplate), - new CompactionRule(Period.ZERO, monthGranularityTemplate) - ) - ); - - startCompactionWithSpec(cascadingTemplate); - runCompactionTasks(12); - - verifyFullyCompacted(); - verifyNumSegmentsWith(Granularities.HOUR, 0); - verifyNumSegmentsWith(Granularities.DAY, 10); - verifyNumSegmentsWith(Granularities.MONTH, 2); - } - - @Test - public void test_ingestHourGranularity_andCompactToDayAndMonth_withCatalogMSQTemplates() - { - dataSource = TestDataSource.WIKI; - - final int numDays = (int) new Duration(MAR_11.getMillis() - JAN_20.getMillis()).getStandardDays(); - createSegments(24 * numDays, Granularities.HOUR, JAN_20); - verifyNumSegmentsWith(Granularities.HOUR, 24 * numDays); - - // Add compaction templates to catalog - final String sqlDayGranularity = - "REPLACE INTO ${dataSource}" - + " OVERWRITE WHERE __time >= TIMESTAMP '${startTimestamp}' AND __time < TIMESTAMP '${endTimestamp}'" - + " SELECT * FROM ${dataSource}" - + " WHERE __time BETWEEN '${startTimestamp}' AND '${endTimestamp}'" - + " PARTITIONED BY DAY"; - final CompactionJobTemplate dayGranularityTemplate = new MSQCompactionJobTemplate( - new ClientSqlQuery(sqlDayGranularity, null, false, false, false, null, null), - createMatcher(Granularities.DAY) - ); - final String sqlMonthGranularity = - "REPLACE INTO ${dataSource}" - + " OVERWRITE WHERE __time >= TIMESTAMP '${startTimestamp}' AND __time < TIMESTAMP '${endTimestamp}'" - + " SELECT * FROM ${dataSource}" - + " WHERE __time BETWEEN '${startTimestamp}' AND '${endTimestamp}'" - + " PARTITIONED BY MONTH"; - final CompactionJobTemplate monthGranularityTemplate = new MSQCompactionJobTemplate( - new ClientSqlQuery(sqlMonthGranularity, null, false, false, false, null, null), - createMatcher(Granularities.MONTH) - ); - - // Compact everything going back to Mar 10 to DAY granularity, rest to MONTH - final DateTime now = DateTimes.nowUtc(); - final Period dayRulePeriod = new Period(now.getMillis() - MAR_11.minusDays(1).minusMinutes(1).getMillis()); - CascadingCompactionTemplate cascadingTemplate = new CascadingCompactionTemplate( - dataSource, - List.of( - new CompactionRule(dayRulePeriod, dayGranularityTemplate), - new CompactionRule(Period.ZERO, monthGranularityTemplate) - ) - ); - - startCompactionWithSpec(cascadingTemplate); - runMSQCompactionJobs(12); - - verifyFullyCompacted(); - verifyNumSegmentsWith(Granularities.HOUR, 0); - verifyNumSegmentsWith(Granularities.DAY, 10); - verifyNumSegmentsWith(Granularities.MONTH, 2); - } - - private void verifyNumSegmentsWith(Granularity granularity, int numExpectedSegments) - { - long numMatchingSegments = segmentStorage - .retrieveAllUsedSegments(dataSource, Segments.ONLY_VISIBLE) - .stream() - .filter(segment -> granularity.isAligned(segment.getInterval())) - .count(); - - Assert.assertEquals( - StringUtils.format("Segment with granularity[%s]", granularity), - numExpectedSegments, - (int) numMatchingSegments - ); - } - - private void verifyFullyCompacted() - { - runScheduledJob(); - int numSegments = segmentStorage.retrieveAllUsedSegments(dataSource, Segments.ONLY_VISIBLE).size(); - - final AutoCompactionSnapshot snapshot = scheduler.getCompactionSnapshot(dataSource); - Assert.assertEquals(0, snapshot.getSegmentCountAwaitingCompaction()); - Assert.assertEquals(0, snapshot.getSegmentCountSkipped()); - Assert.assertEquals(numSegments, snapshot.getSegmentCountCompacted()); - } - private void createSegments(int numSegments, Granularity granularity, DateTime firstSegmentStart) { final List segments = CreateDataSegments @@ -638,14 +486,6 @@ private void createSegments(int numSegments, Granularity granularity, DateTime f segmentStorage.commitSegments(Set.copyOf(segments), null); } - private void startCompactionWithSpec(DataSourceCompactionConfig config) - { - scheduler.becomeLeader(); - final CompactionSupervisorSpec compactionSupervisor - = new CompactionSupervisorSpec(config, false, scheduler); - scheduler.startCompaction(config.getDataSource(), compactionSupervisor.createSupervisor()); - } - private void runCompactionTasks(int expectedCount) { runScheduledJob(); @@ -694,52 +534,6 @@ private void runCompactionTask(String taskId, Interval compactionInterval, Granu } } - private void runMSQCompactionJobs(int numExpectedJobs) - { - runScheduledJob(); - serviceEmitter.verifySum("compact/task/count", numExpectedJobs); - - ArgumentCaptor queryArgumentCaptor = ArgumentCaptor.forClass(ClientSqlQuery.class); - Mockito.verify(brokerClient, Mockito.times(numExpectedJobs)) - .submitSqlTask(queryArgumentCaptor.capture()); - - for (ClientSqlQuery job : queryArgumentCaptor.getAllValues()) { - final String query = job.getQuery(); - - final Granularity segmentGranularity; - if (query.contains("PARTITIONED BY DAY")) { - segmentGranularity = Granularities.DAY; - } else if (query.contains("PARTITIONED BY MONTH")) { - segmentGranularity = Granularities.MONTH; - } else { - segmentGranularity = Granularities.HOUR; - } - - final Interval compactionInterval = - (Interval) job.getContext().get(CompactSegments.COMPACTION_INTERVAL_KEY); - runCompactionTask( - submittedMsqTaskIds.get(compactionInterval), - compactionInterval, - segmentGranularity - ); - } - - segmentStorage.getManager().forceUpdateDataSourcesSnapshot(); - } - - private static CompactionStateMatcher createMatcher(Granularity segmentGranularity) - { - return new CompactionStateMatcher( - null, - null, - null, - null, - null, - new UserCompactionTaskGranularityConfig(segmentGranularity, null, null), - null - ); - } - private void disableScheduler() { compactionConfig.set(new ClusterCompactionConfig(null, null, null, false, null)); diff --git a/stuff.diff b/stuff.diff new file mode 100644 index 000000000000..a4b264dced42 --- /dev/null +++ b/stuff.diff @@ -0,0 +1,7184 @@ +diff --git a/benchmarks/src/test/java/org/apache/druid/server/coordinator/NewestSegmentFirstPolicyBenchmark.java b/benchmarks/src/test/java/org/apache/druid/server/coordinator/NewestSegmentFirstPolicyBenchmark.java +index a0a1da9369..c9c4599fad 100644 +--- a/benchmarks/src/test/java/org/apache/druid/server/coordinator/NewestSegmentFirstPolicyBenchmark.java ++++ b/benchmarks/src/test/java/org/apache/druid/server/coordinator/NewestSegmentFirstPolicyBenchmark.java +@@ -21,11 +21,9 @@ package org.apache.druid.server.coordinator; + + import com.google.common.collect.ImmutableList; + import org.apache.druid.client.DataSourcesSnapshot; +-import org.apache.druid.jackson.DefaultObjectMapper; + import org.apache.druid.java.util.common.DateTimes; + import org.apache.druid.server.compaction.CompactionCandidateSearchPolicy; + import org.apache.druid.server.compaction.CompactionSegmentIterator; +-import org.apache.druid.server.compaction.CompactionStatusTracker; + import org.apache.druid.server.compaction.NewestSegmentFirstPolicy; + import org.apache.druid.server.compaction.PriorityBasedCompactionSegmentIterator; + import org.apache.druid.timeline.DataSegment; +@@ -137,8 +135,7 @@ public class NewestSegmentFirstPolicyBenchmark + policy, + compactionConfigs, + dataSources, +- Collections.emptyMap(), +- new CompactionStatusTracker(new DefaultObjectMapper()) ++ Collections.emptyMap() + ); + for (int i = 0; i < numCompactionTaskSlots && iterator.hasNext(); i++) { + blackhole.consume(iterator.next()); +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 +new file mode 100644 +index 0000000000..35ba77274f +--- /dev/null ++++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionSupervisorTest.java +@@ -0,0 +1,345 @@ ++/* ++ * 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.testing.embedded.compact; ++ ++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.indexing.common.task.IndexTask; ++import org.apache.druid.indexing.compact.CascadingCompactionTemplate; ++import org.apache.druid.indexing.compact.CompactionJobTemplate; ++import org.apache.druid.indexing.compact.CompactionRule; ++import org.apache.druid.indexing.compact.CompactionStateMatcher; ++import org.apache.druid.indexing.compact.CompactionSupervisorSpec; ++import org.apache.druid.indexing.compact.InlineCompactionJobTemplate; ++import org.apache.druid.indexing.compact.MSQCompactionJobTemplate; ++import org.apache.druid.indexing.overlord.Segments; ++import org.apache.druid.java.util.common.DateTimes; ++import org.apache.druid.java.util.common.granularity.Granularities; ++import org.apache.druid.java.util.common.granularity.Granularity; ++import org.apache.druid.msq.guice.IndexerMemoryManagementModule; ++import org.apache.druid.msq.guice.MSQDurableStorageModule; ++import org.apache.druid.msq.guice.MSQIndexingModule; ++import org.apache.druid.msq.guice.MSQSqlModule; ++import org.apache.druid.msq.guice.SqlTaskModule; ++import org.apache.druid.query.DruidMetrics; ++import org.apache.druid.query.http.ClientSqlQuery; ++import org.apache.druid.rpc.UpdateResponse; ++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.UserCompactionTaskGranularityConfig; ++import org.apache.druid.testing.embedded.EmbeddedBroker; ++import org.apache.druid.testing.embedded.EmbeddedCoordinator; ++import org.apache.druid.testing.embedded.EmbeddedDruidCluster; ++import org.apache.druid.testing.embedded.EmbeddedHistorical; ++import org.apache.druid.testing.embedded.EmbeddedIndexer; ++import org.apache.druid.testing.embedded.EmbeddedOverlord; ++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.hamcrest.Matcher; ++import org.hamcrest.Matchers; ++import org.joda.time.DateTime; ++import org.joda.time.Period; ++import org.junit.jupiter.api.Assertions; ++import org.junit.jupiter.api.BeforeAll; ++import org.junit.jupiter.api.Test; ++ ++import java.util.List; ++import java.util.Map; ++ ++/** ++ * Embedded test that runs compaction supervisors of various types. ++ */ ++public class CompactionSupervisorTest extends EmbeddedClusterTestBase ++{ ++ private final EmbeddedBroker broker = new EmbeddedBroker(); ++ private final EmbeddedIndexer indexer = new EmbeddedIndexer() ++ .setServerMemory(2_000_000_000L) ++ .addProperty("druid.worker.capacity", "20"); ++ private final EmbeddedOverlord overlord = new EmbeddedOverlord() ++ .addProperty("druid.manager.segments.pollDuration", "PT1s") ++ .addProperty("druid.manager.segments.useIncrementalCache", "always"); ++ private final EmbeddedHistorical historical = new EmbeddedHistorical(); ++ private final EmbeddedCoordinator coordinator = new EmbeddedCoordinator() ++ .addProperty("druid.manager.segments.useIncrementalCache", "always"); ++ ++ @Override ++ public EmbeddedDruidCluster createCluster() ++ { ++ return EmbeddedDruidCluster.withEmbeddedDerbyAndZookeeper() ++ .useLatchableEmitter() ++ .useDefaultTimeoutForLatchableEmitter(600) ++ .addExtensions( ++ CatalogClientModule.class, ++ CatalogCoordinatorModule.class, ++ IndexerMemoryManagementModule.class, ++ MSQDurableStorageModule.class, ++ MSQIndexingModule.class, ++ MSQSqlModule.class, ++ SqlTaskModule.class ++ ) ++ .addServer(coordinator) ++ .addServer(overlord) ++ .addServer(indexer) ++ .addServer(historical) ++ .addServer(broker) ++ .addServer(new EmbeddedRouter()); ++ } ++ ++ @BeforeAll ++ public void enableCompactionSupervisors() ++ { ++ final UpdateResponse updateResponse = cluster.callApi().onLeaderOverlord( ++ o -> o.updateClusterCompactionConfig(new ClusterCompactionConfig(1.0, 100, null, true, null)) ++ ); ++ Assertions.assertTrue(updateResponse.isSuccess()); ++ } ++ ++ @Test ++ public void test_ingestDayGranularity_andCompactToMonthGranularity_withInlineConfig() ++ { ++ // Ingest data at DAY granularity and verify ++ runIngestionAtGranularity( ++ "DAY", ++ "2025-06-01T00:00:00.000Z,shirt,105" ++ + "\n2025-06-02T00:00:00.000Z,trousers,210" ++ + "\n2025-06-03T00:00:00.000Z,jeans,150" ++ ); ++ Assertions.assertEquals(3, getNumSegmentsWith(Granularities.DAY)); ++ ++ // Create a compaction config with MONTH granularity ++ InlineSchemaDataSourceCompactionConfig compactionConfig = ++ InlineSchemaDataSourceCompactionConfig ++ .builder() ++ .forDataSource(dataSource) ++ .withSkipOffsetFromLatest(Period.seconds(0)) ++ .withGranularitySpec( ++ new UserCompactionTaskGranularityConfig(Granularities.MONTH, null, null) ++ ) ++ .build(); ++ ++ runCompactionWithSpec(compactionConfig); ++ waitForAllCompactionTasksToFinish(); ++ ++ Assertions.assertEquals(0, getNumSegmentsWith(Granularities.DAY)); ++ Assertions.assertEquals(1, getNumSegmentsWith(Granularities.MONTH)); ++ } ++ ++ @Test ++ public void test_ingestHourGranularity_andCompactToDayAndMonth_withInlineTemplates() ++ { ++ // Create a cascading template with DAY and MONTH granularity ++ CascadingCompactionTemplate cascadingTemplate = new CascadingCompactionTemplate( ++ dataSource, ++ List.of( ++ new CompactionRule(Period.days(1), new InlineCompactionJobTemplate(createMatcher(Granularities.DAY))), ++ new CompactionRule(Period.days(50), new InlineCompactionJobTemplate(createMatcher(Granularities.MONTH))) ++ ) ++ ); ++ ++ ingestHourSegments(1000); ++ runCompactionWithSpec(cascadingTemplate); ++ waitForAllCompactionTasksToFinish(); ++ ++ Assertions.assertEquals(0, getNumSegmentsWith(Granularities.HOUR)); ++ Assertions.assertTrue(getNumSegmentsWith(Granularities.DAY) >= 1); ++ Assertions.assertTrue(getNumSegmentsWith(Granularities.MONTH) >= 1); ++ } ++ ++ @Test ++ public void test_ingestHourGranularity_andCompactToDayAndMonth_withMSQTemplates() ++ { ++ ingestHourSegments(1200); ++ ++ // Add compaction templates to catalog ++ final String sqlDayGranularity = ++ "REPLACE INTO ${dataSource}" ++ + " OVERWRITE WHERE __time >= TIMESTAMP '${startTimestamp}' AND __time < TIMESTAMP '${endTimestamp}'" ++ + " SELECT * FROM ${dataSource}" ++ + " WHERE __time BETWEEN '${startTimestamp}' AND '${endTimestamp}'" ++ + " PARTITIONED BY DAY"; ++ final CompactionJobTemplate dayGranularityTemplate = new MSQCompactionJobTemplate( ++ new ClientSqlQuery(sqlDayGranularity, null, false, false, false, null, null), ++ createMatcher(Granularities.DAY) ++ ); ++ final String sqlMonthGranularity = ++ "REPLACE INTO ${dataSource}" ++ + " OVERWRITE WHERE __time >= TIMESTAMP '${startTimestamp}' AND __time < TIMESTAMP '${endTimestamp}'" ++ + " SELECT * FROM ${dataSource}" ++ + " WHERE __time >= TIMESTAMP '${startTimestamp}' AND __time < TIMESTAMP '${endTimestamp}'" ++ + " PARTITIONED BY MONTH"; ++ final CompactionJobTemplate monthGranularityTemplate = new MSQCompactionJobTemplate( ++ new ClientSqlQuery(sqlMonthGranularity, null, false, false, false, null, null), ++ createMatcher(Granularities.MONTH) ++ ); ++ ++ // Create a cascading template with DAY and MONTH granularity ++ CascadingCompactionTemplate cascadingTemplate = new CascadingCompactionTemplate( ++ dataSource, ++ List.of( ++ new CompactionRule(Period.days(1), dayGranularityTemplate), ++ new CompactionRule(Period.days(50), monthGranularityTemplate) ++ ) ++ ); ++ ++ runCompactionWithSpec(cascadingTemplate); ++ waitForAllCompactionTasksToFinish(); ++ ++ Assertions.assertEquals(0, getNumSegmentsWith(Granularities.HOUR)); ++ Assertions.assertTrue(getNumSegmentsWith(Granularities.DAY) >= 1); ++ Assertions.assertTrue(getNumSegmentsWith(Granularities.MONTH) >= 1); ++ } ++ ++ @Test ++ public void test_ingestHourGranularity_andCompactToDayAndMonth_withMixedTemplates() ++ { ++ ingestHourSegments(1200); ++ ++ // Add compaction templates to catalog ++ final String sqlDayGranularity = ++ "REPLACE INTO ${dataSource}" ++ + " OVERWRITE WHERE __time >= TIMESTAMP '${startTimestamp}' AND __time < TIMESTAMP '${endTimestamp}'" ++ + " SELECT * FROM ${dataSource}" ++ + " WHERE __time BETWEEN '${startTimestamp}' AND '${endTimestamp}'" ++ + " PARTITIONED BY DAY"; ++ final MSQCompactionJobTemplate dayTemplate = new MSQCompactionJobTemplate( ++ new ClientSqlQuery(sqlDayGranularity, null, false, false, false, null, null), ++ createMatcher(Granularities.DAY) ++ ); ++ final CompactionJobTemplate weekTemplate = ++ new InlineCompactionJobTemplate(createMatcher(Granularities.WEEK)); ++ final InlineCompactionJobTemplate monthTemplate = ++ new InlineCompactionJobTemplate(createMatcher(Granularities.MONTH)); ++ ++ // Compact last 1 day to DAY, next 14 days to WEEK, then 1 more DAY, rest to MONTH ++ CascadingCompactionTemplate cascadingTemplate = new CascadingCompactionTemplate( ++ dataSource, ++ List.of( ++ new CompactionRule(Period.days(1), dayTemplate), ++ new CompactionRule(Period.days(15), weekTemplate), ++ new CompactionRule(Period.days(16), dayTemplate), ++ new CompactionRule(Period.ZERO, monthTemplate) ++ ) ++ ); ++ ++ runCompactionWithSpec(cascadingTemplate); ++ waitForAllCompactionTasksToFinish(); ++ ++ Assertions.assertEquals(0, getNumSegmentsWith(Granularities.HOUR)); ++ Assertions.assertTrue(getNumSegmentsWith(Granularities.DAY) >= 1); ++ Assertions.assertTrue(getNumSegmentsWith(Granularities.WEEK) >= 1); ++ Assertions.assertTrue(getNumSegmentsWith(Granularities.MONTH) >= 1); ++ } ++ ++ private void ingestHourSegments(int numSegments) ++ { ++ runIngestionAtGranularity( ++ "HOUR", ++ createHourlyInlineDataCsv(DateTimes.nowUtc(), numSegments) ++ ); ++ } ++ ++ private void runCompactionWithSpec(DataSourceCompactionConfig config) ++ { ++ final CompactionSupervisorSpec compactionSupervisor ++ = new CompactionSupervisorSpec(config, false, null); ++ cluster.callApi().postSupervisor(compactionSupervisor); ++ } ++ ++ private void waitForAllCompactionTasksToFinish() ++ { ++ // Wait for all intervals to be compacted ++ overlord.latchableEmitter().waitForEvent( ++ event -> event.hasMetricName("interval/waitCompact/count") ++ .hasDimension(DruidMetrics.DATASOURCE, dataSource) ++ .hasValueMatching(Matchers.equalTo(0L)) ++ ); ++ ++ // Wait for all submitted compaction jobs to finish ++ int numSubmittedTasks = overlord.latchableEmitter().getMetricValues( ++ "compact/task/count", ++ Map.of(DruidMetrics.DATASOURCE, dataSource) ++ ).stream().mapToInt(Number::intValue).sum(); ++ ++ final Matcher taskTypeMatcher = Matchers.anyOf( ++ Matchers.equalTo("query_controller"), ++ Matchers.equalTo("compact") ++ ); ++ overlord.latchableEmitter().waitForEventAggregate( ++ event -> event.hasMetricName("task/run/time") ++ .hasDimensionMatching(DruidMetrics.TASK_TYPE, taskTypeMatcher) ++ .hasDimension(DruidMetrics.DATASOURCE, dataSource), ++ agg -> agg.hasCountAtLeast(numSubmittedTasks) ++ ); ++ } ++ ++ private int getNumSegmentsWith(Granularity granularity) ++ { ++ return (int) overlord ++ .bindings() ++ .segmentsMetadataStorage() ++ .retrieveAllUsedSegments(dataSource, Segments.ONLY_VISIBLE) ++ .stream() ++ .filter(segment -> granularity.isAligned(segment.getInterval())) ++ .count(); ++ } ++ ++ private void runIngestionAtGranularity( ++ String granularity, ++ String inlineDataCsv ++ ) ++ { ++ final IndexTask task = MoreResources.Task.BASIC_INDEX ++ .get() ++ .segmentGranularity(granularity) ++ .inlineInputSourceWithData(inlineDataCsv) ++ .dataSource(dataSource) ++ .withId(IdUtils.getRandomId()); ++ cluster.callApi().runTask(task, overlord); ++ } ++ ++ private String createHourlyInlineDataCsv(DateTime latestRecordTimestamp, int numRecords) ++ { ++ final StringBuilder builder = new StringBuilder(); ++ for (int i = 0; i < numRecords; ++i) { ++ builder.append(latestRecordTimestamp.minusHours(i)) ++ .append(",").append("item_").append(IdUtils.getRandomId()) ++ .append(",").append(0) ++ .append("\n"); ++ } ++ ++ return builder.toString(); ++ } ++ ++ private static CompactionStateMatcher createMatcher(Granularity segmentGranularity) ++ { ++ return new CompactionStateMatcher( ++ null, ++ null, ++ null, ++ null, ++ null, ++ new UserCompactionTaskGranularityConfig(segmentGranularity, null, null), ++ null ++ ); ++ } ++} +diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaClusterMetricsTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaClusterMetricsTest.java +index 44c9be9a9c..11e25120aa 100644 +--- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaClusterMetricsTest.java ++++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaClusterMetricsTest.java +@@ -237,7 +237,7 @@ public class KafkaClusterMetricsTest extends EmbeddedClusterTestBase + event -> event.hasMetricName("task/run/time") + .hasDimension(DruidMetrics.TASK_TYPE, "compact") + .hasDimension(DruidMetrics.TASK_STATUS, "SUCCESS"), +- agg -> agg.hasCountAtLeast(2) ++ agg -> agg.hasCountAtLeast(10) + ); + + // Verify that some segments have been upgraded due to Concurrent Append and Replace +diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/server/HighAvailabilityTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/server/HighAvailabilityTest.java +index 303ad16afa..2a0f9d3cbe 100644 +--- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/server/HighAvailabilityTest.java ++++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/server/HighAvailabilityTest.java +@@ -43,6 +43,7 @@ import org.apache.druid.testing.embedded.EmbeddedOverlord; + import org.apache.druid.testing.embedded.EmbeddedRouter; + import org.apache.druid.testing.embedded.indexing.Resources; + import org.apache.druid.testing.embedded.junit5.EmbeddedClusterTestBase; ++import org.hamcrest.Matchers; + import org.jboss.netty.handler.codec.http.HttpMethod; + import org.jboss.netty.handler.codec.http.HttpResponseStatus; + import org.junit.jupiter.api.Assertions; +@@ -128,7 +129,7 @@ public class HighAvailabilityTest extends EmbeddedClusterTestBase + coordinator1.latchableEmitter().waitForEvent( + event -> event.hasMetricName("segment/metadataCache/used/count") + .hasDimension(DruidMetrics.DATASOURCE, dataSource) +- .hasValueAtLeast(10) ++ .hasValueMatching(Matchers.greaterThanOrEqualTo(10L)) + ); + + // Run sys queries, switch leaders, repeat +diff --git a/extensions-core/druid-catalog/pom.xml b/extensions-core/druid-catalog/pom.xml +index 3227fe6216..2c36306634 100644 +--- a/extensions-core/druid-catalog/pom.xml ++++ b/extensions-core/druid-catalog/pom.xml +@@ -193,6 +193,11 @@ + easymock + test + ++ ++ org.hamcrest ++ hamcrest-all ++ test ++ + + org.apache.curator + curator-test +diff --git a/extensions-core/druid-catalog/src/test/java/org/apache/druid/catalog/compact/CatalogCompactionTest.java b/extensions-core/druid-catalog/src/test/java/org/apache/druid/catalog/compact/CatalogCompactionTest.java +index 9f3106b884..ece6a9ef05 100644 +--- a/extensions-core/druid-catalog/src/test/java/org/apache/druid/catalog/compact/CatalogCompactionTest.java ++++ b/extensions-core/druid-catalog/src/test/java/org/apache/druid/catalog/compact/CatalogCompactionTest.java +@@ -32,6 +32,7 @@ import org.apache.druid.indexing.compact.CompactionSupervisorSpec; + import org.apache.druid.indexing.overlord.Segments; + import org.apache.druid.java.util.common.StringUtils; + import org.apache.druid.java.util.common.granularity.Granularities; ++import org.apache.druid.query.DruidMetrics; + import org.apache.druid.rpc.UpdateResponse; + import org.apache.druid.server.coordinator.CatalogDataSourceCompactionConfig; + import org.apache.druid.server.coordinator.ClusterCompactionConfig; +@@ -52,7 +53,8 @@ import java.util.List; + public class CatalogCompactionTest extends EmbeddedClusterTestBase + { + private final EmbeddedOverlord overlord = new EmbeddedOverlord() +- .addProperty("druid.catalog.client.maxSyncRetries", "0"); ++ .addProperty("druid.catalog.client.maxSyncRetries", "0") ++ .addProperty("druid.manager.segments.pollDuration", "PT1s"); + private final EmbeddedCoordinator coordinator = new EmbeddedCoordinator() + .addProperty("druid.manager.segments.useIncrementalCache", "always"); + private final EmbeddedBroker broker = new EmbeddedBroker() +@@ -119,7 +121,8 @@ public class CatalogCompactionTest extends EmbeddedClusterTestBase + // Wait for compaction to finish + overlord.latchableEmitter().waitForEvent( + event -> event.hasMetricName("task/run/time") +- .hasDimension("taskType", "compact") ++ .hasDimension(DruidMetrics.TASK_TYPE, "compact") ++ .hasDimension(DruidMetrics.DATASOURCE, dataSource) + ); + + // Verify that segments are now compacted to MONTH granularity +diff --git a/indexing-service/src/main/java/org/apache/druid/guice/SupervisorModule.java b/indexing-service/src/main/java/org/apache/druid/guice/SupervisorModule.java +index 73e8e06e89..5642fe1126 100644 +--- a/indexing-service/src/main/java/org/apache/druid/guice/SupervisorModule.java ++++ b/indexing-service/src/main/java/org/apache/druid/guice/SupervisorModule.java +@@ -25,7 +25,10 @@ import com.fasterxml.jackson.databind.jsontype.NamedType; + import com.fasterxml.jackson.databind.module.SimpleModule; + import com.google.common.collect.ImmutableList; + import com.google.inject.Binder; ++import org.apache.druid.indexing.compact.CascadingCompactionTemplate; + import org.apache.druid.indexing.compact.CompactionSupervisorSpec; ++import org.apache.druid.indexing.compact.InlineCompactionJobTemplate; ++import org.apache.druid.indexing.compact.MSQCompactionJobTemplate; + import org.apache.druid.indexing.overlord.supervisor.SupervisorStateManagerConfig; + import org.apache.druid.indexing.scheduledbatch.ScheduledBatchSupervisorSpec; + import org.apache.druid.initialization.DruidModule; +@@ -46,6 +49,9 @@ public class SupervisorModule implements DruidModule + return ImmutableList.of( + new SimpleModule(getClass().getSimpleName()) + .registerSubtypes( ++ new NamedType(InlineCompactionJobTemplate.class, InlineCompactionJobTemplate.TYPE), ++ new NamedType(MSQCompactionJobTemplate.class, MSQCompactionJobTemplate.TYPE), ++ new NamedType(CascadingCompactionTemplate.class, CascadingCompactionTemplate.TYPE), + new NamedType(CompactionSupervisorSpec.class, CompactionSupervisorSpec.TYPE), + new NamedType(ScheduledBatchSupervisorSpec.class, ScheduledBatchSupervisorSpec.TYPE) + ) +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 1c65264f6b..75b148d6e5 100644 +--- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java ++++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java +@@ -27,7 +27,6 @@ import com.fasterxml.jackson.annotation.JsonInclude.Include; + import com.fasterxml.jackson.annotation.JsonProperty; + import com.google.common.annotations.VisibleForTesting; + import com.google.common.base.Preconditions; +-import com.google.common.base.Verify; + import com.google.common.collect.BiMap; + import com.google.common.collect.HashBiMap; + import com.google.common.collect.ImmutableList; +@@ -95,8 +94,8 @@ import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager; + import org.apache.druid.segment.transform.CompactionTransformSpec; + import org.apache.druid.segment.transform.TransformSpec; + import org.apache.druid.segment.writeout.SegmentWriteOutMediumFactory; ++import org.apache.druid.server.compaction.CompactionSlotManager; + import org.apache.druid.server.coordinator.CompactionConfigValidationResult; +-import org.apache.druid.server.coordinator.duty.CompactSegments; + import org.apache.druid.server.lookup.cache.LookupLoadingSpec; + import org.apache.druid.server.security.ResourceAction; + import org.apache.druid.timeline.DataSegment; +@@ -133,7 +132,7 @@ import java.util.stream.IntStream; + */ + public class CompactionTask extends AbstractBatchIndexTask implements PendingSegmentAllocatingTask + { +- public static final String TYPE = "compact"; ++ public static final String TYPE = CompactionSlotManager.COMPACTION_TASK_TYPE; + private static final Logger log = new Logger(CompactionTask.class); + + /** +@@ -148,10 +147,6 @@ public class CompactionTask extends AbstractBatchIndexTask implements PendingSeg + */ + public static final String CTX_KEY_APPENDERATOR_TRACKING_TASK_ID = "appenderatorTrackingTaskId"; + +- static { +- Verify.verify(TYPE.equals(CompactSegments.COMPACTION_TASK_TYPE)); +- } +- + private final CompactionIOConfig ioConfig; + @Nullable + private final DimensionsSpec dimensionsSpec; +diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/Tasks.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/Tasks.java +index 90fb67116b..b45eb45dc0 100644 +--- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/Tasks.java ++++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/Tasks.java +@@ -47,7 +47,6 @@ public class Tasks + public static final long DEFAULT_SUB_TASK_TIMEOUT_MILLIS = 0; + public static final boolean DEFAULT_FORCE_TIME_CHUNK_LOCK = true; + public static final boolean DEFAULT_STORE_COMPACTION_STATE = false; +- public static final boolean DEFAULT_USE_MAX_MEMORY_ESTIMATES = false; + public static final TaskLockType DEFAULT_TASK_LOCK_TYPE = TaskLockType.EXCLUSIVE; + public static final boolean DEFAULT_USE_CONCURRENT_LOCKS = false; + +diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CascadingCompactionTemplate.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CascadingCompactionTemplate.java +new file mode 100644 +index 0000000000..89d3897436 +--- /dev/null ++++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CascadingCompactionTemplate.java +@@ -0,0 +1,246 @@ ++/* ++ * 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.compact; ++ ++import com.fasterxml.jackson.annotation.JsonCreator; ++import com.fasterxml.jackson.annotation.JsonProperty; ++import org.apache.druid.data.input.impl.AggregateProjectionSpec; ++import org.apache.druid.error.InvalidInput; ++import org.apache.druid.indexer.CompactionEngine; ++import org.apache.druid.indexing.input.DruidInputSource; ++import org.apache.druid.java.util.common.DateTimes; ++import org.apache.druid.java.util.common.granularity.Granularity; ++import org.apache.druid.query.aggregation.AggregatorFactory; ++import org.apache.druid.segment.transform.CompactionTransformSpec; ++import org.apache.druid.server.coordinator.DataSourceCompactionConfig; ++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; ++import org.joda.time.DateTime; ++import org.joda.time.Interval; ++import org.joda.time.Period; ++ ++import javax.annotation.Nullable; ++import java.util.ArrayList; ++import java.util.List; ++import java.util.Map; ++import java.util.Objects; ++ ++/** ++ * Template to perform period-based cascading compaction. Contains a list of ++ * {@link CompactionRule} which divide the segment timeline into compactible ++ * intervals. Each rule specifies a period relative to the current time which is ++ * used to determine its applicable interval: ++ *
    ++ *
  • Rule 1: range = [now - p1, +inf)
  • ++ *
  • Rule 2: range = [now - p2, now - p1)
  • ++ *
  • ...
  • ++ *
  • Rule n: range = (-inf, now - p(n - 1))
  • ++ *
++ * ++ * If two adjacent rules explicitly specify a segment granularity, the boundary ++ * between them may be {@linkplain CompactionRule#computeStartTime adjusted} ++ * to ensure that there are no uncompacted gaps in the timeline. ++ *

++ * This template never needs to be deserialized as a {@code BatchIndexingJobTemplate}, ++ * only as a {@link DataSourceCompactionConfig} in {@link CompactionSupervisorSpec}. ++ */ ++public class CascadingCompactionTemplate implements CompactionJobTemplate, DataSourceCompactionConfig ++{ ++ public static final String TYPE = "compactCascade"; ++ ++ private final String dataSource; ++ private final List rules; ++ ++ @JsonCreator ++ public CascadingCompactionTemplate( ++ @JsonProperty("dataSource") String dataSource, ++ @JsonProperty("rules") List rules ++ ) ++ { ++ this.rules = rules; ++ this.dataSource = Objects.requireNonNull(dataSource, "'dataSource' cannot be null"); ++ ++ InvalidInput.conditionalException(rules != null && !rules.isEmpty(), "'rules' cannot be empty"); ++ } ++ ++ @Override ++ @JsonProperty ++ public String getDataSource() ++ { ++ return dataSource; ++ } ++ ++ @JsonProperty ++ public List getRules() ++ { ++ return rules; ++ } ++ ++ @Override ++ public List createCompactionJobs( ++ DruidInputSource source, ++ CompactionJobParams jobParams ++ ) ++ { ++ final List allJobs = new ArrayList<>(); ++ ++ // Include future dates in the first rule ++ final DateTime currentTime = jobParams.getScheduleStartTime(); ++ DateTime previousRuleStartTime = DateTimes.MAX; ++ for (int i = 0; i < rules.size() - 1; ++i) { ++ final CompactionRule rule = rules.get(i); ++ final DateTime ruleStartTime = rule.computeStartTime(currentTime, rules.get(i + 1)); ++ final Interval ruleInterval = new Interval(ruleStartTime, previousRuleStartTime); ++ ++ allJobs.addAll( ++ createJobsForSearchInterval(rule.getTemplate(), ruleInterval, source, jobParams) ++ ); ++ ++ previousRuleStartTime = ruleStartTime; ++ } ++ ++ // Include past dates in the last rule ++ final CompactionRule lastRule = rules.get(rules.size() - 1); ++ final Interval lastRuleInterval = new Interval(DateTimes.MIN, previousRuleStartTime); ++ allJobs.addAll( ++ createJobsForSearchInterval(lastRule.getTemplate(), lastRuleInterval, source, jobParams) ++ ); ++ ++ return allJobs; ++ } ++ ++ private List createJobsForSearchInterval( ++ CompactionJobTemplate template, ++ Interval searchInterval, ++ DruidInputSource inputSource, ++ CompactionJobParams jobParams ++ ) ++ { ++ return template.createCompactionJobs( ++ inputSource.withInterval(searchInterval), ++ jobParams ++ ); ++ } ++ ++ @Override ++ public String getType() ++ { ++ return TYPE; ++ } ++ ++ // Legacy fields from DataSourceCompactionConfig that are not used by this template ++ ++ @Nullable ++ @Override ++ public CompactionEngine getEngine() ++ { ++ return null; ++ } ++ ++ @Override ++ public int getTaskPriority() ++ { ++ return 0; ++ } ++ ++ @Override ++ public long getInputSegmentSizeBytes() ++ { ++ return 0; ++ } ++ ++ @Nullable ++ @Override ++ public Integer getMaxRowsPerSegment() ++ { ++ return 0; ++ } ++ ++ @Override ++ public Period getSkipOffsetFromLatest() ++ { ++ return null; ++ } ++ ++ @Nullable ++ @Override ++ public UserCompactionTaskQueryTuningConfig getTuningConfig() ++ { ++ return null; ++ } ++ ++ @Nullable ++ @Override ++ public UserCompactionTaskIOConfig getIoConfig() ++ { ++ return null; ++ } ++ ++ @Nullable ++ @Override ++ public Map getTaskContext() ++ { ++ return Map.of(); ++ } ++ ++ @Nullable ++ @Override ++ public Granularity getSegmentGranularity() ++ { ++ return null; ++ } ++ ++ @Nullable ++ @Override ++ public UserCompactionTaskGranularityConfig getGranularitySpec() ++ { ++ return null; ++ } ++ ++ @Nullable ++ @Override ++ public List getProjections() ++ { ++ return List.of(); ++ } ++ ++ @Nullable ++ @Override ++ public CompactionTransformSpec getTransformSpec() ++ { ++ return null; ++ } ++ ++ @Nullable ++ @Override ++ public UserCompactionTaskDimensionsConfig getDimensionsSpec() ++ { ++ return null; ++ } ++ ++ @Nullable ++ @Override ++ public AggregatorFactory[] getMetricsSpec() ++ { ++ return new AggregatorFactory[0]; ++ } ++} +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 +new file mode 100644 +index 0000000000..c8563290a6 +--- /dev/null ++++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionConfigBasedJobTemplate.java +@@ -0,0 +1,156 @@ ++/* ++ * 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.compact; ++ ++import org.apache.druid.client.indexing.ClientCompactionTaskQuery; ++import org.apache.druid.error.DruidException; ++import org.apache.druid.error.InvalidInput; ++import org.apache.druid.indexing.input.DruidInputSource; ++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.CompactionSlotManager; ++import org.apache.druid.server.compaction.DataSourceCompactibleSegmentIterator; ++import org.apache.druid.server.compaction.NewestSegmentFirstPolicy; ++import org.apache.druid.server.coordinator.DataSourceCompactionConfig; ++import org.apache.druid.server.coordinator.InlineSchemaDataSourceCompactionConfig; ++import org.apache.druid.server.coordinator.duty.CompactSegments; ++import org.apache.druid.timeline.SegmentTimeline; ++import org.joda.time.Interval; ++import org.joda.time.Period; ++ ++import javax.annotation.Nullable; ++import java.util.ArrayList; ++import java.util.List; ++import java.util.Objects; ++ ++/** ++ * This template never needs to be deserialized as a {@code BatchIndexingJobTemplate}. ++ * It is just a delegating template that uses a {@link DataSourceCompactionConfig} ++ * to create compaction jobs. ++ */ ++public class CompactionConfigBasedJobTemplate implements CompactionJobTemplate ++{ ++ private final DataSourceCompactionConfig config; ++ ++ public CompactionConfigBasedJobTemplate(DataSourceCompactionConfig config) ++ { ++ this.config = config; ++ } ++ ++ public static CompactionConfigBasedJobTemplate create(String dataSource, CompactionStateMatcher stateMatcher) ++ { ++ return new CompactionConfigBasedJobTemplate( ++ InlineSchemaDataSourceCompactionConfig ++ .builder() ++ .forDataSource(dataSource) ++ .withSkipOffsetFromLatest(Period.ZERO) ++ .withTransformSpec(stateMatcher.getTransformSpec()) ++ .withProjections(stateMatcher.getProjections()) ++ .withMetricsSpec(stateMatcher.getMetricsSpec()) ++ .withGranularitySpec(stateMatcher.getGranularitySpec()) ++ .build() ++ ); ++ } ++ ++ @Nullable ++ @Override ++ public Granularity getSegmentGranularity() ++ { ++ return config.getSegmentGranularity(); ++ } ++ ++ @Override ++ public List createCompactionJobs( ++ DruidInputSource source, ++ CompactionJobParams params ++ ) ++ { ++ final DataSourceCompactibleSegmentIterator segmentIterator = getCompactibleCandidates(source, params); ++ ++ final List jobs = new ArrayList<>(); ++ ++ // Create a job for each CompactionCandidate ++ while (segmentIterator.hasNext()) { ++ final CompactionCandidate candidate = segmentIterator.next(); ++ ++ ClientCompactionTaskQuery taskPayload ++ = CompactSegments.createCompactionTask(candidate, config, params.getClusterCompactionConfig().getEngine()); ++ jobs.add( ++ new CompactionJob( ++ taskPayload, ++ candidate, ++ CompactionSlotManager.getMaxTaskSlotsForNativeCompactionTask(taskPayload.getTuningConfig()) ++ ) ++ ); ++ } ++ ++ return jobs; ++ } ++ ++ @Override ++ public String getType() ++ { ++ throw DruidException.defensive( ++ "This template cannot be serialized. It is an adapter used to create jobs" ++ + " using a legacy DataSourceCompactionConfig. Do not use this template" ++ + " in a supervisor spec directly. Use types [compactCatalog], [compactMsq]" ++ + " or [compactInline] instead." ++ ); ++ } ++ ++ /** ++ * Creates an iterator over the compactible candidate segments for the given ++ * params. Adds stats for segments that are already compacted to the ++ * {@link CompactionJobParams#getSnapshotBuilder()}. ++ */ ++ DataSourceCompactibleSegmentIterator getCompactibleCandidates( ++ DruidInputSource source, ++ CompactionJobParams params ++ ) ++ { ++ validateInput(source); ++ ++ final Interval searchInterval = Objects.requireNonNull(source.getInterval()); ++ ++ final SegmentTimeline timeline = params.getTimeline(config.getDataSource()); ++ final DataSourceCompactibleSegmentIterator iterator = new DataSourceCompactibleSegmentIterator( ++ config, ++ timeline, ++ Intervals.complementOf(searchInterval), ++ new NewestSegmentFirstPolicy(null) ++ ); ++ ++ // Collect stats for segments that are already compacted ++ iterator.getCompactedSegments().forEach(entry -> params.getSnapshotBuilder().addToComplete(entry)); ++ ++ return iterator; ++ } ++ ++ private void validateInput(DruidInputSource druidInputSource) ++ { ++ if (!druidInputSource.getDataSource().equals(config.getDataSource())) { ++ throw InvalidInput.exception( ++ "Datasource[%s] in compaction config does not match datasource[%s] in input source", ++ config.getDataSource(), druidInputSource.getDataSource() ++ ); ++ } ++ } ++} +diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionJob.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionJob.java +new file mode 100644 +index 0000000000..7a7e7fdc1e +--- /dev/null ++++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionJob.java +@@ -0,0 +1,81 @@ ++/* ++ * Licensed to the Apache Software Foundation (ASF) under one ++ * or more contributor license agreements. See the NOTICE file ++ * distributed with this work for additional information ++ * regarding copyright ownership. The ASF licenses this file ++ * to you under the Apache License, Version 2.0 (the ++ * "License"); you may not use this file except in compliance ++ * with the License. You may obtain a copy of the License at ++ * ++ * http://www.apache.org/licenses/LICENSE-2.0 ++ * ++ * Unless required by applicable law or agreed to in writing, ++ * software distributed under the License is distributed on an ++ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY ++ * KIND, either express or implied. See the License for the ++ * specific language governing permissions and limitations ++ * under the License. ++ */ ++ ++package org.apache.druid.indexing.compact; ++ ++import org.apache.druid.client.indexing.ClientCompactionTaskQuery; ++import org.apache.druid.indexing.template.BatchIndexingJob; ++import org.apache.druid.query.http.ClientSqlQuery; ++import org.apache.druid.server.compaction.CompactionCandidate; ++ ++/** ++ * {@link BatchIndexingJob} to compact an interval of a datasource. ++ */ ++public class CompactionJob extends BatchIndexingJob ++{ ++ private final CompactionCandidate candidate; ++ private final int maxRequiredTaskSlots; ++ ++ public CompactionJob( ++ ClientCompactionTaskQuery task, ++ CompactionCandidate candidate, ++ int maxRequiredTaskSlots ++ ) ++ { ++ super(task, null); ++ this.candidate = candidate; ++ this.maxRequiredTaskSlots = maxRequiredTaskSlots; ++ } ++ ++ public CompactionJob( ++ ClientSqlQuery msqQuery, ++ CompactionCandidate candidate, ++ int maxRequiredTaskSlots ++ ) ++ { ++ super(null, msqQuery); ++ this.candidate = candidate; ++ this.maxRequiredTaskSlots = maxRequiredTaskSlots; ++ } ++ ++ public String getDataSource() ++ { ++ return candidate.getDataSource(); ++ } ++ ++ public CompactionCandidate getCandidate() ++ { ++ return candidate; ++ } ++ ++ public int getMaxRequiredTaskSlots() ++ { ++ return maxRequiredTaskSlots; ++ } ++ ++ @Override ++ public String toString() ++ { ++ return "CompactionJob{" + ++ super.toString() + ++ ", candidate=" + candidate + ++ ", maxRequiredTaskSlots=" + maxRequiredTaskSlots + ++ '}'; ++ } ++} +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 +new file mode 100644 +index 0000000000..0113f1b78b +--- /dev/null ++++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionJobParams.java +@@ -0,0 +1,96 @@ ++/* ++ * 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.compact; ++ ++import org.apache.druid.server.compaction.CompactionSnapshotBuilder; ++import org.apache.druid.server.coordinator.ClusterCompactionConfig; ++import org.apache.druid.timeline.SegmentTimeline; ++import org.joda.time.DateTime; ++ ++/** ++ * Parameters used while creating a {@link CompactionJob} using a {@link CompactionJobTemplate}. ++ */ ++public class CompactionJobParams ++{ ++ private final DateTime scheduleStartTime; ++ private final TimelineProvider timelineProvider; ++ private final ClusterCompactionConfig clusterCompactionConfig; ++ private final CompactionSnapshotBuilder snapshotBuilder; ++ ++ public CompactionJobParams( ++ DateTime scheduleStartTime, ++ ClusterCompactionConfig clusterCompactionConfig, ++ TimelineProvider timelineProvider, ++ CompactionSnapshotBuilder snapshotBuilder ++ ) ++ { ++ this.scheduleStartTime = scheduleStartTime; ++ this.clusterCompactionConfig = clusterCompactionConfig; ++ this.timelineProvider = timelineProvider; ++ this.snapshotBuilder = snapshotBuilder; ++ } ++ ++ /** ++ * Timestamp denoting the start of the current run of the scheduler which has ++ * triggered creation of jobs using these {@link CompactionJobParams}. ++ */ ++ public DateTime getScheduleStartTime() ++ { ++ return scheduleStartTime; ++ } ++ ++ /** ++ * Cluster-level compaction config containing details such as the engine, ++ * compaction search policy, etc. to use while creating {@link CompactionJob}. ++ */ ++ public ClusterCompactionConfig getClusterCompactionConfig() ++ { ++ return clusterCompactionConfig; ++ } ++ ++ /** ++ * Provides the full {@link SegmentTimeline} of used segments for the given ++ * datasource. This timeline is used to identify eligible intervals for which ++ * compaction jobs should be created. ++ */ ++ public SegmentTimeline getTimeline(String dataSource) ++ { ++ return timelineProvider.getTimelineForDataSource(dataSource); ++ } ++ ++ /** ++ * 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 ++ * candidate intervals are identified as compacted, skipped or pending, they ++ * should be updated in this snapshot builder by invoking ++ * {@link CompactionSnapshotBuilder#addToComplete}, {@link CompactionSnapshotBuilder#addToSkipped} ++ * and {@link CompactionSnapshotBuilder#addToPending} respectively. ++ */ ++ public CompactionSnapshotBuilder getSnapshotBuilder() ++ { ++ return snapshotBuilder; ++ } ++ ++ @FunctionalInterface ++ public interface TimelineProvider ++ { ++ SegmentTimeline getTimelineForDataSource(String dataSource); ++ } ++} +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 +new file mode 100644 +index 0000000000..77886af1a0 +--- /dev/null ++++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionJobQueue.java +@@ -0,0 +1,373 @@ ++/* ++ * 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.compact; ++ ++import com.fasterxml.jackson.databind.ObjectMapper; ++import org.apache.druid.client.DataSourcesSnapshot; ++import org.apache.druid.client.broker.BrokerClient; ++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.indexer.TaskState; ++import org.apache.druid.indexer.TaskStatus; ++import org.apache.druid.indexing.common.actions.TaskActionClientFactory; ++import org.apache.druid.indexing.common.task.Task; ++import org.apache.druid.indexing.input.DruidInputSource; ++import org.apache.druid.indexing.overlord.GlobalTaskLockbox; ++import org.apache.druid.indexing.template.BatchIndexingJob; ++import org.apache.druid.java.util.common.DateTimes; ++import org.apache.druid.java.util.common.Stopwatch; ++import org.apache.druid.java.util.common.logger.Logger; ++import org.apache.druid.rpc.indexing.OverlordClient; ++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.CompactionSnapshotBuilder; ++import org.apache.druid.server.compaction.CompactionStatus; ++import org.apache.druid.server.compaction.CompactionStatusTracker; ++import org.apache.druid.server.coordinator.AutoCompactionSnapshot; ++import org.apache.druid.server.coordinator.ClusterCompactionConfig; ++import org.apache.druid.server.coordinator.CompactionConfigValidationResult; ++import org.apache.druid.server.coordinator.stats.CoordinatorRunStats; ++import org.apache.druid.server.coordinator.stats.Dimension; ++import org.apache.druid.server.coordinator.stats.RowKey; ++import org.apache.druid.server.coordinator.stats.Stats; ++ ++import javax.annotation.Nullable; ++import javax.annotation.concurrent.NotThreadSafe; ++import java.util.ArrayList; ++import java.util.HashMap; ++import java.util.HashSet; ++import java.util.List; ++import java.util.Map; ++import java.util.PriorityQueue; ++import java.util.Set; ++import java.util.stream.Collectors; ++ ++/** ++ * Iterates over all eligible compaction jobs in order of their priority. ++ * A fresh instance of this class must be used in every run of the ++ * {@link CompactionScheduler}. ++ *

++ * Unlike the Coordinator duty {@code CompactSegments}, the job queue currently ++ * does not cancel running compaction tasks even if their target segment ++ * granularity has changed. This has not been done here for simplicity since the ++ * {@code CompactionJobQueue} uses compaction templates and may have a different ++ * target segment granulariy for different intervals of the same datasource. ++ * The cancellation of invalid tasks has been left as a future enhancement. ++ */ ++@NotThreadSafe ++public class CompactionJobQueue ++{ ++ private static final Logger log = new Logger(CompactionJobQueue.class); ++ ++ private final CompactionJobParams jobParams; ++ private final CompactionCandidateSearchPolicy searchPolicy; ++ private final ClusterCompactionConfig clusterCompactionConfig; ++ ++ private final ObjectMapper objectMapper; ++ private final CompactionStatusTracker statusTracker; ++ private final TaskActionClientFactory taskActionClientFactory; ++ private final OverlordClient overlordClient; ++ private final GlobalTaskLockbox taskLockbox; ++ private final BrokerClient brokerClient; ++ ++ private final CompactionSnapshotBuilder snapshotBuilder; ++ private final PriorityQueue queue; ++ private final CoordinatorRunStats runStats; ++ ++ private final Set activeSupervisors; ++ private final Map submittedTaskIdToJob; ++ ++ public CompactionJobQueue( ++ DataSourcesSnapshot dataSourcesSnapshot, ++ ClusterCompactionConfig clusterCompactionConfig, ++ CompactionStatusTracker statusTracker, ++ TaskActionClientFactory taskActionClientFactory, ++ GlobalTaskLockbox taskLockbox, ++ OverlordClient overlordClient, ++ BrokerClient brokerClient, ++ ObjectMapper objectMapper ++ ) ++ { ++ this.runStats = new CoordinatorRunStats(); ++ this.snapshotBuilder = new CompactionSnapshotBuilder(runStats); ++ this.clusterCompactionConfig = clusterCompactionConfig; ++ this.searchPolicy = clusterCompactionConfig.getCompactionPolicy(); ++ this.queue = new PriorityQueue<>( ++ (o1, o2) -> searchPolicy.compareCandidates(o1.getCandidate(), o2.getCandidate()) ++ ); ++ this.submittedTaskIdToJob = new HashMap<>(); ++ this.activeSupervisors = new HashSet<>(); ++ this.jobParams = new CompactionJobParams( ++ DateTimes.nowUtc(), ++ clusterCompactionConfig, ++ dataSourcesSnapshot.getUsedSegmentsTimelinesPerDataSource()::get, ++ snapshotBuilder ++ ); ++ ++ this.taskActionClientFactory = taskActionClientFactory; ++ this.overlordClient = overlordClient; ++ this.brokerClient = brokerClient; ++ this.statusTracker = statusTracker; ++ this.objectMapper = objectMapper; ++ this.taskLockbox = taskLockbox; ++ } ++ ++ /** ++ * Creates jobs for the given {@link CompactionSupervisor} and adds them to ++ * the job queue. ++ *

++ * This method is idempotent. If jobs for the given supervisor already exist ++ * in the queue, the method does nothing. ++ */ ++ public void createAndEnqueueJobs( ++ CompactionSupervisor supervisor, ++ DruidInputSource source ++ ) ++ { ++ final Stopwatch jobCreationTime = Stopwatch.createStarted(); ++ final String supervisorId = supervisor.getSpec().getId(); ++ try { ++ if (supervisor.shouldCreateJobs() && !activeSupervisors.contains(supervisorId)) { ++ // Queue fresh jobs ++ final List jobs = supervisor.createJobs(source, jobParams); ++ jobs.forEach(job -> snapshotBuilder.addToPending(job.getCandidate())); ++ ++ queue.addAll(jobs); ++ activeSupervisors.add(supervisorId); ++ ++ runStats.add( ++ Stats.Compaction.CREATED_JOBS, ++ RowKey.of(Dimension.DATASOURCE, source.getDataSource()), ++ jobs.size() ++ ); ++ } else { ++ log.debug("Skipping job creation for supervisor[%s]", supervisorId); ++ } ++ } ++ catch (Exception e) { ++ log.error(e, "Error while creating jobs for supervisor[%s]", supervisorId); ++ } ++ finally { ++ runStats.add( ++ Stats.Compaction.JOB_CREATION_TIME, ++ RowKey.of(Dimension.DATASOURCE, source.getDataSource()), ++ jobCreationTime.millisElapsed() ++ ); ++ } ++ } ++ ++ /** ++ * Removes all existing jobs for the given datasource from the queue. ++ */ ++ public void removeJobs(String dataSource) ++ { ++ final List jobsToRemove = queue ++ .stream() ++ .filter(job -> job.getDataSource().equals(dataSource)) ++ .collect(Collectors.toList()); ++ ++ queue.removeAll(jobsToRemove); ++ log.info("Removed [%d] jobs for datasource[%s] from queue.", jobsToRemove.size(), dataSource); ++ } ++ ++ /** ++ * Submits jobs which are ready to either the Overlord or a Broker (if it is ++ * an MSQ SQL job). ++ */ ++ public void runReadyJobs() ++ { ++ final CompactionSlotManager slotManager = new CompactionSlotManager( ++ overlordClient, ++ statusTracker, ++ clusterCompactionConfig ++ ); ++ slotManager.reserveTaskSlotsForRunningCompactionTasks(); ++ ++ final List pendingJobs = new ArrayList<>(); ++ while (!queue.isEmpty()) { ++ final CompactionJob job = queue.poll(); ++ if (startJobIfPendingAndReady(job, searchPolicy, pendingJobs, slotManager)) { ++ runStats.add(Stats.Compaction.SUBMITTED_TASKS, RowKey.of(Dimension.DATASOURCE, job.getDataSource()), 1); ++ } ++ } ++ ++ // Requeue pending jobs so that they can be launched when slots become available ++ queue.addAll(pendingJobs); ++ } ++ ++ /** ++ * Notifies completion of the given so that the compaction snapshots may be ++ * updated. ++ */ ++ public void onTaskFinished(String taskId, TaskStatus taskStatus) ++ { ++ final CompactionJob job = submittedTaskIdToJob.remove(taskId); ++ if (job == null || !taskStatus.getStatusCode().isComplete()) { ++ // This is an unknown task ID ++ return; ++ } ++ ++ if (taskStatus.getStatusCode() == TaskState.FAILED) { ++ // Add this job back to the queue ++ queue.add(job); ++ } else { ++ snapshotBuilder.moveFromPendingToCompleted(job.getCandidate()); ++ } ++ } ++ ++ public CoordinatorRunStats getRunStats() ++ { ++ return runStats; ++ } ++ ++ /** ++ * Builds compaction snapshots for all the datasources being tracked by this ++ * queue. ++ */ ++ public Map getSnapshots() ++ { ++ return snapshotBuilder.build(); ++ } ++ ++ /** ++ * Starts a job if it is ready and is not already in progress. ++ * ++ * @return true if the job was submitted successfully for execution ++ */ ++ private boolean startJobIfPendingAndReady( ++ CompactionJob job, ++ CompactionCandidateSearchPolicy policy, ++ List pendingJobs, ++ CompactionSlotManager slotManager ++ ) ++ { ++ // Check if the job is a valid compaction job ++ final CompactionCandidate candidate = job.getCandidate(); ++ final CompactionConfigValidationResult validationResult = validateCompactionJob(job); ++ if (!validationResult.isValid()) { ++ log.error("Skipping invalid compaction job[%s] due to reason[%s].", job, validationResult.getReason()); ++ snapshotBuilder.moveFromPendingToSkipped(candidate); ++ return false; ++ } ++ ++ // Check if the job is already running, completed or skipped ++ final CompactionStatus compactionStatus = getCurrentStatusForJob(job, policy); ++ switch (compactionStatus.getState()) { ++ case RUNNING: ++ return false; ++ case COMPLETE: ++ snapshotBuilder.moveFromPendingToCompleted(candidate); ++ return false; ++ case SKIPPED: ++ snapshotBuilder.moveFromPendingToSkipped(candidate); ++ return false; ++ default: ++ break; ++ } ++ ++ // Check if enough compaction task slots are available ++ if (job.getMaxRequiredTaskSlots() > slotManager.getNumAvailableTaskSlots()) { ++ pendingJobs.add(job); ++ return false; ++ } ++ ++ // Reserve task slots and try to start the task ++ slotManager.reserveTaskSlots(job.getMaxRequiredTaskSlots()); ++ final String taskId = startTaskIfReady(job); ++ if (taskId == null) { ++ // Mark the job as skipped for now as the intervals might be locked by other tasks ++ snapshotBuilder.moveFromPendingToSkipped(candidate); ++ return false; ++ } else { ++ statusTracker.onTaskSubmitted(taskId, job.getCandidate()); ++ submittedTaskIdToJob.put(taskId, job); ++ return true; ++ } ++ } ++ ++ /** ++ * Starts the given job if the underlying Task is able to acquire locks. ++ * ++ * @return Non-null taskId if the Task was submitted successfully. ++ */ ++ @Nullable ++ private String startTaskIfReady(CompactionJob job) ++ { ++ // Assume MSQ jobs to be always ready ++ if (job.isMsq()) { ++ try { ++ return FutureUtils.getUnchecked(brokerClient.submitSqlTask(job.getNonNullMsqQuery()), true) ++ .getTaskId(); ++ } ++ catch (Exception e) { ++ log.error(e, "Error while submitting query[%s] to Broker", job.getNonNullMsqQuery()); ++ } ++ } ++ ++ final ClientTaskQuery taskQuery = job.getNonNullTask(); ++ final Task task = objectMapper.convertValue(taskQuery, Task.class); ++ ++ log.debug( ++ "Checking readiness of task[%s] with interval[%s]", ++ task.getId(), job.getCandidate().getCompactionInterval() ++ ); ++ try { ++ taskLockbox.add(task); ++ if (task.isReady(taskActionClientFactory.create(task))) { ++ // Hold the locks acquired by task.isReady() as we will reacquire them anyway ++ FutureUtils.getUnchecked(overlordClient.runTask(task.getId(), task), true); ++ return task.getId(); ++ } else { ++ taskLockbox.unlockAll(task); ++ return null; ++ } ++ } ++ catch (Exception e) { ++ log.error(e, "Error while submitting task[%s] to Overlord", task.getId()); ++ taskLockbox.unlockAll(task); ++ return null; ++ } ++ } ++ ++ public CompactionStatus getCurrentStatusForJob(CompactionJob job, CompactionCandidateSearchPolicy policy) ++ { ++ final CompactionStatus compactionStatus = statusTracker.computeCompactionStatus(job.getCandidate(), policy); ++ final CompactionCandidate candidatesWithStatus = job.getCandidate().withCurrentStatus(null); ++ statusTracker.onCompactionStatusComputed(candidatesWithStatus, null); ++ return compactionStatus; ++ } ++ ++ public static CompactionConfigValidationResult validateCompactionJob(BatchIndexingJob job) ++ { ++ // For MSQ jobs, do not perform any validation ++ if (job.isMsq()) { ++ return CompactionConfigValidationResult.success(); ++ } ++ ++ final ClientTaskQuery task = job.getNonNullTask(); ++ if (!(task instanceof ClientCompactionTaskQuery)) { ++ return CompactionConfigValidationResult.failure("Invalid task type[%s]", task.getType()); ++ } ++ ++ return CompactionConfigValidationResult.success(); ++ } ++} +diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionJobTemplate.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionJobTemplate.java +new file mode 100644 +index 0000000000..cd5096a8cf +--- /dev/null ++++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionJobTemplate.java +@@ -0,0 +1,50 @@ ++/* ++ * 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.compact; ++ ++import org.apache.druid.indexing.input.DruidInputSource; ++import org.apache.druid.indexing.template.BatchIndexingJobTemplate; ++import org.apache.druid.java.util.common.granularity.Granularity; ++ ++import javax.annotation.Nullable; ++import java.util.List; ++ ++/** ++ * Base indexing template for creating {@link CompactionJob}. ++ */ ++public interface CompactionJobTemplate extends BatchIndexingJobTemplate ++{ ++ /** ++ * Creates compaction jobs with this template for the given datasource. ++ */ ++ List createCompactionJobs( ++ DruidInputSource source, ++ CompactionJobParams jobParams ++ ); ++ ++ /** ++ * Granularity of segments created upon successful compaction. ++ * ++ * @return null only if this template does not change segment granularity upon ++ * successful compaction. ++ */ ++ @Nullable ++ Granularity getSegmentGranularity(); ++} +diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionRule.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionRule.java +new file mode 100644 +index 0000000000..10099566ea +--- /dev/null ++++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionRule.java +@@ -0,0 +1,88 @@ ++/* ++ * 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.compact; ++ ++import com.fasterxml.jackson.annotation.JsonCreator; ++import com.fasterxml.jackson.annotation.JsonProperty; ++import org.apache.druid.java.util.common.granularity.Granularity; ++import org.joda.time.DateTime; ++import org.joda.time.Period; ++ ++/** ++ * A single rule used inside {@link CascadingCompactionTemplate}. ++ */ ++public class CompactionRule ++{ ++ private final Period period; ++ private final CompactionJobTemplate template; ++ ++ @JsonCreator ++ public CompactionRule( ++ @JsonProperty("period") Period period, ++ @JsonProperty("template") CompactionJobTemplate template ++ ) ++ { ++ this.period = period; ++ this.template = template; ++ } ++ ++ @JsonProperty ++ public CompactionJobTemplate getTemplate() ++ { ++ return template; ++ } ++ ++ @JsonProperty ++ public Period getPeriod() ++ { ++ return period; ++ } ++ ++ /** ++ * Computes the start time of this rule by subtracting its period from the ++ * reference timestamp. ++ *

++ * If both this rule and the {@code beforeRule} explicitly specify a target ++ * segment granularity, the start time may be adjusted to ensure that there ++ * are no uncompacted gaps left in the timeline. ++ * ++ * @param referenceTime Current time when the rules are being evaluated ++ * @param beforeRule The rule before this one in chronological order ++ */ ++ public DateTime computeStartTime(DateTime referenceTime, CompactionRule beforeRule) ++ { ++ final Granularity granularity = template.getSegmentGranularity(); ++ final Granularity beforeGranularity = beforeRule.template.getSegmentGranularity(); ++ ++ final DateTime calculatedStartTime = referenceTime.minus(period); ++ ++ if (granularity == null || beforeGranularity == null) { ++ return calculatedStartTime; ++ } else { ++ // The gap can be filled only if it is bigger than the granularity of this rule. ++ // If beforeGranularity > granularity, gap would always be smaller than both ++ final DateTime beforeRuleEffectiveEnd = beforeGranularity.bucketStart(calculatedStartTime); ++ final DateTime possibleStartTime = granularity.bucketStart(beforeRuleEffectiveEnd); ++ return possibleStartTime.isBefore(beforeRuleEffectiveEnd) ++ ? granularity.increment(possibleStartTime) ++ : possibleStartTime; ++ } ++ } ++} +diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionScheduler.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionScheduler.java +index 5f0aa6e3ea..6f5ed1a7a6 100644 +--- a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionScheduler.java ++++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionScheduler.java +@@ -65,7 +65,7 @@ public interface CompactionScheduler + /** + * Starts compaction for a datasource if not already running. + */ +- void startCompaction(String dataSourceName, DataSourceCompactionConfig compactionConfig); ++ void startCompaction(String dataSourceName, CompactionSupervisor supervisor); + + /** + * Stops compaction for a datasource if currently running. +diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionStateMatcher.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionStateMatcher.java +new file mode 100644 +index 0000000000..84a95e2f6e +--- /dev/null ++++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionStateMatcher.java +@@ -0,0 +1,186 @@ ++/* ++ * 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.compact; ++ ++import com.fasterxml.jackson.annotation.JsonCreator; ++import com.fasterxml.jackson.annotation.JsonInclude; ++import com.fasterxml.jackson.annotation.JsonProperty; ++import org.apache.druid.data.input.impl.AggregateProjectionSpec; ++import org.apache.druid.data.input.impl.DimensionsSpec; ++import org.apache.druid.indexer.partitions.PartitionsSpec; ++import org.apache.druid.java.util.common.granularity.Granularity; ++import org.apache.druid.query.aggregation.AggregatorFactory; ++import org.apache.druid.segment.IndexSpec; ++import org.apache.druid.segment.transform.CompactionTransformSpec; ++import org.apache.druid.server.coordinator.UserCompactionTaskGranularityConfig; ++ ++import javax.annotation.Nullable; ++import java.util.Arrays; ++import java.util.List; ++import java.util.Objects; ++ ++/** ++ * Target compacted state of segments used to determine if compaction is needed ++ * for an interval. An explicitly defined target state helps avoid superfluous ++ * compaction when only the job definition has changed. ++ *

++ * This class is mostly a duplicate of {@code CompactionState} but is kept ++ * separate to allow: ++ *

    ++ *
  • fields to be nullable so that only non-null fields are used for matching
  • ++ *
  • legacy "compaction-incompatible" fields such as {@link #transformSpec} to ++ * be removed in the future. These fields do not just change the layout/partitioning ++ * of the data but may also alter its meaning which does not fall in the purview ++ * of compaction.
  • ++ *
++ */ ++@JsonInclude(JsonInclude.Include.NON_NULL) ++public class CompactionStateMatcher ++{ ++ private final PartitionsSpec partitionsSpec; ++ private final DimensionsSpec dimensionsSpec; ++ private final CompactionTransformSpec transformSpec; ++ private final IndexSpec indexSpec; ++ private final UserCompactionTaskGranularityConfig granularitySpec; ++ private final AggregatorFactory[] metricsSpec; ++ private final List projections; ++ ++ @JsonCreator ++ public CompactionStateMatcher( ++ @JsonProperty("partitionsSpec") @Nullable PartitionsSpec partitionsSpec, ++ @JsonProperty("dimensionsSpec") @Nullable DimensionsSpec dimensionsSpec, ++ @JsonProperty("metricsSpec") @Nullable AggregatorFactory[] metricsSpec, ++ @JsonProperty("transformSpec") @Nullable CompactionTransformSpec transformSpec, ++ @JsonProperty("indexSpec") @Nullable IndexSpec indexSpec, ++ @JsonProperty("granularitySpec") @Nullable UserCompactionTaskGranularityConfig granularitySpec, ++ @JsonProperty("projections") @Nullable List projections ++ ) ++ { ++ this.partitionsSpec = partitionsSpec; ++ this.dimensionsSpec = dimensionsSpec; ++ this.metricsSpec = metricsSpec; ++ this.transformSpec = transformSpec; ++ this.indexSpec = indexSpec; ++ this.granularitySpec = granularitySpec; ++ this.projections = projections; ++ } ++ ++ @Nullable ++ @JsonProperty ++ public PartitionsSpec getPartitionsSpec() ++ { ++ return partitionsSpec; ++ } ++ ++ @Nullable ++ @JsonProperty ++ public DimensionsSpec getDimensionsSpec() ++ { ++ return dimensionsSpec; ++ } ++ ++ @Nullable ++ @JsonProperty ++ public AggregatorFactory[] getMetricsSpec() ++ { ++ return metricsSpec; ++ } ++ ++ @Nullable ++ @JsonProperty ++ public CompactionTransformSpec getTransformSpec() ++ { ++ return transformSpec; ++ } ++ ++ @Nullable ++ @JsonProperty ++ public IndexSpec getIndexSpec() ++ { ++ return indexSpec; ++ } ++ ++ @Nullable ++ @JsonProperty ++ public UserCompactionTaskGranularityConfig getGranularitySpec() ++ { ++ return granularitySpec; ++ } ++ ++ @JsonProperty ++ @Nullable ++ public List getProjections() ++ { ++ return projections; ++ } ++ ++ @Nullable ++ public Granularity getSegmentGranularity() ++ { ++ return granularitySpec == null ? null : granularitySpec.getSegmentGranularity(); ++ } ++ ++ @Override ++ public boolean equals(Object o) ++ { ++ if (this == o) { ++ return true; ++ } ++ if (o == null || getClass() != o.getClass()) { ++ return false; ++ } ++ CompactionStateMatcher that = (CompactionStateMatcher) o; ++ return Objects.equals(partitionsSpec, that.partitionsSpec) && ++ Objects.equals(dimensionsSpec, that.dimensionsSpec) && ++ Objects.equals(transformSpec, that.transformSpec) && ++ Objects.equals(indexSpec, that.indexSpec) && ++ Objects.equals(granularitySpec, that.granularitySpec) && ++ Arrays.equals(metricsSpec, that.metricsSpec) && ++ Objects.equals(projections, that.projections); ++ } ++ ++ @Override ++ public int hashCode() ++ { ++ return Objects.hash( ++ partitionsSpec, ++ dimensionsSpec, ++ transformSpec, ++ indexSpec, ++ granularitySpec, ++ Arrays.hashCode(metricsSpec), ++ projections ++ ); ++ } ++ ++ @Override ++ public String toString() ++ { ++ return "CompactionState{" + ++ "partitionsSpec=" + partitionsSpec + ++ ", dimensionsSpec=" + dimensionsSpec + ++ ", transformSpec=" + transformSpec + ++ ", indexSpec=" + indexSpec + ++ ", granularitySpec=" + granularitySpec + ++ ", metricsSpec=" + Arrays.toString(metricsSpec) + ++ ", projections=" + projections + ++ '}'; ++ } ++} +diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionSupervisor.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionSupervisor.java +index 851b3920b1..f9ba0eee6d 100644 +--- a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionSupervisor.java ++++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionSupervisor.java +@@ -19,6 +19,7 @@ + + package org.apache.druid.indexing.compact; + ++import org.apache.druid.indexing.input.DruidInputSource; + import org.apache.druid.indexing.overlord.DataSourceMetadata; + import org.apache.druid.indexing.overlord.supervisor.Supervisor; + import org.apache.druid.indexing.overlord.supervisor.SupervisorReport; +@@ -29,6 +30,7 @@ import org.apache.druid.java.util.common.logger.Logger; + import org.apache.druid.server.coordinator.AutoCompactionSnapshot; + + import javax.annotation.Nullable; ++import java.util.List; + + /** + * Supervisor for compaction of a single datasource. +@@ -51,6 +53,31 @@ public class CompactionSupervisor implements Supervisor + this.dataSource = supervisorSpec.getSpec().getDataSource(); + } + ++ public CompactionSupervisorSpec getSpec() ++ { ++ return supervisorSpec; ++ } ++ ++ /** ++ * Checks if this supervisor is ready to create jobs in the current run of the ++ * scheduler. ++ */ ++ public boolean shouldCreateJobs() ++ { ++ return !supervisorSpec.isSuspended(); ++ } ++ ++ /** ++ * Creates compaction jobs for this supervisor. ++ */ ++ public List createJobs( ++ DruidInputSource inputSource, ++ CompactionJobParams jobParams ++ ) ++ { ++ return supervisorSpec.getTemplate().createCompactionJobs(inputSource, jobParams); ++ } ++ + @Override + public void start() + { +@@ -66,7 +93,7 @@ public class CompactionSupervisor implements Supervisor + ); + } else { + log.info("Starting compaction for dataSource[%s].", dataSource); +- scheduler.startCompaction(dataSource, supervisorSpec.getSpec()); ++ scheduler.startCompaction(dataSource, this); + } + } + +diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionSupervisorSpec.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionSupervisorSpec.java +index ca6008cc9d..998d7e0c7e 100644 +--- a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionSupervisorSpec.java ++++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionSupervisorSpec.java +@@ -93,6 +93,18 @@ public class CompactionSupervisorSpec implements SupervisorSpec + return new CompactionSupervisor(this, scheduler); + } + ++ /** ++ * @return {@link CompactionJobTemplate} used to create jobs for the supervisor. ++ */ ++ public CompactionJobTemplate getTemplate() ++ { ++ if (spec instanceof CascadingCompactionTemplate) { ++ return (CascadingCompactionTemplate) spec; ++ } else { ++ return new CompactionConfigBasedJobTemplate(spec); ++ } ++ } ++ + @Override + public List getDataSources() + { +diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/DruidInputSourceFactory.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/DruidInputSourceFactory.java +new file mode 100644 +index 0000000000..4089146404 +--- /dev/null ++++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/DruidInputSourceFactory.java +@@ -0,0 +1,74 @@ ++/* ++ * 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.compact; ++ ++import com.google.inject.Inject; ++import org.apache.druid.client.coordinator.CoordinatorClient; ++import org.apache.druid.indexing.common.SegmentCacheManagerFactory; ++import org.apache.druid.indexing.common.config.TaskConfig; ++import org.apache.druid.indexing.input.DruidInputSource; ++import org.apache.druid.segment.IndexIO; ++import org.joda.time.Interval; ++ ++/** ++ * Factory for creating {@link DruidInputSource} for a given datasource and ++ * interval used by {@link CompactionJobTemplate}. ++ */ ++public class DruidInputSourceFactory ++{ ++ private final IndexIO indexIO; ++ private final TaskConfig taskConfig; ++ private final CoordinatorClient coordinatorClient; ++ private final SegmentCacheManagerFactory segmentCacheManagerFactory; ++ ++ @Inject ++ public DruidInputSourceFactory( ++ IndexIO indexIO, ++ TaskConfig taskConfig, ++ CoordinatorClient coordinatorClient, ++ SegmentCacheManagerFactory segmentCacheManagerFactory ++ ) ++ { ++ this.indexIO = indexIO; ++ this.coordinatorClient = coordinatorClient; ++ this.segmentCacheManagerFactory = segmentCacheManagerFactory; ++ this.taskConfig = taskConfig; ++ } ++ ++ /** ++ * Creates a new {@link DruidInputSource} for the given {@code dataSource} and ++ * {@code interval}. ++ */ ++ public DruidInputSource create(String dataSource, Interval interval) ++ { ++ return new DruidInputSource( ++ dataSource, ++ interval, ++ null, ++ null, ++ null, ++ null, ++ indexIO, ++ coordinatorClient, ++ segmentCacheManagerFactory, ++ taskConfig ++ ); ++ } ++} +diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/InlineCompactionJobTemplate.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/InlineCompactionJobTemplate.java +new file mode 100644 +index 0000000000..43e4c6f8c2 +--- /dev/null ++++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/InlineCompactionJobTemplate.java +@@ -0,0 +1,98 @@ ++/* ++ * 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.compact; ++ ++import com.fasterxml.jackson.annotation.JsonCreator; ++import com.fasterxml.jackson.annotation.JsonProperty; ++import org.apache.druid.indexing.input.DruidInputSource; ++import org.apache.druid.java.util.common.granularity.Granularity; ++ ++import javax.annotation.Nullable; ++import java.util.List; ++import java.util.Objects; ++ ++/** ++ * Template to create compaction jobs using inline specifications. This template ++ * does not fetch any information from the Druid catalog while creating jobs. ++ */ ++public class InlineCompactionJobTemplate implements CompactionJobTemplate ++{ ++ public static final String TYPE = "compactInline"; ++ ++ private final CompactionStateMatcher targetState; ++ ++ @JsonCreator ++ public InlineCompactionJobTemplate( ++ @JsonProperty("targetState") CompactionStateMatcher targetState ++ ) ++ { ++ this.targetState = targetState; ++ } ++ ++ @JsonProperty ++ public CompactionStateMatcher getTargetState() ++ { ++ return targetState; ++ } ++ ++ @Nullable ++ @Override ++ public Granularity getSegmentGranularity() ++ { ++ return targetState.getSegmentGranularity(); ++ } ++ ++ @Override ++ public List createCompactionJobs( ++ DruidInputSource source, ++ CompactionJobParams jobParams ++ ) ++ { ++ final String dataSource = source.getDataSource(); ++ return CompactionConfigBasedJobTemplate ++ .create(dataSource, targetState) ++ .createCompactionJobs(source, jobParams); ++ } ++ ++ @Override ++ public boolean equals(Object object) ++ { ++ if (this == object) { ++ return true; ++ } ++ if (object == null || getClass() != object.getClass()) { ++ return false; ++ } ++ InlineCompactionJobTemplate that = (InlineCompactionJobTemplate) object; ++ return Objects.equals(this.targetState, that.targetState); ++ } ++ ++ @Override ++ public int hashCode() ++ { ++ return Objects.hash(targetState); ++ } ++ ++ @Override ++ public String getType() ++ { ++ return TYPE; ++ } ++} +diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/LocalOverlordClient.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/LocalOverlordClient.java +index 3f1427c7c3..eb200577bb 100644 +--- a/indexing-service/src/main/java/org/apache/druid/indexing/compact/LocalOverlordClient.java ++++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/LocalOverlordClient.java +@@ -72,10 +72,12 @@ class LocalOverlordClient extends NoopOverlordClient + @Override + public ListenableFuture runTask(String taskId, Object clientTaskQuery) + { ++ final CompactionTask task = ++ clientTaskQuery instanceof CompactionTask ++ ? (CompactionTask) clientTaskQuery ++ : convertTask(clientTaskQuery, ClientCompactionTaskQuery.class, CompactionTask.class); + return futureOf(() -> { +- getValidTaskQueue().add( +- convertTask(clientTaskQuery, ClientCompactionTaskQuery.class, CompactionTask.class) +- ); ++ getValidTaskQueue().add(task); + return null; + }); + } +diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/MSQCompactionJobTemplate.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/MSQCompactionJobTemplate.java +new file mode 100644 +index 0000000000..ea7193e443 +--- /dev/null ++++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/MSQCompactionJobTemplate.java +@@ -0,0 +1,191 @@ ++/* ++ * 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.compact; ++ ++import com.fasterxml.jackson.annotation.JsonCreator; ++import com.fasterxml.jackson.annotation.JsonProperty; ++import org.apache.druid.indexing.input.DruidInputSource; ++import org.apache.druid.java.util.common.StringUtils; ++import org.apache.druid.java.util.common.granularity.Granularity; ++import org.apache.druid.query.http.ClientSqlQuery; ++import org.apache.druid.server.compaction.CompactionCandidate; ++import org.apache.druid.server.compaction.CompactionSlotManager; ++import org.apache.druid.server.compaction.DataSourceCompactibleSegmentIterator; ++import org.apache.druid.server.coordinator.duty.CompactSegments; ++import org.joda.time.Interval; ++import org.joda.time.format.DateTimeFormat; ++import org.joda.time.format.DateTimeFormatter; ++ ++import javax.annotation.Nullable; ++import java.util.ArrayList; ++import java.util.HashMap; ++import java.util.List; ++import java.util.Map; ++import java.util.Objects; ++ ++/** ++ * Compaction template that creates MSQ SQL jobs using a templatized SQL with ++ * variables of the format {@code ${variableName}} for fields such as datasource ++ * name and start timestamp. ++ *

++ * Compaction is triggered for an interval only if the current compaction state ++ * of the underlying segments DOES NOT match with the {@link #targetState}. ++ */ ++public class MSQCompactionJobTemplate implements CompactionJobTemplate ++{ ++ public static final String TYPE = "compactMsq"; ++ ++ public static final String VAR_DATASOURCE = "${dataSource}"; ++ public static final String VAR_START_TIMESTAMP = "${startTimestamp}"; ++ public static final String VAR_END_TIMESTAMP = "${endTimestamp}"; ++ ++ private static final DateTimeFormatter TIMESTAMP_FORMATTER = ++ DateTimeFormat.forPattern("yyyy-MM-dd HH:mm:ss.SSS"); ++ ++ private final ClientSqlQuery sqlTemplate; ++ private final CompactionStateMatcher targetState; ++ ++ @JsonCreator ++ public MSQCompactionJobTemplate( ++ @JsonProperty("sqlTemplate") ClientSqlQuery sqlTemplate, ++ @JsonProperty("targetState") CompactionStateMatcher targetState ++ ) ++ { ++ this.sqlTemplate = sqlTemplate; ++ this.targetState = targetState; ++ } ++ ++ @JsonProperty ++ public ClientSqlQuery getSqlTemplate() ++ { ++ return sqlTemplate; ++ } ++ ++ @JsonProperty ++ public CompactionStateMatcher getTargetState() ++ { ++ return targetState; ++ } ++ ++ @Nullable ++ @Override ++ public Granularity getSegmentGranularity() ++ { ++ return targetState.getSegmentGranularity(); ++ } ++ ++ @Override ++ public List createCompactionJobs( ++ DruidInputSource source, ++ CompactionJobParams jobParams ++ ) ++ { ++ final String dataSource = source.getDataSource(); ++ ++ // Identify the compactible candidate segments ++ final CompactionConfigBasedJobTemplate delegate = ++ CompactionConfigBasedJobTemplate.create(dataSource, targetState); ++ final DataSourceCompactibleSegmentIterator candidateIterator = ++ delegate.getCompactibleCandidates(source, jobParams); ++ ++ // Create MSQ jobs for each candidate by interpolating the template variables ++ final List jobs = new ArrayList<>(); ++ while (candidateIterator.hasNext()) { ++ final CompactionCandidate candidate = candidateIterator.next(); ++ jobs.add( ++ new CompactionJob( ++ createQueryForJob(dataSource, candidate.getCompactionInterval()), ++ candidate, ++ CompactionSlotManager.getMaxTaskSlotsForMSQCompactionTask(sqlTemplate.getContext()) ++ ) ++ ); ++ } ++ ++ return jobs; ++ } ++ ++ private ClientSqlQuery createQueryForJob(String dataSource, Interval compactionInterval) ++ { ++ final String formattedSql = formatSql( ++ sqlTemplate.getQuery(), ++ Map.of( ++ VAR_DATASOURCE, dataSource, ++ VAR_START_TIMESTAMP, compactionInterval.getStart().toString(TIMESTAMP_FORMATTER), ++ VAR_END_TIMESTAMP, compactionInterval.getEnd().toString(TIMESTAMP_FORMATTER) ++ ) ++ ); ++ ++ final Map context = new HashMap<>(); ++ if (sqlTemplate.getContext() != null) { ++ context.putAll(sqlTemplate.getContext()); ++ } ++ context.put(CompactSegments.STORE_COMPACTION_STATE_KEY, true); ++ context.put(CompactSegments.COMPACTION_INTERVAL_KEY, compactionInterval); ++ ++ return new ClientSqlQuery( ++ formattedSql, ++ sqlTemplate.getResultFormat(), ++ sqlTemplate.isHeader(), ++ sqlTemplate.isTypesHeader(), ++ sqlTemplate.isSqlTypesHeader(), ++ context, ++ sqlTemplate.getParameters() ++ ); ++ } ++ ++ /** ++ * Formats the given SQL by replacing the template variables. ++ */ ++ public static String formatSql(String sqlTemplate, Map templateVariables) ++ { ++ String sql = sqlTemplate; ++ for (Map.Entry variable : templateVariables.entrySet()) { ++ sql = StringUtils.replace(sql, variable.getKey(), variable.getValue()); ++ } ++ ++ return sql; ++ } ++ ++ @Override ++ public boolean equals(Object object) ++ { ++ if (this == object) { ++ return true; ++ } ++ if (object == null || getClass() != object.getClass()) { ++ return false; ++ } ++ MSQCompactionJobTemplate that = (MSQCompactionJobTemplate) object; ++ return Objects.equals(sqlTemplate, that.sqlTemplate) ++ && Objects.equals(targetState, that.targetState); ++ } ++ ++ @Override ++ public int hashCode() ++ { ++ return Objects.hash(sqlTemplate, targetState); ++ } ++ ++ @Override ++ public String getType() ++ { ++ return TYPE; ++ } ++} +diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/OverlordCompactionScheduler.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/OverlordCompactionScheduler.java +index 95ea3d1169..c3bce6a09d 100644 +--- a/indexing-service/src/main/java/org/apache/druid/indexing/compact/OverlordCompactionScheduler.java ++++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/OverlordCompactionScheduler.java +@@ -24,15 +24,18 @@ import com.google.common.base.Optional; + import com.google.common.base.Supplier; + import com.google.inject.Inject; + import org.apache.druid.client.DataSourcesSnapshot; ++import org.apache.druid.client.broker.BrokerClient; + import org.apache.druid.client.indexing.ClientCompactionRunnerInfo; + import org.apache.druid.indexer.TaskLocation; + import org.apache.druid.indexer.TaskStatus; ++import org.apache.druid.indexing.common.actions.TaskActionClientFactory; ++import org.apache.druid.indexing.overlord.GlobalTaskLockbox; + import org.apache.druid.indexing.overlord.TaskMaster; + import org.apache.druid.indexing.overlord.TaskQueryTool; + import org.apache.druid.indexing.overlord.TaskRunner; + import org.apache.druid.indexing.overlord.TaskRunnerListener; ++import org.apache.druid.java.util.common.Intervals; + import org.apache.druid.java.util.common.Stopwatch; +-import org.apache.druid.java.util.common.concurrent.Execs; + import org.apache.druid.java.util.common.concurrent.ScheduledExecutorFactory; + import org.apache.druid.java.util.common.lifecycle.LifecycleStart; + import org.apache.druid.java.util.common.lifecycle.LifecycleStop; +@@ -40,6 +43,7 @@ import org.apache.druid.java.util.common.logger.Logger; + import org.apache.druid.java.util.emitter.service.ServiceEmitter; + import org.apache.druid.java.util.emitter.service.ServiceMetricEvent; + import org.apache.druid.metadata.SegmentsMetadataManager; ++import org.apache.druid.metadata.SegmentsMetadataManagerConfig; + import org.apache.druid.server.compaction.CompactionRunSimulator; + import org.apache.druid.server.compaction.CompactionSimulateResult; + import org.apache.druid.server.compaction.CompactionStatusTracker; +@@ -50,19 +54,20 @@ import org.apache.druid.server.coordinator.CoordinatorOverlordServiceConfig; + import org.apache.druid.server.coordinator.DataSourceCompactionConfig; + import org.apache.druid.server.coordinator.DruidCompactionConfig; + import org.apache.druid.server.coordinator.duty.CompactSegments; +-import org.apache.druid.server.coordinator.stats.CoordinatorRunStats; + import org.apache.druid.server.coordinator.stats.CoordinatorStat; +-import org.apache.druid.server.coordinator.stats.Dimension; ++import org.apache.druid.server.coordinator.stats.RowKey; + import org.apache.druid.server.coordinator.stats.Stats; +-import org.joda.time.Duration; + +-import java.util.ArrayList; + import java.util.Collections; ++import java.util.List; + import java.util.Map; + import java.util.concurrent.ConcurrentHashMap; + import java.util.concurrent.ScheduledExecutorService; + import java.util.concurrent.TimeUnit; + import java.util.concurrent.atomic.AtomicBoolean; ++import java.util.concurrent.atomic.AtomicReference; ++import java.util.function.Consumer; ++import java.util.stream.Collectors; + + /** + * Implementation of {@link CompactionScheduler}. +@@ -81,16 +86,31 @@ public class OverlordCompactionScheduler implements CompactionScheduler + { + private static final Logger log = new Logger(OverlordCompactionScheduler.class); + +- private static final long SCHEDULE_PERIOD_SECONDS = 5; +- private static final Duration METRIC_EMISSION_PERIOD = Duration.standardMinutes(5); ++ /** ++ * Scheduler run period is 15 minutes. It has been kept high to avoid eager ++ * recomputation of the queue as it may be a very compute-intensive operation ++ * taking upto several minutes on clusters with a large number of used segments. ++ * Jobs for a single supervisor may still be recomputed when the supervisor is updated. ++ */ ++ private static final long DEFAULT_SCHEDULE_PERIOD_MILLIS = 15 * 60_000; + + private final SegmentsMetadataManager segmentManager; + private final LocalOverlordClient overlordClient; ++ private final BrokerClient brokerClient; + private final ServiceEmitter emitter; ++ private final ObjectMapper objectMapper; + private final TaskMaster taskMaster; + + private final Supplier compactionConfigSupplier; +- private final ConcurrentHashMap activeDatasourceConfigs; ++ private final ConcurrentHashMap activeSupervisors; ++ ++ private final AtomicReference> datasourceToCompactionSnapshot; ++ private final AtomicBoolean shouldRecomputeJobsForAnyDatasource = new AtomicBoolean(false); ++ ++ /** ++ * Compaction job queue built in the last invocation of {@link #resetCompactionJobQueue()}. ++ */ ++ private final AtomicReference latestJobQueue; + + /** + * Single-threaded executor to process the compaction queue. +@@ -98,6 +118,9 @@ public class OverlordCompactionScheduler implements CompactionScheduler + private final ScheduledExecutorService executor; + + private final CompactionStatusTracker statusTracker; ++ private final TaskActionClientFactory taskActionClientFactory; ++ private final DruidInputSourceFactory druidInputSourceFactory; ++ private final GlobalTaskLockbox taskLockbox; + + /** + * Listener to watch task completion events and update CompactionStatusTracker. +@@ -107,7 +130,6 @@ public class OverlordCompactionScheduler implements CompactionScheduler + + private final AtomicBoolean isLeader = new AtomicBoolean(false); + private final AtomicBoolean started = new AtomicBoolean(false); +- private final CompactSegments duty; + + /** + * The scheduler should enable/disable polling of segments only if the Overlord +@@ -115,25 +137,35 @@ public class OverlordCompactionScheduler implements CompactionScheduler + * class itself. + */ + private final boolean shouldPollSegments; +- +- private final Stopwatch sinceStatsEmitted = Stopwatch.createUnstarted(); ++ private final long schedulePeriodMillis; + + @Inject + public OverlordCompactionScheduler( + TaskMaster taskMaster, ++ GlobalTaskLockbox taskLockbox, + TaskQueryTool taskQueryTool, + SegmentsMetadataManager segmentManager, ++ SegmentsMetadataManagerConfig segmentManagerConfig, + Supplier compactionConfigSupplier, + CompactionStatusTracker statusTracker, + CoordinatorOverlordServiceConfig coordinatorOverlordServiceConfig, ++ TaskActionClientFactory taskActionClientFactory, ++ DruidInputSourceFactory druidInputSourceFactory, + ScheduledExecutorFactory executorFactory, ++ BrokerClient brokerClient, + ServiceEmitter emitter, + ObjectMapper objectMapper + ) + { ++ final long segmentPollPeriodMillis = ++ segmentManagerConfig.getPollDuration().toStandardDuration().getMillis(); ++ this.schedulePeriodMillis = Math.min(DEFAULT_SCHEDULE_PERIOD_MILLIS, segmentPollPeriodMillis); ++ + this.segmentManager = segmentManager; + this.emitter = emitter; ++ this.objectMapper = objectMapper; + this.taskMaster = taskMaster; ++ this.taskLockbox = taskLockbox; + this.compactionConfigSupplier = compactionConfigSupplier; + + this.executor = executorFactory.create(1, "CompactionScheduler-%s"); +@@ -141,9 +173,13 @@ public class OverlordCompactionScheduler implements CompactionScheduler + this.shouldPollSegments = segmentManager != null + && !coordinatorOverlordServiceConfig.isEnabled(); + this.overlordClient = new LocalOverlordClient(taskMaster, taskQueryTool, objectMapper); +- this.duty = new CompactSegments(this.statusTracker, overlordClient); +- this.activeDatasourceConfigs = new ConcurrentHashMap<>(); ++ this.brokerClient = brokerClient; ++ this.activeSupervisors = new ConcurrentHashMap<>(); ++ this.datasourceToCompactionSnapshot = new AtomicReference<>(); ++ this.latestJobQueue = new AtomicReference<>(); + ++ this.taskActionClientFactory = taskActionClientFactory; ++ this.druidInputSourceFactory = druidInputSourceFactory; + this.taskRunnerListener = new TaskRunnerListener() + { + @Override +@@ -162,7 +198,8 @@ public class OverlordCompactionScheduler implements CompactionScheduler + public void statusChanged(String taskId, TaskStatus status) + { + if (status.isComplete()) { +- statusTracker.onTaskFinished(taskId, status); ++ onTaskFinished(taskId, status); ++ launchPendingJobs(); + } + } + }; +@@ -184,7 +221,8 @@ public class OverlordCompactionScheduler implements CompactionScheduler + public void becomeLeader() + { + if (isLeader.compareAndSet(false, true)) { +- scheduleOnExecutor(this::scheduledRun, SCHEDULE_PERIOD_SECONDS); ++ // Schedule first run after a small delay ++ scheduleOnExecutor(this::scheduledRun, 1_000L); + } + } + +@@ -208,25 +246,31 @@ public class OverlordCompactionScheduler implements CompactionScheduler + } else { + return ClientCompactionRunnerInfo.validateCompactionConfig( + compactionConfig, +- getLatestConfig().getEngine() ++ getLatestClusterConfig().getEngine() + ); + } + } + + @Override +- public void startCompaction(String dataSourceName, DataSourceCompactionConfig config) ++ public void startCompaction(String dataSourceName, CompactionSupervisor supervisor) + { +- // Track active datasources even if scheduler has not started yet because ++ // Track active supervisors even if scheduler has not started yet because + // SupervisorManager is started before the scheduler + if (isEnabled()) { +- activeDatasourceConfigs.put(dataSourceName, config); ++ activeSupervisors.put(dataSourceName, supervisor); ++ ++ if (started.get()) { ++ shouldRecomputeJobsForAnyDatasource.set(true); ++ scheduleOnExecutor(() -> recreateJobs(dataSourceName, supervisor), 0L); ++ } + } + } + + @Override + public void stopCompaction(String dataSourceName) + { +- activeDatasourceConfigs.remove(dataSourceName); ++ activeSupervisors.remove(dataSourceName); ++ updateQueueIfComputed(queue -> queue.removeJobs(dataSourceName)); + statusTracker.removeDatasource(dataSourceName); + } + +@@ -239,10 +283,10 @@ public class OverlordCompactionScheduler implements CompactionScheduler + return; + } + +- log.info("Starting compaction scheduler."); ++ log.info("Starting compaction scheduler with period [%d] millis.", schedulePeriodMillis); + final Optional taskRunnerOptional = taskMaster.getTaskRunner(); + if (taskRunnerOptional.isPresent()) { +- taskRunnerOptional.get().registerListener(taskRunnerListener, Execs.directExecutor()); ++ taskRunnerOptional.get().registerListener(taskRunnerListener, executor); + } + if (shouldPollSegments) { + segmentManager.startPollingDatabasePeriodically(); +@@ -264,7 +308,8 @@ public class OverlordCompactionScheduler implements CompactionScheduler + taskRunnerOptional.get().unregisterListener(taskRunnerListener.getListenerId()); + } + statusTracker.stop(); +- activeDatasourceConfigs.clear(); ++ activeSupervisors.clear(); ++ latestJobQueue.set(null); + + if (shouldPollSegments) { + segmentManager.stopPollingDatabasePeriodically(); +@@ -291,54 +336,119 @@ public class OverlordCompactionScheduler implements CompactionScheduler + if (isEnabled()) { + initState(); + try { +- runCompactionDuty(); ++ resetCompactionJobQueue(); + } + catch (Exception e) { + log.error(e, "Error processing compaction queue. Continuing schedule."); + } +- scheduleOnExecutor(this::scheduledRun, SCHEDULE_PERIOD_SECONDS); ++ scheduleOnExecutor(this::scheduledRun, schedulePeriodMillis); + } else { + cleanupState(); +- scheduleOnExecutor(this::scheduledRun, SCHEDULE_PERIOD_SECONDS * 4); ++ scheduleOnExecutor(this::scheduledRun, schedulePeriodMillis); + } + } + + /** +- * Runs the compaction duty and emits stats if {@link #METRIC_EMISSION_PERIOD} +- * has elapsed. ++ * Creates and launches eligible compaction jobs. + */ +- private synchronized void runCompactionDuty() ++ private synchronized void resetCompactionJobQueue() + { +- final CoordinatorRunStats stats = new CoordinatorRunStats(); +- duty.run(getLatestConfig(), getDatasourceSnapshot(), getLatestConfig().getEngine(), stats); +- +- // Emit stats only if emission period has elapsed +- if (!sinceStatsEmitted.isRunning() || sinceStatsEmitted.hasElapsed(METRIC_EMISSION_PERIOD)) { +- stats.forEachStat( +- (stat, dimensions, value) -> { +- if (stat.shouldEmit()) { +- emitStat(stat, dimensions.getValues(), value); +- } +- } +- ); +- sinceStatsEmitted.restart(); +- } else { +- // Always emit number of submitted tasks +- long numSubmittedTasks = stats.get(Stats.Compaction.SUBMITTED_TASKS); +- emitStat(Stats.Compaction.SUBMITTED_TASKS, Collections.emptyMap(), numSubmittedTasks); ++ // Remove the old queue so that no more jobs are added to it ++ latestJobQueue.set(null); ++ ++ final Stopwatch runDuration = Stopwatch.createStarted(); ++ final DataSourcesSnapshot dataSourcesSnapshot = getDatasourceSnapshot(); ++ final CompactionJobQueue queue = new CompactionJobQueue( ++ dataSourcesSnapshot, ++ getLatestClusterConfig(), ++ statusTracker, ++ taskActionClientFactory, ++ taskLockbox, ++ overlordClient, ++ brokerClient, ++ objectMapper ++ ); ++ latestJobQueue.set(queue); ++ ++ statusTracker.resetActiveDatasources(activeSupervisors.keySet()); ++ statusTracker.onSegmentTimelineUpdated(dataSourcesSnapshot.getSnapshotTime()); ++ ++ // Jobs for all active supervisors are being freshly created ++ // recomputation will not be needed ++ shouldRecomputeJobsForAnyDatasource.set(false); ++ activeSupervisors.forEach(this::createAndEnqueueJobs); ++ ++ launchPendingJobs(); ++ queue.getRunStats().forEachStat(this::emitStat); ++ emitStat(Stats.Compaction.SCHEDULER_RUN_TIME, RowKey.empty(), runDuration.millisElapsed()); ++ } ++ ++ /** ++ * Launches pending compaction jobs if compaction task slots become available. ++ * This method uses the jobs created by the last invocation of {@link #resetCompactionJobQueue()}. ++ */ ++ private synchronized void launchPendingJobs() ++ { ++ updateQueueIfComputed(queue -> { ++ queue.runReadyJobs(); ++ updateCompactionSnapshots(queue); ++ }); ++ } ++ ++ private synchronized void recreateJobs(String dataSource, CompactionSupervisor supervisor) ++ { ++ if (shouldRecomputeJobsForAnyDatasource.get()) { ++ createAndEnqueueJobs(dataSource, supervisor); ++ } ++ } ++ ++ private synchronized void createAndEnqueueJobs(String dataSource, CompactionSupervisor supervisor) ++ { ++ updateQueueIfComputed( ++ queue -> queue.createAndEnqueueJobs( ++ supervisor, ++ druidInputSourceFactory.create(dataSource, Intervals.ETERNITY) ++ ) ++ ); ++ } ++ ++ /** ++ * Performs an operation on the {@link #latestJobQueue} if it has been already ++ * computed. ++ */ ++ private void updateQueueIfComputed(Consumer operation) ++ { ++ final CompactionJobQueue queue = latestJobQueue.get(); ++ if (queue != null) { ++ operation.accept(queue); + } + } + ++ private void onTaskFinished(String taskId, TaskStatus taskStatus) ++ { ++ statusTracker.onTaskFinished(taskId, taskStatus); ++ ++ updateQueueIfComputed(queue -> { ++ queue.onTaskFinished(taskId, taskStatus); ++ updateCompactionSnapshots(queue); ++ }); ++ } ++ ++ private void updateCompactionSnapshots(CompactionJobQueue queue) ++ { ++ datasourceToCompactionSnapshot.set(queue.getSnapshots()); ++ } ++ + @Override + public AutoCompactionSnapshot getCompactionSnapshot(String dataSource) + { +- if (!activeDatasourceConfigs.containsKey(dataSource)) { ++ if (!activeSupervisors.containsKey(dataSource)) { + return AutoCompactionSnapshot.builder(dataSource) + .withStatus(AutoCompactionSnapshot.ScheduleStatus.NOT_ENABLED) + .build(); + } + +- final AutoCompactionSnapshot snapshot = duty.getAutoCompactionSnapshot(dataSource); ++ final AutoCompactionSnapshot snapshot = datasourceToCompactionSnapshot.get().get(dataSource); + if (snapshot == null) { + final AutoCompactionSnapshot.ScheduleStatus status = + isEnabled() +@@ -353,7 +463,7 @@ public class OverlordCompactionScheduler implements CompactionScheduler + @Override + public Map getAllCompactionSnapshots() + { +- return duty.getAutoCompactionSnapshot(); ++ return Map.copyOf(datasourceToCompactionSnapshot.get()); + } + + @Override +@@ -363,17 +473,21 @@ public class OverlordCompactionScheduler implements CompactionScheduler + return new CompactionRunSimulator(statusTracker, overlordClient).simulateRunWithConfig( + getLatestConfig().withClusterConfig(updateRequest), + getDatasourceSnapshot(), +- getLatestConfig().getEngine() ++ updateRequest.getEngine() + ); + } else { + return new CompactionSimulateResult(Collections.emptyMap()); + } + } + +- private void emitStat(CoordinatorStat stat, Map dimensionValues, long value) ++ private void emitStat(CoordinatorStat stat, RowKey rowKey, long value) + { ++ if (!stat.shouldEmit()) { ++ return; ++ } ++ + ServiceMetricEvent.Builder eventBuilder = new ServiceMetricEvent.Builder(); +- dimensionValues.forEach( ++ rowKey.getValues().forEach( + (dim, dimValue) -> eventBuilder.setDimension(dim.reportedName(), dimValue) + ); + emitter.emit(eventBuilder.setMetric(stat.getMetricName(), value)); +@@ -381,10 +495,20 @@ public class OverlordCompactionScheduler implements CompactionScheduler + + private DruidCompactionConfig getLatestConfig() + { ++ final List configs = activeSupervisors ++ .values() ++ .stream() ++ .map(s -> s.getSpec().getSpec()) ++ .collect(Collectors.toList()); + return DruidCompactionConfig + .empty() +- .withClusterConfig(compactionConfigSupplier.get().clusterConfig()) +- .withDatasourceConfigs(new ArrayList<>(activeDatasourceConfigs.values())); ++ .withClusterConfig(getLatestClusterConfig()) ++ .withDatasourceConfigs(configs); ++ } ++ ++ private ClusterCompactionConfig getLatestClusterConfig() ++ { ++ return compactionConfigSupplier.get().clusterConfig(); + } + + private DataSourcesSnapshot getDatasourceSnapshot() +@@ -392,7 +516,7 @@ public class OverlordCompactionScheduler implements CompactionScheduler + return segmentManager.getRecentDataSourcesSnapshot(); + } + +- private void scheduleOnExecutor(Runnable runnable, long delaySeconds) ++ private void scheduleOnExecutor(Runnable runnable, long delayMillis) + { + executor.schedule( + () -> { +@@ -403,8 +527,8 @@ public class OverlordCompactionScheduler implements CompactionScheduler + log.error(t, "Error while executing runnable"); + } + }, +- delaySeconds, +- TimeUnit.SECONDS ++ delayMillis, ++ TimeUnit.MILLISECONDS + ); + } + } +diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/input/DruidInputSource.java b/indexing-service/src/main/java/org/apache/druid/indexing/input/DruidInputSource.java +index 84c17f1a8f..fea6f30ef4 100644 +--- a/indexing-service/src/main/java/org/apache/druid/indexing/input/DruidInputSource.java ++++ b/indexing-service/src/main/java/org/apache/druid/indexing/input/DruidInputSource.java +@@ -291,6 +291,25 @@ public class DruidInputSource extends AbstractInputSource implements SplittableI + ); + } + ++ /** ++ * Creates a new {@link DruidInputSource} with the given interval. ++ */ ++ public DruidInputSource withInterval(Interval interval) ++ { ++ return new DruidInputSource( ++ this.dataSource, ++ interval, ++ null, ++ this.dimFilter, ++ this.dimensions, ++ this.metrics, ++ this.indexIO, ++ this.coordinatorClient, ++ this.segmentCacheManagerFactory, ++ this.taskConfig ++ ); ++ } ++ + @Override + protected InputSourceReader fixedFormatReader(InputRowSchema inputRowSchema, @Nullable File temporaryDirectory) + { +diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskQueue.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskQueue.java +index 0cfe481a8f..c923e70fca 100644 +--- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskQueue.java ++++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskQueue.java +@@ -1023,6 +1023,19 @@ public class TaskQueue + } + + /** ++ * List of all active tasks currently being managed by this TaskQueue. ++ */ ++ public List getActiveTasks() ++ { ++ return activeTasks.values() ++ .stream() ++ .filter(entry -> !entry.isComplete) ++ .map(TaskEntry::getTask) ++ .collect(Collectors.toList()); ++ } ++ ++ /** ++ * Returns the list of currently active tasks for the given datasource. + * List of all active and completed task infos currently being managed by this TaskQueue. + */ + public List getTaskInfos() +diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/compact/CompactionSupervisorSpecTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/compact/CompactionSupervisorSpecTest.java +index b1732acabb..b008d74829 100644 +--- a/indexing-service/src/test/java/org/apache/druid/indexing/compact/CompactionSupervisorSpecTest.java ++++ b/indexing-service/src/test/java/org/apache/druid/indexing/compact/CompactionSupervisorSpecTest.java +@@ -137,7 +137,7 @@ public class CompactionSupervisorSpecTest + supervisor.start(); + supervisor.stop(false); + +- Mockito.verify(scheduler, Mockito.times(1)).startCompaction(TestDataSource.WIKI, spec); ++ Mockito.verify(scheduler, Mockito.times(1)).startCompaction(TestDataSource.WIKI, supervisor); + Mockito.verify(scheduler, Mockito.times(1)).stopCompaction(TestDataSource.WIKI); + } + +@@ -157,7 +157,7 @@ public class CompactionSupervisorSpecTest + supervisor.start(); + supervisor.stop(false); + +- Mockito.verify(scheduler, Mockito.times(1)).startCompaction(TestDataSource.WIKI, spec); ++ Mockito.verify(scheduler, Mockito.times(1)).startCompaction(TestDataSource.WIKI, supervisor); + Mockito.verify(scheduler, Mockito.times(1)).stopCompaction(TestDataSource.WIKI); + } + +diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/compact/OverlordCompactionSchedulerTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/compact/OverlordCompactionSchedulerTest.java +index 9353953fc2..1484a719db 100644 +--- a/indexing-service/src/test/java/org/apache/druid/indexing/compact/OverlordCompactionSchedulerTest.java ++++ b/indexing-service/src/test/java/org/apache/druid/indexing/compact/OverlordCompactionSchedulerTest.java +@@ -21,15 +21,31 @@ package org.apache.druid.indexing.compact; + + import com.fasterxml.jackson.databind.InjectableValues; + import com.fasterxml.jackson.databind.ObjectMapper; ++import com.google.common.base.Optional; ++import com.google.common.util.concurrent.Futures; ++import org.apache.druid.client.broker.BrokerClient; ++import org.apache.druid.client.coordinator.NoopCoordinatorClient; + import org.apache.druid.client.indexing.ClientMSQContext; ++import org.apache.druid.common.utils.IdUtils; + import org.apache.druid.guice.IndexingServiceTuningConfigModule; ++import org.apache.druid.guice.SupervisorModule; + import org.apache.druid.indexer.CompactionEngine; ++import org.apache.druid.indexer.TaskState; ++import org.apache.druid.indexer.TaskStatus; + import org.apache.druid.indexing.common.SegmentCacheManagerFactory; ++import org.apache.druid.indexing.common.TimeChunkLock; ++import org.apache.druid.indexing.common.actions.RetrieveUsedSegmentsAction; ++import org.apache.druid.indexing.common.actions.TaskAction; ++import org.apache.druid.indexing.common.actions.TaskActionClient; ++import org.apache.druid.indexing.common.actions.TaskActionClientFactory; ++import org.apache.druid.indexing.common.actions.TimeChunkLockTryAcquireAction; ++import org.apache.druid.indexing.common.config.TaskConfig; + import org.apache.druid.indexing.common.config.TaskStorageConfig; + import org.apache.druid.indexing.common.task.CompactionTask; + import org.apache.druid.indexing.common.task.Task; + import org.apache.druid.indexing.overlord.GlobalTaskLockbox; + import org.apache.druid.indexing.overlord.HeapMemoryTaskStorage; ++import org.apache.druid.indexing.overlord.Segments; + import org.apache.druid.indexing.overlord.TaskMaster; + import org.apache.druid.indexing.overlord.TaskQueryTool; + import org.apache.druid.indexing.overlord.TaskQueue; +@@ -38,9 +54,16 @@ import org.apache.druid.indexing.overlord.setup.DefaultWorkerBehaviorConfig; + import org.apache.druid.indexing.overlord.setup.WorkerBehaviorConfig; + import org.apache.druid.indexing.test.TestIndexerMetadataStorageCoordinator; + import org.apache.druid.jackson.DefaultObjectMapper; ++import org.apache.druid.java.util.common.DateTimes; + import org.apache.druid.java.util.common.Intervals; ++import org.apache.druid.java.util.common.StringUtils; + import org.apache.druid.java.util.common.granularity.Granularities; ++import org.apache.druid.java.util.common.granularity.Granularity; + import org.apache.druid.java.util.metrics.StubServiceEmitter; ++import org.apache.druid.metadata.SegmentsMetadataManager; ++import org.apache.druid.metadata.SegmentsMetadataManagerConfig; ++import org.apache.druid.query.http.ClientSqlQuery; ++import org.apache.druid.query.http.SqlTaskStatus; + import org.apache.druid.segment.TestDataSource; + import org.apache.druid.segment.TestIndex; + import org.apache.druid.server.compaction.CompactionSimulateResult; +@@ -56,11 +79,15 @@ import org.apache.druid.server.coordinator.CreateDataSegments; + import org.apache.druid.server.coordinator.DataSourceCompactionConfig; + import org.apache.druid.server.coordinator.DruidCompactionConfig; + import org.apache.druid.server.coordinator.InlineSchemaDataSourceCompactionConfig; ++import org.apache.druid.server.coordinator.UserCompactionTaskGranularityConfig; ++import org.apache.druid.server.coordinator.duty.CompactSegments; + import org.apache.druid.server.coordinator.simulate.BlockingExecutorService; +-import org.apache.druid.server.coordinator.simulate.TestSegmentsMetadataManager; + import org.apache.druid.server.coordinator.simulate.WrappingScheduledExecutorService; + import org.apache.druid.server.coordinator.stats.Stats; + import org.apache.druid.timeline.DataSegment; ++import org.joda.time.DateTime; ++import org.joda.time.Duration; ++import org.joda.time.Interval; + import org.joda.time.Period; + import org.junit.Assert; + import org.junit.Before; +@@ -71,7 +98,10 @@ import org.mockito.Mockito; + + import java.util.Arrays; + import java.util.Collections; ++import java.util.HashMap; + import java.util.List; ++import java.util.Map; ++import java.util.Set; + import java.util.concurrent.atomic.AtomicReference; + + public class OverlordCompactionSchedulerTest +@@ -81,6 +111,7 @@ public class OverlordCompactionSchedulerTest + static { + OBJECT_MAPPER = new DefaultObjectMapper(); + OBJECT_MAPPER.registerModules(new IndexingServiceTuningConfigModule().getJacksonModules()); ++ OBJECT_MAPPER.registerModules(new SupervisorModule().getJacksonModules()); + OBJECT_MAPPER.setInjectableValues( + new InjectableValues + .Std() +@@ -91,25 +122,53 @@ public class OverlordCompactionSchedulerTest + ); + } + ++ private static final DateTime JAN_20 = DateTimes.of("2025-01-20"); ++ private static final DateTime MAR_11 = DateTimes.of("2025-03-11"); ++ + private AtomicReference compactionConfig; + private CoordinatorOverlordServiceConfig coordinatorOverlordServiceConfig; + + private TaskMaster taskMaster; + private TaskQueue taskQueue; ++ private BrokerClient brokerClient; ++ private TaskActionClientFactory taskActionClientFactory; + private BlockingExecutorService executor; + + private HeapMemoryTaskStorage taskStorage; +- private TestSegmentsMetadataManager segmentsMetadataManager; ++ private TestIndexerMetadataStorageCoordinator segmentStorage; ++ private SegmentsMetadataManager segmentsMetadataManager; + private StubServiceEmitter serviceEmitter; + ++ private String dataSource; + private OverlordCompactionScheduler scheduler; + ++ private Map submittedMsqTaskIds; ++ + @Before + public void setUp() + { ++ dataSource = "wiki_" + IdUtils.getRandomId(); ++ + final TaskRunner taskRunner = Mockito.mock(TaskRunner.class); ++ Mockito.when(taskRunner.getTotalCapacity()).thenReturn(100); ++ Mockito.when(taskRunner.getMaximumCapacityWithAutoscale()).thenReturn(100); ++ + taskQueue = Mockito.mock(TaskQueue.class); + ++ submittedMsqTaskIds = new HashMap<>(); ++ brokerClient = Mockito.mock(BrokerClient.class); ++ Mockito.when(brokerClient.submitSqlTask(ArgumentMatchers.any(ClientSqlQuery.class))).thenAnswer( ++ arg -> { ++ final ClientSqlQuery query = arg.getArgument(0); ++ final Interval compactionInterval = ++ (Interval) query.getContext().get(CompactSegments.COMPACTION_INTERVAL_KEY); ++ ++ final String taskId = IdUtils.getRandomId(); ++ submittedMsqTaskIds.put(compactionInterval, taskId); ++ return Futures.immediateFuture(new SqlTaskStatus(taskId, TaskState.RUNNING, null)); ++ } ++ ); ++ + taskMaster = new TaskMaster(null, null); + Assert.assertFalse(taskMaster.isHalfOrFullLeader()); + Assert.assertFalse(taskMaster.isFullLeader()); +@@ -126,11 +185,39 @@ public class OverlordCompactionSchedulerTest + + executor = new BlockingExecutorService("test"); + serviceEmitter = new StubServiceEmitter(); +- segmentsMetadataManager = new TestSegmentsMetadataManager(); ++ segmentStorage = new TestIndexerMetadataStorageCoordinator(); ++ segmentsMetadataManager = segmentStorage.getManager(); + +- compactionConfig = new AtomicReference<>(new ClusterCompactionConfig(null, null, null, true, null)); ++ compactionConfig = new AtomicReference<>(new ClusterCompactionConfig(1.0, 100, null, true, null)); + coordinatorOverlordServiceConfig = new CoordinatorOverlordServiceConfig(false, null); + ++ taskActionClientFactory = task -> new TaskActionClient() ++ { ++ @Override ++ @SuppressWarnings("unchecked") ++ public RetType submit(TaskAction taskAction) ++ { ++ if (taskAction instanceof RetrieveUsedSegmentsAction) { ++ return (RetType) segmentStorage.retrieveAllUsedSegments( ++ ((RetrieveUsedSegmentsAction) taskAction).getDataSource(), ++ Segments.ONLY_VISIBLE ++ ); ++ } else if (taskAction instanceof TimeChunkLockTryAcquireAction) { ++ final TimeChunkLockTryAcquireAction lockAcquireAction = (TimeChunkLockTryAcquireAction) taskAction; ++ return (RetType) new TimeChunkLock( ++ null, ++ task.getGroupId(), ++ task.getDataSource(), ++ lockAcquireAction.getInterval(), ++ DateTimes.nowUtc().toString(), ++ 1 ++ ); ++ } else { ++ return null; ++ } ++ } ++ }; ++ + initScheduler(); + } + +@@ -142,19 +229,29 @@ public class OverlordCompactionSchedulerTest + = new DefaultWorkerBehaviorConfig(WorkerBehaviorConfig.DEFAULT_STRATEGY, null); + scheduler = new OverlordCompactionScheduler( + taskMaster, ++ taskLockbox, + new TaskQueryTool(taskStorage, taskLockbox, taskMaster, null, () -> defaultWorkerConfig), + segmentsMetadataManager, ++ new SegmentsMetadataManagerConfig(null, null, null), + () -> DruidCompactionConfig.empty().withClusterConfig(compactionConfig.get()), +- new CompactionStatusTracker(OBJECT_MAPPER), ++ new CompactionStatusTracker(), + coordinatorOverlordServiceConfig, ++ taskActionClientFactory, ++ new DruidInputSourceFactory( ++ TestIndex.INDEX_IO, ++ Mockito.mock(TaskConfig.class), ++ new NoopCoordinatorClient(), ++ new SegmentCacheManagerFactory(TestIndex.INDEX_IO, OBJECT_MAPPER) ++ ), + (nameFormat, numThreads) -> new WrappingScheduledExecutorService("test", executor, false), ++ brokerClient, + serviceEmitter, + OBJECT_MAPPER + ); + } + + @Test +- public void testBecomeLeader_triggersStart_ifEnabled() ++ public void test_becomeLeader_triggersStart_ifEnabled() + { + Assert.assertTrue(scheduler.isEnabled()); + +@@ -168,7 +265,7 @@ public class OverlordCompactionSchedulerTest + } + + @Test +- public void testBecomeLeader_doesNotTriggerStart_ifDisabled() ++ public void test_becomeLeader_doesNotTriggerStart_ifDisabled() + { + disableScheduler(); + Assert.assertFalse(scheduler.isEnabled()); +@@ -182,7 +279,7 @@ public class OverlordCompactionSchedulerTest + } + + @Test +- public void testStopBeingLeader_triggersStop() ++ public void test_stopBeingLeader_triggersStop() + { + Assert.assertFalse(scheduler.isRunning()); + +@@ -198,7 +295,7 @@ public class OverlordCompactionSchedulerTest + } + + @Test +- public void testDisablingScheduler_triggersStop() ++ public void test_disableSupervisors_triggersStop() + { + // Start scheduler + scheduler.becomeLeader(); +@@ -216,7 +313,7 @@ public class OverlordCompactionSchedulerTest + } + + @Test +- public void testEnablingScheduler_triggersStart() ++ public void test_enableSupervisors_triggersStart() + { + disableScheduler(); + +@@ -235,7 +332,7 @@ public class OverlordCompactionSchedulerTest + } + + @Test +- public void testSegmentsAreNotPolled_ifSupervisorsAreDisabled() ++ public void test_disableSupervisors_disablesSegmentPolling() + { + disableScheduler(); + +@@ -243,7 +340,7 @@ public class OverlordCompactionSchedulerTest + } + + @Test +- public void testSegmentsArePolled_whenRunningInStandaloneMode() ++ public void test_enableSupervisors_inStandaloneMode_enablesSegmentPolling() + { + coordinatorOverlordServiceConfig = new CoordinatorOverlordServiceConfig(false, null); + initScheduler(); +@@ -252,7 +349,7 @@ public class OverlordCompactionSchedulerTest + } + + @Test +- public void testSegmentsAreNotPolled_whenRunningInCoordinatorMode() ++ public void test_enableSupervisors_inCoordinatorMode_disablesSegmentPolling() + { + coordinatorOverlordServiceConfig = new CoordinatorOverlordServiceConfig(true, "overlord"); + initScheduler(); +@@ -272,7 +369,7 @@ public class OverlordCompactionSchedulerTest + } + + @Test +- public void testNullCompactionConfigIsInvalid() ++ public void test_validateCompactionConfig_returnsInvalid_forNullConfig() + { + final CompactionConfigValidationResult result = scheduler.validateCompactionConfig(null); + Assert.assertFalse(result.isValid()); +@@ -280,11 +377,11 @@ public class OverlordCompactionSchedulerTest + } + + @Test +- public void testMsqCompactionConfigWithOneMaxTasksIsInvalid() ++ public void test_validateCompactionConfig_returnsInvalid_forMSQConfigWithOneMaxTasks() + { + final DataSourceCompactionConfig datasourceConfig = InlineSchemaDataSourceCompactionConfig + .builder() +- .forDataSource(TestDataSource.WIKI) ++ .forDataSource(dataSource) + .withEngine(CompactionEngine.MSQ) + .withTaskContext(Collections.singletonMap(ClientMSQContext.CTX_MAX_NUM_TASKS, 1)) + .build(); +@@ -298,75 +395,50 @@ public class OverlordCompactionSchedulerTest + } + + @Test +- public void testStartCompaction() ++ public void test_startCompaction_enablesTaskSubmission_forDatasource() + { +- final List wikiSegments = CreateDataSegments.ofDatasource(TestDataSource.WIKI).eachOfSizeInMb(100); +- wikiSegments.forEach(segmentsMetadataManager::addSegment); ++ createSegments(1, Granularities.DAY, JAN_20); + + scheduler.becomeLeader(); +- scheduler.startCompaction( +- TestDataSource.WIKI, +- InlineSchemaDataSourceCompactionConfig.builder() +- .forDataSource(TestDataSource.WIKI) +- .withSkipOffsetFromLatest(Period.seconds(0)) +- .build() +- ); +- +- executor.finishNextPendingTask(); ++ scheduler.startCompaction(dataSource, createSupervisorWithInlineSpec()); + +- ArgumentCaptor taskArgumentCaptor = ArgumentCaptor.forClass(Task.class); +- Mockito.verify(taskQueue, Mockito.times(1)).add(taskArgumentCaptor.capture()); +- +- Task submittedTask = taskArgumentCaptor.getValue(); +- Assert.assertNotNull(submittedTask); +- Assert.assertTrue(submittedTask instanceof CompactionTask); +- +- final CompactionTask compactionTask = (CompactionTask) submittedTask; +- Assert.assertEquals(TestDataSource.WIKI, compactionTask.getDataSource()); ++ runCompactionTasks(1); + +- final AutoCompactionSnapshot.Builder expectedSnapshot = AutoCompactionSnapshot.builder(TestDataSource.WIKI); +- expectedSnapshot.incrementCompactedStats(CompactionStatistics.create(100_000_000, 1, 1)); ++ final AutoCompactionSnapshot.Builder expectedSnapshot = AutoCompactionSnapshot.builder(dataSource); ++ expectedSnapshot.incrementWaitingStats(CompactionStatistics.create(100_000_000, 1, 1)); + + Assert.assertEquals( + expectedSnapshot.build(), +- scheduler.getCompactionSnapshot(TestDataSource.WIKI) ++ scheduler.getCompactionSnapshot(dataSource) + ); + Assert.assertEquals( +- Collections.singletonMap(TestDataSource.WIKI, expectedSnapshot.build()), ++ Map.of(dataSource, expectedSnapshot.build()), + scheduler.getAllCompactionSnapshots() + ); + + serviceEmitter.verifyValue(Stats.Compaction.SUBMITTED_TASKS.getMetricName(), 1L); +- serviceEmitter.verifyValue(Stats.Compaction.COMPACTED_BYTES.getMetricName(), 100_000_000L); ++ serviceEmitter.verifyValue(Stats.Compaction.PENDING_BYTES.getMetricName(), 100_000_000L); + + scheduler.stopBeingLeader(); + } + + @Test +- public void testStopCompaction() ++ public void test_stopCompaction_disablesTaskSubmission_forDatasource() + { +- final List wikiSegments = CreateDataSegments.ofDatasource(TestDataSource.WIKI).eachOfSizeInMb(100); +- wikiSegments.forEach(segmentsMetadataManager::addSegment); ++ createSegments(1, Granularities.DAY, JAN_20); + + scheduler.becomeLeader(); +- scheduler.startCompaction( +- TestDataSource.WIKI, +- InlineSchemaDataSourceCompactionConfig.builder() +- .forDataSource(TestDataSource.WIKI) +- .withSkipOffsetFromLatest(Period.seconds(0)) +- .build() +- ); +- scheduler.stopCompaction(TestDataSource.WIKI); +- +- executor.finishNextPendingTask(); ++ scheduler.startCompaction(dataSource, createSupervisorWithInlineSpec()); ++ scheduler.stopCompaction(dataSource); + ++ runScheduledJob(); + Mockito.verify(taskQueue, Mockito.never()).add(ArgumentMatchers.any()); + + Assert.assertEquals( +- AutoCompactionSnapshot.builder(TestDataSource.WIKI) ++ AutoCompactionSnapshot.builder(dataSource) + .withStatus(AutoCompactionSnapshot.ScheduleStatus.NOT_ENABLED) + .build(), +- scheduler.getCompactionSnapshot(TestDataSource.WIKI) ++ scheduler.getCompactionSnapshot(dataSource) + ); + Assert.assertTrue(scheduler.getAllCompactionSnapshots().isEmpty()); + +@@ -377,26 +449,14 @@ public class OverlordCompactionSchedulerTest + } + + @Test +- public void testSimulateRun() ++ public void test_simulateRunWithConfigUpdate() + { +- final List wikiSegments = CreateDataSegments +- .ofDatasource(TestDataSource.WIKI) +- .forIntervals(1, Granularities.DAY) +- .startingAt("2013-01-01") +- .withNumPartitions(10) +- .eachOfSizeInMb(100); +- wikiSegments.forEach(segmentsMetadataManager::addSegment); ++ createSegments(1, Granularities.DAY, DateTimes.of("2013-01-01")); + + scheduler.becomeLeader(); + runScheduledJob(); + +- scheduler.startCompaction( +- TestDataSource.WIKI, +- InlineSchemaDataSourceCompactionConfig.builder() +- .forDataSource(TestDataSource.WIKI) +- .withSkipOffsetFromLatest(Period.seconds(0)) +- .build() +- ); ++ scheduler.startCompaction(dataSource, createSupervisorWithInlineSpec()); + + final CompactionSimulateResult simulateResult = scheduler.simulateRunWithConfigUpdate( + new ClusterCompactionConfig(null, null, null, null, null) +@@ -410,10 +470,10 @@ public class OverlordCompactionSchedulerTest + Assert.assertEquals( + Collections.singletonList( + Arrays.asList( +- TestDataSource.WIKI, ++ dataSource, + Intervals.of("2013-01-01/P1D"), +- 10, +- 1_000_000_000L, ++ 1, ++ 100_000_000L, + 1, + "not compacted yet" + ) +@@ -421,7 +481,7 @@ public class OverlordCompactionSchedulerTest + pendingCompactionTable.getRows() + ); + +- scheduler.stopCompaction(TestDataSource.WIKI); ++ scheduler.stopCompaction(dataSource); + + final CompactionSimulateResult simulateResultWhenDisabled = scheduler.simulateRunWithConfigUpdate( + new ClusterCompactionConfig(null, null, null, null, null) +@@ -431,6 +491,255 @@ public class OverlordCompactionSchedulerTest + scheduler.stopBeingLeader(); + } + ++ @Test ++ public void test_ingestHourGranularity_andCompactToDayAndMonth_withInlineTemplates() ++ { ++ final int numDays = (int) new Duration(MAR_11.getMillis() - JAN_20.getMillis()).getStandardDays(); ++ createSegments(24 * numDays, Granularities.HOUR, JAN_20); ++ verifyNumSegmentsWith(Granularities.HOUR, 24 * numDays); ++ ++ // Compact everything going back to Mar 10 to DAY granularity, rest to MONTH ++ final DateTime now = DateTimes.nowUtc(); ++ final Period dayRulePeriod = new Period(now.getMillis() - MAR_11.minusDays(1).minusMinutes(1).getMillis()); ++ CascadingCompactionTemplate cascadingTemplate = new CascadingCompactionTemplate( ++ dataSource, ++ List.of( ++ new CompactionRule(dayRulePeriod, new InlineCompactionJobTemplate(createMatcher(Granularities.DAY))), ++ new CompactionRule(Period.ZERO, new InlineCompactionJobTemplate(createMatcher(Granularities.MONTH))) ++ ) ++ ); ++ ++ startCompactionWithSpec(cascadingTemplate); ++ runCompactionTasks(12); ++ ++ verifyFullyCompacted(); ++ verifyNumSegmentsWith(Granularities.HOUR, 0); ++ verifyNumSegmentsWith(Granularities.DAY, 10); ++ verifyNumSegmentsWith(Granularities.MONTH, 2); ++ } ++ ++ @Test ++ public void test_ingestHourGranularity_andCompactToDayAndMonth_withCatalogTemplates() ++ { ++ final int numDays = (int) new Duration(MAR_11.getMillis() - JAN_20.getMillis()).getStandardDays(); ++ createSegments(24 * numDays, Granularities.HOUR, JAN_20); ++ verifyNumSegmentsWith(Granularities.HOUR, 24 * numDays); ++ ++ // Add compaction templates to catalog ++ final CompactionJobTemplate dayGranularityTemplate = ++ new InlineCompactionJobTemplate(createMatcher(Granularities.DAY)); ++ final CompactionJobTemplate monthGranularityTemplate = ++ new InlineCompactionJobTemplate(createMatcher(Granularities.MONTH)); ++ ++ // Compact everything going back to Mar 10 to DAY granularity, rest to MONTH ++ final DateTime now = DateTimes.nowUtc(); ++ final Period dayRulePeriod = new Period(now.getMillis() - MAR_11.minusDays(1).minusMinutes(1).getMillis()); ++ CascadingCompactionTemplate cascadingTemplate = new CascadingCompactionTemplate( ++ dataSource, ++ List.of( ++ new CompactionRule(dayRulePeriod, dayGranularityTemplate), ++ new CompactionRule(Period.ZERO, monthGranularityTemplate) ++ ) ++ ); ++ ++ startCompactionWithSpec(cascadingTemplate); ++ runCompactionTasks(12); ++ ++ verifyFullyCompacted(); ++ verifyNumSegmentsWith(Granularities.HOUR, 0); ++ verifyNumSegmentsWith(Granularities.DAY, 10); ++ verifyNumSegmentsWith(Granularities.MONTH, 2); ++ } ++ ++ @Test ++ public void test_ingestHourGranularity_andCompactToDayAndMonth_withCatalogMSQTemplates() ++ { ++ dataSource = TestDataSource.WIKI; ++ ++ final int numDays = (int) new Duration(MAR_11.getMillis() - JAN_20.getMillis()).getStandardDays(); ++ createSegments(24 * numDays, Granularities.HOUR, JAN_20); ++ verifyNumSegmentsWith(Granularities.HOUR, 24 * numDays); ++ ++ // Add compaction templates to catalog ++ final String sqlDayGranularity = ++ "REPLACE INTO ${dataSource}" ++ + " OVERWRITE WHERE __time >= TIMESTAMP '${startTimestamp}' AND __time < TIMESTAMP '${endTimestamp}'" ++ + " SELECT * FROM ${dataSource}" ++ + " WHERE __time BETWEEN '${startTimestamp}' AND '${endTimestamp}'" ++ + " PARTITIONED BY DAY"; ++ final CompactionJobTemplate dayGranularityTemplate = new MSQCompactionJobTemplate( ++ new ClientSqlQuery(sqlDayGranularity, null, false, false, false, null, null), ++ createMatcher(Granularities.DAY) ++ ); ++ final String sqlMonthGranularity = ++ "REPLACE INTO ${dataSource}" ++ + " OVERWRITE WHERE __time >= TIMESTAMP '${startTimestamp}' AND __time < TIMESTAMP '${endTimestamp}'" ++ + " SELECT * FROM ${dataSource}" ++ + " WHERE __time BETWEEN '${startTimestamp}' AND '${endTimestamp}'" ++ + " PARTITIONED BY MONTH"; ++ final CompactionJobTemplate monthGranularityTemplate = new MSQCompactionJobTemplate( ++ new ClientSqlQuery(sqlMonthGranularity, null, false, false, false, null, null), ++ createMatcher(Granularities.MONTH) ++ ); ++ ++ // Compact everything going back to Mar 10 to DAY granularity, rest to MONTH ++ final DateTime now = DateTimes.nowUtc(); ++ final Period dayRulePeriod = new Period(now.getMillis() - MAR_11.minusDays(1).minusMinutes(1).getMillis()); ++ CascadingCompactionTemplate cascadingTemplate = new CascadingCompactionTemplate( ++ dataSource, ++ List.of( ++ new CompactionRule(dayRulePeriod, dayGranularityTemplate), ++ new CompactionRule(Period.ZERO, monthGranularityTemplate) ++ ) ++ ); ++ ++ startCompactionWithSpec(cascadingTemplate); ++ runMSQCompactionJobs(12); ++ ++ verifyFullyCompacted(); ++ verifyNumSegmentsWith(Granularities.HOUR, 0); ++ verifyNumSegmentsWith(Granularities.DAY, 10); ++ verifyNumSegmentsWith(Granularities.MONTH, 2); ++ } ++ ++ private void verifyNumSegmentsWith(Granularity granularity, int numExpectedSegments) ++ { ++ long numMatchingSegments = segmentStorage ++ .retrieveAllUsedSegments(dataSource, Segments.ONLY_VISIBLE) ++ .stream() ++ .filter(segment -> granularity.isAligned(segment.getInterval())) ++ .count(); ++ ++ Assert.assertEquals( ++ StringUtils.format("Segment with granularity[%s]", granularity), ++ numExpectedSegments, ++ (int) numMatchingSegments ++ ); ++ } ++ ++ private void verifyFullyCompacted() ++ { ++ runScheduledJob(); ++ int numSegments = segmentStorage.retrieveAllUsedSegments(dataSource, Segments.ONLY_VISIBLE).size(); ++ ++ final AutoCompactionSnapshot snapshot = scheduler.getCompactionSnapshot(dataSource); ++ Assert.assertEquals(0, snapshot.getSegmentCountAwaitingCompaction()); ++ Assert.assertEquals(0, snapshot.getSegmentCountSkipped()); ++ Assert.assertEquals(numSegments, snapshot.getSegmentCountCompacted()); ++ } ++ ++ private void createSegments(int numSegments, Granularity granularity, DateTime firstSegmentStart) ++ { ++ final List segments = CreateDataSegments ++ .ofDatasource(dataSource) ++ .forIntervals(numSegments, granularity) ++ .startingAt(firstSegmentStart) ++ .eachOfSizeInMb(100); ++ segmentStorage.commitSegments(Set.copyOf(segments), null); ++ } ++ ++ private void startCompactionWithSpec(DataSourceCompactionConfig config) ++ { ++ scheduler.becomeLeader(); ++ final CompactionSupervisorSpec compactionSupervisor ++ = new CompactionSupervisorSpec(config, false, scheduler); ++ scheduler.startCompaction(config.getDataSource(), compactionSupervisor.createSupervisor()); ++ } ++ ++ private void runCompactionTasks(int expectedCount) ++ { ++ runScheduledJob(); ++ serviceEmitter.verifySum("compact/task/count", expectedCount); ++ ++ ArgumentCaptor taskArgumentCaptor = ArgumentCaptor.forClass(Task.class); ++ Mockito.verify(taskQueue, Mockito.times(expectedCount)).add(taskArgumentCaptor.capture()); ++ ++ for (Task task : taskArgumentCaptor.getAllValues()) { ++ Assert.assertTrue(task instanceof CompactionTask); ++ Assert.assertEquals(dataSource, task.getDataSource()); ++ ++ final CompactionTask compactionTask = (CompactionTask) task; ++ runCompactionTask( ++ compactionTask.getId(), ++ compactionTask.getIoConfig().getInputSpec().findInterval(dataSource), ++ compactionTask.getSegmentGranularity() ++ ); ++ } ++ ++ segmentStorage.getManager().forceUpdateDataSourcesSnapshot(); ++ } ++ ++ private void runCompactionTask(String taskId, Interval compactionInterval, Granularity segmentGranularity) ++ { ++ // Update status of task in TaskQueue ++ Mockito.when(taskQueue.getTaskStatus(taskId)) ++ .thenReturn(Optional.of(TaskStatus.success(taskId))); ++ ++ // Determine interval and granularity and apply it to the timeline ++ if (segmentGranularity == null) { ++ // Nothing to do ++ return; ++ } ++ ++ for (Interval replaceInterval : segmentGranularity.getIterable(compactionInterval)) { ++ // Create a single segment in this interval ++ DataSegment replaceSegment = CreateDataSegments ++ .ofDatasource(dataSource) ++ .forIntervals(1, segmentGranularity) ++ .startingAt(replaceInterval.getStart()) ++ .withVersion("2") ++ .eachOfSizeInMb(100) ++ .get(0); ++ segmentStorage.commitSegments(Set.of(replaceSegment), null); ++ } ++ } ++ ++ private void runMSQCompactionJobs(int numExpectedJobs) ++ { ++ runScheduledJob(); ++ serviceEmitter.verifySum("compact/task/count", numExpectedJobs); ++ ++ ArgumentCaptor queryArgumentCaptor = ArgumentCaptor.forClass(ClientSqlQuery.class); ++ Mockito.verify(brokerClient, Mockito.times(numExpectedJobs)) ++ .submitSqlTask(queryArgumentCaptor.capture()); ++ ++ for (ClientSqlQuery job : queryArgumentCaptor.getAllValues()) { ++ final String query = job.getQuery(); ++ ++ final Granularity segmentGranularity; ++ if (query.contains("PARTITIONED BY DAY")) { ++ segmentGranularity = Granularities.DAY; ++ } else if (query.contains("PARTITIONED BY MONTH")) { ++ segmentGranularity = Granularities.MONTH; ++ } else { ++ segmentGranularity = Granularities.HOUR; ++ } ++ ++ final Interval compactionInterval = ++ (Interval) job.getContext().get(CompactSegments.COMPACTION_INTERVAL_KEY); ++ runCompactionTask( ++ submittedMsqTaskIds.get(compactionInterval), ++ compactionInterval, ++ segmentGranularity ++ ); ++ } ++ ++ segmentStorage.getManager().forceUpdateDataSourcesSnapshot(); ++ } ++ ++ private static CompactionStateMatcher createMatcher(Granularity segmentGranularity) ++ { ++ return new CompactionStateMatcher( ++ null, ++ null, ++ null, ++ null, ++ null, ++ new UserCompactionTaskGranularityConfig(segmentGranularity, null, null), ++ null ++ ); ++ } ++ + private void disableScheduler() + { + compactionConfig.set(new ClusterCompactionConfig(null, null, null, false, null)); +@@ -446,4 +755,16 @@ public class OverlordCompactionSchedulerTest + executor.finishNextPendingTask(); + } + ++ private CompactionSupervisor createSupervisorWithInlineSpec() ++ { ++ return new CompactionSupervisorSpec( ++ InlineSchemaDataSourceCompactionConfig ++ .builder() ++ .forDataSource(dataSource) ++ .withSkipOffsetFromLatest(Period.seconds(0)) ++ .build(), ++ false, ++ scheduler ++ ).createSupervisor(); ++ } + } +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 e7ecc49df8..ee9e85204e 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 +@@ -27,9 +27,11 @@ import org.apache.druid.indexing.overlord.SegmentCreateRequest; + import org.apache.druid.indexing.overlord.SegmentPublishResult; + import org.apache.druid.indexing.overlord.Segments; + import org.apache.druid.jackson.DefaultObjectMapper; ++import org.apache.druid.java.util.common.Intervals; + import org.apache.druid.java.util.common.Pair; + import org.apache.druid.metadata.PendingSegmentRecord; + import org.apache.druid.metadata.ReplaceTaskLock; ++import org.apache.druid.metadata.SegmentsMetadataManager; + import org.apache.druid.metadata.SortOrder; + import org.apache.druid.segment.SegmentSchemaMapping; + import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; +@@ -56,6 +58,11 @@ public class TestIndexerMetadataStorageCoordinator implements IndexerMetadataSto + + private int deleteSegmentsCount = 0; + ++ public SegmentsMetadataManager getManager() ++ { ++ return segmentsMetadataManager; ++ } ++ + @Override + public Set retrieveAllDatasourceNames() + { +@@ -106,9 +113,15 @@ public class TestIndexerMetadataStorageCoordinator implements IndexerMetadataSto + @Override + public Set retrieveAllUsedSegments(String dataSource, Segments visibility) + { +- return Set.copyOf( +- segmentsMetadataManager.getRecentDataSourcesSnapshot().getDataSource(dataSource).getSegments() +- ); ++ if (visibility == Segments.ONLY_VISIBLE) { ++ return segmentsMetadataManager ++ .getRecentDataSourcesSnapshot() ++ .getAllUsedNonOvershadowedSegments(dataSource, Intervals.ETERNITY); ++ } else { ++ return Set.copyOf( ++ segmentsMetadataManager.getRecentDataSourcesSnapshot().getDataSource(dataSource).getSegments() ++ ); ++ } + } + + @Override +diff --git a/processing/src/main/java/org/apache/druid/java/util/common/Intervals.java b/processing/src/main/java/org/apache/druid/java/util/common/Intervals.java +index 0960f63b49..ce34fc4798 100644 +--- a/processing/src/main/java/org/apache/druid/java/util/common/Intervals.java ++++ b/processing/src/main/java/org/apache/druid/java/util/common/Intervals.java +@@ -29,6 +29,7 @@ import org.joda.time.format.DateTimeFormatter; + import org.joda.time.format.ISODateTimeFormat; + + import javax.annotation.Nullable; ++import java.util.List; + + public final class Intervals + { +@@ -167,6 +168,26 @@ public final class Intervals + return null; + } + ++ /** ++ * @return List of intervals which when added to the given interval create ++ * {@link Intervals#ETERNITY}. ++ */ ++ public static List complementOf(Interval interval) ++ { ++ if (isEternity(interval)) { ++ return List.of(); ++ } else if (DateTimes.MIN.equals(interval.getStart())) { ++ return List.of(new Interval(interval.getEnd(), DateTimes.MAX)); ++ } else if (DateTimes.MAX.equals(interval.getEnd())) { ++ return List.of(new Interval(DateTimes.MIN, interval.getStart())); ++ } else { ++ return List.of( ++ new Interval(DateTimes.MIN, interval.getStart()), ++ new Interval(interval.getEnd(), DateTimes.MAX) ++ ); ++ } ++ } ++ + private Intervals() + { + } +diff --git a/processing/src/main/java/org/apache/druid/query/http/ClientSqlQuery.java b/processing/src/main/java/org/apache/druid/query/http/ClientSqlQuery.java +index 3b3a2a2398..663714b4d9 100644 +--- a/processing/src/main/java/org/apache/druid/query/http/ClientSqlQuery.java ++++ b/processing/src/main/java/org/apache/druid/query/http/ClientSqlQuery.java +@@ -109,7 +109,6 @@ public class ClientSqlQuery + return parameters; + } + +- + @Override + public boolean equals(final Object o) + { +diff --git a/processing/src/test/java/org/apache/druid/java/util/common/IntervalsTest.java b/processing/src/test/java/org/apache/druid/java/util/common/IntervalsTest.java +index 3d591a319f..de0a27bcb5 100644 +--- a/processing/src/test/java/org/apache/druid/java/util/common/IntervalsTest.java ++++ b/processing/src/test/java/org/apache/druid/java/util/common/IntervalsTest.java +@@ -26,6 +26,7 @@ import org.junit.Assert; + import org.junit.Test; + + import java.util.Arrays; ++import java.util.List; + + public class IntervalsTest + { +@@ -124,4 +125,36 @@ public class IntervalsTest + () -> Intervals.of("invalid string") + ); + } ++ ++ @Test ++ public void testComplementOf() ++ { ++ Assert.assertEquals( ++ List.of(), ++ Intervals.complementOf(Intervals.ETERNITY) ++ ); ++ ++ testComplementOf("2020/P1Y"); ++ testComplementOf("2001/2001-01"); ++ testComplementOf("2001-01-02/2001-02"); ++ } ++ ++ private void testComplementOf(String interval) ++ { ++ final Interval testInterval = Intervals.of(interval); ++ final List complement = List.of( ++ new Interval(DateTimes.MIN, testInterval.getStart()), ++ new Interval(testInterval.getEnd(), DateTimes.MAX) ++ ); ++ Assert.assertEquals( ++ complement, ++ Intervals.complementOf(testInterval) ++ ); ++ Assert.assertEquals( ++ Intervals.ONLY_ETERNITY, ++ JodaUtils.condenseIntervals( ++ List.of(complement.get(0), complement.get(1), testInterval) ++ ) ++ ); ++ } + } +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 7ff9ff424f..7a7f655723 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 +@@ -22,67 +22,24 @@ package org.apache.druid.client.indexing; + import com.fasterxml.jackson.annotation.JsonCreator; + import com.fasterxml.jackson.annotation.JsonProperty; + import org.apache.druid.java.util.common.IAE; +-import org.apache.druid.java.util.common.JodaUtils; +-import org.apache.druid.java.util.common.granularity.Granularity; +-import org.apache.druid.java.util.common.logger.Logger; +-import org.apache.druid.timeline.DataSegment; + 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. + */ + public class ClientCompactionIntervalSpec + { +- private static final Logger LOGGER = new Logger(ClientCompactionIntervalSpec.class); +- + private static final String TYPE = "interval"; + + private final Interval interval; + @Nullable + private final String sha256OfSortedSegmentIds; + +- public static ClientCompactionIntervalSpec fromSegments(List segments, @Nullable Granularity segmentGranularity) +- { +- Interval interval = JodaUtils.umbrellaInterval(segments.stream().map(DataSegment::getInterval).collect(Collectors.toList())); +- LOGGER.info("Original umbrella interval %s in compaction task for datasource %s", interval, segments.get(0).getDataSource()); +- if (segmentGranularity != null) { +- // If segmentGranularity is set, then the segmentGranularity of the segments may not align with the configured segmentGranularity +- // We must adjust the interval of the compaction task to fully cover and align with the segmentGranularity +- // For example, +- // - The umbrella interval of the segments is 2015-04-11/2015-04-12 but configured segmentGranularity is YEAR, +- // if the compaction task's interval is 2015-04-11/2015-04-12 then we can run into race condition where after submitting +- // the compaction task, a new segment outside of the interval (i.e. 2015-02-11/2015-02-12) got created will be lost as it is +- // overshadowed by the compacted segment (compacted segment has interval 2015-01-01/2016-01-01. +- // Hence, in this case, we must adjust the compaction task interval to 2015-01-01/2016-01-01. +- // - The segment to be compacted has MONTH segmentGranularity with the interval 2015-02-01/2015-03-01 but configured +- // segmentGranularity is WEEK. If the compaction task's interval is 2015-02-01/2015-03-01 then compacted segments created will be +- // 2015-01-26/2015-02-02, 2015-02-02/2015-02-09, 2015-02-09/2015-02-16, 2015-02-16/2015-02-23, 2015-02-23/2015-03-02. +- // This is because Druid's WEEK segments alway start and end on Monday. In the above example, 2015-01-26 and 2015-03-02 +- // are Mondays but 2015-02-01 and 2015-03-01 are not. Hence, the WEEK segments have to start and end on 2015-01-26 and 2015-03-02. +- // If the compaction task's interval is 2015-02-01/2015-03-01, then the compacted segment would cause existing data +- // from 2015-01-26 to 2015-02-01 and 2015-03-01 to 2015-03-02 to be lost. Hence, in this case, +- // we must adjust the compaction task interval to 2015-01-26/2015-03-02 +- interval = JodaUtils.umbrellaInterval(segmentGranularity.getIterable(interval)); +- LOGGER.info( +- "Interval adjusted to %s in compaction task for datasource %s with configured segmentGranularity %s", +- interval, +- segments.get(0).getDataSource(), +- segmentGranularity +- ); +- } +- return new ClientCompactionIntervalSpec( +- interval, +- null +- ); +- } +- + @JsonCreator + public ClientCompactionIntervalSpec( + @JsonProperty("interval") Interval interval, +diff --git a/server/src/main/java/org/apache/druid/indexing/template/BatchIndexingJob.java b/server/src/main/java/org/apache/druid/indexing/template/BatchIndexingJob.java +new file mode 100644 +index 0000000000..509226da10 +--- /dev/null ++++ b/server/src/main/java/org/apache/druid/indexing/template/BatchIndexingJob.java +@@ -0,0 +1,90 @@ ++/* ++ * Licensed to the Apache Software Foundation (ASF) under one ++ * or more contributor license agreements. See the NOTICE file ++ * distributed with this work for additional information ++ * regarding copyright ownership. The ASF licenses this file ++ * to you under the Apache License, Version 2.0 (the ++ * "License"); you may not use this file except in compliance ++ * with the License. You may obtain a copy of the License at ++ * ++ * http://www.apache.org/licenses/LICENSE-2.0 ++ * ++ * Unless required by applicable law or agreed to in writing, ++ * software distributed under the License is distributed on an ++ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY ++ * KIND, either express or implied. See the License for the ++ * specific language governing permissions and limitations ++ * under the License. ++ */ ++ ++package org.apache.druid.indexing.template; ++ ++import org.apache.druid.client.indexing.ClientTaskQuery; ++import org.apache.druid.error.InvalidInput; ++import org.apache.druid.query.http.ClientSqlQuery; ++ ++import javax.annotation.Nullable; ++import java.util.Objects; ++ ++/** ++ * A batch indexing job that can be launched by the Overlord as a task. ++ * A job may contain the {@link ClientTaskQuery} itself or an MSQ query that gets converted ++ * by the Broker to a {@code ControllerTask} and is then submitted to the Overlord. ++ */ ++public class BatchIndexingJob ++{ ++ private final boolean isMsq; ++ private final ClientSqlQuery msqQuery; ++ private final ClientTaskQuery task; ++ ++ protected BatchIndexingJob( ++ @Nullable ClientTaskQuery task, ++ @Nullable ClientSqlQuery msqQuery ++ ) ++ { ++ this.isMsq = task == null; ++ this.msqQuery = msqQuery; ++ this.task = task; ++ ++ InvalidInput.conditionalException( ++ (task == null || msqQuery == null) && (task != null || msqQuery != null), ++ "Exactly one of 'task' or 'msqQuery' must be non-null" ++ ); ++ } ++ ++ /** ++ * @return MSQ query to be run in this job, if any. ++ * @throws NullPointerException if this not an MSQ job. ++ */ ++ public ClientSqlQuery getNonNullMsqQuery() ++ { ++ return Objects.requireNonNull(msqQuery); ++ } ++ ++ /** ++ * @return Task to be run in this job, if any. ++ * @throws NullPointerException if this is an MSQ job. ++ */ ++ public ClientTaskQuery getNonNullTask() ++ { ++ return Objects.requireNonNull(task); ++ } ++ ++ /** ++ * @return true if this is an MSQ job. ++ */ ++ public boolean isMsq() ++ { ++ return isMsq; ++ } ++ ++ @Override ++ public String toString() ++ { ++ return "BatchIndexingJob{" + ++ "isMsq=" + isMsq + ++ ", msqQuery=" + msqQuery + ++ ", task=" + task + ++ '}'; ++ } ++} +diff --git a/server/src/main/java/org/apache/druid/indexing/template/BatchIndexingJobTemplate.java b/server/src/main/java/org/apache/druid/indexing/template/BatchIndexingJobTemplate.java +new file mode 100644 +index 0000000000..e8a09a6bb1 +--- /dev/null ++++ b/server/src/main/java/org/apache/druid/indexing/template/BatchIndexingJobTemplate.java +@@ -0,0 +1,41 @@ ++/* ++ * 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.template; ++ ++import com.fasterxml.jackson.annotation.JsonProperty; ++import com.fasterxml.jackson.annotation.JsonTypeInfo; ++import org.apache.druid.data.input.InputSource; ++ ++/** ++ * ETL template to create a {@link BatchIndexingJob} that indexes data from an ++ * {@link InputSource} into an output destination. ++ *

++ * This interface currently defines no methods and is used only as the root type ++ * for serialization of compaction template objects. ++ */ ++@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type") ++public interface BatchIndexingJobTemplate ++{ ++ /** ++ * Unique type name of this template used for JSON serialization. ++ */ ++ @JsonProperty ++ String getType(); ++} +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 4cd9b22df8..f936f3d49a 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 +@@ -21,12 +21,14 @@ package org.apache.druid.server.compaction; + + import org.apache.druid.error.InvalidInput; + import org.apache.druid.java.util.common.JodaUtils; ++import org.apache.druid.java.util.common.granularity.Granularity; + import org.apache.druid.segment.SegmentUtils; + import org.apache.druid.timeline.DataSegment; + import org.joda.time.Interval; + + import javax.annotation.Nullable; + import java.util.List; ++import java.util.Set; + import java.util.stream.Collectors; + + /** +@@ -37,29 +39,54 @@ public class CompactionCandidate + { + private final List segments; + private final Interval umbrellaInterval; ++ private final Interval compactionInterval; + private final String dataSource; + private final long totalBytes; + private final int numIntervals; + + private final CompactionStatus currentStatus; + +- public static CompactionCandidate from(List segments) ++ public static CompactionCandidate from( ++ List segments, ++ @Nullable Granularity targetSegmentGranularity ++ ) + { + if (segments == null || segments.isEmpty()) { + throw InvalidInput.exception("Segments to compact must be non-empty"); +- } else { +- return new CompactionCandidate(segments, null); + } ++ ++ final Set segmentIntervals = ++ segments.stream().map(DataSegment::getInterval).collect(Collectors.toSet()); ++ final Interval umbrellaInterval = JodaUtils.umbrellaInterval(segmentIntervals); ++ final Interval compactionInterval = ++ targetSegmentGranularity == null ++ ? umbrellaInterval ++ : JodaUtils.umbrellaInterval(targetSegmentGranularity.getIterable(umbrellaInterval)); ++ ++ return new CompactionCandidate( ++ segments, ++ umbrellaInterval, ++ compactionInterval, ++ segmentIntervals.size(), ++ null ++ ); + } + +- private CompactionCandidate(List segments, @Nullable CompactionStatus currentStatus) ++ private CompactionCandidate( ++ List segments, ++ Interval umbrellaInterval, ++ Interval compactionInterval, ++ int numDistinctSegmentIntervals, ++ @Nullable CompactionStatus currentStatus ++ ) + { + this.segments = segments; + this.totalBytes = segments.stream().mapToLong(DataSegment::getSize).sum(); +- this.umbrellaInterval = JodaUtils.umbrellaInterval( +- segments.stream().map(DataSegment::getInterval).collect(Collectors.toList()) +- ); +- this.numIntervals = (int) segments.stream().map(DataSegment::getInterval).distinct().count(); ++ ++ this.umbrellaInterval = umbrellaInterval; ++ this.compactionInterval = compactionInterval; ++ ++ this.numIntervals = numDistinctSegmentIntervals; + this.dataSource = segments.get(0).getDataSource(); + this.currentStatus = currentStatus; + } +@@ -91,6 +118,15 @@ public class CompactionCandidate + return umbrellaInterval; + } + ++ /** ++ * Interval aligned to the target segment granularity used for the compaction ++ * task. This interval completely contains the {@link #umbrellaInterval}. ++ */ ++ public Interval getCompactionInterval() ++ { ++ return compactionInterval; ++ } ++ + public String getDataSource() + { + return dataSource; +@@ -115,7 +151,7 @@ public class CompactionCandidate + */ + public CompactionCandidate withCurrentStatus(CompactionStatus status) + { +- return new CompactionCandidate(this.segments, status); ++ return new CompactionCandidate(segments, umbrellaInterval, compactionInterval, numIntervals, status); + } + + @Override +diff --git a/server/src/main/java/org/apache/druid/server/compaction/CompactionRunSimulator.java b/server/src/main/java/org/apache/druid/server/compaction/CompactionRunSimulator.java +index 646319eec3..781019e12f 100644 +--- a/server/src/main/java/org/apache/druid/server/compaction/CompactionRunSimulator.java ++++ b/server/src/main/java/org/apache/druid/server/compaction/CompactionRunSimulator.java +@@ -22,7 +22,6 @@ package org.apache.druid.server.compaction; + import com.google.common.util.concurrent.Futures; + import com.google.common.util.concurrent.ListenableFuture; + import org.apache.druid.client.DataSourcesSnapshot; +-import org.apache.druid.client.indexing.ClientCompactionTaskQuery; + import org.apache.druid.client.indexing.ClientCompactionTaskQueryTuningConfig; + import org.apache.druid.client.indexing.IndexingTotalWorkerCapacityInfo; + import org.apache.druid.client.indexing.TaskPayloadResponse; +@@ -86,16 +85,15 @@ public class CompactionRunSimulator + + // Add a read-only wrapper over the actual status tracker so that we can + // account for the active tasks +- final CompactionStatusTracker simulationStatusTracker = new CompactionStatusTracker(null) ++ final CompactionStatusTracker simulationStatusTracker = new CompactionStatusTracker() + { + @Override + public CompactionStatus computeCompactionStatus( + CompactionCandidate candidate, +- DataSourceCompactionConfig config, + CompactionCandidateSearchPolicy searchPolicy + ) + { +- return statusTracker.computeCompactionStatus(candidate, config, searchPolicy); ++ return statusTracker.computeCompactionStatus(candidate, searchPolicy); + } + + @Override +@@ -123,12 +121,12 @@ public class CompactionRunSimulator + } + + @Override +- public void onTaskSubmitted(ClientCompactionTaskQuery taskPayload, CompactionCandidate candidateSegments) ++ public void onTaskSubmitted(String taskId, CompactionCandidate candidateSegments) + { + // Add a row for each task in order of submission + final CompactionStatus status = candidateSegments.getCurrentStatus(); + queuedIntervals.addRow( +- createRow(candidateSegments, taskPayload.getTuningConfig(), status == null ? "" : status.getReason()) ++ createRow(candidateSegments, null, status == null ? "" : status.getReason()) + ); + } + }; +@@ -171,12 +169,10 @@ public class CompactionRunSimulator + { + final List row = new ArrayList<>(); + row.add(candidate.getDataSource()); +- row.add(candidate.getUmbrellaInterval()); ++ row.add(candidate.getCompactionInterval()); + row.add(candidate.numSegments()); + row.add(candidate.getTotalBytes()); +- if (tuningConfig != null) { +- row.add(CompactSegments.findMaxNumTaskSlotsUsedByOneNativeCompactionTask(tuningConfig)); +- } ++ row.add(CompactionSlotManager.getMaxTaskSlotsForNativeCompactionTask(tuningConfig)); + if (reason != null) { + row.add(reason); + } +diff --git a/server/src/main/java/org/apache/druid/server/compaction/CompactionSlotManager.java b/server/src/main/java/org/apache/druid/server/compaction/CompactionSlotManager.java +new file mode 100644 +index 0000000000..428367db8e +--- /dev/null ++++ b/server/src/main/java/org/apache/druid/server/compaction/CompactionSlotManager.java +@@ -0,0 +1,376 @@ ++/* ++ * Licensed to the Apache Software Foundation (ASF) under one ++ * or more contributor license agreements. See the NOTICE file ++ * distributed with this work for additional information ++ * regarding copyright ownership. The ASF licenses this file ++ * to you under the Apache License, Version 2.0 (the ++ * "License"); you may not use this file except in compliance ++ * with the License. You may obtain a copy of the License at ++ * ++ * http://www.apache.org/licenses/LICENSE-2.0 ++ * ++ * Unless required by applicable law or agreed to in writing, ++ * software distributed under the License is distributed on an ++ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY ++ * KIND, either express or implied. See the License for the ++ * specific language governing permissions and limitations ++ * under the License. ++ */ ++ ++package org.apache.druid.server.compaction; ++ ++import com.google.common.annotations.VisibleForTesting; ++import org.apache.druid.client.indexing.ClientCompactionTaskQuery; ++import org.apache.druid.client.indexing.ClientCompactionTaskQueryTuningConfig; ++import org.apache.druid.client.indexing.ClientMSQContext; ++import org.apache.druid.client.indexing.TaskPayloadResponse; ++import org.apache.druid.common.guava.FutureUtils; ++import org.apache.druid.indexer.CompactionEngine; ++import org.apache.druid.indexer.TaskStatus; ++import org.apache.druid.indexer.TaskStatusPlus; ++import org.apache.druid.indexer.partitions.DimensionRangePartitionsSpec; ++import org.apache.druid.java.util.common.ISE; ++import org.apache.druid.java.util.common.granularity.Granularity; ++import org.apache.druid.java.util.common.logger.Logger; ++import org.apache.druid.metadata.LockFilterPolicy; ++import org.apache.druid.rpc.indexing.OverlordClient; ++import org.apache.druid.server.coordinator.ClusterCompactionConfig; ++import org.apache.druid.server.coordinator.DataSourceCompactionConfig; ++import org.apache.druid.server.coordinator.duty.CoordinatorDutyUtils; ++import org.joda.time.Interval; ++ ++import javax.annotation.Nullable; ++import java.util.ArrayList; ++import java.util.HashMap; ++import java.util.HashSet; ++import java.util.List; ++import java.util.Map; ++import java.util.Set; ++import java.util.stream.Collectors; ++ ++/** ++ * Fetches running compaction tasks from the Overlord and tracks their compaction ++ * intervals and task slots. ++ */ ++public class CompactionSlotManager ++{ ++ /** ++ * Task type for native compaction tasks. ++ */ ++ public static final String COMPACTION_TASK_TYPE = "compact"; ++ ++ private static final Logger log = new Logger(CompactionSlotManager.class); ++ ++ private final OverlordClient overlordClient; ++ private final CompactionStatusTracker statusTracker; ++ ++ private final Map> intervalsToSkipCompaction; ++ ++ private int numAvailableTaskSlots; ++ ++ public CompactionSlotManager( ++ OverlordClient overlordClient, ++ CompactionStatusTracker statusTracker, ++ ClusterCompactionConfig clusterCompactionConfig ++ ) ++ { ++ this.overlordClient = overlordClient; ++ this.statusTracker = statusTracker; ++ this.numAvailableTaskSlots = getCompactionTaskCapacity(clusterCompactionConfig); ++ this.intervalsToSkipCompaction = new HashMap<>(); ++ } ++ ++ public int getNumAvailableTaskSlots() ++ { ++ return numAvailableTaskSlots; ++ } ++ ++ public Map> getDatasourceIntervalsToSkipCompaction() ++ { ++ return intervalsToSkipCompaction; ++ } ++ ++ public void reserveTaskSlots(int numSlotsToReserve) ++ { ++ numAvailableTaskSlots -= numSlotsToReserve; ++ } ++ ++ /** ++ * Reserves task slots for the given task from the overall compaction task capacity. ++ */ ++ public void reserveTaskSlots(ClientCompactionTaskQuery compactionTaskQuery) ++ { ++ // Note: The default compactionRunnerType used here should match the default runner used in CompactionTask when ++ // no runner is provided there. ++ CompactionEngine compactionRunnerType = compactionTaskQuery.getCompactionRunner() == null ++ ? CompactionEngine.NATIVE ++ : compactionTaskQuery.getCompactionRunner().getType(); ++ if (compactionRunnerType == CompactionEngine.NATIVE) { ++ numAvailableTaskSlots -= ++ getMaxTaskSlotsForNativeCompactionTask(compactionTaskQuery.getTuningConfig()); ++ } else { ++ numAvailableTaskSlots -= ++ getMaxTaskSlotsForMSQCompactionTask(compactionTaskQuery.getContext()); ++ } ++ } ++ ++ /** ++ * Reserves task slots for all running compaction tasks. ++ */ ++ public void reserveTaskSlotsForRunningCompactionTasks() ++ { ++ for (ClientCompactionTaskQuery task : fetchRunningCompactionTasks()) { ++ reserveTaskSlots(task); ++ } ++ } ++ ++ /** ++ * Retrieves currently running tasks of type {@link #COMPACTION_TASK_TYPE} from ++ * the Overlord. ++ *

++ * Also queries the Overlord for the status of all tasks that were submitted ++ * recently but are not active anymore. The statuses are then updated in the ++ * {@link CompactionStatusTracker}. ++ */ ++ public List fetchRunningCompactionTasks() ++ { ++ // Fetch currently running compaction tasks ++ final List compactionTasks = CoordinatorDutyUtils.getStatusOfActiveTasks( ++ overlordClient, ++ status -> status != null && COMPACTION_TASK_TYPE.equals(status.getType()) ++ ); ++ ++ final Set activeTaskIds ++ = compactionTasks.stream().map(TaskStatusPlus::getId).collect(Collectors.toSet()); ++ trackStatusOfCompletedTasks(activeTaskIds); ++ ++ final List runningCompactTasks = new ArrayList<>(); ++ for (TaskStatusPlus status : compactionTasks) { ++ final TaskPayloadResponse response = ++ FutureUtils.getUnchecked(overlordClient.taskPayload(status.getId()), true); ++ if (response == null) { ++ throw new ISE("Could not find payload for active compaction task[%s]", status.getId()); ++ } else if (!COMPACTION_TASK_TYPE.equals(response.getPayload().getType())) { ++ throw new ISE( ++ "Payload of active compaction task[%s] is of invalid type[%s]", ++ status.getId(), response.getPayload().getType() ++ ); ++ } ++ ++ runningCompactTasks.add((ClientCompactionTaskQuery) response.getPayload()); ++ } ++ ++ return runningCompactTasks; ++ } ++ ++ /** ++ * Cancels a currently running compaction task only if the segment granularity ++ * has changed in the datasource compaction config. Otherwise, the task is ++ * retained and its intervals are skipped from the current round of compaction. ++ * ++ * @return true if the task was canceled, false otherwise. ++ */ ++ public boolean cancelTaskOnlyIfGranularityChanged( ++ ClientCompactionTaskQuery compactionTaskQuery, ++ DataSourceCompactionConfig dataSourceCompactionConfig ++ ) ++ { ++ if (dataSourceCompactionConfig == null ++ || dataSourceCompactionConfig.getGranularitySpec() == null ++ || compactionTaskQuery.getGranularitySpec() == null) { ++ skipTaskInterval(compactionTaskQuery); ++ reserveTaskSlots(compactionTaskQuery); ++ return false; ++ } ++ ++ Granularity configuredSegmentGranularity = dataSourceCompactionConfig.getGranularitySpec() ++ .getSegmentGranularity(); ++ Granularity taskSegmentGranularity = compactionTaskQuery.getGranularitySpec().getSegmentGranularity(); ++ if (configuredSegmentGranularity == null || configuredSegmentGranularity.equals(taskSegmentGranularity)) { ++ skipTaskInterval(compactionTaskQuery); ++ reserveTaskSlots(compactionTaskQuery); ++ return false; ++ } ++ ++ log.info( ++ "Cancelling task[%s] as task segmentGranularity[%s] differs from compaction config segmentGranularity[%s].", ++ compactionTaskQuery.getId(), taskSegmentGranularity, configuredSegmentGranularity ++ ); ++ overlordClient.cancelTask(compactionTaskQuery.getId()); ++ return true; ++ } ++ ++ /** ++ * Retrieves the list of intervals locked by higher priority tasks for each datasource. ++ * Since compaction tasks submitted for these Intervals would have to wait anyway, ++ * we skip these Intervals until the next compaction run by adding them to ++ * {@link #intervalsToSkipCompaction}. ++ *

++ * This method must be called after invalid compaction tasks have already been ++ * cancelled using {@link #cancelTaskOnlyIfGranularityChanged} so that their ++ * intervals are not considered locked. ++ */ ++ public void skipLockedIntervals(List compactionConfigs) ++ { ++ final List lockFilterPolicies = compactionConfigs ++ .stream() ++ .map( ++ config -> new LockFilterPolicy( ++ config.getDataSource(), ++ config.getTaskPriority(), ++ null, ++ config.getTaskContext() ++ ) ++ ) ++ .collect(Collectors.toList()); ++ final Map> datasourceToLockedIntervals = new HashMap<>( ++ FutureUtils.getUnchecked(overlordClient.findLockedIntervals(lockFilterPolicies), true) ++ ); ++ log.debug( ++ "Skipping the following intervals for Compaction as they are currently locked: %s", ++ datasourceToLockedIntervals ++ ); ++ ++ // Skip all the intervals locked by higher priority tasks for each datasource ++ datasourceToLockedIntervals.forEach( ++ (dataSource, intervals) -> ++ intervalsToSkipCompaction ++ .computeIfAbsent(dataSource, ds -> new ArrayList<>()) ++ .addAll(intervals) ++ ); ++ } ++ ++ /** ++ * Adds the compaction interval of this task to {@link #intervalsToSkipCompaction} ++ */ ++ private void skipTaskInterval(ClientCompactionTaskQuery compactionTask) ++ { ++ final Interval interval = compactionTask.getIoConfig().getInputSpec().getInterval(); ++ intervalsToSkipCompaction.computeIfAbsent(compactionTask.getDataSource(), k -> new ArrayList<>()) ++ .add(interval); ++ } ++ ++ /** ++ * Computes overall compaction task capacity for the cluster. ++ * ++ * @return A value >= 1. ++ */ ++ private int getCompactionTaskCapacity(ClusterCompactionConfig clusterConfig) ++ { ++ int totalWorkerCapacity = CoordinatorDutyUtils.getTotalWorkerCapacity(overlordClient); ++ ++ int compactionTaskCapacity = Math.min( ++ (int) (totalWorkerCapacity * clusterConfig.getCompactionTaskSlotRatio()), ++ clusterConfig.getMaxCompactionTaskSlots() ++ ); ++ ++ // Always consider atleast one slot available for compaction ++ return Math.max(1, compactionTaskCapacity); ++ } ++ ++ /** ++ * Queries the Overlord for the status of all tasks that were submitted ++ * recently but are not active anymore. The statuses are then updated in the ++ * {@link #statusTracker}. ++ */ ++ private void trackStatusOfCompletedTasks(Set activeTaskIds) ++ { ++ final Set finishedTaskIds = new HashSet<>(statusTracker.getSubmittedTaskIds()); ++ finishedTaskIds.removeAll(activeTaskIds); ++ ++ if (finishedTaskIds.isEmpty()) { ++ return; ++ } ++ ++ final Map taskStatusMap ++ = FutureUtils.getUnchecked(overlordClient.taskStatuses(finishedTaskIds), true); ++ for (String taskId : finishedTaskIds) { ++ // Assume unknown task to have finished successfully ++ final TaskStatus taskStatus = taskStatusMap.getOrDefault(taskId, TaskStatus.success(taskId)); ++ if (taskStatus.isComplete()) { ++ statusTracker.onTaskFinished(taskId, taskStatus); ++ } ++ } ++ } ++ ++ /** ++ * @return Maximum number of task slots used by a native compaction task at ++ * any time when the task is run with the given tuning config. ++ */ ++ public static int getMaxTaskSlotsForNativeCompactionTask( ++ @Nullable ClientCompactionTaskQueryTuningConfig tuningConfig ++ ) ++ { ++ if (isParallelMode(tuningConfig)) { ++ Integer maxNumConcurrentSubTasks = tuningConfig.getMaxNumConcurrentSubTasks(); ++ // Max number of task slots used in parallel mode = maxNumConcurrentSubTasks + 1 (supervisor task) ++ return (maxNumConcurrentSubTasks == null ? 1 : maxNumConcurrentSubTasks) + 1; ++ } else { ++ return 1; ++ } ++ } ++ ++ /** ++ * @return Maximum number of task slots used by an MSQ compaction task at any ++ * time when the task is run with the given context. ++ */ ++ public static int getMaxTaskSlotsForMSQCompactionTask(@Nullable Map context) ++ { ++ return context == null ++ ? ClientMSQContext.DEFAULT_MAX_NUM_TASKS ++ : (int) context.getOrDefault(ClientMSQContext.CTX_MAX_NUM_TASKS, ClientMSQContext.DEFAULT_MAX_NUM_TASKS); ++ } ++ ++ ++ /** ++ * Returns true if the compaction task can run in the parallel mode with the given tuningConfig. ++ * This method should be synchronized with ParallelIndexSupervisorTask.isParallelMode(InputSource, ParallelIndexTuningConfig). ++ */ ++ @VisibleForTesting ++ public static boolean isParallelMode(@Nullable ClientCompactionTaskQueryTuningConfig tuningConfig) ++ { ++ if (null == tuningConfig) { ++ return false; ++ } ++ boolean useRangePartitions = useRangePartitions(tuningConfig); ++ int minRequiredNumConcurrentSubTasks = useRangePartitions ? 1 : 2; ++ return tuningConfig.getMaxNumConcurrentSubTasks() != null ++ && tuningConfig.getMaxNumConcurrentSubTasks() >= minRequiredNumConcurrentSubTasks; ++ } ++ ++ private static boolean useRangePartitions(ClientCompactionTaskQueryTuningConfig tuningConfig) ++ { ++ // dynamic partitionsSpec will be used if getPartitionsSpec() returns null ++ return tuningConfig.getPartitionsSpec() instanceof DimensionRangePartitionsSpec; ++ } ++ ++ public int computeSlotsRequiredForTask( ++ ClientCompactionTaskQuery task, ++ DataSourceCompactionConfig config ++ ) ++ { ++ if (task.getCompactionRunner().getType() == CompactionEngine.MSQ) { ++ final Map autoCompactionContext = task.getContext(); ++ if (autoCompactionContext.containsKey(ClientMSQContext.CTX_MAX_NUM_TASKS)) { ++ return (int) autoCompactionContext.get(ClientMSQContext.CTX_MAX_NUM_TASKS); ++ } else { ++ // Since MSQ needs all task slots for the calculated #tasks to be available upfront, allot all available ++ // compaction slots (upto a max of MAX_TASK_SLOTS_FOR_MSQ_COMPACTION) to current compaction task to avoid ++ // stalling. Setting "taskAssignment" to "auto" has the problem of not being able to determine the actual ++ // count, which is required for subsequent tasks. ++ final int slotsRequiredForCurrentTask = Math.min( ++ // Update the slots to 2 (min required for MSQ) if only 1 slot is available. ++ numAvailableTaskSlots == 1 ? 2 : numAvailableTaskSlots, ++ ClientMSQContext.MAX_TASK_SLOTS_FOR_MSQ_COMPACTION_TASK ++ ); ++ autoCompactionContext.put(ClientMSQContext.CTX_MAX_NUM_TASKS, slotsRequiredForCurrentTask); ++ ++ return slotsRequiredForCurrentTask; ++ } ++ } else { ++ return CompactionSlotManager.getMaxTaskSlotsForNativeCompactionTask( ++ config.getTuningConfig() ++ ); ++ } ++ } ++} +diff --git a/server/src/main/java/org/apache/druid/server/compaction/CompactionSnapshotBuilder.java b/server/src/main/java/org/apache/druid/server/compaction/CompactionSnapshotBuilder.java +new file mode 100644 +index 0000000000..d186587cb8 +--- /dev/null ++++ b/server/src/main/java/org/apache/druid/server/compaction/CompactionSnapshotBuilder.java +@@ -0,0 +1,108 @@ ++/* ++ * Licensed to the Apache Software Foundation (ASF) under one ++ * or more contributor license agreements. See the NOTICE file ++ * distributed with this work for additional information ++ * regarding copyright ownership. The ASF licenses this file ++ * to you under the Apache License, Version 2.0 (the ++ * "License"); you may not use this file except in compliance ++ * with the License. You may obtain a copy of the License at ++ * ++ * http://www.apache.org/licenses/LICENSE-2.0 ++ * ++ * Unless required by applicable law or agreed to in writing, ++ * software distributed under the License is distributed on an ++ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY ++ * KIND, either express or implied. See the License for the ++ * specific language governing permissions and limitations ++ * under the License. ++ */ ++ ++package org.apache.druid.server.compaction; ++ ++import org.apache.druid.server.coordinator.AutoCompactionSnapshot; ++import org.apache.druid.server.coordinator.stats.CoordinatorRunStats; ++import org.apache.druid.server.coordinator.stats.Dimension; ++import org.apache.druid.server.coordinator.stats.RowKey; ++import org.apache.druid.server.coordinator.stats.Stats; ++ ++import java.util.HashMap; ++import java.util.Map; ++ ++/** ++ * Builds {@link AutoCompactionSnapshot} for multiple datasources using the ++ * identified {@link CompactionCandidate} list. ++ */ ++public class CompactionSnapshotBuilder ++{ ++ private final CoordinatorRunStats stats; ++ private final Map datasourceToBuilder = new HashMap<>(); ++ ++ public CompactionSnapshotBuilder(CoordinatorRunStats runStats) ++ { ++ this.stats = runStats; ++ } ++ ++ public void addToComplete(CompactionCandidate candidate) ++ { ++ getBuilderForDatasource(candidate.getDataSource()) ++ .incrementCompactedStats(candidate.getStats()); ++ } ++ ++ public void addToPending(CompactionCandidate candidate) ++ { ++ getBuilderForDatasource(candidate.getDataSource()) ++ .incrementWaitingStats(candidate.getStats()); ++ } ++ ++ public void addToSkipped(CompactionCandidate candidate) ++ { ++ getBuilderForDatasource(candidate.getDataSource()) ++ .incrementSkippedStats(candidate.getStats()); ++ } ++ ++ public void moveFromPendingToSkipped(CompactionCandidate candidate) ++ { ++ getBuilderForDatasource(candidate.getDataSource()) ++ .decrementWaitingStats(candidate.getStats()); ++ addToSkipped(candidate); ++ } ++ ++ public void moveFromPendingToCompleted(CompactionCandidate candidate) ++ { ++ getBuilderForDatasource(candidate.getDataSource()) ++ .decrementWaitingStats(candidate.getStats()); ++ addToComplete(candidate); ++ } ++ ++ public Map build() ++ { ++ final Map datasourceToSnapshot = new HashMap<>(); ++ datasourceToBuilder.forEach((dataSource, builder) -> { ++ final AutoCompactionSnapshot autoCompactionSnapshot = builder.build(); ++ datasourceToSnapshot.put(dataSource, autoCompactionSnapshot); ++ collectSnapshotStats(autoCompactionSnapshot); ++ }); ++ ++ return datasourceToSnapshot; ++ } ++ ++ private AutoCompactionSnapshot.Builder getBuilderForDatasource(String dataSource) ++ { ++ return datasourceToBuilder.computeIfAbsent(dataSource, AutoCompactionSnapshot::builder); ++ } ++ ++ private void collectSnapshotStats(AutoCompactionSnapshot autoCompactionSnapshot) ++ { ++ final RowKey rowKey = RowKey.of(Dimension.DATASOURCE, autoCompactionSnapshot.getDataSource()); ++ ++ stats.add(Stats.Compaction.PENDING_BYTES, rowKey, autoCompactionSnapshot.getBytesAwaitingCompaction()); ++ stats.add(Stats.Compaction.PENDING_SEGMENTS, rowKey, autoCompactionSnapshot.getSegmentCountAwaitingCompaction()); ++ stats.add(Stats.Compaction.PENDING_INTERVALS, rowKey, autoCompactionSnapshot.getIntervalCountAwaitingCompaction()); ++ stats.add(Stats.Compaction.COMPACTED_BYTES, rowKey, autoCompactionSnapshot.getBytesCompacted()); ++ stats.add(Stats.Compaction.COMPACTED_SEGMENTS, rowKey, autoCompactionSnapshot.getSegmentCountCompacted()); ++ stats.add(Stats.Compaction.COMPACTED_INTERVALS, rowKey, autoCompactionSnapshot.getIntervalCountCompacted()); ++ stats.add(Stats.Compaction.SKIPPED_BYTES, rowKey, autoCompactionSnapshot.getBytesSkipped()); ++ stats.add(Stats.Compaction.SKIPPED_SEGMENTS, rowKey, autoCompactionSnapshot.getSegmentCountSkipped()); ++ stats.add(Stats.Compaction.SKIPPED_INTERVALS, rowKey, autoCompactionSnapshot.getIntervalCountSkipped()); ++ } ++} +diff --git a/server/src/main/java/org/apache/druid/server/compaction/CompactionStatistics.java b/server/src/main/java/org/apache/druid/server/compaction/CompactionStatistics.java +index 23f1b7fe9e..d7e5165586 100644 +--- a/server/src/main/java/org/apache/druid/server/compaction/CompactionStatistics.java ++++ b/server/src/main/java/org/apache/druid/server/compaction/CompactionStatistics.java +@@ -58,4 +58,11 @@ public class CompactionStatistics + numIntervals += other.getNumIntervals(); + numSegments += other.getNumSegments(); + } ++ ++ public void decrement(CompactionStatistics other) ++ { ++ totalBytes -= other.getTotalBytes(); ++ numIntervals -= other.getNumIntervals(); ++ numSegments -= other.getNumSegments(); ++ } + } +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 3c1616b6e7..be4acd00e2 100644 +--- a/server/src/main/java/org/apache/druid/server/compaction/CompactionStatus.java ++++ b/server/src/main/java/org/apache/druid/server/compaction/CompactionStatus.java +@@ -19,9 +19,7 @@ + + package org.apache.druid.server.compaction; + +-import com.fasterxml.jackson.databind.ObjectMapper; + import org.apache.commons.lang3.ArrayUtils; +-import org.apache.druid.client.indexing.ClientCompactionTaskGranularitySpec; + import org.apache.druid.client.indexing.ClientCompactionTaskQueryTuningConfig; + import org.apache.druid.common.config.Configs; + import org.apache.druid.data.input.impl.DimensionSchema; +@@ -64,6 +62,7 @@ public class CompactionStatus + * The order of the checks must be honored while evaluating them. + */ + private static final List> CHECKS = Arrays.asList( ++ Evaluator::inputBytesAreWithinLimit, + Evaluator::segmentsHaveBeenCompactedAtLeastOnce, + Evaluator::allCandidatesHaveSameCompactionState, + Evaluator::partitionsSpecIsUpToDate, +@@ -115,12 +114,17 @@ public class CompactionStatus + '}'; + } + +- private static CompactionStatus incomplete(String reasonFormat, Object... args) ++ public static CompactionStatus pending(String reasonFormat, Object... args) + { + return new CompactionStatus(State.PENDING, StringUtils.format(reasonFormat, args)); + } + +- private static CompactionStatus completeIfEqual( ++ /** ++ * Computes compaction status for the given field. The status is assumed to be ++ * COMPLETE (i.e. no further compaction is required) if the configured value ++ * of the field is null or equal to the current value. ++ */ ++ private static CompactionStatus completeIfNullOrEqual( + String field, + T configured, + T current, +@@ -141,7 +145,7 @@ public class CompactionStatus + Function stringFunction + ) + { +- return CompactionStatus.incomplete( ++ return CompactionStatus.pending( + "'%s' mismatch: required[%s], current[%s]", + field, + target == null ? null : stringFunction.apply(target), +@@ -187,14 +191,19 @@ public class CompactionStatus + } + } + +- static CompactionStatus skipped(String reasonFormat, Object... args) ++ public static CompactionStatus skipped(String reasonFormat, Object... args) + { + return new CompactionStatus(State.SKIPPED, StringUtils.format(reasonFormat, args)); + } + +- static CompactionStatus running(String reasonForCompaction) ++ public static CompactionStatus running(String message) ++ { ++ return new CompactionStatus(State.RUNNING, message); ++ } ++ ++ public static CompactionStatus complete(String message) + { +- return new CompactionStatus(State.RUNNING, reasonForCompaction); ++ return new CompactionStatus(State.COMPLETE, message); + } + + /** +@@ -204,23 +213,30 @@ public class CompactionStatus + */ + static CompactionStatus compute( + CompactionCandidate candidateSegments, +- DataSourceCompactionConfig config, +- ObjectMapper objectMapper ++ DataSourceCompactionConfig config + ) + { +- final Evaluator evaluator = new Evaluator(candidateSegments, config, objectMapper); ++ final Evaluator evaluator = new Evaluator(candidateSegments, config); + return CHECKS.stream() + .map(f -> f.apply(evaluator)) + .filter(status -> !status.isComplete()) + .findFirst().orElse(COMPLETE); + } + ++ @Nullable + static PartitionsSpec findPartitionsSpecFromConfig(ClientCompactionTaskQueryTuningConfig tuningConfig) + { + final PartitionsSpec partitionsSpecFromTuningConfig = tuningConfig.getPartitionsSpec(); + if (partitionsSpecFromTuningConfig == null) { +- final long maxTotalRows = Configs.valueOrDefault(tuningConfig.getMaxTotalRows(), Long.MAX_VALUE); +- return new DynamicPartitionsSpec(tuningConfig.getMaxRowsPerSegment(), maxTotalRows); ++ final Long maxTotalRows = tuningConfig.getMaxTotalRows(); ++ final Integer maxRowsPerSegment = tuningConfig.getMaxRowsPerSegment(); ++ ++ if (maxTotalRows == null && maxRowsPerSegment == null) { ++ // If not specified, return null so that partitionsSpec is not compared ++ return null; ++ } else { ++ return new DynamicPartitionsSpec(maxRowsPerSegment, maxTotalRows); ++ } + } else if (partitionsSpecFromTuningConfig instanceof DynamicPartitionsSpec) { + return new DynamicPartitionsSpec( + partitionsSpecFromTuningConfig.getMaxRowsPerSegment(), +@@ -276,22 +292,19 @@ public class CompactionStatus + */ + private static class Evaluator + { +- private final ObjectMapper objectMapper; + private final DataSourceCompactionConfig compactionConfig; + private final CompactionCandidate candidateSegments; + private final CompactionState lastCompactionState; + private final ClientCompactionTaskQueryTuningConfig tuningConfig; +- private final ClientCompactionTaskGranularitySpec existingGranularitySpec; ++ private final UserCompactionTaskGranularityConfig existingGranularitySpec; + private final UserCompactionTaskGranularityConfig configuredGranularitySpec; + + private Evaluator( + CompactionCandidate candidateSegments, +- DataSourceCompactionConfig compactionConfig, +- ObjectMapper objectMapper ++ DataSourceCompactionConfig compactionConfig + ) + { + this.candidateSegments = candidateSegments; +- this.objectMapper = objectMapper; + this.lastCompactionState = candidateSegments.getSegments().get(0).getLastCompactionState(); + this.compactionConfig = compactionConfig; + this.tuningConfig = ClientCompactionTaskQueryTuningConfig.from(compactionConfig); +@@ -299,9 +312,8 @@ public class CompactionStatus + if (lastCompactionState == null) { + this.existingGranularitySpec = null; + } else { +- this.existingGranularitySpec = convertIfNotNull( +- lastCompactionState.getGranularitySpec(), +- ClientCompactionTaskGranularitySpec.class ++ this.existingGranularitySpec = UserCompactionTaskGranularityConfig.from( ++ lastCompactionState.getGranularitySpec() + ); + } + } +@@ -309,7 +321,7 @@ public class CompactionStatus + private CompactionStatus segmentsHaveBeenCompactedAtLeastOnce() + { + if (lastCompactionState == null) { +- return CompactionStatus.incomplete("not compacted yet"); ++ return CompactionStatus.pending("not compacted yet"); + } else { + return COMPLETE; + } +@@ -323,7 +335,7 @@ public class CompactionStatus + if (allHaveSameCompactionState) { + return COMPLETE; + } else { +- return CompactionStatus.incomplete("segments have different last compaction states"); ++ return CompactionStatus.pending("segments have different last compaction states"); + } + } + +@@ -337,7 +349,7 @@ public class CompactionStatus + existingPartionsSpec.getMaxRowsPerSegment(), + ((DynamicPartitionsSpec) existingPartionsSpec).getMaxTotalRowsOr(Long.MAX_VALUE)); + } +- return CompactionStatus.completeIfEqual( ++ return CompactionStatus.completeIfNullOrEqual( + "partitionsSpec", + findPartitionsSpecFromConfig(tuningConfig), + existingPartionsSpec, +@@ -347,17 +359,17 @@ public class CompactionStatus + + private CompactionStatus indexSpecIsUpToDate() + { +- return CompactionStatus.completeIfEqual( ++ return CompactionStatus.completeIfNullOrEqual( + "indexSpec", + Configs.valueOrDefault(tuningConfig.getIndexSpec(), IndexSpec.getDefault()).getEffectiveSpec(), +- objectMapper.convertValue(lastCompactionState.getIndexSpec(), IndexSpec.class).getEffectiveSpec(), ++ lastCompactionState.getIndexSpec().getEffectiveSpec(), + String::valueOf + ); + } + + private CompactionStatus projectionsAreUpToDate() + { +- return CompactionStatus.completeIfEqual( ++ return CompactionStatus.completeIfNullOrEqual( + "projections", + compactionConfig.getProjections(), + lastCompactionState.getProjections(), +@@ -365,6 +377,19 @@ public class CompactionStatus + ); + } + ++ private CompactionStatus inputBytesAreWithinLimit() ++ { ++ final long inputSegmentSize = compactionConfig.getInputSegmentSizeBytes(); ++ if (candidateSegments.getTotalBytes() > inputSegmentSize) { ++ return CompactionStatus.skipped( ++ "'inputSegmentSize' exceeded: Total segment size[%d] is larger than allowed inputSegmentSize[%d]", ++ candidateSegments.getTotalBytes(), inputSegmentSize ++ ); ++ } else { ++ return COMPLETE; ++ } ++ } ++ + private CompactionStatus segmentGranularityIsUpToDate() + { + if (configuredGranularitySpec == null +@@ -385,7 +410,7 @@ public class CompactionStatus + segment -> !configuredSegmentGranularity.isAligned(segment.getInterval()) + ); + if (needsCompaction) { +- return CompactionStatus.incomplete( ++ return CompactionStatus.pending( + "segmentGranularity: segments do not align with target[%s]", + asString(configuredSegmentGranularity) + ); +@@ -407,7 +432,7 @@ public class CompactionStatus + if (configuredGranularitySpec == null) { + return COMPLETE; + } else { +- return CompactionStatus.completeIfEqual( ++ return CompactionStatus.completeIfNullOrEqual( + "rollup", + configuredGranularitySpec.isRollup(), + existingGranularitySpec == null ? null : existingGranularitySpec.isRollup(), +@@ -421,7 +446,7 @@ public class CompactionStatus + if (configuredGranularitySpec == null) { + return COMPLETE; + } else { +- return CompactionStatus.completeIfEqual( ++ return CompactionStatus.completeIfNullOrEqual( + "queryGranularity", + configuredGranularitySpec.getQueryGranularity(), + existingGranularitySpec == null ? null : existingGranularitySpec.getQueryGranularity(), +@@ -454,7 +479,7 @@ public class CompactionStatus + ? IndexSpec.getDefault() + : compactionConfig.getTuningConfig().getIndexSpec() + ); +- return CompactionStatus.completeIfEqual( ++ return CompactionStatus.completeIfNullOrEqual( + "dimensionsSpec", + configuredDimensions, + existingDimensions, +@@ -493,26 +518,13 @@ public class CompactionStatus + return COMPLETE; + } + +- CompactionTransformSpec existingTransformSpec = convertIfNotNull( +- lastCompactionState.getTransformSpec(), +- CompactionTransformSpec.class +- ); +- return CompactionStatus.completeIfEqual( ++ CompactionTransformSpec existingTransformSpec = lastCompactionState.getTransformSpec(); ++ return CompactionStatus.completeIfNullOrEqual( + "transformSpec filter", + compactionConfig.getTransformSpec().getFilter(), + existingTransformSpec == null ? null : existingTransformSpec.getFilter(), + String::valueOf + ); + } +- +- @Nullable +- private T convertIfNotNull(Object object, Class clazz) +- { +- if (object == null) { +- return null; +- } else { +- return objectMapper.convertValue(object, clazz); +- } +- } + } + } +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 cbf5f25f9d..401f413e7f 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 +@@ -19,14 +19,10 @@ + + package org.apache.druid.server.compaction; + +-import com.fasterxml.jackson.databind.ObjectMapper; +-import com.google.inject.Inject; +-import org.apache.druid.client.indexing.ClientCompactionTaskQuery; + import org.apache.druid.indexer.TaskState; + import org.apache.druid.indexer.TaskStatus; + import org.apache.druid.java.util.common.DateTimes; + import org.apache.druid.server.coordinator.DataSourceCompactionConfig; +-import org.apache.druid.server.coordinator.DruidCompactionConfig; + import org.joda.time.DateTime; + import org.joda.time.Duration; + import org.joda.time.Interval; +@@ -45,7 +41,6 @@ public class CompactionStatusTracker + { + private static final Duration MAX_STATUS_RETAIN_DURATION = Duration.standardHours(12); + +- private final ObjectMapper objectMapper; + private final ConcurrentHashMap datasourceStatuses + = new ConcurrentHashMap<>(); + private final ConcurrentHashMap submittedTaskIdToSegments +@@ -53,12 +48,6 @@ public class CompactionStatusTracker + + private final AtomicReference segmentSnapshotTime = new AtomicReference<>(); + +- @Inject +- public CompactionStatusTracker(ObjectMapper objectMapper) +- { +- this.objectMapper = objectMapper; +- } +- + public void stop() + { + datasourceStatuses.clear(); +@@ -74,7 +63,7 @@ public class CompactionStatusTracker + return datasourceStatuses + .getOrDefault(candidates.getDataSource(), DatasourceStatus.EMPTY) + .intervalToTaskStatus +- .get(candidates.getUmbrellaInterval()); ++ .get(candidates.getCompactionInterval()); + } + + /** +@@ -86,30 +75,20 @@ public class CompactionStatusTracker + return submittedTaskIdToSegments.keySet(); + } + ++ /** ++ * Checks if compaction can be started for the given {@link CompactionCandidate}. ++ * This method assumes that the given candidate is eligible for compaction ++ * based on the current compaction config/supervisor of the datasource. ++ */ + public CompactionStatus computeCompactionStatus( + CompactionCandidate candidate, +- DataSourceCompactionConfig config, + CompactionCandidateSearchPolicy searchPolicy + ) + { +- final CompactionStatus compactionStatus = CompactionStatus.compute(candidate, config, objectMapper); +- if (compactionStatus.isComplete()) { +- return compactionStatus; +- } +- +- // Skip intervals that violate max allowed input segment size +- final long inputSegmentSize = config.getInputSegmentSizeBytes(); +- if (candidate.getTotalBytes() > inputSegmentSize) { +- return CompactionStatus.skipped( +- "'inputSegmentSize' exceeded: Total segment size[%d] is larger than allowed inputSegmentSize[%d]", +- candidate.getTotalBytes(), inputSegmentSize +- ); +- } +- + // Skip intervals that already have a running task + final CompactionTaskStatus lastTaskStatus = getLatestTaskStatus(candidate); + if (lastTaskStatus != null && lastTaskStatus.getState() == TaskState.RUNNING) { +- return CompactionStatus.skipped("Task for interval is already running"); ++ return CompactionStatus.running("Task for interval is already running"); + } + + // Skip intervals that have been recently compacted if segment timeline is not updated yet +@@ -117,19 +96,23 @@ public class CompactionStatusTracker + if (lastTaskStatus != null + && lastTaskStatus.getState() == TaskState.SUCCESS + && snapshotTime != null && snapshotTime.isBefore(lastTaskStatus.getUpdatedTime())) { +- return CompactionStatus.skipped( ++ return CompactionStatus.complete( + "Segment timeline not updated since last compaction task succeeded" + ); + } + + // Skip intervals that have been filtered out by the policy +- if (!searchPolicy.isEligibleForCompaction(candidate, compactionStatus, lastTaskStatus)) { ++ if (!searchPolicy.isEligibleForCompaction(candidate, CompactionStatus.pending(""), lastTaskStatus)) { + return CompactionStatus.skipped("Rejected by search policy"); + } + +- return compactionStatus; ++ return CompactionStatus.pending("Not compacted yet"); + } + ++ /** ++ * Tracks the latest compaction status of the given compaction candidates. ++ * Used only by the {@link CompactionRunSimulator}. ++ */ + public void onCompactionStatusComputed( + CompactionCandidate candidateSegments, + DataSourceCompactionConfig config +@@ -143,17 +126,15 @@ public class CompactionStatusTracker + this.segmentSnapshotTime.set(snapshotTime); + } + +- public void onCompactionConfigUpdated(DruidCompactionConfig compactionConfig) ++ /** ++ * Updates the set of datasources that have compaction enabled and cleans up ++ * stale task statuses. ++ */ ++ public void resetActiveDatasources(Set compactionEnabledDatasources) + { +- final Set compactionEnabledDatasources = new HashSet<>(); +- if (compactionConfig.getCompactionConfigs() != null) { +- compactionConfig.getCompactionConfigs().forEach(config -> { +- getOrComputeDatasourceStatus(config.getDataSource()) +- .cleanupStaleTaskStatuses(); +- +- compactionEnabledDatasources.add(config.getDataSource()); +- }); +- } ++ compactionEnabledDatasources.forEach( ++ dataSource -> getOrComputeDatasourceStatus(dataSource).cleanupStaleTaskStatuses() ++ ); + + // Clean up state for datasources where compaction has been disabled + final Set allDatasources = new HashSet<>(datasourceStatuses.keySet()); +@@ -165,12 +146,12 @@ public class CompactionStatusTracker + } + + public void onTaskSubmitted( +- ClientCompactionTaskQuery taskPayload, ++ String taskId, + CompactionCandidate candidateSegments + ) + { +- submittedTaskIdToSegments.put(taskPayload.getId(), candidateSegments); +- getOrComputeDatasourceStatus(taskPayload.getDataSource()) ++ submittedTaskIdToSegments.put(taskId, candidateSegments); ++ getOrComputeDatasourceStatus(candidateSegments.getDataSource()) + .handleSubmittedTask(candidateSegments); + } + +@@ -186,7 +167,7 @@ public class CompactionStatusTracker + return; + } + +- final Interval compactionInterval = candidateSegments.getUmbrellaInterval(); ++ final Interval compactionInterval = candidateSegments.getCompactionInterval(); + getOrComputeDatasourceStatus(candidateSegments.getDataSource()) + .handleCompletedTask(compactionInterval, taskStatus); + } +@@ -229,7 +210,7 @@ public class CompactionStatusTracker + + void handleSubmittedTask(CompactionCandidate candidateSegments) + { +- final Interval interval = candidateSegments.getUmbrellaInterval(); ++ final Interval interval = candidateSegments.getCompactionInterval(); + final CompactionTaskStatus lastStatus = intervalToTaskStatus.get(interval); + + final DateTime now = DateTimes.nowUtc(); +diff --git a/server/src/main/java/org/apache/druid/server/compaction/DataSourceCompactibleSegmentIterator.java b/server/src/main/java/org/apache/druid/server/compaction/DataSourceCompactibleSegmentIterator.java +index e15d310b33..40867eb5c0 100644 +--- a/server/src/main/java/org/apache/druid/server/compaction/DataSourceCompactibleSegmentIterator.java ++++ b/server/src/main/java/org/apache/druid/server/compaction/DataSourceCompactibleSegmentIterator.java +@@ -65,8 +65,6 @@ public class DataSourceCompactibleSegmentIterator implements CompactionSegmentIt + + private final String dataSource; + private final DataSourceCompactionConfig config; +- private final CompactionStatusTracker statusTracker; +- private final CompactionCandidateSearchPolicy searchPolicy; + + private final List compactedSegments = new ArrayList<>(); + private final List skippedSegments = new ArrayList<>(); +@@ -83,14 +81,11 @@ public class DataSourceCompactibleSegmentIterator implements CompactionSegmentIt + DataSourceCompactionConfig config, + SegmentTimeline timeline, + List skipIntervals, +- CompactionCandidateSearchPolicy searchPolicy, +- CompactionStatusTracker statusTracker ++ CompactionCandidateSearchPolicy searchPolicy + ) + { +- this.statusTracker = statusTracker; + this.config = config; + this.dataSource = config.getDataSource(); +- this.searchPolicy = searchPolicy; + this.queue = new PriorityQueue<>(searchPolicy::compareCandidates); + + populateQueue(timeline, skipIntervals); +@@ -117,11 +112,12 @@ public class DataSourceCompactibleSegmentIterator implements CompactionSegmentIt + } + } + if (!partialEternitySegments.isEmpty()) { +- CompactionCandidate candidatesWithStatus = CompactionCandidate.from(partialEternitySegments).withCurrentStatus( +- CompactionStatus.skipped("Segments have partial-eternity intervals") +- ); ++ // Do not use the target segment granularity in the CompactionCandidate ++ // as Granularities.getIterable() will cause OOM due to the above issue ++ CompactionCandidate candidatesWithStatus = CompactionCandidate ++ .from(partialEternitySegments, null) ++ .withCurrentStatus(CompactionStatus.skipped("Segments have partial-eternity intervals")); + skippedSegments.add(candidatesWithStatus); +- statusTracker.onCompactionStatusComputed(candidatesWithStatus, config); + return; + } + +@@ -315,11 +311,9 @@ public class DataSourceCompactibleSegmentIterator implements CompactionSegmentIt + continue; + } + +- final CompactionCandidate candidates = CompactionCandidate.from(segments); +- final CompactionStatus compactionStatus +- = statusTracker.computeCompactionStatus(candidates, config, searchPolicy); ++ final CompactionCandidate candidates = CompactionCandidate.from(segments, config.getSegmentGranularity()); ++ final CompactionStatus compactionStatus = CompactionStatus.compute(candidates, config); + final CompactionCandidate candidatesWithStatus = candidates.withCurrentStatus(compactionStatus); +- statusTracker.onCompactionStatusComputed(candidatesWithStatus, config); + + if (compactionStatus.isComplete()) { + compactedSegments.add(candidatesWithStatus); +@@ -360,10 +354,10 @@ public class DataSourceCompactibleSegmentIterator implements CompactionSegmentIt + timeline.findNonOvershadowedObjectsInInterval(skipInterval, Partitions.ONLY_COMPLETE) + ); + if (!CollectionUtils.isNullOrEmpty(segments)) { +- final CompactionCandidate candidates = CompactionCandidate.from(segments); ++ final CompactionCandidate candidates = CompactionCandidate.from(segments, config.getSegmentGranularity()); + + final CompactionStatus reason; +- if (candidates.getUmbrellaInterval().overlaps(latestSkipInterval)) { ++ if (candidates.getCompactionInterval().overlaps(latestSkipInterval)) { + reason = CompactionStatus.skipped("skip offset from latest[%s]", skipOffset); + } else { + reason = CompactionStatus.skipped("interval locked by another task"); +@@ -371,7 +365,6 @@ public class DataSourceCompactibleSegmentIterator implements CompactionSegmentIt + + final CompactionCandidate candidatesWithStatus = candidates.withCurrentStatus(reason); + skippedSegments.add(candidatesWithStatus); +- statusTracker.onCompactionStatusComputed(candidatesWithStatus, config); + } + } + +diff --git a/server/src/main/java/org/apache/druid/server/compaction/PriorityBasedCompactionSegmentIterator.java b/server/src/main/java/org/apache/druid/server/compaction/PriorityBasedCompactionSegmentIterator.java +index 1e91df7e38..49d936fda0 100644 +--- a/server/src/main/java/org/apache/druid/server/compaction/PriorityBasedCompactionSegmentIterator.java ++++ b/server/src/main/java/org/apache/druid/server/compaction/PriorityBasedCompactionSegmentIterator.java +@@ -48,8 +48,7 @@ public class PriorityBasedCompactionSegmentIterator implements CompactionSegment + CompactionCandidateSearchPolicy searchPolicy, + Map compactionConfigs, + Map datasourceToTimeline, +- Map> skipIntervals, +- CompactionStatusTracker statusTracker ++ Map> skipIntervals + ) + { + this.queue = new PriorityQueue<>(searchPolicy::compareCandidates); +@@ -70,8 +69,7 @@ public class PriorityBasedCompactionSegmentIterator implements CompactionSegment + compactionConfigs.get(datasource), + timeline, + skipIntervals.getOrDefault(datasource, Collections.emptyList()), +- searchPolicy, +- statusTracker ++ searchPolicy + ) + ); + addNextItemForDatasourceToQueue(datasource); +diff --git a/server/src/main/java/org/apache/druid/server/coordinator/AutoCompactionSnapshot.java b/server/src/main/java/org/apache/druid/server/coordinator/AutoCompactionSnapshot.java +index e31a7919f2..b450dd5a50 100644 +--- a/server/src/main/java/org/apache/druid/server/coordinator/AutoCompactionSnapshot.java ++++ b/server/src/main/java/org/apache/druid/server/coordinator/AutoCompactionSnapshot.java +@@ -204,6 +204,25 @@ public class AutoCompactionSnapshot + ); + } + ++ @Override ++ public String toString() ++ { ++ return "AutoCompactionSnapshot{" + ++ "dataSource='" + dataSource + '\'' + ++ ", scheduleStatus=" + scheduleStatus + ++ ", message='" + message + '\'' + ++ ", bytesAwaitingCompaction=" + bytesAwaitingCompaction + ++ ", bytesCompacted=" + bytesCompacted + ++ ", bytesSkipped=" + bytesSkipped + ++ ", segmentCountAwaitingCompaction=" + segmentCountAwaitingCompaction + ++ ", segmentCountCompacted=" + segmentCountCompacted + ++ ", segmentCountSkipped=" + segmentCountSkipped + ++ ", intervalCountAwaitingCompaction=" + intervalCountAwaitingCompaction + ++ ", intervalCountCompacted=" + intervalCountCompacted + ++ ", intervalCountSkipped=" + intervalCountSkipped + ++ '}'; ++ } ++ + public static class Builder + { + private final String dataSource; +@@ -241,6 +260,11 @@ public class AutoCompactionSnapshot + waitingStats.increment(entry); + } + ++ public void decrementWaitingStats(CompactionStatistics entry) ++ { ++ waitingStats.decrement(entry); ++ } ++ + public void incrementCompactedStats(CompactionStatistics entry) + { + compactedStats.increment(entry); +diff --git a/server/src/main/java/org/apache/druid/server/coordinator/UserCompactionTaskGranularityConfig.java b/server/src/main/java/org/apache/druid/server/coordinator/UserCompactionTaskGranularityConfig.java +index be3c7e1af9..4ad57f3adf 100644 +--- a/server/src/main/java/org/apache/druid/server/coordinator/UserCompactionTaskGranularityConfig.java ++++ b/server/src/main/java/org/apache/druid/server/coordinator/UserCompactionTaskGranularityConfig.java +@@ -20,10 +20,12 @@ + package org.apache.druid.server.coordinator; + + import com.fasterxml.jackson.annotation.JsonCreator; ++import com.fasterxml.jackson.annotation.JsonInclude; + import com.fasterxml.jackson.annotation.JsonProperty; + import org.apache.druid.indexer.granularity.GranularitySpec; + import org.apache.druid.java.util.common.granularity.Granularity; + ++import javax.annotation.Nullable; + import java.util.Objects; + + /** +@@ -35,6 +37,7 @@ import java.util.Objects; + * Note that this class is not the same as {@link GranularitySpec}. This class simply holds Granularity configs + * and pass it to compaction task spec. This class does not do bucketing, group events or knows how to partition data. + */ ++@JsonInclude(JsonInclude.Include.NON_NULL) + public class UserCompactionTaskGranularityConfig + { + private final Granularity segmentGranularity; +@@ -71,6 +74,20 @@ public class UserCompactionTaskGranularityConfig + return rollup; + } + ++ @Nullable ++ public static UserCompactionTaskGranularityConfig from(GranularitySpec granularitySpec) ++ { ++ if (granularitySpec == null) { ++ return null; ++ } else { ++ return new UserCompactionTaskGranularityConfig( ++ granularitySpec.getSegmentGranularity(), ++ granularitySpec.getQueryGranularity(), ++ granularitySpec.isRollup() ++ ); ++ } ++ } ++ + @Override + public boolean equals(Object o) + { +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 135852c1de..1dca23090c 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 +@@ -23,7 +23,6 @@ import com.fasterxml.jackson.annotation.JacksonInject; + import com.fasterxml.jackson.annotation.JsonCreator; + import com.google.common.annotations.VisibleForTesting; + import com.google.common.base.Preconditions; +-import com.google.common.base.Predicate; + import org.apache.druid.client.DataSourcesSnapshot; + import org.apache.druid.client.indexing.ClientCompactionIOConfig; + import org.apache.druid.client.indexing.ClientCompactionIntervalSpec; +@@ -32,26 +31,22 @@ import org.apache.druid.client.indexing.ClientCompactionTaskDimensionsSpec; + import org.apache.druid.client.indexing.ClientCompactionTaskGranularitySpec; + import org.apache.druid.client.indexing.ClientCompactionTaskQuery; + import org.apache.druid.client.indexing.ClientCompactionTaskQueryTuningConfig; +-import org.apache.druid.client.indexing.ClientMSQContext; +-import org.apache.druid.client.indexing.TaskPayloadResponse; + 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.indexer.CompactionEngine; +-import org.apache.druid.indexer.TaskStatus; +-import org.apache.druid.indexer.TaskStatusPlus; +-import org.apache.druid.indexer.partitions.DimensionRangePartitionsSpec; +-import org.apache.druid.java.util.common.ISE; + 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.metadata.LockFilterPolicy; + 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.CompactionSegmentIterator; ++import org.apache.druid.server.compaction.CompactionSlotManager; ++import org.apache.druid.server.compaction.CompactionSnapshotBuilder; ++import org.apache.druid.server.compaction.CompactionStatus; + import org.apache.druid.server.compaction.CompactionStatusTracker; + import org.apache.druid.server.compaction.PriorityBasedCompactionSegmentIterator; + import org.apache.druid.server.coordinator.AutoCompactionSnapshot; +@@ -66,10 +61,8 @@ import org.apache.druid.timeline.DataSegment; + import org.joda.time.Interval; + + import javax.annotation.Nullable; +-import java.util.ArrayList; + import java.util.Collections; + import java.util.HashMap; +-import java.util.HashSet; + import java.util.List; + import java.util.Map; + import java.util.Set; +@@ -79,21 +72,16 @@ import java.util.stream.Collectors; + + public class CompactSegments implements CoordinatorCustomDuty + { +- /** +- * Must be the same as org.apache.druid.indexing.common.task.CompactionTask.TYPE. +- */ +- public static final String COMPACTION_TASK_TYPE = "compact"; + /** + * Must be the same as org.apache.druid.indexing.common.task.Tasks.STORE_COMPACTION_STATE_KEY + */ + public static final String STORE_COMPACTION_STATE_KEY = "storeCompactionState"; ++ public static final String COMPACTION_INTERVAL_KEY = "compactionInterval"; + private static final String COMPACTION_REASON_KEY = "compactionReason"; + + private static final Logger LOG = new Logger(CompactSegments.class); + + private static final String TASK_ID_PREFIX = "coordinator-issued"; +- private static final Predicate IS_COMPACTION_TASK = +- status -> null != status && COMPACTION_TASK_TYPE.equals(status.getType()); + + private final CompactionStatusTracker statusTracker; + private final OverlordClient overlordClient; +@@ -152,75 +140,36 @@ public class CompactSegments implements CoordinatorCustomDuty + } + + statusTracker.onSegmentTimelineUpdated(dataSources.getSnapshotTime()); +- statusTracker.onCompactionConfigUpdated(dynamicConfig); + List compactionConfigList = dynamicConfig.getCompactionConfigs(); + if (compactionConfigList == null || compactionConfigList.isEmpty()) { + resetCompactionSnapshot(); ++ statusTracker.resetActiveDatasources(Set.of()); + return; + } + + Map compactionConfigs = compactionConfigList + .stream() + .collect(Collectors.toMap(DataSourceCompactionConfig::getDataSource, Function.identity())); ++ statusTracker.resetActiveDatasources(compactionConfigs.keySet()); + +- // Map from dataSource to list of intervals for which compaction will be skipped in this run +- final Map> intervalsToSkipCompaction = new HashMap<>(); +- +- // Fetch currently running compaction tasks +- int busyCompactionTaskSlots = 0; +- final List compactionTasks = CoordinatorDutyUtils.getStatusOfActiveTasks( ++ final CompactionSlotManager slotManager = new CompactionSlotManager( + overlordClient, +- IS_COMPACTION_TASK ++ statusTracker, ++ dynamicConfig.clusterConfig() + ); ++ stats.add(Stats.Compaction.MAX_SLOTS, slotManager.getNumAvailableTaskSlots()); + +- final Set activeTaskIds +- = compactionTasks.stream().map(TaskStatusPlus::getId).collect(Collectors.toSet()); +- trackStatusOfCompletedTasks(activeTaskIds); +- +- for (TaskStatusPlus status : compactionTasks) { +- final TaskPayloadResponse response = +- FutureUtils.getUnchecked(overlordClient.taskPayload(status.getId()), true); +- if (response == null) { +- throw new ISE("Could not find payload for active compaction task[%s]", status.getId()); +- } else if (!COMPACTION_TASK_TYPE.equals(response.getPayload().getType())) { +- throw new ISE( +- "Payload of active compaction task[%s] is of invalid type[%s]", +- status.getId(), response.getPayload().getType() +- ); +- } +- +- final ClientCompactionTaskQuery compactionTaskQuery = (ClientCompactionTaskQuery) response.getPayload(); +- DataSourceCompactionConfig dataSourceCompactionConfig = compactionConfigs.get(status.getDataSource()); +- if (cancelTaskIfGranularityChanged(compactionTaskQuery, dataSourceCompactionConfig)) { +- continue; +- } +- +- // Skip this interval as the current active compaction task is good +- final Interval interval = compactionTaskQuery.getIoConfig().getInputSpec().getInterval(); +- intervalsToSkipCompaction.computeIfAbsent(status.getDataSource(), k -> new ArrayList<>()) +- .add(interval); +- // Note: The default compactionRunnerType used here should match the default runner used in CompactionTask when +- // no runner is provided there. +- CompactionEngine compactionRunnerType = compactionTaskQuery.getCompactionRunner() == null +- ? CompactionEngine.NATIVE +- : compactionTaskQuery.getCompactionRunner().getType(); +- if (compactionRunnerType == CompactionEngine.NATIVE) { +- busyCompactionTaskSlots += +- findMaxNumTaskSlotsUsedByOneNativeCompactionTask(compactionTaskQuery.getTuningConfig()); +- } else { +- busyCompactionTaskSlots += findMaxNumTaskSlotsUsedByOneMsqCompactionTask(compactionTaskQuery.getContext()); ++ // Fetch currently running compaction tasks ++ for (ClientCompactionTaskQuery compactionTaskQuery : slotManager.fetchRunningCompactionTasks()) { ++ final String dataSource = compactionTaskQuery.getDataSource(); ++ DataSourceCompactionConfig dataSourceCompactionConfig = compactionConfigs.get(dataSource); ++ if (slotManager.cancelTaskOnlyIfGranularityChanged(compactionTaskQuery, dataSourceCompactionConfig)) { ++ stats.add(Stats.Compaction.CANCELLED_TASKS, RowKey.of(Dimension.DATASOURCE, dataSource), 1L); + } + } ++ stats.add(Stats.Compaction.AVAILABLE_SLOTS, slotManager.getNumAvailableTaskSlots()); + +- // Skip all the intervals locked by higher priority tasks for each datasource +- // This must be done after the invalid compaction tasks are cancelled +- // in the loop above so that their intervals are not considered locked +- getLockedIntervals(compactionConfigList).forEach( +- (dataSource, intervals) -> +- intervalsToSkipCompaction +- .computeIfAbsent(dataSource, ds -> new ArrayList<>()) +- .addAll(intervals) +- ); ++ slotManager.skipLockedIntervals(compactionConfigList); + + // Get iterator over segments to compact and submit compaction tasks + final CompactionCandidateSearchPolicy policy = dynamicConfig.getCompactionPolicy(); +@@ -228,27 +177,21 @@ public class CompactSegments implements CoordinatorCustomDuty + policy, + compactionConfigs, + dataSources.getUsedSegmentsTimelinesPerDataSource(), +- intervalsToSkipCompaction, +- statusTracker ++ slotManager.getDatasourceIntervalsToSkipCompaction() + ); + +- final int compactionTaskCapacity = getCompactionTaskCapacity(dynamicConfig); +- final int availableCompactionTaskSlots +- = getAvailableCompactionTaskSlots(compactionTaskCapacity, busyCompactionTaskSlots); +- +- final Map currentRunAutoCompactionSnapshotBuilders = new HashMap<>(); ++ final CompactionSnapshotBuilder compactionSnapshotBuilder = new CompactionSnapshotBuilder(stats); + final int numSubmittedCompactionTasks = submitCompactionTasks( + compactionConfigs, +- currentRunAutoCompactionSnapshotBuilders, +- availableCompactionTaskSlots, ++ compactionSnapshotBuilder, ++ slotManager, + iterator, ++ policy, + defaultEngine + ); + +- stats.add(Stats.Compaction.MAX_SLOTS, compactionTaskCapacity); +- stats.add(Stats.Compaction.AVAILABLE_SLOTS, availableCompactionTaskSlots); + stats.add(Stats.Compaction.SUBMITTED_TASKS, numSubmittedCompactionTasks); +- updateCompactionSnapshotStats(currentRunAutoCompactionSnapshotBuilders, iterator, stats); ++ updateCompactionSnapshotStats(compactionSnapshotBuilder, iterator, compactionConfigs); + } + + private void resetCompactionSnapshot() +@@ -271,334 +214,168 @@ public class CompactSegments implements CoordinatorCustomDuty + } + } + +- /** +- * Queries the Overlord for the status of all tasks that were submitted +- * recently but are not active anymore. The statuses are then updated in the +- * {@link #statusTracker}. +- */ +- private void trackStatusOfCompletedTasks(Set activeTaskIds) +- { +- final Set finishedTaskIds = new HashSet<>(statusTracker.getSubmittedTaskIds()); +- finishedTaskIds.removeAll(activeTaskIds); +- +- if (finishedTaskIds.isEmpty()) { +- return; +- } +- +- final Map taskStatusMap +- = FutureUtils.getUnchecked(overlordClient.taskStatuses(finishedTaskIds), true); +- for (String taskId : finishedTaskIds) { +- // Assume unknown task to have finished successfully +- final TaskStatus taskStatus = taskStatusMap.getOrDefault(taskId, TaskStatus.success(taskId)); +- if (taskStatus.isComplete()) { +- statusTracker.onTaskFinished(taskId, taskStatus); +- } +- } +- } +- +- /** +- * Cancels a currently running compaction task if the segment granularity +- * for this datasource has changed in the compaction config. +- * +- * @return true if the task was canceled, false otherwise. +- */ +- private boolean cancelTaskIfGranularityChanged( +- ClientCompactionTaskQuery compactionTaskQuery, +- DataSourceCompactionConfig dataSourceCompactionConfig +- ) +- { +- if (dataSourceCompactionConfig == null +- || dataSourceCompactionConfig.getGranularitySpec() == null +- || compactionTaskQuery.getGranularitySpec() == null) { +- return false; +- } +- +- Granularity configuredSegmentGranularity = dataSourceCompactionConfig.getGranularitySpec() +- .getSegmentGranularity(); +- Granularity taskSegmentGranularity = compactionTaskQuery.getGranularitySpec().getSegmentGranularity(); +- if (configuredSegmentGranularity == null || configuredSegmentGranularity.equals(taskSegmentGranularity)) { +- return false; +- } +- +- LOG.info( +- "Cancelling task[%s] as task segmentGranularity[%s] differs from compaction config segmentGranularity[%s].", +- compactionTaskQuery.getId(), taskSegmentGranularity, configuredSegmentGranularity +- ); +- overlordClient.cancelTask(compactionTaskQuery.getId()); +- return true; +- } +- +- /** +- * Gets a List of Intervals locked by higher priority tasks for each datasource. +- * However, when using a REPLACE lock for compaction, intervals locked with any APPEND lock will not be returned +- * Since compaction tasks submitted for these Intervals would have to wait anyway, +- * we skip these Intervals until the next compaction run. +- *

+- * For now, Segment Locks are being treated the same as Time Chunk Locks even +- * though they lock only a Segment and not the entire Interval. Thus, +- * a compaction task will not be submitted for an Interval if +- *

    +- *
  • either the whole Interval is locked by a higher priority Task with an incompatible lock type
  • +- *
  • or there is atleast one Segment in the Interval that is locked by a +- * higher priority Task
  • +- *
+- */ +- private Map> getLockedIntervals( +- List compactionConfigs +- ) +- { +- final List lockFilterPolicies = compactionConfigs +- .stream() +- .map(config -> +- new LockFilterPolicy(config.getDataSource(), config.getTaskPriority(), null, config.getTaskContext())) +- .collect(Collectors.toList()); +- final Map> datasourceToLockedIntervals = +- new HashMap<>(FutureUtils.getUnchecked(overlordClient.findLockedIntervals(lockFilterPolicies), true)); +- LOG.debug( +- "Skipping the following intervals for Compaction as they are currently locked: %s", +- datasourceToLockedIntervals +- ); +- +- return datasourceToLockedIntervals; +- } +- +- /** +- * Returns the maximum number of task slots used by one native compaction task at any time when the task is +- * issued with the given tuningConfig. +- */ +- public static int findMaxNumTaskSlotsUsedByOneNativeCompactionTask( +- @Nullable ClientCompactionTaskQueryTuningConfig tuningConfig +- ) +- { +- if (isParallelMode(tuningConfig)) { +- @Nullable +- Integer maxNumConcurrentSubTasks = tuningConfig.getMaxNumConcurrentSubTasks(); +- // Max number of task slots used in parallel mode = maxNumConcurrentSubTasks + 1 (supervisor task) +- return (maxNumConcurrentSubTasks == null ? 1 : maxNumConcurrentSubTasks) + 1; +- } else { +- return 1; +- } +- } +- +- /** +- * Returns the maximum number of task slots used by one MSQ compaction task at any time when the task is +- * issued with the given context. +- */ +- static int findMaxNumTaskSlotsUsedByOneMsqCompactionTask(@Nullable Map context) +- { +- return context == null +- ? ClientMSQContext.DEFAULT_MAX_NUM_TASKS +- : (int) context.getOrDefault(ClientMSQContext.CTX_MAX_NUM_TASKS, ClientMSQContext.DEFAULT_MAX_NUM_TASKS); +- } +- +- +- /** +- * Returns true if the compaction task can run in the parallel mode with the given tuningConfig. +- * This method should be synchronized with ParallelIndexSupervisorTask.isParallelMode(InputSource, ParallelIndexTuningConfig). +- */ +- @VisibleForTesting +- static boolean isParallelMode(@Nullable ClientCompactionTaskQueryTuningConfig tuningConfig) +- { +- if (null == tuningConfig) { +- return false; +- } +- boolean useRangePartitions = useRangePartitions(tuningConfig); +- int minRequiredNumConcurrentSubTasks = useRangePartitions ? 1 : 2; +- return tuningConfig.getMaxNumConcurrentSubTasks() != null +- && tuningConfig.getMaxNumConcurrentSubTasks() >= minRequiredNumConcurrentSubTasks; +- } +- +- private static boolean useRangePartitions(ClientCompactionTaskQueryTuningConfig tuningConfig) +- { +- // dynamic partitionsSpec will be used if getPartitionsSpec() returns null +- return tuningConfig.getPartitionsSpec() instanceof DimensionRangePartitionsSpec; +- } +- +- private int getCompactionTaskCapacity(DruidCompactionConfig dynamicConfig) +- { +- int totalWorkerCapacity = CoordinatorDutyUtils.getTotalWorkerCapacity(overlordClient); +- +- return Math.min( +- (int) (totalWorkerCapacity * dynamicConfig.getCompactionTaskSlotRatio()), +- dynamicConfig.getMaxCompactionTaskSlots() +- ); +- } +- +- private int getAvailableCompactionTaskSlots(int compactionTaskCapacity, int busyCompactionTaskSlots) +- { +- final int availableCompactionTaskSlots; +- if (busyCompactionTaskSlots > 0) { +- availableCompactionTaskSlots = Math.max(0, compactionTaskCapacity - busyCompactionTaskSlots); +- } else { +- // compactionTaskCapacity might be 0 if totalWorkerCapacity is low. +- // This guarantees that at least one slot is available if +- // compaction is enabled and estimatedIncompleteCompactionTasks is 0. +- availableCompactionTaskSlots = Math.max(1, compactionTaskCapacity); +- } +- LOG.debug( +- "Found [%d] available task slots for compaction out of max compaction task capacity [%d]", +- availableCompactionTaskSlots, compactionTaskCapacity +- ); +- +- return availableCompactionTaskSlots; +- } +- + /** + * Submits compaction tasks to the Overlord. Returns total number of tasks submitted. + */ + private int submitCompactionTasks( + Map compactionConfigs, +- Map currentRunAutoCompactionSnapshotBuilders, +- int numAvailableCompactionTaskSlots, ++ CompactionSnapshotBuilder snapshotBuilder, ++ CompactionSlotManager slotManager, + CompactionSegmentIterator iterator, ++ CompactionCandidateSearchPolicy policy, + CompactionEngine defaultEngine + ) + { +- if (numAvailableCompactionTaskSlots <= 0) { ++ if (slotManager.getNumAvailableTaskSlots() <= 0) { + return 0; + } + + int numSubmittedTasks = 0; + int totalTaskSlotsAssigned = 0; + +- while (iterator.hasNext() && totalTaskSlotsAssigned < numAvailableCompactionTaskSlots) { ++ while (iterator.hasNext() && totalTaskSlotsAssigned < slotManager.getNumAvailableTaskSlots()) { + final CompactionCandidate entry = iterator.next(); + final String dataSourceName = entry.getDataSource(); ++ final DataSourceCompactionConfig config = compactionConfigs.get(dataSourceName); + +- // As these segments will be compacted, we will aggregate the statistic to the Compacted statistics +- currentRunAutoCompactionSnapshotBuilders +- .computeIfAbsent(dataSourceName, AutoCompactionSnapshot::builder) +- .incrementCompactedStats(entry.getStats()); ++ final CompactionStatus compactionStatus = ++ statusTracker.computeCompactionStatus(entry, policy); ++ final CompactionCandidate candidatesWithStatus = entry.withCurrentStatus(compactionStatus); ++ statusTracker.onCompactionStatusComputed(candidatesWithStatus, config); + +- final DataSourceCompactionConfig config = compactionConfigs.get(dataSourceName); +- final List segmentsToCompact = entry.getSegments(); +- // ^ feed segmentsToCompact to config to get grouping of configs to segments +- +- // Create granularitySpec to send to compaction task +- Granularity segmentGranularityToUse = null; +- if (config.getGranularitySpec() == null || config.getGranularitySpec().getSegmentGranularity() == null) { +- // Determines segmentGranularity from the segmentsToCompact +- // Each batch of segmentToCompact from CompactionSegmentIterator will contain the same interval as +- // segmentGranularity is not set in the compaction config +- Interval interval = segmentsToCompact.get(0).getInterval(); +- if (segmentsToCompact.stream().allMatch(segment -> interval.overlaps(segment.getInterval()))) { +- try { +- segmentGranularityToUse = GranularityType.fromPeriod(interval.toPeriod()).getDefaultGranularity(); +- } +- catch (IllegalArgumentException iae) { +- // This case can happen if the existing segment interval result in complicated periods. +- // Fall back to setting segmentGranularity as null +- LOG.warn("Cannot determine segmentGranularity from interval[%s].", interval); +- } +- } else { +- LOG.warn( +- "Not setting 'segmentGranularity' for auto-compaction task as" +- + " the segments to compact do not have the same interval." +- ); +- } ++ if (compactionStatus.isComplete()) { ++ snapshotBuilder.addToComplete(candidatesWithStatus); ++ } else if (compactionStatus.isSkipped()) { ++ snapshotBuilder.addToSkipped(candidatesWithStatus); + } else { +- segmentGranularityToUse = config.getGranularitySpec().getSegmentGranularity(); ++ // As these segments will be compacted, we will aggregate the statistic to the Compacted statistics ++ snapshotBuilder.addToComplete(entry); + } +- final ClientCompactionTaskGranularitySpec granularitySpec = new ClientCompactionTaskGranularitySpec( +- segmentGranularityToUse, +- config.getGranularitySpec() != null ? config.getGranularitySpec().getQueryGranularity() : null, +- config.getGranularitySpec() != null ? config.getGranularitySpec().isRollup() : null +- ); + +- // Create dimensionsSpec to send to compaction task +- final ClientCompactionTaskDimensionsSpec dimensionsSpec; +- if (config.getDimensionsSpec() != null) { +- dimensionsSpec = new ClientCompactionTaskDimensionsSpec( +- config.getDimensionsSpec().getDimensions() +- ); +- } else { +- dimensionsSpec = null; +- } ++ final ClientCompactionTaskQuery taskPayload = createCompactionTask(entry, config, defaultEngine); + +- Boolean dropExisting = null; +- if (config.getIoConfig() != null) { +- dropExisting = config.getIoConfig().isDropExisting(); +- } ++ final String taskId = taskPayload.getId(); ++ FutureUtils.getUnchecked(overlordClient.runTask(taskId, taskPayload), true); ++ statusTracker.onTaskSubmitted(taskId, entry); + +- // If all the segments found to be compacted are tombstones then dropExisting +- // needs to be forced to true. This forcing needs to happen in the case that +- // the flag is null, or it is false. It is needed when it is null to avoid the +- // possibility of the code deciding to default it to false later. +- // Forcing the flag to true will enable the task ingestion code to generate new, compacted, tombstones to +- // cover the tombstones found to be compacted as well as to mark them +- // as compacted (update their lastCompactionState). If we don't force the +- // flag then every time this compact duty runs it will find the same tombstones +- // in the interval since their lastCompactionState +- // was not set repeating this over and over and the duty will not make progress; it +- // will become stuck on this set of tombstones. +- // This forcing code should be revised +- // when/if the autocompaction code policy to decide which segments to compact changes +- if (dropExisting == null || !dropExisting) { +- if (segmentsToCompact.stream().allMatch(DataSegment::isTombstone)) { +- dropExisting = true; +- LOG.info("Forcing dropExisting to true since all segments to compact are tombstones."); +- } +- } ++ LOG.debug( ++ "Submitted a compaction task[%s] for [%d] segments in datasource[%s], umbrella interval[%s].", ++ taskId, entry.numSegments(), dataSourceName, entry.getUmbrellaInterval() ++ ); ++ LOG.debugSegments(entry.getSegments(), "Compacting segments"); ++ numSubmittedTasks++; ++ totalTaskSlotsAssigned += slotManager.computeSlotsRequiredForTask(taskPayload, config); ++ } + +- final CompactionEngine compactionEngine = config.getEngine() == null ? defaultEngine : config.getEngine(); +- final Map autoCompactionContext = newAutoCompactionContext(config.getTaskContext()); +- int slotsRequiredForCurrentTask; +- +- if (compactionEngine == CompactionEngine.MSQ) { +- if (autoCompactionContext.containsKey(ClientMSQContext.CTX_MAX_NUM_TASKS)) { +- slotsRequiredForCurrentTask = (int) autoCompactionContext.get(ClientMSQContext.CTX_MAX_NUM_TASKS); +- } else { +- // Since MSQ needs all task slots for the calculated #tasks to be available upfront, allot all available +- // compaction slots (upto a max of MAX_TASK_SLOTS_FOR_MSQ_COMPACTION) to current compaction task to avoid +- // stalling. Setting "taskAssignment" to "auto" has the problem of not being able to determine the actual +- // count, which is required for subsequent tasks. +- slotsRequiredForCurrentTask = Math.min( +- // Update the slots to 2 (min required for MSQ) if only 1 slot is available. +- numAvailableCompactionTaskSlots == 1 ? 2 : numAvailableCompactionTaskSlots, +- ClientMSQContext.MAX_TASK_SLOTS_FOR_MSQ_COMPACTION_TASK +- ); +- autoCompactionContext.put(ClientMSQContext.CTX_MAX_NUM_TASKS, slotsRequiredForCurrentTask); ++ LOG.info("Submitted a total of [%d] compaction tasks.", numSubmittedTasks); ++ return numSubmittedTasks; ++ } ++ ++ /** ++ * Creates a {@link ClientCompactionTaskQuery} which can be submitted to an ++ * {@link OverlordClient} to start a compaction task. ++ */ ++ public static ClientCompactionTaskQuery createCompactionTask( ++ CompactionCandidate candidate, ++ DataSourceCompactionConfig config, ++ CompactionEngine defaultEngine ++ ) ++ { ++ final List segmentsToCompact = candidate.getSegments(); ++ ++ // Create granularitySpec to send to compaction task ++ Granularity segmentGranularityToUse = null; ++ if (config.getGranularitySpec() == null || config.getGranularitySpec().getSegmentGranularity() == null) { ++ // Determines segmentGranularity from the segmentsToCompact ++ // Each batch of segmentToCompact from CompactionSegmentIterator will contain the same interval as ++ // segmentGranularity is not set in the compaction config ++ Interval interval = segmentsToCompact.get(0).getInterval(); ++ if (segmentsToCompact.stream().allMatch(segment -> interval.overlaps(segment.getInterval()))) { ++ try { ++ segmentGranularityToUse = GranularityType.fromPeriod(interval.toPeriod()).getDefaultGranularity(); ++ } ++ catch (IllegalArgumentException iae) { ++ // This case can happen if the existing segment interval result in complicated periods. ++ // Fall back to setting segmentGranularity as null ++ LOG.warn("Cannot determine segmentGranularity from interval[%s].", interval); + } + } else { +- slotsRequiredForCurrentTask = findMaxNumTaskSlotsUsedByOneNativeCompactionTask(config.getTuningConfig()); ++ LOG.warn( ++ "Not setting 'segmentGranularity' for auto-compaction task as" ++ + " the segments to compact do not have the same interval." ++ ); + } ++ } else { ++ segmentGranularityToUse = config.getGranularitySpec().getSegmentGranularity(); ++ } ++ final ClientCompactionTaskGranularitySpec granularitySpec = new ClientCompactionTaskGranularitySpec( ++ segmentGranularityToUse, ++ config.getGranularitySpec() != null ? config.getGranularitySpec().getQueryGranularity() : null, ++ config.getGranularitySpec() != null ? config.getGranularitySpec().isRollup() : null ++ ); ++ ++ // Create dimensionsSpec to send to compaction task ++ final ClientCompactionTaskDimensionsSpec dimensionsSpec; ++ if (config.getDimensionsSpec() != null) { ++ dimensionsSpec = new ClientCompactionTaskDimensionsSpec( ++ config.getDimensionsSpec().getDimensions() ++ ); ++ } else { ++ dimensionsSpec = null; ++ } + +- if (entry.getCurrentStatus() != null) { +- autoCompactionContext.put(COMPACTION_REASON_KEY, entry.getCurrentStatus().getReason()); ++ Boolean dropExisting = null; ++ if (config.getIoConfig() != null) { ++ dropExisting = config.getIoConfig().isDropExisting(); ++ } ++ ++ // If all the segments found to be compacted are tombstones then dropExisting ++ // needs to be forced to true. This forcing needs to happen in the case that ++ // the flag is null, or it is false. It is needed when it is null to avoid the ++ // possibility of the code deciding to default it to false later. ++ // Forcing the flag to true will enable the task ingestion code to generate new, compacted, tombstones to ++ // cover the tombstones found to be compacted as well as to mark them ++ // as compacted (update their lastCompactionState). If we don't force the ++ // flag then every time this compact duty runs it will find the same tombstones ++ // in the interval since their lastCompactionState ++ // was not set repeating this over and over and the duty will not make progress; it ++ // will become stuck on this set of tombstones. ++ // This forcing code should be revised ++ // when/if the autocompaction code policy to decide which segments to compact changes ++ if (dropExisting == null || !dropExisting) { ++ if (segmentsToCompact.stream().allMatch(DataSegment::isTombstone)) { ++ dropExisting = true; ++ LOG.info("Forcing dropExisting to true since all segments to compact are tombstones."); + } ++ } + +- final String taskId = compactSegments( +- entry, +- config.getTaskPriority(), +- ClientCompactionTaskQueryTuningConfig.from( +- config.getTuningConfig(), +- config.getMaxRowsPerSegment(), +- config.getMetricsSpec() != null +- ), +- granularitySpec, +- dimensionsSpec, +- config.getMetricsSpec(), +- config.getTransformSpec(), +- config.getProjections(), +- dropExisting, +- autoCompactionContext, +- new ClientCompactionRunnerInfo(compactionEngine) +- ); ++ final CompactionEngine compactionEngine = config.getEngine() == null ? defaultEngine : config.getEngine(); ++ final Map autoCompactionContext = newAutoCompactionContext(config.getTaskContext()); + +- LOG.debug( +- "Submitted a compaction task[%s] for [%d] segments in datasource[%s], umbrella interval[%s].", +- taskId, segmentsToCompact.size(), dataSourceName, entry.getUmbrellaInterval() +- ); +- LOG.debugSegments(segmentsToCompact, "Compacting segments"); +- numSubmittedTasks++; +- totalTaskSlotsAssigned += slotsRequiredForCurrentTask; ++ if (candidate.getCurrentStatus() != null) { ++ autoCompactionContext.put(COMPACTION_REASON_KEY, candidate.getCurrentStatus().getReason()); + } + +- LOG.info("Submitted a total of [%d] compaction tasks.", numSubmittedTasks); +- return numSubmittedTasks; ++ return compactSegments( ++ candidate, ++ config.getTaskPriority(), ++ ClientCompactionTaskQueryTuningConfig.from( ++ config.getTuningConfig(), ++ config.getMaxRowsPerSegment(), ++ config.getMetricsSpec() != null ++ ), ++ granularitySpec, ++ dimensionsSpec, ++ config.getMetricsSpec(), ++ config.getTransformSpec(), ++ config.getProjections(), ++ dropExisting, ++ autoCompactionContext, ++ new ClientCompactionRunnerInfo(compactionEngine) ++ ); + } + +- private Map newAutoCompactionContext(@Nullable Map configuredContext) ++ private static Map newAutoCompactionContext(@Nullable Map configuredContext) + { + final Map newContext = configuredContext == null + ? new HashMap<>() +@@ -608,60 +385,23 @@ public class CompactSegments implements CoordinatorCustomDuty + } + + private void updateCompactionSnapshotStats( +- Map currentRunAutoCompactionSnapshotBuilders, ++ CompactionSnapshotBuilder snapshotBuilder, + CompactionSegmentIterator iterator, +- CoordinatorRunStats stats ++ Map datasourceToConfig + ) + { + // Mark all the segments remaining in the iterator as "awaiting compaction" + while (iterator.hasNext()) { +- final CompactionCandidate entry = iterator.next(); +- currentRunAutoCompactionSnapshotBuilders +- .computeIfAbsent(entry.getDataSource(), AutoCompactionSnapshot::builder) +- .incrementWaitingStats(entry.getStats()); ++ snapshotBuilder.addToPending(iterator.next()); + } +- +- // Statistics of all segments considered compacted after this run +- iterator.getCompactedSegments().forEach( +- candidateSegments -> currentRunAutoCompactionSnapshotBuilders +- .computeIfAbsent(candidateSegments.getDataSource(), AutoCompactionSnapshot::builder) +- .incrementCompactedStats(candidateSegments.getStats()) +- ); +- +- // Statistics of all segments considered skipped after this run +- iterator.getSkippedSegments().forEach( +- candidateSegments -> currentRunAutoCompactionSnapshotBuilders +- .computeIfAbsent(candidateSegments.getDataSource(), AutoCompactionSnapshot::builder) +- .incrementSkippedStats(candidateSegments.getStats()) +- ); +- +- final Map currentAutoCompactionSnapshotPerDataSource = new HashMap<>(); +- currentRunAutoCompactionSnapshotBuilders.forEach((dataSource, builder) -> { +- final AutoCompactionSnapshot autoCompactionSnapshot = builder.build(); +- currentAutoCompactionSnapshotPerDataSource.put(dataSource, autoCompactionSnapshot); +- collectSnapshotStats(autoCompactionSnapshot, stats); ++ iterator.getCompactedSegments().forEach(snapshotBuilder::addToComplete); ++ iterator.getSkippedSegments().forEach(entry -> { ++ statusTracker.onCompactionStatusComputed(entry, datasourceToConfig.get(entry.getDataSource())); ++ snapshotBuilder.addToSkipped(entry); + }); + + // Atomic update of autoCompactionSnapshotPerDataSource with the latest from this coordinator run +- autoCompactionSnapshotPerDataSource.set(currentAutoCompactionSnapshotPerDataSource); +- } +- +- private void collectSnapshotStats( +- AutoCompactionSnapshot autoCompactionSnapshot, +- CoordinatorRunStats stats +- ) +- { +- final RowKey rowKey = RowKey.of(Dimension.DATASOURCE, autoCompactionSnapshot.getDataSource()); +- +- stats.add(Stats.Compaction.PENDING_BYTES, rowKey, autoCompactionSnapshot.getBytesAwaitingCompaction()); +- stats.add(Stats.Compaction.PENDING_SEGMENTS, rowKey, autoCompactionSnapshot.getSegmentCountAwaitingCompaction()); +- stats.add(Stats.Compaction.PENDING_INTERVALS, rowKey, autoCompactionSnapshot.getIntervalCountAwaitingCompaction()); +- stats.add(Stats.Compaction.COMPACTED_BYTES, rowKey, autoCompactionSnapshot.getBytesCompacted()); +- stats.add(Stats.Compaction.COMPACTED_SEGMENTS, rowKey, autoCompactionSnapshot.getSegmentCountCompacted()); +- stats.add(Stats.Compaction.COMPACTED_INTERVALS, rowKey, autoCompactionSnapshot.getIntervalCountCompacted()); +- stats.add(Stats.Compaction.SKIPPED_BYTES, rowKey, autoCompactionSnapshot.getBytesSkipped()); +- stats.add(Stats.Compaction.SKIPPED_SEGMENTS, rowKey, autoCompactionSnapshot.getSegmentCountSkipped()); +- stats.add(Stats.Compaction.SKIPPED_INTERVALS, rowKey, autoCompactionSnapshot.getIntervalCountSkipped()); ++ autoCompactionSnapshotPerDataSource.set(snapshotBuilder.build()); + } + + @Nullable +@@ -675,7 +415,7 @@ public class CompactSegments implements CoordinatorCustomDuty + return autoCompactionSnapshotPerDataSource.get(); + } + +- private String compactSegments( ++ private static ClientCompactionTaskQuery compactSegments( + CompactionCandidate entry, + int compactionTaskPriority, + ClientCompactionTaskQueryTuningConfig tuningConfig, +@@ -685,7 +425,7 @@ public class CompactSegments implements CoordinatorCustomDuty + @Nullable CompactionTransformSpec transformSpec, + @Nullable List projectionSpecs, + @Nullable Boolean dropExisting, +- @Nullable Map context, ++ Map context, + ClientCompactionRunnerInfo compactionRunner + ) + { +@@ -698,17 +438,15 @@ public class CompactSegments implements CoordinatorCustomDuty + "Segments must have the same dataSource" + ); + +- context = context == null ? new HashMap<>() : context; + context.put("priority", compactionTaskPriority); + + final String taskId = IdUtils.newTaskId(TASK_ID_PREFIX, ClientCompactionTaskQuery.TYPE, dataSource, null); +- final Granularity segmentGranularity = granularitySpec == null ? null : granularitySpec.getSegmentGranularity(); + +- final ClientCompactionTaskQuery taskPayload = new ClientCompactionTaskQuery( ++ return new ClientCompactionTaskQuery( + taskId, + dataSource, + new ClientCompactionIOConfig( +- ClientCompactionIntervalSpec.fromSegments(segments, segmentGranularity), ++ new ClientCompactionIntervalSpec(entry.getCompactionInterval(), null), + dropExisting + ), + tuningConfig, +@@ -720,9 +458,5 @@ public class CompactSegments implements CoordinatorCustomDuty + context, + compactionRunner + ); +- FutureUtils.getUnchecked(overlordClient.runTask(taskId, taskPayload), true); +- statusTracker.onTaskSubmitted(taskPayload, entry); +- +- return taskId; + } + } +diff --git a/server/src/main/java/org/apache/druid/server/coordinator/stats/CoordinatorRunStats.java b/server/src/main/java/org/apache/druid/server/coordinator/stats/CoordinatorRunStats.java +index 76d788cb43..a9473b298c 100644 +--- a/server/src/main/java/org/apache/druid/server/coordinator/stats/CoordinatorRunStats.java ++++ b/server/src/main/java/org/apache/druid/server/coordinator/stats/CoordinatorRunStats.java +@@ -95,6 +95,20 @@ public class CoordinatorRunStats + return statValues == null ? 0 : statValues.getLong(stat); + } + ++ /** ++ * Invokes the {@link StatHandler} for each value of the specified stat. ++ */ ++ public void forEachEntry(CoordinatorStat stat, StatHandler handler) ++ { ++ allStats.forEach( ++ (rowKey, stats) -> stats.object2LongEntrySet().fastForEach(entry -> { ++ if (entry.getKey().equals(stat)) { ++ handler.handle(stat, rowKey, entry.getLongValue()); ++ } ++ }) ++ ); ++ } ++ + public void forEachStat(StatHandler handler) + { + allStats.forEach( +diff --git a/server/src/main/java/org/apache/druid/server/coordinator/stats/Stats.java b/server/src/main/java/org/apache/druid/server/coordinator/stats/Stats.java +index f4ae1b5f8a..3851b38fc9 100644 +--- a/server/src/main/java/org/apache/druid/server/coordinator/stats/Stats.java ++++ b/server/src/main/java/org/apache/druid/server/coordinator/stats/Stats.java +@@ -110,8 +110,17 @@ public class Stats + + public static class Compaction + { ++ public static final CoordinatorStat JOB_CREATION_TIME ++ = CoordinatorStat.toDebugAndEmit("jobCreationTime", "compact/createJobs/time"); ++ public static final CoordinatorStat CREATED_JOBS ++ = CoordinatorStat.toDebugAndEmit("jobQueueSize", "compact/createJobs/count"); ++ public static final CoordinatorStat SCHEDULER_RUN_TIME ++ = CoordinatorStat.toDebugAndEmit("schedulerRunTime", "compact/runScheduler/time"); ++ + public static final CoordinatorStat SUBMITTED_TASKS + = CoordinatorStat.toDebugAndEmit("compactTasks", "compact/task/count"); ++ public static final CoordinatorStat CANCELLED_TASKS ++ = CoordinatorStat.toDebugAndEmit("compactCancelled", "compactTask/cancelled/count"); + public static final CoordinatorStat MAX_SLOTS + = CoordinatorStat.toDebugAndEmit("compactMaxSlots", "compactTask/maxSlot/count"); + public static final CoordinatorStat AVAILABLE_SLOTS +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 646e798aa6..46ecc64d72 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 +@@ -24,6 +24,7 @@ import org.apache.druid.java.util.common.DateTimes; + import org.apache.druid.java.util.common.Intervals; + import org.apache.druid.java.util.common.granularity.Granularities; + import org.apache.druid.segment.IndexIO; ++import org.apache.druid.server.compaction.CompactionCandidate; + import org.apache.druid.timeline.DataSegment; + import org.apache.druid.timeline.partition.NoneShardSpec; + import org.junit.Assert; +@@ -72,43 +73,43 @@ public class ClientCompactionIntervalSpecTest + public void testFromSegmentWithNoSegmentGranularity() + { + // The umbrella interval of segments is 2015-02-12/2015-04-14 +- ClientCompactionIntervalSpec actual = ClientCompactionIntervalSpec.fromSegments(ImmutableList.of(dataSegment1, dataSegment2, dataSegment3), null); +- Assert.assertEquals(Intervals.of("2015-02-12/2015-04-14"), actual.getInterval()); ++ CompactionCandidate actual = CompactionCandidate.from(ImmutableList.of(dataSegment1, dataSegment2, dataSegment3), null); ++ Assert.assertEquals(Intervals.of("2015-02-12/2015-04-14"), actual.getCompactionInterval()); + } + + @Test + public void testFromSegmentWitSegmentGranularitySameAsSegment() + { + // The umbrella interval of segments is 2015-04-11/2015-04-12 +- ClientCompactionIntervalSpec actual = ClientCompactionIntervalSpec.fromSegments(ImmutableList.of(dataSegment1), Granularities.DAY); +- Assert.assertEquals(Intervals.of("2015-04-11/2015-04-12"), actual.getInterval()); ++ CompactionCandidate actual = CompactionCandidate.from(ImmutableList.of(dataSegment1), Granularities.DAY); ++ Assert.assertEquals(Intervals.of("2015-04-11/2015-04-12"), actual.getCompactionInterval()); + } + + @Test + public void testFromSegmentWithCoarserSegmentGranularity() + { + // The umbrella interval of segments is 2015-02-12/2015-04-14 +- ClientCompactionIntervalSpec actual = ClientCompactionIntervalSpec.fromSegments(ImmutableList.of(dataSegment1, dataSegment2, dataSegment3), Granularities.YEAR); ++ CompactionCandidate actual = CompactionCandidate.from(ImmutableList.of(dataSegment1, dataSegment2, dataSegment3), Granularities.YEAR); + // The compaction interval should be expanded to start of the year and end of the year to cover the segmentGranularity +- Assert.assertEquals(Intervals.of("2015-01-01/2016-01-01"), actual.getInterval()); ++ Assert.assertEquals(Intervals.of("2015-01-01/2016-01-01"), actual.getCompactionInterval()); + } + + @Test + public void testFromSegmentWithFinerSegmentGranularityAndUmbrellaIntervalAlign() + { + // The umbrella interval of segments is 2015-02-12/2015-04-14 +- ClientCompactionIntervalSpec actual = ClientCompactionIntervalSpec.fromSegments(ImmutableList.of(dataSegment1, dataSegment2, dataSegment3), Granularities.DAY); ++ CompactionCandidate actual = CompactionCandidate.from(ImmutableList.of(dataSegment1, dataSegment2, dataSegment3), Granularities.DAY); + // The segmentGranularity of DAY align with the umbrella interval (umbrella interval can be evenly divide into the segmentGranularity) +- Assert.assertEquals(Intervals.of("2015-02-12/2015-04-14"), actual.getInterval()); ++ Assert.assertEquals(Intervals.of("2015-02-12/2015-04-14"), actual.getCompactionInterval()); + } + + @Test + public void testFromSegmentWithFinerSegmentGranularityAndUmbrellaIntervalNotAlign() + { + // The umbrella interval of segments is 2015-02-12/2015-04-14 +- ClientCompactionIntervalSpec actual = ClientCompactionIntervalSpec.fromSegments(ImmutableList.of(dataSegment1, dataSegment2, dataSegment3), Granularities.WEEK); ++ CompactionCandidate actual = CompactionCandidate.from(ImmutableList.of(dataSegment1, dataSegment2, dataSegment3), Granularities.WEEK); + // The segmentGranularity of WEEK does not align with the umbrella interval (umbrella interval cannot be evenly divide into the segmentGranularity) + // Hence the compaction interval is modified to aling with the segmentGranularity +- Assert.assertEquals(Intervals.of("2015-02-09/2015-04-20"), actual.getInterval()); ++ Assert.assertEquals(Intervals.of("2015-02-09/2015-04-20"), actual.getCompactionInterval()); + } + } +diff --git a/server/src/test/java/org/apache/druid/server/compaction/CompactionRunSimulatorTest.java b/server/src/test/java/org/apache/druid/server/compaction/CompactionRunSimulatorTest.java +index fd491ef930..bab2e351df 100644 +--- a/server/src/test/java/org/apache/druid/server/compaction/CompactionRunSimulatorTest.java ++++ b/server/src/test/java/org/apache/druid/server/compaction/CompactionRunSimulatorTest.java +@@ -19,14 +19,12 @@ + + package org.apache.druid.server.compaction; + +-import com.fasterxml.jackson.databind.ObjectMapper; + import com.google.common.util.concurrent.Futures; + import com.google.common.util.concurrent.ListenableFuture; + import org.apache.druid.client.indexing.TaskPayloadResponse; + import org.apache.druid.indexer.CompactionEngine; + import org.apache.druid.indexer.TaskStatus; + import org.apache.druid.indexer.TaskStatusPlus; +-import org.apache.druid.jackson.DefaultObjectMapper; + import org.apache.druid.java.util.common.CloseableIterators; + import org.apache.druid.java.util.common.Intervals; + import org.apache.druid.java.util.common.granularity.Granularities; +@@ -51,10 +49,8 @@ import java.util.Set; + + public class CompactionRunSimulatorTest + { +- private static final ObjectMapper OBJECT_MAPPER = new DefaultObjectMapper(); +- + private final CompactionRunSimulator simulator = new CompactionRunSimulator( +- new CompactionStatusTracker(OBJECT_MAPPER), ++ new CompactionStatusTracker(), + new TestOverlordClient() + ); + +@@ -115,7 +111,7 @@ public class CompactionRunSimulatorTest + ); + Assert.assertEquals( + Collections.singletonList( +- Arrays.asList("wiki", Intervals.of("2013-01-10/P1D"), 10, 1_000_000_000L, "skip offset from latest[P1D]") ++ Arrays.asList("wiki", Intervals.of("2013-01-10/P1D"), 10, 1_000_000_000L, 1, "skip offset from latest[P1D]") + ), + skippedTable.getRows() + ); +diff --git a/server/src/test/java/org/apache/druid/server/compaction/CompactionStatusTest.java b/server/src/test/java/org/apache/druid/server/compaction/CompactionStatusTest.java +index 14af1c78da..d201b84135 100644 +--- a/server/src/test/java/org/apache/druid/server/compaction/CompactionStatusTest.java ++++ b/server/src/test/java/org/apache/druid/server/compaction/CompactionStatusTest.java +@@ -19,7 +19,6 @@ + + package org.apache.druid.server.compaction; + +-import com.fasterxml.jackson.databind.ObjectMapper; + import org.apache.druid.client.indexing.ClientCompactionTaskQueryTuningConfig; + import org.apache.druid.data.input.impl.AggregateProjectionSpec; + import org.apache.druid.data.input.impl.DimensionsSpec; +@@ -31,7 +30,6 @@ import org.apache.druid.indexer.partitions.DimensionRangePartitionsSpec; + import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; + import org.apache.druid.indexer.partitions.HashedPartitionsSpec; + import org.apache.druid.indexer.partitions.PartitionsSpec; +-import org.apache.druid.jackson.DefaultObjectMapper; + import org.apache.druid.java.util.common.Intervals; + import org.apache.druid.java.util.common.granularity.Granularities; + import org.apache.druid.query.aggregation.LongSumAggregatorFactory; +@@ -47,6 +45,7 @@ import org.apache.druid.server.coordinator.UserCompactionTaskGranularityConfig; + import org.apache.druid.server.coordinator.UserCompactionTaskQueryTuningConfig; + import org.apache.druid.timeline.CompactionState; + import org.apache.druid.timeline.DataSegment; ++import org.apache.druid.timeline.SegmentId; + import org.junit.Assert; + import org.junit.Test; + +@@ -55,14 +54,9 @@ import java.util.List; + + public class CompactionStatusTest + { +- private static final ObjectMapper OBJECT_MAPPER = new DefaultObjectMapper(); +- + private static final DataSegment WIKI_SEGMENT +- = DataSegment.builder() +- .dataSource(TestDataSource.WIKI) +- .interval(Intervals.of("2013-01-01/PT1H")) ++ = DataSegment.builder(SegmentId.of(TestDataSource.WIKI, Intervals.of("2013-01-01/PT1H"), "v1", 0)) + .size(100_000_000L) +- .version("v1") + .build(); + + @Test +@@ -70,8 +64,7 @@ public class CompactionStatusTest + { + final ClientCompactionTaskQueryTuningConfig tuningConfig + = ClientCompactionTaskQueryTuningConfig.from(null); +- Assert.assertEquals( +- new DynamicPartitionsSpec(null, Long.MAX_VALUE), ++ Assert.assertNull( + CompactionStatus.findPartitionsSpecFromConfig(tuningConfig) + ); + } +@@ -203,9 +196,14 @@ public class CompactionStatusTest + @Test + public void testStatusWhenLastCompactionStateIsEmpty() + { ++ final PartitionsSpec requiredPartitionsSpec = new DynamicPartitionsSpec(5_000_000, null); + verifyCompactionStatusIsPendingBecause( + new CompactionState(null, null, null, null, null, null, null), +- InlineSchemaDataSourceCompactionConfig.builder().forDataSource(TestDataSource.WIKI).build(), ++ InlineSchemaDataSourceCompactionConfig ++ .builder() ++ .withTuningConfig(createTuningConfig(requiredPartitionsSpec, null)) ++ .forDataSource(TestDataSource.WIKI) ++ .build(), + "'partitionsSpec' mismatch: required['dynamic' with 5,000,000 rows], current[null]" + ); + } +@@ -213,12 +211,16 @@ public class CompactionStatusTest + @Test + public void testStatusOnPartitionsSpecMismatch() + { ++ final PartitionsSpec requiredPartitionsSpec = new DynamicPartitionsSpec(5_000_000, null); + final PartitionsSpec currentPartitionsSpec = new DynamicPartitionsSpec(100, null); + + final CompactionState lastCompactionState + = new CompactionState(currentPartitionsSpec, null, null, null, null, null, null); +- final DataSourceCompactionConfig compactionConfig +- = InlineSchemaDataSourceCompactionConfig.builder().forDataSource(TestDataSource.WIKI).build(); ++ final DataSourceCompactionConfig compactionConfig = InlineSchemaDataSourceCompactionConfig ++ .builder() ++ .withTuningConfig(createTuningConfig(requiredPartitionsSpec, null)) ++ .forDataSource(TestDataSource.WIKI) ++ .build(); + + verifyCompactionStatusIsPendingBecause( + lastCompactionState, +@@ -323,9 +325,8 @@ public class CompactionStatusTest + + final DataSegment segment = DataSegment.builder(WIKI_SEGMENT).lastCompactionState(lastCompactionState).build(); + final CompactionStatus status = CompactionStatus.compute( +- CompactionCandidate.from(Collections.singletonList(segment)), +- compactionConfig, +- OBJECT_MAPPER ++ CompactionCandidate.from(List.of(segment), Granularities.HOUR), ++ compactionConfig + ); + Assert.assertTrue(status.isComplete()); + } +@@ -373,9 +374,8 @@ public class CompactionStatusTest + + final DataSegment segment = DataSegment.builder(WIKI_SEGMENT).lastCompactionState(lastCompactionState).build(); + final CompactionStatus status = CompactionStatus.compute( +- CompactionCandidate.from(Collections.singletonList(segment)), +- compactionConfig, +- OBJECT_MAPPER ++ CompactionCandidate.from(List.of(segment), Granularities.HOUR), ++ compactionConfig + ); + Assert.assertTrue(status.isComplete()); + } +@@ -428,9 +428,8 @@ public class CompactionStatusTest + + final DataSegment segment = DataSegment.builder(WIKI_SEGMENT).lastCompactionState(lastCompactionState).build(); + final CompactionStatus status = CompactionStatus.compute( +- CompactionCandidate.from(Collections.singletonList(segment)), +- compactionConfig, +- OBJECT_MAPPER ++ CompactionCandidate.from(List.of(segment), Granularities.HOUR), ++ compactionConfig + ); + Assert.assertFalse(status.isComplete()); + } +@@ -482,9 +481,8 @@ public class CompactionStatusTest + + final DataSegment segment = DataSegment.builder(WIKI_SEGMENT).lastCompactionState(lastCompactionState).build(); + final CompactionStatus status = CompactionStatus.compute( +- CompactionCandidate.from(Collections.singletonList(segment)), +- compactionConfig, +- OBJECT_MAPPER ++ CompactionCandidate.from(List.of(segment), null), ++ compactionConfig + ); + Assert.assertTrue(status.isComplete()); + } +@@ -536,9 +534,8 @@ public class CompactionStatusTest + + final DataSegment segment = DataSegment.builder(WIKI_SEGMENT).lastCompactionState(lastCompactionState).build(); + final CompactionStatus status = CompactionStatus.compute( +- CompactionCandidate.from(Collections.singletonList(segment)), +- compactionConfig, +- OBJECT_MAPPER ++ CompactionCandidate.from(List.of(segment), null), ++ compactionConfig + ); + Assert.assertFalse(status.isComplete()); + } +@@ -554,9 +551,8 @@ public class CompactionStatusTest + .lastCompactionState(lastCompactionState) + .build(); + final CompactionStatus status = CompactionStatus.compute( +- CompactionCandidate.from(Collections.singletonList(segment)), +- compactionConfig, +- OBJECT_MAPPER ++ CompactionCandidate.from(List.of(segment), null), ++ compactionConfig + ); + + Assert.assertFalse(status.isComplete()); +diff --git a/server/src/test/java/org/apache/druid/server/compaction/CompactionStatusTrackerTest.java b/server/src/test/java/org/apache/druid/server/compaction/CompactionStatusTrackerTest.java +index 1878aab8c0..c0496b0705 100644 +--- a/server/src/test/java/org/apache/druid/server/compaction/CompactionStatusTrackerTest.java ++++ b/server/src/test/java/org/apache/druid/server/compaction/CompactionStatusTrackerTest.java +@@ -19,26 +19,20 @@ + + package org.apache.druid.server.compaction; + +-import com.fasterxml.jackson.databind.ObjectMapper; +-import org.apache.druid.client.indexing.ClientCompactionTaskQuery; + import org.apache.druid.indexer.TaskState; + import org.apache.druid.indexer.TaskStatus; +-import org.apache.druid.jackson.DefaultObjectMapper; + import org.apache.druid.java.util.common.DateTimes; + import org.apache.druid.segment.TestDataSource; + import org.apache.druid.server.coordinator.CreateDataSegments; +-import org.apache.druid.server.coordinator.DataSourceCompactionConfig; +-import org.apache.druid.server.coordinator.InlineSchemaDataSourceCompactionConfig; + import org.apache.druid.timeline.DataSegment; + import org.junit.Assert; + import org.junit.Before; + import org.junit.Test; + +-import java.util.Collections; ++import java.util.List; + + public class CompactionStatusTrackerTest + { +- private static final ObjectMapper MAPPER = new DefaultObjectMapper(); + private static final DataSegment WIKI_SEGMENT + = CreateDataSegments.ofDatasource(TestDataSource.WIKI).eachOfSizeInMb(100).get(0); + +@@ -47,15 +41,15 @@ public class CompactionStatusTrackerTest + @Before + public void setup() + { +- statusTracker = new CompactionStatusTracker(MAPPER); ++ statusTracker = new CompactionStatusTracker(); + } + + @Test + public void testGetLatestTaskStatusForSubmittedTask() + { + final CompactionCandidate candidateSegments +- = CompactionCandidate.from(Collections.singletonList(WIKI_SEGMENT)); +- statusTracker.onTaskSubmitted(createCompactionTask("task1"), candidateSegments); ++ = CompactionCandidate.from(List.of(WIKI_SEGMENT), null); ++ statusTracker.onTaskSubmitted("task1", candidateSegments); + + CompactionTaskStatus status = statusTracker.getLatestTaskStatus(candidateSegments); + Assert.assertEquals(TaskState.RUNNING, status.getState()); +@@ -65,8 +59,8 @@ public class CompactionStatusTrackerTest + public void testGetLatestTaskStatusForSuccessfulTask() + { + final CompactionCandidate candidateSegments +- = CompactionCandidate.from(Collections.singletonList(WIKI_SEGMENT)); +- statusTracker.onTaskSubmitted(createCompactionTask("task1"), candidateSegments); ++ = CompactionCandidate.from(List.of(WIKI_SEGMENT), null); ++ statusTracker.onTaskSubmitted("task1", candidateSegments); + statusTracker.onTaskFinished("task1", TaskStatus.success("task1")); + + CompactionTaskStatus status = statusTracker.getLatestTaskStatus(candidateSegments); +@@ -77,8 +71,8 @@ public class CompactionStatusTrackerTest + public void testGetLatestTaskStatusForFailedTask() + { + final CompactionCandidate candidateSegments +- = CompactionCandidate.from(Collections.singletonList(WIKI_SEGMENT)); +- statusTracker.onTaskSubmitted(createCompactionTask("task1"), candidateSegments); ++ = CompactionCandidate.from(List.of(WIKI_SEGMENT), null); ++ statusTracker.onTaskSubmitted("task1", candidateSegments); + statusTracker.onTaskFinished("task1", TaskStatus.failure("task1", "some failure")); + + CompactionTaskStatus status = statusTracker.getLatestTaskStatus(candidateSegments); +@@ -90,12 +84,12 @@ public class CompactionStatusTrackerTest + public void testGetLatestTaskStatusForRepeatedlyFailingTask() + { + final CompactionCandidate candidateSegments +- = CompactionCandidate.from(Collections.singletonList(WIKI_SEGMENT)); ++ = CompactionCandidate.from(List.of(WIKI_SEGMENT), null); + +- statusTracker.onTaskSubmitted(createCompactionTask("task1"), candidateSegments); ++ statusTracker.onTaskSubmitted("task1", candidateSegments); + statusTracker.onTaskFinished("task1", TaskStatus.failure("task1", "some failure")); + +- statusTracker.onTaskSubmitted(createCompactionTask("task2"), candidateSegments); ++ statusTracker.onTaskSubmitted("task2", candidateSegments); + CompactionTaskStatus status = statusTracker.getLatestTaskStatus(candidateSegments); + Assert.assertEquals(TaskState.RUNNING, status.getState()); + Assert.assertEquals(1, status.getNumConsecutiveFailures()); +@@ -110,25 +104,23 @@ public class CompactionStatusTrackerTest + @Test + public void testComputeCompactionStatusForSuccessfulTask() + { +- final DataSourceCompactionConfig compactionConfig +- = InlineSchemaDataSourceCompactionConfig.builder().forDataSource(TestDataSource.WIKI).build(); + final NewestSegmentFirstPolicy policy = new NewestSegmentFirstPolicy(null); + final CompactionCandidate candidateSegments +- = CompactionCandidate.from(Collections.singletonList(WIKI_SEGMENT)); ++ = CompactionCandidate.from(List.of(WIKI_SEGMENT), null); + + // Verify that interval is originally eligible for compaction + CompactionStatus status +- = statusTracker.computeCompactionStatus(candidateSegments, compactionConfig, policy); ++ = statusTracker.computeCompactionStatus(candidateSegments, policy); + Assert.assertEquals(CompactionStatus.State.PENDING, status.getState()); +- Assert.assertEquals("not compacted yet", status.getReason()); ++ Assert.assertEquals("Not compacted yet", status.getReason()); + + // Verify that interval is skipped for compaction after task has finished + statusTracker.onSegmentTimelineUpdated(DateTimes.nowUtc().minusMinutes(1)); +- statusTracker.onTaskSubmitted(createCompactionTask("task1"), candidateSegments); ++ statusTracker.onTaskSubmitted("task1", candidateSegments); + statusTracker.onTaskFinished("task1", TaskStatus.success("task1")); + +- status = statusTracker.computeCompactionStatus(candidateSegments, compactionConfig, policy); +- Assert.assertEquals(CompactionStatus.State.SKIPPED, status.getState()); ++ status = statusTracker.computeCompactionStatus(candidateSegments, policy); ++ Assert.assertEquals(CompactionStatus.State.COMPLETE, status.getState()); + Assert.assertEquals( + "Segment timeline not updated since last compaction task succeeded", + status.getReason() +@@ -136,26 +128,7 @@ public class CompactionStatusTrackerTest + + // Verify that interval becomes eligible again after timeline has been updated + statusTracker.onSegmentTimelineUpdated(DateTimes.nowUtc()); +- status = statusTracker.computeCompactionStatus(candidateSegments, compactionConfig, policy); ++ status = statusTracker.computeCompactionStatus(candidateSegments, policy); + Assert.assertEquals(CompactionStatus.State.PENDING, status.getState()); + } +- +- private ClientCompactionTaskQuery createCompactionTask( +- String taskId +- ) +- { +- return new ClientCompactionTaskQuery( +- taskId, +- TestDataSource.WIKI, +- null, +- null, +- null, +- null, +- null, +- null, +- null, +- null, +- null +- ); +- } + } +diff --git a/server/src/test/java/org/apache/druid/server/compaction/NewestSegmentFirstPolicyTest.java b/server/src/test/java/org/apache/druid/server/compaction/NewestSegmentFirstPolicyTest.java +index c655dc8ae4..1c92c9a249 100644 +--- a/server/src/test/java/org/apache/druid/server/compaction/NewestSegmentFirstPolicyTest.java ++++ b/server/src/test/java/org/apache/druid/server/compaction/NewestSegmentFirstPolicyTest.java +@@ -65,7 +65,6 @@ import org.joda.time.DateTimeZone; + import org.joda.time.Interval; + import org.joda.time.Period; + import org.junit.Assert; +-import org.junit.Before; + import org.junit.Test; + + import java.util.ArrayList; +@@ -82,13 +81,6 @@ public class NewestSegmentFirstPolicyTest + private static final int DEFAULT_NUM_SEGMENTS_PER_SHARD = 4; + private final ObjectMapper mapper = new DefaultObjectMapper(); + private final NewestSegmentFirstPolicy policy = new NewestSegmentFirstPolicy(null); +- private CompactionStatusTracker statusTracker; +- +- @Before +- public void setup() +- { +- statusTracker = new CompactionStatusTracker(mapper); +- } + + @Test + public void testLargeOffsetAndSmallSegmentInterval() +@@ -284,8 +276,7 @@ public class NewestSegmentFirstPolicyTest + .withNumPartitions(4) + ) + ), +- Collections.emptyMap(), +- statusTracker ++ Collections.emptyMap() + ); + + assertCompactSegmentIntervals( +@@ -517,8 +508,7 @@ public class NewestSegmentFirstPolicyTest + Intervals.of("2017-11-15T00:00:00/2017-11-15T20:00:00"), + Intervals.of("2017-11-13T00:00:00/2017-11-14T01:00:00") + ) +- ), +- statusTracker ++ ) + ); + + assertCompactSegmentIntervals( +@@ -557,8 +547,7 @@ public class NewestSegmentFirstPolicyTest + Intervals.of("2017-11-16T04:00:00/2017-11-16T10:00:00"), + Intervals.of("2017-11-16T14:00:00/2017-11-16T20:00:00") + ) +- ), +- statusTracker ++ ) + ); + + assertCompactSegmentIntervals( +@@ -1756,7 +1745,7 @@ public class NewestSegmentFirstPolicyTest + null, + new OnheapIncrementalIndex.Spec(true), + null, +- 1000L, ++ null, + null, + partitionsSpec, + IndexSpec.getDefault(), +@@ -1784,7 +1773,7 @@ public class NewestSegmentFirstPolicyTest + null, + new OnheapIncrementalIndex.Spec(false), + null, +- 1000L, ++ null, + null, + partitionsSpec, + IndexSpec.getDefault(), +@@ -2063,8 +2052,7 @@ public class NewestSegmentFirstPolicyTest + TestDataSource.WIKI, SegmentTimeline.forSegments(wikiSegments), + TestDataSource.KOALA, SegmentTimeline.forSegments(koalaSegments) + ), +- Collections.emptyMap(), +- statusTracker ++ Collections.emptyMap() + ); + + // Verify that the segments of WIKI are preferred even though they are older +@@ -2085,8 +2073,7 @@ public class NewestSegmentFirstPolicyTest + policy, + Collections.singletonMap(TestDataSource.WIKI, config), + Collections.singletonMap(TestDataSource.WIKI, timeline), +- Collections.emptyMap(), +- statusTracker ++ Collections.emptyMap() + ); + } + +diff --git a/server/src/test/java/org/apache/druid/server/coordinator/CreateDataSegments.java b/server/src/test/java/org/apache/druid/server/coordinator/CreateDataSegments.java +index 371d251b06..380533c729 100644 +--- a/server/src/test/java/org/apache/druid/server/coordinator/CreateDataSegments.java ++++ b/server/src/test/java/org/apache/druid/server/coordinator/CreateDataSegments.java +@@ -105,6 +105,10 @@ public class CreateDataSegments + return this; + } + ++ /** ++ * Specifies the version to use for creating the segments. Default version is ++ * {@code "1"}. ++ */ + public CreateDataSegments withVersion(String version) + { + this.version = version; +diff --git a/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorTest.java b/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorTest.java +index f77da3140a..39213ff861 100644 +--- a/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorTest.java ++++ b/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorTest.java +@@ -139,8 +139,7 @@ public class DruidCoordinatorTest + ) + ).andReturn(new AtomicReference<>(DruidCompactionConfig.empty())).anyTimes(); + EasyMock.replay(configManager); +- final ObjectMapper objectMapper = new DefaultObjectMapper(); +- statusTracker = new CompactionStatusTracker(objectMapper); ++ statusTracker = new CompactionStatusTracker(); + druidCoordinatorConfig = new DruidCoordinatorConfig( + new CoordinatorRunConfig(new Duration(COORDINATOR_START_DELAY), new Duration(COORDINATOR_PERIOD)), + new CoordinatorPeriodConfig(null, null), +@@ -169,7 +168,7 @@ public class DruidCoordinatorTest + new TestDruidLeaderSelector(), + null, + CentralizedDatasourceSchemaConfig.create(), +- new CompactionStatusTracker(OBJECT_MAPPER), ++ new CompactionStatusTracker(), + EasyMock.niceMock(CoordinatorDynamicConfigSyncer.class), + EasyMock.niceMock(CloneStatusManager.class) + ); +@@ -481,7 +480,7 @@ public class DruidCoordinatorTest + new TestDruidLeaderSelector(), + null, + CentralizedDatasourceSchemaConfig.create(), +- new CompactionStatusTracker(OBJECT_MAPPER), ++ new CompactionStatusTracker(), + EasyMock.niceMock(CoordinatorDynamicConfigSyncer.class), + EasyMock.niceMock(CloneStatusManager.class) + ); +@@ -533,7 +532,7 @@ public class DruidCoordinatorTest + new TestDruidLeaderSelector(), + null, + CentralizedDatasourceSchemaConfig.create(), +- new CompactionStatusTracker(OBJECT_MAPPER), ++ new CompactionStatusTracker(), + EasyMock.niceMock(CoordinatorDynamicConfigSyncer.class), + EasyMock.niceMock(CloneStatusManager.class) + ); +@@ -585,7 +584,7 @@ public class DruidCoordinatorTest + new TestDruidLeaderSelector(), + null, + CentralizedDatasourceSchemaConfig.create(), +- new CompactionStatusTracker(OBJECT_MAPPER), ++ new CompactionStatusTracker(), + EasyMock.niceMock(CoordinatorDynamicConfigSyncer.class), + EasyMock.niceMock(CloneStatusManager.class) + ); +@@ -695,7 +694,7 @@ public class DruidCoordinatorTest + new TestDruidLeaderSelector(), + null, + CentralizedDatasourceSchemaConfig.create(), +- new CompactionStatusTracker(OBJECT_MAPPER), ++ new CompactionStatusTracker(), + EasyMock.niceMock(CoordinatorDynamicConfigSyncer.class), + EasyMock.niceMock(CloneStatusManager.class) + ); +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 0d00228960..301dd77493 100644 +--- a/server/src/test/java/org/apache/druid/server/coordinator/duty/CompactSegmentsTest.java ++++ b/server/src/test/java/org/apache/druid/server/coordinator/duty/CompactSegmentsTest.java +@@ -80,7 +80,9 @@ import org.apache.druid.segment.IndexSpec; + import org.apache.druid.segment.incremental.OnheapIncrementalIndex; + import org.apache.druid.segment.indexing.BatchIOConfig; + import org.apache.druid.segment.transform.CompactionTransformSpec; ++import org.apache.druid.server.compaction.CompactionCandidate; + import org.apache.druid.server.compaction.CompactionCandidateSearchPolicy; ++import org.apache.druid.server.compaction.CompactionSlotManager; + import org.apache.druid.server.compaction.CompactionStatusTracker; + import org.apache.druid.server.compaction.FixedIntervalOrderPolicy; + import org.apache.druid.server.compaction.NewestSegmentFirstPolicy; +@@ -226,7 +228,7 @@ public class CompactSegmentsTest + } + } + dataSources = DataSourcesSnapshot.fromUsedSegments(allSegments); +- statusTracker = new CompactionStatusTracker(JSON_MAPPER); ++ statusTracker = new CompactionStatusTracker(); + policy = new NewestSegmentFirstPolicy(null); + } + +@@ -870,8 +872,8 @@ public class CompactSegmentsTest + // All segments is compact at the same time since we changed the segment granularity to YEAR and all segment + // are within the same year + Assert.assertEquals( +- ClientCompactionIntervalSpec.fromSegments(datasourceToSegments.get(dataSource), Granularities.YEAR), +- taskPayload.getIoConfig().getInputSpec() ++ CompactionCandidate.from(datasourceToSegments.get(dataSource), Granularities.YEAR).getCompactionInterval(), ++ taskPayload.getIoConfig().getInputSpec().getInterval() + ); + + ClientCompactionTaskGranularitySpec expectedGranularitySpec = +@@ -1061,8 +1063,8 @@ public class CompactSegmentsTest + // All segments is compact at the same time since we changed the segment granularity to YEAR and all segment + // are within the same year + Assert.assertEquals( +- ClientCompactionIntervalSpec.fromSegments(datasourceToSegments.get(dataSource), Granularities.YEAR), +- taskPayload.getIoConfig().getInputSpec() ++ CompactionCandidate.from(datasourceToSegments.get(dataSource), Granularities.YEAR).getCompactionInterval(), ++ taskPayload.getIoConfig().getInputSpec().getInterval() + ); + + ClientCompactionTaskGranularitySpec expectedGranularitySpec = +@@ -1156,8 +1158,8 @@ public class CompactSegmentsTest + // All segments is compact at the same time since we changed the segment granularity to YEAR and all segment + // are within the same year + Assert.assertEquals( +- ClientCompactionIntervalSpec.fromSegments(datasourceToSegments.get(dataSource), Granularities.YEAR), +- taskPayload.getIoConfig().getInputSpec() ++ CompactionCandidate.from(datasourceToSegments.get(dataSource), Granularities.YEAR).getCompactionInterval(), ++ taskPayload.getIoConfig().getInputSpec().getInterval() + ); + + ClientCompactionTaskGranularitySpec expectedGranularitySpec = +@@ -1397,8 +1399,8 @@ public class CompactSegmentsTest + ClientCompactionTaskQuery taskPayload = (ClientCompactionTaskQuery) payloadCaptor.getValue(); + + Assert.assertEquals( +- ClientCompactionIntervalSpec.fromSegments(segments, Granularities.DAY), +- taskPayload.getIoConfig().getInputSpec() ++ CompactionCandidate.from(segments, Granularities.DAY).getCompactionInterval(), ++ taskPayload.getIoConfig().getInputSpec().getInterval() + ); + + ClientCompactionTaskGranularitySpec expectedGranularitySpec = +@@ -1460,8 +1462,8 @@ public class CompactSegmentsTest + ClientCompactionTaskQuery taskPayload = (ClientCompactionTaskQuery) payloadCaptor.getValue(); + + Assert.assertEquals( +- ClientCompactionIntervalSpec.fromSegments(segments, Granularities.YEAR), +- taskPayload.getIoConfig().getInputSpec() ++ CompactionCandidate.from(segments, Granularities.YEAR).getCompactionInterval(), ++ taskPayload.getIoConfig().getInputSpec().getInterval() + ); + + ClientCompactionTaskGranularitySpec expectedGranularitySpec = +@@ -2004,7 +2006,7 @@ public class CompactSegmentsTest + @Test + public void testIsParalleModeNullTuningConfigReturnFalse() + { +- Assert.assertFalse(CompactSegments.isParallelMode(null)); ++ Assert.assertFalse(CompactionSlotManager.isParallelMode(null)); + } + + @Test +@@ -2012,7 +2014,7 @@ public class CompactSegmentsTest + { + ClientCompactionTaskQueryTuningConfig tuningConfig = Mockito.mock(ClientCompactionTaskQueryTuningConfig.class); + Mockito.when(tuningConfig.getPartitionsSpec()).thenReturn(null); +- Assert.assertFalse(CompactSegments.isParallelMode(tuningConfig)); ++ Assert.assertFalse(CompactionSlotManager.isParallelMode(tuningConfig)); + } + + @Test +@@ -2022,13 +2024,13 @@ public class CompactSegmentsTest + Mockito.when(tuningConfig.getPartitionsSpec()).thenReturn(Mockito.mock(PartitionsSpec.class)); + + Mockito.when(tuningConfig.getMaxNumConcurrentSubTasks()).thenReturn(null); +- Assert.assertFalse(CompactSegments.isParallelMode(tuningConfig)); ++ Assert.assertFalse(CompactionSlotManager.isParallelMode(tuningConfig)); + + Mockito.when(tuningConfig.getMaxNumConcurrentSubTasks()).thenReturn(1); +- Assert.assertFalse(CompactSegments.isParallelMode(tuningConfig)); ++ Assert.assertFalse(CompactionSlotManager.isParallelMode(tuningConfig)); + + Mockito.when(tuningConfig.getMaxNumConcurrentSubTasks()).thenReturn(2); +- Assert.assertTrue(CompactSegments.isParallelMode(tuningConfig)); ++ Assert.assertTrue(CompactionSlotManager.isParallelMode(tuningConfig)); + } + + @Test +@@ -2038,13 +2040,13 @@ public class CompactSegmentsTest + Mockito.when(tuningConfig.getPartitionsSpec()).thenReturn(Mockito.mock(SingleDimensionPartitionsSpec.class)); + + Mockito.when(tuningConfig.getMaxNumConcurrentSubTasks()).thenReturn(null); +- Assert.assertFalse(CompactSegments.isParallelMode(tuningConfig)); ++ Assert.assertFalse(CompactionSlotManager.isParallelMode(tuningConfig)); + + Mockito.when(tuningConfig.getMaxNumConcurrentSubTasks()).thenReturn(1); +- Assert.assertTrue(CompactSegments.isParallelMode(tuningConfig)); ++ Assert.assertTrue(CompactionSlotManager.isParallelMode(tuningConfig)); + + Mockito.when(tuningConfig.getMaxNumConcurrentSubTasks()).thenReturn(2); +- Assert.assertTrue(CompactSegments.isParallelMode(tuningConfig)); ++ Assert.assertTrue(CompactionSlotManager.isParallelMode(tuningConfig)); + } + + @Test +@@ -2053,7 +2055,7 @@ public class CompactSegmentsTest + ClientCompactionTaskQueryTuningConfig tuningConfig = Mockito.mock(ClientCompactionTaskQueryTuningConfig.class); + Mockito.when(tuningConfig.getPartitionsSpec()).thenReturn(Mockito.mock(PartitionsSpec.class)); + Mockito.when(tuningConfig.getMaxNumConcurrentSubTasks()).thenReturn(2); +- Assert.assertEquals(3, CompactSegments.findMaxNumTaskSlotsUsedByOneNativeCompactionTask(tuningConfig)); ++ Assert.assertEquals(3, CompactionSlotManager.getMaxTaskSlotsForNativeCompactionTask(tuningConfig)); + } + + @Test +@@ -2062,7 +2064,7 @@ public class CompactSegmentsTest + ClientCompactionTaskQueryTuningConfig tuningConfig = Mockito.mock(ClientCompactionTaskQueryTuningConfig.class); + Mockito.when(tuningConfig.getPartitionsSpec()).thenReturn(Mockito.mock(PartitionsSpec.class)); + Mockito.when(tuningConfig.getMaxNumConcurrentSubTasks()).thenReturn(1); +- Assert.assertEquals(1, CompactSegments.findMaxNumTaskSlotsUsedByOneNativeCompactionTask(tuningConfig)); ++ Assert.assertEquals(1, CompactionSlotManager.getMaxTaskSlotsForNativeCompactionTask(tuningConfig)); + } + } + +diff --git a/server/src/test/java/org/apache/druid/server/coordinator/simulate/CoordinatorSimulationBuilder.java b/server/src/test/java/org/apache/druid/server/coordinator/simulate/CoordinatorSimulationBuilder.java +index 8d63492375..0838b41402 100644 +--- a/server/src/test/java/org/apache/druid/server/coordinator/simulate/CoordinatorSimulationBuilder.java ++++ b/server/src/test/java/org/apache/druid/server/coordinator/simulate/CoordinatorSimulationBuilder.java +@@ -223,7 +223,7 @@ public class CoordinatorSimulationBuilder + env.leaderSelector, + null, + CentralizedDatasourceSchemaConfig.create(), +- new CompactionStatusTracker(OBJECT_MAPPER), ++ new CompactionStatusTracker(), + env.configSyncer, + env.cloneStatusManager + ); +diff --git a/server/src/test/java/org/apache/druid/server/metrics/LatchableEmitter.java b/server/src/test/java/org/apache/druid/server/metrics/LatchableEmitter.java +index 247b74d8e5..5d8b110e17 100644 +--- a/server/src/test/java/org/apache/druid/server/metrics/LatchableEmitter.java ++++ b/server/src/test/java/org/apache/druid/server/metrics/LatchableEmitter.java +@@ -24,6 +24,8 @@ import org.apache.druid.java.util.common.logger.Logger; + import org.apache.druid.java.util.emitter.core.Event; + import org.apache.druid.java.util.emitter.service.ServiceMetricEvent; + import org.apache.druid.java.util.metrics.StubServiceEmitter; ++import org.hamcrest.Matcher; ++import org.hamcrest.Matchers; + import org.junit.jupiter.api.Timeout; + + import java.util.ArrayList; +@@ -245,8 +247,8 @@ public class LatchableEmitter extends StubServiceEmitter + private String host; + private String service; + private String metricName; +- private Long metricValue; +- private final Map dimensions = new HashMap<>(); ++ private Matcher valueMatcher; ++ private final Map> dimensionMatchers = new HashMap<>(); + + private final AtomicReference matchingEvent = new AtomicReference<>(); + +@@ -260,12 +262,11 @@ public class LatchableEmitter extends StubServiceEmitter + } + + /** +- * Matches an event only if it has a metric value equal to or greater than +- * the given value. ++ * Matches an event only if the metric value satisfies the given matcher. + */ +- public EventMatcher hasValueAtLeast(long metricValue) ++ public EventMatcher hasValueMatching(Matcher valueMatcher) + { +- this.metricValue = metricValue; ++ this.valueMatcher = valueMatcher; + return this; + } + +@@ -274,7 +275,16 @@ public class LatchableEmitter extends StubServiceEmitter + */ + public EventMatcher hasDimension(String dimension, Object value) + { +- dimensions.put(dimension, value); ++ dimensionMatchers.put(dimension, Matchers.equalTo(value)); ++ return this; ++ } ++ ++ /** ++ * Matches an event if the value of the given dimension satisfies the matcher. ++ */ ++ public EventMatcher hasDimensionMatching(String dimension, Matcher matcher) ++ { ++ dimensionMatchers.put(dimension, matcher); + return this; + } + +@@ -301,7 +311,7 @@ public class LatchableEmitter extends StubServiceEmitter + { + if (metricName != null && !event.getMetric().equals(metricName)) { + return false; +- } else if (metricValue != null && event.getValue().longValue() < metricValue) { ++ } else if (valueMatcher != null && !valueMatcher.matches(event.getValue())) { + return false; + } else if (service != null && !service.equals(event.getService())) { + return false; +@@ -309,10 +319,8 @@ public class LatchableEmitter extends StubServiceEmitter + return false; + } + +- final boolean matches = dimensions.entrySet().stream().allMatch( +- dimValue -> event.getUserDims() +- .getOrDefault(dimValue.getKey(), "") +- .equals(dimValue.getValue()) ++ final boolean matches = dimensionMatchers.entrySet().stream().allMatch( ++ dimValue -> dimValue.getValue().matches(event.getUserDims().get(dimValue.getKey())) + ); + + if (matches) { +diff --git a/services/src/main/java/org/apache/druid/cli/CliOverlord.java b/services/src/main/java/org/apache/druid/cli/CliOverlord.java +index 81a0c13671..63a1d28bfc 100644 +--- a/services/src/main/java/org/apache/druid/cli/CliOverlord.java ++++ b/services/src/main/java/org/apache/druid/cli/CliOverlord.java +@@ -70,6 +70,7 @@ import org.apache.druid.indexing.common.task.batch.parallel.ShuffleClient; + import org.apache.druid.indexing.common.tasklogs.SwitchingTaskLogStreamer; + import org.apache.druid.indexing.common.tasklogs.TaskRunnerTaskLogStreamer; + import org.apache.druid.indexing.compact.CompactionScheduler; ++import org.apache.druid.indexing.compact.DruidInputSourceFactory; + import org.apache.druid.indexing.compact.OverlordCompactionScheduler; + import org.apache.druid.indexing.overlord.DruidOverlord; + import org.apache.druid.indexing.overlord.ForkingTaskRunnerFactory; +@@ -247,6 +248,7 @@ public class CliOverlord extends ServerRunnable + binder.bind(TaskQueryTool.class).in(LazySingleton.class); + binder.bind(IndexerMetadataStorageAdapter.class).in(LazySingleton.class); + binder.bind(CompactionScheduler.class).to(OverlordCompactionScheduler.class).in(ManageLifecycle.class); ++ binder.bind(DruidInputSourceFactory.class).in(LazySingleton.class); + binder.bind(ScheduledBatchTaskManager.class).in(LazySingleton.class); + binder.bind(SupervisorManager.class).in(LazySingleton.class); + +diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/schema/DruidCalciteSchemaModuleTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/schema/DruidCalciteSchemaModuleTest.java +index 2333561540..eb0ab607e8 100644 +--- a/sql/src/test/java/org/apache/druid/sql/calcite/schema/DruidCalciteSchemaModuleTest.java ++++ b/sql/src/test/java/org/apache/druid/sql/calcite/schema/DruidCalciteSchemaModuleTest.java +@@ -28,6 +28,8 @@ import com.google.inject.Key; + import com.google.inject.Scopes; + import com.google.inject.TypeLiteral; + import com.google.inject.name.Names; ++import org.apache.druid.catalog.MapMetadataCatalog; ++import org.apache.druid.catalog.MetadataCatalog; + import org.apache.druid.client.FilteredServerInventoryView; + import org.apache.druid.client.TimelineServerView; + import org.apache.druid.client.coordinator.CoordinatorClient; +@@ -56,7 +58,7 @@ import org.apache.druid.sql.calcite.view.ViewManager; + import org.easymock.EasyMock; + import org.easymock.EasyMockExtension; + import org.easymock.Mock; +-import org.junit.Assert; ++import org.junit.jupiter.api.Assertions; + import org.junit.jupiter.api.BeforeEach; + import org.junit.jupiter.api.Test; + import org.junit.jupiter.api.extension.ExtendWith; +@@ -120,6 +122,7 @@ public class DruidCalciteSchemaModuleTest extends CalciteTestBase + binder.bindScope(LazySingleton.class, Scopes.SINGLETON); + binder.bind(LookupExtractorFactoryContainerProvider.class).toInstance(lookupReferencesManager); + binder.bind(CatalogResolver.class).toInstance(CatalogResolver.NULL_RESOLVER); ++ binder.bind(MetadataCatalog.class).toInstance(new MapMetadataCatalog(objectMapper)); + binder.bind(ServiceEmitter.class).toInstance(new ServiceEmitter("", "", null)); + binder.bind(OverlordClient.class).to(NoopOverlordClient.class); + binder.bind(CoordinatorClient.class).to(NoopCoordinatorClient.class); +@@ -134,35 +137,39 @@ public class DruidCalciteSchemaModuleTest extends CalciteTestBase + public void testDruidSchemaNameIsInjected() + { + String schemaName = injector.getInstance(Key.get(String.class, DruidSchemaName.class)); +- Assert.assertEquals(DRUID_SCHEMA_NAME, schemaName); ++ Assertions.assertEquals(DRUID_SCHEMA_NAME, schemaName); + } + + @Test + public void testDruidSqlSchemaIsInjectedAsSingleton() + { + NamedDruidSchema namedDruidSchema = injector.getInstance(NamedDruidSchema.class); +- Assert.assertNotNull(namedDruidSchema); ++ Assertions.assertNotNull(namedDruidSchema); + NamedDruidSchema other = injector.getInstance(NamedDruidSchema.class); +- Assert.assertSame(other, namedDruidSchema); ++ Assertions.assertSame(other, namedDruidSchema); + } + + @Test + public void testSystemSqlSchemaIsInjectedAsSingleton() + { + NamedSystemSchema namedSystemSchema = injector.getInstance(NamedSystemSchema.class); +- Assert.assertNotNull(namedSystemSchema); ++ Assertions.assertNotNull(namedSystemSchema); + NamedSystemSchema other = injector.getInstance(NamedSystemSchema.class); +- Assert.assertSame(other, namedSystemSchema); ++ Assertions.assertSame(other, namedSystemSchema); + } + + @Test + public void testDruidCalciteSchemasAreInjected() + { + Set sqlSchemas = injector.getInstance(Key.get(new TypeLiteral<>() {})); +- Set> expectedSchemas = +- ImmutableSet.of(NamedSystemSchema.class, NamedDruidSchema.class, NamedLookupSchema.class, NamedViewSchema.class); +- Assert.assertEquals(expectedSchemas.size(), sqlSchemas.size()); +- Assert.assertEquals( ++ Set> expectedSchemas = Set.of( ++ NamedSystemSchema.class, ++ NamedDruidSchema.class, ++ NamedLookupSchema.class, ++ NamedViewSchema.class ++ ); ++ Assertions.assertEquals(expectedSchemas.size(), sqlSchemas.size()); ++ Assertions.assertEquals( + expectedSchemas, + sqlSchemas.stream().map(NamedSchema::getClass).collect(Collectors.toSet())); + } +@@ -171,36 +178,36 @@ public class DruidCalciteSchemaModuleTest extends CalciteTestBase + public void testDruidSchemaIsInjectedAsSingleton() + { + DruidSchema schema = injector.getInstance(DruidSchema.class); +- Assert.assertNotNull(schema); ++ Assertions.assertNotNull(schema); + DruidSchema other = injector.getInstance(DruidSchema.class); +- Assert.assertSame(other, schema); ++ Assertions.assertSame(other, schema); + } + + @Test + public void testSystemSchemaIsInjectedAsSingleton() + { + SystemSchema schema = injector.getInstance(SystemSchema.class); +- Assert.assertNotNull(schema); ++ Assertions.assertNotNull(schema); + SystemSchema other = injector.getInstance(SystemSchema.class); +- Assert.assertSame(other, schema); ++ Assertions.assertSame(other, schema); + } + + @Test + public void testInformationSchemaIsInjectedAsSingleton() + { + InformationSchema schema = injector.getInstance(InformationSchema.class); +- Assert.assertNotNull(schema); ++ Assertions.assertNotNull(schema); + InformationSchema other = injector.getInstance(InformationSchema.class); +- Assert.assertSame(other, schema); ++ Assertions.assertSame(other, schema); + } + + @Test + public void testLookupSchemaIsInjectedAsSingleton() + { + LookupSchema schema = injector.getInstance(LookupSchema.class); +- Assert.assertNotNull(schema); ++ Assertions.assertNotNull(schema); + LookupSchema other = injector.getInstance(LookupSchema.class); +- Assert.assertSame(other, schema); ++ Assertions.assertSame(other, schema); + } + + @Test +@@ -209,22 +216,22 @@ public class DruidCalciteSchemaModuleTest extends CalciteTestBase + DruidSchemaCatalog rootSchema = injector.getInstance( + Key.get(DruidSchemaCatalog.class, Names.named(DruidCalciteSchemaModule.INCOMPLETE_SCHEMA)) + ); +- Assert.assertNotNull(rootSchema); ++ Assertions.assertNotNull(rootSchema); + DruidSchemaCatalog other = injector.getInstance( + Key.get(DruidSchemaCatalog.class, Names.named(DruidCalciteSchemaModule.INCOMPLETE_SCHEMA)) + ); +- Assert.assertSame(other, rootSchema); ++ Assertions.assertSame(other, rootSchema); + } + + @Test + public void testRootSchemaIsInjectedAsSingleton() + { + DruidSchemaCatalog rootSchema = injector.getInstance(Key.get(DruidSchemaCatalog.class)); +- Assert.assertNotNull(rootSchema); ++ Assertions.assertNotNull(rootSchema); + DruidSchemaCatalog other = injector.getInstance( + Key.get(DruidSchemaCatalog.class, Names.named(DruidCalciteSchemaModule.INCOMPLETE_SCHEMA)) + ); +- Assert.assertSame(other, rootSchema); ++ Assertions.assertSame(other, rootSchema); + } + + @Test +@@ -232,7 +239,7 @@ public class DruidCalciteSchemaModuleTest extends CalciteTestBase + { + DruidSchemaCatalog rootSchema = injector.getInstance(Key.get(DruidSchemaCatalog.class)); + InformationSchema expectedSchema = injector.getInstance(InformationSchema.class); +- Assert.assertNotNull(rootSchema); +- Assert.assertSame(expectedSchema, rootSchema.getSubSchema("INFORMATION_SCHEMA").unwrap(InformationSchema.class)); ++ Assertions.assertNotNull(rootSchema); ++ Assertions.assertSame(expectedSchema, rootSchema.getSubSchema("INFORMATION_SCHEMA").unwrap(InformationSchema.class)); + } + } +diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/util/SqlTestFramework.java b/sql/src/test/java/org/apache/druid/sql/calcite/util/SqlTestFramework.java +index 7a7cf16459..38ddba751b 100644 +--- a/sql/src/test/java/org/apache/druid/sql/calcite/util/SqlTestFramework.java ++++ b/sql/src/test/java/org/apache/druid/sql/calcite/util/SqlTestFramework.java +@@ -35,6 +35,7 @@ import org.apache.druid.client.cache.CacheConfig; + import org.apache.druid.collections.BlockingPool; + import org.apache.druid.collections.NonBlockingPool; + import org.apache.druid.guice.BuiltInTypesModule; ++import org.apache.druid.guice.CatalogCoreModule; + import org.apache.druid.guice.DruidInjectorBuilder; + import org.apache.druid.guice.ExpressionModule; + import org.apache.druid.guice.LazySingleton; +@@ -409,6 +410,7 @@ public class SqlTestFramework + new LookylooModule(), + new SegmentWranglerModule(), + new ExpressionModule(), ++ new CatalogCoreModule(), + DruidModule.override( + new QueryRunnerFactoryModule(), + new Module() +diff --git a/sql/src/test/java/org/apache/druid/sql/guice/SqlModuleTest.java b/sql/src/test/java/org/apache/druid/sql/guice/SqlModuleTest.java +index 28e8923f63..f0057eb9c5 100644 +--- a/sql/src/test/java/org/apache/druid/sql/guice/SqlModuleTest.java ++++ b/sql/src/test/java/org/apache/druid/sql/guice/SqlModuleTest.java +@@ -32,6 +32,7 @@ import org.apache.druid.client.TimelineServerView; + import org.apache.druid.client.coordinator.CoordinatorClient; + import org.apache.druid.client.coordinator.NoopCoordinatorClient; + import org.apache.druid.discovery.DruidNodeDiscoveryProvider; ++import org.apache.druid.guice.CatalogCoreModule; + import org.apache.druid.guice.DruidGuiceExtensions; + import org.apache.druid.guice.JsonConfigurator; + import org.apache.druid.guice.LazySingleton; +@@ -182,6 +183,7 @@ public class SqlModuleTest + new JacksonModule(), + new PolicyModule(), + new AuthenticatorMapperModule(), ++ new CatalogCoreModule(), + binder -> { + binder.bind(Validator.class).toInstance(Validation.buildDefaultValidatorFactory().getValidator()); + binder.bind(JsonConfigurator.class).in(LazySingleton.class); From 3cbf9558723d8a3c079fd60086355b86a9fa33d0 Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Thu, 30 Oct 2025 20:57:26 +0530 Subject: [PATCH 25/25] Remove extra file --- stuff.diff | 7184 ---------------------------------------------------- 1 file changed, 7184 deletions(-) delete mode 100644 stuff.diff diff --git a/stuff.diff b/stuff.diff deleted file mode 100644 index a4b264dced42..000000000000 --- a/stuff.diff +++ /dev/null @@ -1,7184 +0,0 @@ -diff --git a/benchmarks/src/test/java/org/apache/druid/server/coordinator/NewestSegmentFirstPolicyBenchmark.java b/benchmarks/src/test/java/org/apache/druid/server/coordinator/NewestSegmentFirstPolicyBenchmark.java -index a0a1da9369..c9c4599fad 100644 ---- a/benchmarks/src/test/java/org/apache/druid/server/coordinator/NewestSegmentFirstPolicyBenchmark.java -+++ b/benchmarks/src/test/java/org/apache/druid/server/coordinator/NewestSegmentFirstPolicyBenchmark.java -@@ -21,11 +21,9 @@ package org.apache.druid.server.coordinator; - - import com.google.common.collect.ImmutableList; - import org.apache.druid.client.DataSourcesSnapshot; --import org.apache.druid.jackson.DefaultObjectMapper; - import org.apache.druid.java.util.common.DateTimes; - import org.apache.druid.server.compaction.CompactionCandidateSearchPolicy; - import org.apache.druid.server.compaction.CompactionSegmentIterator; --import org.apache.druid.server.compaction.CompactionStatusTracker; - import org.apache.druid.server.compaction.NewestSegmentFirstPolicy; - import org.apache.druid.server.compaction.PriorityBasedCompactionSegmentIterator; - import org.apache.druid.timeline.DataSegment; -@@ -137,8 +135,7 @@ public class NewestSegmentFirstPolicyBenchmark - policy, - compactionConfigs, - dataSources, -- Collections.emptyMap(), -- new CompactionStatusTracker(new DefaultObjectMapper()) -+ Collections.emptyMap() - ); - for (int i = 0; i < numCompactionTaskSlots && iterator.hasNext(); i++) { - blackhole.consume(iterator.next()); -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 -new file mode 100644 -index 0000000000..35ba77274f ---- /dev/null -+++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionSupervisorTest.java -@@ -0,0 +1,345 @@ -+/* -+ * 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.testing.embedded.compact; -+ -+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.indexing.common.task.IndexTask; -+import org.apache.druid.indexing.compact.CascadingCompactionTemplate; -+import org.apache.druid.indexing.compact.CompactionJobTemplate; -+import org.apache.druid.indexing.compact.CompactionRule; -+import org.apache.druid.indexing.compact.CompactionStateMatcher; -+import org.apache.druid.indexing.compact.CompactionSupervisorSpec; -+import org.apache.druid.indexing.compact.InlineCompactionJobTemplate; -+import org.apache.druid.indexing.compact.MSQCompactionJobTemplate; -+import org.apache.druid.indexing.overlord.Segments; -+import org.apache.druid.java.util.common.DateTimes; -+import org.apache.druid.java.util.common.granularity.Granularities; -+import org.apache.druid.java.util.common.granularity.Granularity; -+import org.apache.druid.msq.guice.IndexerMemoryManagementModule; -+import org.apache.druid.msq.guice.MSQDurableStorageModule; -+import org.apache.druid.msq.guice.MSQIndexingModule; -+import org.apache.druid.msq.guice.MSQSqlModule; -+import org.apache.druid.msq.guice.SqlTaskModule; -+import org.apache.druid.query.DruidMetrics; -+import org.apache.druid.query.http.ClientSqlQuery; -+import org.apache.druid.rpc.UpdateResponse; -+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.UserCompactionTaskGranularityConfig; -+import org.apache.druid.testing.embedded.EmbeddedBroker; -+import org.apache.druid.testing.embedded.EmbeddedCoordinator; -+import org.apache.druid.testing.embedded.EmbeddedDruidCluster; -+import org.apache.druid.testing.embedded.EmbeddedHistorical; -+import org.apache.druid.testing.embedded.EmbeddedIndexer; -+import org.apache.druid.testing.embedded.EmbeddedOverlord; -+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.hamcrest.Matcher; -+import org.hamcrest.Matchers; -+import org.joda.time.DateTime; -+import org.joda.time.Period; -+import org.junit.jupiter.api.Assertions; -+import org.junit.jupiter.api.BeforeAll; -+import org.junit.jupiter.api.Test; -+ -+import java.util.List; -+import java.util.Map; -+ -+/** -+ * Embedded test that runs compaction supervisors of various types. -+ */ -+public class CompactionSupervisorTest extends EmbeddedClusterTestBase -+{ -+ private final EmbeddedBroker broker = new EmbeddedBroker(); -+ private final EmbeddedIndexer indexer = new EmbeddedIndexer() -+ .setServerMemory(2_000_000_000L) -+ .addProperty("druid.worker.capacity", "20"); -+ private final EmbeddedOverlord overlord = new EmbeddedOverlord() -+ .addProperty("druid.manager.segments.pollDuration", "PT1s") -+ .addProperty("druid.manager.segments.useIncrementalCache", "always"); -+ private final EmbeddedHistorical historical = new EmbeddedHistorical(); -+ private final EmbeddedCoordinator coordinator = new EmbeddedCoordinator() -+ .addProperty("druid.manager.segments.useIncrementalCache", "always"); -+ -+ @Override -+ public EmbeddedDruidCluster createCluster() -+ { -+ return EmbeddedDruidCluster.withEmbeddedDerbyAndZookeeper() -+ .useLatchableEmitter() -+ .useDefaultTimeoutForLatchableEmitter(600) -+ .addExtensions( -+ CatalogClientModule.class, -+ CatalogCoordinatorModule.class, -+ IndexerMemoryManagementModule.class, -+ MSQDurableStorageModule.class, -+ MSQIndexingModule.class, -+ MSQSqlModule.class, -+ SqlTaskModule.class -+ ) -+ .addServer(coordinator) -+ .addServer(overlord) -+ .addServer(indexer) -+ .addServer(historical) -+ .addServer(broker) -+ .addServer(new EmbeddedRouter()); -+ } -+ -+ @BeforeAll -+ public void enableCompactionSupervisors() -+ { -+ final UpdateResponse updateResponse = cluster.callApi().onLeaderOverlord( -+ o -> o.updateClusterCompactionConfig(new ClusterCompactionConfig(1.0, 100, null, true, null)) -+ ); -+ Assertions.assertTrue(updateResponse.isSuccess()); -+ } -+ -+ @Test -+ public void test_ingestDayGranularity_andCompactToMonthGranularity_withInlineConfig() -+ { -+ // Ingest data at DAY granularity and verify -+ runIngestionAtGranularity( -+ "DAY", -+ "2025-06-01T00:00:00.000Z,shirt,105" -+ + "\n2025-06-02T00:00:00.000Z,trousers,210" -+ + "\n2025-06-03T00:00:00.000Z,jeans,150" -+ ); -+ Assertions.assertEquals(3, getNumSegmentsWith(Granularities.DAY)); -+ -+ // Create a compaction config with MONTH granularity -+ InlineSchemaDataSourceCompactionConfig compactionConfig = -+ InlineSchemaDataSourceCompactionConfig -+ .builder() -+ .forDataSource(dataSource) -+ .withSkipOffsetFromLatest(Period.seconds(0)) -+ .withGranularitySpec( -+ new UserCompactionTaskGranularityConfig(Granularities.MONTH, null, null) -+ ) -+ .build(); -+ -+ runCompactionWithSpec(compactionConfig); -+ waitForAllCompactionTasksToFinish(); -+ -+ Assertions.assertEquals(0, getNumSegmentsWith(Granularities.DAY)); -+ Assertions.assertEquals(1, getNumSegmentsWith(Granularities.MONTH)); -+ } -+ -+ @Test -+ public void test_ingestHourGranularity_andCompactToDayAndMonth_withInlineTemplates() -+ { -+ // Create a cascading template with DAY and MONTH granularity -+ CascadingCompactionTemplate cascadingTemplate = new CascadingCompactionTemplate( -+ dataSource, -+ List.of( -+ new CompactionRule(Period.days(1), new InlineCompactionJobTemplate(createMatcher(Granularities.DAY))), -+ new CompactionRule(Period.days(50), new InlineCompactionJobTemplate(createMatcher(Granularities.MONTH))) -+ ) -+ ); -+ -+ ingestHourSegments(1000); -+ runCompactionWithSpec(cascadingTemplate); -+ waitForAllCompactionTasksToFinish(); -+ -+ Assertions.assertEquals(0, getNumSegmentsWith(Granularities.HOUR)); -+ Assertions.assertTrue(getNumSegmentsWith(Granularities.DAY) >= 1); -+ Assertions.assertTrue(getNumSegmentsWith(Granularities.MONTH) >= 1); -+ } -+ -+ @Test -+ public void test_ingestHourGranularity_andCompactToDayAndMonth_withMSQTemplates() -+ { -+ ingestHourSegments(1200); -+ -+ // Add compaction templates to catalog -+ final String sqlDayGranularity = -+ "REPLACE INTO ${dataSource}" -+ + " OVERWRITE WHERE __time >= TIMESTAMP '${startTimestamp}' AND __time < TIMESTAMP '${endTimestamp}'" -+ + " SELECT * FROM ${dataSource}" -+ + " WHERE __time BETWEEN '${startTimestamp}' AND '${endTimestamp}'" -+ + " PARTITIONED BY DAY"; -+ final CompactionJobTemplate dayGranularityTemplate = new MSQCompactionJobTemplate( -+ new ClientSqlQuery(sqlDayGranularity, null, false, false, false, null, null), -+ createMatcher(Granularities.DAY) -+ ); -+ final String sqlMonthGranularity = -+ "REPLACE INTO ${dataSource}" -+ + " OVERWRITE WHERE __time >= TIMESTAMP '${startTimestamp}' AND __time < TIMESTAMP '${endTimestamp}'" -+ + " SELECT * FROM ${dataSource}" -+ + " WHERE __time >= TIMESTAMP '${startTimestamp}' AND __time < TIMESTAMP '${endTimestamp}'" -+ + " PARTITIONED BY MONTH"; -+ final CompactionJobTemplate monthGranularityTemplate = new MSQCompactionJobTemplate( -+ new ClientSqlQuery(sqlMonthGranularity, null, false, false, false, null, null), -+ createMatcher(Granularities.MONTH) -+ ); -+ -+ // Create a cascading template with DAY and MONTH granularity -+ CascadingCompactionTemplate cascadingTemplate = new CascadingCompactionTemplate( -+ dataSource, -+ List.of( -+ new CompactionRule(Period.days(1), dayGranularityTemplate), -+ new CompactionRule(Period.days(50), monthGranularityTemplate) -+ ) -+ ); -+ -+ runCompactionWithSpec(cascadingTemplate); -+ waitForAllCompactionTasksToFinish(); -+ -+ Assertions.assertEquals(0, getNumSegmentsWith(Granularities.HOUR)); -+ Assertions.assertTrue(getNumSegmentsWith(Granularities.DAY) >= 1); -+ Assertions.assertTrue(getNumSegmentsWith(Granularities.MONTH) >= 1); -+ } -+ -+ @Test -+ public void test_ingestHourGranularity_andCompactToDayAndMonth_withMixedTemplates() -+ { -+ ingestHourSegments(1200); -+ -+ // Add compaction templates to catalog -+ final String sqlDayGranularity = -+ "REPLACE INTO ${dataSource}" -+ + " OVERWRITE WHERE __time >= TIMESTAMP '${startTimestamp}' AND __time < TIMESTAMP '${endTimestamp}'" -+ + " SELECT * FROM ${dataSource}" -+ + " WHERE __time BETWEEN '${startTimestamp}' AND '${endTimestamp}'" -+ + " PARTITIONED BY DAY"; -+ final MSQCompactionJobTemplate dayTemplate = new MSQCompactionJobTemplate( -+ new ClientSqlQuery(sqlDayGranularity, null, false, false, false, null, null), -+ createMatcher(Granularities.DAY) -+ ); -+ final CompactionJobTemplate weekTemplate = -+ new InlineCompactionJobTemplate(createMatcher(Granularities.WEEK)); -+ final InlineCompactionJobTemplate monthTemplate = -+ new InlineCompactionJobTemplate(createMatcher(Granularities.MONTH)); -+ -+ // Compact last 1 day to DAY, next 14 days to WEEK, then 1 more DAY, rest to MONTH -+ CascadingCompactionTemplate cascadingTemplate = new CascadingCompactionTemplate( -+ dataSource, -+ List.of( -+ new CompactionRule(Period.days(1), dayTemplate), -+ new CompactionRule(Period.days(15), weekTemplate), -+ new CompactionRule(Period.days(16), dayTemplate), -+ new CompactionRule(Period.ZERO, monthTemplate) -+ ) -+ ); -+ -+ runCompactionWithSpec(cascadingTemplate); -+ waitForAllCompactionTasksToFinish(); -+ -+ Assertions.assertEquals(0, getNumSegmentsWith(Granularities.HOUR)); -+ Assertions.assertTrue(getNumSegmentsWith(Granularities.DAY) >= 1); -+ Assertions.assertTrue(getNumSegmentsWith(Granularities.WEEK) >= 1); -+ Assertions.assertTrue(getNumSegmentsWith(Granularities.MONTH) >= 1); -+ } -+ -+ private void ingestHourSegments(int numSegments) -+ { -+ runIngestionAtGranularity( -+ "HOUR", -+ createHourlyInlineDataCsv(DateTimes.nowUtc(), numSegments) -+ ); -+ } -+ -+ private void runCompactionWithSpec(DataSourceCompactionConfig config) -+ { -+ final CompactionSupervisorSpec compactionSupervisor -+ = new CompactionSupervisorSpec(config, false, null); -+ cluster.callApi().postSupervisor(compactionSupervisor); -+ } -+ -+ private void waitForAllCompactionTasksToFinish() -+ { -+ // Wait for all intervals to be compacted -+ overlord.latchableEmitter().waitForEvent( -+ event -> event.hasMetricName("interval/waitCompact/count") -+ .hasDimension(DruidMetrics.DATASOURCE, dataSource) -+ .hasValueMatching(Matchers.equalTo(0L)) -+ ); -+ -+ // Wait for all submitted compaction jobs to finish -+ int numSubmittedTasks = overlord.latchableEmitter().getMetricValues( -+ "compact/task/count", -+ Map.of(DruidMetrics.DATASOURCE, dataSource) -+ ).stream().mapToInt(Number::intValue).sum(); -+ -+ final Matcher taskTypeMatcher = Matchers.anyOf( -+ Matchers.equalTo("query_controller"), -+ Matchers.equalTo("compact") -+ ); -+ overlord.latchableEmitter().waitForEventAggregate( -+ event -> event.hasMetricName("task/run/time") -+ .hasDimensionMatching(DruidMetrics.TASK_TYPE, taskTypeMatcher) -+ .hasDimension(DruidMetrics.DATASOURCE, dataSource), -+ agg -> agg.hasCountAtLeast(numSubmittedTasks) -+ ); -+ } -+ -+ private int getNumSegmentsWith(Granularity granularity) -+ { -+ return (int) overlord -+ .bindings() -+ .segmentsMetadataStorage() -+ .retrieveAllUsedSegments(dataSource, Segments.ONLY_VISIBLE) -+ .stream() -+ .filter(segment -> granularity.isAligned(segment.getInterval())) -+ .count(); -+ } -+ -+ private void runIngestionAtGranularity( -+ String granularity, -+ String inlineDataCsv -+ ) -+ { -+ final IndexTask task = MoreResources.Task.BASIC_INDEX -+ .get() -+ .segmentGranularity(granularity) -+ .inlineInputSourceWithData(inlineDataCsv) -+ .dataSource(dataSource) -+ .withId(IdUtils.getRandomId()); -+ cluster.callApi().runTask(task, overlord); -+ } -+ -+ private String createHourlyInlineDataCsv(DateTime latestRecordTimestamp, int numRecords) -+ { -+ final StringBuilder builder = new StringBuilder(); -+ for (int i = 0; i < numRecords; ++i) { -+ builder.append(latestRecordTimestamp.minusHours(i)) -+ .append(",").append("item_").append(IdUtils.getRandomId()) -+ .append(",").append(0) -+ .append("\n"); -+ } -+ -+ return builder.toString(); -+ } -+ -+ private static CompactionStateMatcher createMatcher(Granularity segmentGranularity) -+ { -+ return new CompactionStateMatcher( -+ null, -+ null, -+ null, -+ null, -+ null, -+ new UserCompactionTaskGranularityConfig(segmentGranularity, null, null), -+ null -+ ); -+ } -+} -diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaClusterMetricsTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaClusterMetricsTest.java -index 44c9be9a9c..11e25120aa 100644 ---- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaClusterMetricsTest.java -+++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/KafkaClusterMetricsTest.java -@@ -237,7 +237,7 @@ public class KafkaClusterMetricsTest extends EmbeddedClusterTestBase - event -> event.hasMetricName("task/run/time") - .hasDimension(DruidMetrics.TASK_TYPE, "compact") - .hasDimension(DruidMetrics.TASK_STATUS, "SUCCESS"), -- agg -> agg.hasCountAtLeast(2) -+ agg -> agg.hasCountAtLeast(10) - ); - - // Verify that some segments have been upgraded due to Concurrent Append and Replace -diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/server/HighAvailabilityTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/server/HighAvailabilityTest.java -index 303ad16afa..2a0f9d3cbe 100644 ---- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/server/HighAvailabilityTest.java -+++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/server/HighAvailabilityTest.java -@@ -43,6 +43,7 @@ import org.apache.druid.testing.embedded.EmbeddedOverlord; - import org.apache.druid.testing.embedded.EmbeddedRouter; - import org.apache.druid.testing.embedded.indexing.Resources; - import org.apache.druid.testing.embedded.junit5.EmbeddedClusterTestBase; -+import org.hamcrest.Matchers; - import org.jboss.netty.handler.codec.http.HttpMethod; - import org.jboss.netty.handler.codec.http.HttpResponseStatus; - import org.junit.jupiter.api.Assertions; -@@ -128,7 +129,7 @@ public class HighAvailabilityTest extends EmbeddedClusterTestBase - coordinator1.latchableEmitter().waitForEvent( - event -> event.hasMetricName("segment/metadataCache/used/count") - .hasDimension(DruidMetrics.DATASOURCE, dataSource) -- .hasValueAtLeast(10) -+ .hasValueMatching(Matchers.greaterThanOrEqualTo(10L)) - ); - - // Run sys queries, switch leaders, repeat -diff --git a/extensions-core/druid-catalog/pom.xml b/extensions-core/druid-catalog/pom.xml -index 3227fe6216..2c36306634 100644 ---- a/extensions-core/druid-catalog/pom.xml -+++ b/extensions-core/druid-catalog/pom.xml -@@ -193,6 +193,11 @@ - easymock - test - -+ -+ org.hamcrest -+ hamcrest-all -+ test -+ - - org.apache.curator - curator-test -diff --git a/extensions-core/druid-catalog/src/test/java/org/apache/druid/catalog/compact/CatalogCompactionTest.java b/extensions-core/druid-catalog/src/test/java/org/apache/druid/catalog/compact/CatalogCompactionTest.java -index 9f3106b884..ece6a9ef05 100644 ---- a/extensions-core/druid-catalog/src/test/java/org/apache/druid/catalog/compact/CatalogCompactionTest.java -+++ b/extensions-core/druid-catalog/src/test/java/org/apache/druid/catalog/compact/CatalogCompactionTest.java -@@ -32,6 +32,7 @@ import org.apache.druid.indexing.compact.CompactionSupervisorSpec; - import org.apache.druid.indexing.overlord.Segments; - import org.apache.druid.java.util.common.StringUtils; - import org.apache.druid.java.util.common.granularity.Granularities; -+import org.apache.druid.query.DruidMetrics; - import org.apache.druid.rpc.UpdateResponse; - import org.apache.druid.server.coordinator.CatalogDataSourceCompactionConfig; - import org.apache.druid.server.coordinator.ClusterCompactionConfig; -@@ -52,7 +53,8 @@ import java.util.List; - public class CatalogCompactionTest extends EmbeddedClusterTestBase - { - private final EmbeddedOverlord overlord = new EmbeddedOverlord() -- .addProperty("druid.catalog.client.maxSyncRetries", "0"); -+ .addProperty("druid.catalog.client.maxSyncRetries", "0") -+ .addProperty("druid.manager.segments.pollDuration", "PT1s"); - private final EmbeddedCoordinator coordinator = new EmbeddedCoordinator() - .addProperty("druid.manager.segments.useIncrementalCache", "always"); - private final EmbeddedBroker broker = new EmbeddedBroker() -@@ -119,7 +121,8 @@ public class CatalogCompactionTest extends EmbeddedClusterTestBase - // Wait for compaction to finish - overlord.latchableEmitter().waitForEvent( - event -> event.hasMetricName("task/run/time") -- .hasDimension("taskType", "compact") -+ .hasDimension(DruidMetrics.TASK_TYPE, "compact") -+ .hasDimension(DruidMetrics.DATASOURCE, dataSource) - ); - - // Verify that segments are now compacted to MONTH granularity -diff --git a/indexing-service/src/main/java/org/apache/druid/guice/SupervisorModule.java b/indexing-service/src/main/java/org/apache/druid/guice/SupervisorModule.java -index 73e8e06e89..5642fe1126 100644 ---- a/indexing-service/src/main/java/org/apache/druid/guice/SupervisorModule.java -+++ b/indexing-service/src/main/java/org/apache/druid/guice/SupervisorModule.java -@@ -25,7 +25,10 @@ import com.fasterxml.jackson.databind.jsontype.NamedType; - import com.fasterxml.jackson.databind.module.SimpleModule; - import com.google.common.collect.ImmutableList; - import com.google.inject.Binder; -+import org.apache.druid.indexing.compact.CascadingCompactionTemplate; - import org.apache.druid.indexing.compact.CompactionSupervisorSpec; -+import org.apache.druid.indexing.compact.InlineCompactionJobTemplate; -+import org.apache.druid.indexing.compact.MSQCompactionJobTemplate; - import org.apache.druid.indexing.overlord.supervisor.SupervisorStateManagerConfig; - import org.apache.druid.indexing.scheduledbatch.ScheduledBatchSupervisorSpec; - import org.apache.druid.initialization.DruidModule; -@@ -46,6 +49,9 @@ public class SupervisorModule implements DruidModule - return ImmutableList.of( - new SimpleModule(getClass().getSimpleName()) - .registerSubtypes( -+ new NamedType(InlineCompactionJobTemplate.class, InlineCompactionJobTemplate.TYPE), -+ new NamedType(MSQCompactionJobTemplate.class, MSQCompactionJobTemplate.TYPE), -+ new NamedType(CascadingCompactionTemplate.class, CascadingCompactionTemplate.TYPE), - new NamedType(CompactionSupervisorSpec.class, CompactionSupervisorSpec.TYPE), - new NamedType(ScheduledBatchSupervisorSpec.class, ScheduledBatchSupervisorSpec.TYPE) - ) -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 1c65264f6b..75b148d6e5 100644 ---- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java -+++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java -@@ -27,7 +27,6 @@ import com.fasterxml.jackson.annotation.JsonInclude.Include; - import com.fasterxml.jackson.annotation.JsonProperty; - import com.google.common.annotations.VisibleForTesting; - import com.google.common.base.Preconditions; --import com.google.common.base.Verify; - import com.google.common.collect.BiMap; - import com.google.common.collect.HashBiMap; - import com.google.common.collect.ImmutableList; -@@ -95,8 +94,8 @@ import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager; - import org.apache.druid.segment.transform.CompactionTransformSpec; - import org.apache.druid.segment.transform.TransformSpec; - import org.apache.druid.segment.writeout.SegmentWriteOutMediumFactory; -+import org.apache.druid.server.compaction.CompactionSlotManager; - import org.apache.druid.server.coordinator.CompactionConfigValidationResult; --import org.apache.druid.server.coordinator.duty.CompactSegments; - import org.apache.druid.server.lookup.cache.LookupLoadingSpec; - import org.apache.druid.server.security.ResourceAction; - import org.apache.druid.timeline.DataSegment; -@@ -133,7 +132,7 @@ import java.util.stream.IntStream; - */ - public class CompactionTask extends AbstractBatchIndexTask implements PendingSegmentAllocatingTask - { -- public static final String TYPE = "compact"; -+ public static final String TYPE = CompactionSlotManager.COMPACTION_TASK_TYPE; - private static final Logger log = new Logger(CompactionTask.class); - - /** -@@ -148,10 +147,6 @@ public class CompactionTask extends AbstractBatchIndexTask implements PendingSeg - */ - public static final String CTX_KEY_APPENDERATOR_TRACKING_TASK_ID = "appenderatorTrackingTaskId"; - -- static { -- Verify.verify(TYPE.equals(CompactSegments.COMPACTION_TASK_TYPE)); -- } -- - private final CompactionIOConfig ioConfig; - @Nullable - private final DimensionsSpec dimensionsSpec; -diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/Tasks.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/Tasks.java -index 90fb67116b..b45eb45dc0 100644 ---- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/Tasks.java -+++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/Tasks.java -@@ -47,7 +47,6 @@ public class Tasks - public static final long DEFAULT_SUB_TASK_TIMEOUT_MILLIS = 0; - public static final boolean DEFAULT_FORCE_TIME_CHUNK_LOCK = true; - public static final boolean DEFAULT_STORE_COMPACTION_STATE = false; -- public static final boolean DEFAULT_USE_MAX_MEMORY_ESTIMATES = false; - public static final TaskLockType DEFAULT_TASK_LOCK_TYPE = TaskLockType.EXCLUSIVE; - public static final boolean DEFAULT_USE_CONCURRENT_LOCKS = false; - -diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CascadingCompactionTemplate.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CascadingCompactionTemplate.java -new file mode 100644 -index 0000000000..89d3897436 ---- /dev/null -+++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CascadingCompactionTemplate.java -@@ -0,0 +1,246 @@ -+/* -+ * 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.compact; -+ -+import com.fasterxml.jackson.annotation.JsonCreator; -+import com.fasterxml.jackson.annotation.JsonProperty; -+import org.apache.druid.data.input.impl.AggregateProjectionSpec; -+import org.apache.druid.error.InvalidInput; -+import org.apache.druid.indexer.CompactionEngine; -+import org.apache.druid.indexing.input.DruidInputSource; -+import org.apache.druid.java.util.common.DateTimes; -+import org.apache.druid.java.util.common.granularity.Granularity; -+import org.apache.druid.query.aggregation.AggregatorFactory; -+import org.apache.druid.segment.transform.CompactionTransformSpec; -+import org.apache.druid.server.coordinator.DataSourceCompactionConfig; -+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; -+import org.joda.time.DateTime; -+import org.joda.time.Interval; -+import org.joda.time.Period; -+ -+import javax.annotation.Nullable; -+import java.util.ArrayList; -+import java.util.List; -+import java.util.Map; -+import java.util.Objects; -+ -+/** -+ * Template to perform period-based cascading compaction. Contains a list of -+ * {@link CompactionRule} which divide the segment timeline into compactible -+ * intervals. Each rule specifies a period relative to the current time which is -+ * used to determine its applicable interval: -+ *
    -+ *
  • Rule 1: range = [now - p1, +inf)
  • -+ *
  • Rule 2: range = [now - p2, now - p1)
  • -+ *
  • ...
  • -+ *
  • Rule n: range = (-inf, now - p(n - 1))
  • -+ *
-+ * -+ * If two adjacent rules explicitly specify a segment granularity, the boundary -+ * between them may be {@linkplain CompactionRule#computeStartTime adjusted} -+ * to ensure that there are no uncompacted gaps in the timeline. -+ *

-+ * This template never needs to be deserialized as a {@code BatchIndexingJobTemplate}, -+ * only as a {@link DataSourceCompactionConfig} in {@link CompactionSupervisorSpec}. -+ */ -+public class CascadingCompactionTemplate implements CompactionJobTemplate, DataSourceCompactionConfig -+{ -+ public static final String TYPE = "compactCascade"; -+ -+ private final String dataSource; -+ private final List rules; -+ -+ @JsonCreator -+ public CascadingCompactionTemplate( -+ @JsonProperty("dataSource") String dataSource, -+ @JsonProperty("rules") List rules -+ ) -+ { -+ this.rules = rules; -+ this.dataSource = Objects.requireNonNull(dataSource, "'dataSource' cannot be null"); -+ -+ InvalidInput.conditionalException(rules != null && !rules.isEmpty(), "'rules' cannot be empty"); -+ } -+ -+ @Override -+ @JsonProperty -+ public String getDataSource() -+ { -+ return dataSource; -+ } -+ -+ @JsonProperty -+ public List getRules() -+ { -+ return rules; -+ } -+ -+ @Override -+ public List createCompactionJobs( -+ DruidInputSource source, -+ CompactionJobParams jobParams -+ ) -+ { -+ final List allJobs = new ArrayList<>(); -+ -+ // Include future dates in the first rule -+ final DateTime currentTime = jobParams.getScheduleStartTime(); -+ DateTime previousRuleStartTime = DateTimes.MAX; -+ for (int i = 0; i < rules.size() - 1; ++i) { -+ final CompactionRule rule = rules.get(i); -+ final DateTime ruleStartTime = rule.computeStartTime(currentTime, rules.get(i + 1)); -+ final Interval ruleInterval = new Interval(ruleStartTime, previousRuleStartTime); -+ -+ allJobs.addAll( -+ createJobsForSearchInterval(rule.getTemplate(), ruleInterval, source, jobParams) -+ ); -+ -+ previousRuleStartTime = ruleStartTime; -+ } -+ -+ // Include past dates in the last rule -+ final CompactionRule lastRule = rules.get(rules.size() - 1); -+ final Interval lastRuleInterval = new Interval(DateTimes.MIN, previousRuleStartTime); -+ allJobs.addAll( -+ createJobsForSearchInterval(lastRule.getTemplate(), lastRuleInterval, source, jobParams) -+ ); -+ -+ return allJobs; -+ } -+ -+ private List createJobsForSearchInterval( -+ CompactionJobTemplate template, -+ Interval searchInterval, -+ DruidInputSource inputSource, -+ CompactionJobParams jobParams -+ ) -+ { -+ return template.createCompactionJobs( -+ inputSource.withInterval(searchInterval), -+ jobParams -+ ); -+ } -+ -+ @Override -+ public String getType() -+ { -+ return TYPE; -+ } -+ -+ // Legacy fields from DataSourceCompactionConfig that are not used by this template -+ -+ @Nullable -+ @Override -+ public CompactionEngine getEngine() -+ { -+ return null; -+ } -+ -+ @Override -+ public int getTaskPriority() -+ { -+ return 0; -+ } -+ -+ @Override -+ public long getInputSegmentSizeBytes() -+ { -+ return 0; -+ } -+ -+ @Nullable -+ @Override -+ public Integer getMaxRowsPerSegment() -+ { -+ return 0; -+ } -+ -+ @Override -+ public Period getSkipOffsetFromLatest() -+ { -+ return null; -+ } -+ -+ @Nullable -+ @Override -+ public UserCompactionTaskQueryTuningConfig getTuningConfig() -+ { -+ return null; -+ } -+ -+ @Nullable -+ @Override -+ public UserCompactionTaskIOConfig getIoConfig() -+ { -+ return null; -+ } -+ -+ @Nullable -+ @Override -+ public Map getTaskContext() -+ { -+ return Map.of(); -+ } -+ -+ @Nullable -+ @Override -+ public Granularity getSegmentGranularity() -+ { -+ return null; -+ } -+ -+ @Nullable -+ @Override -+ public UserCompactionTaskGranularityConfig getGranularitySpec() -+ { -+ return null; -+ } -+ -+ @Nullable -+ @Override -+ public List getProjections() -+ { -+ return List.of(); -+ } -+ -+ @Nullable -+ @Override -+ public CompactionTransformSpec getTransformSpec() -+ { -+ return null; -+ } -+ -+ @Nullable -+ @Override -+ public UserCompactionTaskDimensionsConfig getDimensionsSpec() -+ { -+ return null; -+ } -+ -+ @Nullable -+ @Override -+ public AggregatorFactory[] getMetricsSpec() -+ { -+ return new AggregatorFactory[0]; -+ } -+} -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 -new file mode 100644 -index 0000000000..c8563290a6 ---- /dev/null -+++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionConfigBasedJobTemplate.java -@@ -0,0 +1,156 @@ -+/* -+ * 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.compact; -+ -+import org.apache.druid.client.indexing.ClientCompactionTaskQuery; -+import org.apache.druid.error.DruidException; -+import org.apache.druid.error.InvalidInput; -+import org.apache.druid.indexing.input.DruidInputSource; -+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.CompactionSlotManager; -+import org.apache.druid.server.compaction.DataSourceCompactibleSegmentIterator; -+import org.apache.druid.server.compaction.NewestSegmentFirstPolicy; -+import org.apache.druid.server.coordinator.DataSourceCompactionConfig; -+import org.apache.druid.server.coordinator.InlineSchemaDataSourceCompactionConfig; -+import org.apache.druid.server.coordinator.duty.CompactSegments; -+import org.apache.druid.timeline.SegmentTimeline; -+import org.joda.time.Interval; -+import org.joda.time.Period; -+ -+import javax.annotation.Nullable; -+import java.util.ArrayList; -+import java.util.List; -+import java.util.Objects; -+ -+/** -+ * This template never needs to be deserialized as a {@code BatchIndexingJobTemplate}. -+ * It is just a delegating template that uses a {@link DataSourceCompactionConfig} -+ * to create compaction jobs. -+ */ -+public class CompactionConfigBasedJobTemplate implements CompactionJobTemplate -+{ -+ private final DataSourceCompactionConfig config; -+ -+ public CompactionConfigBasedJobTemplate(DataSourceCompactionConfig config) -+ { -+ this.config = config; -+ } -+ -+ public static CompactionConfigBasedJobTemplate create(String dataSource, CompactionStateMatcher stateMatcher) -+ { -+ return new CompactionConfigBasedJobTemplate( -+ InlineSchemaDataSourceCompactionConfig -+ .builder() -+ .forDataSource(dataSource) -+ .withSkipOffsetFromLatest(Period.ZERO) -+ .withTransformSpec(stateMatcher.getTransformSpec()) -+ .withProjections(stateMatcher.getProjections()) -+ .withMetricsSpec(stateMatcher.getMetricsSpec()) -+ .withGranularitySpec(stateMatcher.getGranularitySpec()) -+ .build() -+ ); -+ } -+ -+ @Nullable -+ @Override -+ public Granularity getSegmentGranularity() -+ { -+ return config.getSegmentGranularity(); -+ } -+ -+ @Override -+ public List createCompactionJobs( -+ DruidInputSource source, -+ CompactionJobParams params -+ ) -+ { -+ final DataSourceCompactibleSegmentIterator segmentIterator = getCompactibleCandidates(source, params); -+ -+ final List jobs = new ArrayList<>(); -+ -+ // Create a job for each CompactionCandidate -+ while (segmentIterator.hasNext()) { -+ final CompactionCandidate candidate = segmentIterator.next(); -+ -+ ClientCompactionTaskQuery taskPayload -+ = CompactSegments.createCompactionTask(candidate, config, params.getClusterCompactionConfig().getEngine()); -+ jobs.add( -+ new CompactionJob( -+ taskPayload, -+ candidate, -+ CompactionSlotManager.getMaxTaskSlotsForNativeCompactionTask(taskPayload.getTuningConfig()) -+ ) -+ ); -+ } -+ -+ return jobs; -+ } -+ -+ @Override -+ public String getType() -+ { -+ throw DruidException.defensive( -+ "This template cannot be serialized. It is an adapter used to create jobs" -+ + " using a legacy DataSourceCompactionConfig. Do not use this template" -+ + " in a supervisor spec directly. Use types [compactCatalog], [compactMsq]" -+ + " or [compactInline] instead." -+ ); -+ } -+ -+ /** -+ * Creates an iterator over the compactible candidate segments for the given -+ * params. Adds stats for segments that are already compacted to the -+ * {@link CompactionJobParams#getSnapshotBuilder()}. -+ */ -+ DataSourceCompactibleSegmentIterator getCompactibleCandidates( -+ DruidInputSource source, -+ CompactionJobParams params -+ ) -+ { -+ validateInput(source); -+ -+ final Interval searchInterval = Objects.requireNonNull(source.getInterval()); -+ -+ final SegmentTimeline timeline = params.getTimeline(config.getDataSource()); -+ final DataSourceCompactibleSegmentIterator iterator = new DataSourceCompactibleSegmentIterator( -+ config, -+ timeline, -+ Intervals.complementOf(searchInterval), -+ new NewestSegmentFirstPolicy(null) -+ ); -+ -+ // Collect stats for segments that are already compacted -+ iterator.getCompactedSegments().forEach(entry -> params.getSnapshotBuilder().addToComplete(entry)); -+ -+ return iterator; -+ } -+ -+ private void validateInput(DruidInputSource druidInputSource) -+ { -+ if (!druidInputSource.getDataSource().equals(config.getDataSource())) { -+ throw InvalidInput.exception( -+ "Datasource[%s] in compaction config does not match datasource[%s] in input source", -+ config.getDataSource(), druidInputSource.getDataSource() -+ ); -+ } -+ } -+} -diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionJob.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionJob.java -new file mode 100644 -index 0000000000..7a7e7fdc1e ---- /dev/null -+++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionJob.java -@@ -0,0 +1,81 @@ -+/* -+ * Licensed to the Apache Software Foundation (ASF) under one -+ * or more contributor license agreements. See the NOTICE file -+ * distributed with this work for additional information -+ * regarding copyright ownership. The ASF licenses this file -+ * to you under the Apache License, Version 2.0 (the -+ * "License"); you may not use this file except in compliance -+ * with the License. You may obtain a copy of the License at -+ * -+ * http://www.apache.org/licenses/LICENSE-2.0 -+ * -+ * Unless required by applicable law or agreed to in writing, -+ * software distributed under the License is distributed on an -+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -+ * KIND, either express or implied. See the License for the -+ * specific language governing permissions and limitations -+ * under the License. -+ */ -+ -+package org.apache.druid.indexing.compact; -+ -+import org.apache.druid.client.indexing.ClientCompactionTaskQuery; -+import org.apache.druid.indexing.template.BatchIndexingJob; -+import org.apache.druid.query.http.ClientSqlQuery; -+import org.apache.druid.server.compaction.CompactionCandidate; -+ -+/** -+ * {@link BatchIndexingJob} to compact an interval of a datasource. -+ */ -+public class CompactionJob extends BatchIndexingJob -+{ -+ private final CompactionCandidate candidate; -+ private final int maxRequiredTaskSlots; -+ -+ public CompactionJob( -+ ClientCompactionTaskQuery task, -+ CompactionCandidate candidate, -+ int maxRequiredTaskSlots -+ ) -+ { -+ super(task, null); -+ this.candidate = candidate; -+ this.maxRequiredTaskSlots = maxRequiredTaskSlots; -+ } -+ -+ public CompactionJob( -+ ClientSqlQuery msqQuery, -+ CompactionCandidate candidate, -+ int maxRequiredTaskSlots -+ ) -+ { -+ super(null, msqQuery); -+ this.candidate = candidate; -+ this.maxRequiredTaskSlots = maxRequiredTaskSlots; -+ } -+ -+ public String getDataSource() -+ { -+ return candidate.getDataSource(); -+ } -+ -+ public CompactionCandidate getCandidate() -+ { -+ return candidate; -+ } -+ -+ public int getMaxRequiredTaskSlots() -+ { -+ return maxRequiredTaskSlots; -+ } -+ -+ @Override -+ public String toString() -+ { -+ return "CompactionJob{" + -+ super.toString() + -+ ", candidate=" + candidate + -+ ", maxRequiredTaskSlots=" + maxRequiredTaskSlots + -+ '}'; -+ } -+} -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 -new file mode 100644 -index 0000000000..0113f1b78b ---- /dev/null -+++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionJobParams.java -@@ -0,0 +1,96 @@ -+/* -+ * 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.compact; -+ -+import org.apache.druid.server.compaction.CompactionSnapshotBuilder; -+import org.apache.druid.server.coordinator.ClusterCompactionConfig; -+import org.apache.druid.timeline.SegmentTimeline; -+import org.joda.time.DateTime; -+ -+/** -+ * Parameters used while creating a {@link CompactionJob} using a {@link CompactionJobTemplate}. -+ */ -+public class CompactionJobParams -+{ -+ private final DateTime scheduleStartTime; -+ private final TimelineProvider timelineProvider; -+ private final ClusterCompactionConfig clusterCompactionConfig; -+ private final CompactionSnapshotBuilder snapshotBuilder; -+ -+ public CompactionJobParams( -+ DateTime scheduleStartTime, -+ ClusterCompactionConfig clusterCompactionConfig, -+ TimelineProvider timelineProvider, -+ CompactionSnapshotBuilder snapshotBuilder -+ ) -+ { -+ this.scheduleStartTime = scheduleStartTime; -+ this.clusterCompactionConfig = clusterCompactionConfig; -+ this.timelineProvider = timelineProvider; -+ this.snapshotBuilder = snapshotBuilder; -+ } -+ -+ /** -+ * Timestamp denoting the start of the current run of the scheduler which has -+ * triggered creation of jobs using these {@link CompactionJobParams}. -+ */ -+ public DateTime getScheduleStartTime() -+ { -+ return scheduleStartTime; -+ } -+ -+ /** -+ * Cluster-level compaction config containing details such as the engine, -+ * compaction search policy, etc. to use while creating {@link CompactionJob}. -+ */ -+ public ClusterCompactionConfig getClusterCompactionConfig() -+ { -+ return clusterCompactionConfig; -+ } -+ -+ /** -+ * Provides the full {@link SegmentTimeline} of used segments for the given -+ * datasource. This timeline is used to identify eligible intervals for which -+ * compaction jobs should be created. -+ */ -+ public SegmentTimeline getTimeline(String dataSource) -+ { -+ return timelineProvider.getTimelineForDataSource(dataSource); -+ } -+ -+ /** -+ * 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 -+ * candidate intervals are identified as compacted, skipped or pending, they -+ * should be updated in this snapshot builder by invoking -+ * {@link CompactionSnapshotBuilder#addToComplete}, {@link CompactionSnapshotBuilder#addToSkipped} -+ * and {@link CompactionSnapshotBuilder#addToPending} respectively. -+ */ -+ public CompactionSnapshotBuilder getSnapshotBuilder() -+ { -+ return snapshotBuilder; -+ } -+ -+ @FunctionalInterface -+ public interface TimelineProvider -+ { -+ SegmentTimeline getTimelineForDataSource(String dataSource); -+ } -+} -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 -new file mode 100644 -index 0000000000..77886af1a0 ---- /dev/null -+++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionJobQueue.java -@@ -0,0 +1,373 @@ -+/* -+ * 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.compact; -+ -+import com.fasterxml.jackson.databind.ObjectMapper; -+import org.apache.druid.client.DataSourcesSnapshot; -+import org.apache.druid.client.broker.BrokerClient; -+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.indexer.TaskState; -+import org.apache.druid.indexer.TaskStatus; -+import org.apache.druid.indexing.common.actions.TaskActionClientFactory; -+import org.apache.druid.indexing.common.task.Task; -+import org.apache.druid.indexing.input.DruidInputSource; -+import org.apache.druid.indexing.overlord.GlobalTaskLockbox; -+import org.apache.druid.indexing.template.BatchIndexingJob; -+import org.apache.druid.java.util.common.DateTimes; -+import org.apache.druid.java.util.common.Stopwatch; -+import org.apache.druid.java.util.common.logger.Logger; -+import org.apache.druid.rpc.indexing.OverlordClient; -+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.CompactionSnapshotBuilder; -+import org.apache.druid.server.compaction.CompactionStatus; -+import org.apache.druid.server.compaction.CompactionStatusTracker; -+import org.apache.druid.server.coordinator.AutoCompactionSnapshot; -+import org.apache.druid.server.coordinator.ClusterCompactionConfig; -+import org.apache.druid.server.coordinator.CompactionConfigValidationResult; -+import org.apache.druid.server.coordinator.stats.CoordinatorRunStats; -+import org.apache.druid.server.coordinator.stats.Dimension; -+import org.apache.druid.server.coordinator.stats.RowKey; -+import org.apache.druid.server.coordinator.stats.Stats; -+ -+import javax.annotation.Nullable; -+import javax.annotation.concurrent.NotThreadSafe; -+import java.util.ArrayList; -+import java.util.HashMap; -+import java.util.HashSet; -+import java.util.List; -+import java.util.Map; -+import java.util.PriorityQueue; -+import java.util.Set; -+import java.util.stream.Collectors; -+ -+/** -+ * Iterates over all eligible compaction jobs in order of their priority. -+ * A fresh instance of this class must be used in every run of the -+ * {@link CompactionScheduler}. -+ *

-+ * Unlike the Coordinator duty {@code CompactSegments}, the job queue currently -+ * does not cancel running compaction tasks even if their target segment -+ * granularity has changed. This has not been done here for simplicity since the -+ * {@code CompactionJobQueue} uses compaction templates and may have a different -+ * target segment granulariy for different intervals of the same datasource. -+ * The cancellation of invalid tasks has been left as a future enhancement. -+ */ -+@NotThreadSafe -+public class CompactionJobQueue -+{ -+ private static final Logger log = new Logger(CompactionJobQueue.class); -+ -+ private final CompactionJobParams jobParams; -+ private final CompactionCandidateSearchPolicy searchPolicy; -+ private final ClusterCompactionConfig clusterCompactionConfig; -+ -+ private final ObjectMapper objectMapper; -+ private final CompactionStatusTracker statusTracker; -+ private final TaskActionClientFactory taskActionClientFactory; -+ private final OverlordClient overlordClient; -+ private final GlobalTaskLockbox taskLockbox; -+ private final BrokerClient brokerClient; -+ -+ private final CompactionSnapshotBuilder snapshotBuilder; -+ private final PriorityQueue queue; -+ private final CoordinatorRunStats runStats; -+ -+ private final Set activeSupervisors; -+ private final Map submittedTaskIdToJob; -+ -+ public CompactionJobQueue( -+ DataSourcesSnapshot dataSourcesSnapshot, -+ ClusterCompactionConfig clusterCompactionConfig, -+ CompactionStatusTracker statusTracker, -+ TaskActionClientFactory taskActionClientFactory, -+ GlobalTaskLockbox taskLockbox, -+ OverlordClient overlordClient, -+ BrokerClient brokerClient, -+ ObjectMapper objectMapper -+ ) -+ { -+ this.runStats = new CoordinatorRunStats(); -+ this.snapshotBuilder = new CompactionSnapshotBuilder(runStats); -+ this.clusterCompactionConfig = clusterCompactionConfig; -+ this.searchPolicy = clusterCompactionConfig.getCompactionPolicy(); -+ this.queue = new PriorityQueue<>( -+ (o1, o2) -> searchPolicy.compareCandidates(o1.getCandidate(), o2.getCandidate()) -+ ); -+ this.submittedTaskIdToJob = new HashMap<>(); -+ this.activeSupervisors = new HashSet<>(); -+ this.jobParams = new CompactionJobParams( -+ DateTimes.nowUtc(), -+ clusterCompactionConfig, -+ dataSourcesSnapshot.getUsedSegmentsTimelinesPerDataSource()::get, -+ snapshotBuilder -+ ); -+ -+ this.taskActionClientFactory = taskActionClientFactory; -+ this.overlordClient = overlordClient; -+ this.brokerClient = brokerClient; -+ this.statusTracker = statusTracker; -+ this.objectMapper = objectMapper; -+ this.taskLockbox = taskLockbox; -+ } -+ -+ /** -+ * Creates jobs for the given {@link CompactionSupervisor} and adds them to -+ * the job queue. -+ *

-+ * This method is idempotent. If jobs for the given supervisor already exist -+ * in the queue, the method does nothing. -+ */ -+ public void createAndEnqueueJobs( -+ CompactionSupervisor supervisor, -+ DruidInputSource source -+ ) -+ { -+ final Stopwatch jobCreationTime = Stopwatch.createStarted(); -+ final String supervisorId = supervisor.getSpec().getId(); -+ try { -+ if (supervisor.shouldCreateJobs() && !activeSupervisors.contains(supervisorId)) { -+ // Queue fresh jobs -+ final List jobs = supervisor.createJobs(source, jobParams); -+ jobs.forEach(job -> snapshotBuilder.addToPending(job.getCandidate())); -+ -+ queue.addAll(jobs); -+ activeSupervisors.add(supervisorId); -+ -+ runStats.add( -+ Stats.Compaction.CREATED_JOBS, -+ RowKey.of(Dimension.DATASOURCE, source.getDataSource()), -+ jobs.size() -+ ); -+ } else { -+ log.debug("Skipping job creation for supervisor[%s]", supervisorId); -+ } -+ } -+ catch (Exception e) { -+ log.error(e, "Error while creating jobs for supervisor[%s]", supervisorId); -+ } -+ finally { -+ runStats.add( -+ Stats.Compaction.JOB_CREATION_TIME, -+ RowKey.of(Dimension.DATASOURCE, source.getDataSource()), -+ jobCreationTime.millisElapsed() -+ ); -+ } -+ } -+ -+ /** -+ * Removes all existing jobs for the given datasource from the queue. -+ */ -+ public void removeJobs(String dataSource) -+ { -+ final List jobsToRemove = queue -+ .stream() -+ .filter(job -> job.getDataSource().equals(dataSource)) -+ .collect(Collectors.toList()); -+ -+ queue.removeAll(jobsToRemove); -+ log.info("Removed [%d] jobs for datasource[%s] from queue.", jobsToRemove.size(), dataSource); -+ } -+ -+ /** -+ * Submits jobs which are ready to either the Overlord or a Broker (if it is -+ * an MSQ SQL job). -+ */ -+ public void runReadyJobs() -+ { -+ final CompactionSlotManager slotManager = new CompactionSlotManager( -+ overlordClient, -+ statusTracker, -+ clusterCompactionConfig -+ ); -+ slotManager.reserveTaskSlotsForRunningCompactionTasks(); -+ -+ final List pendingJobs = new ArrayList<>(); -+ while (!queue.isEmpty()) { -+ final CompactionJob job = queue.poll(); -+ if (startJobIfPendingAndReady(job, searchPolicy, pendingJobs, slotManager)) { -+ runStats.add(Stats.Compaction.SUBMITTED_TASKS, RowKey.of(Dimension.DATASOURCE, job.getDataSource()), 1); -+ } -+ } -+ -+ // Requeue pending jobs so that they can be launched when slots become available -+ queue.addAll(pendingJobs); -+ } -+ -+ /** -+ * Notifies completion of the given so that the compaction snapshots may be -+ * updated. -+ */ -+ public void onTaskFinished(String taskId, TaskStatus taskStatus) -+ { -+ final CompactionJob job = submittedTaskIdToJob.remove(taskId); -+ if (job == null || !taskStatus.getStatusCode().isComplete()) { -+ // This is an unknown task ID -+ return; -+ } -+ -+ if (taskStatus.getStatusCode() == TaskState.FAILED) { -+ // Add this job back to the queue -+ queue.add(job); -+ } else { -+ snapshotBuilder.moveFromPendingToCompleted(job.getCandidate()); -+ } -+ } -+ -+ public CoordinatorRunStats getRunStats() -+ { -+ return runStats; -+ } -+ -+ /** -+ * Builds compaction snapshots for all the datasources being tracked by this -+ * queue. -+ */ -+ public Map getSnapshots() -+ { -+ return snapshotBuilder.build(); -+ } -+ -+ /** -+ * Starts a job if it is ready and is not already in progress. -+ * -+ * @return true if the job was submitted successfully for execution -+ */ -+ private boolean startJobIfPendingAndReady( -+ CompactionJob job, -+ CompactionCandidateSearchPolicy policy, -+ List pendingJobs, -+ CompactionSlotManager slotManager -+ ) -+ { -+ // Check if the job is a valid compaction job -+ final CompactionCandidate candidate = job.getCandidate(); -+ final CompactionConfigValidationResult validationResult = validateCompactionJob(job); -+ if (!validationResult.isValid()) { -+ log.error("Skipping invalid compaction job[%s] due to reason[%s].", job, validationResult.getReason()); -+ snapshotBuilder.moveFromPendingToSkipped(candidate); -+ return false; -+ } -+ -+ // Check if the job is already running, completed or skipped -+ final CompactionStatus compactionStatus = getCurrentStatusForJob(job, policy); -+ switch (compactionStatus.getState()) { -+ case RUNNING: -+ return false; -+ case COMPLETE: -+ snapshotBuilder.moveFromPendingToCompleted(candidate); -+ return false; -+ case SKIPPED: -+ snapshotBuilder.moveFromPendingToSkipped(candidate); -+ return false; -+ default: -+ break; -+ } -+ -+ // Check if enough compaction task slots are available -+ if (job.getMaxRequiredTaskSlots() > slotManager.getNumAvailableTaskSlots()) { -+ pendingJobs.add(job); -+ return false; -+ } -+ -+ // Reserve task slots and try to start the task -+ slotManager.reserveTaskSlots(job.getMaxRequiredTaskSlots()); -+ final String taskId = startTaskIfReady(job); -+ if (taskId == null) { -+ // Mark the job as skipped for now as the intervals might be locked by other tasks -+ snapshotBuilder.moveFromPendingToSkipped(candidate); -+ return false; -+ } else { -+ statusTracker.onTaskSubmitted(taskId, job.getCandidate()); -+ submittedTaskIdToJob.put(taskId, job); -+ return true; -+ } -+ } -+ -+ /** -+ * Starts the given job if the underlying Task is able to acquire locks. -+ * -+ * @return Non-null taskId if the Task was submitted successfully. -+ */ -+ @Nullable -+ private String startTaskIfReady(CompactionJob job) -+ { -+ // Assume MSQ jobs to be always ready -+ if (job.isMsq()) { -+ try { -+ return FutureUtils.getUnchecked(brokerClient.submitSqlTask(job.getNonNullMsqQuery()), true) -+ .getTaskId(); -+ } -+ catch (Exception e) { -+ log.error(e, "Error while submitting query[%s] to Broker", job.getNonNullMsqQuery()); -+ } -+ } -+ -+ final ClientTaskQuery taskQuery = job.getNonNullTask(); -+ final Task task = objectMapper.convertValue(taskQuery, Task.class); -+ -+ log.debug( -+ "Checking readiness of task[%s] with interval[%s]", -+ task.getId(), job.getCandidate().getCompactionInterval() -+ ); -+ try { -+ taskLockbox.add(task); -+ if (task.isReady(taskActionClientFactory.create(task))) { -+ // Hold the locks acquired by task.isReady() as we will reacquire them anyway -+ FutureUtils.getUnchecked(overlordClient.runTask(task.getId(), task), true); -+ return task.getId(); -+ } else { -+ taskLockbox.unlockAll(task); -+ return null; -+ } -+ } -+ catch (Exception e) { -+ log.error(e, "Error while submitting task[%s] to Overlord", task.getId()); -+ taskLockbox.unlockAll(task); -+ return null; -+ } -+ } -+ -+ public CompactionStatus getCurrentStatusForJob(CompactionJob job, CompactionCandidateSearchPolicy policy) -+ { -+ final CompactionStatus compactionStatus = statusTracker.computeCompactionStatus(job.getCandidate(), policy); -+ final CompactionCandidate candidatesWithStatus = job.getCandidate().withCurrentStatus(null); -+ statusTracker.onCompactionStatusComputed(candidatesWithStatus, null); -+ return compactionStatus; -+ } -+ -+ public static CompactionConfigValidationResult validateCompactionJob(BatchIndexingJob job) -+ { -+ // For MSQ jobs, do not perform any validation -+ if (job.isMsq()) { -+ return CompactionConfigValidationResult.success(); -+ } -+ -+ final ClientTaskQuery task = job.getNonNullTask(); -+ if (!(task instanceof ClientCompactionTaskQuery)) { -+ return CompactionConfigValidationResult.failure("Invalid task type[%s]", task.getType()); -+ } -+ -+ return CompactionConfigValidationResult.success(); -+ } -+} -diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionJobTemplate.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionJobTemplate.java -new file mode 100644 -index 0000000000..cd5096a8cf ---- /dev/null -+++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionJobTemplate.java -@@ -0,0 +1,50 @@ -+/* -+ * 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.compact; -+ -+import org.apache.druid.indexing.input.DruidInputSource; -+import org.apache.druid.indexing.template.BatchIndexingJobTemplate; -+import org.apache.druid.java.util.common.granularity.Granularity; -+ -+import javax.annotation.Nullable; -+import java.util.List; -+ -+/** -+ * Base indexing template for creating {@link CompactionJob}. -+ */ -+public interface CompactionJobTemplate extends BatchIndexingJobTemplate -+{ -+ /** -+ * Creates compaction jobs with this template for the given datasource. -+ */ -+ List createCompactionJobs( -+ DruidInputSource source, -+ CompactionJobParams jobParams -+ ); -+ -+ /** -+ * Granularity of segments created upon successful compaction. -+ * -+ * @return null only if this template does not change segment granularity upon -+ * successful compaction. -+ */ -+ @Nullable -+ Granularity getSegmentGranularity(); -+} -diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionRule.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionRule.java -new file mode 100644 -index 0000000000..10099566ea ---- /dev/null -+++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionRule.java -@@ -0,0 +1,88 @@ -+/* -+ * 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.compact; -+ -+import com.fasterxml.jackson.annotation.JsonCreator; -+import com.fasterxml.jackson.annotation.JsonProperty; -+import org.apache.druid.java.util.common.granularity.Granularity; -+import org.joda.time.DateTime; -+import org.joda.time.Period; -+ -+/** -+ * A single rule used inside {@link CascadingCompactionTemplate}. -+ */ -+public class CompactionRule -+{ -+ private final Period period; -+ private final CompactionJobTemplate template; -+ -+ @JsonCreator -+ public CompactionRule( -+ @JsonProperty("period") Period period, -+ @JsonProperty("template") CompactionJobTemplate template -+ ) -+ { -+ this.period = period; -+ this.template = template; -+ } -+ -+ @JsonProperty -+ public CompactionJobTemplate getTemplate() -+ { -+ return template; -+ } -+ -+ @JsonProperty -+ public Period getPeriod() -+ { -+ return period; -+ } -+ -+ /** -+ * Computes the start time of this rule by subtracting its period from the -+ * reference timestamp. -+ *

-+ * If both this rule and the {@code beforeRule} explicitly specify a target -+ * segment granularity, the start time may be adjusted to ensure that there -+ * are no uncompacted gaps left in the timeline. -+ * -+ * @param referenceTime Current time when the rules are being evaluated -+ * @param beforeRule The rule before this one in chronological order -+ */ -+ public DateTime computeStartTime(DateTime referenceTime, CompactionRule beforeRule) -+ { -+ final Granularity granularity = template.getSegmentGranularity(); -+ final Granularity beforeGranularity = beforeRule.template.getSegmentGranularity(); -+ -+ final DateTime calculatedStartTime = referenceTime.minus(period); -+ -+ if (granularity == null || beforeGranularity == null) { -+ return calculatedStartTime; -+ } else { -+ // The gap can be filled only if it is bigger than the granularity of this rule. -+ // If beforeGranularity > granularity, gap would always be smaller than both -+ final DateTime beforeRuleEffectiveEnd = beforeGranularity.bucketStart(calculatedStartTime); -+ final DateTime possibleStartTime = granularity.bucketStart(beforeRuleEffectiveEnd); -+ return possibleStartTime.isBefore(beforeRuleEffectiveEnd) -+ ? granularity.increment(possibleStartTime) -+ : possibleStartTime; -+ } -+ } -+} -diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionScheduler.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionScheduler.java -index 5f0aa6e3ea..6f5ed1a7a6 100644 ---- a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionScheduler.java -+++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionScheduler.java -@@ -65,7 +65,7 @@ public interface CompactionScheduler - /** - * Starts compaction for a datasource if not already running. - */ -- void startCompaction(String dataSourceName, DataSourceCompactionConfig compactionConfig); -+ void startCompaction(String dataSourceName, CompactionSupervisor supervisor); - - /** - * Stops compaction for a datasource if currently running. -diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionStateMatcher.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionStateMatcher.java -new file mode 100644 -index 0000000000..84a95e2f6e ---- /dev/null -+++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionStateMatcher.java -@@ -0,0 +1,186 @@ -+/* -+ * 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.compact; -+ -+import com.fasterxml.jackson.annotation.JsonCreator; -+import com.fasterxml.jackson.annotation.JsonInclude; -+import com.fasterxml.jackson.annotation.JsonProperty; -+import org.apache.druid.data.input.impl.AggregateProjectionSpec; -+import org.apache.druid.data.input.impl.DimensionsSpec; -+import org.apache.druid.indexer.partitions.PartitionsSpec; -+import org.apache.druid.java.util.common.granularity.Granularity; -+import org.apache.druid.query.aggregation.AggregatorFactory; -+import org.apache.druid.segment.IndexSpec; -+import org.apache.druid.segment.transform.CompactionTransformSpec; -+import org.apache.druid.server.coordinator.UserCompactionTaskGranularityConfig; -+ -+import javax.annotation.Nullable; -+import java.util.Arrays; -+import java.util.List; -+import java.util.Objects; -+ -+/** -+ * Target compacted state of segments used to determine if compaction is needed -+ * for an interval. An explicitly defined target state helps avoid superfluous -+ * compaction when only the job definition has changed. -+ *

-+ * This class is mostly a duplicate of {@code CompactionState} but is kept -+ * separate to allow: -+ *

    -+ *
  • fields to be nullable so that only non-null fields are used for matching
  • -+ *
  • legacy "compaction-incompatible" fields such as {@link #transformSpec} to -+ * be removed in the future. These fields do not just change the layout/partitioning -+ * of the data but may also alter its meaning which does not fall in the purview -+ * of compaction.
  • -+ *
-+ */ -+@JsonInclude(JsonInclude.Include.NON_NULL) -+public class CompactionStateMatcher -+{ -+ private final PartitionsSpec partitionsSpec; -+ private final DimensionsSpec dimensionsSpec; -+ private final CompactionTransformSpec transformSpec; -+ private final IndexSpec indexSpec; -+ private final UserCompactionTaskGranularityConfig granularitySpec; -+ private final AggregatorFactory[] metricsSpec; -+ private final List projections; -+ -+ @JsonCreator -+ public CompactionStateMatcher( -+ @JsonProperty("partitionsSpec") @Nullable PartitionsSpec partitionsSpec, -+ @JsonProperty("dimensionsSpec") @Nullable DimensionsSpec dimensionsSpec, -+ @JsonProperty("metricsSpec") @Nullable AggregatorFactory[] metricsSpec, -+ @JsonProperty("transformSpec") @Nullable CompactionTransformSpec transformSpec, -+ @JsonProperty("indexSpec") @Nullable IndexSpec indexSpec, -+ @JsonProperty("granularitySpec") @Nullable UserCompactionTaskGranularityConfig granularitySpec, -+ @JsonProperty("projections") @Nullable List projections -+ ) -+ { -+ this.partitionsSpec = partitionsSpec; -+ this.dimensionsSpec = dimensionsSpec; -+ this.metricsSpec = metricsSpec; -+ this.transformSpec = transformSpec; -+ this.indexSpec = indexSpec; -+ this.granularitySpec = granularitySpec; -+ this.projections = projections; -+ } -+ -+ @Nullable -+ @JsonProperty -+ public PartitionsSpec getPartitionsSpec() -+ { -+ return partitionsSpec; -+ } -+ -+ @Nullable -+ @JsonProperty -+ public DimensionsSpec getDimensionsSpec() -+ { -+ return dimensionsSpec; -+ } -+ -+ @Nullable -+ @JsonProperty -+ public AggregatorFactory[] getMetricsSpec() -+ { -+ return metricsSpec; -+ } -+ -+ @Nullable -+ @JsonProperty -+ public CompactionTransformSpec getTransformSpec() -+ { -+ return transformSpec; -+ } -+ -+ @Nullable -+ @JsonProperty -+ public IndexSpec getIndexSpec() -+ { -+ return indexSpec; -+ } -+ -+ @Nullable -+ @JsonProperty -+ public UserCompactionTaskGranularityConfig getGranularitySpec() -+ { -+ return granularitySpec; -+ } -+ -+ @JsonProperty -+ @Nullable -+ public List getProjections() -+ { -+ return projections; -+ } -+ -+ @Nullable -+ public Granularity getSegmentGranularity() -+ { -+ return granularitySpec == null ? null : granularitySpec.getSegmentGranularity(); -+ } -+ -+ @Override -+ public boolean equals(Object o) -+ { -+ if (this == o) { -+ return true; -+ } -+ if (o == null || getClass() != o.getClass()) { -+ return false; -+ } -+ CompactionStateMatcher that = (CompactionStateMatcher) o; -+ return Objects.equals(partitionsSpec, that.partitionsSpec) && -+ Objects.equals(dimensionsSpec, that.dimensionsSpec) && -+ Objects.equals(transformSpec, that.transformSpec) && -+ Objects.equals(indexSpec, that.indexSpec) && -+ Objects.equals(granularitySpec, that.granularitySpec) && -+ Arrays.equals(metricsSpec, that.metricsSpec) && -+ Objects.equals(projections, that.projections); -+ } -+ -+ @Override -+ public int hashCode() -+ { -+ return Objects.hash( -+ partitionsSpec, -+ dimensionsSpec, -+ transformSpec, -+ indexSpec, -+ granularitySpec, -+ Arrays.hashCode(metricsSpec), -+ projections -+ ); -+ } -+ -+ @Override -+ public String toString() -+ { -+ return "CompactionState{" + -+ "partitionsSpec=" + partitionsSpec + -+ ", dimensionsSpec=" + dimensionsSpec + -+ ", transformSpec=" + transformSpec + -+ ", indexSpec=" + indexSpec + -+ ", granularitySpec=" + granularitySpec + -+ ", metricsSpec=" + Arrays.toString(metricsSpec) + -+ ", projections=" + projections + -+ '}'; -+ } -+} -diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionSupervisor.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionSupervisor.java -index 851b3920b1..f9ba0eee6d 100644 ---- a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionSupervisor.java -+++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionSupervisor.java -@@ -19,6 +19,7 @@ - - package org.apache.druid.indexing.compact; - -+import org.apache.druid.indexing.input.DruidInputSource; - import org.apache.druid.indexing.overlord.DataSourceMetadata; - import org.apache.druid.indexing.overlord.supervisor.Supervisor; - import org.apache.druid.indexing.overlord.supervisor.SupervisorReport; -@@ -29,6 +30,7 @@ import org.apache.druid.java.util.common.logger.Logger; - import org.apache.druid.server.coordinator.AutoCompactionSnapshot; - - import javax.annotation.Nullable; -+import java.util.List; - - /** - * Supervisor for compaction of a single datasource. -@@ -51,6 +53,31 @@ public class CompactionSupervisor implements Supervisor - this.dataSource = supervisorSpec.getSpec().getDataSource(); - } - -+ public CompactionSupervisorSpec getSpec() -+ { -+ return supervisorSpec; -+ } -+ -+ /** -+ * Checks if this supervisor is ready to create jobs in the current run of the -+ * scheduler. -+ */ -+ public boolean shouldCreateJobs() -+ { -+ return !supervisorSpec.isSuspended(); -+ } -+ -+ /** -+ * Creates compaction jobs for this supervisor. -+ */ -+ public List createJobs( -+ DruidInputSource inputSource, -+ CompactionJobParams jobParams -+ ) -+ { -+ return supervisorSpec.getTemplate().createCompactionJobs(inputSource, jobParams); -+ } -+ - @Override - public void start() - { -@@ -66,7 +93,7 @@ public class CompactionSupervisor implements Supervisor - ); - } else { - log.info("Starting compaction for dataSource[%s].", dataSource); -- scheduler.startCompaction(dataSource, supervisorSpec.getSpec()); -+ scheduler.startCompaction(dataSource, this); - } - } - -diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionSupervisorSpec.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionSupervisorSpec.java -index ca6008cc9d..998d7e0c7e 100644 ---- a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionSupervisorSpec.java -+++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionSupervisorSpec.java -@@ -93,6 +93,18 @@ public class CompactionSupervisorSpec implements SupervisorSpec - return new CompactionSupervisor(this, scheduler); - } - -+ /** -+ * @return {@link CompactionJobTemplate} used to create jobs for the supervisor. -+ */ -+ public CompactionJobTemplate getTemplate() -+ { -+ if (spec instanceof CascadingCompactionTemplate) { -+ return (CascadingCompactionTemplate) spec; -+ } else { -+ return new CompactionConfigBasedJobTemplate(spec); -+ } -+ } -+ - @Override - public List getDataSources() - { -diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/DruidInputSourceFactory.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/DruidInputSourceFactory.java -new file mode 100644 -index 0000000000..4089146404 ---- /dev/null -+++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/DruidInputSourceFactory.java -@@ -0,0 +1,74 @@ -+/* -+ * 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.compact; -+ -+import com.google.inject.Inject; -+import org.apache.druid.client.coordinator.CoordinatorClient; -+import org.apache.druid.indexing.common.SegmentCacheManagerFactory; -+import org.apache.druid.indexing.common.config.TaskConfig; -+import org.apache.druid.indexing.input.DruidInputSource; -+import org.apache.druid.segment.IndexIO; -+import org.joda.time.Interval; -+ -+/** -+ * Factory for creating {@link DruidInputSource} for a given datasource and -+ * interval used by {@link CompactionJobTemplate}. -+ */ -+public class DruidInputSourceFactory -+{ -+ private final IndexIO indexIO; -+ private final TaskConfig taskConfig; -+ private final CoordinatorClient coordinatorClient; -+ private final SegmentCacheManagerFactory segmentCacheManagerFactory; -+ -+ @Inject -+ public DruidInputSourceFactory( -+ IndexIO indexIO, -+ TaskConfig taskConfig, -+ CoordinatorClient coordinatorClient, -+ SegmentCacheManagerFactory segmentCacheManagerFactory -+ ) -+ { -+ this.indexIO = indexIO; -+ this.coordinatorClient = coordinatorClient; -+ this.segmentCacheManagerFactory = segmentCacheManagerFactory; -+ this.taskConfig = taskConfig; -+ } -+ -+ /** -+ * Creates a new {@link DruidInputSource} for the given {@code dataSource} and -+ * {@code interval}. -+ */ -+ public DruidInputSource create(String dataSource, Interval interval) -+ { -+ return new DruidInputSource( -+ dataSource, -+ interval, -+ null, -+ null, -+ null, -+ null, -+ indexIO, -+ coordinatorClient, -+ segmentCacheManagerFactory, -+ taskConfig -+ ); -+ } -+} -diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/InlineCompactionJobTemplate.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/InlineCompactionJobTemplate.java -new file mode 100644 -index 0000000000..43e4c6f8c2 ---- /dev/null -+++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/InlineCompactionJobTemplate.java -@@ -0,0 +1,98 @@ -+/* -+ * 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.compact; -+ -+import com.fasterxml.jackson.annotation.JsonCreator; -+import com.fasterxml.jackson.annotation.JsonProperty; -+import org.apache.druid.indexing.input.DruidInputSource; -+import org.apache.druid.java.util.common.granularity.Granularity; -+ -+import javax.annotation.Nullable; -+import java.util.List; -+import java.util.Objects; -+ -+/** -+ * Template to create compaction jobs using inline specifications. This template -+ * does not fetch any information from the Druid catalog while creating jobs. -+ */ -+public class InlineCompactionJobTemplate implements CompactionJobTemplate -+{ -+ public static final String TYPE = "compactInline"; -+ -+ private final CompactionStateMatcher targetState; -+ -+ @JsonCreator -+ public InlineCompactionJobTemplate( -+ @JsonProperty("targetState") CompactionStateMatcher targetState -+ ) -+ { -+ this.targetState = targetState; -+ } -+ -+ @JsonProperty -+ public CompactionStateMatcher getTargetState() -+ { -+ return targetState; -+ } -+ -+ @Nullable -+ @Override -+ public Granularity getSegmentGranularity() -+ { -+ return targetState.getSegmentGranularity(); -+ } -+ -+ @Override -+ public List createCompactionJobs( -+ DruidInputSource source, -+ CompactionJobParams jobParams -+ ) -+ { -+ final String dataSource = source.getDataSource(); -+ return CompactionConfigBasedJobTemplate -+ .create(dataSource, targetState) -+ .createCompactionJobs(source, jobParams); -+ } -+ -+ @Override -+ public boolean equals(Object object) -+ { -+ if (this == object) { -+ return true; -+ } -+ if (object == null || getClass() != object.getClass()) { -+ return false; -+ } -+ InlineCompactionJobTemplate that = (InlineCompactionJobTemplate) object; -+ return Objects.equals(this.targetState, that.targetState); -+ } -+ -+ @Override -+ public int hashCode() -+ { -+ return Objects.hash(targetState); -+ } -+ -+ @Override -+ public String getType() -+ { -+ return TYPE; -+ } -+} -diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/LocalOverlordClient.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/LocalOverlordClient.java -index 3f1427c7c3..eb200577bb 100644 ---- a/indexing-service/src/main/java/org/apache/druid/indexing/compact/LocalOverlordClient.java -+++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/LocalOverlordClient.java -@@ -72,10 +72,12 @@ class LocalOverlordClient extends NoopOverlordClient - @Override - public ListenableFuture runTask(String taskId, Object clientTaskQuery) - { -+ final CompactionTask task = -+ clientTaskQuery instanceof CompactionTask -+ ? (CompactionTask) clientTaskQuery -+ : convertTask(clientTaskQuery, ClientCompactionTaskQuery.class, CompactionTask.class); - return futureOf(() -> { -- getValidTaskQueue().add( -- convertTask(clientTaskQuery, ClientCompactionTaskQuery.class, CompactionTask.class) -- ); -+ getValidTaskQueue().add(task); - return null; - }); - } -diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/MSQCompactionJobTemplate.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/MSQCompactionJobTemplate.java -new file mode 100644 -index 0000000000..ea7193e443 ---- /dev/null -+++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/MSQCompactionJobTemplate.java -@@ -0,0 +1,191 @@ -+/* -+ * 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.compact; -+ -+import com.fasterxml.jackson.annotation.JsonCreator; -+import com.fasterxml.jackson.annotation.JsonProperty; -+import org.apache.druid.indexing.input.DruidInputSource; -+import org.apache.druid.java.util.common.StringUtils; -+import org.apache.druid.java.util.common.granularity.Granularity; -+import org.apache.druid.query.http.ClientSqlQuery; -+import org.apache.druid.server.compaction.CompactionCandidate; -+import org.apache.druid.server.compaction.CompactionSlotManager; -+import org.apache.druid.server.compaction.DataSourceCompactibleSegmentIterator; -+import org.apache.druid.server.coordinator.duty.CompactSegments; -+import org.joda.time.Interval; -+import org.joda.time.format.DateTimeFormat; -+import org.joda.time.format.DateTimeFormatter; -+ -+import javax.annotation.Nullable; -+import java.util.ArrayList; -+import java.util.HashMap; -+import java.util.List; -+import java.util.Map; -+import java.util.Objects; -+ -+/** -+ * Compaction template that creates MSQ SQL jobs using a templatized SQL with -+ * variables of the format {@code ${variableName}} for fields such as datasource -+ * name and start timestamp. -+ *

-+ * Compaction is triggered for an interval only if the current compaction state -+ * of the underlying segments DOES NOT match with the {@link #targetState}. -+ */ -+public class MSQCompactionJobTemplate implements CompactionJobTemplate -+{ -+ public static final String TYPE = "compactMsq"; -+ -+ public static final String VAR_DATASOURCE = "${dataSource}"; -+ public static final String VAR_START_TIMESTAMP = "${startTimestamp}"; -+ public static final String VAR_END_TIMESTAMP = "${endTimestamp}"; -+ -+ private static final DateTimeFormatter TIMESTAMP_FORMATTER = -+ DateTimeFormat.forPattern("yyyy-MM-dd HH:mm:ss.SSS"); -+ -+ private final ClientSqlQuery sqlTemplate; -+ private final CompactionStateMatcher targetState; -+ -+ @JsonCreator -+ public MSQCompactionJobTemplate( -+ @JsonProperty("sqlTemplate") ClientSqlQuery sqlTemplate, -+ @JsonProperty("targetState") CompactionStateMatcher targetState -+ ) -+ { -+ this.sqlTemplate = sqlTemplate; -+ this.targetState = targetState; -+ } -+ -+ @JsonProperty -+ public ClientSqlQuery getSqlTemplate() -+ { -+ return sqlTemplate; -+ } -+ -+ @JsonProperty -+ public CompactionStateMatcher getTargetState() -+ { -+ return targetState; -+ } -+ -+ @Nullable -+ @Override -+ public Granularity getSegmentGranularity() -+ { -+ return targetState.getSegmentGranularity(); -+ } -+ -+ @Override -+ public List createCompactionJobs( -+ DruidInputSource source, -+ CompactionJobParams jobParams -+ ) -+ { -+ final String dataSource = source.getDataSource(); -+ -+ // Identify the compactible candidate segments -+ final CompactionConfigBasedJobTemplate delegate = -+ CompactionConfigBasedJobTemplate.create(dataSource, targetState); -+ final DataSourceCompactibleSegmentIterator candidateIterator = -+ delegate.getCompactibleCandidates(source, jobParams); -+ -+ // Create MSQ jobs for each candidate by interpolating the template variables -+ final List jobs = new ArrayList<>(); -+ while (candidateIterator.hasNext()) { -+ final CompactionCandidate candidate = candidateIterator.next(); -+ jobs.add( -+ new CompactionJob( -+ createQueryForJob(dataSource, candidate.getCompactionInterval()), -+ candidate, -+ CompactionSlotManager.getMaxTaskSlotsForMSQCompactionTask(sqlTemplate.getContext()) -+ ) -+ ); -+ } -+ -+ return jobs; -+ } -+ -+ private ClientSqlQuery createQueryForJob(String dataSource, Interval compactionInterval) -+ { -+ final String formattedSql = formatSql( -+ sqlTemplate.getQuery(), -+ Map.of( -+ VAR_DATASOURCE, dataSource, -+ VAR_START_TIMESTAMP, compactionInterval.getStart().toString(TIMESTAMP_FORMATTER), -+ VAR_END_TIMESTAMP, compactionInterval.getEnd().toString(TIMESTAMP_FORMATTER) -+ ) -+ ); -+ -+ final Map context = new HashMap<>(); -+ if (sqlTemplate.getContext() != null) { -+ context.putAll(sqlTemplate.getContext()); -+ } -+ context.put(CompactSegments.STORE_COMPACTION_STATE_KEY, true); -+ context.put(CompactSegments.COMPACTION_INTERVAL_KEY, compactionInterval); -+ -+ return new ClientSqlQuery( -+ formattedSql, -+ sqlTemplate.getResultFormat(), -+ sqlTemplate.isHeader(), -+ sqlTemplate.isTypesHeader(), -+ sqlTemplate.isSqlTypesHeader(), -+ context, -+ sqlTemplate.getParameters() -+ ); -+ } -+ -+ /** -+ * Formats the given SQL by replacing the template variables. -+ */ -+ public static String formatSql(String sqlTemplate, Map templateVariables) -+ { -+ String sql = sqlTemplate; -+ for (Map.Entry variable : templateVariables.entrySet()) { -+ sql = StringUtils.replace(sql, variable.getKey(), variable.getValue()); -+ } -+ -+ return sql; -+ } -+ -+ @Override -+ public boolean equals(Object object) -+ { -+ if (this == object) { -+ return true; -+ } -+ if (object == null || getClass() != object.getClass()) { -+ return false; -+ } -+ MSQCompactionJobTemplate that = (MSQCompactionJobTemplate) object; -+ return Objects.equals(sqlTemplate, that.sqlTemplate) -+ && Objects.equals(targetState, that.targetState); -+ } -+ -+ @Override -+ public int hashCode() -+ { -+ return Objects.hash(sqlTemplate, targetState); -+ } -+ -+ @Override -+ public String getType() -+ { -+ return TYPE; -+ } -+} -diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/OverlordCompactionScheduler.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/OverlordCompactionScheduler.java -index 95ea3d1169..c3bce6a09d 100644 ---- a/indexing-service/src/main/java/org/apache/druid/indexing/compact/OverlordCompactionScheduler.java -+++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/OverlordCompactionScheduler.java -@@ -24,15 +24,18 @@ import com.google.common.base.Optional; - import com.google.common.base.Supplier; - import com.google.inject.Inject; - import org.apache.druid.client.DataSourcesSnapshot; -+import org.apache.druid.client.broker.BrokerClient; - import org.apache.druid.client.indexing.ClientCompactionRunnerInfo; - import org.apache.druid.indexer.TaskLocation; - import org.apache.druid.indexer.TaskStatus; -+import org.apache.druid.indexing.common.actions.TaskActionClientFactory; -+import org.apache.druid.indexing.overlord.GlobalTaskLockbox; - import org.apache.druid.indexing.overlord.TaskMaster; - import org.apache.druid.indexing.overlord.TaskQueryTool; - import org.apache.druid.indexing.overlord.TaskRunner; - import org.apache.druid.indexing.overlord.TaskRunnerListener; -+import org.apache.druid.java.util.common.Intervals; - import org.apache.druid.java.util.common.Stopwatch; --import org.apache.druid.java.util.common.concurrent.Execs; - import org.apache.druid.java.util.common.concurrent.ScheduledExecutorFactory; - import org.apache.druid.java.util.common.lifecycle.LifecycleStart; - import org.apache.druid.java.util.common.lifecycle.LifecycleStop; -@@ -40,6 +43,7 @@ import org.apache.druid.java.util.common.logger.Logger; - import org.apache.druid.java.util.emitter.service.ServiceEmitter; - import org.apache.druid.java.util.emitter.service.ServiceMetricEvent; - import org.apache.druid.metadata.SegmentsMetadataManager; -+import org.apache.druid.metadata.SegmentsMetadataManagerConfig; - import org.apache.druid.server.compaction.CompactionRunSimulator; - import org.apache.druid.server.compaction.CompactionSimulateResult; - import org.apache.druid.server.compaction.CompactionStatusTracker; -@@ -50,19 +54,20 @@ import org.apache.druid.server.coordinator.CoordinatorOverlordServiceConfig; - import org.apache.druid.server.coordinator.DataSourceCompactionConfig; - import org.apache.druid.server.coordinator.DruidCompactionConfig; - import org.apache.druid.server.coordinator.duty.CompactSegments; --import org.apache.druid.server.coordinator.stats.CoordinatorRunStats; - import org.apache.druid.server.coordinator.stats.CoordinatorStat; --import org.apache.druid.server.coordinator.stats.Dimension; -+import org.apache.druid.server.coordinator.stats.RowKey; - import org.apache.druid.server.coordinator.stats.Stats; --import org.joda.time.Duration; - --import java.util.ArrayList; - import java.util.Collections; -+import java.util.List; - import java.util.Map; - import java.util.concurrent.ConcurrentHashMap; - import java.util.concurrent.ScheduledExecutorService; - import java.util.concurrent.TimeUnit; - import java.util.concurrent.atomic.AtomicBoolean; -+import java.util.concurrent.atomic.AtomicReference; -+import java.util.function.Consumer; -+import java.util.stream.Collectors; - - /** - * Implementation of {@link CompactionScheduler}. -@@ -81,16 +86,31 @@ public class OverlordCompactionScheduler implements CompactionScheduler - { - private static final Logger log = new Logger(OverlordCompactionScheduler.class); - -- private static final long SCHEDULE_PERIOD_SECONDS = 5; -- private static final Duration METRIC_EMISSION_PERIOD = Duration.standardMinutes(5); -+ /** -+ * Scheduler run period is 15 minutes. It has been kept high to avoid eager -+ * recomputation of the queue as it may be a very compute-intensive operation -+ * taking upto several minutes on clusters with a large number of used segments. -+ * Jobs for a single supervisor may still be recomputed when the supervisor is updated. -+ */ -+ private static final long DEFAULT_SCHEDULE_PERIOD_MILLIS = 15 * 60_000; - - private final SegmentsMetadataManager segmentManager; - private final LocalOverlordClient overlordClient; -+ private final BrokerClient brokerClient; - private final ServiceEmitter emitter; -+ private final ObjectMapper objectMapper; - private final TaskMaster taskMaster; - - private final Supplier compactionConfigSupplier; -- private final ConcurrentHashMap activeDatasourceConfigs; -+ private final ConcurrentHashMap activeSupervisors; -+ -+ private final AtomicReference> datasourceToCompactionSnapshot; -+ private final AtomicBoolean shouldRecomputeJobsForAnyDatasource = new AtomicBoolean(false); -+ -+ /** -+ * Compaction job queue built in the last invocation of {@link #resetCompactionJobQueue()}. -+ */ -+ private final AtomicReference latestJobQueue; - - /** - * Single-threaded executor to process the compaction queue. -@@ -98,6 +118,9 @@ public class OverlordCompactionScheduler implements CompactionScheduler - private final ScheduledExecutorService executor; - - private final CompactionStatusTracker statusTracker; -+ private final TaskActionClientFactory taskActionClientFactory; -+ private final DruidInputSourceFactory druidInputSourceFactory; -+ private final GlobalTaskLockbox taskLockbox; - - /** - * Listener to watch task completion events and update CompactionStatusTracker. -@@ -107,7 +130,6 @@ public class OverlordCompactionScheduler implements CompactionScheduler - - private final AtomicBoolean isLeader = new AtomicBoolean(false); - private final AtomicBoolean started = new AtomicBoolean(false); -- private final CompactSegments duty; - - /** - * The scheduler should enable/disable polling of segments only if the Overlord -@@ -115,25 +137,35 @@ public class OverlordCompactionScheduler implements CompactionScheduler - * class itself. - */ - private final boolean shouldPollSegments; -- -- private final Stopwatch sinceStatsEmitted = Stopwatch.createUnstarted(); -+ private final long schedulePeriodMillis; - - @Inject - public OverlordCompactionScheduler( - TaskMaster taskMaster, -+ GlobalTaskLockbox taskLockbox, - TaskQueryTool taskQueryTool, - SegmentsMetadataManager segmentManager, -+ SegmentsMetadataManagerConfig segmentManagerConfig, - Supplier compactionConfigSupplier, - CompactionStatusTracker statusTracker, - CoordinatorOverlordServiceConfig coordinatorOverlordServiceConfig, -+ TaskActionClientFactory taskActionClientFactory, -+ DruidInputSourceFactory druidInputSourceFactory, - ScheduledExecutorFactory executorFactory, -+ BrokerClient brokerClient, - ServiceEmitter emitter, - ObjectMapper objectMapper - ) - { -+ final long segmentPollPeriodMillis = -+ segmentManagerConfig.getPollDuration().toStandardDuration().getMillis(); -+ this.schedulePeriodMillis = Math.min(DEFAULT_SCHEDULE_PERIOD_MILLIS, segmentPollPeriodMillis); -+ - this.segmentManager = segmentManager; - this.emitter = emitter; -+ this.objectMapper = objectMapper; - this.taskMaster = taskMaster; -+ this.taskLockbox = taskLockbox; - this.compactionConfigSupplier = compactionConfigSupplier; - - this.executor = executorFactory.create(1, "CompactionScheduler-%s"); -@@ -141,9 +173,13 @@ public class OverlordCompactionScheduler implements CompactionScheduler - this.shouldPollSegments = segmentManager != null - && !coordinatorOverlordServiceConfig.isEnabled(); - this.overlordClient = new LocalOverlordClient(taskMaster, taskQueryTool, objectMapper); -- this.duty = new CompactSegments(this.statusTracker, overlordClient); -- this.activeDatasourceConfigs = new ConcurrentHashMap<>(); -+ this.brokerClient = brokerClient; -+ this.activeSupervisors = new ConcurrentHashMap<>(); -+ this.datasourceToCompactionSnapshot = new AtomicReference<>(); -+ this.latestJobQueue = new AtomicReference<>(); - -+ this.taskActionClientFactory = taskActionClientFactory; -+ this.druidInputSourceFactory = druidInputSourceFactory; - this.taskRunnerListener = new TaskRunnerListener() - { - @Override -@@ -162,7 +198,8 @@ public class OverlordCompactionScheduler implements CompactionScheduler - public void statusChanged(String taskId, TaskStatus status) - { - if (status.isComplete()) { -- statusTracker.onTaskFinished(taskId, status); -+ onTaskFinished(taskId, status); -+ launchPendingJobs(); - } - } - }; -@@ -184,7 +221,8 @@ public class OverlordCompactionScheduler implements CompactionScheduler - public void becomeLeader() - { - if (isLeader.compareAndSet(false, true)) { -- scheduleOnExecutor(this::scheduledRun, SCHEDULE_PERIOD_SECONDS); -+ // Schedule first run after a small delay -+ scheduleOnExecutor(this::scheduledRun, 1_000L); - } - } - -@@ -208,25 +246,31 @@ public class OverlordCompactionScheduler implements CompactionScheduler - } else { - return ClientCompactionRunnerInfo.validateCompactionConfig( - compactionConfig, -- getLatestConfig().getEngine() -+ getLatestClusterConfig().getEngine() - ); - } - } - - @Override -- public void startCompaction(String dataSourceName, DataSourceCompactionConfig config) -+ public void startCompaction(String dataSourceName, CompactionSupervisor supervisor) - { -- // Track active datasources even if scheduler has not started yet because -+ // Track active supervisors even if scheduler has not started yet because - // SupervisorManager is started before the scheduler - if (isEnabled()) { -- activeDatasourceConfigs.put(dataSourceName, config); -+ activeSupervisors.put(dataSourceName, supervisor); -+ -+ if (started.get()) { -+ shouldRecomputeJobsForAnyDatasource.set(true); -+ scheduleOnExecutor(() -> recreateJobs(dataSourceName, supervisor), 0L); -+ } - } - } - - @Override - public void stopCompaction(String dataSourceName) - { -- activeDatasourceConfigs.remove(dataSourceName); -+ activeSupervisors.remove(dataSourceName); -+ updateQueueIfComputed(queue -> queue.removeJobs(dataSourceName)); - statusTracker.removeDatasource(dataSourceName); - } - -@@ -239,10 +283,10 @@ public class OverlordCompactionScheduler implements CompactionScheduler - return; - } - -- log.info("Starting compaction scheduler."); -+ log.info("Starting compaction scheduler with period [%d] millis.", schedulePeriodMillis); - final Optional taskRunnerOptional = taskMaster.getTaskRunner(); - if (taskRunnerOptional.isPresent()) { -- taskRunnerOptional.get().registerListener(taskRunnerListener, Execs.directExecutor()); -+ taskRunnerOptional.get().registerListener(taskRunnerListener, executor); - } - if (shouldPollSegments) { - segmentManager.startPollingDatabasePeriodically(); -@@ -264,7 +308,8 @@ public class OverlordCompactionScheduler implements CompactionScheduler - taskRunnerOptional.get().unregisterListener(taskRunnerListener.getListenerId()); - } - statusTracker.stop(); -- activeDatasourceConfigs.clear(); -+ activeSupervisors.clear(); -+ latestJobQueue.set(null); - - if (shouldPollSegments) { - segmentManager.stopPollingDatabasePeriodically(); -@@ -291,54 +336,119 @@ public class OverlordCompactionScheduler implements CompactionScheduler - if (isEnabled()) { - initState(); - try { -- runCompactionDuty(); -+ resetCompactionJobQueue(); - } - catch (Exception e) { - log.error(e, "Error processing compaction queue. Continuing schedule."); - } -- scheduleOnExecutor(this::scheduledRun, SCHEDULE_PERIOD_SECONDS); -+ scheduleOnExecutor(this::scheduledRun, schedulePeriodMillis); - } else { - cleanupState(); -- scheduleOnExecutor(this::scheduledRun, SCHEDULE_PERIOD_SECONDS * 4); -+ scheduleOnExecutor(this::scheduledRun, schedulePeriodMillis); - } - } - - /** -- * Runs the compaction duty and emits stats if {@link #METRIC_EMISSION_PERIOD} -- * has elapsed. -+ * Creates and launches eligible compaction jobs. - */ -- private synchronized void runCompactionDuty() -+ private synchronized void resetCompactionJobQueue() - { -- final CoordinatorRunStats stats = new CoordinatorRunStats(); -- duty.run(getLatestConfig(), getDatasourceSnapshot(), getLatestConfig().getEngine(), stats); -- -- // Emit stats only if emission period has elapsed -- if (!sinceStatsEmitted.isRunning() || sinceStatsEmitted.hasElapsed(METRIC_EMISSION_PERIOD)) { -- stats.forEachStat( -- (stat, dimensions, value) -> { -- if (stat.shouldEmit()) { -- emitStat(stat, dimensions.getValues(), value); -- } -- } -- ); -- sinceStatsEmitted.restart(); -- } else { -- // Always emit number of submitted tasks -- long numSubmittedTasks = stats.get(Stats.Compaction.SUBMITTED_TASKS); -- emitStat(Stats.Compaction.SUBMITTED_TASKS, Collections.emptyMap(), numSubmittedTasks); -+ // Remove the old queue so that no more jobs are added to it -+ latestJobQueue.set(null); -+ -+ final Stopwatch runDuration = Stopwatch.createStarted(); -+ final DataSourcesSnapshot dataSourcesSnapshot = getDatasourceSnapshot(); -+ final CompactionJobQueue queue = new CompactionJobQueue( -+ dataSourcesSnapshot, -+ getLatestClusterConfig(), -+ statusTracker, -+ taskActionClientFactory, -+ taskLockbox, -+ overlordClient, -+ brokerClient, -+ objectMapper -+ ); -+ latestJobQueue.set(queue); -+ -+ statusTracker.resetActiveDatasources(activeSupervisors.keySet()); -+ statusTracker.onSegmentTimelineUpdated(dataSourcesSnapshot.getSnapshotTime()); -+ -+ // Jobs for all active supervisors are being freshly created -+ // recomputation will not be needed -+ shouldRecomputeJobsForAnyDatasource.set(false); -+ activeSupervisors.forEach(this::createAndEnqueueJobs); -+ -+ launchPendingJobs(); -+ queue.getRunStats().forEachStat(this::emitStat); -+ emitStat(Stats.Compaction.SCHEDULER_RUN_TIME, RowKey.empty(), runDuration.millisElapsed()); -+ } -+ -+ /** -+ * Launches pending compaction jobs if compaction task slots become available. -+ * This method uses the jobs created by the last invocation of {@link #resetCompactionJobQueue()}. -+ */ -+ private synchronized void launchPendingJobs() -+ { -+ updateQueueIfComputed(queue -> { -+ queue.runReadyJobs(); -+ updateCompactionSnapshots(queue); -+ }); -+ } -+ -+ private synchronized void recreateJobs(String dataSource, CompactionSupervisor supervisor) -+ { -+ if (shouldRecomputeJobsForAnyDatasource.get()) { -+ createAndEnqueueJobs(dataSource, supervisor); -+ } -+ } -+ -+ private synchronized void createAndEnqueueJobs(String dataSource, CompactionSupervisor supervisor) -+ { -+ updateQueueIfComputed( -+ queue -> queue.createAndEnqueueJobs( -+ supervisor, -+ druidInputSourceFactory.create(dataSource, Intervals.ETERNITY) -+ ) -+ ); -+ } -+ -+ /** -+ * Performs an operation on the {@link #latestJobQueue} if it has been already -+ * computed. -+ */ -+ private void updateQueueIfComputed(Consumer operation) -+ { -+ final CompactionJobQueue queue = latestJobQueue.get(); -+ if (queue != null) { -+ operation.accept(queue); - } - } - -+ private void onTaskFinished(String taskId, TaskStatus taskStatus) -+ { -+ statusTracker.onTaskFinished(taskId, taskStatus); -+ -+ updateQueueIfComputed(queue -> { -+ queue.onTaskFinished(taskId, taskStatus); -+ updateCompactionSnapshots(queue); -+ }); -+ } -+ -+ private void updateCompactionSnapshots(CompactionJobQueue queue) -+ { -+ datasourceToCompactionSnapshot.set(queue.getSnapshots()); -+ } -+ - @Override - public AutoCompactionSnapshot getCompactionSnapshot(String dataSource) - { -- if (!activeDatasourceConfigs.containsKey(dataSource)) { -+ if (!activeSupervisors.containsKey(dataSource)) { - return AutoCompactionSnapshot.builder(dataSource) - .withStatus(AutoCompactionSnapshot.ScheduleStatus.NOT_ENABLED) - .build(); - } - -- final AutoCompactionSnapshot snapshot = duty.getAutoCompactionSnapshot(dataSource); -+ final AutoCompactionSnapshot snapshot = datasourceToCompactionSnapshot.get().get(dataSource); - if (snapshot == null) { - final AutoCompactionSnapshot.ScheduleStatus status = - isEnabled() -@@ -353,7 +463,7 @@ public class OverlordCompactionScheduler implements CompactionScheduler - @Override - public Map getAllCompactionSnapshots() - { -- return duty.getAutoCompactionSnapshot(); -+ return Map.copyOf(datasourceToCompactionSnapshot.get()); - } - - @Override -@@ -363,17 +473,21 @@ public class OverlordCompactionScheduler implements CompactionScheduler - return new CompactionRunSimulator(statusTracker, overlordClient).simulateRunWithConfig( - getLatestConfig().withClusterConfig(updateRequest), - getDatasourceSnapshot(), -- getLatestConfig().getEngine() -+ updateRequest.getEngine() - ); - } else { - return new CompactionSimulateResult(Collections.emptyMap()); - } - } - -- private void emitStat(CoordinatorStat stat, Map dimensionValues, long value) -+ private void emitStat(CoordinatorStat stat, RowKey rowKey, long value) - { -+ if (!stat.shouldEmit()) { -+ return; -+ } -+ - ServiceMetricEvent.Builder eventBuilder = new ServiceMetricEvent.Builder(); -- dimensionValues.forEach( -+ rowKey.getValues().forEach( - (dim, dimValue) -> eventBuilder.setDimension(dim.reportedName(), dimValue) - ); - emitter.emit(eventBuilder.setMetric(stat.getMetricName(), value)); -@@ -381,10 +495,20 @@ public class OverlordCompactionScheduler implements CompactionScheduler - - private DruidCompactionConfig getLatestConfig() - { -+ final List configs = activeSupervisors -+ .values() -+ .stream() -+ .map(s -> s.getSpec().getSpec()) -+ .collect(Collectors.toList()); - return DruidCompactionConfig - .empty() -- .withClusterConfig(compactionConfigSupplier.get().clusterConfig()) -- .withDatasourceConfigs(new ArrayList<>(activeDatasourceConfigs.values())); -+ .withClusterConfig(getLatestClusterConfig()) -+ .withDatasourceConfigs(configs); -+ } -+ -+ private ClusterCompactionConfig getLatestClusterConfig() -+ { -+ return compactionConfigSupplier.get().clusterConfig(); - } - - private DataSourcesSnapshot getDatasourceSnapshot() -@@ -392,7 +516,7 @@ public class OverlordCompactionScheduler implements CompactionScheduler - return segmentManager.getRecentDataSourcesSnapshot(); - } - -- private void scheduleOnExecutor(Runnable runnable, long delaySeconds) -+ private void scheduleOnExecutor(Runnable runnable, long delayMillis) - { - executor.schedule( - () -> { -@@ -403,8 +527,8 @@ public class OverlordCompactionScheduler implements CompactionScheduler - log.error(t, "Error while executing runnable"); - } - }, -- delaySeconds, -- TimeUnit.SECONDS -+ delayMillis, -+ TimeUnit.MILLISECONDS - ); - } - } -diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/input/DruidInputSource.java b/indexing-service/src/main/java/org/apache/druid/indexing/input/DruidInputSource.java -index 84c17f1a8f..fea6f30ef4 100644 ---- a/indexing-service/src/main/java/org/apache/druid/indexing/input/DruidInputSource.java -+++ b/indexing-service/src/main/java/org/apache/druid/indexing/input/DruidInputSource.java -@@ -291,6 +291,25 @@ public class DruidInputSource extends AbstractInputSource implements SplittableI - ); - } - -+ /** -+ * Creates a new {@link DruidInputSource} with the given interval. -+ */ -+ public DruidInputSource withInterval(Interval interval) -+ { -+ return new DruidInputSource( -+ this.dataSource, -+ interval, -+ null, -+ this.dimFilter, -+ this.dimensions, -+ this.metrics, -+ this.indexIO, -+ this.coordinatorClient, -+ this.segmentCacheManagerFactory, -+ this.taskConfig -+ ); -+ } -+ - @Override - protected InputSourceReader fixedFormatReader(InputRowSchema inputRowSchema, @Nullable File temporaryDirectory) - { -diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskQueue.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskQueue.java -index 0cfe481a8f..c923e70fca 100644 ---- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskQueue.java -+++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskQueue.java -@@ -1023,6 +1023,19 @@ public class TaskQueue - } - - /** -+ * List of all active tasks currently being managed by this TaskQueue. -+ */ -+ public List getActiveTasks() -+ { -+ return activeTasks.values() -+ .stream() -+ .filter(entry -> !entry.isComplete) -+ .map(TaskEntry::getTask) -+ .collect(Collectors.toList()); -+ } -+ -+ /** -+ * Returns the list of currently active tasks for the given datasource. - * List of all active and completed task infos currently being managed by this TaskQueue. - */ - public List getTaskInfos() -diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/compact/CompactionSupervisorSpecTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/compact/CompactionSupervisorSpecTest.java -index b1732acabb..b008d74829 100644 ---- a/indexing-service/src/test/java/org/apache/druid/indexing/compact/CompactionSupervisorSpecTest.java -+++ b/indexing-service/src/test/java/org/apache/druid/indexing/compact/CompactionSupervisorSpecTest.java -@@ -137,7 +137,7 @@ public class CompactionSupervisorSpecTest - supervisor.start(); - supervisor.stop(false); - -- Mockito.verify(scheduler, Mockito.times(1)).startCompaction(TestDataSource.WIKI, spec); -+ Mockito.verify(scheduler, Mockito.times(1)).startCompaction(TestDataSource.WIKI, supervisor); - Mockito.verify(scheduler, Mockito.times(1)).stopCompaction(TestDataSource.WIKI); - } - -@@ -157,7 +157,7 @@ public class CompactionSupervisorSpecTest - supervisor.start(); - supervisor.stop(false); - -- Mockito.verify(scheduler, Mockito.times(1)).startCompaction(TestDataSource.WIKI, spec); -+ Mockito.verify(scheduler, Mockito.times(1)).startCompaction(TestDataSource.WIKI, supervisor); - Mockito.verify(scheduler, Mockito.times(1)).stopCompaction(TestDataSource.WIKI); - } - -diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/compact/OverlordCompactionSchedulerTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/compact/OverlordCompactionSchedulerTest.java -index 9353953fc2..1484a719db 100644 ---- a/indexing-service/src/test/java/org/apache/druid/indexing/compact/OverlordCompactionSchedulerTest.java -+++ b/indexing-service/src/test/java/org/apache/druid/indexing/compact/OverlordCompactionSchedulerTest.java -@@ -21,15 +21,31 @@ package org.apache.druid.indexing.compact; - - import com.fasterxml.jackson.databind.InjectableValues; - import com.fasterxml.jackson.databind.ObjectMapper; -+import com.google.common.base.Optional; -+import com.google.common.util.concurrent.Futures; -+import org.apache.druid.client.broker.BrokerClient; -+import org.apache.druid.client.coordinator.NoopCoordinatorClient; - import org.apache.druid.client.indexing.ClientMSQContext; -+import org.apache.druid.common.utils.IdUtils; - import org.apache.druid.guice.IndexingServiceTuningConfigModule; -+import org.apache.druid.guice.SupervisorModule; - import org.apache.druid.indexer.CompactionEngine; -+import org.apache.druid.indexer.TaskState; -+import org.apache.druid.indexer.TaskStatus; - import org.apache.druid.indexing.common.SegmentCacheManagerFactory; -+import org.apache.druid.indexing.common.TimeChunkLock; -+import org.apache.druid.indexing.common.actions.RetrieveUsedSegmentsAction; -+import org.apache.druid.indexing.common.actions.TaskAction; -+import org.apache.druid.indexing.common.actions.TaskActionClient; -+import org.apache.druid.indexing.common.actions.TaskActionClientFactory; -+import org.apache.druid.indexing.common.actions.TimeChunkLockTryAcquireAction; -+import org.apache.druid.indexing.common.config.TaskConfig; - import org.apache.druid.indexing.common.config.TaskStorageConfig; - import org.apache.druid.indexing.common.task.CompactionTask; - import org.apache.druid.indexing.common.task.Task; - import org.apache.druid.indexing.overlord.GlobalTaskLockbox; - import org.apache.druid.indexing.overlord.HeapMemoryTaskStorage; -+import org.apache.druid.indexing.overlord.Segments; - import org.apache.druid.indexing.overlord.TaskMaster; - import org.apache.druid.indexing.overlord.TaskQueryTool; - import org.apache.druid.indexing.overlord.TaskQueue; -@@ -38,9 +54,16 @@ import org.apache.druid.indexing.overlord.setup.DefaultWorkerBehaviorConfig; - import org.apache.druid.indexing.overlord.setup.WorkerBehaviorConfig; - import org.apache.druid.indexing.test.TestIndexerMetadataStorageCoordinator; - import org.apache.druid.jackson.DefaultObjectMapper; -+import org.apache.druid.java.util.common.DateTimes; - import org.apache.druid.java.util.common.Intervals; -+import org.apache.druid.java.util.common.StringUtils; - import org.apache.druid.java.util.common.granularity.Granularities; -+import org.apache.druid.java.util.common.granularity.Granularity; - import org.apache.druid.java.util.metrics.StubServiceEmitter; -+import org.apache.druid.metadata.SegmentsMetadataManager; -+import org.apache.druid.metadata.SegmentsMetadataManagerConfig; -+import org.apache.druid.query.http.ClientSqlQuery; -+import org.apache.druid.query.http.SqlTaskStatus; - import org.apache.druid.segment.TestDataSource; - import org.apache.druid.segment.TestIndex; - import org.apache.druid.server.compaction.CompactionSimulateResult; -@@ -56,11 +79,15 @@ import org.apache.druid.server.coordinator.CreateDataSegments; - import org.apache.druid.server.coordinator.DataSourceCompactionConfig; - import org.apache.druid.server.coordinator.DruidCompactionConfig; - import org.apache.druid.server.coordinator.InlineSchemaDataSourceCompactionConfig; -+import org.apache.druid.server.coordinator.UserCompactionTaskGranularityConfig; -+import org.apache.druid.server.coordinator.duty.CompactSegments; - import org.apache.druid.server.coordinator.simulate.BlockingExecutorService; --import org.apache.druid.server.coordinator.simulate.TestSegmentsMetadataManager; - import org.apache.druid.server.coordinator.simulate.WrappingScheduledExecutorService; - import org.apache.druid.server.coordinator.stats.Stats; - import org.apache.druid.timeline.DataSegment; -+import org.joda.time.DateTime; -+import org.joda.time.Duration; -+import org.joda.time.Interval; - import org.joda.time.Period; - import org.junit.Assert; - import org.junit.Before; -@@ -71,7 +98,10 @@ import org.mockito.Mockito; - - import java.util.Arrays; - import java.util.Collections; -+import java.util.HashMap; - import java.util.List; -+import java.util.Map; -+import java.util.Set; - import java.util.concurrent.atomic.AtomicReference; - - public class OverlordCompactionSchedulerTest -@@ -81,6 +111,7 @@ public class OverlordCompactionSchedulerTest - static { - OBJECT_MAPPER = new DefaultObjectMapper(); - OBJECT_MAPPER.registerModules(new IndexingServiceTuningConfigModule().getJacksonModules()); -+ OBJECT_MAPPER.registerModules(new SupervisorModule().getJacksonModules()); - OBJECT_MAPPER.setInjectableValues( - new InjectableValues - .Std() -@@ -91,25 +122,53 @@ public class OverlordCompactionSchedulerTest - ); - } - -+ private static final DateTime JAN_20 = DateTimes.of("2025-01-20"); -+ private static final DateTime MAR_11 = DateTimes.of("2025-03-11"); -+ - private AtomicReference compactionConfig; - private CoordinatorOverlordServiceConfig coordinatorOverlordServiceConfig; - - private TaskMaster taskMaster; - private TaskQueue taskQueue; -+ private BrokerClient brokerClient; -+ private TaskActionClientFactory taskActionClientFactory; - private BlockingExecutorService executor; - - private HeapMemoryTaskStorage taskStorage; -- private TestSegmentsMetadataManager segmentsMetadataManager; -+ private TestIndexerMetadataStorageCoordinator segmentStorage; -+ private SegmentsMetadataManager segmentsMetadataManager; - private StubServiceEmitter serviceEmitter; - -+ private String dataSource; - private OverlordCompactionScheduler scheduler; - -+ private Map submittedMsqTaskIds; -+ - @Before - public void setUp() - { -+ dataSource = "wiki_" + IdUtils.getRandomId(); -+ - final TaskRunner taskRunner = Mockito.mock(TaskRunner.class); -+ Mockito.when(taskRunner.getTotalCapacity()).thenReturn(100); -+ Mockito.when(taskRunner.getMaximumCapacityWithAutoscale()).thenReturn(100); -+ - taskQueue = Mockito.mock(TaskQueue.class); - -+ submittedMsqTaskIds = new HashMap<>(); -+ brokerClient = Mockito.mock(BrokerClient.class); -+ Mockito.when(brokerClient.submitSqlTask(ArgumentMatchers.any(ClientSqlQuery.class))).thenAnswer( -+ arg -> { -+ final ClientSqlQuery query = arg.getArgument(0); -+ final Interval compactionInterval = -+ (Interval) query.getContext().get(CompactSegments.COMPACTION_INTERVAL_KEY); -+ -+ final String taskId = IdUtils.getRandomId(); -+ submittedMsqTaskIds.put(compactionInterval, taskId); -+ return Futures.immediateFuture(new SqlTaskStatus(taskId, TaskState.RUNNING, null)); -+ } -+ ); -+ - taskMaster = new TaskMaster(null, null); - Assert.assertFalse(taskMaster.isHalfOrFullLeader()); - Assert.assertFalse(taskMaster.isFullLeader()); -@@ -126,11 +185,39 @@ public class OverlordCompactionSchedulerTest - - executor = new BlockingExecutorService("test"); - serviceEmitter = new StubServiceEmitter(); -- segmentsMetadataManager = new TestSegmentsMetadataManager(); -+ segmentStorage = new TestIndexerMetadataStorageCoordinator(); -+ segmentsMetadataManager = segmentStorage.getManager(); - -- compactionConfig = new AtomicReference<>(new ClusterCompactionConfig(null, null, null, true, null)); -+ compactionConfig = new AtomicReference<>(new ClusterCompactionConfig(1.0, 100, null, true, null)); - coordinatorOverlordServiceConfig = new CoordinatorOverlordServiceConfig(false, null); - -+ taskActionClientFactory = task -> new TaskActionClient() -+ { -+ @Override -+ @SuppressWarnings("unchecked") -+ public RetType submit(TaskAction taskAction) -+ { -+ if (taskAction instanceof RetrieveUsedSegmentsAction) { -+ return (RetType) segmentStorage.retrieveAllUsedSegments( -+ ((RetrieveUsedSegmentsAction) taskAction).getDataSource(), -+ Segments.ONLY_VISIBLE -+ ); -+ } else if (taskAction instanceof TimeChunkLockTryAcquireAction) { -+ final TimeChunkLockTryAcquireAction lockAcquireAction = (TimeChunkLockTryAcquireAction) taskAction; -+ return (RetType) new TimeChunkLock( -+ null, -+ task.getGroupId(), -+ task.getDataSource(), -+ lockAcquireAction.getInterval(), -+ DateTimes.nowUtc().toString(), -+ 1 -+ ); -+ } else { -+ return null; -+ } -+ } -+ }; -+ - initScheduler(); - } - -@@ -142,19 +229,29 @@ public class OverlordCompactionSchedulerTest - = new DefaultWorkerBehaviorConfig(WorkerBehaviorConfig.DEFAULT_STRATEGY, null); - scheduler = new OverlordCompactionScheduler( - taskMaster, -+ taskLockbox, - new TaskQueryTool(taskStorage, taskLockbox, taskMaster, null, () -> defaultWorkerConfig), - segmentsMetadataManager, -+ new SegmentsMetadataManagerConfig(null, null, null), - () -> DruidCompactionConfig.empty().withClusterConfig(compactionConfig.get()), -- new CompactionStatusTracker(OBJECT_MAPPER), -+ new CompactionStatusTracker(), - coordinatorOverlordServiceConfig, -+ taskActionClientFactory, -+ new DruidInputSourceFactory( -+ TestIndex.INDEX_IO, -+ Mockito.mock(TaskConfig.class), -+ new NoopCoordinatorClient(), -+ new SegmentCacheManagerFactory(TestIndex.INDEX_IO, OBJECT_MAPPER) -+ ), - (nameFormat, numThreads) -> new WrappingScheduledExecutorService("test", executor, false), -+ brokerClient, - serviceEmitter, - OBJECT_MAPPER - ); - } - - @Test -- public void testBecomeLeader_triggersStart_ifEnabled() -+ public void test_becomeLeader_triggersStart_ifEnabled() - { - Assert.assertTrue(scheduler.isEnabled()); - -@@ -168,7 +265,7 @@ public class OverlordCompactionSchedulerTest - } - - @Test -- public void testBecomeLeader_doesNotTriggerStart_ifDisabled() -+ public void test_becomeLeader_doesNotTriggerStart_ifDisabled() - { - disableScheduler(); - Assert.assertFalse(scheduler.isEnabled()); -@@ -182,7 +279,7 @@ public class OverlordCompactionSchedulerTest - } - - @Test -- public void testStopBeingLeader_triggersStop() -+ public void test_stopBeingLeader_triggersStop() - { - Assert.assertFalse(scheduler.isRunning()); - -@@ -198,7 +295,7 @@ public class OverlordCompactionSchedulerTest - } - - @Test -- public void testDisablingScheduler_triggersStop() -+ public void test_disableSupervisors_triggersStop() - { - // Start scheduler - scheduler.becomeLeader(); -@@ -216,7 +313,7 @@ public class OverlordCompactionSchedulerTest - } - - @Test -- public void testEnablingScheduler_triggersStart() -+ public void test_enableSupervisors_triggersStart() - { - disableScheduler(); - -@@ -235,7 +332,7 @@ public class OverlordCompactionSchedulerTest - } - - @Test -- public void testSegmentsAreNotPolled_ifSupervisorsAreDisabled() -+ public void test_disableSupervisors_disablesSegmentPolling() - { - disableScheduler(); - -@@ -243,7 +340,7 @@ public class OverlordCompactionSchedulerTest - } - - @Test -- public void testSegmentsArePolled_whenRunningInStandaloneMode() -+ public void test_enableSupervisors_inStandaloneMode_enablesSegmentPolling() - { - coordinatorOverlordServiceConfig = new CoordinatorOverlordServiceConfig(false, null); - initScheduler(); -@@ -252,7 +349,7 @@ public class OverlordCompactionSchedulerTest - } - - @Test -- public void testSegmentsAreNotPolled_whenRunningInCoordinatorMode() -+ public void test_enableSupervisors_inCoordinatorMode_disablesSegmentPolling() - { - coordinatorOverlordServiceConfig = new CoordinatorOverlordServiceConfig(true, "overlord"); - initScheduler(); -@@ -272,7 +369,7 @@ public class OverlordCompactionSchedulerTest - } - - @Test -- public void testNullCompactionConfigIsInvalid() -+ public void test_validateCompactionConfig_returnsInvalid_forNullConfig() - { - final CompactionConfigValidationResult result = scheduler.validateCompactionConfig(null); - Assert.assertFalse(result.isValid()); -@@ -280,11 +377,11 @@ public class OverlordCompactionSchedulerTest - } - - @Test -- public void testMsqCompactionConfigWithOneMaxTasksIsInvalid() -+ public void test_validateCompactionConfig_returnsInvalid_forMSQConfigWithOneMaxTasks() - { - final DataSourceCompactionConfig datasourceConfig = InlineSchemaDataSourceCompactionConfig - .builder() -- .forDataSource(TestDataSource.WIKI) -+ .forDataSource(dataSource) - .withEngine(CompactionEngine.MSQ) - .withTaskContext(Collections.singletonMap(ClientMSQContext.CTX_MAX_NUM_TASKS, 1)) - .build(); -@@ -298,75 +395,50 @@ public class OverlordCompactionSchedulerTest - } - - @Test -- public void testStartCompaction() -+ public void test_startCompaction_enablesTaskSubmission_forDatasource() - { -- final List wikiSegments = CreateDataSegments.ofDatasource(TestDataSource.WIKI).eachOfSizeInMb(100); -- wikiSegments.forEach(segmentsMetadataManager::addSegment); -+ createSegments(1, Granularities.DAY, JAN_20); - - scheduler.becomeLeader(); -- scheduler.startCompaction( -- TestDataSource.WIKI, -- InlineSchemaDataSourceCompactionConfig.builder() -- .forDataSource(TestDataSource.WIKI) -- .withSkipOffsetFromLatest(Period.seconds(0)) -- .build() -- ); -- -- executor.finishNextPendingTask(); -+ scheduler.startCompaction(dataSource, createSupervisorWithInlineSpec()); - -- ArgumentCaptor taskArgumentCaptor = ArgumentCaptor.forClass(Task.class); -- Mockito.verify(taskQueue, Mockito.times(1)).add(taskArgumentCaptor.capture()); -- -- Task submittedTask = taskArgumentCaptor.getValue(); -- Assert.assertNotNull(submittedTask); -- Assert.assertTrue(submittedTask instanceof CompactionTask); -- -- final CompactionTask compactionTask = (CompactionTask) submittedTask; -- Assert.assertEquals(TestDataSource.WIKI, compactionTask.getDataSource()); -+ runCompactionTasks(1); - -- final AutoCompactionSnapshot.Builder expectedSnapshot = AutoCompactionSnapshot.builder(TestDataSource.WIKI); -- expectedSnapshot.incrementCompactedStats(CompactionStatistics.create(100_000_000, 1, 1)); -+ final AutoCompactionSnapshot.Builder expectedSnapshot = AutoCompactionSnapshot.builder(dataSource); -+ expectedSnapshot.incrementWaitingStats(CompactionStatistics.create(100_000_000, 1, 1)); - - Assert.assertEquals( - expectedSnapshot.build(), -- scheduler.getCompactionSnapshot(TestDataSource.WIKI) -+ scheduler.getCompactionSnapshot(dataSource) - ); - Assert.assertEquals( -- Collections.singletonMap(TestDataSource.WIKI, expectedSnapshot.build()), -+ Map.of(dataSource, expectedSnapshot.build()), - scheduler.getAllCompactionSnapshots() - ); - - serviceEmitter.verifyValue(Stats.Compaction.SUBMITTED_TASKS.getMetricName(), 1L); -- serviceEmitter.verifyValue(Stats.Compaction.COMPACTED_BYTES.getMetricName(), 100_000_000L); -+ serviceEmitter.verifyValue(Stats.Compaction.PENDING_BYTES.getMetricName(), 100_000_000L); - - scheduler.stopBeingLeader(); - } - - @Test -- public void testStopCompaction() -+ public void test_stopCompaction_disablesTaskSubmission_forDatasource() - { -- final List wikiSegments = CreateDataSegments.ofDatasource(TestDataSource.WIKI).eachOfSizeInMb(100); -- wikiSegments.forEach(segmentsMetadataManager::addSegment); -+ createSegments(1, Granularities.DAY, JAN_20); - - scheduler.becomeLeader(); -- scheduler.startCompaction( -- TestDataSource.WIKI, -- InlineSchemaDataSourceCompactionConfig.builder() -- .forDataSource(TestDataSource.WIKI) -- .withSkipOffsetFromLatest(Period.seconds(0)) -- .build() -- ); -- scheduler.stopCompaction(TestDataSource.WIKI); -- -- executor.finishNextPendingTask(); -+ scheduler.startCompaction(dataSource, createSupervisorWithInlineSpec()); -+ scheduler.stopCompaction(dataSource); - -+ runScheduledJob(); - Mockito.verify(taskQueue, Mockito.never()).add(ArgumentMatchers.any()); - - Assert.assertEquals( -- AutoCompactionSnapshot.builder(TestDataSource.WIKI) -+ AutoCompactionSnapshot.builder(dataSource) - .withStatus(AutoCompactionSnapshot.ScheduleStatus.NOT_ENABLED) - .build(), -- scheduler.getCompactionSnapshot(TestDataSource.WIKI) -+ scheduler.getCompactionSnapshot(dataSource) - ); - Assert.assertTrue(scheduler.getAllCompactionSnapshots().isEmpty()); - -@@ -377,26 +449,14 @@ public class OverlordCompactionSchedulerTest - } - - @Test -- public void testSimulateRun() -+ public void test_simulateRunWithConfigUpdate() - { -- final List wikiSegments = CreateDataSegments -- .ofDatasource(TestDataSource.WIKI) -- .forIntervals(1, Granularities.DAY) -- .startingAt("2013-01-01") -- .withNumPartitions(10) -- .eachOfSizeInMb(100); -- wikiSegments.forEach(segmentsMetadataManager::addSegment); -+ createSegments(1, Granularities.DAY, DateTimes.of("2013-01-01")); - - scheduler.becomeLeader(); - runScheduledJob(); - -- scheduler.startCompaction( -- TestDataSource.WIKI, -- InlineSchemaDataSourceCompactionConfig.builder() -- .forDataSource(TestDataSource.WIKI) -- .withSkipOffsetFromLatest(Period.seconds(0)) -- .build() -- ); -+ scheduler.startCompaction(dataSource, createSupervisorWithInlineSpec()); - - final CompactionSimulateResult simulateResult = scheduler.simulateRunWithConfigUpdate( - new ClusterCompactionConfig(null, null, null, null, null) -@@ -410,10 +470,10 @@ public class OverlordCompactionSchedulerTest - Assert.assertEquals( - Collections.singletonList( - Arrays.asList( -- TestDataSource.WIKI, -+ dataSource, - Intervals.of("2013-01-01/P1D"), -- 10, -- 1_000_000_000L, -+ 1, -+ 100_000_000L, - 1, - "not compacted yet" - ) -@@ -421,7 +481,7 @@ public class OverlordCompactionSchedulerTest - pendingCompactionTable.getRows() - ); - -- scheduler.stopCompaction(TestDataSource.WIKI); -+ scheduler.stopCompaction(dataSource); - - final CompactionSimulateResult simulateResultWhenDisabled = scheduler.simulateRunWithConfigUpdate( - new ClusterCompactionConfig(null, null, null, null, null) -@@ -431,6 +491,255 @@ public class OverlordCompactionSchedulerTest - scheduler.stopBeingLeader(); - } - -+ @Test -+ public void test_ingestHourGranularity_andCompactToDayAndMonth_withInlineTemplates() -+ { -+ final int numDays = (int) new Duration(MAR_11.getMillis() - JAN_20.getMillis()).getStandardDays(); -+ createSegments(24 * numDays, Granularities.HOUR, JAN_20); -+ verifyNumSegmentsWith(Granularities.HOUR, 24 * numDays); -+ -+ // Compact everything going back to Mar 10 to DAY granularity, rest to MONTH -+ final DateTime now = DateTimes.nowUtc(); -+ final Period dayRulePeriod = new Period(now.getMillis() - MAR_11.minusDays(1).minusMinutes(1).getMillis()); -+ CascadingCompactionTemplate cascadingTemplate = new CascadingCompactionTemplate( -+ dataSource, -+ List.of( -+ new CompactionRule(dayRulePeriod, new InlineCompactionJobTemplate(createMatcher(Granularities.DAY))), -+ new CompactionRule(Period.ZERO, new InlineCompactionJobTemplate(createMatcher(Granularities.MONTH))) -+ ) -+ ); -+ -+ startCompactionWithSpec(cascadingTemplate); -+ runCompactionTasks(12); -+ -+ verifyFullyCompacted(); -+ verifyNumSegmentsWith(Granularities.HOUR, 0); -+ verifyNumSegmentsWith(Granularities.DAY, 10); -+ verifyNumSegmentsWith(Granularities.MONTH, 2); -+ } -+ -+ @Test -+ public void test_ingestHourGranularity_andCompactToDayAndMonth_withCatalogTemplates() -+ { -+ final int numDays = (int) new Duration(MAR_11.getMillis() - JAN_20.getMillis()).getStandardDays(); -+ createSegments(24 * numDays, Granularities.HOUR, JAN_20); -+ verifyNumSegmentsWith(Granularities.HOUR, 24 * numDays); -+ -+ // Add compaction templates to catalog -+ final CompactionJobTemplate dayGranularityTemplate = -+ new InlineCompactionJobTemplate(createMatcher(Granularities.DAY)); -+ final CompactionJobTemplate monthGranularityTemplate = -+ new InlineCompactionJobTemplate(createMatcher(Granularities.MONTH)); -+ -+ // Compact everything going back to Mar 10 to DAY granularity, rest to MONTH -+ final DateTime now = DateTimes.nowUtc(); -+ final Period dayRulePeriod = new Period(now.getMillis() - MAR_11.minusDays(1).minusMinutes(1).getMillis()); -+ CascadingCompactionTemplate cascadingTemplate = new CascadingCompactionTemplate( -+ dataSource, -+ List.of( -+ new CompactionRule(dayRulePeriod, dayGranularityTemplate), -+ new CompactionRule(Period.ZERO, monthGranularityTemplate) -+ ) -+ ); -+ -+ startCompactionWithSpec(cascadingTemplate); -+ runCompactionTasks(12); -+ -+ verifyFullyCompacted(); -+ verifyNumSegmentsWith(Granularities.HOUR, 0); -+ verifyNumSegmentsWith(Granularities.DAY, 10); -+ verifyNumSegmentsWith(Granularities.MONTH, 2); -+ } -+ -+ @Test -+ public void test_ingestHourGranularity_andCompactToDayAndMonth_withCatalogMSQTemplates() -+ { -+ dataSource = TestDataSource.WIKI; -+ -+ final int numDays = (int) new Duration(MAR_11.getMillis() - JAN_20.getMillis()).getStandardDays(); -+ createSegments(24 * numDays, Granularities.HOUR, JAN_20); -+ verifyNumSegmentsWith(Granularities.HOUR, 24 * numDays); -+ -+ // Add compaction templates to catalog -+ final String sqlDayGranularity = -+ "REPLACE INTO ${dataSource}" -+ + " OVERWRITE WHERE __time >= TIMESTAMP '${startTimestamp}' AND __time < TIMESTAMP '${endTimestamp}'" -+ + " SELECT * FROM ${dataSource}" -+ + " WHERE __time BETWEEN '${startTimestamp}' AND '${endTimestamp}'" -+ + " PARTITIONED BY DAY"; -+ final CompactionJobTemplate dayGranularityTemplate = new MSQCompactionJobTemplate( -+ new ClientSqlQuery(sqlDayGranularity, null, false, false, false, null, null), -+ createMatcher(Granularities.DAY) -+ ); -+ final String sqlMonthGranularity = -+ "REPLACE INTO ${dataSource}" -+ + " OVERWRITE WHERE __time >= TIMESTAMP '${startTimestamp}' AND __time < TIMESTAMP '${endTimestamp}'" -+ + " SELECT * FROM ${dataSource}" -+ + " WHERE __time BETWEEN '${startTimestamp}' AND '${endTimestamp}'" -+ + " PARTITIONED BY MONTH"; -+ final CompactionJobTemplate monthGranularityTemplate = new MSQCompactionJobTemplate( -+ new ClientSqlQuery(sqlMonthGranularity, null, false, false, false, null, null), -+ createMatcher(Granularities.MONTH) -+ ); -+ -+ // Compact everything going back to Mar 10 to DAY granularity, rest to MONTH -+ final DateTime now = DateTimes.nowUtc(); -+ final Period dayRulePeriod = new Period(now.getMillis() - MAR_11.minusDays(1).minusMinutes(1).getMillis()); -+ CascadingCompactionTemplate cascadingTemplate = new CascadingCompactionTemplate( -+ dataSource, -+ List.of( -+ new CompactionRule(dayRulePeriod, dayGranularityTemplate), -+ new CompactionRule(Period.ZERO, monthGranularityTemplate) -+ ) -+ ); -+ -+ startCompactionWithSpec(cascadingTemplate); -+ runMSQCompactionJobs(12); -+ -+ verifyFullyCompacted(); -+ verifyNumSegmentsWith(Granularities.HOUR, 0); -+ verifyNumSegmentsWith(Granularities.DAY, 10); -+ verifyNumSegmentsWith(Granularities.MONTH, 2); -+ } -+ -+ private void verifyNumSegmentsWith(Granularity granularity, int numExpectedSegments) -+ { -+ long numMatchingSegments = segmentStorage -+ .retrieveAllUsedSegments(dataSource, Segments.ONLY_VISIBLE) -+ .stream() -+ .filter(segment -> granularity.isAligned(segment.getInterval())) -+ .count(); -+ -+ Assert.assertEquals( -+ StringUtils.format("Segment with granularity[%s]", granularity), -+ numExpectedSegments, -+ (int) numMatchingSegments -+ ); -+ } -+ -+ private void verifyFullyCompacted() -+ { -+ runScheduledJob(); -+ int numSegments = segmentStorage.retrieveAllUsedSegments(dataSource, Segments.ONLY_VISIBLE).size(); -+ -+ final AutoCompactionSnapshot snapshot = scheduler.getCompactionSnapshot(dataSource); -+ Assert.assertEquals(0, snapshot.getSegmentCountAwaitingCompaction()); -+ Assert.assertEquals(0, snapshot.getSegmentCountSkipped()); -+ Assert.assertEquals(numSegments, snapshot.getSegmentCountCompacted()); -+ } -+ -+ private void createSegments(int numSegments, Granularity granularity, DateTime firstSegmentStart) -+ { -+ final List segments = CreateDataSegments -+ .ofDatasource(dataSource) -+ .forIntervals(numSegments, granularity) -+ .startingAt(firstSegmentStart) -+ .eachOfSizeInMb(100); -+ segmentStorage.commitSegments(Set.copyOf(segments), null); -+ } -+ -+ private void startCompactionWithSpec(DataSourceCompactionConfig config) -+ { -+ scheduler.becomeLeader(); -+ final CompactionSupervisorSpec compactionSupervisor -+ = new CompactionSupervisorSpec(config, false, scheduler); -+ scheduler.startCompaction(config.getDataSource(), compactionSupervisor.createSupervisor()); -+ } -+ -+ private void runCompactionTasks(int expectedCount) -+ { -+ runScheduledJob(); -+ serviceEmitter.verifySum("compact/task/count", expectedCount); -+ -+ ArgumentCaptor taskArgumentCaptor = ArgumentCaptor.forClass(Task.class); -+ Mockito.verify(taskQueue, Mockito.times(expectedCount)).add(taskArgumentCaptor.capture()); -+ -+ for (Task task : taskArgumentCaptor.getAllValues()) { -+ Assert.assertTrue(task instanceof CompactionTask); -+ Assert.assertEquals(dataSource, task.getDataSource()); -+ -+ final CompactionTask compactionTask = (CompactionTask) task; -+ runCompactionTask( -+ compactionTask.getId(), -+ compactionTask.getIoConfig().getInputSpec().findInterval(dataSource), -+ compactionTask.getSegmentGranularity() -+ ); -+ } -+ -+ segmentStorage.getManager().forceUpdateDataSourcesSnapshot(); -+ } -+ -+ private void runCompactionTask(String taskId, Interval compactionInterval, Granularity segmentGranularity) -+ { -+ // Update status of task in TaskQueue -+ Mockito.when(taskQueue.getTaskStatus(taskId)) -+ .thenReturn(Optional.of(TaskStatus.success(taskId))); -+ -+ // Determine interval and granularity and apply it to the timeline -+ if (segmentGranularity == null) { -+ // Nothing to do -+ return; -+ } -+ -+ for (Interval replaceInterval : segmentGranularity.getIterable(compactionInterval)) { -+ // Create a single segment in this interval -+ DataSegment replaceSegment = CreateDataSegments -+ .ofDatasource(dataSource) -+ .forIntervals(1, segmentGranularity) -+ .startingAt(replaceInterval.getStart()) -+ .withVersion("2") -+ .eachOfSizeInMb(100) -+ .get(0); -+ segmentStorage.commitSegments(Set.of(replaceSegment), null); -+ } -+ } -+ -+ private void runMSQCompactionJobs(int numExpectedJobs) -+ { -+ runScheduledJob(); -+ serviceEmitter.verifySum("compact/task/count", numExpectedJobs); -+ -+ ArgumentCaptor queryArgumentCaptor = ArgumentCaptor.forClass(ClientSqlQuery.class); -+ Mockito.verify(brokerClient, Mockito.times(numExpectedJobs)) -+ .submitSqlTask(queryArgumentCaptor.capture()); -+ -+ for (ClientSqlQuery job : queryArgumentCaptor.getAllValues()) { -+ final String query = job.getQuery(); -+ -+ final Granularity segmentGranularity; -+ if (query.contains("PARTITIONED BY DAY")) { -+ segmentGranularity = Granularities.DAY; -+ } else if (query.contains("PARTITIONED BY MONTH")) { -+ segmentGranularity = Granularities.MONTH; -+ } else { -+ segmentGranularity = Granularities.HOUR; -+ } -+ -+ final Interval compactionInterval = -+ (Interval) job.getContext().get(CompactSegments.COMPACTION_INTERVAL_KEY); -+ runCompactionTask( -+ submittedMsqTaskIds.get(compactionInterval), -+ compactionInterval, -+ segmentGranularity -+ ); -+ } -+ -+ segmentStorage.getManager().forceUpdateDataSourcesSnapshot(); -+ } -+ -+ private static CompactionStateMatcher createMatcher(Granularity segmentGranularity) -+ { -+ return new CompactionStateMatcher( -+ null, -+ null, -+ null, -+ null, -+ null, -+ new UserCompactionTaskGranularityConfig(segmentGranularity, null, null), -+ null -+ ); -+ } -+ - private void disableScheduler() - { - compactionConfig.set(new ClusterCompactionConfig(null, null, null, false, null)); -@@ -446,4 +755,16 @@ public class OverlordCompactionSchedulerTest - executor.finishNextPendingTask(); - } - -+ private CompactionSupervisor createSupervisorWithInlineSpec() -+ { -+ return new CompactionSupervisorSpec( -+ InlineSchemaDataSourceCompactionConfig -+ .builder() -+ .forDataSource(dataSource) -+ .withSkipOffsetFromLatest(Period.seconds(0)) -+ .build(), -+ false, -+ scheduler -+ ).createSupervisor(); -+ } - } -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 e7ecc49df8..ee9e85204e 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 -@@ -27,9 +27,11 @@ import org.apache.druid.indexing.overlord.SegmentCreateRequest; - import org.apache.druid.indexing.overlord.SegmentPublishResult; - import org.apache.druid.indexing.overlord.Segments; - import org.apache.druid.jackson.DefaultObjectMapper; -+import org.apache.druid.java.util.common.Intervals; - import org.apache.druid.java.util.common.Pair; - import org.apache.druid.metadata.PendingSegmentRecord; - import org.apache.druid.metadata.ReplaceTaskLock; -+import org.apache.druid.metadata.SegmentsMetadataManager; - import org.apache.druid.metadata.SortOrder; - import org.apache.druid.segment.SegmentSchemaMapping; - import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; -@@ -56,6 +58,11 @@ public class TestIndexerMetadataStorageCoordinator implements IndexerMetadataSto - - private int deleteSegmentsCount = 0; - -+ public SegmentsMetadataManager getManager() -+ { -+ return segmentsMetadataManager; -+ } -+ - @Override - public Set retrieveAllDatasourceNames() - { -@@ -106,9 +113,15 @@ public class TestIndexerMetadataStorageCoordinator implements IndexerMetadataSto - @Override - public Set retrieveAllUsedSegments(String dataSource, Segments visibility) - { -- return Set.copyOf( -- segmentsMetadataManager.getRecentDataSourcesSnapshot().getDataSource(dataSource).getSegments() -- ); -+ if (visibility == Segments.ONLY_VISIBLE) { -+ return segmentsMetadataManager -+ .getRecentDataSourcesSnapshot() -+ .getAllUsedNonOvershadowedSegments(dataSource, Intervals.ETERNITY); -+ } else { -+ return Set.copyOf( -+ segmentsMetadataManager.getRecentDataSourcesSnapshot().getDataSource(dataSource).getSegments() -+ ); -+ } - } - - @Override -diff --git a/processing/src/main/java/org/apache/druid/java/util/common/Intervals.java b/processing/src/main/java/org/apache/druid/java/util/common/Intervals.java -index 0960f63b49..ce34fc4798 100644 ---- a/processing/src/main/java/org/apache/druid/java/util/common/Intervals.java -+++ b/processing/src/main/java/org/apache/druid/java/util/common/Intervals.java -@@ -29,6 +29,7 @@ import org.joda.time.format.DateTimeFormatter; - import org.joda.time.format.ISODateTimeFormat; - - import javax.annotation.Nullable; -+import java.util.List; - - public final class Intervals - { -@@ -167,6 +168,26 @@ public final class Intervals - return null; - } - -+ /** -+ * @return List of intervals which when added to the given interval create -+ * {@link Intervals#ETERNITY}. -+ */ -+ public static List complementOf(Interval interval) -+ { -+ if (isEternity(interval)) { -+ return List.of(); -+ } else if (DateTimes.MIN.equals(interval.getStart())) { -+ return List.of(new Interval(interval.getEnd(), DateTimes.MAX)); -+ } else if (DateTimes.MAX.equals(interval.getEnd())) { -+ return List.of(new Interval(DateTimes.MIN, interval.getStart())); -+ } else { -+ return List.of( -+ new Interval(DateTimes.MIN, interval.getStart()), -+ new Interval(interval.getEnd(), DateTimes.MAX) -+ ); -+ } -+ } -+ - private Intervals() - { - } -diff --git a/processing/src/main/java/org/apache/druid/query/http/ClientSqlQuery.java b/processing/src/main/java/org/apache/druid/query/http/ClientSqlQuery.java -index 3b3a2a2398..663714b4d9 100644 ---- a/processing/src/main/java/org/apache/druid/query/http/ClientSqlQuery.java -+++ b/processing/src/main/java/org/apache/druid/query/http/ClientSqlQuery.java -@@ -109,7 +109,6 @@ public class ClientSqlQuery - return parameters; - } - -- - @Override - public boolean equals(final Object o) - { -diff --git a/processing/src/test/java/org/apache/druid/java/util/common/IntervalsTest.java b/processing/src/test/java/org/apache/druid/java/util/common/IntervalsTest.java -index 3d591a319f..de0a27bcb5 100644 ---- a/processing/src/test/java/org/apache/druid/java/util/common/IntervalsTest.java -+++ b/processing/src/test/java/org/apache/druid/java/util/common/IntervalsTest.java -@@ -26,6 +26,7 @@ import org.junit.Assert; - import org.junit.Test; - - import java.util.Arrays; -+import java.util.List; - - public class IntervalsTest - { -@@ -124,4 +125,36 @@ public class IntervalsTest - () -> Intervals.of("invalid string") - ); - } -+ -+ @Test -+ public void testComplementOf() -+ { -+ Assert.assertEquals( -+ List.of(), -+ Intervals.complementOf(Intervals.ETERNITY) -+ ); -+ -+ testComplementOf("2020/P1Y"); -+ testComplementOf("2001/2001-01"); -+ testComplementOf("2001-01-02/2001-02"); -+ } -+ -+ private void testComplementOf(String interval) -+ { -+ final Interval testInterval = Intervals.of(interval); -+ final List complement = List.of( -+ new Interval(DateTimes.MIN, testInterval.getStart()), -+ new Interval(testInterval.getEnd(), DateTimes.MAX) -+ ); -+ Assert.assertEquals( -+ complement, -+ Intervals.complementOf(testInterval) -+ ); -+ Assert.assertEquals( -+ Intervals.ONLY_ETERNITY, -+ JodaUtils.condenseIntervals( -+ List.of(complement.get(0), complement.get(1), testInterval) -+ ) -+ ); -+ } - } -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 7ff9ff424f..7a7f655723 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 -@@ -22,67 +22,24 @@ package org.apache.druid.client.indexing; - import com.fasterxml.jackson.annotation.JsonCreator; - import com.fasterxml.jackson.annotation.JsonProperty; - import org.apache.druid.java.util.common.IAE; --import org.apache.druid.java.util.common.JodaUtils; --import org.apache.druid.java.util.common.granularity.Granularity; --import org.apache.druid.java.util.common.logger.Logger; --import org.apache.druid.timeline.DataSegment; - 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. - */ - public class ClientCompactionIntervalSpec - { -- private static final Logger LOGGER = new Logger(ClientCompactionIntervalSpec.class); -- - private static final String TYPE = "interval"; - - private final Interval interval; - @Nullable - private final String sha256OfSortedSegmentIds; - -- public static ClientCompactionIntervalSpec fromSegments(List segments, @Nullable Granularity segmentGranularity) -- { -- Interval interval = JodaUtils.umbrellaInterval(segments.stream().map(DataSegment::getInterval).collect(Collectors.toList())); -- LOGGER.info("Original umbrella interval %s in compaction task for datasource %s", interval, segments.get(0).getDataSource()); -- if (segmentGranularity != null) { -- // If segmentGranularity is set, then the segmentGranularity of the segments may not align with the configured segmentGranularity -- // We must adjust the interval of the compaction task to fully cover and align with the segmentGranularity -- // For example, -- // - The umbrella interval of the segments is 2015-04-11/2015-04-12 but configured segmentGranularity is YEAR, -- // if the compaction task's interval is 2015-04-11/2015-04-12 then we can run into race condition where after submitting -- // the compaction task, a new segment outside of the interval (i.e. 2015-02-11/2015-02-12) got created will be lost as it is -- // overshadowed by the compacted segment (compacted segment has interval 2015-01-01/2016-01-01. -- // Hence, in this case, we must adjust the compaction task interval to 2015-01-01/2016-01-01. -- // - The segment to be compacted has MONTH segmentGranularity with the interval 2015-02-01/2015-03-01 but configured -- // segmentGranularity is WEEK. If the compaction task's interval is 2015-02-01/2015-03-01 then compacted segments created will be -- // 2015-01-26/2015-02-02, 2015-02-02/2015-02-09, 2015-02-09/2015-02-16, 2015-02-16/2015-02-23, 2015-02-23/2015-03-02. -- // This is because Druid's WEEK segments alway start and end on Monday. In the above example, 2015-01-26 and 2015-03-02 -- // are Mondays but 2015-02-01 and 2015-03-01 are not. Hence, the WEEK segments have to start and end on 2015-01-26 and 2015-03-02. -- // If the compaction task's interval is 2015-02-01/2015-03-01, then the compacted segment would cause existing data -- // from 2015-01-26 to 2015-02-01 and 2015-03-01 to 2015-03-02 to be lost. Hence, in this case, -- // we must adjust the compaction task interval to 2015-01-26/2015-03-02 -- interval = JodaUtils.umbrellaInterval(segmentGranularity.getIterable(interval)); -- LOGGER.info( -- "Interval adjusted to %s in compaction task for datasource %s with configured segmentGranularity %s", -- interval, -- segments.get(0).getDataSource(), -- segmentGranularity -- ); -- } -- return new ClientCompactionIntervalSpec( -- interval, -- null -- ); -- } -- - @JsonCreator - public ClientCompactionIntervalSpec( - @JsonProperty("interval") Interval interval, -diff --git a/server/src/main/java/org/apache/druid/indexing/template/BatchIndexingJob.java b/server/src/main/java/org/apache/druid/indexing/template/BatchIndexingJob.java -new file mode 100644 -index 0000000000..509226da10 ---- /dev/null -+++ b/server/src/main/java/org/apache/druid/indexing/template/BatchIndexingJob.java -@@ -0,0 +1,90 @@ -+/* -+ * Licensed to the Apache Software Foundation (ASF) under one -+ * or more contributor license agreements. See the NOTICE file -+ * distributed with this work for additional information -+ * regarding copyright ownership. The ASF licenses this file -+ * to you under the Apache License, Version 2.0 (the -+ * "License"); you may not use this file except in compliance -+ * with the License. You may obtain a copy of the License at -+ * -+ * http://www.apache.org/licenses/LICENSE-2.0 -+ * -+ * Unless required by applicable law or agreed to in writing, -+ * software distributed under the License is distributed on an -+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -+ * KIND, either express or implied. See the License for the -+ * specific language governing permissions and limitations -+ * under the License. -+ */ -+ -+package org.apache.druid.indexing.template; -+ -+import org.apache.druid.client.indexing.ClientTaskQuery; -+import org.apache.druid.error.InvalidInput; -+import org.apache.druid.query.http.ClientSqlQuery; -+ -+import javax.annotation.Nullable; -+import java.util.Objects; -+ -+/** -+ * A batch indexing job that can be launched by the Overlord as a task. -+ * A job may contain the {@link ClientTaskQuery} itself or an MSQ query that gets converted -+ * by the Broker to a {@code ControllerTask} and is then submitted to the Overlord. -+ */ -+public class BatchIndexingJob -+{ -+ private final boolean isMsq; -+ private final ClientSqlQuery msqQuery; -+ private final ClientTaskQuery task; -+ -+ protected BatchIndexingJob( -+ @Nullable ClientTaskQuery task, -+ @Nullable ClientSqlQuery msqQuery -+ ) -+ { -+ this.isMsq = task == null; -+ this.msqQuery = msqQuery; -+ this.task = task; -+ -+ InvalidInput.conditionalException( -+ (task == null || msqQuery == null) && (task != null || msqQuery != null), -+ "Exactly one of 'task' or 'msqQuery' must be non-null" -+ ); -+ } -+ -+ /** -+ * @return MSQ query to be run in this job, if any. -+ * @throws NullPointerException if this not an MSQ job. -+ */ -+ public ClientSqlQuery getNonNullMsqQuery() -+ { -+ return Objects.requireNonNull(msqQuery); -+ } -+ -+ /** -+ * @return Task to be run in this job, if any. -+ * @throws NullPointerException if this is an MSQ job. -+ */ -+ public ClientTaskQuery getNonNullTask() -+ { -+ return Objects.requireNonNull(task); -+ } -+ -+ /** -+ * @return true if this is an MSQ job. -+ */ -+ public boolean isMsq() -+ { -+ return isMsq; -+ } -+ -+ @Override -+ public String toString() -+ { -+ return "BatchIndexingJob{" + -+ "isMsq=" + isMsq + -+ ", msqQuery=" + msqQuery + -+ ", task=" + task + -+ '}'; -+ } -+} -diff --git a/server/src/main/java/org/apache/druid/indexing/template/BatchIndexingJobTemplate.java b/server/src/main/java/org/apache/druid/indexing/template/BatchIndexingJobTemplate.java -new file mode 100644 -index 0000000000..e8a09a6bb1 ---- /dev/null -+++ b/server/src/main/java/org/apache/druid/indexing/template/BatchIndexingJobTemplate.java -@@ -0,0 +1,41 @@ -+/* -+ * 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.template; -+ -+import com.fasterxml.jackson.annotation.JsonProperty; -+import com.fasterxml.jackson.annotation.JsonTypeInfo; -+import org.apache.druid.data.input.InputSource; -+ -+/** -+ * ETL template to create a {@link BatchIndexingJob} that indexes data from an -+ * {@link InputSource} into an output destination. -+ *

-+ * This interface currently defines no methods and is used only as the root type -+ * for serialization of compaction template objects. -+ */ -+@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type") -+public interface BatchIndexingJobTemplate -+{ -+ /** -+ * Unique type name of this template used for JSON serialization. -+ */ -+ @JsonProperty -+ String getType(); -+} -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 4cd9b22df8..f936f3d49a 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 -@@ -21,12 +21,14 @@ package org.apache.druid.server.compaction; - - import org.apache.druid.error.InvalidInput; - import org.apache.druid.java.util.common.JodaUtils; -+import org.apache.druid.java.util.common.granularity.Granularity; - import org.apache.druid.segment.SegmentUtils; - import org.apache.druid.timeline.DataSegment; - import org.joda.time.Interval; - - import javax.annotation.Nullable; - import java.util.List; -+import java.util.Set; - import java.util.stream.Collectors; - - /** -@@ -37,29 +39,54 @@ public class CompactionCandidate - { - private final List segments; - private final Interval umbrellaInterval; -+ private final Interval compactionInterval; - private final String dataSource; - private final long totalBytes; - private final int numIntervals; - - private final CompactionStatus currentStatus; - -- public static CompactionCandidate from(List segments) -+ public static CompactionCandidate from( -+ List segments, -+ @Nullable Granularity targetSegmentGranularity -+ ) - { - if (segments == null || segments.isEmpty()) { - throw InvalidInput.exception("Segments to compact must be non-empty"); -- } else { -- return new CompactionCandidate(segments, null); - } -+ -+ final Set segmentIntervals = -+ segments.stream().map(DataSegment::getInterval).collect(Collectors.toSet()); -+ final Interval umbrellaInterval = JodaUtils.umbrellaInterval(segmentIntervals); -+ final Interval compactionInterval = -+ targetSegmentGranularity == null -+ ? umbrellaInterval -+ : JodaUtils.umbrellaInterval(targetSegmentGranularity.getIterable(umbrellaInterval)); -+ -+ return new CompactionCandidate( -+ segments, -+ umbrellaInterval, -+ compactionInterval, -+ segmentIntervals.size(), -+ null -+ ); - } - -- private CompactionCandidate(List segments, @Nullable CompactionStatus currentStatus) -+ private CompactionCandidate( -+ List segments, -+ Interval umbrellaInterval, -+ Interval compactionInterval, -+ int numDistinctSegmentIntervals, -+ @Nullable CompactionStatus currentStatus -+ ) - { - this.segments = segments; - this.totalBytes = segments.stream().mapToLong(DataSegment::getSize).sum(); -- this.umbrellaInterval = JodaUtils.umbrellaInterval( -- segments.stream().map(DataSegment::getInterval).collect(Collectors.toList()) -- ); -- this.numIntervals = (int) segments.stream().map(DataSegment::getInterval).distinct().count(); -+ -+ this.umbrellaInterval = umbrellaInterval; -+ this.compactionInterval = compactionInterval; -+ -+ this.numIntervals = numDistinctSegmentIntervals; - this.dataSource = segments.get(0).getDataSource(); - this.currentStatus = currentStatus; - } -@@ -91,6 +118,15 @@ public class CompactionCandidate - return umbrellaInterval; - } - -+ /** -+ * Interval aligned to the target segment granularity used for the compaction -+ * task. This interval completely contains the {@link #umbrellaInterval}. -+ */ -+ public Interval getCompactionInterval() -+ { -+ return compactionInterval; -+ } -+ - public String getDataSource() - { - return dataSource; -@@ -115,7 +151,7 @@ public class CompactionCandidate - */ - public CompactionCandidate withCurrentStatus(CompactionStatus status) - { -- return new CompactionCandidate(this.segments, status); -+ return new CompactionCandidate(segments, umbrellaInterval, compactionInterval, numIntervals, status); - } - - @Override -diff --git a/server/src/main/java/org/apache/druid/server/compaction/CompactionRunSimulator.java b/server/src/main/java/org/apache/druid/server/compaction/CompactionRunSimulator.java -index 646319eec3..781019e12f 100644 ---- a/server/src/main/java/org/apache/druid/server/compaction/CompactionRunSimulator.java -+++ b/server/src/main/java/org/apache/druid/server/compaction/CompactionRunSimulator.java -@@ -22,7 +22,6 @@ package org.apache.druid.server.compaction; - import com.google.common.util.concurrent.Futures; - import com.google.common.util.concurrent.ListenableFuture; - import org.apache.druid.client.DataSourcesSnapshot; --import org.apache.druid.client.indexing.ClientCompactionTaskQuery; - import org.apache.druid.client.indexing.ClientCompactionTaskQueryTuningConfig; - import org.apache.druid.client.indexing.IndexingTotalWorkerCapacityInfo; - import org.apache.druid.client.indexing.TaskPayloadResponse; -@@ -86,16 +85,15 @@ public class CompactionRunSimulator - - // Add a read-only wrapper over the actual status tracker so that we can - // account for the active tasks -- final CompactionStatusTracker simulationStatusTracker = new CompactionStatusTracker(null) -+ final CompactionStatusTracker simulationStatusTracker = new CompactionStatusTracker() - { - @Override - public CompactionStatus computeCompactionStatus( - CompactionCandidate candidate, -- DataSourceCompactionConfig config, - CompactionCandidateSearchPolicy searchPolicy - ) - { -- return statusTracker.computeCompactionStatus(candidate, config, searchPolicy); -+ return statusTracker.computeCompactionStatus(candidate, searchPolicy); - } - - @Override -@@ -123,12 +121,12 @@ public class CompactionRunSimulator - } - - @Override -- public void onTaskSubmitted(ClientCompactionTaskQuery taskPayload, CompactionCandidate candidateSegments) -+ public void onTaskSubmitted(String taskId, CompactionCandidate candidateSegments) - { - // Add a row for each task in order of submission - final CompactionStatus status = candidateSegments.getCurrentStatus(); - queuedIntervals.addRow( -- createRow(candidateSegments, taskPayload.getTuningConfig(), status == null ? "" : status.getReason()) -+ createRow(candidateSegments, null, status == null ? "" : status.getReason()) - ); - } - }; -@@ -171,12 +169,10 @@ public class CompactionRunSimulator - { - final List row = new ArrayList<>(); - row.add(candidate.getDataSource()); -- row.add(candidate.getUmbrellaInterval()); -+ row.add(candidate.getCompactionInterval()); - row.add(candidate.numSegments()); - row.add(candidate.getTotalBytes()); -- if (tuningConfig != null) { -- row.add(CompactSegments.findMaxNumTaskSlotsUsedByOneNativeCompactionTask(tuningConfig)); -- } -+ row.add(CompactionSlotManager.getMaxTaskSlotsForNativeCompactionTask(tuningConfig)); - if (reason != null) { - row.add(reason); - } -diff --git a/server/src/main/java/org/apache/druid/server/compaction/CompactionSlotManager.java b/server/src/main/java/org/apache/druid/server/compaction/CompactionSlotManager.java -new file mode 100644 -index 0000000000..428367db8e ---- /dev/null -+++ b/server/src/main/java/org/apache/druid/server/compaction/CompactionSlotManager.java -@@ -0,0 +1,376 @@ -+/* -+ * Licensed to the Apache Software Foundation (ASF) under one -+ * or more contributor license agreements. See the NOTICE file -+ * distributed with this work for additional information -+ * regarding copyright ownership. The ASF licenses this file -+ * to you under the Apache License, Version 2.0 (the -+ * "License"); you may not use this file except in compliance -+ * with the License. You may obtain a copy of the License at -+ * -+ * http://www.apache.org/licenses/LICENSE-2.0 -+ * -+ * Unless required by applicable law or agreed to in writing, -+ * software distributed under the License is distributed on an -+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -+ * KIND, either express or implied. See the License for the -+ * specific language governing permissions and limitations -+ * under the License. -+ */ -+ -+package org.apache.druid.server.compaction; -+ -+import com.google.common.annotations.VisibleForTesting; -+import org.apache.druid.client.indexing.ClientCompactionTaskQuery; -+import org.apache.druid.client.indexing.ClientCompactionTaskQueryTuningConfig; -+import org.apache.druid.client.indexing.ClientMSQContext; -+import org.apache.druid.client.indexing.TaskPayloadResponse; -+import org.apache.druid.common.guava.FutureUtils; -+import org.apache.druid.indexer.CompactionEngine; -+import org.apache.druid.indexer.TaskStatus; -+import org.apache.druid.indexer.TaskStatusPlus; -+import org.apache.druid.indexer.partitions.DimensionRangePartitionsSpec; -+import org.apache.druid.java.util.common.ISE; -+import org.apache.druid.java.util.common.granularity.Granularity; -+import org.apache.druid.java.util.common.logger.Logger; -+import org.apache.druid.metadata.LockFilterPolicy; -+import org.apache.druid.rpc.indexing.OverlordClient; -+import org.apache.druid.server.coordinator.ClusterCompactionConfig; -+import org.apache.druid.server.coordinator.DataSourceCompactionConfig; -+import org.apache.druid.server.coordinator.duty.CoordinatorDutyUtils; -+import org.joda.time.Interval; -+ -+import javax.annotation.Nullable; -+import java.util.ArrayList; -+import java.util.HashMap; -+import java.util.HashSet; -+import java.util.List; -+import java.util.Map; -+import java.util.Set; -+import java.util.stream.Collectors; -+ -+/** -+ * Fetches running compaction tasks from the Overlord and tracks their compaction -+ * intervals and task slots. -+ */ -+public class CompactionSlotManager -+{ -+ /** -+ * Task type for native compaction tasks. -+ */ -+ public static final String COMPACTION_TASK_TYPE = "compact"; -+ -+ private static final Logger log = new Logger(CompactionSlotManager.class); -+ -+ private final OverlordClient overlordClient; -+ private final CompactionStatusTracker statusTracker; -+ -+ private final Map> intervalsToSkipCompaction; -+ -+ private int numAvailableTaskSlots; -+ -+ public CompactionSlotManager( -+ OverlordClient overlordClient, -+ CompactionStatusTracker statusTracker, -+ ClusterCompactionConfig clusterCompactionConfig -+ ) -+ { -+ this.overlordClient = overlordClient; -+ this.statusTracker = statusTracker; -+ this.numAvailableTaskSlots = getCompactionTaskCapacity(clusterCompactionConfig); -+ this.intervalsToSkipCompaction = new HashMap<>(); -+ } -+ -+ public int getNumAvailableTaskSlots() -+ { -+ return numAvailableTaskSlots; -+ } -+ -+ public Map> getDatasourceIntervalsToSkipCompaction() -+ { -+ return intervalsToSkipCompaction; -+ } -+ -+ public void reserveTaskSlots(int numSlotsToReserve) -+ { -+ numAvailableTaskSlots -= numSlotsToReserve; -+ } -+ -+ /** -+ * Reserves task slots for the given task from the overall compaction task capacity. -+ */ -+ public void reserveTaskSlots(ClientCompactionTaskQuery compactionTaskQuery) -+ { -+ // Note: The default compactionRunnerType used here should match the default runner used in CompactionTask when -+ // no runner is provided there. -+ CompactionEngine compactionRunnerType = compactionTaskQuery.getCompactionRunner() == null -+ ? CompactionEngine.NATIVE -+ : compactionTaskQuery.getCompactionRunner().getType(); -+ if (compactionRunnerType == CompactionEngine.NATIVE) { -+ numAvailableTaskSlots -= -+ getMaxTaskSlotsForNativeCompactionTask(compactionTaskQuery.getTuningConfig()); -+ } else { -+ numAvailableTaskSlots -= -+ getMaxTaskSlotsForMSQCompactionTask(compactionTaskQuery.getContext()); -+ } -+ } -+ -+ /** -+ * Reserves task slots for all running compaction tasks. -+ */ -+ public void reserveTaskSlotsForRunningCompactionTasks() -+ { -+ for (ClientCompactionTaskQuery task : fetchRunningCompactionTasks()) { -+ reserveTaskSlots(task); -+ } -+ } -+ -+ /** -+ * Retrieves currently running tasks of type {@link #COMPACTION_TASK_TYPE} from -+ * the Overlord. -+ *

-+ * Also queries the Overlord for the status of all tasks that were submitted -+ * recently but are not active anymore. The statuses are then updated in the -+ * {@link CompactionStatusTracker}. -+ */ -+ public List fetchRunningCompactionTasks() -+ { -+ // Fetch currently running compaction tasks -+ final List compactionTasks = CoordinatorDutyUtils.getStatusOfActiveTasks( -+ overlordClient, -+ status -> status != null && COMPACTION_TASK_TYPE.equals(status.getType()) -+ ); -+ -+ final Set activeTaskIds -+ = compactionTasks.stream().map(TaskStatusPlus::getId).collect(Collectors.toSet()); -+ trackStatusOfCompletedTasks(activeTaskIds); -+ -+ final List runningCompactTasks = new ArrayList<>(); -+ for (TaskStatusPlus status : compactionTasks) { -+ final TaskPayloadResponse response = -+ FutureUtils.getUnchecked(overlordClient.taskPayload(status.getId()), true); -+ if (response == null) { -+ throw new ISE("Could not find payload for active compaction task[%s]", status.getId()); -+ } else if (!COMPACTION_TASK_TYPE.equals(response.getPayload().getType())) { -+ throw new ISE( -+ "Payload of active compaction task[%s] is of invalid type[%s]", -+ status.getId(), response.getPayload().getType() -+ ); -+ } -+ -+ runningCompactTasks.add((ClientCompactionTaskQuery) response.getPayload()); -+ } -+ -+ return runningCompactTasks; -+ } -+ -+ /** -+ * Cancels a currently running compaction task only if the segment granularity -+ * has changed in the datasource compaction config. Otherwise, the task is -+ * retained and its intervals are skipped from the current round of compaction. -+ * -+ * @return true if the task was canceled, false otherwise. -+ */ -+ public boolean cancelTaskOnlyIfGranularityChanged( -+ ClientCompactionTaskQuery compactionTaskQuery, -+ DataSourceCompactionConfig dataSourceCompactionConfig -+ ) -+ { -+ if (dataSourceCompactionConfig == null -+ || dataSourceCompactionConfig.getGranularitySpec() == null -+ || compactionTaskQuery.getGranularitySpec() == null) { -+ skipTaskInterval(compactionTaskQuery); -+ reserveTaskSlots(compactionTaskQuery); -+ return false; -+ } -+ -+ Granularity configuredSegmentGranularity = dataSourceCompactionConfig.getGranularitySpec() -+ .getSegmentGranularity(); -+ Granularity taskSegmentGranularity = compactionTaskQuery.getGranularitySpec().getSegmentGranularity(); -+ if (configuredSegmentGranularity == null || configuredSegmentGranularity.equals(taskSegmentGranularity)) { -+ skipTaskInterval(compactionTaskQuery); -+ reserveTaskSlots(compactionTaskQuery); -+ return false; -+ } -+ -+ log.info( -+ "Cancelling task[%s] as task segmentGranularity[%s] differs from compaction config segmentGranularity[%s].", -+ compactionTaskQuery.getId(), taskSegmentGranularity, configuredSegmentGranularity -+ ); -+ overlordClient.cancelTask(compactionTaskQuery.getId()); -+ return true; -+ } -+ -+ /** -+ * Retrieves the list of intervals locked by higher priority tasks for each datasource. -+ * Since compaction tasks submitted for these Intervals would have to wait anyway, -+ * we skip these Intervals until the next compaction run by adding them to -+ * {@link #intervalsToSkipCompaction}. -+ *

-+ * This method must be called after invalid compaction tasks have already been -+ * cancelled using {@link #cancelTaskOnlyIfGranularityChanged} so that their -+ * intervals are not considered locked. -+ */ -+ public void skipLockedIntervals(List compactionConfigs) -+ { -+ final List lockFilterPolicies = compactionConfigs -+ .stream() -+ .map( -+ config -> new LockFilterPolicy( -+ config.getDataSource(), -+ config.getTaskPriority(), -+ null, -+ config.getTaskContext() -+ ) -+ ) -+ .collect(Collectors.toList()); -+ final Map> datasourceToLockedIntervals = new HashMap<>( -+ FutureUtils.getUnchecked(overlordClient.findLockedIntervals(lockFilterPolicies), true) -+ ); -+ log.debug( -+ "Skipping the following intervals for Compaction as they are currently locked: %s", -+ datasourceToLockedIntervals -+ ); -+ -+ // Skip all the intervals locked by higher priority tasks for each datasource -+ datasourceToLockedIntervals.forEach( -+ (dataSource, intervals) -> -+ intervalsToSkipCompaction -+ .computeIfAbsent(dataSource, ds -> new ArrayList<>()) -+ .addAll(intervals) -+ ); -+ } -+ -+ /** -+ * Adds the compaction interval of this task to {@link #intervalsToSkipCompaction} -+ */ -+ private void skipTaskInterval(ClientCompactionTaskQuery compactionTask) -+ { -+ final Interval interval = compactionTask.getIoConfig().getInputSpec().getInterval(); -+ intervalsToSkipCompaction.computeIfAbsent(compactionTask.getDataSource(), k -> new ArrayList<>()) -+ .add(interval); -+ } -+ -+ /** -+ * Computes overall compaction task capacity for the cluster. -+ * -+ * @return A value >= 1. -+ */ -+ private int getCompactionTaskCapacity(ClusterCompactionConfig clusterConfig) -+ { -+ int totalWorkerCapacity = CoordinatorDutyUtils.getTotalWorkerCapacity(overlordClient); -+ -+ int compactionTaskCapacity = Math.min( -+ (int) (totalWorkerCapacity * clusterConfig.getCompactionTaskSlotRatio()), -+ clusterConfig.getMaxCompactionTaskSlots() -+ ); -+ -+ // Always consider atleast one slot available for compaction -+ return Math.max(1, compactionTaskCapacity); -+ } -+ -+ /** -+ * Queries the Overlord for the status of all tasks that were submitted -+ * recently but are not active anymore. The statuses are then updated in the -+ * {@link #statusTracker}. -+ */ -+ private void trackStatusOfCompletedTasks(Set activeTaskIds) -+ { -+ final Set finishedTaskIds = new HashSet<>(statusTracker.getSubmittedTaskIds()); -+ finishedTaskIds.removeAll(activeTaskIds); -+ -+ if (finishedTaskIds.isEmpty()) { -+ return; -+ } -+ -+ final Map taskStatusMap -+ = FutureUtils.getUnchecked(overlordClient.taskStatuses(finishedTaskIds), true); -+ for (String taskId : finishedTaskIds) { -+ // Assume unknown task to have finished successfully -+ final TaskStatus taskStatus = taskStatusMap.getOrDefault(taskId, TaskStatus.success(taskId)); -+ if (taskStatus.isComplete()) { -+ statusTracker.onTaskFinished(taskId, taskStatus); -+ } -+ } -+ } -+ -+ /** -+ * @return Maximum number of task slots used by a native compaction task at -+ * any time when the task is run with the given tuning config. -+ */ -+ public static int getMaxTaskSlotsForNativeCompactionTask( -+ @Nullable ClientCompactionTaskQueryTuningConfig tuningConfig -+ ) -+ { -+ if (isParallelMode(tuningConfig)) { -+ Integer maxNumConcurrentSubTasks = tuningConfig.getMaxNumConcurrentSubTasks(); -+ // Max number of task slots used in parallel mode = maxNumConcurrentSubTasks + 1 (supervisor task) -+ return (maxNumConcurrentSubTasks == null ? 1 : maxNumConcurrentSubTasks) + 1; -+ } else { -+ return 1; -+ } -+ } -+ -+ /** -+ * @return Maximum number of task slots used by an MSQ compaction task at any -+ * time when the task is run with the given context. -+ */ -+ public static int getMaxTaskSlotsForMSQCompactionTask(@Nullable Map context) -+ { -+ return context == null -+ ? ClientMSQContext.DEFAULT_MAX_NUM_TASKS -+ : (int) context.getOrDefault(ClientMSQContext.CTX_MAX_NUM_TASKS, ClientMSQContext.DEFAULT_MAX_NUM_TASKS); -+ } -+ -+ -+ /** -+ * Returns true if the compaction task can run in the parallel mode with the given tuningConfig. -+ * This method should be synchronized with ParallelIndexSupervisorTask.isParallelMode(InputSource, ParallelIndexTuningConfig). -+ */ -+ @VisibleForTesting -+ public static boolean isParallelMode(@Nullable ClientCompactionTaskQueryTuningConfig tuningConfig) -+ { -+ if (null == tuningConfig) { -+ return false; -+ } -+ boolean useRangePartitions = useRangePartitions(tuningConfig); -+ int minRequiredNumConcurrentSubTasks = useRangePartitions ? 1 : 2; -+ return tuningConfig.getMaxNumConcurrentSubTasks() != null -+ && tuningConfig.getMaxNumConcurrentSubTasks() >= minRequiredNumConcurrentSubTasks; -+ } -+ -+ private static boolean useRangePartitions(ClientCompactionTaskQueryTuningConfig tuningConfig) -+ { -+ // dynamic partitionsSpec will be used if getPartitionsSpec() returns null -+ return tuningConfig.getPartitionsSpec() instanceof DimensionRangePartitionsSpec; -+ } -+ -+ public int computeSlotsRequiredForTask( -+ ClientCompactionTaskQuery task, -+ DataSourceCompactionConfig config -+ ) -+ { -+ if (task.getCompactionRunner().getType() == CompactionEngine.MSQ) { -+ final Map autoCompactionContext = task.getContext(); -+ if (autoCompactionContext.containsKey(ClientMSQContext.CTX_MAX_NUM_TASKS)) { -+ return (int) autoCompactionContext.get(ClientMSQContext.CTX_MAX_NUM_TASKS); -+ } else { -+ // Since MSQ needs all task slots for the calculated #tasks to be available upfront, allot all available -+ // compaction slots (upto a max of MAX_TASK_SLOTS_FOR_MSQ_COMPACTION) to current compaction task to avoid -+ // stalling. Setting "taskAssignment" to "auto" has the problem of not being able to determine the actual -+ // count, which is required for subsequent tasks. -+ final int slotsRequiredForCurrentTask = Math.min( -+ // Update the slots to 2 (min required for MSQ) if only 1 slot is available. -+ numAvailableTaskSlots == 1 ? 2 : numAvailableTaskSlots, -+ ClientMSQContext.MAX_TASK_SLOTS_FOR_MSQ_COMPACTION_TASK -+ ); -+ autoCompactionContext.put(ClientMSQContext.CTX_MAX_NUM_TASKS, slotsRequiredForCurrentTask); -+ -+ return slotsRequiredForCurrentTask; -+ } -+ } else { -+ return CompactionSlotManager.getMaxTaskSlotsForNativeCompactionTask( -+ config.getTuningConfig() -+ ); -+ } -+ } -+} -diff --git a/server/src/main/java/org/apache/druid/server/compaction/CompactionSnapshotBuilder.java b/server/src/main/java/org/apache/druid/server/compaction/CompactionSnapshotBuilder.java -new file mode 100644 -index 0000000000..d186587cb8 ---- /dev/null -+++ b/server/src/main/java/org/apache/druid/server/compaction/CompactionSnapshotBuilder.java -@@ -0,0 +1,108 @@ -+/* -+ * Licensed to the Apache Software Foundation (ASF) under one -+ * or more contributor license agreements. See the NOTICE file -+ * distributed with this work for additional information -+ * regarding copyright ownership. The ASF licenses this file -+ * to you under the Apache License, Version 2.0 (the -+ * "License"); you may not use this file except in compliance -+ * with the License. You may obtain a copy of the License at -+ * -+ * http://www.apache.org/licenses/LICENSE-2.0 -+ * -+ * Unless required by applicable law or agreed to in writing, -+ * software distributed under the License is distributed on an -+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -+ * KIND, either express or implied. See the License for the -+ * specific language governing permissions and limitations -+ * under the License. -+ */ -+ -+package org.apache.druid.server.compaction; -+ -+import org.apache.druid.server.coordinator.AutoCompactionSnapshot; -+import org.apache.druid.server.coordinator.stats.CoordinatorRunStats; -+import org.apache.druid.server.coordinator.stats.Dimension; -+import org.apache.druid.server.coordinator.stats.RowKey; -+import org.apache.druid.server.coordinator.stats.Stats; -+ -+import java.util.HashMap; -+import java.util.Map; -+ -+/** -+ * Builds {@link AutoCompactionSnapshot} for multiple datasources using the -+ * identified {@link CompactionCandidate} list. -+ */ -+public class CompactionSnapshotBuilder -+{ -+ private final CoordinatorRunStats stats; -+ private final Map datasourceToBuilder = new HashMap<>(); -+ -+ public CompactionSnapshotBuilder(CoordinatorRunStats runStats) -+ { -+ this.stats = runStats; -+ } -+ -+ public void addToComplete(CompactionCandidate candidate) -+ { -+ getBuilderForDatasource(candidate.getDataSource()) -+ .incrementCompactedStats(candidate.getStats()); -+ } -+ -+ public void addToPending(CompactionCandidate candidate) -+ { -+ getBuilderForDatasource(candidate.getDataSource()) -+ .incrementWaitingStats(candidate.getStats()); -+ } -+ -+ public void addToSkipped(CompactionCandidate candidate) -+ { -+ getBuilderForDatasource(candidate.getDataSource()) -+ .incrementSkippedStats(candidate.getStats()); -+ } -+ -+ public void moveFromPendingToSkipped(CompactionCandidate candidate) -+ { -+ getBuilderForDatasource(candidate.getDataSource()) -+ .decrementWaitingStats(candidate.getStats()); -+ addToSkipped(candidate); -+ } -+ -+ public void moveFromPendingToCompleted(CompactionCandidate candidate) -+ { -+ getBuilderForDatasource(candidate.getDataSource()) -+ .decrementWaitingStats(candidate.getStats()); -+ addToComplete(candidate); -+ } -+ -+ public Map build() -+ { -+ final Map datasourceToSnapshot = new HashMap<>(); -+ datasourceToBuilder.forEach((dataSource, builder) -> { -+ final AutoCompactionSnapshot autoCompactionSnapshot = builder.build(); -+ datasourceToSnapshot.put(dataSource, autoCompactionSnapshot); -+ collectSnapshotStats(autoCompactionSnapshot); -+ }); -+ -+ return datasourceToSnapshot; -+ } -+ -+ private AutoCompactionSnapshot.Builder getBuilderForDatasource(String dataSource) -+ { -+ return datasourceToBuilder.computeIfAbsent(dataSource, AutoCompactionSnapshot::builder); -+ } -+ -+ private void collectSnapshotStats(AutoCompactionSnapshot autoCompactionSnapshot) -+ { -+ final RowKey rowKey = RowKey.of(Dimension.DATASOURCE, autoCompactionSnapshot.getDataSource()); -+ -+ stats.add(Stats.Compaction.PENDING_BYTES, rowKey, autoCompactionSnapshot.getBytesAwaitingCompaction()); -+ stats.add(Stats.Compaction.PENDING_SEGMENTS, rowKey, autoCompactionSnapshot.getSegmentCountAwaitingCompaction()); -+ stats.add(Stats.Compaction.PENDING_INTERVALS, rowKey, autoCompactionSnapshot.getIntervalCountAwaitingCompaction()); -+ stats.add(Stats.Compaction.COMPACTED_BYTES, rowKey, autoCompactionSnapshot.getBytesCompacted()); -+ stats.add(Stats.Compaction.COMPACTED_SEGMENTS, rowKey, autoCompactionSnapshot.getSegmentCountCompacted()); -+ stats.add(Stats.Compaction.COMPACTED_INTERVALS, rowKey, autoCompactionSnapshot.getIntervalCountCompacted()); -+ stats.add(Stats.Compaction.SKIPPED_BYTES, rowKey, autoCompactionSnapshot.getBytesSkipped()); -+ stats.add(Stats.Compaction.SKIPPED_SEGMENTS, rowKey, autoCompactionSnapshot.getSegmentCountSkipped()); -+ stats.add(Stats.Compaction.SKIPPED_INTERVALS, rowKey, autoCompactionSnapshot.getIntervalCountSkipped()); -+ } -+} -diff --git a/server/src/main/java/org/apache/druid/server/compaction/CompactionStatistics.java b/server/src/main/java/org/apache/druid/server/compaction/CompactionStatistics.java -index 23f1b7fe9e..d7e5165586 100644 ---- a/server/src/main/java/org/apache/druid/server/compaction/CompactionStatistics.java -+++ b/server/src/main/java/org/apache/druid/server/compaction/CompactionStatistics.java -@@ -58,4 +58,11 @@ public class CompactionStatistics - numIntervals += other.getNumIntervals(); - numSegments += other.getNumSegments(); - } -+ -+ public void decrement(CompactionStatistics other) -+ { -+ totalBytes -= other.getTotalBytes(); -+ numIntervals -= other.getNumIntervals(); -+ numSegments -= other.getNumSegments(); -+ } - } -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 3c1616b6e7..be4acd00e2 100644 ---- a/server/src/main/java/org/apache/druid/server/compaction/CompactionStatus.java -+++ b/server/src/main/java/org/apache/druid/server/compaction/CompactionStatus.java -@@ -19,9 +19,7 @@ - - package org.apache.druid.server.compaction; - --import com.fasterxml.jackson.databind.ObjectMapper; - import org.apache.commons.lang3.ArrayUtils; --import org.apache.druid.client.indexing.ClientCompactionTaskGranularitySpec; - import org.apache.druid.client.indexing.ClientCompactionTaskQueryTuningConfig; - import org.apache.druid.common.config.Configs; - import org.apache.druid.data.input.impl.DimensionSchema; -@@ -64,6 +62,7 @@ public class CompactionStatus - * The order of the checks must be honored while evaluating them. - */ - private static final List> CHECKS = Arrays.asList( -+ Evaluator::inputBytesAreWithinLimit, - Evaluator::segmentsHaveBeenCompactedAtLeastOnce, - Evaluator::allCandidatesHaveSameCompactionState, - Evaluator::partitionsSpecIsUpToDate, -@@ -115,12 +114,17 @@ public class CompactionStatus - '}'; - } - -- private static CompactionStatus incomplete(String reasonFormat, Object... args) -+ public static CompactionStatus pending(String reasonFormat, Object... args) - { - return new CompactionStatus(State.PENDING, StringUtils.format(reasonFormat, args)); - } - -- private static CompactionStatus completeIfEqual( -+ /** -+ * Computes compaction status for the given field. The status is assumed to be -+ * COMPLETE (i.e. no further compaction is required) if the configured value -+ * of the field is null or equal to the current value. -+ */ -+ private static CompactionStatus completeIfNullOrEqual( - String field, - T configured, - T current, -@@ -141,7 +145,7 @@ public class CompactionStatus - Function stringFunction - ) - { -- return CompactionStatus.incomplete( -+ return CompactionStatus.pending( - "'%s' mismatch: required[%s], current[%s]", - field, - target == null ? null : stringFunction.apply(target), -@@ -187,14 +191,19 @@ public class CompactionStatus - } - } - -- static CompactionStatus skipped(String reasonFormat, Object... args) -+ public static CompactionStatus skipped(String reasonFormat, Object... args) - { - return new CompactionStatus(State.SKIPPED, StringUtils.format(reasonFormat, args)); - } - -- static CompactionStatus running(String reasonForCompaction) -+ public static CompactionStatus running(String message) -+ { -+ return new CompactionStatus(State.RUNNING, message); -+ } -+ -+ public static CompactionStatus complete(String message) - { -- return new CompactionStatus(State.RUNNING, reasonForCompaction); -+ return new CompactionStatus(State.COMPLETE, message); - } - - /** -@@ -204,23 +213,30 @@ public class CompactionStatus - */ - static CompactionStatus compute( - CompactionCandidate candidateSegments, -- DataSourceCompactionConfig config, -- ObjectMapper objectMapper -+ DataSourceCompactionConfig config - ) - { -- final Evaluator evaluator = new Evaluator(candidateSegments, config, objectMapper); -+ final Evaluator evaluator = new Evaluator(candidateSegments, config); - return CHECKS.stream() - .map(f -> f.apply(evaluator)) - .filter(status -> !status.isComplete()) - .findFirst().orElse(COMPLETE); - } - -+ @Nullable - static PartitionsSpec findPartitionsSpecFromConfig(ClientCompactionTaskQueryTuningConfig tuningConfig) - { - final PartitionsSpec partitionsSpecFromTuningConfig = tuningConfig.getPartitionsSpec(); - if (partitionsSpecFromTuningConfig == null) { -- final long maxTotalRows = Configs.valueOrDefault(tuningConfig.getMaxTotalRows(), Long.MAX_VALUE); -- return new DynamicPartitionsSpec(tuningConfig.getMaxRowsPerSegment(), maxTotalRows); -+ final Long maxTotalRows = tuningConfig.getMaxTotalRows(); -+ final Integer maxRowsPerSegment = tuningConfig.getMaxRowsPerSegment(); -+ -+ if (maxTotalRows == null && maxRowsPerSegment == null) { -+ // If not specified, return null so that partitionsSpec is not compared -+ return null; -+ } else { -+ return new DynamicPartitionsSpec(maxRowsPerSegment, maxTotalRows); -+ } - } else if (partitionsSpecFromTuningConfig instanceof DynamicPartitionsSpec) { - return new DynamicPartitionsSpec( - partitionsSpecFromTuningConfig.getMaxRowsPerSegment(), -@@ -276,22 +292,19 @@ public class CompactionStatus - */ - private static class Evaluator - { -- private final ObjectMapper objectMapper; - private final DataSourceCompactionConfig compactionConfig; - private final CompactionCandidate candidateSegments; - private final CompactionState lastCompactionState; - private final ClientCompactionTaskQueryTuningConfig tuningConfig; -- private final ClientCompactionTaskGranularitySpec existingGranularitySpec; -+ private final UserCompactionTaskGranularityConfig existingGranularitySpec; - private final UserCompactionTaskGranularityConfig configuredGranularitySpec; - - private Evaluator( - CompactionCandidate candidateSegments, -- DataSourceCompactionConfig compactionConfig, -- ObjectMapper objectMapper -+ DataSourceCompactionConfig compactionConfig - ) - { - this.candidateSegments = candidateSegments; -- this.objectMapper = objectMapper; - this.lastCompactionState = candidateSegments.getSegments().get(0).getLastCompactionState(); - this.compactionConfig = compactionConfig; - this.tuningConfig = ClientCompactionTaskQueryTuningConfig.from(compactionConfig); -@@ -299,9 +312,8 @@ public class CompactionStatus - if (lastCompactionState == null) { - this.existingGranularitySpec = null; - } else { -- this.existingGranularitySpec = convertIfNotNull( -- lastCompactionState.getGranularitySpec(), -- ClientCompactionTaskGranularitySpec.class -+ this.existingGranularitySpec = UserCompactionTaskGranularityConfig.from( -+ lastCompactionState.getGranularitySpec() - ); - } - } -@@ -309,7 +321,7 @@ public class CompactionStatus - private CompactionStatus segmentsHaveBeenCompactedAtLeastOnce() - { - if (lastCompactionState == null) { -- return CompactionStatus.incomplete("not compacted yet"); -+ return CompactionStatus.pending("not compacted yet"); - } else { - return COMPLETE; - } -@@ -323,7 +335,7 @@ public class CompactionStatus - if (allHaveSameCompactionState) { - return COMPLETE; - } else { -- return CompactionStatus.incomplete("segments have different last compaction states"); -+ return CompactionStatus.pending("segments have different last compaction states"); - } - } - -@@ -337,7 +349,7 @@ public class CompactionStatus - existingPartionsSpec.getMaxRowsPerSegment(), - ((DynamicPartitionsSpec) existingPartionsSpec).getMaxTotalRowsOr(Long.MAX_VALUE)); - } -- return CompactionStatus.completeIfEqual( -+ return CompactionStatus.completeIfNullOrEqual( - "partitionsSpec", - findPartitionsSpecFromConfig(tuningConfig), - existingPartionsSpec, -@@ -347,17 +359,17 @@ public class CompactionStatus - - private CompactionStatus indexSpecIsUpToDate() - { -- return CompactionStatus.completeIfEqual( -+ return CompactionStatus.completeIfNullOrEqual( - "indexSpec", - Configs.valueOrDefault(tuningConfig.getIndexSpec(), IndexSpec.getDefault()).getEffectiveSpec(), -- objectMapper.convertValue(lastCompactionState.getIndexSpec(), IndexSpec.class).getEffectiveSpec(), -+ lastCompactionState.getIndexSpec().getEffectiveSpec(), - String::valueOf - ); - } - - private CompactionStatus projectionsAreUpToDate() - { -- return CompactionStatus.completeIfEqual( -+ return CompactionStatus.completeIfNullOrEqual( - "projections", - compactionConfig.getProjections(), - lastCompactionState.getProjections(), -@@ -365,6 +377,19 @@ public class CompactionStatus - ); - } - -+ private CompactionStatus inputBytesAreWithinLimit() -+ { -+ final long inputSegmentSize = compactionConfig.getInputSegmentSizeBytes(); -+ if (candidateSegments.getTotalBytes() > inputSegmentSize) { -+ return CompactionStatus.skipped( -+ "'inputSegmentSize' exceeded: Total segment size[%d] is larger than allowed inputSegmentSize[%d]", -+ candidateSegments.getTotalBytes(), inputSegmentSize -+ ); -+ } else { -+ return COMPLETE; -+ } -+ } -+ - private CompactionStatus segmentGranularityIsUpToDate() - { - if (configuredGranularitySpec == null -@@ -385,7 +410,7 @@ public class CompactionStatus - segment -> !configuredSegmentGranularity.isAligned(segment.getInterval()) - ); - if (needsCompaction) { -- return CompactionStatus.incomplete( -+ return CompactionStatus.pending( - "segmentGranularity: segments do not align with target[%s]", - asString(configuredSegmentGranularity) - ); -@@ -407,7 +432,7 @@ public class CompactionStatus - if (configuredGranularitySpec == null) { - return COMPLETE; - } else { -- return CompactionStatus.completeIfEqual( -+ return CompactionStatus.completeIfNullOrEqual( - "rollup", - configuredGranularitySpec.isRollup(), - existingGranularitySpec == null ? null : existingGranularitySpec.isRollup(), -@@ -421,7 +446,7 @@ public class CompactionStatus - if (configuredGranularitySpec == null) { - return COMPLETE; - } else { -- return CompactionStatus.completeIfEqual( -+ return CompactionStatus.completeIfNullOrEqual( - "queryGranularity", - configuredGranularitySpec.getQueryGranularity(), - existingGranularitySpec == null ? null : existingGranularitySpec.getQueryGranularity(), -@@ -454,7 +479,7 @@ public class CompactionStatus - ? IndexSpec.getDefault() - : compactionConfig.getTuningConfig().getIndexSpec() - ); -- return CompactionStatus.completeIfEqual( -+ return CompactionStatus.completeIfNullOrEqual( - "dimensionsSpec", - configuredDimensions, - existingDimensions, -@@ -493,26 +518,13 @@ public class CompactionStatus - return COMPLETE; - } - -- CompactionTransformSpec existingTransformSpec = convertIfNotNull( -- lastCompactionState.getTransformSpec(), -- CompactionTransformSpec.class -- ); -- return CompactionStatus.completeIfEqual( -+ CompactionTransformSpec existingTransformSpec = lastCompactionState.getTransformSpec(); -+ return CompactionStatus.completeIfNullOrEqual( - "transformSpec filter", - compactionConfig.getTransformSpec().getFilter(), - existingTransformSpec == null ? null : existingTransformSpec.getFilter(), - String::valueOf - ); - } -- -- @Nullable -- private T convertIfNotNull(Object object, Class clazz) -- { -- if (object == null) { -- return null; -- } else { -- return objectMapper.convertValue(object, clazz); -- } -- } - } - } -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 cbf5f25f9d..401f413e7f 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 -@@ -19,14 +19,10 @@ - - package org.apache.druid.server.compaction; - --import com.fasterxml.jackson.databind.ObjectMapper; --import com.google.inject.Inject; --import org.apache.druid.client.indexing.ClientCompactionTaskQuery; - import org.apache.druid.indexer.TaskState; - import org.apache.druid.indexer.TaskStatus; - import org.apache.druid.java.util.common.DateTimes; - import org.apache.druid.server.coordinator.DataSourceCompactionConfig; --import org.apache.druid.server.coordinator.DruidCompactionConfig; - import org.joda.time.DateTime; - import org.joda.time.Duration; - import org.joda.time.Interval; -@@ -45,7 +41,6 @@ public class CompactionStatusTracker - { - private static final Duration MAX_STATUS_RETAIN_DURATION = Duration.standardHours(12); - -- private final ObjectMapper objectMapper; - private final ConcurrentHashMap datasourceStatuses - = new ConcurrentHashMap<>(); - private final ConcurrentHashMap submittedTaskIdToSegments -@@ -53,12 +48,6 @@ public class CompactionStatusTracker - - private final AtomicReference segmentSnapshotTime = new AtomicReference<>(); - -- @Inject -- public CompactionStatusTracker(ObjectMapper objectMapper) -- { -- this.objectMapper = objectMapper; -- } -- - public void stop() - { - datasourceStatuses.clear(); -@@ -74,7 +63,7 @@ public class CompactionStatusTracker - return datasourceStatuses - .getOrDefault(candidates.getDataSource(), DatasourceStatus.EMPTY) - .intervalToTaskStatus -- .get(candidates.getUmbrellaInterval()); -+ .get(candidates.getCompactionInterval()); - } - - /** -@@ -86,30 +75,20 @@ public class CompactionStatusTracker - return submittedTaskIdToSegments.keySet(); - } - -+ /** -+ * Checks if compaction can be started for the given {@link CompactionCandidate}. -+ * This method assumes that the given candidate is eligible for compaction -+ * based on the current compaction config/supervisor of the datasource. -+ */ - public CompactionStatus computeCompactionStatus( - CompactionCandidate candidate, -- DataSourceCompactionConfig config, - CompactionCandidateSearchPolicy searchPolicy - ) - { -- final CompactionStatus compactionStatus = CompactionStatus.compute(candidate, config, objectMapper); -- if (compactionStatus.isComplete()) { -- return compactionStatus; -- } -- -- // Skip intervals that violate max allowed input segment size -- final long inputSegmentSize = config.getInputSegmentSizeBytes(); -- if (candidate.getTotalBytes() > inputSegmentSize) { -- return CompactionStatus.skipped( -- "'inputSegmentSize' exceeded: Total segment size[%d] is larger than allowed inputSegmentSize[%d]", -- candidate.getTotalBytes(), inputSegmentSize -- ); -- } -- - // Skip intervals that already have a running task - final CompactionTaskStatus lastTaskStatus = getLatestTaskStatus(candidate); - if (lastTaskStatus != null && lastTaskStatus.getState() == TaskState.RUNNING) { -- return CompactionStatus.skipped("Task for interval is already running"); -+ return CompactionStatus.running("Task for interval is already running"); - } - - // Skip intervals that have been recently compacted if segment timeline is not updated yet -@@ -117,19 +96,23 @@ public class CompactionStatusTracker - if (lastTaskStatus != null - && lastTaskStatus.getState() == TaskState.SUCCESS - && snapshotTime != null && snapshotTime.isBefore(lastTaskStatus.getUpdatedTime())) { -- return CompactionStatus.skipped( -+ return CompactionStatus.complete( - "Segment timeline not updated since last compaction task succeeded" - ); - } - - // Skip intervals that have been filtered out by the policy -- if (!searchPolicy.isEligibleForCompaction(candidate, compactionStatus, lastTaskStatus)) { -+ if (!searchPolicy.isEligibleForCompaction(candidate, CompactionStatus.pending(""), lastTaskStatus)) { - return CompactionStatus.skipped("Rejected by search policy"); - } - -- return compactionStatus; -+ return CompactionStatus.pending("Not compacted yet"); - } - -+ /** -+ * Tracks the latest compaction status of the given compaction candidates. -+ * Used only by the {@link CompactionRunSimulator}. -+ */ - public void onCompactionStatusComputed( - CompactionCandidate candidateSegments, - DataSourceCompactionConfig config -@@ -143,17 +126,15 @@ public class CompactionStatusTracker - this.segmentSnapshotTime.set(snapshotTime); - } - -- public void onCompactionConfigUpdated(DruidCompactionConfig compactionConfig) -+ /** -+ * Updates the set of datasources that have compaction enabled and cleans up -+ * stale task statuses. -+ */ -+ public void resetActiveDatasources(Set compactionEnabledDatasources) - { -- final Set compactionEnabledDatasources = new HashSet<>(); -- if (compactionConfig.getCompactionConfigs() != null) { -- compactionConfig.getCompactionConfigs().forEach(config -> { -- getOrComputeDatasourceStatus(config.getDataSource()) -- .cleanupStaleTaskStatuses(); -- -- compactionEnabledDatasources.add(config.getDataSource()); -- }); -- } -+ compactionEnabledDatasources.forEach( -+ dataSource -> getOrComputeDatasourceStatus(dataSource).cleanupStaleTaskStatuses() -+ ); - - // Clean up state for datasources where compaction has been disabled - final Set allDatasources = new HashSet<>(datasourceStatuses.keySet()); -@@ -165,12 +146,12 @@ public class CompactionStatusTracker - } - - public void onTaskSubmitted( -- ClientCompactionTaskQuery taskPayload, -+ String taskId, - CompactionCandidate candidateSegments - ) - { -- submittedTaskIdToSegments.put(taskPayload.getId(), candidateSegments); -- getOrComputeDatasourceStatus(taskPayload.getDataSource()) -+ submittedTaskIdToSegments.put(taskId, candidateSegments); -+ getOrComputeDatasourceStatus(candidateSegments.getDataSource()) - .handleSubmittedTask(candidateSegments); - } - -@@ -186,7 +167,7 @@ public class CompactionStatusTracker - return; - } - -- final Interval compactionInterval = candidateSegments.getUmbrellaInterval(); -+ final Interval compactionInterval = candidateSegments.getCompactionInterval(); - getOrComputeDatasourceStatus(candidateSegments.getDataSource()) - .handleCompletedTask(compactionInterval, taskStatus); - } -@@ -229,7 +210,7 @@ public class CompactionStatusTracker - - void handleSubmittedTask(CompactionCandidate candidateSegments) - { -- final Interval interval = candidateSegments.getUmbrellaInterval(); -+ final Interval interval = candidateSegments.getCompactionInterval(); - final CompactionTaskStatus lastStatus = intervalToTaskStatus.get(interval); - - final DateTime now = DateTimes.nowUtc(); -diff --git a/server/src/main/java/org/apache/druid/server/compaction/DataSourceCompactibleSegmentIterator.java b/server/src/main/java/org/apache/druid/server/compaction/DataSourceCompactibleSegmentIterator.java -index e15d310b33..40867eb5c0 100644 ---- a/server/src/main/java/org/apache/druid/server/compaction/DataSourceCompactibleSegmentIterator.java -+++ b/server/src/main/java/org/apache/druid/server/compaction/DataSourceCompactibleSegmentIterator.java -@@ -65,8 +65,6 @@ public class DataSourceCompactibleSegmentIterator implements CompactionSegmentIt - - private final String dataSource; - private final DataSourceCompactionConfig config; -- private final CompactionStatusTracker statusTracker; -- private final CompactionCandidateSearchPolicy searchPolicy; - - private final List compactedSegments = new ArrayList<>(); - private final List skippedSegments = new ArrayList<>(); -@@ -83,14 +81,11 @@ public class DataSourceCompactibleSegmentIterator implements CompactionSegmentIt - DataSourceCompactionConfig config, - SegmentTimeline timeline, - List skipIntervals, -- CompactionCandidateSearchPolicy searchPolicy, -- CompactionStatusTracker statusTracker -+ CompactionCandidateSearchPolicy searchPolicy - ) - { -- this.statusTracker = statusTracker; - this.config = config; - this.dataSource = config.getDataSource(); -- this.searchPolicy = searchPolicy; - this.queue = new PriorityQueue<>(searchPolicy::compareCandidates); - - populateQueue(timeline, skipIntervals); -@@ -117,11 +112,12 @@ public class DataSourceCompactibleSegmentIterator implements CompactionSegmentIt - } - } - if (!partialEternitySegments.isEmpty()) { -- CompactionCandidate candidatesWithStatus = CompactionCandidate.from(partialEternitySegments).withCurrentStatus( -- CompactionStatus.skipped("Segments have partial-eternity intervals") -- ); -+ // Do not use the target segment granularity in the CompactionCandidate -+ // as Granularities.getIterable() will cause OOM due to the above issue -+ CompactionCandidate candidatesWithStatus = CompactionCandidate -+ .from(partialEternitySegments, null) -+ .withCurrentStatus(CompactionStatus.skipped("Segments have partial-eternity intervals")); - skippedSegments.add(candidatesWithStatus); -- statusTracker.onCompactionStatusComputed(candidatesWithStatus, config); - return; - } - -@@ -315,11 +311,9 @@ public class DataSourceCompactibleSegmentIterator implements CompactionSegmentIt - continue; - } - -- final CompactionCandidate candidates = CompactionCandidate.from(segments); -- final CompactionStatus compactionStatus -- = statusTracker.computeCompactionStatus(candidates, config, searchPolicy); -+ final CompactionCandidate candidates = CompactionCandidate.from(segments, config.getSegmentGranularity()); -+ final CompactionStatus compactionStatus = CompactionStatus.compute(candidates, config); - final CompactionCandidate candidatesWithStatus = candidates.withCurrentStatus(compactionStatus); -- statusTracker.onCompactionStatusComputed(candidatesWithStatus, config); - - if (compactionStatus.isComplete()) { - compactedSegments.add(candidatesWithStatus); -@@ -360,10 +354,10 @@ public class DataSourceCompactibleSegmentIterator implements CompactionSegmentIt - timeline.findNonOvershadowedObjectsInInterval(skipInterval, Partitions.ONLY_COMPLETE) - ); - if (!CollectionUtils.isNullOrEmpty(segments)) { -- final CompactionCandidate candidates = CompactionCandidate.from(segments); -+ final CompactionCandidate candidates = CompactionCandidate.from(segments, config.getSegmentGranularity()); - - final CompactionStatus reason; -- if (candidates.getUmbrellaInterval().overlaps(latestSkipInterval)) { -+ if (candidates.getCompactionInterval().overlaps(latestSkipInterval)) { - reason = CompactionStatus.skipped("skip offset from latest[%s]", skipOffset); - } else { - reason = CompactionStatus.skipped("interval locked by another task"); -@@ -371,7 +365,6 @@ public class DataSourceCompactibleSegmentIterator implements CompactionSegmentIt - - final CompactionCandidate candidatesWithStatus = candidates.withCurrentStatus(reason); - skippedSegments.add(candidatesWithStatus); -- statusTracker.onCompactionStatusComputed(candidatesWithStatus, config); - } - } - -diff --git a/server/src/main/java/org/apache/druid/server/compaction/PriorityBasedCompactionSegmentIterator.java b/server/src/main/java/org/apache/druid/server/compaction/PriorityBasedCompactionSegmentIterator.java -index 1e91df7e38..49d936fda0 100644 ---- a/server/src/main/java/org/apache/druid/server/compaction/PriorityBasedCompactionSegmentIterator.java -+++ b/server/src/main/java/org/apache/druid/server/compaction/PriorityBasedCompactionSegmentIterator.java -@@ -48,8 +48,7 @@ public class PriorityBasedCompactionSegmentIterator implements CompactionSegment - CompactionCandidateSearchPolicy searchPolicy, - Map compactionConfigs, - Map datasourceToTimeline, -- Map> skipIntervals, -- CompactionStatusTracker statusTracker -+ Map> skipIntervals - ) - { - this.queue = new PriorityQueue<>(searchPolicy::compareCandidates); -@@ -70,8 +69,7 @@ public class PriorityBasedCompactionSegmentIterator implements CompactionSegment - compactionConfigs.get(datasource), - timeline, - skipIntervals.getOrDefault(datasource, Collections.emptyList()), -- searchPolicy, -- statusTracker -+ searchPolicy - ) - ); - addNextItemForDatasourceToQueue(datasource); -diff --git a/server/src/main/java/org/apache/druid/server/coordinator/AutoCompactionSnapshot.java b/server/src/main/java/org/apache/druid/server/coordinator/AutoCompactionSnapshot.java -index e31a7919f2..b450dd5a50 100644 ---- a/server/src/main/java/org/apache/druid/server/coordinator/AutoCompactionSnapshot.java -+++ b/server/src/main/java/org/apache/druid/server/coordinator/AutoCompactionSnapshot.java -@@ -204,6 +204,25 @@ public class AutoCompactionSnapshot - ); - } - -+ @Override -+ public String toString() -+ { -+ return "AutoCompactionSnapshot{" + -+ "dataSource='" + dataSource + '\'' + -+ ", scheduleStatus=" + scheduleStatus + -+ ", message='" + message + '\'' + -+ ", bytesAwaitingCompaction=" + bytesAwaitingCompaction + -+ ", bytesCompacted=" + bytesCompacted + -+ ", bytesSkipped=" + bytesSkipped + -+ ", segmentCountAwaitingCompaction=" + segmentCountAwaitingCompaction + -+ ", segmentCountCompacted=" + segmentCountCompacted + -+ ", segmentCountSkipped=" + segmentCountSkipped + -+ ", intervalCountAwaitingCompaction=" + intervalCountAwaitingCompaction + -+ ", intervalCountCompacted=" + intervalCountCompacted + -+ ", intervalCountSkipped=" + intervalCountSkipped + -+ '}'; -+ } -+ - public static class Builder - { - private final String dataSource; -@@ -241,6 +260,11 @@ public class AutoCompactionSnapshot - waitingStats.increment(entry); - } - -+ public void decrementWaitingStats(CompactionStatistics entry) -+ { -+ waitingStats.decrement(entry); -+ } -+ - public void incrementCompactedStats(CompactionStatistics entry) - { - compactedStats.increment(entry); -diff --git a/server/src/main/java/org/apache/druid/server/coordinator/UserCompactionTaskGranularityConfig.java b/server/src/main/java/org/apache/druid/server/coordinator/UserCompactionTaskGranularityConfig.java -index be3c7e1af9..4ad57f3adf 100644 ---- a/server/src/main/java/org/apache/druid/server/coordinator/UserCompactionTaskGranularityConfig.java -+++ b/server/src/main/java/org/apache/druid/server/coordinator/UserCompactionTaskGranularityConfig.java -@@ -20,10 +20,12 @@ - package org.apache.druid.server.coordinator; - - import com.fasterxml.jackson.annotation.JsonCreator; -+import com.fasterxml.jackson.annotation.JsonInclude; - import com.fasterxml.jackson.annotation.JsonProperty; - import org.apache.druid.indexer.granularity.GranularitySpec; - import org.apache.druid.java.util.common.granularity.Granularity; - -+import javax.annotation.Nullable; - import java.util.Objects; - - /** -@@ -35,6 +37,7 @@ import java.util.Objects; - * Note that this class is not the same as {@link GranularitySpec}. This class simply holds Granularity configs - * and pass it to compaction task spec. This class does not do bucketing, group events or knows how to partition data. - */ -+@JsonInclude(JsonInclude.Include.NON_NULL) - public class UserCompactionTaskGranularityConfig - { - private final Granularity segmentGranularity; -@@ -71,6 +74,20 @@ public class UserCompactionTaskGranularityConfig - return rollup; - } - -+ @Nullable -+ public static UserCompactionTaskGranularityConfig from(GranularitySpec granularitySpec) -+ { -+ if (granularitySpec == null) { -+ return null; -+ } else { -+ return new UserCompactionTaskGranularityConfig( -+ granularitySpec.getSegmentGranularity(), -+ granularitySpec.getQueryGranularity(), -+ granularitySpec.isRollup() -+ ); -+ } -+ } -+ - @Override - public boolean equals(Object o) - { -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 135852c1de..1dca23090c 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 -@@ -23,7 +23,6 @@ import com.fasterxml.jackson.annotation.JacksonInject; - import com.fasterxml.jackson.annotation.JsonCreator; - import com.google.common.annotations.VisibleForTesting; - import com.google.common.base.Preconditions; --import com.google.common.base.Predicate; - import org.apache.druid.client.DataSourcesSnapshot; - import org.apache.druid.client.indexing.ClientCompactionIOConfig; - import org.apache.druid.client.indexing.ClientCompactionIntervalSpec; -@@ -32,26 +31,22 @@ import org.apache.druid.client.indexing.ClientCompactionTaskDimensionsSpec; - import org.apache.druid.client.indexing.ClientCompactionTaskGranularitySpec; - import org.apache.druid.client.indexing.ClientCompactionTaskQuery; - import org.apache.druid.client.indexing.ClientCompactionTaskQueryTuningConfig; --import org.apache.druid.client.indexing.ClientMSQContext; --import org.apache.druid.client.indexing.TaskPayloadResponse; - 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.indexer.CompactionEngine; --import org.apache.druid.indexer.TaskStatus; --import org.apache.druid.indexer.TaskStatusPlus; --import org.apache.druid.indexer.partitions.DimensionRangePartitionsSpec; --import org.apache.druid.java.util.common.ISE; - 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.metadata.LockFilterPolicy; - 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.CompactionSegmentIterator; -+import org.apache.druid.server.compaction.CompactionSlotManager; -+import org.apache.druid.server.compaction.CompactionSnapshotBuilder; -+import org.apache.druid.server.compaction.CompactionStatus; - import org.apache.druid.server.compaction.CompactionStatusTracker; - import org.apache.druid.server.compaction.PriorityBasedCompactionSegmentIterator; - import org.apache.druid.server.coordinator.AutoCompactionSnapshot; -@@ -66,10 +61,8 @@ import org.apache.druid.timeline.DataSegment; - import org.joda.time.Interval; - - import javax.annotation.Nullable; --import java.util.ArrayList; - import java.util.Collections; - import java.util.HashMap; --import java.util.HashSet; - import java.util.List; - import java.util.Map; - import java.util.Set; -@@ -79,21 +72,16 @@ import java.util.stream.Collectors; - - public class CompactSegments implements CoordinatorCustomDuty - { -- /** -- * Must be the same as org.apache.druid.indexing.common.task.CompactionTask.TYPE. -- */ -- public static final String COMPACTION_TASK_TYPE = "compact"; - /** - * Must be the same as org.apache.druid.indexing.common.task.Tasks.STORE_COMPACTION_STATE_KEY - */ - public static final String STORE_COMPACTION_STATE_KEY = "storeCompactionState"; -+ public static final String COMPACTION_INTERVAL_KEY = "compactionInterval"; - private static final String COMPACTION_REASON_KEY = "compactionReason"; - - private static final Logger LOG = new Logger(CompactSegments.class); - - private static final String TASK_ID_PREFIX = "coordinator-issued"; -- private static final Predicate IS_COMPACTION_TASK = -- status -> null != status && COMPACTION_TASK_TYPE.equals(status.getType()); - - private final CompactionStatusTracker statusTracker; - private final OverlordClient overlordClient; -@@ -152,75 +140,36 @@ public class CompactSegments implements CoordinatorCustomDuty - } - - statusTracker.onSegmentTimelineUpdated(dataSources.getSnapshotTime()); -- statusTracker.onCompactionConfigUpdated(dynamicConfig); - List compactionConfigList = dynamicConfig.getCompactionConfigs(); - if (compactionConfigList == null || compactionConfigList.isEmpty()) { - resetCompactionSnapshot(); -+ statusTracker.resetActiveDatasources(Set.of()); - return; - } - - Map compactionConfigs = compactionConfigList - .stream() - .collect(Collectors.toMap(DataSourceCompactionConfig::getDataSource, Function.identity())); -+ statusTracker.resetActiveDatasources(compactionConfigs.keySet()); - -- // Map from dataSource to list of intervals for which compaction will be skipped in this run -- final Map> intervalsToSkipCompaction = new HashMap<>(); -- -- // Fetch currently running compaction tasks -- int busyCompactionTaskSlots = 0; -- final List compactionTasks = CoordinatorDutyUtils.getStatusOfActiveTasks( -+ final CompactionSlotManager slotManager = new CompactionSlotManager( - overlordClient, -- IS_COMPACTION_TASK -+ statusTracker, -+ dynamicConfig.clusterConfig() - ); -+ stats.add(Stats.Compaction.MAX_SLOTS, slotManager.getNumAvailableTaskSlots()); - -- final Set activeTaskIds -- = compactionTasks.stream().map(TaskStatusPlus::getId).collect(Collectors.toSet()); -- trackStatusOfCompletedTasks(activeTaskIds); -- -- for (TaskStatusPlus status : compactionTasks) { -- final TaskPayloadResponse response = -- FutureUtils.getUnchecked(overlordClient.taskPayload(status.getId()), true); -- if (response == null) { -- throw new ISE("Could not find payload for active compaction task[%s]", status.getId()); -- } else if (!COMPACTION_TASK_TYPE.equals(response.getPayload().getType())) { -- throw new ISE( -- "Payload of active compaction task[%s] is of invalid type[%s]", -- status.getId(), response.getPayload().getType() -- ); -- } -- -- final ClientCompactionTaskQuery compactionTaskQuery = (ClientCompactionTaskQuery) response.getPayload(); -- DataSourceCompactionConfig dataSourceCompactionConfig = compactionConfigs.get(status.getDataSource()); -- if (cancelTaskIfGranularityChanged(compactionTaskQuery, dataSourceCompactionConfig)) { -- continue; -- } -- -- // Skip this interval as the current active compaction task is good -- final Interval interval = compactionTaskQuery.getIoConfig().getInputSpec().getInterval(); -- intervalsToSkipCompaction.computeIfAbsent(status.getDataSource(), k -> new ArrayList<>()) -- .add(interval); -- // Note: The default compactionRunnerType used here should match the default runner used in CompactionTask when -- // no runner is provided there. -- CompactionEngine compactionRunnerType = compactionTaskQuery.getCompactionRunner() == null -- ? CompactionEngine.NATIVE -- : compactionTaskQuery.getCompactionRunner().getType(); -- if (compactionRunnerType == CompactionEngine.NATIVE) { -- busyCompactionTaskSlots += -- findMaxNumTaskSlotsUsedByOneNativeCompactionTask(compactionTaskQuery.getTuningConfig()); -- } else { -- busyCompactionTaskSlots += findMaxNumTaskSlotsUsedByOneMsqCompactionTask(compactionTaskQuery.getContext()); -+ // Fetch currently running compaction tasks -+ for (ClientCompactionTaskQuery compactionTaskQuery : slotManager.fetchRunningCompactionTasks()) { -+ final String dataSource = compactionTaskQuery.getDataSource(); -+ DataSourceCompactionConfig dataSourceCompactionConfig = compactionConfigs.get(dataSource); -+ if (slotManager.cancelTaskOnlyIfGranularityChanged(compactionTaskQuery, dataSourceCompactionConfig)) { -+ stats.add(Stats.Compaction.CANCELLED_TASKS, RowKey.of(Dimension.DATASOURCE, dataSource), 1L); - } - } -+ stats.add(Stats.Compaction.AVAILABLE_SLOTS, slotManager.getNumAvailableTaskSlots()); - -- // Skip all the intervals locked by higher priority tasks for each datasource -- // This must be done after the invalid compaction tasks are cancelled -- // in the loop above so that their intervals are not considered locked -- getLockedIntervals(compactionConfigList).forEach( -- (dataSource, intervals) -> -- intervalsToSkipCompaction -- .computeIfAbsent(dataSource, ds -> new ArrayList<>()) -- .addAll(intervals) -- ); -+ slotManager.skipLockedIntervals(compactionConfigList); - - // Get iterator over segments to compact and submit compaction tasks - final CompactionCandidateSearchPolicy policy = dynamicConfig.getCompactionPolicy(); -@@ -228,27 +177,21 @@ public class CompactSegments implements CoordinatorCustomDuty - policy, - compactionConfigs, - dataSources.getUsedSegmentsTimelinesPerDataSource(), -- intervalsToSkipCompaction, -- statusTracker -+ slotManager.getDatasourceIntervalsToSkipCompaction() - ); - -- final int compactionTaskCapacity = getCompactionTaskCapacity(dynamicConfig); -- final int availableCompactionTaskSlots -- = getAvailableCompactionTaskSlots(compactionTaskCapacity, busyCompactionTaskSlots); -- -- final Map currentRunAutoCompactionSnapshotBuilders = new HashMap<>(); -+ final CompactionSnapshotBuilder compactionSnapshotBuilder = new CompactionSnapshotBuilder(stats); - final int numSubmittedCompactionTasks = submitCompactionTasks( - compactionConfigs, -- currentRunAutoCompactionSnapshotBuilders, -- availableCompactionTaskSlots, -+ compactionSnapshotBuilder, -+ slotManager, - iterator, -+ policy, - defaultEngine - ); - -- stats.add(Stats.Compaction.MAX_SLOTS, compactionTaskCapacity); -- stats.add(Stats.Compaction.AVAILABLE_SLOTS, availableCompactionTaskSlots); - stats.add(Stats.Compaction.SUBMITTED_TASKS, numSubmittedCompactionTasks); -- updateCompactionSnapshotStats(currentRunAutoCompactionSnapshotBuilders, iterator, stats); -+ updateCompactionSnapshotStats(compactionSnapshotBuilder, iterator, compactionConfigs); - } - - private void resetCompactionSnapshot() -@@ -271,334 +214,168 @@ public class CompactSegments implements CoordinatorCustomDuty - } - } - -- /** -- * Queries the Overlord for the status of all tasks that were submitted -- * recently but are not active anymore. The statuses are then updated in the -- * {@link #statusTracker}. -- */ -- private void trackStatusOfCompletedTasks(Set activeTaskIds) -- { -- final Set finishedTaskIds = new HashSet<>(statusTracker.getSubmittedTaskIds()); -- finishedTaskIds.removeAll(activeTaskIds); -- -- if (finishedTaskIds.isEmpty()) { -- return; -- } -- -- final Map taskStatusMap -- = FutureUtils.getUnchecked(overlordClient.taskStatuses(finishedTaskIds), true); -- for (String taskId : finishedTaskIds) { -- // Assume unknown task to have finished successfully -- final TaskStatus taskStatus = taskStatusMap.getOrDefault(taskId, TaskStatus.success(taskId)); -- if (taskStatus.isComplete()) { -- statusTracker.onTaskFinished(taskId, taskStatus); -- } -- } -- } -- -- /** -- * Cancels a currently running compaction task if the segment granularity -- * for this datasource has changed in the compaction config. -- * -- * @return true if the task was canceled, false otherwise. -- */ -- private boolean cancelTaskIfGranularityChanged( -- ClientCompactionTaskQuery compactionTaskQuery, -- DataSourceCompactionConfig dataSourceCompactionConfig -- ) -- { -- if (dataSourceCompactionConfig == null -- || dataSourceCompactionConfig.getGranularitySpec() == null -- || compactionTaskQuery.getGranularitySpec() == null) { -- return false; -- } -- -- Granularity configuredSegmentGranularity = dataSourceCompactionConfig.getGranularitySpec() -- .getSegmentGranularity(); -- Granularity taskSegmentGranularity = compactionTaskQuery.getGranularitySpec().getSegmentGranularity(); -- if (configuredSegmentGranularity == null || configuredSegmentGranularity.equals(taskSegmentGranularity)) { -- return false; -- } -- -- LOG.info( -- "Cancelling task[%s] as task segmentGranularity[%s] differs from compaction config segmentGranularity[%s].", -- compactionTaskQuery.getId(), taskSegmentGranularity, configuredSegmentGranularity -- ); -- overlordClient.cancelTask(compactionTaskQuery.getId()); -- return true; -- } -- -- /** -- * Gets a List of Intervals locked by higher priority tasks for each datasource. -- * However, when using a REPLACE lock for compaction, intervals locked with any APPEND lock will not be returned -- * Since compaction tasks submitted for these Intervals would have to wait anyway, -- * we skip these Intervals until the next compaction run. -- *

-- * For now, Segment Locks are being treated the same as Time Chunk Locks even -- * though they lock only a Segment and not the entire Interval. Thus, -- * a compaction task will not be submitted for an Interval if -- *

    -- *
  • either the whole Interval is locked by a higher priority Task with an incompatible lock type
  • -- *
  • or there is atleast one Segment in the Interval that is locked by a -- * higher priority Task
  • -- *
-- */ -- private Map> getLockedIntervals( -- List compactionConfigs -- ) -- { -- final List lockFilterPolicies = compactionConfigs -- .stream() -- .map(config -> -- new LockFilterPolicy(config.getDataSource(), config.getTaskPriority(), null, config.getTaskContext())) -- .collect(Collectors.toList()); -- final Map> datasourceToLockedIntervals = -- new HashMap<>(FutureUtils.getUnchecked(overlordClient.findLockedIntervals(lockFilterPolicies), true)); -- LOG.debug( -- "Skipping the following intervals for Compaction as they are currently locked: %s", -- datasourceToLockedIntervals -- ); -- -- return datasourceToLockedIntervals; -- } -- -- /** -- * Returns the maximum number of task slots used by one native compaction task at any time when the task is -- * issued with the given tuningConfig. -- */ -- public static int findMaxNumTaskSlotsUsedByOneNativeCompactionTask( -- @Nullable ClientCompactionTaskQueryTuningConfig tuningConfig -- ) -- { -- if (isParallelMode(tuningConfig)) { -- @Nullable -- Integer maxNumConcurrentSubTasks = tuningConfig.getMaxNumConcurrentSubTasks(); -- // Max number of task slots used in parallel mode = maxNumConcurrentSubTasks + 1 (supervisor task) -- return (maxNumConcurrentSubTasks == null ? 1 : maxNumConcurrentSubTasks) + 1; -- } else { -- return 1; -- } -- } -- -- /** -- * Returns the maximum number of task slots used by one MSQ compaction task at any time when the task is -- * issued with the given context. -- */ -- static int findMaxNumTaskSlotsUsedByOneMsqCompactionTask(@Nullable Map context) -- { -- return context == null -- ? ClientMSQContext.DEFAULT_MAX_NUM_TASKS -- : (int) context.getOrDefault(ClientMSQContext.CTX_MAX_NUM_TASKS, ClientMSQContext.DEFAULT_MAX_NUM_TASKS); -- } -- -- -- /** -- * Returns true if the compaction task can run in the parallel mode with the given tuningConfig. -- * This method should be synchronized with ParallelIndexSupervisorTask.isParallelMode(InputSource, ParallelIndexTuningConfig). -- */ -- @VisibleForTesting -- static boolean isParallelMode(@Nullable ClientCompactionTaskQueryTuningConfig tuningConfig) -- { -- if (null == tuningConfig) { -- return false; -- } -- boolean useRangePartitions = useRangePartitions(tuningConfig); -- int minRequiredNumConcurrentSubTasks = useRangePartitions ? 1 : 2; -- return tuningConfig.getMaxNumConcurrentSubTasks() != null -- && tuningConfig.getMaxNumConcurrentSubTasks() >= minRequiredNumConcurrentSubTasks; -- } -- -- private static boolean useRangePartitions(ClientCompactionTaskQueryTuningConfig tuningConfig) -- { -- // dynamic partitionsSpec will be used if getPartitionsSpec() returns null -- return tuningConfig.getPartitionsSpec() instanceof DimensionRangePartitionsSpec; -- } -- -- private int getCompactionTaskCapacity(DruidCompactionConfig dynamicConfig) -- { -- int totalWorkerCapacity = CoordinatorDutyUtils.getTotalWorkerCapacity(overlordClient); -- -- return Math.min( -- (int) (totalWorkerCapacity * dynamicConfig.getCompactionTaskSlotRatio()), -- dynamicConfig.getMaxCompactionTaskSlots() -- ); -- } -- -- private int getAvailableCompactionTaskSlots(int compactionTaskCapacity, int busyCompactionTaskSlots) -- { -- final int availableCompactionTaskSlots; -- if (busyCompactionTaskSlots > 0) { -- availableCompactionTaskSlots = Math.max(0, compactionTaskCapacity - busyCompactionTaskSlots); -- } else { -- // compactionTaskCapacity might be 0 if totalWorkerCapacity is low. -- // This guarantees that at least one slot is available if -- // compaction is enabled and estimatedIncompleteCompactionTasks is 0. -- availableCompactionTaskSlots = Math.max(1, compactionTaskCapacity); -- } -- LOG.debug( -- "Found [%d] available task slots for compaction out of max compaction task capacity [%d]", -- availableCompactionTaskSlots, compactionTaskCapacity -- ); -- -- return availableCompactionTaskSlots; -- } -- - /** - * Submits compaction tasks to the Overlord. Returns total number of tasks submitted. - */ - private int submitCompactionTasks( - Map compactionConfigs, -- Map currentRunAutoCompactionSnapshotBuilders, -- int numAvailableCompactionTaskSlots, -+ CompactionSnapshotBuilder snapshotBuilder, -+ CompactionSlotManager slotManager, - CompactionSegmentIterator iterator, -+ CompactionCandidateSearchPolicy policy, - CompactionEngine defaultEngine - ) - { -- if (numAvailableCompactionTaskSlots <= 0) { -+ if (slotManager.getNumAvailableTaskSlots() <= 0) { - return 0; - } - - int numSubmittedTasks = 0; - int totalTaskSlotsAssigned = 0; - -- while (iterator.hasNext() && totalTaskSlotsAssigned < numAvailableCompactionTaskSlots) { -+ while (iterator.hasNext() && totalTaskSlotsAssigned < slotManager.getNumAvailableTaskSlots()) { - final CompactionCandidate entry = iterator.next(); - final String dataSourceName = entry.getDataSource(); -+ final DataSourceCompactionConfig config = compactionConfigs.get(dataSourceName); - -- // As these segments will be compacted, we will aggregate the statistic to the Compacted statistics -- currentRunAutoCompactionSnapshotBuilders -- .computeIfAbsent(dataSourceName, AutoCompactionSnapshot::builder) -- .incrementCompactedStats(entry.getStats()); -+ final CompactionStatus compactionStatus = -+ statusTracker.computeCompactionStatus(entry, policy); -+ final CompactionCandidate candidatesWithStatus = entry.withCurrentStatus(compactionStatus); -+ statusTracker.onCompactionStatusComputed(candidatesWithStatus, config); - -- final DataSourceCompactionConfig config = compactionConfigs.get(dataSourceName); -- final List segmentsToCompact = entry.getSegments(); -- // ^ feed segmentsToCompact to config to get grouping of configs to segments -- -- // Create granularitySpec to send to compaction task -- Granularity segmentGranularityToUse = null; -- if (config.getGranularitySpec() == null || config.getGranularitySpec().getSegmentGranularity() == null) { -- // Determines segmentGranularity from the segmentsToCompact -- // Each batch of segmentToCompact from CompactionSegmentIterator will contain the same interval as -- // segmentGranularity is not set in the compaction config -- Interval interval = segmentsToCompact.get(0).getInterval(); -- if (segmentsToCompact.stream().allMatch(segment -> interval.overlaps(segment.getInterval()))) { -- try { -- segmentGranularityToUse = GranularityType.fromPeriod(interval.toPeriod()).getDefaultGranularity(); -- } -- catch (IllegalArgumentException iae) { -- // This case can happen if the existing segment interval result in complicated periods. -- // Fall back to setting segmentGranularity as null -- LOG.warn("Cannot determine segmentGranularity from interval[%s].", interval); -- } -- } else { -- LOG.warn( -- "Not setting 'segmentGranularity' for auto-compaction task as" -- + " the segments to compact do not have the same interval." -- ); -- } -+ if (compactionStatus.isComplete()) { -+ snapshotBuilder.addToComplete(candidatesWithStatus); -+ } else if (compactionStatus.isSkipped()) { -+ snapshotBuilder.addToSkipped(candidatesWithStatus); - } else { -- segmentGranularityToUse = config.getGranularitySpec().getSegmentGranularity(); -+ // As these segments will be compacted, we will aggregate the statistic to the Compacted statistics -+ snapshotBuilder.addToComplete(entry); - } -- final ClientCompactionTaskGranularitySpec granularitySpec = new ClientCompactionTaskGranularitySpec( -- segmentGranularityToUse, -- config.getGranularitySpec() != null ? config.getGranularitySpec().getQueryGranularity() : null, -- config.getGranularitySpec() != null ? config.getGranularitySpec().isRollup() : null -- ); - -- // Create dimensionsSpec to send to compaction task -- final ClientCompactionTaskDimensionsSpec dimensionsSpec; -- if (config.getDimensionsSpec() != null) { -- dimensionsSpec = new ClientCompactionTaskDimensionsSpec( -- config.getDimensionsSpec().getDimensions() -- ); -- } else { -- dimensionsSpec = null; -- } -+ final ClientCompactionTaskQuery taskPayload = createCompactionTask(entry, config, defaultEngine); - -- Boolean dropExisting = null; -- if (config.getIoConfig() != null) { -- dropExisting = config.getIoConfig().isDropExisting(); -- } -+ final String taskId = taskPayload.getId(); -+ FutureUtils.getUnchecked(overlordClient.runTask(taskId, taskPayload), true); -+ statusTracker.onTaskSubmitted(taskId, entry); - -- // If all the segments found to be compacted are tombstones then dropExisting -- // needs to be forced to true. This forcing needs to happen in the case that -- // the flag is null, or it is false. It is needed when it is null to avoid the -- // possibility of the code deciding to default it to false later. -- // Forcing the flag to true will enable the task ingestion code to generate new, compacted, tombstones to -- // cover the tombstones found to be compacted as well as to mark them -- // as compacted (update their lastCompactionState). If we don't force the -- // flag then every time this compact duty runs it will find the same tombstones -- // in the interval since their lastCompactionState -- // was not set repeating this over and over and the duty will not make progress; it -- // will become stuck on this set of tombstones. -- // This forcing code should be revised -- // when/if the autocompaction code policy to decide which segments to compact changes -- if (dropExisting == null || !dropExisting) { -- if (segmentsToCompact.stream().allMatch(DataSegment::isTombstone)) { -- dropExisting = true; -- LOG.info("Forcing dropExisting to true since all segments to compact are tombstones."); -- } -- } -+ LOG.debug( -+ "Submitted a compaction task[%s] for [%d] segments in datasource[%s], umbrella interval[%s].", -+ taskId, entry.numSegments(), dataSourceName, entry.getUmbrellaInterval() -+ ); -+ LOG.debugSegments(entry.getSegments(), "Compacting segments"); -+ numSubmittedTasks++; -+ totalTaskSlotsAssigned += slotManager.computeSlotsRequiredForTask(taskPayload, config); -+ } - -- final CompactionEngine compactionEngine = config.getEngine() == null ? defaultEngine : config.getEngine(); -- final Map autoCompactionContext = newAutoCompactionContext(config.getTaskContext()); -- int slotsRequiredForCurrentTask; -- -- if (compactionEngine == CompactionEngine.MSQ) { -- if (autoCompactionContext.containsKey(ClientMSQContext.CTX_MAX_NUM_TASKS)) { -- slotsRequiredForCurrentTask = (int) autoCompactionContext.get(ClientMSQContext.CTX_MAX_NUM_TASKS); -- } else { -- // Since MSQ needs all task slots for the calculated #tasks to be available upfront, allot all available -- // compaction slots (upto a max of MAX_TASK_SLOTS_FOR_MSQ_COMPACTION) to current compaction task to avoid -- // stalling. Setting "taskAssignment" to "auto" has the problem of not being able to determine the actual -- // count, which is required for subsequent tasks. -- slotsRequiredForCurrentTask = Math.min( -- // Update the slots to 2 (min required for MSQ) if only 1 slot is available. -- numAvailableCompactionTaskSlots == 1 ? 2 : numAvailableCompactionTaskSlots, -- ClientMSQContext.MAX_TASK_SLOTS_FOR_MSQ_COMPACTION_TASK -- ); -- autoCompactionContext.put(ClientMSQContext.CTX_MAX_NUM_TASKS, slotsRequiredForCurrentTask); -+ LOG.info("Submitted a total of [%d] compaction tasks.", numSubmittedTasks); -+ return numSubmittedTasks; -+ } -+ -+ /** -+ * Creates a {@link ClientCompactionTaskQuery} which can be submitted to an -+ * {@link OverlordClient} to start a compaction task. -+ */ -+ public static ClientCompactionTaskQuery createCompactionTask( -+ CompactionCandidate candidate, -+ DataSourceCompactionConfig config, -+ CompactionEngine defaultEngine -+ ) -+ { -+ final List segmentsToCompact = candidate.getSegments(); -+ -+ // Create granularitySpec to send to compaction task -+ Granularity segmentGranularityToUse = null; -+ if (config.getGranularitySpec() == null || config.getGranularitySpec().getSegmentGranularity() == null) { -+ // Determines segmentGranularity from the segmentsToCompact -+ // Each batch of segmentToCompact from CompactionSegmentIterator will contain the same interval as -+ // segmentGranularity is not set in the compaction config -+ Interval interval = segmentsToCompact.get(0).getInterval(); -+ if (segmentsToCompact.stream().allMatch(segment -> interval.overlaps(segment.getInterval()))) { -+ try { -+ segmentGranularityToUse = GranularityType.fromPeriod(interval.toPeriod()).getDefaultGranularity(); -+ } -+ catch (IllegalArgumentException iae) { -+ // This case can happen if the existing segment interval result in complicated periods. -+ // Fall back to setting segmentGranularity as null -+ LOG.warn("Cannot determine segmentGranularity from interval[%s].", interval); - } - } else { -- slotsRequiredForCurrentTask = findMaxNumTaskSlotsUsedByOneNativeCompactionTask(config.getTuningConfig()); -+ LOG.warn( -+ "Not setting 'segmentGranularity' for auto-compaction task as" -+ + " the segments to compact do not have the same interval." -+ ); - } -+ } else { -+ segmentGranularityToUse = config.getGranularitySpec().getSegmentGranularity(); -+ } -+ final ClientCompactionTaskGranularitySpec granularitySpec = new ClientCompactionTaskGranularitySpec( -+ segmentGranularityToUse, -+ config.getGranularitySpec() != null ? config.getGranularitySpec().getQueryGranularity() : null, -+ config.getGranularitySpec() != null ? config.getGranularitySpec().isRollup() : null -+ ); -+ -+ // Create dimensionsSpec to send to compaction task -+ final ClientCompactionTaskDimensionsSpec dimensionsSpec; -+ if (config.getDimensionsSpec() != null) { -+ dimensionsSpec = new ClientCompactionTaskDimensionsSpec( -+ config.getDimensionsSpec().getDimensions() -+ ); -+ } else { -+ dimensionsSpec = null; -+ } - -- if (entry.getCurrentStatus() != null) { -- autoCompactionContext.put(COMPACTION_REASON_KEY, entry.getCurrentStatus().getReason()); -+ Boolean dropExisting = null; -+ if (config.getIoConfig() != null) { -+ dropExisting = config.getIoConfig().isDropExisting(); -+ } -+ -+ // If all the segments found to be compacted are tombstones then dropExisting -+ // needs to be forced to true. This forcing needs to happen in the case that -+ // the flag is null, or it is false. It is needed when it is null to avoid the -+ // possibility of the code deciding to default it to false later. -+ // Forcing the flag to true will enable the task ingestion code to generate new, compacted, tombstones to -+ // cover the tombstones found to be compacted as well as to mark them -+ // as compacted (update their lastCompactionState). If we don't force the -+ // flag then every time this compact duty runs it will find the same tombstones -+ // in the interval since their lastCompactionState -+ // was not set repeating this over and over and the duty will not make progress; it -+ // will become stuck on this set of tombstones. -+ // This forcing code should be revised -+ // when/if the autocompaction code policy to decide which segments to compact changes -+ if (dropExisting == null || !dropExisting) { -+ if (segmentsToCompact.stream().allMatch(DataSegment::isTombstone)) { -+ dropExisting = true; -+ LOG.info("Forcing dropExisting to true since all segments to compact are tombstones."); - } -+ } - -- final String taskId = compactSegments( -- entry, -- config.getTaskPriority(), -- ClientCompactionTaskQueryTuningConfig.from( -- config.getTuningConfig(), -- config.getMaxRowsPerSegment(), -- config.getMetricsSpec() != null -- ), -- granularitySpec, -- dimensionsSpec, -- config.getMetricsSpec(), -- config.getTransformSpec(), -- config.getProjections(), -- dropExisting, -- autoCompactionContext, -- new ClientCompactionRunnerInfo(compactionEngine) -- ); -+ final CompactionEngine compactionEngine = config.getEngine() == null ? defaultEngine : config.getEngine(); -+ final Map autoCompactionContext = newAutoCompactionContext(config.getTaskContext()); - -- LOG.debug( -- "Submitted a compaction task[%s] for [%d] segments in datasource[%s], umbrella interval[%s].", -- taskId, segmentsToCompact.size(), dataSourceName, entry.getUmbrellaInterval() -- ); -- LOG.debugSegments(segmentsToCompact, "Compacting segments"); -- numSubmittedTasks++; -- totalTaskSlotsAssigned += slotsRequiredForCurrentTask; -+ if (candidate.getCurrentStatus() != null) { -+ autoCompactionContext.put(COMPACTION_REASON_KEY, candidate.getCurrentStatus().getReason()); - } - -- LOG.info("Submitted a total of [%d] compaction tasks.", numSubmittedTasks); -- return numSubmittedTasks; -+ return compactSegments( -+ candidate, -+ config.getTaskPriority(), -+ ClientCompactionTaskQueryTuningConfig.from( -+ config.getTuningConfig(), -+ config.getMaxRowsPerSegment(), -+ config.getMetricsSpec() != null -+ ), -+ granularitySpec, -+ dimensionsSpec, -+ config.getMetricsSpec(), -+ config.getTransformSpec(), -+ config.getProjections(), -+ dropExisting, -+ autoCompactionContext, -+ new ClientCompactionRunnerInfo(compactionEngine) -+ ); - } - -- private Map newAutoCompactionContext(@Nullable Map configuredContext) -+ private static Map newAutoCompactionContext(@Nullable Map configuredContext) - { - final Map newContext = configuredContext == null - ? new HashMap<>() -@@ -608,60 +385,23 @@ public class CompactSegments implements CoordinatorCustomDuty - } - - private void updateCompactionSnapshotStats( -- Map currentRunAutoCompactionSnapshotBuilders, -+ CompactionSnapshotBuilder snapshotBuilder, - CompactionSegmentIterator iterator, -- CoordinatorRunStats stats -+ Map datasourceToConfig - ) - { - // Mark all the segments remaining in the iterator as "awaiting compaction" - while (iterator.hasNext()) { -- final CompactionCandidate entry = iterator.next(); -- currentRunAutoCompactionSnapshotBuilders -- .computeIfAbsent(entry.getDataSource(), AutoCompactionSnapshot::builder) -- .incrementWaitingStats(entry.getStats()); -+ snapshotBuilder.addToPending(iterator.next()); - } -- -- // Statistics of all segments considered compacted after this run -- iterator.getCompactedSegments().forEach( -- candidateSegments -> currentRunAutoCompactionSnapshotBuilders -- .computeIfAbsent(candidateSegments.getDataSource(), AutoCompactionSnapshot::builder) -- .incrementCompactedStats(candidateSegments.getStats()) -- ); -- -- // Statistics of all segments considered skipped after this run -- iterator.getSkippedSegments().forEach( -- candidateSegments -> currentRunAutoCompactionSnapshotBuilders -- .computeIfAbsent(candidateSegments.getDataSource(), AutoCompactionSnapshot::builder) -- .incrementSkippedStats(candidateSegments.getStats()) -- ); -- -- final Map currentAutoCompactionSnapshotPerDataSource = new HashMap<>(); -- currentRunAutoCompactionSnapshotBuilders.forEach((dataSource, builder) -> { -- final AutoCompactionSnapshot autoCompactionSnapshot = builder.build(); -- currentAutoCompactionSnapshotPerDataSource.put(dataSource, autoCompactionSnapshot); -- collectSnapshotStats(autoCompactionSnapshot, stats); -+ iterator.getCompactedSegments().forEach(snapshotBuilder::addToComplete); -+ iterator.getSkippedSegments().forEach(entry -> { -+ statusTracker.onCompactionStatusComputed(entry, datasourceToConfig.get(entry.getDataSource())); -+ snapshotBuilder.addToSkipped(entry); - }); - - // Atomic update of autoCompactionSnapshotPerDataSource with the latest from this coordinator run -- autoCompactionSnapshotPerDataSource.set(currentAutoCompactionSnapshotPerDataSource); -- } -- -- private void collectSnapshotStats( -- AutoCompactionSnapshot autoCompactionSnapshot, -- CoordinatorRunStats stats -- ) -- { -- final RowKey rowKey = RowKey.of(Dimension.DATASOURCE, autoCompactionSnapshot.getDataSource()); -- -- stats.add(Stats.Compaction.PENDING_BYTES, rowKey, autoCompactionSnapshot.getBytesAwaitingCompaction()); -- stats.add(Stats.Compaction.PENDING_SEGMENTS, rowKey, autoCompactionSnapshot.getSegmentCountAwaitingCompaction()); -- stats.add(Stats.Compaction.PENDING_INTERVALS, rowKey, autoCompactionSnapshot.getIntervalCountAwaitingCompaction()); -- stats.add(Stats.Compaction.COMPACTED_BYTES, rowKey, autoCompactionSnapshot.getBytesCompacted()); -- stats.add(Stats.Compaction.COMPACTED_SEGMENTS, rowKey, autoCompactionSnapshot.getSegmentCountCompacted()); -- stats.add(Stats.Compaction.COMPACTED_INTERVALS, rowKey, autoCompactionSnapshot.getIntervalCountCompacted()); -- stats.add(Stats.Compaction.SKIPPED_BYTES, rowKey, autoCompactionSnapshot.getBytesSkipped()); -- stats.add(Stats.Compaction.SKIPPED_SEGMENTS, rowKey, autoCompactionSnapshot.getSegmentCountSkipped()); -- stats.add(Stats.Compaction.SKIPPED_INTERVALS, rowKey, autoCompactionSnapshot.getIntervalCountSkipped()); -+ autoCompactionSnapshotPerDataSource.set(snapshotBuilder.build()); - } - - @Nullable -@@ -675,7 +415,7 @@ public class CompactSegments implements CoordinatorCustomDuty - return autoCompactionSnapshotPerDataSource.get(); - } - -- private String compactSegments( -+ private static ClientCompactionTaskQuery compactSegments( - CompactionCandidate entry, - int compactionTaskPriority, - ClientCompactionTaskQueryTuningConfig tuningConfig, -@@ -685,7 +425,7 @@ public class CompactSegments implements CoordinatorCustomDuty - @Nullable CompactionTransformSpec transformSpec, - @Nullable List projectionSpecs, - @Nullable Boolean dropExisting, -- @Nullable Map context, -+ Map context, - ClientCompactionRunnerInfo compactionRunner - ) - { -@@ -698,17 +438,15 @@ public class CompactSegments implements CoordinatorCustomDuty - "Segments must have the same dataSource" - ); - -- context = context == null ? new HashMap<>() : context; - context.put("priority", compactionTaskPriority); - - final String taskId = IdUtils.newTaskId(TASK_ID_PREFIX, ClientCompactionTaskQuery.TYPE, dataSource, null); -- final Granularity segmentGranularity = granularitySpec == null ? null : granularitySpec.getSegmentGranularity(); - -- final ClientCompactionTaskQuery taskPayload = new ClientCompactionTaskQuery( -+ return new ClientCompactionTaskQuery( - taskId, - dataSource, - new ClientCompactionIOConfig( -- ClientCompactionIntervalSpec.fromSegments(segments, segmentGranularity), -+ new ClientCompactionIntervalSpec(entry.getCompactionInterval(), null), - dropExisting - ), - tuningConfig, -@@ -720,9 +458,5 @@ public class CompactSegments implements CoordinatorCustomDuty - context, - compactionRunner - ); -- FutureUtils.getUnchecked(overlordClient.runTask(taskId, taskPayload), true); -- statusTracker.onTaskSubmitted(taskPayload, entry); -- -- return taskId; - } - } -diff --git a/server/src/main/java/org/apache/druid/server/coordinator/stats/CoordinatorRunStats.java b/server/src/main/java/org/apache/druid/server/coordinator/stats/CoordinatorRunStats.java -index 76d788cb43..a9473b298c 100644 ---- a/server/src/main/java/org/apache/druid/server/coordinator/stats/CoordinatorRunStats.java -+++ b/server/src/main/java/org/apache/druid/server/coordinator/stats/CoordinatorRunStats.java -@@ -95,6 +95,20 @@ public class CoordinatorRunStats - return statValues == null ? 0 : statValues.getLong(stat); - } - -+ /** -+ * Invokes the {@link StatHandler} for each value of the specified stat. -+ */ -+ public void forEachEntry(CoordinatorStat stat, StatHandler handler) -+ { -+ allStats.forEach( -+ (rowKey, stats) -> stats.object2LongEntrySet().fastForEach(entry -> { -+ if (entry.getKey().equals(stat)) { -+ handler.handle(stat, rowKey, entry.getLongValue()); -+ } -+ }) -+ ); -+ } -+ - public void forEachStat(StatHandler handler) - { - allStats.forEach( -diff --git a/server/src/main/java/org/apache/druid/server/coordinator/stats/Stats.java b/server/src/main/java/org/apache/druid/server/coordinator/stats/Stats.java -index f4ae1b5f8a..3851b38fc9 100644 ---- a/server/src/main/java/org/apache/druid/server/coordinator/stats/Stats.java -+++ b/server/src/main/java/org/apache/druid/server/coordinator/stats/Stats.java -@@ -110,8 +110,17 @@ public class Stats - - public static class Compaction - { -+ public static final CoordinatorStat JOB_CREATION_TIME -+ = CoordinatorStat.toDebugAndEmit("jobCreationTime", "compact/createJobs/time"); -+ public static final CoordinatorStat CREATED_JOBS -+ = CoordinatorStat.toDebugAndEmit("jobQueueSize", "compact/createJobs/count"); -+ public static final CoordinatorStat SCHEDULER_RUN_TIME -+ = CoordinatorStat.toDebugAndEmit("schedulerRunTime", "compact/runScheduler/time"); -+ - public static final CoordinatorStat SUBMITTED_TASKS - = CoordinatorStat.toDebugAndEmit("compactTasks", "compact/task/count"); -+ public static final CoordinatorStat CANCELLED_TASKS -+ = CoordinatorStat.toDebugAndEmit("compactCancelled", "compactTask/cancelled/count"); - public static final CoordinatorStat MAX_SLOTS - = CoordinatorStat.toDebugAndEmit("compactMaxSlots", "compactTask/maxSlot/count"); - public static final CoordinatorStat AVAILABLE_SLOTS -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 646e798aa6..46ecc64d72 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 -@@ -24,6 +24,7 @@ import org.apache.druid.java.util.common.DateTimes; - import org.apache.druid.java.util.common.Intervals; - import org.apache.druid.java.util.common.granularity.Granularities; - import org.apache.druid.segment.IndexIO; -+import org.apache.druid.server.compaction.CompactionCandidate; - import org.apache.druid.timeline.DataSegment; - import org.apache.druid.timeline.partition.NoneShardSpec; - import org.junit.Assert; -@@ -72,43 +73,43 @@ public class ClientCompactionIntervalSpecTest - public void testFromSegmentWithNoSegmentGranularity() - { - // The umbrella interval of segments is 2015-02-12/2015-04-14 -- ClientCompactionIntervalSpec actual = ClientCompactionIntervalSpec.fromSegments(ImmutableList.of(dataSegment1, dataSegment2, dataSegment3), null); -- Assert.assertEquals(Intervals.of("2015-02-12/2015-04-14"), actual.getInterval()); -+ CompactionCandidate actual = CompactionCandidate.from(ImmutableList.of(dataSegment1, dataSegment2, dataSegment3), null); -+ Assert.assertEquals(Intervals.of("2015-02-12/2015-04-14"), actual.getCompactionInterval()); - } - - @Test - public void testFromSegmentWitSegmentGranularitySameAsSegment() - { - // The umbrella interval of segments is 2015-04-11/2015-04-12 -- ClientCompactionIntervalSpec actual = ClientCompactionIntervalSpec.fromSegments(ImmutableList.of(dataSegment1), Granularities.DAY); -- Assert.assertEquals(Intervals.of("2015-04-11/2015-04-12"), actual.getInterval()); -+ CompactionCandidate actual = CompactionCandidate.from(ImmutableList.of(dataSegment1), Granularities.DAY); -+ Assert.assertEquals(Intervals.of("2015-04-11/2015-04-12"), actual.getCompactionInterval()); - } - - @Test - public void testFromSegmentWithCoarserSegmentGranularity() - { - // The umbrella interval of segments is 2015-02-12/2015-04-14 -- ClientCompactionIntervalSpec actual = ClientCompactionIntervalSpec.fromSegments(ImmutableList.of(dataSegment1, dataSegment2, dataSegment3), Granularities.YEAR); -+ CompactionCandidate actual = CompactionCandidate.from(ImmutableList.of(dataSegment1, dataSegment2, dataSegment3), Granularities.YEAR); - // The compaction interval should be expanded to start of the year and end of the year to cover the segmentGranularity -- Assert.assertEquals(Intervals.of("2015-01-01/2016-01-01"), actual.getInterval()); -+ Assert.assertEquals(Intervals.of("2015-01-01/2016-01-01"), actual.getCompactionInterval()); - } - - @Test - public void testFromSegmentWithFinerSegmentGranularityAndUmbrellaIntervalAlign() - { - // The umbrella interval of segments is 2015-02-12/2015-04-14 -- ClientCompactionIntervalSpec actual = ClientCompactionIntervalSpec.fromSegments(ImmutableList.of(dataSegment1, dataSegment2, dataSegment3), Granularities.DAY); -+ CompactionCandidate actual = CompactionCandidate.from(ImmutableList.of(dataSegment1, dataSegment2, dataSegment3), Granularities.DAY); - // The segmentGranularity of DAY align with the umbrella interval (umbrella interval can be evenly divide into the segmentGranularity) -- Assert.assertEquals(Intervals.of("2015-02-12/2015-04-14"), actual.getInterval()); -+ Assert.assertEquals(Intervals.of("2015-02-12/2015-04-14"), actual.getCompactionInterval()); - } - - @Test - public void testFromSegmentWithFinerSegmentGranularityAndUmbrellaIntervalNotAlign() - { - // The umbrella interval of segments is 2015-02-12/2015-04-14 -- ClientCompactionIntervalSpec actual = ClientCompactionIntervalSpec.fromSegments(ImmutableList.of(dataSegment1, dataSegment2, dataSegment3), Granularities.WEEK); -+ CompactionCandidate actual = CompactionCandidate.from(ImmutableList.of(dataSegment1, dataSegment2, dataSegment3), Granularities.WEEK); - // The segmentGranularity of WEEK does not align with the umbrella interval (umbrella interval cannot be evenly divide into the segmentGranularity) - // Hence the compaction interval is modified to aling with the segmentGranularity -- Assert.assertEquals(Intervals.of("2015-02-09/2015-04-20"), actual.getInterval()); -+ Assert.assertEquals(Intervals.of("2015-02-09/2015-04-20"), actual.getCompactionInterval()); - } - } -diff --git a/server/src/test/java/org/apache/druid/server/compaction/CompactionRunSimulatorTest.java b/server/src/test/java/org/apache/druid/server/compaction/CompactionRunSimulatorTest.java -index fd491ef930..bab2e351df 100644 ---- a/server/src/test/java/org/apache/druid/server/compaction/CompactionRunSimulatorTest.java -+++ b/server/src/test/java/org/apache/druid/server/compaction/CompactionRunSimulatorTest.java -@@ -19,14 +19,12 @@ - - package org.apache.druid.server.compaction; - --import com.fasterxml.jackson.databind.ObjectMapper; - import com.google.common.util.concurrent.Futures; - import com.google.common.util.concurrent.ListenableFuture; - import org.apache.druid.client.indexing.TaskPayloadResponse; - import org.apache.druid.indexer.CompactionEngine; - import org.apache.druid.indexer.TaskStatus; - import org.apache.druid.indexer.TaskStatusPlus; --import org.apache.druid.jackson.DefaultObjectMapper; - import org.apache.druid.java.util.common.CloseableIterators; - import org.apache.druid.java.util.common.Intervals; - import org.apache.druid.java.util.common.granularity.Granularities; -@@ -51,10 +49,8 @@ import java.util.Set; - - public class CompactionRunSimulatorTest - { -- private static final ObjectMapper OBJECT_MAPPER = new DefaultObjectMapper(); -- - private final CompactionRunSimulator simulator = new CompactionRunSimulator( -- new CompactionStatusTracker(OBJECT_MAPPER), -+ new CompactionStatusTracker(), - new TestOverlordClient() - ); - -@@ -115,7 +111,7 @@ public class CompactionRunSimulatorTest - ); - Assert.assertEquals( - Collections.singletonList( -- Arrays.asList("wiki", Intervals.of("2013-01-10/P1D"), 10, 1_000_000_000L, "skip offset from latest[P1D]") -+ Arrays.asList("wiki", Intervals.of("2013-01-10/P1D"), 10, 1_000_000_000L, 1, "skip offset from latest[P1D]") - ), - skippedTable.getRows() - ); -diff --git a/server/src/test/java/org/apache/druid/server/compaction/CompactionStatusTest.java b/server/src/test/java/org/apache/druid/server/compaction/CompactionStatusTest.java -index 14af1c78da..d201b84135 100644 ---- a/server/src/test/java/org/apache/druid/server/compaction/CompactionStatusTest.java -+++ b/server/src/test/java/org/apache/druid/server/compaction/CompactionStatusTest.java -@@ -19,7 +19,6 @@ - - package org.apache.druid.server.compaction; - --import com.fasterxml.jackson.databind.ObjectMapper; - import org.apache.druid.client.indexing.ClientCompactionTaskQueryTuningConfig; - import org.apache.druid.data.input.impl.AggregateProjectionSpec; - import org.apache.druid.data.input.impl.DimensionsSpec; -@@ -31,7 +30,6 @@ import org.apache.druid.indexer.partitions.DimensionRangePartitionsSpec; - import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; - import org.apache.druid.indexer.partitions.HashedPartitionsSpec; - import org.apache.druid.indexer.partitions.PartitionsSpec; --import org.apache.druid.jackson.DefaultObjectMapper; - import org.apache.druid.java.util.common.Intervals; - import org.apache.druid.java.util.common.granularity.Granularities; - import org.apache.druid.query.aggregation.LongSumAggregatorFactory; -@@ -47,6 +45,7 @@ import org.apache.druid.server.coordinator.UserCompactionTaskGranularityConfig; - import org.apache.druid.server.coordinator.UserCompactionTaskQueryTuningConfig; - import org.apache.druid.timeline.CompactionState; - import org.apache.druid.timeline.DataSegment; -+import org.apache.druid.timeline.SegmentId; - import org.junit.Assert; - import org.junit.Test; - -@@ -55,14 +54,9 @@ import java.util.List; - - public class CompactionStatusTest - { -- private static final ObjectMapper OBJECT_MAPPER = new DefaultObjectMapper(); -- - private static final DataSegment WIKI_SEGMENT -- = DataSegment.builder() -- .dataSource(TestDataSource.WIKI) -- .interval(Intervals.of("2013-01-01/PT1H")) -+ = DataSegment.builder(SegmentId.of(TestDataSource.WIKI, Intervals.of("2013-01-01/PT1H"), "v1", 0)) - .size(100_000_000L) -- .version("v1") - .build(); - - @Test -@@ -70,8 +64,7 @@ public class CompactionStatusTest - { - final ClientCompactionTaskQueryTuningConfig tuningConfig - = ClientCompactionTaskQueryTuningConfig.from(null); -- Assert.assertEquals( -- new DynamicPartitionsSpec(null, Long.MAX_VALUE), -+ Assert.assertNull( - CompactionStatus.findPartitionsSpecFromConfig(tuningConfig) - ); - } -@@ -203,9 +196,14 @@ public class CompactionStatusTest - @Test - public void testStatusWhenLastCompactionStateIsEmpty() - { -+ final PartitionsSpec requiredPartitionsSpec = new DynamicPartitionsSpec(5_000_000, null); - verifyCompactionStatusIsPendingBecause( - new CompactionState(null, null, null, null, null, null, null), -- InlineSchemaDataSourceCompactionConfig.builder().forDataSource(TestDataSource.WIKI).build(), -+ InlineSchemaDataSourceCompactionConfig -+ .builder() -+ .withTuningConfig(createTuningConfig(requiredPartitionsSpec, null)) -+ .forDataSource(TestDataSource.WIKI) -+ .build(), - "'partitionsSpec' mismatch: required['dynamic' with 5,000,000 rows], current[null]" - ); - } -@@ -213,12 +211,16 @@ public class CompactionStatusTest - @Test - public void testStatusOnPartitionsSpecMismatch() - { -+ final PartitionsSpec requiredPartitionsSpec = new DynamicPartitionsSpec(5_000_000, null); - final PartitionsSpec currentPartitionsSpec = new DynamicPartitionsSpec(100, null); - - final CompactionState lastCompactionState - = new CompactionState(currentPartitionsSpec, null, null, null, null, null, null); -- final DataSourceCompactionConfig compactionConfig -- = InlineSchemaDataSourceCompactionConfig.builder().forDataSource(TestDataSource.WIKI).build(); -+ final DataSourceCompactionConfig compactionConfig = InlineSchemaDataSourceCompactionConfig -+ .builder() -+ .withTuningConfig(createTuningConfig(requiredPartitionsSpec, null)) -+ .forDataSource(TestDataSource.WIKI) -+ .build(); - - verifyCompactionStatusIsPendingBecause( - lastCompactionState, -@@ -323,9 +325,8 @@ public class CompactionStatusTest - - final DataSegment segment = DataSegment.builder(WIKI_SEGMENT).lastCompactionState(lastCompactionState).build(); - final CompactionStatus status = CompactionStatus.compute( -- CompactionCandidate.from(Collections.singletonList(segment)), -- compactionConfig, -- OBJECT_MAPPER -+ CompactionCandidate.from(List.of(segment), Granularities.HOUR), -+ compactionConfig - ); - Assert.assertTrue(status.isComplete()); - } -@@ -373,9 +374,8 @@ public class CompactionStatusTest - - final DataSegment segment = DataSegment.builder(WIKI_SEGMENT).lastCompactionState(lastCompactionState).build(); - final CompactionStatus status = CompactionStatus.compute( -- CompactionCandidate.from(Collections.singletonList(segment)), -- compactionConfig, -- OBJECT_MAPPER -+ CompactionCandidate.from(List.of(segment), Granularities.HOUR), -+ compactionConfig - ); - Assert.assertTrue(status.isComplete()); - } -@@ -428,9 +428,8 @@ public class CompactionStatusTest - - final DataSegment segment = DataSegment.builder(WIKI_SEGMENT).lastCompactionState(lastCompactionState).build(); - final CompactionStatus status = CompactionStatus.compute( -- CompactionCandidate.from(Collections.singletonList(segment)), -- compactionConfig, -- OBJECT_MAPPER -+ CompactionCandidate.from(List.of(segment), Granularities.HOUR), -+ compactionConfig - ); - Assert.assertFalse(status.isComplete()); - } -@@ -482,9 +481,8 @@ public class CompactionStatusTest - - final DataSegment segment = DataSegment.builder(WIKI_SEGMENT).lastCompactionState(lastCompactionState).build(); - final CompactionStatus status = CompactionStatus.compute( -- CompactionCandidate.from(Collections.singletonList(segment)), -- compactionConfig, -- OBJECT_MAPPER -+ CompactionCandidate.from(List.of(segment), null), -+ compactionConfig - ); - Assert.assertTrue(status.isComplete()); - } -@@ -536,9 +534,8 @@ public class CompactionStatusTest - - final DataSegment segment = DataSegment.builder(WIKI_SEGMENT).lastCompactionState(lastCompactionState).build(); - final CompactionStatus status = CompactionStatus.compute( -- CompactionCandidate.from(Collections.singletonList(segment)), -- compactionConfig, -- OBJECT_MAPPER -+ CompactionCandidate.from(List.of(segment), null), -+ compactionConfig - ); - Assert.assertFalse(status.isComplete()); - } -@@ -554,9 +551,8 @@ public class CompactionStatusTest - .lastCompactionState(lastCompactionState) - .build(); - final CompactionStatus status = CompactionStatus.compute( -- CompactionCandidate.from(Collections.singletonList(segment)), -- compactionConfig, -- OBJECT_MAPPER -+ CompactionCandidate.from(List.of(segment), null), -+ compactionConfig - ); - - Assert.assertFalse(status.isComplete()); -diff --git a/server/src/test/java/org/apache/druid/server/compaction/CompactionStatusTrackerTest.java b/server/src/test/java/org/apache/druid/server/compaction/CompactionStatusTrackerTest.java -index 1878aab8c0..c0496b0705 100644 ---- a/server/src/test/java/org/apache/druid/server/compaction/CompactionStatusTrackerTest.java -+++ b/server/src/test/java/org/apache/druid/server/compaction/CompactionStatusTrackerTest.java -@@ -19,26 +19,20 @@ - - package org.apache.druid.server.compaction; - --import com.fasterxml.jackson.databind.ObjectMapper; --import org.apache.druid.client.indexing.ClientCompactionTaskQuery; - import org.apache.druid.indexer.TaskState; - import org.apache.druid.indexer.TaskStatus; --import org.apache.druid.jackson.DefaultObjectMapper; - import org.apache.druid.java.util.common.DateTimes; - import org.apache.druid.segment.TestDataSource; - import org.apache.druid.server.coordinator.CreateDataSegments; --import org.apache.druid.server.coordinator.DataSourceCompactionConfig; --import org.apache.druid.server.coordinator.InlineSchemaDataSourceCompactionConfig; - import org.apache.druid.timeline.DataSegment; - import org.junit.Assert; - import org.junit.Before; - import org.junit.Test; - --import java.util.Collections; -+import java.util.List; - - public class CompactionStatusTrackerTest - { -- private static final ObjectMapper MAPPER = new DefaultObjectMapper(); - private static final DataSegment WIKI_SEGMENT - = CreateDataSegments.ofDatasource(TestDataSource.WIKI).eachOfSizeInMb(100).get(0); - -@@ -47,15 +41,15 @@ public class CompactionStatusTrackerTest - @Before - public void setup() - { -- statusTracker = new CompactionStatusTracker(MAPPER); -+ statusTracker = new CompactionStatusTracker(); - } - - @Test - public void testGetLatestTaskStatusForSubmittedTask() - { - final CompactionCandidate candidateSegments -- = CompactionCandidate.from(Collections.singletonList(WIKI_SEGMENT)); -- statusTracker.onTaskSubmitted(createCompactionTask("task1"), candidateSegments); -+ = CompactionCandidate.from(List.of(WIKI_SEGMENT), null); -+ statusTracker.onTaskSubmitted("task1", candidateSegments); - - CompactionTaskStatus status = statusTracker.getLatestTaskStatus(candidateSegments); - Assert.assertEquals(TaskState.RUNNING, status.getState()); -@@ -65,8 +59,8 @@ public class CompactionStatusTrackerTest - public void testGetLatestTaskStatusForSuccessfulTask() - { - final CompactionCandidate candidateSegments -- = CompactionCandidate.from(Collections.singletonList(WIKI_SEGMENT)); -- statusTracker.onTaskSubmitted(createCompactionTask("task1"), candidateSegments); -+ = CompactionCandidate.from(List.of(WIKI_SEGMENT), null); -+ statusTracker.onTaskSubmitted("task1", candidateSegments); - statusTracker.onTaskFinished("task1", TaskStatus.success("task1")); - - CompactionTaskStatus status = statusTracker.getLatestTaskStatus(candidateSegments); -@@ -77,8 +71,8 @@ public class CompactionStatusTrackerTest - public void testGetLatestTaskStatusForFailedTask() - { - final CompactionCandidate candidateSegments -- = CompactionCandidate.from(Collections.singletonList(WIKI_SEGMENT)); -- statusTracker.onTaskSubmitted(createCompactionTask("task1"), candidateSegments); -+ = CompactionCandidate.from(List.of(WIKI_SEGMENT), null); -+ statusTracker.onTaskSubmitted("task1", candidateSegments); - statusTracker.onTaskFinished("task1", TaskStatus.failure("task1", "some failure")); - - CompactionTaskStatus status = statusTracker.getLatestTaskStatus(candidateSegments); -@@ -90,12 +84,12 @@ public class CompactionStatusTrackerTest - public void testGetLatestTaskStatusForRepeatedlyFailingTask() - { - final CompactionCandidate candidateSegments -- = CompactionCandidate.from(Collections.singletonList(WIKI_SEGMENT)); -+ = CompactionCandidate.from(List.of(WIKI_SEGMENT), null); - -- statusTracker.onTaskSubmitted(createCompactionTask("task1"), candidateSegments); -+ statusTracker.onTaskSubmitted("task1", candidateSegments); - statusTracker.onTaskFinished("task1", TaskStatus.failure("task1", "some failure")); - -- statusTracker.onTaskSubmitted(createCompactionTask("task2"), candidateSegments); -+ statusTracker.onTaskSubmitted("task2", candidateSegments); - CompactionTaskStatus status = statusTracker.getLatestTaskStatus(candidateSegments); - Assert.assertEquals(TaskState.RUNNING, status.getState()); - Assert.assertEquals(1, status.getNumConsecutiveFailures()); -@@ -110,25 +104,23 @@ public class CompactionStatusTrackerTest - @Test - public void testComputeCompactionStatusForSuccessfulTask() - { -- final DataSourceCompactionConfig compactionConfig -- = InlineSchemaDataSourceCompactionConfig.builder().forDataSource(TestDataSource.WIKI).build(); - final NewestSegmentFirstPolicy policy = new NewestSegmentFirstPolicy(null); - final CompactionCandidate candidateSegments -- = CompactionCandidate.from(Collections.singletonList(WIKI_SEGMENT)); -+ = CompactionCandidate.from(List.of(WIKI_SEGMENT), null); - - // Verify that interval is originally eligible for compaction - CompactionStatus status -- = statusTracker.computeCompactionStatus(candidateSegments, compactionConfig, policy); -+ = statusTracker.computeCompactionStatus(candidateSegments, policy); - Assert.assertEquals(CompactionStatus.State.PENDING, status.getState()); -- Assert.assertEquals("not compacted yet", status.getReason()); -+ Assert.assertEquals("Not compacted yet", status.getReason()); - - // Verify that interval is skipped for compaction after task has finished - statusTracker.onSegmentTimelineUpdated(DateTimes.nowUtc().minusMinutes(1)); -- statusTracker.onTaskSubmitted(createCompactionTask("task1"), candidateSegments); -+ statusTracker.onTaskSubmitted("task1", candidateSegments); - statusTracker.onTaskFinished("task1", TaskStatus.success("task1")); - -- status = statusTracker.computeCompactionStatus(candidateSegments, compactionConfig, policy); -- Assert.assertEquals(CompactionStatus.State.SKIPPED, status.getState()); -+ status = statusTracker.computeCompactionStatus(candidateSegments, policy); -+ Assert.assertEquals(CompactionStatus.State.COMPLETE, status.getState()); - Assert.assertEquals( - "Segment timeline not updated since last compaction task succeeded", - status.getReason() -@@ -136,26 +128,7 @@ public class CompactionStatusTrackerTest - - // Verify that interval becomes eligible again after timeline has been updated - statusTracker.onSegmentTimelineUpdated(DateTimes.nowUtc()); -- status = statusTracker.computeCompactionStatus(candidateSegments, compactionConfig, policy); -+ status = statusTracker.computeCompactionStatus(candidateSegments, policy); - Assert.assertEquals(CompactionStatus.State.PENDING, status.getState()); - } -- -- private ClientCompactionTaskQuery createCompactionTask( -- String taskId -- ) -- { -- return new ClientCompactionTaskQuery( -- taskId, -- TestDataSource.WIKI, -- null, -- null, -- null, -- null, -- null, -- null, -- null, -- null, -- null -- ); -- } - } -diff --git a/server/src/test/java/org/apache/druid/server/compaction/NewestSegmentFirstPolicyTest.java b/server/src/test/java/org/apache/druid/server/compaction/NewestSegmentFirstPolicyTest.java -index c655dc8ae4..1c92c9a249 100644 ---- a/server/src/test/java/org/apache/druid/server/compaction/NewestSegmentFirstPolicyTest.java -+++ b/server/src/test/java/org/apache/druid/server/compaction/NewestSegmentFirstPolicyTest.java -@@ -65,7 +65,6 @@ import org.joda.time.DateTimeZone; - import org.joda.time.Interval; - import org.joda.time.Period; - import org.junit.Assert; --import org.junit.Before; - import org.junit.Test; - - import java.util.ArrayList; -@@ -82,13 +81,6 @@ public class NewestSegmentFirstPolicyTest - private static final int DEFAULT_NUM_SEGMENTS_PER_SHARD = 4; - private final ObjectMapper mapper = new DefaultObjectMapper(); - private final NewestSegmentFirstPolicy policy = new NewestSegmentFirstPolicy(null); -- private CompactionStatusTracker statusTracker; -- -- @Before -- public void setup() -- { -- statusTracker = new CompactionStatusTracker(mapper); -- } - - @Test - public void testLargeOffsetAndSmallSegmentInterval() -@@ -284,8 +276,7 @@ public class NewestSegmentFirstPolicyTest - .withNumPartitions(4) - ) - ), -- Collections.emptyMap(), -- statusTracker -+ Collections.emptyMap() - ); - - assertCompactSegmentIntervals( -@@ -517,8 +508,7 @@ public class NewestSegmentFirstPolicyTest - Intervals.of("2017-11-15T00:00:00/2017-11-15T20:00:00"), - Intervals.of("2017-11-13T00:00:00/2017-11-14T01:00:00") - ) -- ), -- statusTracker -+ ) - ); - - assertCompactSegmentIntervals( -@@ -557,8 +547,7 @@ public class NewestSegmentFirstPolicyTest - Intervals.of("2017-11-16T04:00:00/2017-11-16T10:00:00"), - Intervals.of("2017-11-16T14:00:00/2017-11-16T20:00:00") - ) -- ), -- statusTracker -+ ) - ); - - assertCompactSegmentIntervals( -@@ -1756,7 +1745,7 @@ public class NewestSegmentFirstPolicyTest - null, - new OnheapIncrementalIndex.Spec(true), - null, -- 1000L, -+ null, - null, - partitionsSpec, - IndexSpec.getDefault(), -@@ -1784,7 +1773,7 @@ public class NewestSegmentFirstPolicyTest - null, - new OnheapIncrementalIndex.Spec(false), - null, -- 1000L, -+ null, - null, - partitionsSpec, - IndexSpec.getDefault(), -@@ -2063,8 +2052,7 @@ public class NewestSegmentFirstPolicyTest - TestDataSource.WIKI, SegmentTimeline.forSegments(wikiSegments), - TestDataSource.KOALA, SegmentTimeline.forSegments(koalaSegments) - ), -- Collections.emptyMap(), -- statusTracker -+ Collections.emptyMap() - ); - - // Verify that the segments of WIKI are preferred even though they are older -@@ -2085,8 +2073,7 @@ public class NewestSegmentFirstPolicyTest - policy, - Collections.singletonMap(TestDataSource.WIKI, config), - Collections.singletonMap(TestDataSource.WIKI, timeline), -- Collections.emptyMap(), -- statusTracker -+ Collections.emptyMap() - ); - } - -diff --git a/server/src/test/java/org/apache/druid/server/coordinator/CreateDataSegments.java b/server/src/test/java/org/apache/druid/server/coordinator/CreateDataSegments.java -index 371d251b06..380533c729 100644 ---- a/server/src/test/java/org/apache/druid/server/coordinator/CreateDataSegments.java -+++ b/server/src/test/java/org/apache/druid/server/coordinator/CreateDataSegments.java -@@ -105,6 +105,10 @@ public class CreateDataSegments - return this; - } - -+ /** -+ * Specifies the version to use for creating the segments. Default version is -+ * {@code "1"}. -+ */ - public CreateDataSegments withVersion(String version) - { - this.version = version; -diff --git a/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorTest.java b/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorTest.java -index f77da3140a..39213ff861 100644 ---- a/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorTest.java -+++ b/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorTest.java -@@ -139,8 +139,7 @@ public class DruidCoordinatorTest - ) - ).andReturn(new AtomicReference<>(DruidCompactionConfig.empty())).anyTimes(); - EasyMock.replay(configManager); -- final ObjectMapper objectMapper = new DefaultObjectMapper(); -- statusTracker = new CompactionStatusTracker(objectMapper); -+ statusTracker = new CompactionStatusTracker(); - druidCoordinatorConfig = new DruidCoordinatorConfig( - new CoordinatorRunConfig(new Duration(COORDINATOR_START_DELAY), new Duration(COORDINATOR_PERIOD)), - new CoordinatorPeriodConfig(null, null), -@@ -169,7 +168,7 @@ public class DruidCoordinatorTest - new TestDruidLeaderSelector(), - null, - CentralizedDatasourceSchemaConfig.create(), -- new CompactionStatusTracker(OBJECT_MAPPER), -+ new CompactionStatusTracker(), - EasyMock.niceMock(CoordinatorDynamicConfigSyncer.class), - EasyMock.niceMock(CloneStatusManager.class) - ); -@@ -481,7 +480,7 @@ public class DruidCoordinatorTest - new TestDruidLeaderSelector(), - null, - CentralizedDatasourceSchemaConfig.create(), -- new CompactionStatusTracker(OBJECT_MAPPER), -+ new CompactionStatusTracker(), - EasyMock.niceMock(CoordinatorDynamicConfigSyncer.class), - EasyMock.niceMock(CloneStatusManager.class) - ); -@@ -533,7 +532,7 @@ public class DruidCoordinatorTest - new TestDruidLeaderSelector(), - null, - CentralizedDatasourceSchemaConfig.create(), -- new CompactionStatusTracker(OBJECT_MAPPER), -+ new CompactionStatusTracker(), - EasyMock.niceMock(CoordinatorDynamicConfigSyncer.class), - EasyMock.niceMock(CloneStatusManager.class) - ); -@@ -585,7 +584,7 @@ public class DruidCoordinatorTest - new TestDruidLeaderSelector(), - null, - CentralizedDatasourceSchemaConfig.create(), -- new CompactionStatusTracker(OBJECT_MAPPER), -+ new CompactionStatusTracker(), - EasyMock.niceMock(CoordinatorDynamicConfigSyncer.class), - EasyMock.niceMock(CloneStatusManager.class) - ); -@@ -695,7 +694,7 @@ public class DruidCoordinatorTest - new TestDruidLeaderSelector(), - null, - CentralizedDatasourceSchemaConfig.create(), -- new CompactionStatusTracker(OBJECT_MAPPER), -+ new CompactionStatusTracker(), - EasyMock.niceMock(CoordinatorDynamicConfigSyncer.class), - EasyMock.niceMock(CloneStatusManager.class) - ); -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 0d00228960..301dd77493 100644 ---- a/server/src/test/java/org/apache/druid/server/coordinator/duty/CompactSegmentsTest.java -+++ b/server/src/test/java/org/apache/druid/server/coordinator/duty/CompactSegmentsTest.java -@@ -80,7 +80,9 @@ import org.apache.druid.segment.IndexSpec; - import org.apache.druid.segment.incremental.OnheapIncrementalIndex; - import org.apache.druid.segment.indexing.BatchIOConfig; - import org.apache.druid.segment.transform.CompactionTransformSpec; -+import org.apache.druid.server.compaction.CompactionCandidate; - import org.apache.druid.server.compaction.CompactionCandidateSearchPolicy; -+import org.apache.druid.server.compaction.CompactionSlotManager; - import org.apache.druid.server.compaction.CompactionStatusTracker; - import org.apache.druid.server.compaction.FixedIntervalOrderPolicy; - import org.apache.druid.server.compaction.NewestSegmentFirstPolicy; -@@ -226,7 +228,7 @@ public class CompactSegmentsTest - } - } - dataSources = DataSourcesSnapshot.fromUsedSegments(allSegments); -- statusTracker = new CompactionStatusTracker(JSON_MAPPER); -+ statusTracker = new CompactionStatusTracker(); - policy = new NewestSegmentFirstPolicy(null); - } - -@@ -870,8 +872,8 @@ public class CompactSegmentsTest - // All segments is compact at the same time since we changed the segment granularity to YEAR and all segment - // are within the same year - Assert.assertEquals( -- ClientCompactionIntervalSpec.fromSegments(datasourceToSegments.get(dataSource), Granularities.YEAR), -- taskPayload.getIoConfig().getInputSpec() -+ CompactionCandidate.from(datasourceToSegments.get(dataSource), Granularities.YEAR).getCompactionInterval(), -+ taskPayload.getIoConfig().getInputSpec().getInterval() - ); - - ClientCompactionTaskGranularitySpec expectedGranularitySpec = -@@ -1061,8 +1063,8 @@ public class CompactSegmentsTest - // All segments is compact at the same time since we changed the segment granularity to YEAR and all segment - // are within the same year - Assert.assertEquals( -- ClientCompactionIntervalSpec.fromSegments(datasourceToSegments.get(dataSource), Granularities.YEAR), -- taskPayload.getIoConfig().getInputSpec() -+ CompactionCandidate.from(datasourceToSegments.get(dataSource), Granularities.YEAR).getCompactionInterval(), -+ taskPayload.getIoConfig().getInputSpec().getInterval() - ); - - ClientCompactionTaskGranularitySpec expectedGranularitySpec = -@@ -1156,8 +1158,8 @@ public class CompactSegmentsTest - // All segments is compact at the same time since we changed the segment granularity to YEAR and all segment - // are within the same year - Assert.assertEquals( -- ClientCompactionIntervalSpec.fromSegments(datasourceToSegments.get(dataSource), Granularities.YEAR), -- taskPayload.getIoConfig().getInputSpec() -+ CompactionCandidate.from(datasourceToSegments.get(dataSource), Granularities.YEAR).getCompactionInterval(), -+ taskPayload.getIoConfig().getInputSpec().getInterval() - ); - - ClientCompactionTaskGranularitySpec expectedGranularitySpec = -@@ -1397,8 +1399,8 @@ public class CompactSegmentsTest - ClientCompactionTaskQuery taskPayload = (ClientCompactionTaskQuery) payloadCaptor.getValue(); - - Assert.assertEquals( -- ClientCompactionIntervalSpec.fromSegments(segments, Granularities.DAY), -- taskPayload.getIoConfig().getInputSpec() -+ CompactionCandidate.from(segments, Granularities.DAY).getCompactionInterval(), -+ taskPayload.getIoConfig().getInputSpec().getInterval() - ); - - ClientCompactionTaskGranularitySpec expectedGranularitySpec = -@@ -1460,8 +1462,8 @@ public class CompactSegmentsTest - ClientCompactionTaskQuery taskPayload = (ClientCompactionTaskQuery) payloadCaptor.getValue(); - - Assert.assertEquals( -- ClientCompactionIntervalSpec.fromSegments(segments, Granularities.YEAR), -- taskPayload.getIoConfig().getInputSpec() -+ CompactionCandidate.from(segments, Granularities.YEAR).getCompactionInterval(), -+ taskPayload.getIoConfig().getInputSpec().getInterval() - ); - - ClientCompactionTaskGranularitySpec expectedGranularitySpec = -@@ -2004,7 +2006,7 @@ public class CompactSegmentsTest - @Test - public void testIsParalleModeNullTuningConfigReturnFalse() - { -- Assert.assertFalse(CompactSegments.isParallelMode(null)); -+ Assert.assertFalse(CompactionSlotManager.isParallelMode(null)); - } - - @Test -@@ -2012,7 +2014,7 @@ public class CompactSegmentsTest - { - ClientCompactionTaskQueryTuningConfig tuningConfig = Mockito.mock(ClientCompactionTaskQueryTuningConfig.class); - Mockito.when(tuningConfig.getPartitionsSpec()).thenReturn(null); -- Assert.assertFalse(CompactSegments.isParallelMode(tuningConfig)); -+ Assert.assertFalse(CompactionSlotManager.isParallelMode(tuningConfig)); - } - - @Test -@@ -2022,13 +2024,13 @@ public class CompactSegmentsTest - Mockito.when(tuningConfig.getPartitionsSpec()).thenReturn(Mockito.mock(PartitionsSpec.class)); - - Mockito.when(tuningConfig.getMaxNumConcurrentSubTasks()).thenReturn(null); -- Assert.assertFalse(CompactSegments.isParallelMode(tuningConfig)); -+ Assert.assertFalse(CompactionSlotManager.isParallelMode(tuningConfig)); - - Mockito.when(tuningConfig.getMaxNumConcurrentSubTasks()).thenReturn(1); -- Assert.assertFalse(CompactSegments.isParallelMode(tuningConfig)); -+ Assert.assertFalse(CompactionSlotManager.isParallelMode(tuningConfig)); - - Mockito.when(tuningConfig.getMaxNumConcurrentSubTasks()).thenReturn(2); -- Assert.assertTrue(CompactSegments.isParallelMode(tuningConfig)); -+ Assert.assertTrue(CompactionSlotManager.isParallelMode(tuningConfig)); - } - - @Test -@@ -2038,13 +2040,13 @@ public class CompactSegmentsTest - Mockito.when(tuningConfig.getPartitionsSpec()).thenReturn(Mockito.mock(SingleDimensionPartitionsSpec.class)); - - Mockito.when(tuningConfig.getMaxNumConcurrentSubTasks()).thenReturn(null); -- Assert.assertFalse(CompactSegments.isParallelMode(tuningConfig)); -+ Assert.assertFalse(CompactionSlotManager.isParallelMode(tuningConfig)); - - Mockito.when(tuningConfig.getMaxNumConcurrentSubTasks()).thenReturn(1); -- Assert.assertTrue(CompactSegments.isParallelMode(tuningConfig)); -+ Assert.assertTrue(CompactionSlotManager.isParallelMode(tuningConfig)); - - Mockito.when(tuningConfig.getMaxNumConcurrentSubTasks()).thenReturn(2); -- Assert.assertTrue(CompactSegments.isParallelMode(tuningConfig)); -+ Assert.assertTrue(CompactionSlotManager.isParallelMode(tuningConfig)); - } - - @Test -@@ -2053,7 +2055,7 @@ public class CompactSegmentsTest - ClientCompactionTaskQueryTuningConfig tuningConfig = Mockito.mock(ClientCompactionTaskQueryTuningConfig.class); - Mockito.when(tuningConfig.getPartitionsSpec()).thenReturn(Mockito.mock(PartitionsSpec.class)); - Mockito.when(tuningConfig.getMaxNumConcurrentSubTasks()).thenReturn(2); -- Assert.assertEquals(3, CompactSegments.findMaxNumTaskSlotsUsedByOneNativeCompactionTask(tuningConfig)); -+ Assert.assertEquals(3, CompactionSlotManager.getMaxTaskSlotsForNativeCompactionTask(tuningConfig)); - } - - @Test -@@ -2062,7 +2064,7 @@ public class CompactSegmentsTest - ClientCompactionTaskQueryTuningConfig tuningConfig = Mockito.mock(ClientCompactionTaskQueryTuningConfig.class); - Mockito.when(tuningConfig.getPartitionsSpec()).thenReturn(Mockito.mock(PartitionsSpec.class)); - Mockito.when(tuningConfig.getMaxNumConcurrentSubTasks()).thenReturn(1); -- Assert.assertEquals(1, CompactSegments.findMaxNumTaskSlotsUsedByOneNativeCompactionTask(tuningConfig)); -+ Assert.assertEquals(1, CompactionSlotManager.getMaxTaskSlotsForNativeCompactionTask(tuningConfig)); - } - } - -diff --git a/server/src/test/java/org/apache/druid/server/coordinator/simulate/CoordinatorSimulationBuilder.java b/server/src/test/java/org/apache/druid/server/coordinator/simulate/CoordinatorSimulationBuilder.java -index 8d63492375..0838b41402 100644 ---- a/server/src/test/java/org/apache/druid/server/coordinator/simulate/CoordinatorSimulationBuilder.java -+++ b/server/src/test/java/org/apache/druid/server/coordinator/simulate/CoordinatorSimulationBuilder.java -@@ -223,7 +223,7 @@ public class CoordinatorSimulationBuilder - env.leaderSelector, - null, - CentralizedDatasourceSchemaConfig.create(), -- new CompactionStatusTracker(OBJECT_MAPPER), -+ new CompactionStatusTracker(), - env.configSyncer, - env.cloneStatusManager - ); -diff --git a/server/src/test/java/org/apache/druid/server/metrics/LatchableEmitter.java b/server/src/test/java/org/apache/druid/server/metrics/LatchableEmitter.java -index 247b74d8e5..5d8b110e17 100644 ---- a/server/src/test/java/org/apache/druid/server/metrics/LatchableEmitter.java -+++ b/server/src/test/java/org/apache/druid/server/metrics/LatchableEmitter.java -@@ -24,6 +24,8 @@ import org.apache.druid.java.util.common.logger.Logger; - import org.apache.druid.java.util.emitter.core.Event; - import org.apache.druid.java.util.emitter.service.ServiceMetricEvent; - import org.apache.druid.java.util.metrics.StubServiceEmitter; -+import org.hamcrest.Matcher; -+import org.hamcrest.Matchers; - import org.junit.jupiter.api.Timeout; - - import java.util.ArrayList; -@@ -245,8 +247,8 @@ public class LatchableEmitter extends StubServiceEmitter - private String host; - private String service; - private String metricName; -- private Long metricValue; -- private final Map dimensions = new HashMap<>(); -+ private Matcher valueMatcher; -+ private final Map> dimensionMatchers = new HashMap<>(); - - private final AtomicReference matchingEvent = new AtomicReference<>(); - -@@ -260,12 +262,11 @@ public class LatchableEmitter extends StubServiceEmitter - } - - /** -- * Matches an event only if it has a metric value equal to or greater than -- * the given value. -+ * Matches an event only if the metric value satisfies the given matcher. - */ -- public EventMatcher hasValueAtLeast(long metricValue) -+ public EventMatcher hasValueMatching(Matcher valueMatcher) - { -- this.metricValue = metricValue; -+ this.valueMatcher = valueMatcher; - return this; - } - -@@ -274,7 +275,16 @@ public class LatchableEmitter extends StubServiceEmitter - */ - public EventMatcher hasDimension(String dimension, Object value) - { -- dimensions.put(dimension, value); -+ dimensionMatchers.put(dimension, Matchers.equalTo(value)); -+ return this; -+ } -+ -+ /** -+ * Matches an event if the value of the given dimension satisfies the matcher. -+ */ -+ public EventMatcher hasDimensionMatching(String dimension, Matcher matcher) -+ { -+ dimensionMatchers.put(dimension, matcher); - return this; - } - -@@ -301,7 +311,7 @@ public class LatchableEmitter extends StubServiceEmitter - { - if (metricName != null && !event.getMetric().equals(metricName)) { - return false; -- } else if (metricValue != null && event.getValue().longValue() < metricValue) { -+ } else if (valueMatcher != null && !valueMatcher.matches(event.getValue())) { - return false; - } else if (service != null && !service.equals(event.getService())) { - return false; -@@ -309,10 +319,8 @@ public class LatchableEmitter extends StubServiceEmitter - return false; - } - -- final boolean matches = dimensions.entrySet().stream().allMatch( -- dimValue -> event.getUserDims() -- .getOrDefault(dimValue.getKey(), "") -- .equals(dimValue.getValue()) -+ final boolean matches = dimensionMatchers.entrySet().stream().allMatch( -+ dimValue -> dimValue.getValue().matches(event.getUserDims().get(dimValue.getKey())) - ); - - if (matches) { -diff --git a/services/src/main/java/org/apache/druid/cli/CliOverlord.java b/services/src/main/java/org/apache/druid/cli/CliOverlord.java -index 81a0c13671..63a1d28bfc 100644 ---- a/services/src/main/java/org/apache/druid/cli/CliOverlord.java -+++ b/services/src/main/java/org/apache/druid/cli/CliOverlord.java -@@ -70,6 +70,7 @@ import org.apache.druid.indexing.common.task.batch.parallel.ShuffleClient; - import org.apache.druid.indexing.common.tasklogs.SwitchingTaskLogStreamer; - import org.apache.druid.indexing.common.tasklogs.TaskRunnerTaskLogStreamer; - import org.apache.druid.indexing.compact.CompactionScheduler; -+import org.apache.druid.indexing.compact.DruidInputSourceFactory; - import org.apache.druid.indexing.compact.OverlordCompactionScheduler; - import org.apache.druid.indexing.overlord.DruidOverlord; - import org.apache.druid.indexing.overlord.ForkingTaskRunnerFactory; -@@ -247,6 +248,7 @@ public class CliOverlord extends ServerRunnable - binder.bind(TaskQueryTool.class).in(LazySingleton.class); - binder.bind(IndexerMetadataStorageAdapter.class).in(LazySingleton.class); - binder.bind(CompactionScheduler.class).to(OverlordCompactionScheduler.class).in(ManageLifecycle.class); -+ binder.bind(DruidInputSourceFactory.class).in(LazySingleton.class); - binder.bind(ScheduledBatchTaskManager.class).in(LazySingleton.class); - binder.bind(SupervisorManager.class).in(LazySingleton.class); - -diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/schema/DruidCalciteSchemaModuleTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/schema/DruidCalciteSchemaModuleTest.java -index 2333561540..eb0ab607e8 100644 ---- a/sql/src/test/java/org/apache/druid/sql/calcite/schema/DruidCalciteSchemaModuleTest.java -+++ b/sql/src/test/java/org/apache/druid/sql/calcite/schema/DruidCalciteSchemaModuleTest.java -@@ -28,6 +28,8 @@ import com.google.inject.Key; - import com.google.inject.Scopes; - import com.google.inject.TypeLiteral; - import com.google.inject.name.Names; -+import org.apache.druid.catalog.MapMetadataCatalog; -+import org.apache.druid.catalog.MetadataCatalog; - import org.apache.druid.client.FilteredServerInventoryView; - import org.apache.druid.client.TimelineServerView; - import org.apache.druid.client.coordinator.CoordinatorClient; -@@ -56,7 +58,7 @@ import org.apache.druid.sql.calcite.view.ViewManager; - import org.easymock.EasyMock; - import org.easymock.EasyMockExtension; - import org.easymock.Mock; --import org.junit.Assert; -+import org.junit.jupiter.api.Assertions; - import org.junit.jupiter.api.BeforeEach; - import org.junit.jupiter.api.Test; - import org.junit.jupiter.api.extension.ExtendWith; -@@ -120,6 +122,7 @@ public class DruidCalciteSchemaModuleTest extends CalciteTestBase - binder.bindScope(LazySingleton.class, Scopes.SINGLETON); - binder.bind(LookupExtractorFactoryContainerProvider.class).toInstance(lookupReferencesManager); - binder.bind(CatalogResolver.class).toInstance(CatalogResolver.NULL_RESOLVER); -+ binder.bind(MetadataCatalog.class).toInstance(new MapMetadataCatalog(objectMapper)); - binder.bind(ServiceEmitter.class).toInstance(new ServiceEmitter("", "", null)); - binder.bind(OverlordClient.class).to(NoopOverlordClient.class); - binder.bind(CoordinatorClient.class).to(NoopCoordinatorClient.class); -@@ -134,35 +137,39 @@ public class DruidCalciteSchemaModuleTest extends CalciteTestBase - public void testDruidSchemaNameIsInjected() - { - String schemaName = injector.getInstance(Key.get(String.class, DruidSchemaName.class)); -- Assert.assertEquals(DRUID_SCHEMA_NAME, schemaName); -+ Assertions.assertEquals(DRUID_SCHEMA_NAME, schemaName); - } - - @Test - public void testDruidSqlSchemaIsInjectedAsSingleton() - { - NamedDruidSchema namedDruidSchema = injector.getInstance(NamedDruidSchema.class); -- Assert.assertNotNull(namedDruidSchema); -+ Assertions.assertNotNull(namedDruidSchema); - NamedDruidSchema other = injector.getInstance(NamedDruidSchema.class); -- Assert.assertSame(other, namedDruidSchema); -+ Assertions.assertSame(other, namedDruidSchema); - } - - @Test - public void testSystemSqlSchemaIsInjectedAsSingleton() - { - NamedSystemSchema namedSystemSchema = injector.getInstance(NamedSystemSchema.class); -- Assert.assertNotNull(namedSystemSchema); -+ Assertions.assertNotNull(namedSystemSchema); - NamedSystemSchema other = injector.getInstance(NamedSystemSchema.class); -- Assert.assertSame(other, namedSystemSchema); -+ Assertions.assertSame(other, namedSystemSchema); - } - - @Test - public void testDruidCalciteSchemasAreInjected() - { - Set sqlSchemas = injector.getInstance(Key.get(new TypeLiteral<>() {})); -- Set> expectedSchemas = -- ImmutableSet.of(NamedSystemSchema.class, NamedDruidSchema.class, NamedLookupSchema.class, NamedViewSchema.class); -- Assert.assertEquals(expectedSchemas.size(), sqlSchemas.size()); -- Assert.assertEquals( -+ Set> expectedSchemas = Set.of( -+ NamedSystemSchema.class, -+ NamedDruidSchema.class, -+ NamedLookupSchema.class, -+ NamedViewSchema.class -+ ); -+ Assertions.assertEquals(expectedSchemas.size(), sqlSchemas.size()); -+ Assertions.assertEquals( - expectedSchemas, - sqlSchemas.stream().map(NamedSchema::getClass).collect(Collectors.toSet())); - } -@@ -171,36 +178,36 @@ public class DruidCalciteSchemaModuleTest extends CalciteTestBase - public void testDruidSchemaIsInjectedAsSingleton() - { - DruidSchema schema = injector.getInstance(DruidSchema.class); -- Assert.assertNotNull(schema); -+ Assertions.assertNotNull(schema); - DruidSchema other = injector.getInstance(DruidSchema.class); -- Assert.assertSame(other, schema); -+ Assertions.assertSame(other, schema); - } - - @Test - public void testSystemSchemaIsInjectedAsSingleton() - { - SystemSchema schema = injector.getInstance(SystemSchema.class); -- Assert.assertNotNull(schema); -+ Assertions.assertNotNull(schema); - SystemSchema other = injector.getInstance(SystemSchema.class); -- Assert.assertSame(other, schema); -+ Assertions.assertSame(other, schema); - } - - @Test - public void testInformationSchemaIsInjectedAsSingleton() - { - InformationSchema schema = injector.getInstance(InformationSchema.class); -- Assert.assertNotNull(schema); -+ Assertions.assertNotNull(schema); - InformationSchema other = injector.getInstance(InformationSchema.class); -- Assert.assertSame(other, schema); -+ Assertions.assertSame(other, schema); - } - - @Test - public void testLookupSchemaIsInjectedAsSingleton() - { - LookupSchema schema = injector.getInstance(LookupSchema.class); -- Assert.assertNotNull(schema); -+ Assertions.assertNotNull(schema); - LookupSchema other = injector.getInstance(LookupSchema.class); -- Assert.assertSame(other, schema); -+ Assertions.assertSame(other, schema); - } - - @Test -@@ -209,22 +216,22 @@ public class DruidCalciteSchemaModuleTest extends CalciteTestBase - DruidSchemaCatalog rootSchema = injector.getInstance( - Key.get(DruidSchemaCatalog.class, Names.named(DruidCalciteSchemaModule.INCOMPLETE_SCHEMA)) - ); -- Assert.assertNotNull(rootSchema); -+ Assertions.assertNotNull(rootSchema); - DruidSchemaCatalog other = injector.getInstance( - Key.get(DruidSchemaCatalog.class, Names.named(DruidCalciteSchemaModule.INCOMPLETE_SCHEMA)) - ); -- Assert.assertSame(other, rootSchema); -+ Assertions.assertSame(other, rootSchema); - } - - @Test - public void testRootSchemaIsInjectedAsSingleton() - { - DruidSchemaCatalog rootSchema = injector.getInstance(Key.get(DruidSchemaCatalog.class)); -- Assert.assertNotNull(rootSchema); -+ Assertions.assertNotNull(rootSchema); - DruidSchemaCatalog other = injector.getInstance( - Key.get(DruidSchemaCatalog.class, Names.named(DruidCalciteSchemaModule.INCOMPLETE_SCHEMA)) - ); -- Assert.assertSame(other, rootSchema); -+ Assertions.assertSame(other, rootSchema); - } - - @Test -@@ -232,7 +239,7 @@ public class DruidCalciteSchemaModuleTest extends CalciteTestBase - { - DruidSchemaCatalog rootSchema = injector.getInstance(Key.get(DruidSchemaCatalog.class)); - InformationSchema expectedSchema = injector.getInstance(InformationSchema.class); -- Assert.assertNotNull(rootSchema); -- Assert.assertSame(expectedSchema, rootSchema.getSubSchema("INFORMATION_SCHEMA").unwrap(InformationSchema.class)); -+ Assertions.assertNotNull(rootSchema); -+ Assertions.assertSame(expectedSchema, rootSchema.getSubSchema("INFORMATION_SCHEMA").unwrap(InformationSchema.class)); - } - } -diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/util/SqlTestFramework.java b/sql/src/test/java/org/apache/druid/sql/calcite/util/SqlTestFramework.java -index 7a7cf16459..38ddba751b 100644 ---- a/sql/src/test/java/org/apache/druid/sql/calcite/util/SqlTestFramework.java -+++ b/sql/src/test/java/org/apache/druid/sql/calcite/util/SqlTestFramework.java -@@ -35,6 +35,7 @@ import org.apache.druid.client.cache.CacheConfig; - import org.apache.druid.collections.BlockingPool; - import org.apache.druid.collections.NonBlockingPool; - import org.apache.druid.guice.BuiltInTypesModule; -+import org.apache.druid.guice.CatalogCoreModule; - import org.apache.druid.guice.DruidInjectorBuilder; - import org.apache.druid.guice.ExpressionModule; - import org.apache.druid.guice.LazySingleton; -@@ -409,6 +410,7 @@ public class SqlTestFramework - new LookylooModule(), - new SegmentWranglerModule(), - new ExpressionModule(), -+ new CatalogCoreModule(), - DruidModule.override( - new QueryRunnerFactoryModule(), - new Module() -diff --git a/sql/src/test/java/org/apache/druid/sql/guice/SqlModuleTest.java b/sql/src/test/java/org/apache/druid/sql/guice/SqlModuleTest.java -index 28e8923f63..f0057eb9c5 100644 ---- a/sql/src/test/java/org/apache/druid/sql/guice/SqlModuleTest.java -+++ b/sql/src/test/java/org/apache/druid/sql/guice/SqlModuleTest.java -@@ -32,6 +32,7 @@ import org.apache.druid.client.TimelineServerView; - import org.apache.druid.client.coordinator.CoordinatorClient; - import org.apache.druid.client.coordinator.NoopCoordinatorClient; - import org.apache.druid.discovery.DruidNodeDiscoveryProvider; -+import org.apache.druid.guice.CatalogCoreModule; - import org.apache.druid.guice.DruidGuiceExtensions; - import org.apache.druid.guice.JsonConfigurator; - import org.apache.druid.guice.LazySingleton; -@@ -182,6 +183,7 @@ public class SqlModuleTest - new JacksonModule(), - new PolicyModule(), - new AuthenticatorMapperModule(), -+ new CatalogCoreModule(), - binder -> { - binder.bind(Validator.class).toInstance(Validation.buildDefaultValidatorFactory().getValidator()); - binder.bind(JsonConfigurator.class).in(LazySingleton.class);