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/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..e11563c03897 --- /dev/null +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionSupervisorTest.java @@ -0,0 +1,195 @@ +/* + * 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.CompactionSupervisorSpec; +import org.apache.druid.indexing.overlord.Segments; +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.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.Period; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; + +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)); + } + + 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); + } +} 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 44c9be9a9c85..11e25120aa11 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 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/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..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 @@ -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) + .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 3227fe6216b8..2c36306634aa 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 9f3106b88441..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 @@ -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; @@ -52,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() @@ -119,7 +121,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/indexing/common/task/CompactionTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java index 1c65264f6b6d..75b148d6e541 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/CompactionConfigBasedJobTemplate.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionConfigBasedJobTemplate.java new file mode 100644 index 000000000000..fd89714dadb6 --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionConfigBasedJobTemplate.java @@ -0,0 +1,139 @@ +/* + * 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.duty.CompactSegments; +import org.apache.druid.timeline.SegmentTimeline; +import org.joda.time.Interval; + +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; + } + + @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 000000000000..7a7e7fdc1eab --- /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 000000000000..0113f1b78bac --- /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 000000000000..77886af1a017 --- /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 000000000000..cd5096a8cf8e --- /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 000000000000..10099566ea71 --- /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 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..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,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.server.coordinator.AutoCompactionSnapshot; import javax.annotation.Nullable; +import java.util.List; /** * Supervisor for compaction of a single datasource. @@ -51,6 +53,31 @@ public CompactionSupervisor( 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 void start() ); } 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 ca6008cc9d79..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 @@ -93,6 +93,14 @@ public CompactionSupervisor createSupervisor() return new CompactionSupervisor(this, scheduler); } + /** + * @return {@link CompactionJobTemplate} used to create jobs for the supervisor. + */ + public CompactionJobTemplate getTemplate() + { + 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 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/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..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 @@ -24,15 +24,18 @@ 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.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.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 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.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 void locationChanged(String taskId, TaskLocation newLocation) public void statusChanged(String taskId, TaskStatus status) { if (status.isComplete()) { - statusTracker.onTaskFinished(taskId, status); + onTaskFinished(taskId, status); + launchPendingJobs(); } } }; @@ -184,7 +221,8 @@ public synchronized void stop() 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 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 + // 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 @@ 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, Execs.directExecutor()); + taskRunnerOptional.get().registerListener(taskRunnerListener, executor); } if (shouldPollSegments) { segmentManager.startPollingDatabasePeriodically(); @@ -264,7 +308,8 @@ private synchronized void cleanupState() taskRunnerOptional.get().unregisterListener(taskRunnerListener.getListenerId()); } statusTracker.stop(); - activeDatasourceConfigs.clear(); + activeSupervisors.clear(); + latestJobQueue.set(null); if (shouldPollSegments) { segmentManager.stopPollingDatabasePeriodically(); @@ -291,54 +336,119 @@ private synchronized void scheduledRun() 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 AutoCompactionSnapshot getCompactionSnapshot(String dataSource) @Override public Map getAllCompactionSnapshots() { - return duty.getAutoCompactionSnapshot(); + return Map.copyOf(datasourceToCompactionSnapshot.get()); } @Override @@ -363,17 +473,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 +495,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() @@ -392,7 +516,7 @@ private DataSourcesSnapshot getDatasourceSnapshot() return segmentManager.getRecentDataSourcesSnapshot(); } - private void scheduleOnExecutor(Runnable runnable, long delaySeconds) + private void scheduleOnExecutor(Runnable runnable, long delayMillis) { executor.schedule( () -> { @@ -403,8 +527,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/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/TaskQueue.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskQueue.java index 0cfe481a8fea..c923e70fcadb 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 Optional getActiveTaskInfo(String taskId) } /** + * 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 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..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 @@ -21,15 +21,31 @@ 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,10 +54,15 @@ 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.common.granularity.Granularity; import org.apache.druid.java.util.metrics.StubServiceEmitter; -import org.apache.druid.segment.TestDataSource; +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.TestIndex; import org.apache.druid.server.compaction.CompactionSimulateResult; import org.apache.druid.server.compaction.CompactionStatistics; @@ -57,10 +78,11 @@ import org.apache.druid.server.coordinator.DruidCompactionConfig; import org.apache.druid.server.coordinator.InlineSchemaDataSourceCompactionConfig; 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.Interval; import org.joda.time.Period; import org.junit.Assert; import org.junit.Before; @@ -72,6 +94,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 @@ -81,6 +105,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 +116,44 @@ public class OverlordCompactionSchedulerTest ); } + private static final DateTime JAN_20 = DateTimes.of("2025-01-20"); + 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; @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); + brokerClient = Mockito.mock(BrokerClient.class); + Mockito.when(brokerClient.submitSqlTask(ArgumentMatchers.any(ClientSqlQuery.class))).thenAnswer( + arg -> { + final String taskId = IdUtils.getRandomId(); + return Futures.immediateFuture(new SqlTaskStatus(taskId, TaskState.RUNNING, null)); + } + ); + taskMaster = new TaskMaster(null, null); Assert.assertFalse(taskMaster.isHalfOrFullLeader()); Assert.assertFalse(taskMaster.isFullLeader()); @@ -126,11 +170,39 @@ 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, 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 +214,29 @@ private void initScheduler() = 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 +250,7 @@ public void testBecomeLeader_triggersStart_ifEnabled() } @Test - public void testBecomeLeader_doesNotTriggerStart_ifDisabled() + public void test_becomeLeader_doesNotTriggerStart_ifDisabled() { disableScheduler(); Assert.assertFalse(scheduler.isEnabled()); @@ -182,7 +264,7 @@ public void testBecomeLeader_doesNotTriggerStart_ifDisabled() } @Test - public void testStopBeingLeader_triggersStop() + public void test_stopBeingLeader_triggersStop() { Assert.assertFalse(scheduler.isRunning()); @@ -198,7 +280,7 @@ public void testStopBeingLeader_triggersStop() } @Test - public void testDisablingScheduler_triggersStop() + public void test_disableSupervisors_triggersStop() { // Start scheduler scheduler.becomeLeader(); @@ -216,7 +298,7 @@ public void testDisablingScheduler_triggersStop() } @Test - public void testEnablingScheduler_triggersStart() + public void test_enableSupervisors_triggersStart() { disableScheduler(); @@ -235,7 +317,7 @@ public void testEnablingScheduler_triggersStart() } @Test - public void testSegmentsAreNotPolled_ifSupervisorsAreDisabled() + public void test_disableSupervisors_disablesSegmentPolling() { disableScheduler(); @@ -243,7 +325,7 @@ public void testSegmentsAreNotPolled_ifSupervisorsAreDisabled() } @Test - public void testSegmentsArePolled_whenRunningInStandaloneMode() + public void test_enableSupervisors_inStandaloneMode_enablesSegmentPolling() { coordinatorOverlordServiceConfig = new CoordinatorOverlordServiceConfig(false, null); initScheduler(); @@ -252,7 +334,7 @@ public void testSegmentsArePolled_whenRunningInStandaloneMode() } @Test - public void testSegmentsAreNotPolled_whenRunningInCoordinatorMode() + public void test_enableSupervisors_inCoordinatorMode_disablesSegmentPolling() { coordinatorOverlordServiceConfig = new CoordinatorOverlordServiceConfig(true, "overlord"); initScheduler(); @@ -272,7 +354,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()); @@ -280,11 +362,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(); @@ -298,75 +380,50 @@ 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, - InlineSchemaDataSourceCompactionConfig.builder() - .forDataSource(TestDataSource.WIKI) - .withSkipOffsetFromLatest(Period.seconds(0)) - .build() - ); + 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); - 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 +434,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, - 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 +455,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" ) @@ -421,7 +466,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) @@ -431,6 +476,64 @@ public void testSimulateRun() scheduler.stopBeingLeader(); } + 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 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 disableScheduler() { compactionConfig.set(new ClusterCompactionConfig(null, null, null, false, null)); @@ -446,4 +549,16 @@ private void runScheduledJob() 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 e7ecc49df8ba..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,9 +27,11 @@ 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 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/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 0960f63b4979..ce34fc4798d5 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.ISODateTimeFormat; import javax.annotation.Nullable; +import java.util.List; public final class Intervals { @@ -167,6 +168,26 @@ 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 (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 3b3a2a239806..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,7 +109,6 @@ public List getParameters() 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 3d591a319f71..de0a27bcb59f 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 { @@ -124,4 +125,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/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/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..e8a09a6bb152 --- /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 4cd9b22df812..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 @@ -21,12 +21,14 @@ 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 Interval getUmbrellaInterval() return umbrellaInterval; } + /** + * Interval aligned to the target segment granularity used for the compaction + * task. This interval completely contains the {@link #umbrellaInterval}. + */ + public Interval getCompactionInterval() + { + return compactionInterval; + } + public String getDataSource() { return dataSource; @@ -115,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 646319eec3b2..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 @@ -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()) ); } }; @@ -171,12 +169,10 @@ 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()); - 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..428367db8ea0 --- /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 000000000000..d186587cb870 --- /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 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/CompactionStatus.java b/server/src/main/java/org/apache/druid/server/compaction/CompactionStatus.java index 3c1616b6e785..be4acd00e21c 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 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,12 +114,17 @@ 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)); } - 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 @@ 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), @@ -187,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, message); + } + + public static CompactionStatus complete(String message) { - return new CompactionStatus(State.RUNNING, reasonForCompaction); + return new CompactionStatus(State.COMPLETE, message); } /** @@ -204,23 +213,30 @@ 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); } + @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 @@ static DimensionRangePartitionsSpec getEffectiveRangePartitionsSpec(DimensionRan */ 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 @@ private Evaluator( if (lastCompactionState == null) { this.existingGranularitySpec = null; } else { - this.existingGranularitySpec = convertIfNotNull( - lastCompactionState.getGranularitySpec(), - ClientCompactionTaskGranularitySpec.class + this.existingGranularitySpec = UserCompactionTaskGranularityConfig.from( + lastCompactionState.getGranularitySpec() ); } } @@ -309,7 +321,7 @@ private Evaluator( 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 @@ 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"); } } @@ -337,7 +349,7 @@ private CompactionStatus partitionsSpecIsUpToDate() existingPartionsSpec.getMaxRowsPerSegment(), ((DynamicPartitionsSpec) existingPartionsSpec).getMaxTotalRowsOr(Long.MAX_VALUE)); } - return CompactionStatus.completeIfEqual( + return CompactionStatus.completeIfNullOrEqual( "partitionsSpec", findPartitionsSpecFromConfig(tuningConfig), existingPartionsSpec, @@ -347,17 +359,17 @@ private CompactionStatus partitionsSpecIsUpToDate() 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 @@ 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 @@ -385,7 +410,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) ); @@ -407,7 +432,7 @@ private CompactionStatus rollupIsUpToDate() if (configuredGranularitySpec == null) { return COMPLETE; } else { - return CompactionStatus.completeIfEqual( + return CompactionStatus.completeIfNullOrEqual( "rollup", configuredGranularitySpec.isRollup(), existingGranularitySpec == null ? null : existingGranularitySpec.isRollup(), @@ -421,7 +446,7 @@ private CompactionStatus queryGranularityIsUpToDate() if (configuredGranularitySpec == null) { return COMPLETE; } else { - return CompactionStatus.completeIfEqual( + return CompactionStatus.completeIfNullOrEqual( "queryGranularity", configuredGranularitySpec.getQueryGranularity(), existingGranularitySpec == null ? null : existingGranularitySpec.getQueryGranularity(), @@ -454,7 +479,7 @@ private CompactionStatus dimensionsSpecIsUpToDate() ? IndexSpec.getDefault() : compactionConfig.getTuningConfig().getIndexSpec() ); - return CompactionStatus.completeIfEqual( + return CompactionStatus.completeIfNullOrEqual( "dimensionsSpec", configuredDimensions, existingDimensions, @@ -493,26 +518,13 @@ private CompactionStatus transformSpecFilterIsUpToDate() 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 cbf5f25f9d7b..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 @@ -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 CompactionTaskStatus getLatestTaskStatus(CompactionCandidate candidates) return datasourceStatuses .getOrDefault(candidates.getDataSource(), DatasourceStatus.EMPTY) .intervalToTaskStatus - .get(candidates.getUmbrellaInterval()); + .get(candidates.getCompactionInterval()); } /** @@ -86,30 +75,20 @@ 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 - ); - } - // 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 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" ); } // 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 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 +146,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); } @@ -186,7 +167,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); } @@ -229,7 +210,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 e15d310b33db..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 @@ -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); @@ -117,11 +112,12 @@ 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); - statusTracker.onCompactionStatusComputed(candidatesWithStatus, config); return; } @@ -315,11 +311,9 @@ private void findAndEnqueueSegmentsToCompact(CompactibleSegmentIterator compacti 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 @@ 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"); @@ -371,7 +365,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/AutoCompactionSnapshot.java b/server/src/main/java/org/apache/druid/server/coordinator/AutoCompactionSnapshot.java index e31a7919f24f..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; @@ -241,6 +260,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); 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..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,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 @@ * 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 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) { 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 135852c1de9c..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 @@ -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,21 +72,16 @@ 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 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 +177,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,334 +214,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(); - // ^ 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 @@ 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 @@ -675,7 +415,7 @@ public Map getAutoCompactionSnapshot() return autoCompactionSnapshotPerDataSource.get(); } - private String compactSegments( + private static ClientCompactionTaskQuery compactSegments( CompactionCandidate entry, int compactionTaskPriority, ClientCompactionTaskQueryTuningConfig tuningConfig, @@ -685,7 +425,7 @@ private String compactSegments( @Nullable CompactionTransformSpec transformSpec, @Nullable List projectionSpecs, @Nullable Boolean dropExisting, - @Nullable Map context, + Map context, ClientCompactionRunnerInfo compactionRunner ) { @@ -698,17 +438,15 @@ private String 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(); - 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 @@ 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..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 @@ -110,8 +110,17 @@ 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 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 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/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 14af1c78da90..d201b84135dc 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.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.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 @@ 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 void testFindPartitionsSpecWhenGivenIsNull() { final ClientCompactionTaskQueryTuningConfig tuningConfig = ClientCompactionTaskQueryTuningConfig.from(null); - Assert.assertEquals( - new DynamicPartitionsSpec(null, Long.MAX_VALUE), + Assert.assertNull( CompactionStatus.findPartitionsSpecFromConfig(tuningConfig) ); } @@ -203,9 +196,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]" ); } @@ -213,12 +211,16 @@ public void testStatusWhenLastCompactionStateIsEmpty() @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 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 + CompactionCandidate.from(List.of(segment), Granularities.HOUR), + compactionConfig ); Assert.assertTrue(status.isComplete()); } @@ -373,9 +374,8 @@ 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 + CompactionCandidate.from(List.of(segment), Granularities.HOUR), + compactionConfig ); Assert.assertTrue(status.isComplete()); } @@ -428,9 +428,8 @@ 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 + CompactionCandidate.from(List.of(segment), Granularities.HOUR), + compactionConfig ); Assert.assertFalse(status.isComplete()); } @@ -482,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()); } @@ -536,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()); } @@ -554,9 +551,8 @@ private void verifyCompactionStatusIsPendingBecause( .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 1878aab8c0c3..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 @@ -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 void testGetLatestTaskStatusForSubmittedTask() 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 void testGetLatestTaskStatusForSuccessfulTask() 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 void testGetLatestTaskStatusForFailedTask() 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 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)); + = 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 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 c655dc8ae479..1c92c9a249c9 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( @@ -1756,7 +1745,7 @@ public void testIteratorDoesNotReturnSegmentWithChangingAppendableIndexSpec() null, new OnheapIncrementalIndex.Spec(true), null, - 1000L, + null, null, partitionsSpec, IndexSpec.getDefault(), @@ -1784,7 +1773,7 @@ public void testIteratorDoesNotReturnSegmentWithChangingAppendableIndexSpec() null, new OnheapIncrementalIndex.Spec(false), null, - 1000L, + null, null, partitionsSpec, IndexSpec.getDefault(), @@ -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/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; 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..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,7 +80,9 @@ 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 void setup() } } dataSources = DataSourcesSnapshot.fromUsedSegments(allSegments); - statusTracker = new CompactionStatusTracker(JSON_MAPPER); + statusTracker = new CompactionStatusTracker(); policy = new NewestSegmentFirstPolicy(null); } @@ -870,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 = @@ -1061,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 = @@ -1156,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 = @@ -1397,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 = @@ -1460,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 = @@ -2004,7 +2006,7 @@ public static class StaticUtilsTest @Test public void testIsParalleModeNullTuningConfigReturnFalse() { - Assert.assertFalse(CompactSegments.isParallelMode(null)); + Assert.assertFalse(CompactionSlotManager.isParallelMode(null)); } @Test @@ -2012,7 +2014,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 +2024,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 +2040,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 +2055,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 +2064,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 ); 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 247b74d8e57f..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 @@ -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; @@ -245,8 +247,8 @@ public static class EventMatcher implements Predicate 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 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 metricValue) + public EventMatcher hasValueMatching(Matcher valueMatcher) { - this.metricValue = metricValue; + this.valueMatcher = valueMatcher; return this; } @@ -274,7 +275,16 @@ public EventMatcher hasValueAtLeast(long metricValue) */ 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 boolean test(ServiceMetricEvent event) { 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 boolean test(ServiceMetricEvent event) 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 81a0c1367165..63a1d28bfcfa 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.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 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); 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..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 @@ -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,39 @@ 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 + ); + Assertions.assertEquals(expectedSchemas.size(), sqlSchemas.size()); + Assertions.assertEquals( expectedSchemas, sqlSchemas.stream().map(NamedSchema::getClass).collect(Collectors.toSet())); } @@ -171,36 +178,36 @@ 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 @@ -209,22 +216,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 +239,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/util/SqlTestFramework.java b/sql/src/test/java/org/apache/druid/sql/calcite/util/SqlTestFramework.java index 7a7cf1645972..38ddba751b5d 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() 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);