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 c9c4599fad76..89adac0ce09f 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 @@ -22,6 +22,7 @@ import com.google.common.collect.ImmutableList; import org.apache.druid.client.DataSourcesSnapshot; import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.segment.metadata.HeapMemoryCompactionStateManager; import org.apache.druid.server.compaction.CompactionCandidateSearchPolicy; import org.apache.druid.server.compaction.CompactionSegmentIterator; import org.apache.druid.server.compaction.NewestSegmentFirstPolicy; @@ -135,7 +136,8 @@ public void measureNewestSegmentFirstPolicy(Blackhole blackhole) policy, compactionConfigs, dataSources, - Collections.emptyMap() + Collections.emptyMap(), + new HeapMemoryCompactionStateManager() ); for (int i = 0; i < numCompactionTaskSlots && iterator.hasNext(); i++) { blackhole.consume(iterator.next()); diff --git a/docs/api-reference/automatic-compaction-api.md b/docs/api-reference/automatic-compaction-api.md index f3744a45f02b..670fbff222a3 100644 --- a/docs/api-reference/automatic-compaction-api.md +++ b/docs/api-reference/automatic-compaction-api.md @@ -889,6 +889,7 @@ This includes the following fields: |`compactionPolicy`|Policy to choose intervals for compaction. Currently, the only supported policy is [Newest segment first](#compaction-policy-newestsegmentfirst).|Newest segment first| |`useSupervisors`|Whether compaction should be run on Overlord using supervisors instead of Coordinator duties.|false| |`engine`|Engine used for running compaction tasks, unless overridden in the datasource-level compaction config. Possible values are `native` and `msq`. `msq` engine can be used for compaction only if `useSupervisors` is `true`.|`native`| +|`legacyPersistLastCompactionStateInSegments`|Whether to persist the full compaction state in segment metadata. When `true` (default), compaction state is stored in both the segment metadata and the compaction states table. This is historically how Druid has worked. When `false`, only a fingerprint reference is stored in the segment metadata, reducing storage overhead in the segments table. The actual compaction state is stored in the compaction states table and can be referenced with the aforementioned fingerprint. Eventually this configuration will be removed and all compaction will use the fingerprint method only. This configuration exists for operators to opt into this future pattern early. **WARNING: if you set this to false and then compact data, rolling back to a Druid version that pre-dates compaction state fingerprinting (< Druid 36) will result in missing compaction states and trigger compaction on segments that may already be compacted.**|`true`| #### Compaction policy `newestSegmentFirst` diff --git a/docs/configuration/index.md b/docs/configuration/index.md index a1d2d3070f61..071ebe18872e 100644 --- a/docs/configuration/index.md +++ b/docs/configuration/index.md @@ -389,6 +389,7 @@ These properties specify the JDBC connection and other configuration around the |`druid.metadata.storage.tables.segments`|The table to use to look for segments.|`druid_segments`| |`druid.metadata.storage.tables.rules`|The table to use to look for segment load/drop rules.|`druid_rules`| |`druid.metadata.storage.tables.config`|The table to use to look for configs.|`druid_config`| +|`druid.metadata.storage.tables.compactionStates`|The table to use to store compaction state fingerprints.|`druid_compactionStates`| |`druid.metadata.storage.tables.tasks`|Used by the indexing service to store tasks.|`druid_tasks`| |`druid.metadata.storage.tables.taskLog`|Used by the indexing service to store task logs.|`druid_tasklogs`| |`druid.metadata.storage.tables.taskLock`|Used by the indexing service to store task locks.|`druid_tasklocks`| @@ -810,6 +811,8 @@ These Coordinator static configurations can be defined in the `coordinator/runti |`druid.manager.rules.pollDuration`|The duration between polls the Coordinator does for updates to the set of active rules. Generally defines the amount of lag time it can take for the Coordinator to notice rules.|`PT1M`| |`druid.manager.rules.defaultRule`|The default rule for the cluster|`_default`| |`druid.manager.rules.alertThreshold`|The duration after a failed poll upon which an alert should be emitted.|`PT10M`| +|`druid.manager.compactionState.cacheSize`|The maximum number of compaction state fingerprints to cache in memory on the coordinator and overlord. Compaction state fingerprints are used to track the compaction configuration applied to segments. Consider increasing this value if you have a large number of datasources with compaction configurations.|`100`| +|`druid.manager.compactionState.prewarmSize`|The number of most recently used compaction state fingerprints to load into cache on Coordinator startup. This pre-warms the cache to improve performance immediately after startup.|`100`| #### Dynamic configuration diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/AutoCompactionTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/AutoCompactionTest.java index 0c7f4a93ff74..868cc48fbfb7 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/AutoCompactionTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/AutoCompactionTest.java @@ -1508,7 +1508,7 @@ public void testAutoCompactionDutyWithDimensionsSpec(CompactionEngine engine) th @ParameterizedTest(name = "useSupervisors={0}") public void testAutoCompactionDutyWithFilter(boolean useSupervisors) throws Exception { - updateClusterConfig(new ClusterCompactionConfig(0.5, 10, null, useSupervisors, null)); + updateClusterConfig(new ClusterCompactionConfig(0.5, 10, null, useSupervisors, null, true)); loadData(INDEX_TASK); try (final Closeable ignored = unloader(fullDatasourceName)) { @@ -1552,7 +1552,7 @@ public void testAutoCompactionDutyWithFilter(boolean useSupervisors) throws Exce @ParameterizedTest(name = "useSupervisors={0}") public void testAutoCompationDutyWithMetricsSpec(boolean useSupervisors) throws Exception { - updateClusterConfig(new ClusterCompactionConfig(0.5, 10, null, useSupervisors, null)); + updateClusterConfig(new ClusterCompactionConfig(0.5, 10, null, useSupervisors, null, true)); loadData(INDEX_TASK); try (final Closeable ignored = unloader(fullDatasourceName)) { @@ -1854,7 +1854,7 @@ private void forceTriggerAutoCompaction( ).collect(Collectors.toList()) ); updateClusterConfig( - new ClusterCompactionConfig(0.5, intervals.size(), policy, true, null) + new ClusterCompactionConfig(0.5, intervals.size(), policy, true, null, true) ); // Wait for scheduler to pick up the compaction job @@ -1864,7 +1864,7 @@ private void forceTriggerAutoCompaction( // Disable all compaction updateClusterConfig( - new ClusterCompactionConfig(0.5, intervals.size(), COMPACT_NOTHING_POLICY, true, null) + new ClusterCompactionConfig(0.5, intervals.size(), COMPACT_NOTHING_POLICY, true, null, true) ); } else { forceTriggerAutoCompaction(numExpectedSegmentsAfterCompaction); @@ -1956,7 +1956,8 @@ private void updateCompactionTaskSlot(double compactionTaskSlotRatio, int maxCom maxCompactionTaskSlots, oldConfig.getCompactionPolicy(), oldConfig.isUseSupervisors(), - oldConfig.getEngine() + oldConfig.getEngine(), + oldConfig.isLegacyPersistLastCompactionStateInSegments() ) ); diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionSupervisorTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionSupervisorTest.java index 9019a0cb5ec5..52a7d04eaa63 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionSupervisorTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionSupervisorTest.java @@ -41,6 +41,7 @@ import org.apache.druid.server.coordinator.InlineSchemaDataSourceCompactionConfig; import org.apache.druid.server.coordinator.UserCompactionTaskGranularityConfig; import org.apache.druid.server.coordinator.UserCompactionTaskQueryTuningConfig; +import org.apache.druid.server.coordinator.duty.CompactSegments; import org.apache.druid.testing.embedded.EmbeddedBroker; import org.apache.druid.testing.embedded.EmbeddedCoordinator; import org.apache.druid.testing.embedded.EmbeddedDruidCluster; @@ -50,6 +51,7 @@ import org.apache.druid.testing.embedded.EmbeddedRouter; import org.apache.druid.testing.embedded.indexing.MoreResources; import org.apache.druid.testing.embedded.junit5.EmbeddedClusterTestBase; +import org.apache.druid.timeline.CompactionState; import org.hamcrest.Matcher; import org.hamcrest.Matchers; import org.joda.time.Period; @@ -111,14 +113,14 @@ public EmbeddedDruidCluster createCluster() private void configureCompaction(CompactionEngine compactionEngine) { final UpdateResponse updateResponse = cluster.callApi().onLeaderOverlord( - o -> o.updateClusterCompactionConfig(new ClusterCompactionConfig(1.0, 100, null, true, compactionEngine)) + o -> o.updateClusterCompactionConfig(new ClusterCompactionConfig(1.0, 100, null, true, compactionEngine, true)) ); Assertions.assertTrue(updateResponse.isSuccess()); } @MethodSource("getEngine") @ParameterizedTest(name = "compactionEngine={0}") - public void test_ingestDayGranularity_andCompactToMonthGranularity_withInlineConfig(CompactionEngine compactionEngine) + public void test_ingestDayGranularity_andCompactToMonthGranularity_andCompactToYearGranularity_withInlineConfig(CompactionEngine compactionEngine) { configureCompaction(compactionEngine); @@ -132,7 +134,7 @@ public void test_ingestDayGranularity_andCompactToMonthGranularity_withInlineCon Assertions.assertEquals(3, getNumSegmentsWith(Granularities.DAY)); // Create a compaction config with MONTH granularity - InlineSchemaDataSourceCompactionConfig compactionConfig = + InlineSchemaDataSourceCompactionConfig monthGranConfig = InlineSchemaDataSourceCompactionConfig .builder() .forDataSource(dataSource) @@ -165,11 +167,170 @@ public void test_ingestDayGranularity_andCompactToMonthGranularity_withInlineCon ) .build(); - runCompactionWithSpec(compactionConfig); + runCompactionWithSpec(monthGranConfig); waitForAllCompactionTasksToFinish(); Assertions.assertEquals(0, getNumSegmentsWith(Granularities.DAY)); Assertions.assertEquals(1, getNumSegmentsWith(Granularities.MONTH)); + + verifyCompactedSegmentsHaveFingerprints(monthGranConfig); + + InlineSchemaDataSourceCompactionConfig yearGranConfig = + InlineSchemaDataSourceCompactionConfig + .builder() + .forDataSource(dataSource) + .withSkipOffsetFromLatest(Period.seconds(0)) + .withGranularitySpec( + new UserCompactionTaskGranularityConfig(Granularities.YEAR, null, null) + ) + .withTuningConfig( + new UserCompactionTaskQueryTuningConfig( + null, + null, + null, + null, + null, + new DimensionRangePartitionsSpec(null, 5000, List.of("item"), false), + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null + ) + ) + .build(); + + overlord.latchableEmitter().flush(); // flush events so wait for works correctly on the next round of compaction + runCompactionWithSpec(yearGranConfig); + waitForAllCompactionTasksToFinish(); + + Assertions.assertEquals(0, getNumSegmentsWith(Granularities.DAY)); + Assertions.assertEquals(0, getNumSegmentsWith(Granularities.MONTH)); + Assertions.assertEquals(1, getNumSegmentsWith(Granularities.YEAR)); + + verifyCompactedSegmentsHaveFingerprints(yearGranConfig); + } + + @MethodSource("getEngine") + @ParameterizedTest(name = "compactionEngine={0}") + public void test_compaction_withPersistLastCompactionStateFalse_storesOnlyFingerprint(CompactionEngine compactionEngine) + throws InterruptedException + { + // Configure cluster with persistLastCompactionState=false + final UpdateResponse updateResponse = cluster.callApi().onLeaderOverlord( + o -> o.updateClusterCompactionConfig( + new ClusterCompactionConfig(1.0, 100, null, true, compactionEngine, false) + ) + ); + Assertions.assertTrue(updateResponse.isSuccess()); + + // Ingest data at DAY granularity + runIngestionAtGranularity( + "DAY", + "2025-06-01T00:00:00.000Z,shirt,105\n" + + "2025-06-02T00:00:00.000Z,trousers,210" + ); + Assertions.assertEquals(2, getNumSegmentsWith(Granularities.DAY)); + + // Create compaction config to compact to MONTH granularity + InlineSchemaDataSourceCompactionConfig monthConfig = + InlineSchemaDataSourceCompactionConfig + .builder() + .forDataSource(dataSource) + .withSkipOffsetFromLatest(Period.seconds(0)) + .withGranularitySpec( + new UserCompactionTaskGranularityConfig(Granularities.MONTH, null, null) + ) + .withTuningConfig( + new UserCompactionTaskQueryTuningConfig( + null, + null, + null, + null, + null, + new DimensionRangePartitionsSpec(1000, null, List.of("item"), false), + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null + ) + ) + .build(); + + runCompactionWithSpec(monthConfig); + waitForAllCompactionTasksToFinish(); + + verifySegmentsHaveNullLastCompactionStateAndNonNullFingerprint(); + } + + private void verifySegmentsHaveNullLastCompactionStateAndNonNullFingerprint() + { + overlord + .bindings() + .segmentsMetadataStorage() + .retrieveAllUsedSegments(dataSource, Segments.ONLY_VISIBLE) + .forEach(segment -> { + Assertions.assertNull( + segment.getLastCompactionState(), + "Segment " + segment.getId() + " should have null lastCompactionState" + ); + Assertions.assertNotNull( + segment.getCompactionStateFingerprint(), + "Segment " + segment.getId() + " should have non-null compactionStateFingerprint" + ); + }); + } + + private void verifyCompactedSegmentsHaveFingerprints(DataSourceCompactionConfig compactionConfig) + { + String expectedFingerprint = CompactionState.generateCompactionStateFingerprint( + CompactSegments.createCompactionStateFromConfig(compactionConfig), + dataSource + ); + + overlord + .bindings() + .segmentsMetadataStorage() + .retrieveAllUsedSegments(dataSource, Segments.ONLY_VISIBLE) + .forEach(segment -> { + String fingerprint = segment.getCompactionStateFingerprint(); + Assertions.assertNotNull( + fingerprint, + "Segment " + segment.getId() + " should have a compaction state fingerprint" + ); + Assertions.assertFalse( + fingerprint.isEmpty(), + "Segment " + segment.getId() + " fingerprint should not be empty" + ); + // SHA-256 fingerprints should be 64 hex characters + Assertions.assertEquals( + 64, + fingerprint.length(), + "Segment " + segment.getId() + " fingerprint should be 64 characters (SHA-256)" + ); + Assertions.assertEquals( + expectedFingerprint, + fingerprint, + "Segment " + segment.getId() + " fingerprint should match expected fingerprint" + ); + }); } private void runCompactionWithSpec(DataSourceCompactionConfig config) 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 8200bb335566..9d2af4d2b21c 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 @@ -212,7 +212,7 @@ public void test_ingestClusterMetrics_withConcurrentCompactionSupervisor_andSkip ); final ClusterCompactionConfig updatedCompactionConfig - = new ClusterCompactionConfig(1.0, 10, null, true, null); + = new ClusterCompactionConfig(1.0, 10, null, true, null, null); final UpdateResponse updateResponse = cluster.callApi().onLeaderOverlord( o -> o.updateClusterCompactionConfig(updatedCompactionConfig) ); @@ -323,7 +323,7 @@ public void test_ingestClusterMetrics_compactionSkipsLockedIntervals() ); final ClusterCompactionConfig updatedCompactionConfig - = new ClusterCompactionConfig(1.0, 10, null, true, null); + = new ClusterCompactionConfig(1.0, 10, null, true, null, null); final UpdateResponse updateResponse = cluster.callApi().onLeaderOverlord( o -> o.updateClusterCompactionConfig(updatedCompactionConfig) ); 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 ece6a9ef0573..d5093d4ffafe 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 @@ -164,7 +164,7 @@ private IndexTask createIndexTaskForInlineData(String taskId) private void enableCompactionSupervisor() { final UpdateResponse updateResponse = cluster.callApi().onLeaderOverlord( - o -> o.updateClusterCompactionConfig(new ClusterCompactionConfig(1.0, 10, null, true, null)) + o -> o.updateClusterCompactionConfig(new ClusterCompactionConfig(1.0, 10, null, true, null, null)) ); Assertions.assertTrue(updateResponse.isSuccess()); } diff --git a/indexing-hadoop/src/main/java/org/apache/druid/indexer/updater/MetadataStorageUpdaterJobSpec.java b/indexing-hadoop/src/main/java/org/apache/druid/indexer/updater/MetadataStorageUpdaterJobSpec.java index dfbdc3e4e85f..105c4ceb9aff 100644 --- a/indexing-hadoop/src/main/java/org/apache/druid/indexer/updater/MetadataStorageUpdaterJobSpec.java +++ b/indexing-hadoop/src/main/java/org/apache/druid/indexer/updater/MetadataStorageUpdaterJobSpec.java @@ -99,6 +99,7 @@ public MetadataStorageTablesConfig getMetadataStorageTablesConfig() null, null, null, + null, null ); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractBatchIndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractBatchIndexTask.java index 97dc70f33578..5067f856e789 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractBatchIndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractBatchIndexTask.java @@ -641,6 +641,21 @@ public static boolean isGuaranteedRollup( return tuningConfig.isForceGuaranteedRollup(); } + public static Function, Set> addCompactionStateFingerprintToSegments( + String compactionStateFingerprint + ) + { + if (compactionStateFingerprint != null) { + return segments -> segments.stream() + .map( + segment -> segment.withCompactionStateFingerprint(compactionStateFingerprint) + ) + .collect(Collectors.toSet()); + } else { + return Function.identity(); + } + } + public static Function, Set> addCompactionStateToSegments( boolean storeCompactionState, TaskToolbox toolbox, diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java index f87ac965ce98..abffa9c7860d 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java @@ -902,11 +902,19 @@ private TaskStatus generateAndPublishSegments( Tasks.STORE_COMPACTION_STATE_KEY, Tasks.DEFAULT_STORE_COMPACTION_STATE ); + + final String compactionStateFingerprint = getContextValue( + Tasks.COMPACTION_STATE_FINGERPRINT_KEY, + null + ); + final Function, Set> annotateFunction = addCompactionStateToSegments( storeCompactionState, toolbox, ingestionSchema + ).andThen( + addCompactionStateFingerprintToSegments(compactionStateFingerprint) ); Set tombStones = Collections.emptySet(); 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 b45eb45dc041..b90e4bfd2f91 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 @@ -68,4 +68,13 @@ public class Tasks static { Verify.verify(STORE_COMPACTION_STATE_KEY.equals(CompactSegments.STORE_COMPACTION_STATE_KEY)); } + + /** + * Context k:v pair that holds the fingerprint of the compaction state to be stored with the segment + */ + public static final String COMPACTION_STATE_FINGERPRINT_KEY = "compactionStateFingerprint"; + + static { + Verify.verify(COMPACTION_STATE_FINGERPRINT_KEY.equals(CompactSegments.COMPACTION_STATE_FINGERPRINT_KEY)); + } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java index 9faa4c2013e1..c74beac9b550 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java @@ -1162,12 +1162,20 @@ private void publishSegments( Tasks.STORE_COMPACTION_STATE_KEY, Tasks.DEFAULT_STORE_COMPACTION_STATE ); - final Function, Set> annotateFunction = addCompactionStateToSegments( - storeCompactionState, - toolbox, - ingestionSchema + final String compactionStateFingerprint = getContextValue( + Tasks.COMPACTION_STATE_FINGERPRINT_KEY, + null ); + final Function, Set> annotateFunction = + addCompactionStateToSegments( + storeCompactionState, + toolbox, + ingestionSchema + ).andThen( + addCompactionStateFingerprintToSegments(compactionStateFingerprint) + ); + Set tombStones = Collections.emptySet(); if (getIngestionMode() == IngestionMode.REPLACE) { TombstoneHelper tombstoneHelper = new TombstoneHelper(toolbox.getTaskActionClient()); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionConfigBasedJobTemplate.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionConfigBasedJobTemplate.java index 6b984a4b6c03..5f5305df5e65 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionConfigBasedJobTemplate.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionConfigBasedJobTemplate.java @@ -23,6 +23,7 @@ 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.DateTimes; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.server.compaction.CompactionCandidate; @@ -31,12 +32,14 @@ import org.apache.druid.server.compaction.NewestSegmentFirstPolicy; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; import org.apache.druid.server.coordinator.duty.CompactSegments; +import org.apache.druid.timeline.CompactionState; 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.Map; import java.util.Objects; /** @@ -70,12 +73,34 @@ public List createCompactionJobs( final List jobs = new ArrayList<>(); + CompactionState compactionState = CompactSegments.createCompactionStateFromConfig(config); + + String compactionStateFingerprint = CompactionState.generateCompactionStateFingerprint( + compactionState, + config.getDataSource() + ); + + if (segmentIterator.hasNext()) { + // If we are going to create compaction jobs for this compaction state, we need to persist the fingerprint -> state + // mapping so compacted segments from these jobs can reference a valid compaction state. + params.getCompactionStateManager().persistCompactionState( + config.getDataSource(), + Map.of(compactionStateFingerprint, compactionState), + DateTimes.nowUtc() + ); + } + // Create a job for each CompactionCandidate while (segmentIterator.hasNext()) { final CompactionCandidate candidate = segmentIterator.next(); - ClientCompactionTaskQuery taskPayload - = CompactSegments.createCompactionTask(candidate, config, params.getClusterCompactionConfig().getEngine()); + ClientCompactionTaskQuery taskPayload = CompactSegments.createCompactionTask( + candidate, + config, + params.getClusterCompactionConfig().getEngine(), + compactionStateFingerprint, + params.getClusterCompactionConfig().isLegacyPersistLastCompactionStateInSegments() + ); jobs.add( new CompactionJob( taskPayload, @@ -120,7 +145,8 @@ DataSourceCompactibleSegmentIterator getCompactibleCandidates( Intervals.complementOf(searchInterval), // This policy is used only while creating jobs // The actual order of jobs is determined by the policy used in CompactionJobQueue - new NewestSegmentFirstPolicy(null) + new NewestSegmentFirstPolicy(null), + params.getCompactionStateManager() ); // Collect stats for segments that are already compacted diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionJobParams.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionJobParams.java index 0113f1b78bac..5b82633b0c85 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionJobParams.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionJobParams.java @@ -19,6 +19,7 @@ package org.apache.druid.indexing.compact; +import org.apache.druid.segment.metadata.CompactionStateManager; import org.apache.druid.server.compaction.CompactionSnapshotBuilder; import org.apache.druid.server.coordinator.ClusterCompactionConfig; import org.apache.druid.timeline.SegmentTimeline; @@ -33,18 +34,21 @@ public class CompactionJobParams private final TimelineProvider timelineProvider; private final ClusterCompactionConfig clusterCompactionConfig; private final CompactionSnapshotBuilder snapshotBuilder; + private final CompactionStateManager compactionStateManager; public CompactionJobParams( DateTime scheduleStartTime, ClusterCompactionConfig clusterCompactionConfig, TimelineProvider timelineProvider, - CompactionSnapshotBuilder snapshotBuilder + CompactionSnapshotBuilder snapshotBuilder, + CompactionStateManager compactionStateManager ) { this.scheduleStartTime = scheduleStartTime; this.clusterCompactionConfig = clusterCompactionConfig; this.timelineProvider = timelineProvider; this.snapshotBuilder = snapshotBuilder; + this.compactionStateManager = compactionStateManager; } /** @@ -88,6 +92,11 @@ public CompactionSnapshotBuilder getSnapshotBuilder() return snapshotBuilder; } + public CompactionStateManager getCompactionStateManager() + { + return compactionStateManager; + } + @FunctionalInterface public interface TimelineProvider { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionJobQueue.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionJobQueue.java index 77886af1a017..78648883e2df 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionJobQueue.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionJobQueue.java @@ -36,6 +36,7 @@ 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.segment.metadata.CompactionStateManager; import org.apache.druid.server.compaction.CompactionCandidate; import org.apache.druid.server.compaction.CompactionCandidateSearchPolicy; import org.apache.druid.server.compaction.CompactionSlotManager; @@ -104,7 +105,8 @@ public CompactionJobQueue( GlobalTaskLockbox taskLockbox, OverlordClient overlordClient, BrokerClient brokerClient, - ObjectMapper objectMapper + ObjectMapper objectMapper, + CompactionStateManager compactionStateManager ) { this.runStats = new CoordinatorRunStats(); @@ -120,7 +122,8 @@ public CompactionJobQueue( DateTimes.nowUtc(), clusterCompactionConfig, dataSourcesSnapshot.getUsedSegmentsTimelinesPerDataSource()::get, - snapshotBuilder + snapshotBuilder, + compactionStateManager ); this.taskActionClientFactory = taskActionClientFactory; 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 c3bce6a09de1..7e8caa3b8806 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 @@ -44,6 +44,7 @@ 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.segment.metadata.CompactionStateManager; import org.apache.druid.server.compaction.CompactionRunSimulator; import org.apache.druid.server.compaction.CompactionSimulateResult; import org.apache.druid.server.compaction.CompactionStatusTracker; @@ -139,6 +140,8 @@ public class OverlordCompactionScheduler implements CompactionScheduler private final boolean shouldPollSegments; private final long schedulePeriodMillis; + private final CompactionStateManager compactionStateManager; + @Inject public OverlordCompactionScheduler( TaskMaster taskMaster, @@ -154,7 +157,8 @@ public OverlordCompactionScheduler( ScheduledExecutorFactory executorFactory, BrokerClient brokerClient, ServiceEmitter emitter, - ObjectMapper objectMapper + ObjectMapper objectMapper, + CompactionStateManager compactionStateManager ) { final long segmentPollPeriodMillis = @@ -180,6 +184,7 @@ public OverlordCompactionScheduler( this.taskActionClientFactory = taskActionClientFactory; this.druidInputSourceFactory = druidInputSourceFactory; + this.compactionStateManager = compactionStateManager; this.taskRunnerListener = new TaskRunnerListener() { @Override @@ -366,7 +371,8 @@ private synchronized void resetCompactionJobQueue() taskLockbox, overlordClient, brokerClient, - objectMapper + objectMapper, + compactionStateManager ); latestJobQueue.set(queue); 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 d154c32ce3eb..98580fa0363a 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 @@ -64,6 +64,7 @@ import org.apache.druid.query.http.ClientSqlQuery; import org.apache.druid.query.http.SqlTaskStatus; import org.apache.druid.segment.TestIndex; +import org.apache.druid.segment.metadata.HeapMemoryCompactionStateManager; import org.apache.druid.server.compaction.CompactionSimulateResult; import org.apache.druid.server.compaction.CompactionStatistics; import org.apache.druid.server.compaction.CompactionStatus; @@ -173,7 +174,7 @@ public void setUp() segmentStorage = new TestIndexerMetadataStorageCoordinator(); segmentsMetadataManager = segmentStorage.getManager(); - compactionConfig = new AtomicReference<>(new ClusterCompactionConfig(1.0, 100, null, true, null)); + compactionConfig = new AtomicReference<>(new ClusterCompactionConfig(1.0, 100, null, true, null, null)); coordinatorOverlordServiceConfig = new CoordinatorOverlordServiceConfig(false, null); taskActionClientFactory = task -> new TaskActionClient() @@ -231,7 +232,8 @@ private void initScheduler() (nameFormat, numThreads) -> new WrappingScheduledExecutorService("test", executor, false), brokerClient, serviceEmitter, - OBJECT_MAPPER + OBJECT_MAPPER, + new HeapMemoryCompactionStateManager() ); } @@ -444,7 +446,7 @@ public void test_simulateRunWithConfigUpdate() scheduler.startCompaction(dataSource, createSupervisorWithInlineSpec()); final CompactionSimulateResult simulateResult = scheduler.simulateRunWithConfigUpdate( - new ClusterCompactionConfig(null, null, null, null, null) + new ClusterCompactionConfig(null, null, null, null, null, null) ); Assert.assertEquals(1, simulateResult.getCompactionStates().size()); final Table pendingCompactionTable = simulateResult.getCompactionStates().get(CompactionStatus.State.PENDING); @@ -469,7 +471,7 @@ public void test_simulateRunWithConfigUpdate() scheduler.stopCompaction(dataSource); final CompactionSimulateResult simulateResultWhenDisabled = scheduler.simulateRunWithConfigUpdate( - new ClusterCompactionConfig(null, null, null, null, null) + new ClusterCompactionConfig(null, null, null, null, null, null) ); Assert.assertTrue(simulateResultWhenDisabled.getCompactionStates().isEmpty()); @@ -536,12 +538,12 @@ private void runCompactionTask(String taskId, Interval compactionInterval, Granu private void disableScheduler() { - compactionConfig.set(new ClusterCompactionConfig(null, null, null, false, null)); + compactionConfig.set(new ClusterCompactionConfig(null, null, null, false, null, null)); } private void enableScheduler() { - compactionConfig.set(new ClusterCompactionConfig(null, null, null, true, null)); + compactionConfig.set(new ClusterCompactionConfig(null, null, null, true, null, null)); } private void runScheduledJob() diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/http/OverlordCompactionResourceTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/http/OverlordCompactionResourceTest.java index a54459ad8891..3518e1dea409 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/http/OverlordCompactionResourceTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/http/OverlordCompactionResourceTest.java @@ -159,7 +159,7 @@ public void test_updateClusterConfig() replayAll(); Response response = compactionResource.updateClusterCompactionConfig( - new ClusterCompactionConfig(0.5, 10, null, true, CompactionEngine.MSQ), + new ClusterCompactionConfig(0.5, 10, null, true, CompactionEngine.MSQ, true), httpRequest ); Assert.assertEquals(200, response.getStatus()); @@ -170,7 +170,7 @@ public void test_updateClusterConfig() public void test_getClusterConfig() { final ClusterCompactionConfig clusterConfig = - new ClusterCompactionConfig(0.4, 100, null, true, CompactionEngine.MSQ); + new ClusterCompactionConfig(0.4, 100, null, true, CompactionEngine.MSQ, true); EasyMock.expect(configManager.getClusterCompactionConfig()) .andReturn(clusterConfig) .once(); diff --git a/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java b/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java index b8238b81b165..399a8e98ae3b 100644 --- a/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java +++ b/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java @@ -1696,6 +1696,12 @@ private void handleQueryResults( Tasks.DEFAULT_STORE_COMPACTION_STATE ); + String compactionStateFingerprint = querySpec.getContext() + .getString( + Tasks.COMPACTION_STATE_FINGERPRINT_KEY, + null + ); + if (storeCompactionState) { DataSourceMSQDestination destination = (DataSourceMSQDestination) querySpec.getDestination(); if (!destination.isReplaceTimeChunks()) { @@ -1721,6 +1727,9 @@ private void handleQueryResults( ); } } + if (compactionStateFingerprint != null) { + compactionStateAnnotateFunction = compactionStateAnnotateFunction.andThen(addCompactionStateFingerprintToSegments(compactionStateFingerprint)); + } log.info("Query [%s] publishing %d segments.", queryDef.getQueryId(), segments.size()); publishAllSegments(segments, compactionStateAnnotateFunction); } else if (MSQControllerTask.isExport(querySpec.getDestination())) { @@ -1751,6 +1760,19 @@ private void handleQueryResults( } } + private static Function, Set> addCompactionStateFingerprintToSegments(String compactionStateFingerprint) + { + if (compactionStateFingerprint != null) { + return segments -> segments.stream() + .map( + segment -> segment.withCompactionStateFingerprint(compactionStateFingerprint) + ) + .collect(Collectors.toSet()); + } else { + return Function.identity(); + } + } + private static Function, Set> addCompactionStateToSegments( MSQSpec querySpec, ObjectMapper jsonMapper, diff --git a/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/DataSegmentWithLocation.java b/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/DataSegmentWithLocation.java index 603a0f33e52e..1714fe072992 100644 --- a/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/DataSegmentWithLocation.java +++ b/multi-stage-query/src/main/java/org/apache/druid/msq/input/table/DataSegmentWithLocation.java @@ -61,6 +61,7 @@ private DataSegmentWithLocation( @JsonProperty("binaryVersion") Integer binaryVersion, @JsonProperty("size") long size, @JsonProperty("servers") Set servers, + @JsonProperty("compactionStateFingerprint") String compactionStateFingerprint, @JacksonInject PruneSpecsHolder pruneSpecsHolder ) { @@ -76,6 +77,7 @@ private DataSegmentWithLocation( lastCompactionState, binaryVersion, size, + compactionStateFingerprint, pruneSpecsHolder ); this.servers = Preconditions.checkNotNull(servers, "servers"); @@ -98,6 +100,7 @@ public DataSegmentWithLocation( null, dataSegment.getBinaryVersion(), dataSegment.getSize(), + dataSegment.getCompactionStateFingerprint(), PruneSpecsHolder.DEFAULT ); this.servers = servers; diff --git a/processing/src/main/java/org/apache/druid/metadata/MetadataStorageConnector.java b/processing/src/main/java/org/apache/druid/metadata/MetadataStorageConnector.java index 1c185f38575b..dfe30ec4af6d 100644 --- a/processing/src/main/java/org/apache/druid/metadata/MetadataStorageConnector.java +++ b/processing/src/main/java/org/apache/druid/metadata/MetadataStorageConnector.java @@ -95,4 +95,9 @@ default void exportTable( * SegmentSchema table is created only when CentralizedDatasourceSchema feature is enabled. */ void createSegmentSchemasTable(); + + /** + * + */ + void createCompactionStatesTable(); } diff --git a/processing/src/main/java/org/apache/druid/metadata/MetadataStorageTablesConfig.java b/processing/src/main/java/org/apache/druid/metadata/MetadataStorageTablesConfig.java index 35915b52b70c..0303da10a610 100644 --- a/processing/src/main/java/org/apache/druid/metadata/MetadataStorageTablesConfig.java +++ b/processing/src/main/java/org/apache/druid/metadata/MetadataStorageTablesConfig.java @@ -32,7 +32,7 @@ public class MetadataStorageTablesConfig public static MetadataStorageTablesConfig fromBase(String base) { - return new MetadataStorageTablesConfig(base, null, null, null, null, null, null, null, null, null, null, null, null); + return new MetadataStorageTablesConfig(base, null, null, null, null, null, null, null, null, null, null, null, null, null); } private static final String DEFAULT_BASE = "druid"; @@ -76,6 +76,9 @@ public static MetadataStorageTablesConfig fromBase(String base) @JsonProperty("useShortIndexNames") private final boolean useShortIndexNames; + @JsonProperty("compactionStates") + private final String compactionStatesTable; + @JsonCreator public MetadataStorageTablesConfig( @JsonProperty("base") String base, @@ -90,7 +93,8 @@ public MetadataStorageTablesConfig( @JsonProperty("supervisors") String supervisorTable, @JsonProperty("upgradeSegments") String upgradeSegmentsTable, @JsonProperty("segmentSchemas") String segmentSchemasTable, - @JsonProperty("useShortIndexNames") Boolean useShortIndexNames + @JsonProperty("useShortIndexNames") Boolean useShortIndexNames, + @JsonProperty("compactionStates") String compactionStatesTable ) { this.base = (base == null) ? DEFAULT_BASE : base; @@ -107,6 +111,7 @@ public MetadataStorageTablesConfig( this.supervisorTable = makeTableName(supervisorTable, "supervisors"); this.segmentSchemasTable = makeTableName(segmentSchemasTable, "segmentSchemas"); this.useShortIndexNames = Configs.valueOrDefault(useShortIndexNames, false); + this.compactionStatesTable = makeTableName(compactionStatesTable, "compactionStates"); } private String makeTableName(String explicitTableName, String defaultSuffix) @@ -181,6 +186,11 @@ public String getSegmentSchemasTable() return segmentSchemasTable; } + public String getCompactionStatesTable() + { + return compactionStatesTable; + } + /** * If enabled, this causes table indices to be created with short, unique SHA-based identifiers. */ diff --git a/processing/src/main/java/org/apache/druid/timeline/CompactionState.java b/processing/src/main/java/org/apache/druid/timeline/CompactionState.java index f99d996a6b73..e4542e553d6a 100644 --- a/processing/src/main/java/org/apache/druid/timeline/CompactionState.java +++ b/processing/src/main/java/org/apache/druid/timeline/CompactionState.java @@ -22,10 +22,19 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonInclude; import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.MapperFeature; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.SerializationFeature; +import com.google.common.hash.Hasher; +import com.google.common.hash.Hashing; +import com.google.common.io.BaseEncoding; import org.apache.druid.data.input.impl.AggregateProjectionSpec; import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.indexer.granularity.GranularitySpec; import org.apache.druid.indexer.partitions.PartitionsSpec; +import org.apache.druid.jackson.DefaultObjectMapper; +import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.transform.CompactionTransformSpec; @@ -50,6 +59,26 @@ */ public class CompactionState { + + /** + * Lazy initialization holder for deterministic ObjectMapper. + * This inner static class is only loaded when first accessed, ensuring all Druid modules + * are properly initialized before the ObjectMapper is created. + * Based on DefaultObjectMapper (with all Druid modules) plus alphabetical sorting for consistency. + */ + private static class DeterministicMapperHolder + { + static final ObjectMapper INSTANCE = createDeterministicMapper(); + + private static ObjectMapper createDeterministicMapper() + { + DefaultObjectMapper baseMapper = new DefaultObjectMapper(); + baseMapper.configure(SerializationFeature.ORDER_MAP_ENTRIES_BY_KEYS, true); + baseMapper.configure(MapperFeature.SORT_PROPERTIES_ALPHABETICALLY, true); + return baseMapper; + } + } + private final PartitionsSpec partitionsSpec; private final DimensionsSpec dimensionsSpec; private final CompactionTransformSpec transformSpec; @@ -123,6 +152,23 @@ public List getProjections() return projections; } + /** + * Returns a deterministic byte representation of this CompactionState for fingerprinting purposes. + * Uses Jackson serialization with sorted properties and map entries to ensure consistency. + * + * @return byte array representing the serialized CompactionState + * @throws RuntimeException if serialization fails + */ + public byte[] getDeterministicBytes() + { + try { + return DeterministicMapperHolder.INSTANCE.writeValueAsBytes(this); + } + catch (JsonProcessingException e) { + throw new RuntimeException("Failed to serialize CompactionState for fingerprinting", e); + } + } + @Override public boolean equals(Object o) { @@ -206,4 +252,22 @@ public static Function, Set> addCompactionStateToS .map(s -> s.withLastCompactionState(compactionState)) .collect(Collectors.toSet()); } + + /** + * Generates a fingerprint string for the given compaction state and data source using SHA-256 hash algorithm. + */ + @SuppressWarnings("UnstableApiUsage") + public static String generateCompactionStateFingerprint(final CompactionState compactionState, final String dataSource) + { + final Hasher hasher = Hashing.sha256().newHasher(); + + hasher.putBytes(StringUtils.toUtf8(dataSource)); + hasher.putByte((byte) 0xff); + + // delegate to compaction state to provide its deterministic bytes + hasher.putBytes(compactionState.getDeterministicBytes()); + hasher.putByte((byte) 0xff); + + return BaseEncoding.base16().encode(hasher.hash().asBytes()); + } } diff --git a/processing/src/main/java/org/apache/druid/timeline/DataSegment.java b/processing/src/main/java/org/apache/druid/timeline/DataSegment.java index ef13b15f773b..19f3a817f6e0 100644 --- a/processing/src/main/java/org/apache/druid/timeline/DataSegment.java +++ b/processing/src/main/java/org/apache/druid/timeline/DataSegment.java @@ -114,6 +114,12 @@ public static class PruneSpecsHolder private final CompactionState lastCompactionState; private final long size; + /** + * SHA-256 fingerprint representation of the CompactionState. + */ + @Nullable + private final String compactionStateFingerprint; + /** * @deprecated use {@link #builder(SegmentId)} or {@link #builder(DataSegment)} instead. */ @@ -142,6 +148,7 @@ public DataSegment( null, binaryVersion, size, + null, PruneSpecsHolder.DEFAULT ); } @@ -175,6 +182,7 @@ public DataSegment( lastCompactionState, binaryVersion, size, + null, PruneSpecsHolder.DEFAULT ); } @@ -196,6 +204,7 @@ private DataSegment( @JsonProperty("lastCompactionState") @Nullable CompactionState lastCompactionState, @JsonProperty("binaryVersion") Integer binaryVersion, @JsonProperty("size") long size, + @JsonProperty("compactionStateFingerprint") @Nullable String compactionStateFingerprint, @JacksonInject PruneSpecsHolder pruneSpecsHolder ) { @@ -211,6 +220,7 @@ private DataSegment( lastCompactionState, binaryVersion, size, + compactionStateFingerprint, pruneSpecsHolder ); } @@ -227,6 +237,7 @@ public DataSegment( @Nullable CompactionState lastCompactionState, Integer binaryVersion, long size, + String compactionStateFingerprint, PruneSpecsHolder pruneSpecsHolder ) { @@ -245,6 +256,9 @@ public DataSegment( this.binaryVersion = binaryVersion; Preconditions.checkArgument(size >= 0); this.size = size; + this.compactionStateFingerprint = compactionStateFingerprint != null + ? STRING_INTERNER.intern(compactionStateFingerprint) + : null; } /** @@ -339,6 +353,14 @@ public boolean isTombstone() return getShardSpec().getType().equals(ShardSpec.Type.TOMBSTONE); } + @Nullable + @JsonProperty + @JsonInclude(JsonInclude.Include.NON_NULL) + public String getCompactionStateFingerprint() + { + return compactionStateFingerprint; + } + @Override public boolean overshadows(DataSegment other) { @@ -433,6 +455,11 @@ public DataSegment withLastCompactionState(CompactionState compactionState) return builder(this).lastCompactionState(compactionState).build(); } + public DataSegment withCompactionStateFingerprint(String compactionStateFingerprint) + { + return builder(this).compactionStateFingerprint(compactionStateFingerprint).build(); + } + @Override public int compareTo(DataSegment dataSegment) { @@ -467,6 +494,7 @@ public String toString() ", shardSpec=" + shardSpec + ", lastCompactionState=" + lastCompactionState + ", size=" + size + + ", compactionStateFingerprint=" + compactionStateFingerprint + '}'; } @@ -540,6 +568,7 @@ public static class Builder private CompactionState lastCompactionState; private Integer binaryVersion; private long size; + private String compactionStateFingerprint; /** * @deprecated use {@link #Builder(SegmentId)} or {@link #Builder(DataSegment)} instead. @@ -565,6 +594,7 @@ private Builder(SegmentId segmentId) this.binaryVersion = 0; this.size = 0; this.lastCompactionState = null; + this.compactionStateFingerprint = null; } private Builder(DataSegment segment) @@ -580,6 +610,7 @@ private Builder(DataSegment segment) this.lastCompactionState = segment.getLastCompactionState(); this.binaryVersion = segment.getBinaryVersion(); this.size = segment.getSize(); + this.compactionStateFingerprint = segment.getCompactionStateFingerprint(); } public Builder dataSource(String dataSource) @@ -648,6 +679,12 @@ public Builder size(long size) return this; } + public Builder compactionStateFingerprint(String compactionStateFingerprint) + { + this.compactionStateFingerprint = compactionStateFingerprint; + return this; + } + public DataSegment build() { // Check stuff that goes into the id, at least. @@ -668,6 +705,7 @@ public DataSegment build() lastCompactionState, binaryVersion, size, + compactionStateFingerprint, PruneSpecsHolder.DEFAULT ); } diff --git a/processing/src/test/java/org/apache/druid/metadata/TestMetadataStorageConnector.java b/processing/src/test/java/org/apache/druid/metadata/TestMetadataStorageConnector.java index d8722a2719f0..1251c4fcf6f7 100644 --- a/processing/src/test/java/org/apache/druid/metadata/TestMetadataStorageConnector.java +++ b/processing/src/test/java/org/apache/druid/metadata/TestMetadataStorageConnector.java @@ -101,4 +101,10 @@ public void createSegmentSchemasTable() { throw new UnsupportedOperationException(); } + + @Override + public void createCompactionStatesTable() + { + throw new UnsupportedOperationException(); + } } diff --git a/processing/src/test/java/org/apache/druid/metadata/TestMetadataStorageTablesConfig.java b/processing/src/test/java/org/apache/druid/metadata/TestMetadataStorageTablesConfig.java index 784b7e2cad69..bbe82add29c9 100644 --- a/processing/src/test/java/org/apache/druid/metadata/TestMetadataStorageTablesConfig.java +++ b/processing/src/test/java/org/apache/druid/metadata/TestMetadataStorageTablesConfig.java @@ -39,6 +39,7 @@ public TestMetadataStorageTablesConfig() null, null, null, + null, null ); } diff --git a/processing/src/test/java/org/apache/druid/timeline/CompactionStateTest.java b/processing/src/test/java/org/apache/druid/timeline/CompactionStateTest.java new file mode 100644 index 000000000000..13aa0aabcc3d --- /dev/null +++ b/processing/src/test/java/org/apache/druid/timeline/CompactionStateTest.java @@ -0,0 +1,199 @@ +/* + * 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.timeline; + +import com.google.common.collect.ImmutableList; +import org.apache.druid.data.input.impl.DimensionsSpec; +import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; +import org.apache.druid.indexer.partitions.HashedPartitionsSpec; +import org.apache.druid.query.aggregation.AggregatorFactory; +import org.apache.druid.query.aggregation.CountAggregatorFactory; +import org.apache.druid.query.aggregation.LongSumAggregatorFactory; +import org.apache.druid.segment.IndexSpec; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +import java.util.Arrays; +import java.util.Collections; +import java.util.List; + +public class CompactionStateTest +{ + @Test + public void test_generateCompactionStateFingerprint_deterministicFingerprinting() + { + CompactionState compactionState1 = createBasicCompactionState(); + CompactionState compactionState2 = createBasicCompactionState(); + + String fingerprint1 = CompactionState.generateCompactionStateFingerprint(compactionState1, "test-ds"); + String fingerprint2 = CompactionState.generateCompactionStateFingerprint(compactionState2, "test-ds"); + + Assertions.assertEquals( + fingerprint1, + fingerprint2, + "Same CompactionState should produce identical fingerprints when datasource is same" + ); + } + + @Test + public void test_generateCompactionStateFingerprint_differentDatasourcesWithSameState_differentFingerprints() + { + CompactionState compactionState = createBasicCompactionState(); + + String fingerprint1 = CompactionState.generateCompactionStateFingerprint(compactionState, "ds1"); + String fingerprint2 = CompactionState.generateCompactionStateFingerprint(compactionState, "ds2"); + + Assertions.assertNotEquals( + fingerprint1, + fingerprint2, + "Different datasources should produce different fingerprints despite same state" + ); + } + + @Test + public void test_generateCompactionStateFingerprint_metricsListOrderDifferenceResultsInNewFingerprint() + { + List metrics1 = Arrays.asList( + new CountAggregatorFactory("count"), + new LongSumAggregatorFactory("sum", "value") + ); + + List metrics2 = Arrays.asList( + new LongSumAggregatorFactory("sum", "value"), + new CountAggregatorFactory("count") + ); + + CompactionState state1 = new CompactionState( + new DynamicPartitionsSpec(null, null), + DimensionsSpec.EMPTY, + metrics1, + null, + IndexSpec.getDefault(), + null, + null + ); + + CompactionState state2 = new CompactionState( + new DynamicPartitionsSpec(null, null), + DimensionsSpec.EMPTY, + metrics2, + null, + IndexSpec.getDefault(), + null, + null + ); + + String fingerprint1 = CompactionState.generateCompactionStateFingerprint(state1, "test-ds"); + String fingerprint2 = CompactionState.generateCompactionStateFingerprint(state2, "test-ds"); + + Assertions.assertNotEquals( + fingerprint1, + fingerprint2, + "Metrics order currently matters (arrays preserve order in JSON)" + ); + } + + @Test + public void test_generateCompactionStateFingerprint_dimensionsListOrderDifferenceResultsInNewFingerprint() + { + DimensionsSpec dimensions1 = new DimensionsSpec( + DimensionsSpec.getDefaultSchemas(ImmutableList.of("dim1", "dim2", "dim3")) + ); + + DimensionsSpec dimensions2 = new DimensionsSpec( + DimensionsSpec.getDefaultSchemas(ImmutableList.of("dim3", "dim2", "dim1")) + ); + + CompactionState state1 = new CompactionState( + new DynamicPartitionsSpec(null, null), + dimensions1, + Collections.singletonList(new CountAggregatorFactory("count")), + null, + IndexSpec.getDefault(), + null, + null + ); + + CompactionState state2 = new CompactionState( + new DynamicPartitionsSpec(null, null), + dimensions2, + Collections.singletonList(new CountAggregatorFactory("count")), + null, + IndexSpec.getDefault(), + null, + null + ); + + String fingerprint1 = CompactionState.generateCompactionStateFingerprint(state1, "test-ds"); + String fingerprint2 = CompactionState.generateCompactionStateFingerprint(state2, "test-ds"); + + Assertions.assertNotEquals( + fingerprint1, + fingerprint2, + "Dimensions order currently matters (arrays preserve order in JSON)" + ); + } + + @Test + public void testGenerateCompactionStateFingerprint_differentPartitionsSpec() + { + CompactionState state1 = new CompactionState( + new DynamicPartitionsSpec(5000000, null), + DimensionsSpec.EMPTY, + Collections.singletonList(new CountAggregatorFactory("count")), + null, + IndexSpec.getDefault(), + null, + null + ); + + CompactionState state2 = new CompactionState( + new HashedPartitionsSpec(null, 2, Collections.singletonList("dim1")), + DimensionsSpec.EMPTY, + Collections.singletonList(new CountAggregatorFactory("count")), + null, + IndexSpec.getDefault(), + null, + null + ); + + String fingerprint1 = CompactionState.generateCompactionStateFingerprint(state1, "test-ds"); + String fingerprint2 = CompactionState.generateCompactionStateFingerprint(state2, "test-ds"); + + Assertions.assertNotEquals( + fingerprint1, + fingerprint2, + "Different PartitionsSpec should produce different fingerprints" + ); + } + + private CompactionState createBasicCompactionState() + { + return new CompactionState( + new DynamicPartitionsSpec(5000000, null), + DimensionsSpec.EMPTY, + Collections.singletonList(new CountAggregatorFactory("count")), + null, + IndexSpec.getDefault(), + null, + null + ); + } +} diff --git a/processing/src/test/java/org/apache/druid/timeline/DataSegmentTest.java b/processing/src/test/java/org/apache/druid/timeline/DataSegmentTest.java index 54caacdf48b5..f4ced3dfd38f 100644 --- a/processing/src/test/java/org/apache/druid/timeline/DataSegmentTest.java +++ b/processing/src/test/java/org/apache/druid/timeline/DataSegmentTest.java @@ -513,6 +513,111 @@ public void testTombstoneType() } + @Test + public void testSerializationWithCompactionStateFingerprint() throws Exception + { + final Interval interval = Intervals.of("2011-10-01/2011-10-02"); + final ImmutableMap loadSpec = ImmutableMap.of("something", "or_other"); + final String fingerprint = "abc123def456"; + final SegmentId segmentId = SegmentId.of("something", interval, "1", new NumberedShardSpec(3, 0)); + + DataSegment segment = DataSegment.builder(segmentId) + .loadSpec(loadSpec) + .dimensions(Arrays.asList("dim1", "dim2")) + .metrics(Arrays.asList("met1", "met2")) + .compactionStateFingerprint(fingerprint) + .binaryVersion(TEST_VERSION) + .size(1) + .build(); + + // Verify fingerprint is present in serialized JSON + final Map objectMap = MAPPER.readValue( + MAPPER.writeValueAsString(segment), + JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT + ); + Assert.assertEquals(fingerprint, objectMap.get("compactionStateFingerprint")); + + // Verify deserialization preserves fingerprint + DataSegment deserializedSegment = MAPPER.readValue(MAPPER.writeValueAsString(segment), DataSegment.class); + Assert.assertEquals(fingerprint, deserializedSegment.getCompactionStateFingerprint()); + Assert.assertEquals(segment.hashCode(), deserializedSegment.hashCode()); + } + + @Test + public void testSerializationWithNullCompactionStateFingerprint() throws Exception + { + final Interval interval = Intervals.of("2011-10-01/2011-10-02"); + final ImmutableMap loadSpec = ImmutableMap.of("something", "or_other"); + final SegmentId segmentId = SegmentId.of("something", interval, "1", new NumberedShardSpec(3, 0)); + + DataSegment segment = DataSegment.builder(segmentId) + .loadSpec(loadSpec) + .dimensions(Arrays.asList("dim1", "dim2")) + .metrics(Arrays.asList("met1", "met2")) + .compactionStateFingerprint(null) + .binaryVersion(TEST_VERSION) + .size(1) + .build(); + + // Verify fingerprint is NOT present in serialized JSON (due to @JsonInclude(NON_NULL)) + final Map objectMap = MAPPER.readValue( + MAPPER.writeValueAsString(segment), + JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT + ); + Assert.assertFalse("compactionStateFingerprint should not be in JSON when null", + objectMap.containsKey("compactionStateFingerprint")); + + // Verify deserialization handles missing fingerprint + DataSegment deserializedSegment = MAPPER.readValue(MAPPER.writeValueAsString(segment), DataSegment.class); + Assert.assertNull(deserializedSegment.getCompactionStateFingerprint()); + Assert.assertEquals(segment.hashCode(), deserializedSegment.hashCode()); + } + + @Test + public void testDeserializationBackwardCompatibility_missingCompactionStateFingerprint() throws Exception + { + // Simulate JSON from old Druid version without compactionStateFingerprint field + String jsonWithoutFingerprint = "{" + + "\"dataSource\": \"something\"," + + "\"interval\": \"2011-10-01T00:00:00.000Z/2011-10-02T00:00:00.000Z\"," + + "\"version\": \"1\"," + + "\"loadSpec\": {\"something\": \"or_other\"}," + + "\"dimensions\": \"dim1,dim2\"," + + "\"metrics\": \"met1,met2\"," + + "\"shardSpec\": {\"type\": \"numbered\", \"partitionNum\": 3, \"partitions\": 0}," + + "\"binaryVersion\": 9," + + "\"size\": 1" + + "}"; + + DataSegment deserializedSegment = MAPPER.readValue(jsonWithoutFingerprint, DataSegment.class); + Assert.assertNull("compactionStateFingerprint should be null for backward compatibility", + deserializedSegment.getCompactionStateFingerprint()); + Assert.assertEquals("something", deserializedSegment.getDataSource()); + Assert.assertEquals(Intervals.of("2011-10-01/2011-10-02"), deserializedSegment.getInterval()); + } + + @Test + public void testWithCompactionStateFingerprint() + { + final String fingerprint = "test_fingerprint_12345"; + final Interval interval = Intervals.of("2012-01-01/2012-01-02"); + final String version = DateTimes.of("2012-01-01T11:22:33.444Z").toString(); + final ShardSpec shardSpec = getShardSpec(7); + final SegmentId segmentId = SegmentId.of("foo", interval, version, shardSpec); + + final DataSegment segment1 = DataSegment.builder(segmentId) + .size(0) + .compactionStateFingerprint(fingerprint) + .build(); + final DataSegment segment2 = DataSegment.builder(segmentId) + .size(0) + .build(); + + DataSegment withFingerprint = segment2.withCompactionStateFingerprint(fingerprint); + Assert.assertEquals(fingerprint, withFingerprint.getCompactionStateFingerprint()); + Assert.assertEquals(segment1, withFingerprint); + } + private static void assertAllFieldsEquals(DataSegment segment1, DataSegment segment2) { Assert.assertEquals(segment1.getDataSource(), segment2.getDataSource()); diff --git a/processing/src/test/java/org/apache/druid/timeline/SegmentStatusInClusterTest.java b/processing/src/test/java/org/apache/druid/timeline/SegmentStatusInClusterTest.java index 5293f9399024..50e1c1781410 100644 --- a/processing/src/test/java/org/apache/druid/timeline/SegmentStatusInClusterTest.java +++ b/processing/src/test/java/org/apache/druid/timeline/SegmentStatusInClusterTest.java @@ -162,7 +162,8 @@ public TestSegment( @JsonProperty("binaryVersion") Integer binaryVersion, @JsonProperty("size") long size, @JsonProperty("overshadowed") boolean overshadowed, - @JsonProperty("replicationFactor") Integer replicationFactor + @JsonProperty("replicationFactor") Integer replicationFactor, + @JsonProperty("compactionStateFingerprint") String compactionStateFingerprint ) { super( @@ -177,6 +178,7 @@ public TestSegment( lastCompactionState, binaryVersion, size, + compactionStateFingerprint, PruneSpecsHolder.DEFAULT ); this.overshadowed = overshadowed; diff --git a/processing/src/test/resources/test.runtime.properties b/processing/src/test/resources/test.runtime.properties index 4f713bc66ddf..3d6a07e1e78c 100644 --- a/processing/src/test/resources/test.runtime.properties +++ b/processing/src/test/resources/test.runtime.properties @@ -31,3 +31,4 @@ druid.metadata.storage.tables.upgradeSegments=jjj_upgradeSegments druid.query.segmentMetadata.defaultAnalysisTypes=["cardinality", "size"] druid.query.segmentMetadata.defaultHistory=P2W druid.metadata.storage.tables.segmentSchemas=kkk_segmentSchemas +druid.metadata.storage.tables.compactionStates=lll_compactionStates diff --git a/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java b/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java index 4ff74ea1f448..14cd95fd7ed7 100644 --- a/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java +++ b/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java @@ -1807,7 +1807,8 @@ protected Set insertSegments( usedSegments.contains(segment), segmentMetadata == null ? null : segmentMetadata.getSchemaFingerprint(), segmentMetadata == null ? null : segmentMetadata.getNumRows(), - null + null, + segment.getCompactionStateFingerprint() ); }).collect(Collectors.toSet()); @@ -1929,7 +1930,8 @@ private Set createNewIdsOfAppendSegmentsAfterReplace( null, oldSegmentMetadata.getSchemaFingerprint(), oldSegmentMetadata.getNumRows(), - upgradedFromSegmentId + upgradedFromSegmentId, + oldSegmentMetadata.getCompactionStateFingerprint() ) ); } @@ -2021,7 +2023,8 @@ private Set insertSegments( true, segmentMetadata == null ? null : segmentMetadata.getSchemaFingerprint(), segmentMetadata == null ? null : segmentMetadata.getNumRows(), - upgradedFromSegmentIdMap.get(segment.getId().toString()) + upgradedFromSegmentIdMap.get(segment.getId().toString()), + segment.getCompactionStateFingerprint() ); }).collect(Collectors.toSet()); diff --git a/server/src/main/java/org/apache/druid/metadata/SQLMetadataConnector.java b/server/src/main/java/org/apache/druid/metadata/SQLMetadataConnector.java index 99fcfe0bc393..9d19ad7f16af 100644 --- a/server/src/main/java/org/apache/druid/metadata/SQLMetadataConnector.java +++ b/server/src/main/java/org/apache/druid/metadata/SQLMetadataConnector.java @@ -355,6 +355,7 @@ public void createSegmentTable(final String tableName) columns.add("used BOOLEAN NOT NULL"); columns.add("payload %2$s NOT NULL"); columns.add("used_status_last_updated VARCHAR(255) NOT NULL"); + columns.add("compaction_state_fingerprint VARCHAR(255)"); if (centralizedDatasourceSchemaConfig.isEnabled()) { columns.add("schema_fingerprint VARCHAR(255)"); @@ -614,6 +615,8 @@ protected void alterSegmentTable() columnNameTypes.put("upgraded_from_segment_id", "VARCHAR(255)"); + columnNameTypes.put("compaction_state_fingerprint", "VARCHAR(255)"); + if (centralizedDatasourceSchemaConfig.isEnabled()) { columnNameTypes.put("schema_fingerprint", "VARCHAR(255)"); columnNameTypes.put("num_rows", "BIGINT"); @@ -1097,6 +1100,55 @@ public void createSegmentSchemasTable() } } + /** + * Creates the compaction states table for storing fingerprinted compaction states + *

+ * This table stores unique compaction states that are referenced by + * segments via fingerprints. + */ + public void createCompactionStatesTable(final String tableName) + { + createTable( + tableName, + ImmutableList.of( + StringUtils.format( + "CREATE TABLE %1$s (\n" + + " id %2$s NOT NULL,\n" + + " created_date VARCHAR(255) NOT NULL,\n" + + " datasource VARCHAR(255) NOT NULL,\n" + + " fingerprint VARCHAR(255) NOT NULL,\n" + + " payload %3$s NOT NULL,\n" + + " used BOOLEAN NOT NULL,\n" + + " used_status_last_updated VARCHAR(255) NOT NULL,\n" + + " PRIMARY KEY (id),\n" + + " UNIQUE (fingerprint)\n" + + ")", + tableName, getSerialType(), getPayloadType() + ) + ) + ); + + createIndex( + tableName, + "IDX_%s_FINGERPRINT", + List.of("fingerprint") + ); + + createIndex( + tableName, + "IDX_%s_USED", + List.of("used", "used_status_last_updated") + ); + } + + @Override + public void createCompactionStatesTable() + { + if (config.get().isCreateTables()) { + createCompactionStatesTable(tablesConfigSupplier.get().getCompactionStatesTable()); + } + } + /** * Get the Set of the index on given table * @@ -1243,12 +1295,12 @@ private void validateSegmentsTable() (tableHasColumn(segmentsTables, "schema_fingerprint") && tableHasColumn(segmentsTables, "num_rows")); - if (tableHasColumn(segmentsTables, "used_status_last_updated") && schemaPersistenceRequirementMet) { + if (tableHasColumn(segmentsTables, "used_status_last_updated") && schemaPersistenceRequirementMet && tableHasColumn(segmentsTables, "compaction_state_fingerprint")) { // do nothing } else { throw new ISE( "Cannot start Druid as table[%s] has an incompatible schema." - + " Reason: One or all of these columns [used_status_last_updated, schema_fingerprint, num_rows] does not exist in table." + + " Reason: One or all of these columns [used_status_last_updated, schema_fingerprint, num_rows, compaction_state_fingerprint] does not exist in table." + " See https://druid.apache.org/docs/latest/operations/upgrade-prep.html for more info on remediation.", tablesConfigSupplier.get().getSegmentsTable() ); diff --git a/server/src/main/java/org/apache/druid/metadata/SqlSegmentsMetadataManagerProvider.java b/server/src/main/java/org/apache/druid/metadata/SqlSegmentsMetadataManagerProvider.java index 8491c5242267..9a632d7909b8 100644 --- a/server/src/main/java/org/apache/druid/metadata/SqlSegmentsMetadataManagerProvider.java +++ b/server/src/main/java/org/apache/druid/metadata/SqlSegmentsMetadataManagerProvider.java @@ -77,6 +77,7 @@ public void start() connector.createSegmentSchemasTable(); connector.createSegmentTable(); connector.createUpgradeSegmentsTable(); + connector.createCompactionStatesTable(); } @Override diff --git a/server/src/main/java/org/apache/druid/metadata/SqlSegmentsMetadataQuery.java b/server/src/main/java/org/apache/druid/metadata/SqlSegmentsMetadataQuery.java index 0eba529c7b4d..6a05bbc41cb0 100644 --- a/server/src/main/java/org/apache/druid/metadata/SqlSegmentsMetadataQuery.java +++ b/server/src/main/java/org/apache/druid/metadata/SqlSegmentsMetadataQuery.java @@ -585,7 +585,7 @@ private CloseableIterator retrieveSegmentBatchById( final Query> query = handle.createQuery( StringUtils.format( "SELECT payload, used, schema_fingerprint, num_rows," - + " upgraded_from_segment_id, used_status_last_updated" + + " upgraded_from_segment_id, used_status_last_updated, compaction_state_fingerprint" + " FROM %s WHERE dataSource = :dataSource %s", dbTables.getSegmentsTable(), getParameterizedInConditionForColumn("id", segmentIds) ) @@ -607,7 +607,8 @@ private CloseableIterator retrieveSegmentBatchById( r.getBoolean(2), schemaFingerprint, numRows, - r.getString(5) + r.getString(5), + r.getString(7) ); } ) @@ -615,7 +616,7 @@ private CloseableIterator retrieveSegmentBatchById( } else { final Query> query = handle.createQuery( StringUtils.format( - "SELECT payload, used, upgraded_from_segment_id, used_status_last_updated, created_date" + "SELECT payload, used, upgraded_from_segment_id, used_status_last_updated, created_date, compaction_state_fingerprint" + " FROM %s WHERE dataSource = :dataSource %s", dbTables.getSegmentsTable(), getParameterizedInConditionForColumn("id", segmentIds) ) @@ -634,7 +635,8 @@ private CloseableIterator retrieveSegmentBatchById( r.getBoolean(2), null, null, - r.getString(3) + r.getString(3), + r.getString(6) ) ) .iterator(); @@ -1722,6 +1724,7 @@ private ResultIterator getDataSegmentPlusResultIterator( used, null, null, + null, null ); } diff --git a/server/src/main/java/org/apache/druid/metadata/segment/SqlSegmentMetadataTransaction.java b/server/src/main/java/org/apache/druid/metadata/segment/SqlSegmentMetadataTransaction.java index 5c9938f8c5a3..ecfa21cc3835 100644 --- a/server/src/main/java/org/apache/druid/metadata/segment/SqlSegmentMetadataTransaction.java +++ b/server/src/main/java/org/apache/druid/metadata/segment/SqlSegmentMetadataTransaction.java @@ -245,10 +245,10 @@ public int insertSegments(Set segments) segments, "INSERT INTO %1$s " + "(id, dataSource, created_date, start, %2$send%2$s, partitioned, " - + "version, used, payload, used_status_last_updated, upgraded_from_segment_id) " + + "version, used, payload, used_status_last_updated, upgraded_from_segment_id, compaction_state_fingerprint) " + "VALUES " + "(:id, :dataSource, :created_date, :start, :end, :partitioned, " - + ":version, :used, :payload, :used_status_last_updated, :upgraded_from_segment_id)" + + ":version, :used, :payload, :used_status_last_updated, :upgraded_from_segment_id, :compaction_state_fingerprint)" ); } @@ -261,11 +261,11 @@ public int insertSegmentsWithMetadata(Set segments) "INSERT INTO %1$s " + "(id, dataSource, created_date, start, %2$send%2$s, partitioned, " + "version, used, payload, used_status_last_updated, upgraded_from_segment_id, " - + "schema_fingerprint, num_rows) " + + "schema_fingerprint, num_rows, compaction_state_fingerprint) " + "VALUES " + "(:id, :dataSource, :created_date, :start, :end, :partitioned, " + ":version, :used, :payload, :used_status_last_updated, :upgraded_from_segment_id, " - + ":schema_fingerprint, :num_rows)" + + ":schema_fingerprint, :num_rows, :compaction_state_fingerprint)" ); } @@ -532,7 +532,8 @@ private int insertSegmentsInBatches( .bind("used", Boolean.TRUE.equals(segmentPlus.getUsed())) .bind("payload", getJsonBytes(segment)) .bind("used_status_last_updated", toNonNullString(segmentPlus.getUsedStatusLastUpdatedDate())) - .bind("upgraded_from_segment_id", segmentPlus.getUpgradedFromSegmentId()); + .bind("upgraded_from_segment_id", segmentPlus.getUpgradedFromSegmentId()) + .bind("compaction_state_fingerprint", segmentPlus.getCompactionStateFingerprint()); if (persistAdditionalMetadata) { preparedBatchPart diff --git a/server/src/main/java/org/apache/druid/metadata/segment/cache/HeapMemorySegmentMetadataCache.java b/server/src/main/java/org/apache/druid/metadata/segment/cache/HeapMemorySegmentMetadataCache.java index 2a1cf50133e4..97d834da4322 100644 --- a/server/src/main/java/org/apache/druid/metadata/segment/cache/HeapMemorySegmentMetadataCache.java +++ b/server/src/main/java/org/apache/druid/metadata/segment/cache/HeapMemorySegmentMetadataCache.java @@ -785,13 +785,13 @@ private void retrieveAllUsedSegments( final String sql; if (useSchemaCache) { sql = StringUtils.format( - "SELECT id, payload, created_date, used_status_last_updated, schema_fingerprint, num_rows" + "SELECT id, payload, created_date, used_status_last_updated, compaction_state_fingerprint, schema_fingerprint, num_rows" + " FROM %s WHERE used = true", tablesConfig.getSegmentsTable() ); } else { sql = StringUtils.format( - "SELECT id, payload, created_date, used_status_last_updated" + "SELECT id, payload, created_date, used_status_last_updated, compaction_state_fingerprint" + " FROM %s WHERE used = true", tablesConfig.getSegmentsTable() ); @@ -1071,9 +1071,10 @@ private DataSegmentPlus mapToSegmentPlus(ResultSet resultSet) DateTimes.of(resultSet.getString(3)), SqlSegmentsMetadataQuery.nullAndEmptySafeDate(resultSet.getString(4)), true, - useSchemaCache ? resultSet.getString(5) : null, - useSchemaCache ? (Long) resultSet.getObject(6) : null, - null + useSchemaCache ? resultSet.getString(6) : null, + useSchemaCache ? (Long) resultSet.getObject(7) : null, + null, + resultSet.getString(5) ); } catch (Throwable t) { diff --git a/server/src/main/java/org/apache/druid/segment/metadata/CompactionStateManager.java b/server/src/main/java/org/apache/druid/segment/metadata/CompactionStateManager.java new file mode 100644 index 000000000000..b3f57734814b --- /dev/null +++ b/server/src/main/java/org/apache/druid/segment/metadata/CompactionStateManager.java @@ -0,0 +1,545 @@ +/* + * 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.segment.metadata; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.cache.Cache; +import com.google.common.cache.CacheBuilder; +import com.google.common.cache.CacheLoader; +import com.google.common.collect.Lists; +import com.google.common.util.concurrent.Striped; +import com.google.inject.Inject; +import org.apache.druid.error.InternalServerError; +import org.apache.druid.guice.ManageLifecycle; +import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.lifecycle.LifecycleStart; +import org.apache.druid.java.util.common.lifecycle.LifecycleStop; +import org.apache.druid.java.util.emitter.EmittingLogger; +import org.apache.druid.metadata.MetadataStorageTablesConfig; +import org.apache.druid.metadata.SQLMetadataConnector; +import org.apache.druid.timeline.CompactionState; +import org.joda.time.DateTime; +import org.skife.jdbi.v2.Handle; +import org.skife.jdbi.v2.PreparedBatch; +import org.skife.jdbi.v2.Query; +import org.skife.jdbi.v2.SQLStatement; +import org.skife.jdbi.v2.Update; + +import javax.annotation.Nonnull; +import javax.annotation.Nullable; +import java.io.IOException; +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; +import java.util.concurrent.locks.Lock; + +/** + * Manages the persistence and retrieval of {@link CompactionState} objects in the metadata storage. + *

+ * Compaction states are uniquely identified by their fingerprints, which are SHA-256 hashes of their content. A cache + * of compaction states using the fingerprints as keys is maintained in memory to optimize retrieval performance. + *

+ *

+ * A striped locking mechanism is used to ensure thread-safe persistence of compaction states on a per-datasource basis. + *

+ */ +@ManageLifecycle +public class CompactionStateManager +{ + private static final EmittingLogger log = new EmittingLogger(CompactionStateManager.class); + private static final int DB_ACTION_PARTITION_SIZE = 100; + + private final MetadataStorageTablesConfig dbTables; + private final ObjectMapper jsonMapper; + private final SQLMetadataConnector connector; + private final CompactionStateManagerConfig config; + private final Cache fingerprintCache; + private final Striped datasourceLocks = Striped.lock(128); + + @Inject + public CompactionStateManager( + @Nonnull MetadataStorageTablesConfig dbTables, + @Nonnull ObjectMapper jsonMapper, + @Nonnull SQLMetadataConnector connector, + @Nonnull CompactionStateManagerConfig config + ) + { + this.dbTables = dbTables; + this.jsonMapper = jsonMapper; + this.connector = connector; + this.config = config; + + this.fingerprintCache = CacheBuilder.newBuilder() + .maximumSize(config.getCacheSize()) + .build(); + } + + @LifecycleStart + public void start() + { + // This is defensive. Since the new table is created during startup after upgrade, we need to defend against + // the table not existing yet. If that is the case we do not pre-warm the cache. + try { + boolean tableExists = connector.retryWithHandle( + handle -> connector.tableExists(handle, dbTables.getCompactionStatesTable()) + ); + if (tableExists) { + log.info("Pre-warming compaction state cache"); + prewarmCache(config.getPrewarmFingerprintCount()); + } else { + log.info("Compaction states table does not exist, skipping pre-warm"); + } + } + catch (Exception e) { + log.warn(e, "Failed to prewarm cache, will load lazily"); + } + } + + @LifecycleStop + public void stop() + { + fingerprintCache.invalidateAll(); + } + + @VisibleForTesting + CompactionStateManager() + { + this.dbTables = null; + this.jsonMapper = null; + this.connector = null; + this.config = null; + this.fingerprintCache = null; + } + + /** + * Persist unique compaction state fingerprints in the DB. + *

+ * This method uses per-datasource locking to prevent concurrent insert race conditions + * when multiple threads attempt to persist the same fingerprints simultaneously. + */ + public void persistCompactionState( + final String dataSource, + final Map fingerprintToStateMap, + final DateTime updateTime + ) + { + if (fingerprintToStateMap.isEmpty()) { + return; + } + + final Lock lock = datasourceLocks.get(dataSource); + lock.lock(); + try { + connector.retryWithHandle(handle -> { + // Fetch already existing compaction state fingerprints + final Set existingFingerprints = getExistingFingerprints( + handle, + fingerprintToStateMap.keySet() + ); + + if (!existingFingerprints.isEmpty()) { + log.info( + "Found already existing compaction state in the DB for dataSource[%s]. Fingerprints: %s.", + dataSource, + existingFingerprints + ); + String setFingerprintsUsedSql = StringUtils.format( + "UPDATE %s SET used = :used, used_status_last_updated = :used_status_last_updated " + + "WHERE fingerprint = :fingerprint", + dbTables.getCompactionStatesTable() + ); + PreparedBatch markUsedBatch = handle.prepareBatch(setFingerprintsUsedSql); + for (String fingerprint : existingFingerprints) { + final String now = updateTime.toString(); + markUsedBatch.add() + .bind("used", true) + .bind("used_status_last_updated", now) + .bind("fingerprint", fingerprint); + } + markUsedBatch.execute(); + } + + Map statesToPersist = new HashMap<>(); + + for (Map.Entry entry : fingerprintToStateMap.entrySet()) { + if (!existingFingerprints.contains(entry.getKey())) { + statesToPersist.put(entry.getKey(), entry.getValue()); + } + } + + if (statesToPersist.isEmpty()) { + log.info("No compaction state to persist for dataSource [%s].", dataSource); + return null; + } + + final List> partitionedFingerprints = Lists.partition( + new ArrayList<>(statesToPersist.keySet()), + DB_ACTION_PARTITION_SIZE + ); + + String insertSql = StringUtils.format( + "INSERT INTO %s (created_date, datasource, fingerprint, payload, used, used_status_last_updated) " + + "VALUES (:created_date, :datasource, :fingerprint, :payload, :used, :used_status_last_updated)", + dbTables.getCompactionStatesTable() + ); + + // Insert compaction states + PreparedBatch stateInsertBatch = handle.prepareBatch(insertSql); + for (List partition : partitionedFingerprints) { + for (String fingerprint : partition) { + final String now = updateTime.toString(); + try { + stateInsertBatch.add() + .bind("created_date", now) + .bind("datasource", dataSource) + .bind("fingerprint", fingerprint) + .bind("payload", jsonMapper.writeValueAsBytes(fingerprintToStateMap.get(fingerprint))) + .bind("used", true) + .bind("used_status_last_updated", now); + } + catch (JsonProcessingException e) { + throw InternalServerError.exception( + e, + "Failed to serialize compaction state for fingerprint[%s]", + fingerprint + ); + } + } + final int[] affectedRows = stateInsertBatch.execute(); + final List failedInserts = new ArrayList<>(); + for (int i = 0; i < partition.size(); ++i) { + if (affectedRows[i] != 1) { + failedInserts.add(partition.get(i)); + } + } + if (failedInserts.isEmpty()) { + log.info( + "Published compaction states %s to DB for datasource[%s].", + partition, + dataSource + ); + } else { + throw new ISE( + "Failed to publish compaction states[%s] to DB for datasource[%s]", + failedInserts, + dataSource + ); + } + } + warmCache(fingerprintToStateMap); + return null; + }); + } + finally { + lock.unlock(); + } + } + + /** + * Marks compaction states as unused if they are not referenced by any used segments. + * + * @return Number of rows updated + */ + public int markUnreferencedCompactionStatesAsUnused() + { + return connector.retryWithHandle( + handle -> + handle.createStatement( + StringUtils.format( + "UPDATE %s SET used = false, used_status_last_updated = :now WHERE used = true " + + "AND fingerprint NOT IN (SELECT DISTINCT compaction_state_fingerprint FROM %s WHERE used = true AND compaction_state_fingerprint IS NOT NULL)", + dbTables.getCompactionStatesTable(), + dbTables.getSegmentsTable() + ) + ) + .bind("now", DateTimes.nowUtc().toString()) + .execute()); + } + + /** + * Finds all compaction state fingerprints which have been marked as unused but are + * still referenced by some used segments. + * + * @return Empty list if no such fingerprint exists + */ + public List findReferencedCompactionStateMarkedAsUnused() + { + return connector.retryWithHandle( + handle -> + handle.createQuery( + StringUtils.format( + "SELECT DISTINCT compaction_state_fingerprint FROM %s WHERE used = true AND compaction_state_fingerprint IN (SELECT fingerprint FROM %s WHERE used = false)", + dbTables.getSegmentsTable(), + dbTables.getCompactionStatesTable() + )) + .mapTo(String.class) + .list() + ); + } + + public int markCompactionStatesAsUsed(List stateFingerprints) + { + if (stateFingerprints.isEmpty()) { + return 0; + } + + return connector.retryWithHandle( + handle -> { + Update statement = handle.createStatement( + StringUtils.format( + "UPDATE %s SET used = true, used_status_last_updated = :now" + + " WHERE fingerprint IN (%s)", + dbTables.getCompactionStatesTable(), + buildParameterizedInClause("fp", stateFingerprints.size()) + ) + ).bind("now", DateTimes.nowUtc().toString()); + + bindValuesToInClause(stateFingerprints, "fp", statement); + + return statement.execute(); + } + ); + } + + public int deleteUnusedCompactionStatesOlderThan(long timestamp) + { + return connector.retryWithHandle( + handle -> handle.createStatement( + StringUtils.format( + "DELETE FROM %s WHERE used = false AND used_status_last_updated < :maxUpdateTime", + dbTables.getCompactionStatesTable() + )) + .bind("maxUpdateTime", DateTimes.utc(timestamp).toString()) + .execute()); + } + + /** + * Gets a compaction state by fingerprint, checking cache first. + */ + @Nullable + public CompactionState getCompactionStateByFingerprint(String fingerprint) + { + try { + return fingerprintCache.get( + fingerprint, + () -> { + CompactionState fromDb = loadCompactionStateFromDatabase(fingerprint); + if (fromDb == null) { + throw new CacheLoader.InvalidCacheLoadException("Fingerprint not found"); // Guava won't cache nulls + } + return fromDb; + } + ); + } + catch (Exception e) { + // Return null for any cache loading failure (ExecutionException, UncheckedExecutionException, InvalidCacheLoadException, etc.) + log.debug(e, "Failed to load compaction state for fingerprint[%s] from cache", fingerprint); + return null; + } + } + + /** + * Warms cache with specific states (after persisting). + */ + private void warmCache(Map fingerprintToStateMap) + { + fingerprintCache.putAll(fingerprintToStateMap); + log.debug("Warmed cache with [%d] compaction states", fingerprintToStateMap.size()); + } + + /** + * Pre-warms the cache by loading the N most recently used fingerprints. + */ + private void prewarmCache(int limit) + { + final long startTime = System.currentTimeMillis(); + log.info("Pre-warming compaction state cache with up to [%d] most recent fingerprints", limit); + + final Map recentStates = connector.retryWithHandle( + handle -> { + final String sql = StringUtils.format( + "SELECT fingerprint, payload FROM %s " + + "WHERE used = true " + + "ORDER BY used_status_last_updated DESC " + + "%s", + dbTables.getCompactionStatesTable(), + connector.limitClause(limit) + ); + + final Map states = new HashMap<>(); + handle.createQuery(sql) + .map((index, r, ctx) -> { + String fingerprint = r.getString("fingerprint"); + byte[] payload = r.getBytes("payload"); + + try { + CompactionState state = jsonMapper.readValue(payload, CompactionState.class); + states.put(fingerprint, state); + } + catch (IOException e) { + log.warn(e, "Failed to deserialize compaction state for fingerprint[%s], skipping", fingerprint); + } + return null; + }) + .list(); + + return states; + } + ); + + // Populate cache + fingerprintCache.putAll(recentStates); + + final long duration = System.currentTimeMillis() - startTime; + log.info( + "Pre-warmed cache with [%d] compaction states in [%d]ms", + recentStates.size(), + duration + ); + + } + + /** + * Invalidates a fingerprint from cache. + */ + public void invalidateFingerprint(String fingerprint) + { + fingerprintCache.invalidate(fingerprint); + } + + /** + * Loads from database. Returns null if not found or unused. + */ + @Nullable + private CompactionState loadCompactionStateFromDatabase(String fingerprint) + { + return connector.retryWithHandle( + handle -> { + List results = handle.createQuery( + StringUtils.format( + "SELECT payload FROM %s WHERE fingerprint = :fingerprint AND used = true", + dbTables.getCompactionStatesTable() + )) + .bind("fingerprint", fingerprint) + .map((index, r, ctx) -> r.getBytes("payload")) + .list(); + + if (results.isEmpty()) { + return null; + } + + try { + return jsonMapper.readValue(results.get(0), CompactionState.class); + } + catch (IOException e) { + log.error(e, "Failed to deserialize compaction state for fingerprint[%s]", fingerprint); + return null; + } + } + ); + } + + /** + * Query the metadata DB to filter the fingerprints that already exist. + **/ + private Set getExistingFingerprints( + final Handle handle, + final Set fingerprintsToInsert + ) + { + if (fingerprintsToInsert.isEmpty()) { + return Collections.emptySet(); + } + + List> partitionedFingerprints = Lists.partition( + new ArrayList<>(fingerprintsToInsert), + DB_ACTION_PARTITION_SIZE + ); + + final Set existingFingerprints = new HashSet<>(); + for (List fingerprintList : partitionedFingerprints) { + Query query = handle.createQuery( + StringUtils.format( + "SELECT fingerprint FROM %s WHERE fingerprint IN (%s)", + dbTables.getCompactionStatesTable(), + buildParameterizedInClause("fp", fingerprintList.size()) + ) + ); + + bindValuesToInClause(fingerprintList, "fp", query); + + query.map((index, r, ctx) -> existingFingerprints.add(r.getString(1))) + .list(); + } + return existingFingerprints; + } + + @VisibleForTesting + protected boolean isCached(String fingerprint) + { + return fingerprintCache.getIfPresent(fingerprint) != null; + } + + /** + * Builds a parameterized IN clause for the specified column with placeholders. + * Must be followed by a call to {@link #bindValuesToInClause(List, String, SQLStatement)}. + * + * @param parameterPrefix prefix for parameter names (e.g., "fingerprint") + * @param valueCount number of values in the IN clause + * @return parameterized IN clause like "(?, ?, ?)" but with named parameters + */ + private static String buildParameterizedInClause(String parameterPrefix, int valueCount) + { + StringBuilder sb = new StringBuilder(); + for (int i = 0; i < valueCount; i++) { + sb.append(":").append(parameterPrefix).append(i); + if (i != valueCount - 1) { + sb.append(","); + } + } + return sb.toString(); + } + + /** + * Binds values to a parameterized IN clause in a SQL query. + * + * @param values list of values to bind + * @param parameterPrefix prefix used when building the IN clause + * @param query the SQL statement to bind values to + */ + private static void bindValuesToInClause( + List values, + String parameterPrefix, + SQLStatement query + ) + { + for (int i = 0; i < values.size(); i++) { + query.bind(parameterPrefix + i, values.get(i)); + } + } +} diff --git a/server/src/main/java/org/apache/druid/segment/metadata/CompactionStateManagerConfig.java b/server/src/main/java/org/apache/druid/segment/metadata/CompactionStateManagerConfig.java new file mode 100644 index 000000000000..04006931b8e5 --- /dev/null +++ b/server/src/main/java/org/apache/druid/segment/metadata/CompactionStateManagerConfig.java @@ -0,0 +1,47 @@ +/* + * 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.segment.metadata; + +import com.fasterxml.jackson.annotation.JsonProperty; + +public class CompactionStateManagerConfig +{ + /** + * The maximum number of fingerprints to keep in the in-memory cache. + */ + @JsonProperty + private int cacheSize = 100; + + /** + * The number of fingerprints to prewarm into the cache on startup. + */ + @JsonProperty + private int prewarmFingerprintCount = 100; + + public int getCacheSize() + { + return cacheSize; + } + + public int getPrewarmFingerprintCount() + { + return prewarmFingerprintCount; + } +} diff --git a/server/src/main/java/org/apache/druid/segment/metadata/HeapMemoryCompactionStateManager.java b/server/src/main/java/org/apache/druid/segment/metadata/HeapMemoryCompactionStateManager.java new file mode 100644 index 000000000000..e0d8fdcf545c --- /dev/null +++ b/server/src/main/java/org/apache/druid/segment/metadata/HeapMemoryCompactionStateManager.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.segment.metadata; + +import org.apache.druid.timeline.CompactionState; +import org.joda.time.DateTime; + +import javax.annotation.Nullable; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; + +/** + * In-memory implementation of {@link CompactionStateManager} that stores + * compaction state fingerprints in heap memory without requiring a database. + *

+ * Useful for simulations and unit tests where database persistence is not needed. + */ +public class HeapMemoryCompactionStateManager extends CompactionStateManager +{ + private final Map fingerprintToStateMap = new ConcurrentHashMap<>(); + + @Override + public void persistCompactionState( + final String dataSource, + final Map fingerprintToStateMap, + final DateTime updateTime + ) + { + // Store in memory for lookup during simulations/tests + this.fingerprintToStateMap.putAll(fingerprintToStateMap); + } + + @Override + @Nullable + public CompactionState getCompactionStateByFingerprint(String fingerprint) + { + return fingerprintToStateMap.get(fingerprint); + } + + /** + * Clears all stored compaction states. Useful for test cleanup or resetting + * state between test runs. + */ + public void clear() + { + fingerprintToStateMap.clear(); + } + + /** + * Returns the number of stored compaction state fingerprints. + */ + public int size() + { + return fingerprintToStateMap.size(); + } + + /** + * Checks if a fingerprint exists in the store. + */ + public boolean containsFingerprint(String fingerprint) + { + return fingerprintToStateMap.containsKey(fingerprint); + } +} 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 58fe28e6ea96..60c97e4766a3 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 @@ -32,6 +32,8 @@ import org.apache.druid.metadata.LockFilterPolicy; import org.apache.druid.rpc.indexing.NoopOverlordClient; import org.apache.druid.rpc.indexing.OverlordClient; +import org.apache.druid.segment.metadata.CompactionStateManager; +import org.apache.druid.segment.metadata.HeapMemoryCompactionStateManager; import org.apache.druid.server.coordinator.ClusterCompactionConfig; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; import org.apache.druid.server.coordinator.DruidCompactionConfig; @@ -138,11 +140,14 @@ public void onTaskSubmitted(String taskId, CompactionCandidate candidateSegments Integer.MAX_VALUE, clusterConfig.getCompactionPolicy(), clusterConfig.isUseSupervisors(), - clusterConfig.getEngine() + clusterConfig.getEngine(), + clusterConfig.isLegacyPersistLastCompactionStateInSegments() ); final CoordinatorRunStats stats = new CoordinatorRunStats(); - new CompactSegments(simulationStatusTracker, readOnlyOverlordClient).run( + // Use an in-memory CompactionStateManager for simulations + final CompactionStateManager inMemoryStateManager = new HeapMemoryCompactionStateManager(); + new CompactSegments(simulationStatusTracker, readOnlyOverlordClient, inMemoryStateManager).run( compactionConfig.withClusterConfig(configWithUnlimitedTaskSlots), dataSourcesSnapshot, defaultEngine, 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 cc52513b16c5..6761c2529f82 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 @@ -30,11 +30,14 @@ import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.common.granularity.GranularityType; +import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.segment.IndexSpec; +import org.apache.druid.segment.metadata.CompactionStateManager; import org.apache.druid.segment.transform.CompactionTransformSpec; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; import org.apache.druid.server.coordinator.UserCompactionTaskGranularityConfig; +import org.apache.druid.server.coordinator.duty.CompactSegments; import org.apache.druid.timeline.CompactionState; import org.apache.druid.timeline.DataSegment; import org.apache.druid.utils.CollectionUtils; @@ -55,6 +58,8 @@ */ public class CompactionStatus { + private static final Logger log = new Logger(CompactionStatus.class); + private static final CompactionStatus COMPLETE = new CompactionStatus(State.COMPLETE, null, null, null); public enum State @@ -62,13 +67,19 @@ public enum State COMPLETE, PENDING, RUNNING, SKIPPED } + /** + * List of checks performed to determine if compaction is already complete based on compaction state fingerprints. + */ + private static final List> FINGERPRINT_CHECKS = List.of( + Evaluator::allFingerprintedCandidatesHaveExpectedFingerprint + ); + /** * List of checks performed to determine if compaction is already complete. *

* The order of the checks must be honored while evaluating them. */ private static final List> CHECKS = Arrays.asList( - Evaluator::segmentsHaveBeenCompactedAtLeastOnce, Evaluator::partitionsSpecIsUpToDate, Evaluator::indexSpecIsUpToDate, Evaluator::segmentGranularityIsUpToDate, @@ -249,14 +260,17 @@ public static CompactionStatus running(String message) */ static CompactionStatus compute( CompactionCandidate candidateSegments, - DataSourceCompactionConfig config + DataSourceCompactionConfig config, + CompactionStateManager compactionStateManager ) { - return new Evaluator(candidateSegments, config).evaluate(); + final CompactionState expectedState = CompactSegments.createCompactionStateFromConfig(config); + final String expectedFingerprint = CompactionState.generateCompactionStateFingerprint(expectedState, config.getDataSource()); + return new Evaluator(candidateSegments, config, expectedFingerprint, compactionStateManager).evaluate(); } @Nullable - static PartitionsSpec findPartitionsSpecFromConfig(ClientCompactionTaskQueryTuningConfig tuningConfig) + public static PartitionsSpec findPartitionsSpecFromConfig(ClientCompactionTaskQueryTuningConfig tuningConfig) { final PartitionsSpec partitionsSpecFromTuningConfig = tuningConfig.getPartitionsSpec(); if (partitionsSpecFromTuningConfig == null) { @@ -330,18 +344,28 @@ private static class Evaluator private final ClientCompactionTaskQueryTuningConfig tuningConfig; private final UserCompactionTaskGranularityConfig configuredGranularitySpec; + private final List fingerprintedSegments = new ArrayList<>(); + private final List compactedSegments = new ArrayList<>(); private final List uncompactedSegments = new ArrayList<>(); private final Map> unknownStateToSegments = new HashMap<>(); + @Nullable + private final String targetFingerprint; + private final CompactionStateManager compactionStateManager; + private Evaluator( CompactionCandidate candidateSegments, - DataSourceCompactionConfig compactionConfig + DataSourceCompactionConfig compactionConfig, + @Nullable String targetFingerprint, + CompactionStateManager compactionStateManager ) { this.candidateSegments = candidateSegments; this.compactionConfig = compactionConfig; this.tuningConfig = ClientCompactionTaskQueryTuningConfig.from(compactionConfig); this.configuredGranularitySpec = compactionConfig.getGranularitySpec(); + this.targetFingerprint = targetFingerprint; + this.compactionStateManager = compactionStateManager; } private CompactionStatus evaluate() @@ -351,40 +375,141 @@ private CompactionStatus evaluate() return inputBytesCheck; } - final List reasonsForCompaction = + List reasonsForCompaction = new ArrayList<>(); + CompactionStatus compactedOnceCheck = segmentsHaveBeenCompactedAtLeastOnce(); + if (!compactedOnceCheck.isComplete()) { + reasonsForCompaction.add(compactedOnceCheck.getReason()); + } + + // First try fingerprint-based evaluation (fast path) + CompactionStatus fingerprintStatus = FINGERPRINT_CHECKS.stream() + .map(f -> f.apply(this)) + .filter(status -> !status.isComplete()) + .findFirst().orElse(COMPLETE); + + if (!fingerprintStatus.isComplete()) { + reasonsForCompaction.add(fingerprintStatus.getReason()); + } + + reasonsForCompaction.addAll( CHECKS.stream() .map(f -> f.apply(this)) .filter(status -> !status.isComplete()) .map(CompactionStatus::getReason) - .collect(Collectors.toList()); + .collect(Collectors.toList()) + ); // Consider segments which have passed all checks to be compacted - final List compactedSegments = unknownStateToSegments - .values() - .stream() - .flatMap(List::stream) - .collect(Collectors.toList()); + // Includes segments with correct fingerprints and segments that passed all state checks + final List allCompactedSegments = new ArrayList<>(this.compactedSegments); + allCompactedSegments.addAll( + unknownStateToSegments + .values() + .stream() + .flatMap(List::stream) + .collect(Collectors.toList()) + ); if (reasonsForCompaction.isEmpty()) { return COMPLETE; } else { return CompactionStatus.pending( - createStats(compactedSegments), + createStats(allCompactedSegments), createStats(uncompactedSegments), reasonsForCompaction.get(0) ); } } + /** + * Evaluates the fingerprints of all fingerprinted candidate segments against the expected fingerprint. + *

+ * If all fingerprinted segments have the expected fingerprint, the check can quickly pass as COMPLETE. However, + * if any fingerprinted segment has a mismatched fingerprint, we need to investigate further by adding them to + * {@link #unknownStateToSegments} where their compaction states will be analyzed. + *

+ */ + private CompactionStatus allFingerprintedCandidatesHaveExpectedFingerprint() + { + Map> mismatchedFingerprintToSegmentMap = new HashMap<>(); + for (DataSegment segment : fingerprintedSegments) { + String fingerprint = segment.getCompactionStateFingerprint(); + if (fingerprint != null && !fingerprint.equals(targetFingerprint)) { + mismatchedFingerprintToSegmentMap + .computeIfAbsent(fingerprint, k -> new ArrayList<>()) + .add(segment); + } else if (fingerprint != null && fingerprint.equals(targetFingerprint)) { + // Segment has correct fingerprint - add to compacted segments + compactedSegments.add(segment); + } + } + + if (mismatchedFingerprintToSegmentMap.isEmpty()) { + return COMPLETE; + } + + boolean fingerprintedSegmentNeedingCompactionFound = false; + + if (compactionStateManager != null) { + for (Map.Entry> e : mismatchedFingerprintToSegmentMap.entrySet()) { + String fingerprint = e.getKey(); + CompactionState stateToValidate = compactionStateManager.getCompactionStateByFingerprint(fingerprint); + if (stateToValidate == null) { + log.warn("No compaction state found for fingerprint[%s]", fingerprint); + fingerprintedSegmentNeedingCompactionFound = true; + uncompactedSegments.addAll(e.getValue()); + } else { + // Note that this does not mean we need compaction yet - we need to validate the state further to determine this + unknownStateToSegments.compute( + stateToValidate, + (state, segments) -> { + if (segments == null) { + segments = new ArrayList<>(); + } + segments.addAll(e.getValue()); + return segments; + } + ); + } + } + } else { + for (Map.Entry> e : mismatchedFingerprintToSegmentMap.entrySet()) { + uncompactedSegments.addAll(e.getValue()); + fingerprintedSegmentNeedingCompactionFound = true; + } + } + + if (fingerprintedSegmentNeedingCompactionFound) { + return CompactionStatus.pending("At least one segment has a mismatched fingerprint and needs compaction"); + } else { + return COMPLETE; + } + } + + /** + * Divvys up segments by certain characteristics and determines if any segments have never been compacted. + *

+ * Segments are categorized into three groups: + *

    + *
  • fingerprinted - segments who have a compaction state fingerprint and need more investigation before adding to {@link #unknownStateToSegments}
  • + *
  • non-fingerprinted with a lastCompactionState - segments who have no fingerprint but have stored a lastCompactionState that needs to be analyzed
  • + *
  • uncompacted - segments who have neither a fingerprint nor a lastCompactionState and thus definitely need compaction
  • + *
+ *

+ */ private CompactionStatus segmentsHaveBeenCompactedAtLeastOnce() { - // Identify the compaction states of all the segments for (DataSegment segment : candidateSegments.getSegments()) { - final CompactionState segmentState = segment.getLastCompactionState(); - if (segmentState == null) { - uncompactedSegments.add(segment); + final String fingerprint = segment.getCompactionStateFingerprint(); + if (fingerprint != null) { + fingerprintedSegments.add(segment); } else { - unknownStateToSegments.computeIfAbsent(segmentState, s -> new ArrayList<>()).add(segment); + final CompactionState segmentState = segment.getLastCompactionState(); + if (segmentState == null) { + uncompactedSegments.add(segment); + } else { + unknownStateToSegments.computeIfAbsent(segmentState, k -> new ArrayList<>()).add(segment); + } } } 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 9b87b0409f69..c88fd264f82b 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 @@ -30,6 +30,7 @@ import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.common.guava.Comparators; import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.segment.metadata.CompactionStateManager; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.Partitions; @@ -68,6 +69,7 @@ public class DataSourceCompactibleSegmentIterator implements CompactionSegmentIt private final String dataSource; private final DataSourceCompactionConfig config; + private final CompactionStateManager compactionStateManager; private final List compactedSegments = new ArrayList<>(); private final List skippedSegments = new ArrayList<>(); @@ -84,12 +86,14 @@ public DataSourceCompactibleSegmentIterator( DataSourceCompactionConfig config, SegmentTimeline timeline, List skipIntervals, - CompactionCandidateSearchPolicy searchPolicy + CompactionCandidateSearchPolicy searchPolicy, + CompactionStateManager compactionStateManager ) { this.config = config; this.dataSource = config.getDataSource(); this.queue = new PriorityQueue<>(searchPolicy::compareCandidates); + this.compactionStateManager = compactionStateManager; populateQueue(timeline, skipIntervals); } @@ -326,7 +330,7 @@ private void findAndEnqueueSegmentsToCompact(CompactibleSegmentIterator compacti } final CompactionCandidate candidates = CompactionCandidate.from(segments, config.getSegmentGranularity()); - final CompactionStatus compactionStatus = CompactionStatus.compute(candidates, config); + final CompactionStatus compactionStatus = CompactionStatus.compute(candidates, config, compactionStateManager); final CompactionCandidate candidatesWithStatus = candidates.withCurrentStatus(compactionStatus); if (compactionStatus.isComplete()) { 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 49d936fda0ac..bdf945a10716 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 @@ -22,6 +22,7 @@ import com.google.common.collect.Maps; import org.apache.druid.error.DruidException; import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.segment.metadata.CompactionStateManager; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; import org.apache.druid.timeline.SegmentTimeline; import org.joda.time.Interval; @@ -48,7 +49,8 @@ public PriorityBasedCompactionSegmentIterator( CompactionCandidateSearchPolicy searchPolicy, Map compactionConfigs, Map datasourceToTimeline, - Map> skipIntervals + Map> skipIntervals, + CompactionStateManager compactionStateManager ) { this.queue = new PriorityQueue<>(searchPolicy::compareCandidates); @@ -69,7 +71,8 @@ public PriorityBasedCompactionSegmentIterator( compactionConfigs.get(datasource), timeline, skipIntervals.getOrDefault(datasource, Collections.emptyList()), - searchPolicy + searchPolicy, + compactionStateManager ) ); addNextItemForDatasourceToQueue(datasource); diff --git a/server/src/main/java/org/apache/druid/server/coordination/LoadableDataSegment.java b/server/src/main/java/org/apache/druid/server/coordination/LoadableDataSegment.java index 1912ecdbe85b..72281b00e98b 100644 --- a/server/src/main/java/org/apache/druid/server/coordination/LoadableDataSegment.java +++ b/server/src/main/java/org/apache/druid/server/coordination/LoadableDataSegment.java @@ -53,7 +53,8 @@ private LoadableDataSegment( @JsonProperty("shardSpec") @Nullable ShardSpec shardSpec, @JsonProperty("lastCompactionState") @Nullable CompactionState lastCompactionState, @JsonProperty("binaryVersion") Integer binaryVersion, - @JsonProperty("size") long size + @JsonProperty("size") long size, + @JsonProperty("compactionStateFingerprint") String compactionStateFingerprint ) { super( @@ -68,6 +69,7 @@ private LoadableDataSegment( lastCompactionState, binaryVersion, size, + compactionStateFingerprint, PruneSpecsHolder.DEFAULT ); } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/ClusterCompactionConfig.java b/server/src/main/java/org/apache/druid/server/coordinator/ClusterCompactionConfig.java index 437849ad1bf2..7e608e6f6a3e 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/ClusterCompactionConfig.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/ClusterCompactionConfig.java @@ -45,6 +45,14 @@ public class ClusterCompactionConfig private final boolean useSupervisors; private final CompactionEngine engine; private final CompactionCandidateSearchPolicy compactionPolicy; + /** + * Whether to persist last compaction state directly in segments for backwards compatibility. + *

+ * In a future release this option will be removed and last compaction state will no longer be persisted in segments. + * Instead, it will only be stored in the metadata store with a fingerprint id that segments will reference. Some + * operators may want to disable this behavior early to begin saving space in segment metadatastore table entries. + */ + private final boolean legacyPersistLastCompactionStateInSegments; @JsonCreator public ClusterCompactionConfig( @@ -52,7 +60,8 @@ public ClusterCompactionConfig( @JsonProperty("maxCompactionTaskSlots") @Nullable Integer maxCompactionTaskSlots, @JsonProperty("compactionPolicy") @Nullable CompactionCandidateSearchPolicy compactionPolicy, @JsonProperty("useSupervisors") @Nullable Boolean useSupervisors, - @JsonProperty("engine") @Nullable CompactionEngine engine + @JsonProperty("engine") @Nullable CompactionEngine engine, + @JsonProperty("legacyPersistLastCompactionStateInSegments") Boolean legacyPersistLastCompactionStateInSegments ) { this.compactionTaskSlotRatio = Configs.valueOrDefault(compactionTaskSlotRatio, 0.1); @@ -60,6 +69,10 @@ public ClusterCompactionConfig( this.compactionPolicy = Configs.valueOrDefault(compactionPolicy, DEFAULT_COMPACTION_POLICY); this.engine = Configs.valueOrDefault(engine, CompactionEngine.NATIVE); this.useSupervisors = Configs.valueOrDefault(useSupervisors, false); + this.legacyPersistLastCompactionStateInSegments = Configs.valueOrDefault( + legacyPersistLastCompactionStateInSegments, + true + ); if (!this.useSupervisors && this.engine == CompactionEngine.MSQ) { throw InvalidInput.exception("MSQ Compaction engine can be used only with compaction supervisors."); @@ -96,6 +109,12 @@ public CompactionEngine getEngine() return engine; } + @JsonProperty + public boolean isLegacyPersistLastCompactionStateInSegments() + { + return legacyPersistLastCompactionStateInSegments; + } + @Override public boolean equals(Object o) { @@ -110,7 +129,8 @@ public boolean equals(Object o) && Objects.equals(maxCompactionTaskSlots, that.maxCompactionTaskSlots) && Objects.equals(compactionPolicy, that.compactionPolicy) && Objects.equals(useSupervisors, that.useSupervisors) - && Objects.equals(engine, that.engine); + && Objects.equals(engine, that.engine) + && Objects.equals(legacyPersistLastCompactionStateInSegments, that.legacyPersistLastCompactionStateInSegments); } @Override @@ -121,7 +141,8 @@ public int hashCode() maxCompactionTaskSlots, compactionPolicy, useSupervisors, - engine + engine, + legacyPersistLastCompactionStateInSegments ); } @@ -134,6 +155,7 @@ public String toString() ", useSupervisors=" + useSupervisors + ", engine=" + engine + ", compactionPolicy=" + compactionPolicy + + ", legacyPersistLastCompactionStateInSegments=" + legacyPersistLastCompactionStateInSegments + '}'; } } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/CoordinatorConfigManager.java b/server/src/main/java/org/apache/druid/server/coordinator/CoordinatorConfigManager.java index 5d435549bacd..65397332b0c9 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/CoordinatorConfigManager.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/CoordinatorConfigManager.java @@ -166,7 +166,8 @@ public boolean updateCompactionTaskSlots( Configs.valueOrDefault(maxCompactionTaskSlots, currentClusterConfig.getMaxCompactionTaskSlots()), currentClusterConfig.getCompactionPolicy(), currentClusterConfig.isUseSupervisors(), - currentClusterConfig.getEngine() + currentClusterConfig.getEngine(), + currentClusterConfig.isLegacyPersistLastCompactionStateInSegments() ); return current.withClusterConfig(updatedClusterConfig); diff --git a/server/src/main/java/org/apache/druid/server/coordinator/DruidCompactionConfig.java b/server/src/main/java/org/apache/druid/server/coordinator/DruidCompactionConfig.java index cc11d9fdf718..9059ab82add6 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/DruidCompactionConfig.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/DruidCompactionConfig.java @@ -40,7 +40,7 @@ public class DruidCompactionConfig public static final String CONFIG_KEY = "coordinator.compaction.config"; private static final DruidCompactionConfig EMPTY_INSTANCE - = new DruidCompactionConfig(List.of(), null, null, null, null, null); + = new DruidCompactionConfig(List.of(), null, null, null, null, null, null); private final List compactionConfigs; private final ClusterCompactionConfig clusterConfig; @@ -86,7 +86,8 @@ public DruidCompactionConfig( @JsonProperty("maxCompactionTaskSlots") @Nullable Integer maxCompactionTaskSlots, @JsonProperty("compactionPolicy") @Nullable CompactionCandidateSearchPolicy compactionPolicy, @JsonProperty("useSupervisors") @Nullable Boolean useSupervisors, - @JsonProperty("engine") @Nullable CompactionEngine engine + @JsonProperty("engine") @Nullable CompactionEngine engine, + @JsonProperty("legacyPersistLastCompactionStateInSegments") @Nullable Boolean legacyPersistLastCompactionStateInSegments ) { this( @@ -96,7 +97,8 @@ public DruidCompactionConfig( maxCompactionTaskSlots, compactionPolicy, useSupervisors, - engine + engine, + legacyPersistLastCompactionStateInSegments ) ); } @@ -140,6 +142,12 @@ public CompactionEngine getEngine() return clusterConfig.getEngine(); } + @JsonProperty + public boolean isLegacyPersistLastCompactionStateInSegments() + { + return clusterConfig.isLegacyPersistLastCompactionStateInSegments(); + } + /** * Returns the cluster-level compaction config. Not used for serialization. */ diff --git a/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java b/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java index de6ef45a0533..20028b60e586 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java @@ -53,6 +53,7 @@ import org.apache.druid.rpc.indexing.OverlordClient; import org.apache.druid.rpc.indexing.SegmentUpdateResponse; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; +import org.apache.druid.segment.metadata.CompactionStateManager; import org.apache.druid.segment.metadata.CoordinatorSegmentMetadataCache; import org.apache.druid.server.DruidNode; import org.apache.druid.server.compaction.CompactionRunSimulator; @@ -76,6 +77,7 @@ import org.apache.druid.server.coordinator.duty.KillRules; import org.apache.druid.server.coordinator.duty.KillStalePendingSegments; import org.apache.druid.server.coordinator.duty.KillSupervisors; +import org.apache.druid.server.coordinator.duty.KillUnreferencedCompactionState; import org.apache.druid.server.coordinator.duty.KillUnreferencedSegmentSchema; import org.apache.druid.server.coordinator.duty.KillUnusedSegments; import org.apache.druid.server.coordinator.duty.MarkEternityTombstonesAsUnused; @@ -144,6 +146,7 @@ public class DruidCoordinator private final CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig; private final CoordinatorDynamicConfigSyncer coordinatorDynamicConfigSyncer; private final CloneStatusManager cloneStatusManager; + private final CompactionStateManager compactionStateManager; private volatile boolean started = false; @@ -190,7 +193,8 @@ public DruidCoordinator( CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig, CompactionStatusTracker compactionStatusTracker, CoordinatorDynamicConfigSyncer coordinatorDynamicConfigSyncer, - CloneStatusManager cloneStatusManager + CloneStatusManager cloneStatusManager, + CompactionStateManager compactionStateManager ) { this.config = config; @@ -215,6 +219,7 @@ public DruidCoordinator( this.coordinatorDynamicConfigSyncer = coordinatorDynamicConfigSyncer; this.cloneStatusManager = cloneStatusManager; + this.compactionStateManager = compactionStateManager; this.compactSegments = initializeCompactSegmentsDuty(this.compactionStatusTracker); } @@ -609,6 +614,7 @@ private List makeMetadataStoreManagementDuties() duties.add( new KillCompactionConfig(killConfigs.compactionConfigs(), metadataManager.indexer(), metadataManager.configs()) ); + duties.add(new KillUnreferencedCompactionState(killConfigs.compactionStates(), metadataManager.compactionStates())); if (centralizedDatasourceSchemaConfig.isEnabled()) { duties.add(new KillUnreferencedSegmentSchema(killConfigs.segmentSchemas(), metadataManager.schemas())); } @@ -619,7 +625,7 @@ private CompactSegments initializeCompactSegmentsDuty(CompactionStatusTracker st { List compactSegmentsDutyFromCustomGroups = getCompactSegmentsDutyFromCustomGroups(); if (compactSegmentsDutyFromCustomGroups.isEmpty()) { - return new CompactSegments(statusTracker, overlordClient); + return new CompactSegments(statusTracker, overlordClient, compactionStateManager); } else { if (compactSegmentsDutyFromCustomGroups.size() > 1) { log.warn( diff --git a/server/src/main/java/org/apache/druid/server/coordinator/MetadataManager.java b/server/src/main/java/org/apache/druid/server/coordinator/MetadataManager.java index 91df6ae428f8..be7eea2f3079 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/MetadataManager.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/MetadataManager.java @@ -26,6 +26,7 @@ import org.apache.druid.metadata.MetadataSupervisorManager; import org.apache.druid.metadata.SegmentsMetadataManager; import org.apache.druid.metadata.segment.cache.SegmentMetadataCache; +import org.apache.druid.segment.metadata.CompactionStateManager; import org.apache.druid.segment.metadata.SegmentSchemaManager; import org.apache.druid.timeline.DataSegment; @@ -42,6 +43,7 @@ public class MetadataManager private final IndexerMetadataStorageCoordinator storageCoordinator; private final SegmentSchemaManager segmentSchemaManager; private final SegmentMetadataCache segmentMetadataCache; + private final CompactionStateManager compactionStateManager; @Inject public MetadataManager( @@ -52,7 +54,8 @@ public MetadataManager( MetadataRuleManager metadataRuleManager, IndexerMetadataStorageCoordinator storageCoordinator, SegmentSchemaManager segmentSchemaManager, - SegmentMetadataCache segmentMetadataCache + SegmentMetadataCache segmentMetadataCache, + CompactionStateManager compactionStateManager ) { this.auditManager = auditManager; @@ -63,6 +66,7 @@ public MetadataManager( this.storageCoordinator = storageCoordinator; this.segmentSchemaManager = segmentSchemaManager; this.segmentMetadataCache = segmentMetadataCache; + this.compactionStateManager = compactionStateManager; } public void onLeaderStart() @@ -131,6 +135,11 @@ public SegmentSchemaManager schemas() return segmentSchemaManager; } + public CompactionStateManager compactionStates() + { + return compactionStateManager; + } + /** * Returns an iterable to go over all segments in all data sources. The order in which segments are iterated is * unspecified. Note: the iteration may not be as trivially cheap as, for example, iteration over an ArrayList. Try diff --git a/server/src/main/java/org/apache/druid/server/coordinator/config/CoordinatorKillConfigs.java b/server/src/main/java/org/apache/druid/server/coordinator/config/CoordinatorKillConfigs.java index c1f5f8bca67b..a7454a7878fa 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/config/CoordinatorKillConfigs.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/config/CoordinatorKillConfigs.java @@ -28,7 +28,7 @@ public class CoordinatorKillConfigs { public static CoordinatorKillConfigs DEFAULT - = new CoordinatorKillConfigs(null, null, null, null, null, null, null, null, null, null, null, null, null, null); + = new CoordinatorKillConfigs(null, null, null, null, null, null, null, null, null, null, null, null, null, null, null); @JsonProperty("supervisor") private final MetadataCleanupConfig supervisors; @@ -74,6 +74,9 @@ public class CoordinatorKillConfigs @JsonProperty("maxInterval") private final Period killUnusedMaxInterval; + @JsonProperty("compactionState") + private final MetadataCleanupConfig compactionStates; + @JsonCreator public CoordinatorKillConfigs( @JsonProperty("pendingSegments") MetadataCleanupConfig pendingSegments, @@ -83,6 +86,7 @@ public CoordinatorKillConfigs( @JsonProperty("rule") MetadataCleanupConfig rules, @JsonProperty("compaction") MetadataCleanupConfig compactionConfigs, @JsonProperty("segmentSchema") MetadataCleanupConfig segmentSchemas, + @JsonProperty("compactionState") MetadataCleanupConfig compactionStates, // Configs for cleanup of unused segments @JsonProperty("on") Boolean killUnusedEnabled, @JsonProperty("period") Duration killUnusedPeriod, @@ -100,6 +104,7 @@ public CoordinatorKillConfigs( this.rules = Configs.valueOrDefault(rules, MetadataCleanupConfig.DEFAULT); this.compactionConfigs = Configs.valueOrDefault(compactionConfigs, MetadataCleanupConfig.DEFAULT); this.segmentSchemas = Configs.valueOrDefault(segmentSchemas, MetadataCleanupConfig.DEFAULT); + this.compactionStates = Configs.valueOrDefault(compactionStates, MetadataCleanupConfig.DEFAULT); this.killUnusedEnabled = killUnusedEnabled; this.killUnusedPeriod = killUnusedPeriod; @@ -145,6 +150,11 @@ public MetadataCleanupConfig segmentSchemas() return segmentSchemas; } + public MetadataCleanupConfig compactionStates() + { + return compactionStates; + } + /** * Creates a KillUnusedSegmentsConfig. This config is initialized lazily as * it uses the indexingPeriod as the default cleanup period. diff --git a/server/src/main/java/org/apache/druid/server/coordinator/config/DruidCoordinatorConfig.java b/server/src/main/java/org/apache/druid/server/coordinator/config/DruidCoordinatorConfig.java index 6004c5b1ba47..29f43014bf06 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/config/DruidCoordinatorConfig.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/config/DruidCoordinatorConfig.java @@ -96,6 +96,7 @@ private void validateKillConfigs() validateKillConfig(killConfigs.rules(), "rule"); validateKillConfig(killConfigs.supervisors(), "supervisor"); validateKillConfig(killConfigs.segmentSchemas(), "segmentSchema"); + validateKillConfig(killConfigs.compactionStates(), "compactionState"); // Validate config for killing unused segments final KillUnusedSegmentsConfig killUnusedConfig 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 9947e521f657..348119f57400 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java @@ -34,12 +34,19 @@ 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.data.input.impl.DimensionsSpec; import org.apache.druid.indexer.CompactionEngine; +import org.apache.druid.indexer.granularity.GranularitySpec; +import org.apache.druid.indexer.granularity.UniformGranularitySpec; +import org.apache.druid.indexer.partitions.PartitionsSpec; +import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.common.granularity.GranularityType; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.rpc.indexing.OverlordClient; +import org.apache.druid.segment.IndexSpec; +import org.apache.druid.segment.metadata.CompactionStateManager; import org.apache.druid.segment.transform.CompactionTransformSpec; import org.apache.druid.server.compaction.CompactionCandidate; import org.apache.druid.server.compaction.CompactionCandidateSearchPolicy; @@ -53,14 +60,17 @@ import org.apache.druid.server.coordinator.DataSourceCompactionConfig; import org.apache.druid.server.coordinator.DruidCompactionConfig; import org.apache.druid.server.coordinator.DruidCoordinatorRuntimeParams; +import org.apache.druid.server.coordinator.UserCompactionTaskGranularityConfig; 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 org.apache.druid.timeline.CompactionState; import org.apache.druid.timeline.DataSegment; import org.joda.time.Interval; import javax.annotation.Nullable; +import java.util.Arrays; import java.util.Collections; import java.util.HashMap; import java.util.List; @@ -76,7 +86,12 @@ public class CompactSegments implements CoordinatorCustomDuty * Must be the same as org.apache.druid.indexing.common.task.Tasks.STORE_COMPACTION_STATE_KEY */ public static final String STORE_COMPACTION_STATE_KEY = "storeCompactionState"; - public static final String COMPACTION_INTERVAL_KEY = "compactionInterval"; + + /** + * Must be the same as org.apache.druid.indexing.common.task.Tasks.COMPACTION_STATE_FINGERPRINT_KEY + */ + public static final String COMPACTION_STATE_FINGERPRINT_KEY = "compactionStateFingerprint"; + private static final String COMPACTION_REASON_KEY = "compactionReason"; private static final Logger LOG = new Logger(CompactSegments.class); @@ -90,14 +105,18 @@ public class CompactSegments implements CoordinatorCustomDuty // read by HTTP threads processing Coordinator API calls. private final AtomicReference> autoCompactionSnapshotPerDataSource = new AtomicReference<>(); + private final CompactionStateManager compactionStateManager; + @JsonCreator public CompactSegments( @JacksonInject CompactionStatusTracker statusTracker, - @JacksonInject OverlordClient overlordClient + @JacksonInject OverlordClient overlordClient, + @JacksonInject CompactionStateManager compactionStateManager ) { this.overlordClient = overlordClient; this.statusTracker = statusTracker; + this.compactionStateManager = compactionStateManager; resetCompactionSnapshot(); } @@ -177,8 +196,9 @@ public void run( policy, compactionConfigs, dataSources.getUsedSegmentsTimelinesPerDataSource(), - slotManager.getDatasourceIntervalsToSkipCompaction() - ); + slotManager.getDatasourceIntervalsToSkipCompaction(), + compactionStateManager + ); final CompactionSnapshotBuilder compactionSnapshotBuilder = new CompactionSnapshotBuilder(stats); final int numSubmittedCompactionTasks = submitCompactionTasks( @@ -187,7 +207,8 @@ public void run( slotManager, iterator, policy, - defaultEngine + defaultEngine, + dynamicConfig.clusterConfig().isLegacyPersistLastCompactionStateInSegments() ); stats.add(Stats.Compaction.SUBMITTED_TASKS, numSubmittedCompactionTasks); @@ -223,7 +244,8 @@ private int submitCompactionTasks( CompactionSlotManager slotManager, CompactionSegmentIterator iterator, CompactionCandidateSearchPolicy policy, - CompactionEngine defaultEngine + CompactionEngine defaultEngine, + boolean persistLastCompactionStateInSegments ) { if (slotManager.getNumAvailableTaskSlots() <= 0) { @@ -254,7 +276,29 @@ private int submitCompactionTasks( snapshotBuilder.addToComplete(entry); } - final ClientCompactionTaskQuery taskPayload = createCompactionTask(entry, config, defaultEngine); + CompactionState compactionState = + createCompactionStateFromConfig(config); + + String compactionStateFingerprint = CompactionState.generateCompactionStateFingerprint( + compactionState, + config.getDataSource() + ); + + // If we are going to create compaction jobs for this compaction state, we need to persist the fingerprint -> state + // mapping so compacted segments from these jobs can reference a valid compaction state. + compactionStateManager.persistCompactionState( + config.getDataSource(), + Map.of(compactionStateFingerprint, compactionState), + DateTimes.nowUtc() + ); + + final ClientCompactionTaskQuery taskPayload = createCompactionTask( + entry, + config, + defaultEngine, + compactionStateFingerprint, + persistLastCompactionStateInSegments + ); final String taskId = taskPayload.getId(); FutureUtils.getUnchecked(overlordClient.runTask(taskId, taskPayload), true); @@ -280,7 +324,9 @@ private int submitCompactionTasks( public static ClientCompactionTaskQuery createCompactionTask( CompactionCandidate candidate, DataSourceCompactionConfig config, - CompactionEngine defaultEngine + CompactionEngine defaultEngine, + String compactionStateFingerprint, + boolean persistLastCompactionStateInSegments ) { final List segmentsToCompact = candidate.getSegments(); @@ -358,6 +404,9 @@ public static ClientCompactionTaskQuery createCompactionTask( autoCompactionContext.put(COMPACTION_REASON_KEY, candidate.getCurrentStatus().getReason()); } + autoCompactionContext.put(STORE_COMPACTION_STATE_KEY, persistLastCompactionStateInSegments); + autoCompactionContext.put(COMPACTION_STATE_FINGERPRINT_KEY, compactionStateFingerprint); + return compactSegments( candidate, config.getTaskPriority(), @@ -461,4 +510,59 @@ private static ClientCompactionTaskQuery compactSegments( compactionRunner ); } + + /** + * Given a {@link DataSourceCompactionConfig}, create a {@link CompactionState} + */ + public static CompactionState createCompactionStateFromConfig(DataSourceCompactionConfig config) + { + ClientCompactionTaskQueryTuningConfig tuningConfig = ClientCompactionTaskQueryTuningConfig.from(config); + + // 1. PartitionsSpec - reuse existing method + PartitionsSpec partitionsSpec = CompactionStatus.findPartitionsSpecFromConfig(tuningConfig); + + // 2. DimensionsSpec + DimensionsSpec dimensionsSpec = null; + if (config.getDimensionsSpec() != null && config.getDimensionsSpec().getDimensions() != null) { + dimensionsSpec = new DimensionsSpec(config.getDimensionsSpec().getDimensions()); + } + + // 3. Metrics + List metricsSpec = config.getMetricsSpec() == null + ? null + : Arrays.asList(config.getMetricsSpec()); + + // 4. Transform + CompactionTransformSpec transformSpec = config.getTransformSpec(); + + // 5. IndexSpec + IndexSpec indexSpec = tuningConfig.getIndexSpec() == null + ? IndexSpec.getDefault() + : tuningConfig.getIndexSpec(); + + // 6. GranularitySpec + GranularitySpec granularitySpec = null; + if (config.getGranularitySpec() != null) { + UserCompactionTaskGranularityConfig userGranularityConfig = config.getGranularitySpec(); + granularitySpec = new UniformGranularitySpec( + userGranularityConfig.getSegmentGranularity(), + userGranularityConfig.getQueryGranularity(), + userGranularityConfig.isRollup(), + null // intervals + ); + } + + // 7. Projections + List projections = config.getProjections(); + + return new CompactionState( + partitionsSpec, + dimensionsSpec, + metricsSpec, + transformSpec, + indexSpec, + granularitySpec, + projections + ); + } } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/duty/KillUnreferencedCompactionState.java b/server/src/main/java/org/apache/druid/server/coordinator/duty/KillUnreferencedCompactionState.java new file mode 100644 index 000000000000..90a03ba3dfc9 --- /dev/null +++ b/server/src/main/java/org/apache/druid/server/coordinator/duty/KillUnreferencedCompactionState.java @@ -0,0 +1,61 @@ +/* + * 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.coordinator.duty; + +import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.segment.metadata.CompactionStateManager; +import org.apache.druid.server.coordinator.config.MetadataCleanupConfig; +import org.apache.druid.server.coordinator.stats.Stats; +import org.joda.time.DateTime; + +import java.util.List; + +public class KillUnreferencedCompactionState extends MetadataCleanupDuty +{ + private static final Logger log = new Logger(KillUnreferencedCompactionState.class); + private final CompactionStateManager compactionStateManager; + + public KillUnreferencedCompactionState( + MetadataCleanupConfig config, + CompactionStateManager compactionStateManager + ) + { + super("compactionState", config, Stats.Kill.COMPACTION_STATE); + this.compactionStateManager = compactionStateManager; + } + + @Override + protected int cleanupEntriesCreatedBefore(DateTime minCreatedTime) + { + // 1: Mark unreferenced states as unused + int unused = compactionStateManager.markUnreferencedCompactionStatesAsUnused(); + log.info("Marked [%s] unreferenced compaction states as unused.", unused); + + // 2: Repair - find unused states still referenced by segments + List stateFingerprints = compactionStateManager.findReferencedCompactionStateMarkedAsUnused(); + if (!stateFingerprints.isEmpty()) { + int numUpdated = compactionStateManager.markCompactionStatesAsUsed(stateFingerprints); + log.info("Marked [%s] unused compaction states referenced by used segments as used.", numUpdated); + } + + // 3: Delete unused states older than threshold + return compactionStateManager.deleteUnusedCompactionStatesOlderThan(minCreatedTime.getMillis()); + } +} 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 3851b38fc91d..d6f379a2838a 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 @@ -170,6 +170,8 @@ public static class Kill = CoordinatorStat.toDebugAndEmit("killedAuditLogs", "metadata/kill/audit/count"); public static final CoordinatorStat DATASOURCES = CoordinatorStat.toDebugAndEmit("killedDatasources", "metadata/kill/datasource/count"); + public static final CoordinatorStat COMPACTION_STATE + = CoordinatorStat.toDebugAndEmit("killedCompactionState", "metadata/kill/compactionState/count"); public static final CoordinatorStat AVAILABLE_SLOTS = CoordinatorStat.toDebugAndEmit("killAvailSlots", "killTask/availableSlot/count"); public static final CoordinatorStat MAX_SLOTS diff --git a/server/src/main/java/org/apache/druid/server/http/DataSegmentPlus.java b/server/src/main/java/org/apache/druid/server/http/DataSegmentPlus.java index bfda5cbf3ad4..4f760b33e31c 100644 --- a/server/src/main/java/org/apache/druid/server/http/DataSegmentPlus.java +++ b/server/src/main/java/org/apache/druid/server/http/DataSegmentPlus.java @@ -58,6 +58,9 @@ public class DataSegmentPlus @Nullable private final String upgradedFromSegmentId; + @Nullable + private final String compactionStateFingerprint; + @JsonCreator public DataSegmentPlus( @JsonProperty("dataSegment") final DataSegment dataSegment, @@ -66,7 +69,8 @@ public DataSegmentPlus( @JsonProperty("used") @Nullable final Boolean used, @JsonProperty("schemaFingerprint") @Nullable final String schemaFingerprint, @JsonProperty("numRows") @Nullable final Long numRows, - @JsonProperty("upgradedFromSegmentId") @Nullable final String upgradedFromSegmentId + @JsonProperty("upgradedFromSegmentId") @Nullable final String upgradedFromSegmentId, + @JsonProperty("compactionStateFingerprint") @Nullable String compactionStateFingerprint ) { this.dataSegment = dataSegment; @@ -76,6 +80,7 @@ public DataSegmentPlus( this.schemaFingerprint = schemaFingerprint; this.numRows = numRows; this.upgradedFromSegmentId = upgradedFromSegmentId; + this.compactionStateFingerprint = compactionStateFingerprint; } @Nullable @@ -126,6 +131,13 @@ public String getUpgradedFromSegmentId() return upgradedFromSegmentId; } + @Nullable + @JsonProperty + public String getCompactionStateFingerprint() + { + return compactionStateFingerprint; + } + @Override public boolean equals(Object o) { @@ -142,7 +154,8 @@ public boolean equals(Object o) && Objects.equals(used, that.getUsed()) && Objects.equals(schemaFingerprint, that.getSchemaFingerprint()) && Objects.equals(numRows, that.getNumRows()) - && Objects.equals(upgradedFromSegmentId, that.getUpgradedFromSegmentId()); + && Objects.equals(upgradedFromSegmentId, that.getUpgradedFromSegmentId()) + && Objects.equals(compactionStateFingerprint, that.getCompactionStateFingerprint()); } @Override @@ -155,7 +168,8 @@ public int hashCode() used, schemaFingerprint, numRows, - upgradedFromSegmentId + upgradedFromSegmentId, + compactionStateFingerprint ); } @@ -170,6 +184,7 @@ public String toString() ", schemaFingerprint=" + getSchemaFingerprint() + ", numRows=" + getNumRows() + ", upgradedFromSegmentId=" + getUpgradedFromSegmentId() + + ", compactionStateFingerprint=" + getCompactionStateFingerprint() + '}'; } } diff --git a/server/src/test/java/org/apache/druid/metadata/IndexerSqlMetadataStorageCoordinatorTestBase.java b/server/src/test/java/org/apache/druid/metadata/IndexerSqlMetadataStorageCoordinatorTestBase.java index 36dd38a85c00..be0120162cf0 100644 --- a/server/src/test/java/org/apache/druid/metadata/IndexerSqlMetadataStorageCoordinatorTestBase.java +++ b/server/src/test/java/org/apache/druid/metadata/IndexerSqlMetadataStorageCoordinatorTestBase.java @@ -617,7 +617,8 @@ public static void insertUsedSegments( true, null, null, - upgradedFromSegmentIdMap.get(segment.getId().toString()) + upgradedFromSegmentIdMap.get(segment.getId().toString()), + null ) ); } diff --git a/server/src/test/java/org/apache/druid/metadata/SQLMetadataConnectorTest.java b/server/src/test/java/org/apache/druid/metadata/SQLMetadataConnectorTest.java index f898d12000c7..523d7220e72d 100644 --- a/server/src/test/java/org/apache/druid/metadata/SQLMetadataConnectorTest.java +++ b/server/src/test/java/org/apache/druid/metadata/SQLMetadataConnectorTest.java @@ -76,6 +76,7 @@ public void testCreateTables() tables.add(tablesConfig.getTasksTable()); tables.add(tablesConfig.getAuditTable()); tables.add(tablesConfig.getSupervisorTable()); + tables.add(tablesConfig.getCompactionStatesTable()); connector.createSegmentTable(); connector.createConfigTable(); @@ -83,6 +84,7 @@ public void testCreateTables() connector.createTaskTables(); connector.createAuditTable(); connector.createSupervisorsTable(); + connector.createCompactionStatesTable(); connector.getDBI().withHandle( handle -> { @@ -187,6 +189,23 @@ public void testAlterSegmentTableAddLastUsed() )); } + /** + * This is a test for the upgrade path where a cluster is upgrading from a version that did not have used_status_last_updated + * in the segments table. + */ + @Test + public void testAlterSegmentTableAddCompactionStateFingerprint() + { + connector.createSegmentTable(); + derbyConnectorRule.segments().update("ALTER TABLE %1$s DROP COLUMN COMPACTION_STATE_FINGERPRINT"); + + connector.alterSegmentTable(); + Assert.assertTrue(connector.tableHasColumn( + derbyConnectorRule.metadataTablesConfigSupplier().get().getSegmentsTable(), + "COMPACTION_STATE_FINGERPRINT" + )); + } + @Test public void testInsertOrUpdate() { @@ -309,7 +328,8 @@ public void test_useShortIndexNames_true_tableIndices_areNotAdded_ifExist() tablesConfig = new MetadataStorageTablesConfig( "druidTest", null, null, null, null, null, null, null, null, null, null, null, - true + true, + null ); connector = new TestDerbyConnector(new MetadataStorageConnectorConfig(), tablesConfig); @@ -343,7 +363,8 @@ public void test_useShortIndexNames_false_tableIndices_areNotAdded_ifExist() tablesConfig = new MetadataStorageTablesConfig( "druidTest", null, null, null, null, null, null, null, null, null, null, null, - false + false, + null ); connector = new TestDerbyConnector(new MetadataStorageConnectorConfig(), tablesConfig); @@ -377,7 +398,8 @@ public void test_useShortIndexNames_true_tableIndices_areAdded_IfNotExist() tablesConfig = new MetadataStorageTablesConfig( "druidTest", null, null, null, null, null, null, null, null, null, null, null, - true + true, + null ); connector = new TestDerbyConnector(new MetadataStorageConnectorConfig(), tablesConfig); @@ -403,7 +425,8 @@ public void test_useShortIndexNames_false_tableIndices_areAdded_IfNotExist() tablesConfig = new MetadataStorageTablesConfig( "druidTest", null, null, null, null, null, null, null, null, null, null, null, - false + false, + null ); connector = new TestDerbyConnector(new MetadataStorageConnectorConfig(), tablesConfig); final String segmentsTable = tablesConfig.getSegmentsTable(); diff --git a/server/src/test/java/org/apache/druid/metadata/segment/cache/HeapMemoryDatasourceSegmentCacheTest.java b/server/src/test/java/org/apache/druid/metadata/segment/cache/HeapMemoryDatasourceSegmentCacheTest.java index c78cf31bfdb4..b375b78df82a 100644 --- a/server/src/test/java/org/apache/druid/metadata/segment/cache/HeapMemoryDatasourceSegmentCacheTest.java +++ b/server/src/test/java/org/apache/druid/metadata/segment/cache/HeapMemoryDatasourceSegmentCacheTest.java @@ -220,6 +220,7 @@ public void testInsertSegments_canMarkItAsUnused() true, null, null, + null, null ); @@ -779,7 +780,8 @@ private static DataSegmentPlus updateSegment(DataSegmentPlus segment, DateTime n segment.getUsed(), segment.getSchemaFingerprint(), segment.getNumRows(), - segment.getUpgradedFromSegmentId() + segment.getUpgradedFromSegmentId(), + segment.getCompactionStateFingerprint() ); } diff --git a/server/src/test/java/org/apache/druid/metadata/segment/cache/HeapMemorySegmentMetadataCacheTest.java b/server/src/test/java/org/apache/druid/metadata/segment/cache/HeapMemorySegmentMetadataCacheTest.java index fab64152ec8a..99ebf7083ac8 100644 --- a/server/src/test/java/org/apache/druid/metadata/segment/cache/HeapMemorySegmentMetadataCacheTest.java +++ b/server/src/test/java/org/apache/druid/metadata/segment/cache/HeapMemorySegmentMetadataCacheTest.java @@ -89,6 +89,7 @@ public void setup() derbyConnector.createSegmentTable(); derbyConnector.createSegmentSchemasTable(); derbyConnector.createPendingSegmentsTable(); + derbyConnector.createCompactionStatesTable(); schemaTestUtils = new SegmentSchemaTestUtils(derbyConnectorRule, derbyConnector, TestHelper.JSON_MAPPER); EmittingLogger.registerEmitter(serviceEmitter); @@ -512,6 +513,7 @@ public void testSync_updatesUsedSegment_ifCacheHasOlderEntry() true, null, null, + null, null ); updateSegmentInMetadataStore(updatedSegment); diff --git a/server/src/test/java/org/apache/druid/rpc/indexing/OverlordClientImplTest.java b/server/src/test/java/org/apache/druid/rpc/indexing/OverlordClientImplTest.java index 294025f25065..2a1af270abaa 100644 --- a/server/src/test/java/org/apache/druid/rpc/indexing/OverlordClientImplTest.java +++ b/server/src/test/java/org/apache/druid/rpc/indexing/OverlordClientImplTest.java @@ -504,7 +504,8 @@ public void test_getClusterCompactionConfig() 101, new NewestSegmentFirstPolicy(null), true, - CompactionEngine.MSQ + CompactionEngine.MSQ, + true ); serviceClient.expectAndRespond( new RequestBuilder(HttpMethod.GET, "/druid/indexer/v1/compaction/config/cluster"), @@ -523,7 +524,7 @@ public void test_getClusterCompactionConfig() public void test_updateClusterCompactionConfig() throws ExecutionException, InterruptedException, JsonProcessingException { - final ClusterCompactionConfig config = new ClusterCompactionConfig(null, null, null, null, null); + final ClusterCompactionConfig config = new ClusterCompactionConfig(null, null, null, null, null, null); serviceClient.expectAndRespond( new RequestBuilder(HttpMethod.POST, "/druid/indexer/v1/compaction/config/cluster") .jsonContent(jsonMapper, config), diff --git a/server/src/test/java/org/apache/druid/segment/metadata/CompactionStateManagerConfigTest.java b/server/src/test/java/org/apache/druid/segment/metadata/CompactionStateManagerConfigTest.java new file mode 100644 index 000000000000..edd7cac699b6 --- /dev/null +++ b/server/src/test/java/org/apache/druid/segment/metadata/CompactionStateManagerConfigTest.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.segment.metadata; + +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.jackson.DefaultObjectMapper; +import org.junit.Assert; +import org.junit.Test; + +public class CompactionStateManagerConfigTest +{ + private final ObjectMapper mapper = new DefaultObjectMapper(); + + @Test + public void testSerdeWithDefaults() throws Exception + { + String json = "{}"; + CompactionStateManagerConfig config = mapper.readValue(json, CompactionStateManagerConfig.class); + Assert.assertEquals(100, config.getCacheSize()); + Assert.assertEquals(100, config.getPrewarmFingerprintCount()); + } + + @Test + public void testSerdeRoundTripWithOverrides() throws Exception + { + String json = "{\"cacheSize\": 1000, \"prewarmFingerprintCount\": 500}"; + CompactionStateManagerConfig config = mapper.readValue(json, CompactionStateManagerConfig.class); + String serialized = mapper.writeValueAsString(config); + CompactionStateManagerConfig deserialized = mapper.readValue(serialized, CompactionStateManagerConfig.class); + Assert.assertEquals(1000, deserialized.getCacheSize()); + Assert.assertEquals(500, deserialized.getPrewarmFingerprintCount()); + } +} diff --git a/server/src/test/java/org/apache/druid/segment/metadata/CompactionStateManagerTest.java b/server/src/test/java/org/apache/druid/segment/metadata/CompactionStateManagerTest.java new file mode 100644 index 000000000000..3aad61e43226 --- /dev/null +++ b/server/src/test/java/org/apache/druid/segment/metadata/CompactionStateManagerTest.java @@ -0,0 +1,417 @@ +/* + * 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.segment.metadata; + +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; +import org.apache.druid.jackson.DefaultObjectMapper; +import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.metadata.MetadataStorageTablesConfig; +import org.apache.druid.metadata.TestDerbyConnector; +import org.apache.druid.segment.IndexSpec; +import org.apache.druid.timeline.CompactionState; +import org.joda.time.DateTime; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.RegisterExtension; + +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertNull; +import static org.junit.jupiter.api.Assertions.assertTrue; + +public class CompactionStateManagerTest +{ + @RegisterExtension + public static final TestDerbyConnector.DerbyConnectorRule5 DERBY_CONNECTOR_RULE = + new TestDerbyConnector.DerbyConnectorRule5(); + + private final ObjectMapper jsonMapper = new DefaultObjectMapper(); + + private static TestDerbyConnector derbyConnector; + private static MetadataStorageTablesConfig tablesConfig; + private CompactionStateManager manager; + + @BeforeAll + public static void setUpClass() + { + derbyConnector = DERBY_CONNECTOR_RULE.getConnector(); + tablesConfig = DERBY_CONNECTOR_RULE.metadataTablesConfigSupplier().get(); + derbyConnector.createCompactionStatesTable(); + derbyConnector.createSegmentTable(); + } + + @BeforeEach + public void setUp() + { + derbyConnector.retryWithHandle(handle -> { + handle.createStatement("DELETE FROM " + tablesConfig.getSegmentsTable()).execute(); + handle.createStatement("DELETE FROM " + tablesConfig.getCompactionStatesTable()).execute(); + return null; + }); + + manager = new CompactionStateManager(tablesConfig, jsonMapper, derbyConnector, new CompactionStateManagerConfig()); + } + + @Test + public void test_persistCompactionState_andThen_getCompactionStateByFingerprint() + { + CompactionState state1 = createTestCompactionState(); + String fingerprint = "fingerprint_abc123"; + + Map fingerprintMap = new HashMap<>(); + fingerprintMap.put(fingerprint, state1); + + derbyConnector.retryWithHandle(handle -> { + manager.persistCompactionState( + "testDatasource", + fingerprintMap, + DateTimes.nowUtc() + ); + return null; + }); + + assertTrue(manager.isCached(fingerprint)); + CompactionState retrieved = manager.getCompactionStateByFingerprint(fingerprint); + assertNotNull(retrieved); + assertEquals(state1, retrieved); + } + + @Test + public void test_persistCompactionState_andThen_confirmCached_andThen_invalidateCache_andThen_confirmNotCached() + { + String fingerprint = "cachemiss_fingerprint"; + CompactionState state = createTestCompactionState(); + + derbyConnector.retryWithHandle(handle -> { + Map map = new HashMap<>(); + map.put(fingerprint, state); + manager.persistCompactionState("ds1", map, DateTimes.nowUtc()); + return null; + }); + + assertTrue(manager.isCached(fingerprint)); + manager.invalidateFingerprint(fingerprint); + assertFalse(manager.isCached(fingerprint)); + CompactionState result = manager.getCompactionStateByFingerprint(fingerprint); + assertNotNull(result); + assertEquals(state, result); + } + + @Test + public void test_persistCompactionState_andThen_markUnreferencedCompactionStateAsUnused_andThen_markCompactionStatesAsUsed() + { + CompactionState state1 = createTestCompactionState(); + String fingerprint = "fingerprint_abc123"; + + Map fingerprintMap = new HashMap<>(); + fingerprintMap.put(fingerprint, state1); + + derbyConnector.retryWithHandle(handle -> { + manager.persistCompactionState( + "testDatasource", + fingerprintMap, + DateTimes.nowUtc() + ); + return null; + }); + assertEquals(1, manager.markUnreferencedCompactionStatesAsUnused()); + assertEquals(1, manager.markCompactionStatesAsUsed(List.of(fingerprint))); + } + + @Test + public void test_findReferencedCompactionStateMarkedAsUnused() + { + CompactionState state1 = createTestCompactionState(); + String fingerprint = "fingerprint_abc123"; + + Map fingerprintMap = new HashMap<>(); + fingerprintMap.put(fingerprint, state1); + + derbyConnector.retryWithHandle(handle -> { + manager.persistCompactionState( + "testDatasource", + fingerprintMap, + DateTimes.nowUtc() + ); + return null; + }); + manager.markUnreferencedCompactionStatesAsUnused(); + assertEquals(0, manager.findReferencedCompactionStateMarkedAsUnused().size()); + + derbyConnector.retryWithHandle(handle -> { + handle.createStatement( + "INSERT INTO " + tablesConfig.getSegmentsTable() + " " + + "(id, dataSource, created_date, start, \"end\", partitioned, version, used, payload, " + + "used_status_last_updated, compaction_state_fingerprint) " + + "VALUES (:id, :dataSource, :created_date, :start, :end, :partitioned, :version, :used, :payload, " + + ":used_status_last_updated, :compaction_state_fingerprint)" + ) + .bind("id", "testSegment_2024-01-01_2024-01-02_v1_0") + .bind("dataSource", "testDatasource") + .bind("created_date", DateTimes.nowUtc().toString()) + .bind("start", "2024-01-01T00:00:00.000Z") + .bind("end", "2024-01-02T00:00:00.000Z") + .bind("partitioned", 0) + .bind("version", "v1") + .bind("used", true) + .bind("payload", new byte[]{}) // Empty payload is fine for this test + .bind("used_status_last_updated", DateTimes.nowUtc().toString()) + .bind("compaction_state_fingerprint", fingerprint) + .execute(); + return null; + }); + + List referenced = manager.findReferencedCompactionStateMarkedAsUnused(); + assertEquals(1, referenced.size()); + assertEquals(fingerprint, referenced.get(0)); + } + + @Test + public void test_deleteCompactionStatesOlderThan_deletesOnlyOldUnusedStates() throws Exception + { + DateTime now = DateTimes.nowUtc(); + DateTime oldTime = now.minusDays(60); + DateTime recentTime = now.minusDays(15); + DateTime cutoffTime = now.minusDays(30); + + String oldFingerprint = "old_fp_should_delete"; + String recentFingerprint = "recent_fp_should_keep"; + + CompactionState oldState = createTestCompactionState(); + CompactionState recentState = createTestCompactionState(); + + // Insert old unused state (60 days old) + derbyConnector.retryWithHandle(handle -> { + handle.createStatement( + "INSERT INTO " + tablesConfig.getCompactionStatesTable() + " " + + "(created_date, datasource, fingerprint, payload, used, used_status_last_updated) " + + "VALUES (:cd, :ds, :fp, :pl, :used, :updated)" + ) + .bind("cd", oldTime.toString()) + .bind("ds", "testDatasource") + .bind("fp", oldFingerprint) + .bind("pl", jsonMapper.writeValueAsBytes(oldState)) + .bind("used", false) + .bind("updated", oldTime.toString()) + .execute(); + return null; + }); + + // Insert recent unused state (15 days old) + derbyConnector.retryWithHandle(handle -> { + handle.createStatement( + "INSERT INTO " + tablesConfig.getCompactionStatesTable() + " " + + "(created_date, datasource, fingerprint, payload, used, used_status_last_updated) " + + "VALUES (:cd, :ds, :fp, :pl, :used, :updated)" + ) + .bind("cd", recentTime.toString()) + .bind("ds", "testDatasource") + .bind("fp", recentFingerprint) + .bind("pl", jsonMapper.writeValueAsBytes(recentState)) + .bind("used", false) + .bind("updated", recentTime.toString()) + .execute(); + return null; + }); + + // Delete states older than 30 days + int deleted = manager.deleteUnusedCompactionStatesOlderThan(cutoffTime.getMillis()); + assertEquals(1, deleted); + + // Verify the old one is gone + CompactionState oldResult = manager.getCompactionStateByFingerprint(oldFingerprint); + assertNull(oldResult); + + // Verify only 1 state remains in the table + Integer count = derbyConnector.retryWithHandle(handle -> + handle.createQuery("SELECT COUNT(*) FROM " + tablesConfig.getCompactionStatesTable()) + .map((i, r, ctx) -> r.getInt(1)) + .first() + ); + assertEquals(1, count); + } + + @Test + public void test_prewarmCache_onModuleLifecycleStart() throws Exception + { + String fingerprint = "prewarm_fingerprint"; + CompactionState state = createTestCompactionState(); + + // Insert a used compaction state directly into the database + derbyConnector.retryWithHandle(handle -> { + handle.createStatement( + "INSERT INTO " + tablesConfig.getCompactionStatesTable() + " " + + "(created_date, datasource, fingerprint, payload, used, used_status_last_updated) " + + "VALUES (:cd, :ds, :fp, :pl, :used, :updated)" + ) + .bind("cd", DateTimes.nowUtc().toString()) + .bind("ds", "testDatasource") + .bind("fp", fingerprint) + .bind("pl", jsonMapper.writeValueAsBytes(state)) + .bind("used", true) // Mark as used so it gets prewarmed + .bind("updated", DateTimes.nowUtc().toString()) + .execute(); + return null; + }); + + // Create a NEW manager (not the shared one) - should prewarm cache in constructor + CompactionStateManager newManager = new CompactionStateManager( + tablesConfig, + jsonMapper, + derbyConnector, + new CompactionStateManagerConfig() + ); + newManager.start(); // normally handled by Guice during startup + + // Verify the state was prewarmed into cache + assertTrue(newManager.isCached(fingerprint)); + + // Verify we can retrieve it + CompactionState retrieved = newManager.getCompactionStateByFingerprint(fingerprint); + assertNotNull(retrieved); + assertEquals(state, retrieved); + } + + @Test + public void test_persistCompactionState_withEmptyMap_doesNothing() + { + // Get initial count + Integer beforeCount = derbyConnector.retryWithHandle(handle -> + handle.createQuery("SELECT COUNT(*) FROM " + tablesConfig.getCompactionStatesTable()) + .map((i, r, ctx) -> r.getInt(1)) + .first() + ); + + // Persist empty map + derbyConnector.retryWithHandle(handle -> { + manager.persistCompactionState("ds", new HashMap<>(), DateTimes.nowUtc()); + return null; + }); + + // Verify count unchanged + Integer afterCount = derbyConnector.retryWithHandle(handle -> + handle.createQuery("SELECT COUNT(*) FROM " + tablesConfig.getCompactionStatesTable()) + .map((i, r, ctx) -> r.getInt(1)) + .first() + ); + + assertEquals(beforeCount, afterCount); + } + + @Test + public void test_getCompactionStateByFingerprint_notFound_returnsNull() + { + // Try to get a fingerprint that doesn't exist + CompactionState result = manager.getCompactionStateByFingerprint("nonexistent_fingerprint"); + + assertNull(result); + + // Verify it's not cached (shouldn't cache nulls) + assertFalse(manager.isCached("nonexistent_fingerprint")); + } + + @Test + public void test_persistCompactionState_verifyExistingFingerprintMarkedUsed() throws Exception + { + String fingerprint = "existing_fingerprint"; + CompactionState state = createTestCompactionState(); + + // Persist initially + derbyConnector.retryWithHandle(handle -> { + Map map = new HashMap<>(); + map.put(fingerprint, state); + manager.persistCompactionState("ds1", map, DateTimes.nowUtc()); + return null; + }); + + // Verify it's marked as used + Boolean usedBefore = derbyConnector.retryWithHandle(handle -> + handle.createQuery( + "SELECT used FROM " + tablesConfig.getCompactionStatesTable() + + " WHERE fingerprint = :fp" + ).bind("fp", fingerprint) + .map((i, r, ctx) -> r.getBoolean("used")) + .first() + ); + assertTrue(usedBefore); + + // Manually mark it as unused + derbyConnector.retryWithHandle(handle -> + handle.createStatement( + "UPDATE " + tablesConfig.getCompactionStatesTable() + + " SET used = false WHERE fingerprint = :fp" + ).bind("fp", fingerprint).execute() + ); + + // Persist again with the same fingerprint (should UPDATE, not INSERT) + derbyConnector.retryWithHandle(handle -> { + Map map = new HashMap<>(); + map.put(fingerprint, state); + manager.persistCompactionState("ds1", map, DateTimes.nowUtc()); + return null; + }); + + // Verify it's marked as used again + Boolean usedAfter = derbyConnector.retryWithHandle(handle -> + handle.createQuery( + "SELECT used FROM " + tablesConfig.getCompactionStatesTable() + + " WHERE fingerprint = :fp" + ).bind("fp", fingerprint) + .map((i, r, ctx) -> r.getBoolean("used")) + .first() + ); + assertTrue(usedAfter); + + // Verify only 1 row exists (no duplicate insert) + Integer count = derbyConnector.retryWithHandle(handle -> + handle.createQuery("SELECT COUNT(*) FROM " + tablesConfig.getCompactionStatesTable()) + .map((i, r, ctx) -> r.getInt(1)) + .first() + ); + assertEquals(1, count); + } + + @Test + public void test_markCompactionStateAsUsed_withEmptyList_returnsZero() + { + assertEquals(0, manager.markCompactionStatesAsUsed(List.of())); + } + + + private CompactionState createTestCompactionState() + { + return new CompactionState( + new DynamicPartitionsSpec(100, null), + null, + null, + null, + IndexSpec.getDefault(), + null, + null + ); + } +} 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 7a68424db9f8..56ec8525bb83 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 @@ -143,7 +143,7 @@ public void testSimulate_withFixedIntervalOrderPolicy() final CompactionSimulateResult simulateResult = simulator.simulateRunWithConfig( DruidCompactionConfig .empty() - .withClusterConfig(new ClusterCompactionConfig(null, null, policy, null, null)) + .withClusterConfig(new ClusterCompactionConfig(null, null, policy, null, null, null)) .withDatasourceConfig( InlineSchemaDataSourceCompactionConfig.builder().forDataSource(dataSource).build() ), 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 d201b84135dc..2f03be6ffa3e 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 @@ -32,21 +32,26 @@ import org.apache.druid.indexer.partitions.PartitionsSpec; 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.query.aggregation.LongSumAggregatorFactory; import org.apache.druid.segment.AutoTypeColumnSchema; import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.TestDataSource; import org.apache.druid.segment.data.CompressionStrategy; +import org.apache.druid.segment.metadata.CompactionStateManager; import org.apache.druid.segment.nested.NestedCommonFormatColumnFormatSpec; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; import org.apache.druid.server.coordinator.InlineSchemaDataSourceCompactionConfig; import org.apache.druid.server.coordinator.UserCompactionTaskDimensionsConfig; import org.apache.druid.server.coordinator.UserCompactionTaskGranularityConfig; import org.apache.druid.server.coordinator.UserCompactionTaskQueryTuningConfig; +import org.apache.druid.server.coordinator.duty.CompactSegments; import org.apache.druid.timeline.CompactionState; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.SegmentId; +import org.easymock.EasyMock; import org.junit.Assert; +import org.junit.Before; import org.junit.Test; import java.util.Collections; @@ -58,6 +63,18 @@ public class CompactionStatusTest = DataSegment.builder(SegmentId.of(TestDataSource.WIKI, Intervals.of("2013-01-01/PT1H"), "v1", 0)) .size(100_000_000L) .build(); + private static final DataSegment WIKI_SEGMENT_2 + = DataSegment.builder(SegmentId.of(TestDataSource.WIKI, Intervals.of("2013-01-01/PT1H"), "v1", 1)) + .size(100_000_000L) + .build(); + + private CompactionStateManager compactionStateManager; + + @Before + public void setUp() + { + compactionStateManager = EasyMock.createMock(CompactionStateManager.class); + } @Test public void testFindPartitionsSpecWhenGivenIsNull() @@ -326,8 +343,9 @@ public void testStatusWhenLastCompactionStateSameAsRequired() final DataSegment segment = DataSegment.builder(WIKI_SEGMENT).lastCompactionState(lastCompactionState).build(); final CompactionStatus status = CompactionStatus.compute( CompactionCandidate.from(List.of(segment), Granularities.HOUR), - compactionConfig - ); + compactionConfig, + compactionStateManager + ); Assert.assertTrue(status.isComplete()); } @@ -375,8 +393,9 @@ public void testStatusWhenProjectionsMatch() final DataSegment segment = DataSegment.builder(WIKI_SEGMENT).lastCompactionState(lastCompactionState).build(); final CompactionStatus status = CompactionStatus.compute( CompactionCandidate.from(List.of(segment), Granularities.HOUR), - compactionConfig - ); + compactionConfig, + compactionStateManager + ); Assert.assertTrue(status.isComplete()); } @@ -429,8 +448,9 @@ public void testStatusWhenProjectionsMismatch() final DataSegment segment = DataSegment.builder(WIKI_SEGMENT).lastCompactionState(lastCompactionState).build(); final CompactionStatus status = CompactionStatus.compute( CompactionCandidate.from(List.of(segment), Granularities.HOUR), - compactionConfig - ); + compactionConfig, + compactionStateManager + ); Assert.assertFalse(status.isComplete()); } @@ -482,8 +502,9 @@ public void testStatusWhenAutoSchemaMatch() final DataSegment segment = DataSegment.builder(WIKI_SEGMENT).lastCompactionState(lastCompactionState).build(); final CompactionStatus status = CompactionStatus.compute( CompactionCandidate.from(List.of(segment), null), - compactionConfig - ); + compactionConfig, + compactionStateManager + ); Assert.assertTrue(status.isComplete()); } @@ -535,9 +556,260 @@ public void testStatusWhenAutoSchemaMismatch() final DataSegment segment = DataSegment.builder(WIKI_SEGMENT).lastCompactionState(lastCompactionState).build(); final CompactionStatus status = CompactionStatus.compute( CompactionCandidate.from(List.of(segment), null), - compactionConfig + compactionConfig, + compactionStateManager + ); + Assert.assertFalse(status.isComplete()); + } + + @Test + public void test_evaluate_needsCompactionWhenAllSegmentsHaveUnexpectedCompactionStateFingerprint() + { + List segments = List.of( + DataSegment.builder(WIKI_SEGMENT).compactionStateFingerprint("wrongFingerprint").build(), + DataSegment.builder(WIKI_SEGMENT_2).compactionStateFingerprint("wrongFingerprint").build() + ); + final DataSourceCompactionConfig compactionConfig = InlineSchemaDataSourceCompactionConfig + .builder() + .forDataSource(TestDataSource.WIKI) + .withGranularitySpec(new UserCompactionTaskGranularityConfig(Granularities.DAY, null, null)) + .build(); + + EasyMock.expect(compactionStateManager.getCompactionStateByFingerprint("wrongFingerprint")).andReturn(createCompactionStateWithGranularity(Granularities.HOUR)); + EasyMock.replay(compactionStateManager); + + verifyEvaluationNeedsCompactionBecauseWithCustomSegments( + CompactionCandidate.from(segments, null), + compactionConfig, + "'segmentGranularity' mismatch: required[DAY], current[HOUR]", + compactionStateManager + ); + } + + @Test + public void test_evaluate_needsCompactionWhenSomeSegmentsHaveUnexpectedCompactionStateFingerprint() + { + final DataSourceCompactionConfig compactionConfig = InlineSchemaDataSourceCompactionConfig + .builder() + .forDataSource(TestDataSource.WIKI) + .withGranularitySpec(new UserCompactionTaskGranularityConfig(Granularities.DAY, null, null)) + .build(); + + CompactionState expectedState = CompactSegments.createCompactionStateFromConfig(compactionConfig); + + String expectedFingerprint = CompactionState.generateCompactionStateFingerprint(expectedState, TestDataSource.WIKI); + + List segments = List.of( + DataSegment.builder(WIKI_SEGMENT).compactionStateFingerprint(expectedFingerprint).build(), + DataSegment.builder(WIKI_SEGMENT_2).compactionStateFingerprint("wrongFingerprint").build() + ); + + EasyMock.expect(compactionStateManager.getCompactionStateByFingerprint("wrongFingerprint")).andReturn(createCompactionStateWithGranularity(Granularities.HOUR)); + EasyMock.replay(compactionStateManager); + + verifyEvaluationNeedsCompactionBecauseWithCustomSegments( + CompactionCandidate.from(segments, null), + compactionConfig, + "'segmentGranularity' mismatch: required[DAY], current[HOUR]", + compactionStateManager ); + } + + @Test + public void test_evaluate_noCompacationIfUnexpectedFingerprintHasExpectedCompactionState() + { + List segments = List.of( + DataSegment.builder(WIKI_SEGMENT).compactionStateFingerprint("wrongFingerprint").build() + ); + final DataSourceCompactionConfig compactionConfig = InlineSchemaDataSourceCompactionConfig + .builder() + .forDataSource(TestDataSource.WIKI) + .withGranularitySpec(new UserCompactionTaskGranularityConfig(Granularities.DAY, null, null)) + .build(); + + EasyMock.expect(compactionStateManager.getCompactionStateByFingerprint("wrongFingerprint")).andReturn(createCompactionStateWithGranularity(Granularities.DAY)); + EasyMock.replay(compactionStateManager); + + final CompactionStatus status = CompactionStatus.compute( + CompactionCandidate.from(segments, null), + compactionConfig, + compactionStateManager + ); + Assert.assertTrue(status.isComplete()); + } + + @Test + public void test_evaluate_needsCompactionWhenUnexpectedFingerprintAndNullCompactionStateManager() + { + List segments = List.of( + DataSegment.builder(WIKI_SEGMENT).compactionStateFingerprint("wrongFingerprint").build() + ); + final DataSourceCompactionConfig compactionConfig = InlineSchemaDataSourceCompactionConfig + .builder() + .forDataSource(TestDataSource.WIKI) + .withGranularitySpec(new UserCompactionTaskGranularityConfig(Granularities.DAY, null, null)) + .build(); + + verifyEvaluationNeedsCompactionBecauseWithCustomSegments( + CompactionCandidate.from(segments, null), + compactionConfig, + "At least one segment has a mismatched fingerprint and needs compaction", + null + ); + } + + @Test + public void test_evaluate_needsCompactionWhenUnexpectedFingerprintAndNoFingerprintInMetadataStore() + { + List segments = List.of( + DataSegment.builder(WIKI_SEGMENT).compactionStateFingerprint("wrongFingerprint").build() + ); + final DataSourceCompactionConfig compactionConfig = InlineSchemaDataSourceCompactionConfig + .builder() + .forDataSource(TestDataSource.WIKI) + .withGranularitySpec(new UserCompactionTaskGranularityConfig(Granularities.DAY, null, null)) + .build(); + + verifyEvaluationNeedsCompactionBecauseWithCustomSegments( + CompactionCandidate.from(segments, null), + compactionConfig, + "At least one segment has a mismatched fingerprint and needs compaction", + compactionStateManager + ); + } + + @Test + public void test_evaluate_noCompactionWhenAllSegmentsHaveExpectedCompactionStateFingerprint() + { + final DataSourceCompactionConfig compactionConfig = InlineSchemaDataSourceCompactionConfig + .builder() + .forDataSource(TestDataSource.WIKI) + .withGranularitySpec(new UserCompactionTaskGranularityConfig(Granularities.DAY, null, null)) + .build(); + + CompactionState expectedState = CompactSegments.createCompactionStateFromConfig(compactionConfig); + + String expectedFingerprint = CompactionState.generateCompactionStateFingerprint(expectedState, TestDataSource.WIKI); + + List segments = List.of( + DataSegment.builder(WIKI_SEGMENT).compactionStateFingerprint(expectedFingerprint).build(), + DataSegment.builder(WIKI_SEGMENT_2).compactionStateFingerprint(expectedFingerprint).build() + ); + + final CompactionStatus status = CompactionStatus.compute( + CompactionCandidate.from(segments, null), + compactionConfig, + compactionStateManager + ); + Assert.assertTrue(status.isComplete()); + } + + @Test + public void test_evaluate_needsCompactionWhenNonFingerprintedSegmentsFailChecksOnLastCompactionState() + { + final DataSourceCompactionConfig compactionConfig = InlineSchemaDataSourceCompactionConfig + .builder() + .forDataSource(TestDataSource.WIKI) + .withGranularitySpec(new UserCompactionTaskGranularityConfig(Granularities.DAY, null, null)) + .build(); + + CompactionState expectedState = CompactSegments.createCompactionStateFromConfig(compactionConfig); + + String expectedFingerprint = CompactionState.generateCompactionStateFingerprint(expectedState, TestDataSource.WIKI); + + List segments = List.of( + DataSegment.builder(WIKI_SEGMENT).compactionStateFingerprint(expectedFingerprint).build(), + DataSegment.builder(WIKI_SEGMENT_2).compactionStateFingerprint(null).lastCompactionState(createCompactionStateWithGranularity(Granularities.HOUR)).build() + ); + + verifyEvaluationNeedsCompactionBecauseWithCustomSegments( + CompactionCandidate.from(segments, null), + compactionConfig, + "'segmentGranularity' mismatch: required[DAY], current[HOUR]", + compactionStateManager + ); + } + + @Test + public void test_evaluate_noCompactionWhenNonFingerprintedSegmentsPassChecksOnLastCompactionState() + { + final DataSourceCompactionConfig compactionConfig = InlineSchemaDataSourceCompactionConfig + .builder() + .forDataSource(TestDataSource.WIKI) + .withGranularitySpec(new UserCompactionTaskGranularityConfig(Granularities.DAY, null, null)) + .build(); + + CompactionState expectedState = CompactSegments.createCompactionStateFromConfig(compactionConfig); + + String expectedFingerprint = CompactionState.generateCompactionStateFingerprint(expectedState, TestDataSource.WIKI); + + List segments = List.of( + DataSegment.builder(WIKI_SEGMENT).compactionStateFingerprint(expectedFingerprint).build(), + DataSegment.builder(WIKI_SEGMENT_2).compactionStateFingerprint(null).lastCompactionState(createCompactionStateWithGranularity(Granularities.DAY)).build() + ); + + final CompactionStatus status = CompactionStatus.compute( + CompactionCandidate.from(segments, null), + compactionConfig, + compactionStateManager + ); + Assert.assertTrue(status.isComplete()); + } + + // ============================ + // SKIPPED status tests + // ============================ + + @Test + public void test_evaluate_isSkippedWhenInputBytesExceedLimit() + { + // Two segments with 100MB each = 200MB total + // inputSegmentSizeBytes is 150MB, so should be skipped + final DataSourceCompactionConfig compactionConfig = InlineSchemaDataSourceCompactionConfig + .builder() + .forDataSource(TestDataSource.WIKI) + .withInputSegmentSizeBytes(150_000_000L) + .withGranularitySpec(new UserCompactionTaskGranularityConfig(Granularities.DAY, null, null)) + .build(); + + final CompactionState lastCompactionState = createCompactionStateWithGranularity(Granularities.HOUR); + List segments = List.of( + DataSegment.builder(WIKI_SEGMENT).lastCompactionState(lastCompactionState).build(), + DataSegment.builder(WIKI_SEGMENT_2).lastCompactionState(lastCompactionState).build() + ); + + final CompactionStatus status = CompactionStatus.compute( + CompactionCandidate.from(segments, null), + compactionConfig, + compactionStateManager + ); + Assert.assertFalse(status.isComplete()); + Assert.assertTrue(status.isSkipped()); + Assert.assertTrue(status.getReason().contains("'inputSegmentSize' exceeded")); + Assert.assertTrue(status.getReason().contains("200000000")); + Assert.assertTrue(status.getReason().contains("150000000")); + } + + /** + * Verify that the evaluation indicates compaction is needed for the expected reason. + * Allows customization of the segments in the compaction candidate. + */ + private void verifyEvaluationNeedsCompactionBecauseWithCustomSegments( + CompactionCandidate candidate, + DataSourceCompactionConfig compactionConfig, + String expectedReason, + CompactionStateManager compactionStateManager + ) + { + final CompactionStatus status = CompactionStatus.compute( + candidate, + compactionConfig, + compactionStateManager + ); + + Assert.assertFalse(status.isComplete()); + Assert.assertEquals(expectedReason, status.getReason()); } private void verifyCompactionStatusIsPendingBecause( @@ -552,8 +824,9 @@ private void verifyCompactionStatusIsPendingBecause( .build(); final CompactionStatus status = CompactionStatus.compute( CompactionCandidate.from(List.of(segment), null), - compactionConfig - ); + compactionConfig, + compactionStateManager + ); Assert.assertFalse(status.isComplete()); Assert.assertEquals(expectedReason, status.getReason()); @@ -580,4 +853,20 @@ private static UserCompactionTaskQueryTuningConfig createTuningConfig( null, null, null, null, null, null, null, null, null, null ); } + + /** + * Simple helper to create a CompactionState with only segmentGranularity set + */ + private static CompactionState createCompactionStateWithGranularity(Granularity segmentGranularity) + { + return new CompactionState( + null, + null, + null, + null, + IndexSpec.getDefault(), + new UniformGranularitySpec(segmentGranularity, 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 1c92c9a249c9..23853a9ca6ba 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 @@ -47,6 +47,8 @@ import org.apache.druid.segment.TestDataSource; import org.apache.druid.segment.data.ConciseBitmapSerdeFactory; import org.apache.druid.segment.incremental.OnheapIncrementalIndex; +import org.apache.druid.segment.metadata.CompactionStateManager; +import org.apache.druid.segment.metadata.HeapMemoryCompactionStateManager; import org.apache.druid.segment.transform.CompactionTransformSpec; import org.apache.druid.server.coordinator.CreateDataSegments; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; @@ -81,6 +83,7 @@ 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 final CompactionStateManager compactionStateManager = new HeapMemoryCompactionStateManager(); @Test public void testLargeOffsetAndSmallSegmentInterval() @@ -276,8 +279,9 @@ public void testSkipDataSourceWithNoSegments() .withNumPartitions(4) ) ), - Collections.emptyMap() - ); + Collections.emptyMap(), + compactionStateManager + ); assertCompactSegmentIntervals( iterator, @@ -508,8 +512,9 @@ 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") ) - ) - ); + ), + compactionStateManager + ); assertCompactSegmentIntervals( iterator, @@ -547,8 +552,9 @@ 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") ) - ) - ); + ), + compactionStateManager + ); assertCompactSegmentIntervals( iterator, @@ -1402,7 +1408,7 @@ public void testIteratorDoesNotReturnsSegmentsWhenPartitionDimensionsPrefixed() } @Test - public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentFilter() throws Exception + public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentFilter() { // Same indexSpec as what is set in the auto compaction config IndexSpec indexSpec = IndexSpec.getDefault(); @@ -2052,8 +2058,9 @@ TestDataSource.KOALA, configBuilder().forDataSource(TestDataSource.KOALA).build( TestDataSource.WIKI, SegmentTimeline.forSegments(wikiSegments), TestDataSource.KOALA, SegmentTimeline.forSegments(koalaSegments) ), - Collections.emptyMap() - ); + Collections.emptyMap(), + compactionStateManager + ); // Verify that the segments of WIKI are preferred even though they are older Assert.assertTrue(iterator.hasNext()); @@ -2073,8 +2080,9 @@ private CompactionSegmentIterator createIterator(DataSourceCompactionConfig conf policy, Collections.singletonMap(TestDataSource.WIKI, config), Collections.singletonMap(TestDataSource.WIKI, timeline), - Collections.emptyMap() - ); + Collections.emptyMap(), + compactionStateManager + ); } private static void assertCompactSegmentIntervals( 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 380533c72905..c14bd4540823 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 @@ -63,6 +63,7 @@ public class CreateDataSegments private String upgradedFromSegmentId; private String schemaFingerprint; private Long numRows; + private String compactionStateFingerprint; public static CreateDataSegments ofDatasource(String datasource) { @@ -127,6 +128,12 @@ public CreateDataSegments withSchemaFingerprint(String schemaFingerprint) return this; } + public CreateDataSegments withCompactionStateFingerprint(String compactionStateFingerprint) + { + this.compactionStateFingerprint = compactionStateFingerprint; + return this; + } + public CreateDataSegments markUnused() { this.used = false; @@ -187,7 +194,8 @@ public List eachOfSize(long sizeInBytes) new NumberedShardSpec(numPartition, numPartitions), ++uniqueIdInInterval, compactionState, - sizeInBytes + sizeInBytes, + compactionStateFingerprint ) ); } @@ -206,7 +214,8 @@ private DataSegmentPlus plus(DataSegment segment) used, schemaFingerprint, numRows, - upgradedFromSegmentId + upgradedFromSegmentId, + compactionStateFingerprint ); } @@ -225,7 +234,8 @@ private NumberedDataSegment( NumberedShardSpec shardSpec, int uniqueId, CompactionState compactionState, - long size + long size, + String compactionStateFingerprint ) { super( @@ -240,6 +250,7 @@ private NumberedDataSegment( compactionState, IndexIO.CURRENT_VERSION_ID, size, + compactionStateFingerprint, PruneSpecsHolder.DEFAULT ); this.uniqueId = uniqueId; diff --git a/server/src/test/java/org/apache/druid/server/coordinator/DataSourceCompactionConfigAuditEntryTest.java b/server/src/test/java/org/apache/druid/server/coordinator/DataSourceCompactionConfigAuditEntryTest.java index 65ea53586d3f..d32f64a49aef 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/DataSourceCompactionConfigAuditEntryTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/DataSourceCompactionConfigAuditEntryTest.java @@ -33,7 +33,7 @@ public class DataSourceCompactionConfigAuditEntryTest private final AuditInfo auditInfo = new AuditInfo("author", "identity", "comment", "ip"); private final DataSourceCompactionConfigAuditEntry firstEntry = new DataSourceCompactionConfigAuditEntry( - new ClusterCompactionConfig(0.1, 9, null, null, null), + new ClusterCompactionConfig(0.1, 9, null, null, null, null), InlineSchemaDataSourceCompactionConfig.builder().forDataSource(TestDataSource.WIKI).build(), auditInfo, DateTimes.nowUtc() @@ -43,7 +43,7 @@ public class DataSourceCompactionConfigAuditEntryTest public void testhasSameConfigWithSameBaseConfigIsTrue() { final DataSourceCompactionConfigAuditEntry secondEntry = new DataSourceCompactionConfigAuditEntry( - new ClusterCompactionConfig(0.1, 9, null, null, null), + new ClusterCompactionConfig(0.1, 9, null, null, null, null), InlineSchemaDataSourceCompactionConfig.builder().forDataSource(TestDataSource.WIKI).build(), auditInfo, DateTimes.nowUtc() @@ -56,7 +56,7 @@ public void testhasSameConfigWithSameBaseConfigIsTrue() public void testhasSameConfigWithDifferentClusterConfigIsFalse() { DataSourceCompactionConfigAuditEntry secondEntry = new DataSourceCompactionConfigAuditEntry( - new ClusterCompactionConfig(0.2, 9, null, null, null), + new ClusterCompactionConfig(0.2, 9, null, null, null, null), InlineSchemaDataSourceCompactionConfig.builder().forDataSource(TestDataSource.WIKI).build(), auditInfo, DateTimes.nowUtc() @@ -65,7 +65,7 @@ public void testhasSameConfigWithDifferentClusterConfigIsFalse() Assert.assertFalse(secondEntry.hasSameConfig(firstEntry)); secondEntry = new DataSourceCompactionConfigAuditEntry( - new ClusterCompactionConfig(0.1, 10, null, null, null), + new ClusterCompactionConfig(0.1, 10, null, null, null, null), InlineSchemaDataSourceCompactionConfig.builder().forDataSource(TestDataSource.WIKI).build(), auditInfo, DateTimes.nowUtc() @@ -78,7 +78,7 @@ public void testhasSameConfigWithDifferentClusterConfigIsFalse() public void testhasSameConfigWithDifferentDatasourceConfigIsFalse() { DataSourceCompactionConfigAuditEntry secondEntry = new DataSourceCompactionConfigAuditEntry( - new ClusterCompactionConfig(0.1, 9, null, null, null), + new ClusterCompactionConfig(0.1, 9, null, null, null, null), InlineSchemaDataSourceCompactionConfig.builder().forDataSource(TestDataSource.KOALA).build(), auditInfo, DateTimes.nowUtc() @@ -91,7 +91,7 @@ public void testhasSameConfigWithDifferentDatasourceConfigIsFalse() public void testhasSameConfigWithNullDatasourceConfigIsFalse() { final DataSourceCompactionConfigAuditEntry secondEntry = new DataSourceCompactionConfigAuditEntry( - new ClusterCompactionConfig(0.1, 9, null, null, null), + new ClusterCompactionConfig(0.1, 9, null, null, null, null), null, auditInfo, DateTimes.nowUtc() diff --git a/server/src/test/java/org/apache/druid/server/coordinator/DataSourceCompactionConfigHistoryTest.java b/server/src/test/java/org/apache/druid/server/coordinator/DataSourceCompactionConfigHistoryTest.java index 7208bdf0bce5..35cb1177af59 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/DataSourceCompactionConfigHistoryTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/DataSourceCompactionConfigHistoryTest.java @@ -177,7 +177,7 @@ public void testAddAndModifyClusterConfigShouldAddTwice() wikiAuditHistory.add(originalConfig, auditInfo, DateTimes.nowUtc()); final DruidCompactionConfig updatedConfig = originalConfig.withClusterConfig( - new ClusterCompactionConfig(0.2, null, null, null, null) + new ClusterCompactionConfig(0.2, null, null, null, null, null) ); wikiAuditHistory.add(updatedConfig, auditInfo, DateTimes.nowUtc()); diff --git a/server/src/test/java/org/apache/druid/server/coordinator/DruidCompactionConfigTest.java b/server/src/test/java/org/apache/druid/server/coordinator/DruidCompactionConfigTest.java index c64a254ae6ee..2f22ed66e6fb 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/DruidCompactionConfigTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/DruidCompactionConfigTest.java @@ -65,6 +65,7 @@ public void testSerdeWithDatasourceConfigs() throws Exception null, null, null, + null, null ); @@ -83,7 +84,8 @@ public void testCopyWithClusterConfig() 10, new NewestSegmentFirstPolicy(null), true, - CompactionEngine.MSQ + CompactionEngine.MSQ, + true ); final DruidCompactionConfig copy = config.withClusterConfig(clusterConfig); @@ -118,5 +120,6 @@ public void testDefaultConfigValues() Assert.assertEquals(CompactionEngine.NATIVE, config.getEngine()); Assert.assertEquals(0.1, config.getCompactionTaskSlotRatio(), 1e-9); Assert.assertEquals(Integer.MAX_VALUE, config.getMaxCompactionTaskSlots()); + Assert.assertTrue(config.isLegacyPersistLastCompactionStateInSegments()); } } diff --git a/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorConfigTest.java b/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorConfigTest.java index d91cb62050a1..9390ffaa20d7 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorConfigTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorConfigTest.java @@ -256,6 +256,10 @@ public void testCoordinatorKillConfigOverrideValues() props.setProperty("druid.coordinator.kill.segmentSchema.period", "PT2H"); props.setProperty("druid.coordinator.kill.segmentSchema.durationToRetain", "PT8H"); + props.setProperty("druid.coordinator.kill.compactionState.on", "false"); + props.setProperty("druid.coordinator.kill.compactionState.period", "PT2H"); + props.setProperty("druid.coordinator.kill.compactionState.durationToRetain", "PT8H"); + final CoordinatorKillConfigs killConfigs = deserializeFrom(props, "druid.coordinator.kill", CoordinatorKillConfigs.class); @@ -283,6 +287,10 @@ public void testCoordinatorKillConfigOverrideValues() new MetadataCleanupConfig(false, Duration.standardHours(2), Duration.standardHours(8)), killConfigs.segmentSchemas() ); + Assert.assertEquals( + new MetadataCleanupConfig(false, Duration.standardHours(2), Duration.standardHours(8)), + killConfigs.compactionStates() + ); Assert.assertFalse(killConfigs.pendingSegments().isCleanupEnabled()); } @@ -330,6 +338,12 @@ public void testCoordinatorConfigFailsWhenCleanupPeriodIsInvalid() "'druid.coordinator.kill.segmentSchema.period'[PT1800S] must be greater than" + " 'druid.coordinator.period.metadataStoreManagementPeriod'[PT3600S]" ); + verifyCoordinatorConfigFailsWith( + createKillConfig().compactionState(cleanupConfig).build(), + periodConfig, + "'druid.coordinator.kill.compactionState.period'[PT1800S] must be greater than" + + " 'druid.coordinator.period.metadataStoreManagementPeriod'[PT3600S]" + ); } @Test @@ -369,6 +383,11 @@ public void testCoordinatorConfigFailsWhenRetainDurationIsNegative() defaultPeriodConfig, "'druid.coordinator.kill.segmentSchema.durationToRetain'[PT-1S] must be 0 milliseconds or higher" ); + verifyCoordinatorConfigFailsWith( + createKillConfig().compactionState(cleanupConfig).build(), + defaultPeriodConfig, + "'druid.coordinator.kill.compactionState.durationToRetain'[PT-1S] must be 0 milliseconds or higher" + ); } @Test @@ -414,6 +433,13 @@ public void testCoordinatorConfigFailsWhenRetainDurationIsHigherThanCurrentTime( + " greater than current time in milliseconds", futureRetainDuration ); + verifyCoordinatorConfigFailsWith( + createKillConfig().compactionState(cleanupConfig).build(), + defaultPeriodConfig, + "'druid.coordinator.kill.compactionState.durationToRetain'[%s] cannot be" + + " greater than current time in milliseconds", + futureRetainDuration + ); } @Test @@ -485,6 +511,7 @@ private static class KillConfigBuilder MetadataCleanupConfig pendingSegments; MetadataCleanupConfig segmentSchema; KillUnusedSegmentsConfig unusedSegments; + MetadataCleanupConfig compactionState; KillConfigBuilder audit(MetadataCleanupConfig config) { @@ -522,6 +549,12 @@ KillConfigBuilder segmentSchema(MetadataCleanupConfig config) return this; } + KillConfigBuilder compactionState(MetadataCleanupConfig config) + { + this.compactionState = config; + return this; + } + KillConfigBuilder unusedSegments(KillUnusedSegmentsConfig config) { this.unusedSegments = config; @@ -538,6 +571,7 @@ CoordinatorKillConfigs build() rules, compaction, segmentSchema, + compactionState, unusedSegments == null ? null : unusedSegments.isCleanupEnabled(), unusedSegments == null ? null : unusedSegments.getCleanupPeriod(), unusedSegments == null ? null : unusedSegments.getDurationToRetain(), 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 39213ff8616d..391f5f66c784 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 @@ -46,6 +46,7 @@ import org.apache.druid.metadata.segment.cache.NoopSegmentMetadataCache; import org.apache.druid.rpc.indexing.OverlordClient; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; +import org.apache.druid.segment.metadata.CompactionStateManager; import org.apache.druid.server.DruidNode; import org.apache.druid.server.compaction.CompactionSimulateResult; import org.apache.druid.server.compaction.CompactionStatusTracker; @@ -112,6 +113,7 @@ public class DruidCoordinatorTest private OverlordClient overlordClient; private CompactionStatusTracker statusTracker; private LatchableServiceEmitter serviceEmitter; + private CompactionStateManager compactionStateManager; @Before public void setUp() throws Exception @@ -122,6 +124,7 @@ public void setUp() throws Exception metadataRuleManager = EasyMock.createNiceMock(MetadataRuleManager.class); loadQueueTaskMaster = EasyMock.createMock(LoadQueueTaskMaster.class); overlordClient = EasyMock.createMock(OverlordClient.class); + compactionStateManager = EasyMock.createMock(CompactionStateManager.class); JacksonConfigManager configManager = EasyMock.createNiceMock(JacksonConfigManager.class); EasyMock.expect( @@ -170,7 +173,8 @@ public void setUp() throws Exception CentralizedDatasourceSchemaConfig.create(), new CompactionStatusTracker(), EasyMock.niceMock(CoordinatorDynamicConfigSyncer.class), - EasyMock.niceMock(CloneStatusManager.class) + EasyMock.niceMock(CloneStatusManager.class), + compactionStateManager ); } @@ -184,7 +188,8 @@ private MetadataManager createMetadataManager(JacksonConfigManager configManager metadataRuleManager, null, null, - NoopSegmentMetadataCache.instance() + NoopSegmentMetadataCache.instance(), + compactionStateManager ); } @@ -482,7 +487,8 @@ public void testCompactSegmentsDutyWhenCustomDutyGroupEmpty() CentralizedDatasourceSchemaConfig.create(), new CompactionStatusTracker(), EasyMock.niceMock(CoordinatorDynamicConfigSyncer.class), - EasyMock.niceMock(CloneStatusManager.class) + EasyMock.niceMock(CloneStatusManager.class), + compactionStateManager ); coordinator.start(); @@ -534,7 +540,8 @@ public void testInitializeCompactSegmentsDutyWhenCustomDutyGroupDoesNotContainsC CentralizedDatasourceSchemaConfig.create(), new CompactionStatusTracker(), EasyMock.niceMock(CoordinatorDynamicConfigSyncer.class), - EasyMock.niceMock(CloneStatusManager.class) + EasyMock.niceMock(CloneStatusManager.class), + compactionStateManager ); coordinator.start(); // Since CompactSegments is not enabled in Custom Duty Group, then CompactSegments must be created in IndexingServiceDuties @@ -565,7 +572,7 @@ public void testInitializeCompactSegmentsDutyWhenCustomDutyGroupContainsCompactS CoordinatorCustomDutyGroup compactSegmentCustomGroup = new CoordinatorCustomDutyGroup( "group1", Duration.standardSeconds(1), - ImmutableList.of(new CompactSegments(statusTracker, null)) + ImmutableList.of(new CompactSegments(statusTracker, null, compactionStateManager)) ); CoordinatorCustomDutyGroups customDutyGroups = new CoordinatorCustomDutyGroups(ImmutableSet.of(compactSegmentCustomGroup)); coordinator = new DruidCoordinator( @@ -586,7 +593,8 @@ public void testInitializeCompactSegmentsDutyWhenCustomDutyGroupContainsCompactS CentralizedDatasourceSchemaConfig.create(), new CompactionStatusTracker(), EasyMock.niceMock(CoordinatorDynamicConfigSyncer.class), - EasyMock.niceMock(CloneStatusManager.class) + EasyMock.niceMock(CloneStatusManager.class), + compactionStateManager ); coordinator.start(); @@ -696,7 +704,8 @@ public void testCoordinatorCustomDutyGroupsRunAsExpected() throws Exception CentralizedDatasourceSchemaConfig.create(), new CompactionStatusTracker(), EasyMock.niceMock(CoordinatorDynamicConfigSyncer.class), - EasyMock.niceMock(CloneStatusManager.class) + EasyMock.niceMock(CloneStatusManager.class), + compactionStateManager ); coordinator.start(); @@ -829,7 +838,7 @@ public void testSimulateRunWithEmptyDatasourceCompactionConfigs() .anyTimes(); EasyMock.replay(segmentsMetadataManager); CompactionSimulateResult result = coordinator.simulateRunWithConfigUpdate( - new ClusterCompactionConfig(0.2, null, null, null, null) + new ClusterCompactionConfig(0.2, null, null, null, null, null) ); Assert.assertEquals(Collections.emptyMap(), result.getCompactionStates()); } 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 301dd77493c2..d39f3400b309 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 @@ -79,6 +79,8 @@ import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.segment.indexing.BatchIOConfig; +import org.apache.druid.segment.metadata.CompactionStateManager; +import org.apache.druid.segment.metadata.HeapMemoryCompactionStateManager; import org.apache.druid.segment.transform.CompactionTransformSpec; import org.apache.druid.server.compaction.CompactionCandidate; import org.apache.druid.server.compaction.CompactionCandidateSearchPolicy; @@ -197,6 +199,7 @@ public static Collection constructorFeeder() private DataSourcesSnapshot dataSources; private CompactionStatusTracker statusTracker; private final Map> datasourceToSegments = new HashMap<>(); + private final CompactionStateManager compactionStateManager = new HeapMemoryCompactionStateManager(); public CompactSegmentsTest( PartitionsSpec partitionsSpec, @@ -273,11 +276,12 @@ public void testSerde() throws Exception .addValue(DruidCoordinatorConfig.class, COORDINATOR_CONFIG) .addValue(OverlordClient.class, overlordClient) .addValue(CompactionStatusTracker.class, statusTracker) + .addValue(CompactionStateManager.class, compactionStateManager) .addValue(MetadataCatalog.class, NullMetadataCatalog.INSTANCE) .addValue(ExprMacroTable.class, TestExprMacroTable.INSTANCE) ); - final CompactSegments compactSegments = new CompactSegments(statusTracker, overlordClient); + final CompactSegments compactSegments = new CompactSegments(statusTracker, overlordClient, compactionStateManager); String compactSegmentString = JSON_MAPPER.writeValueAsString(compactSegments); CompactSegments serdeCompactSegments = JSON_MAPPER.readValue(compactSegmentString, CompactSegments.class); @@ -289,7 +293,7 @@ public void testSerde() throws Exception public void testRun() { final TestOverlordClient overlordClient = new TestOverlordClient(JSON_MAPPER); - final CompactSegments compactSegments = new CompactSegments(statusTracker, overlordClient); + final CompactSegments compactSegments = new CompactSegments(statusTracker, overlordClient, compactionStateManager); final Supplier expectedVersionSupplier = new Supplier<>() { @@ -372,7 +376,7 @@ public void testRun_withFixedIntervalOrderPolicy() public void testMakeStats() { final TestOverlordClient overlordClient = new TestOverlordClient(JSON_MAPPER); - final CompactSegments compactSegments = new CompactSegments(statusTracker, overlordClient); + final CompactSegments compactSegments = new CompactSegments(statusTracker, overlordClient, compactionStateManager); // Before any compaction, we do not have any snapshot of compactions Map autoCompactionSnapshots = compactSegments.getAutoCompactionSnapshot(); @@ -496,7 +500,7 @@ public void testMakeStatsForDataSourceWithCompactedIntervalBetweenNonCompactedIn dataSources = DataSourcesSnapshot.fromUsedSegments(segments); final TestOverlordClient overlordClient = new TestOverlordClient(JSON_MAPPER); - final CompactSegments compactSegments = new CompactSegments(statusTracker, overlordClient); + final CompactSegments compactSegments = new CompactSegments(statusTracker, overlordClient, compactionStateManager); // Before any compaction, we do not have any snapshot of compactions Map autoCompactionSnapshots = compactSegments.getAutoCompactionSnapshot(); @@ -558,7 +562,7 @@ public void testMakeStatsForDataSourceWithCompactedIntervalBetweenNonCompactedIn public void testMakeStatsWithDeactivatedDatasource() { final TestOverlordClient overlordClient = new TestOverlordClient(JSON_MAPPER); - final CompactSegments compactSegments = new CompactSegments(statusTracker, overlordClient); + final CompactSegments compactSegments = new CompactSegments(statusTracker, overlordClient, compactionStateManager); // Before any compaction, we do not have any snapshot of compactions Map autoCompactionSnapshots = compactSegments.getAutoCompactionSnapshot(); @@ -650,7 +654,7 @@ public void testMakeStatsForDataSourceWithSkipped() dataSources = DataSourcesSnapshot.fromUsedSegments(segments); final TestOverlordClient overlordClient = new TestOverlordClient(JSON_MAPPER); - final CompactSegments compactSegments = new CompactSegments(statusTracker, overlordClient); + final CompactSegments compactSegments = new CompactSegments(statusTracker, overlordClient, compactionStateManager); // Before any compaction, we do not have any snapshot of compactions Map autoCompactionSnapshots = compactSegments.getAutoCompactionSnapshot(); @@ -709,7 +713,7 @@ public void testMakeStatsForDataSourceWithSkipped() public void testRunMultipleCompactionTaskSlots() { final TestOverlordClient overlordClient = new TestOverlordClient(JSON_MAPPER); - final CompactSegments compactSegments = new CompactSegments(statusTracker, overlordClient); + final CompactSegments compactSegments = new CompactSegments(statusTracker, overlordClient, compactionStateManager); final CoordinatorRunStats stats = doCompactSegments(compactSegments, 3); Assert.assertEquals(3, stats.get(Stats.Compaction.AVAILABLE_SLOTS)); @@ -729,7 +733,7 @@ public void testRunMultipleCompactionTaskSlotsWithUseAutoScaleSlotsOverMaxSlot() int maxCompactionSlot = 3; Assert.assertTrue(maxCompactionSlot < MAXIMUM_CAPACITY_WITH_AUTO_SCALE); final TestOverlordClient overlordClient = new TestOverlordClient(JSON_MAPPER); - final CompactSegments compactSegments = new CompactSegments(statusTracker, overlordClient); + final CompactSegments compactSegments = new CompactSegments(statusTracker, overlordClient, compactionStateManager); final CoordinatorRunStats stats = doCompactSegments(compactSegments, createCompactionConfigs(), maxCompactionSlot); Assert.assertEquals(maxCompactionSlot, stats.get(Stats.Compaction.AVAILABLE_SLOTS)); @@ -749,7 +753,7 @@ public void testRunMultipleCompactionTaskSlotsWithUseAutoScaleSlotsUnderMaxSlot( int maxCompactionSlot = 100; Assert.assertFalse(maxCompactionSlot < MAXIMUM_CAPACITY_WITH_AUTO_SCALE); final TestOverlordClient overlordClient = new TestOverlordClient(JSON_MAPPER); - final CompactSegments compactSegments = new CompactSegments(statusTracker, overlordClient); + final CompactSegments compactSegments = new CompactSegments(statusTracker, overlordClient, compactionStateManager); final CoordinatorRunStats stats = doCompactSegments(compactSegments, createCompactionConfigs(), maxCompactionSlot); Assert.assertEquals(MAXIMUM_CAPACITY_WITH_AUTO_SCALE, stats.get(Stats.Compaction.AVAILABLE_SLOTS)); @@ -771,7 +775,7 @@ public void testCompactWithoutGranularitySpec() { final OverlordClient mockClient = Mockito.mock(OverlordClient.class); final ArgumentCaptor payloadCaptor = setUpMockClient(mockClient); - final CompactSegments compactSegments = new CompactSegments(statusTracker, mockClient); + final CompactSegments compactSegments = new CompactSegments(statusTracker, mockClient, compactionStateManager); final List compactionConfigs = new ArrayList<>(); final String dataSource = DATA_SOURCE_PREFIX + 0; compactionConfigs.add( @@ -802,7 +806,7 @@ public void testCompactWithNotNullIOConfig() { final OverlordClient mockClient = Mockito.mock(OverlordClient.class); final ArgumentCaptor payloadCaptor = setUpMockClient(mockClient); - final CompactSegments compactSegments = new CompactSegments(statusTracker, mockClient); + final CompactSegments compactSegments = new CompactSegments(statusTracker, mockClient, compactionStateManager); final List compactionConfigs = new ArrayList<>(); final String dataSource = DATA_SOURCE_PREFIX + 0; compactionConfigs.add( @@ -826,7 +830,7 @@ public void testCompactWithNullIOConfig() { final OverlordClient mockClient = Mockito.mock(OverlordClient.class); final ArgumentCaptor payloadCaptor = setUpMockClient(mockClient); - final CompactSegments compactSegments = new CompactSegments(statusTracker, mockClient); + final CompactSegments compactSegments = new CompactSegments(statusTracker, mockClient, compactionStateManager); final List compactionConfigs = new ArrayList<>(); final String dataSource = DATA_SOURCE_PREFIX + 0; compactionConfigs.add( @@ -849,7 +853,7 @@ public void testCompactWithGranularitySpec() { final OverlordClient mockClient = Mockito.mock(OverlordClient.class); final ArgumentCaptor payloadCaptor = setUpMockClient(mockClient); - final CompactSegments compactSegments = new CompactSegments(statusTracker, mockClient); + final CompactSegments compactSegments = new CompactSegments(statusTracker, mockClient, compactionStateManager); final List compactionConfigs = new ArrayList<>(); final String dataSource = DATA_SOURCE_PREFIX + 0; compactionConfigs.add( @@ -886,7 +890,7 @@ public void testCompactWithDimensionSpec() { final OverlordClient mockClient = Mockito.mock(OverlordClient.class); final ArgumentCaptor payloadCaptor = setUpMockClient(mockClient); - final CompactSegments compactSegments = new CompactSegments(statusTracker, mockClient); + final CompactSegments compactSegments = new CompactSegments(statusTracker, mockClient, compactionStateManager); final List compactionConfigs = new ArrayList<>(); final String dataSource = DATA_SOURCE_PREFIX + 0; compactionConfigs.add( @@ -917,7 +921,7 @@ public void testCompactWithoutDimensionSpec() { final OverlordClient mockClient = Mockito.mock(OverlordClient.class); final ArgumentCaptor payloadCaptor = setUpMockClient(mockClient); - final CompactSegments compactSegments = new CompactSegments(statusTracker, mockClient); + final CompactSegments compactSegments = new CompactSegments(statusTracker, mockClient, compactionStateManager); final List compactionConfigs = new ArrayList<>(); final String dataSource = DATA_SOURCE_PREFIX + 0; compactionConfigs.add( @@ -940,7 +944,7 @@ public void testCompactWithProjections() { final OverlordClient mockClient = Mockito.mock(OverlordClient.class); final ArgumentCaptor payloadCaptor = setUpMockClient(mockClient); - final CompactSegments compactSegments = new CompactSegments(statusTracker, mockClient); + final CompactSegments compactSegments = new CompactSegments(statusTracker, mockClient, compactionStateManager); final List compactionConfigs = new ArrayList<>(); final String dataSource = DATA_SOURCE_PREFIX + 0; final List projections = List.of( @@ -1015,7 +1019,7 @@ public void testCompactWithCatalogProjections() ) .buildSpec() ); - final CompactSegments compactSegments = new CompactSegments(statusTracker, mockClient); + final CompactSegments compactSegments = new CompactSegments(statusTracker, mockClient, compactionStateManager); final List compactionConfigs = new ArrayList<>(); compactionConfigs.add( new CatalogDataSourceCompactionConfig( @@ -1041,7 +1045,7 @@ public void testCompactWithRollupInGranularitySpec() { final OverlordClient mockClient = Mockito.mock(OverlordClient.class); final ArgumentCaptor payloadCaptor = setUpMockClient(mockClient); - final CompactSegments compactSegments = new CompactSegments(statusTracker, mockClient); + final CompactSegments compactSegments = new CompactSegments(statusTracker, mockClient, compactionStateManager); final List compactionConfigs = new ArrayList<>(); final String dataSource = DATA_SOURCE_PREFIX + 0; compactionConfigs.add( @@ -1132,7 +1136,7 @@ public void testCompactWithGranularitySpecConflictWithActiveCompactionTask() Mockito.when(mockClient.taskPayload(ArgumentMatchers.eq(conflictTaskId))) .thenReturn(Futures.immediateFuture(runningConflictCompactionTaskPayload)); - final CompactSegments compactSegments = new CompactSegments(statusTracker, mockClient); + final CompactSegments compactSegments = new CompactSegments(statusTracker, mockClient, compactionStateManager); final List compactionConfigs = new ArrayList<>(); compactionConfigs.add( InlineSchemaDataSourceCompactionConfig.builder() @@ -1171,7 +1175,7 @@ public void testCompactWithGranularitySpecConflictWithActiveCompactionTask() public void testIntervalIsCompactedAgainWhenSegmentIsAdded() { final TestOverlordClient overlordClient = new TestOverlordClient(JSON_MAPPER); - final CompactSegments compactSegments = new CompactSegments(statusTracker, overlordClient); + final CompactSegments compactSegments = new CompactSegments(statusTracker, overlordClient, compactionStateManager); final String dataSource = DATA_SOURCE_PREFIX + 0; final DataSourceCompactionConfig compactionConfig = InlineSchemaDataSourceCompactionConfig @@ -1217,7 +1221,7 @@ public void testIntervalIsCompactedAgainWhenSegmentIsAdded() public void testRunParallelCompactionMultipleCompactionTaskSlots() { final TestOverlordClient overlordClient = new TestOverlordClient(JSON_MAPPER); - final CompactSegments compactSegments = new CompactSegments(statusTracker, overlordClient); + final CompactSegments compactSegments = new CompactSegments(statusTracker, overlordClient, compactionStateManager); final CoordinatorRunStats stats; // Native uses maxNumConcurrentSubTasks for task slots whereas MSQ uses maxNumTasks. if (engine == CompactionEngine.NATIVE) { @@ -1254,7 +1258,7 @@ public void testRunWithLockedIntervals() // Verify that locked intervals are skipped and only one compaction task // is submitted for dataSource_0 - CompactSegments compactSegments = new CompactSegments(statusTracker, overlordClient); + CompactSegments compactSegments = new CompactSegments(statusTracker, overlordClient, compactionStateManager); final CoordinatorRunStats stats = doCompactSegments(compactSegments, createcompactionConfigsForNative(2), 4); Assert.assertEquals(1, stats.get(Stats.Compaction.SUBMITTED_TASKS)); @@ -1273,7 +1277,7 @@ public void testCompactWithTransformSpec() { final OverlordClient mockClient = Mockito.mock(OverlordClient.class); final ArgumentCaptor payloadCaptor = setUpMockClient(mockClient); - final CompactSegments compactSegments = new CompactSegments(statusTracker, mockClient); + final CompactSegments compactSegments = new CompactSegments(statusTracker, mockClient, compactionStateManager); final List compactionConfigs = new ArrayList<>(); final String dataSource = DATA_SOURCE_PREFIX + 0; compactionConfigs.add( @@ -1302,7 +1306,7 @@ public void testCompactWithoutCustomSpecs() { final OverlordClient mockClient = Mockito.mock(OverlordClient.class); final ArgumentCaptor payloadCaptor = setUpMockClient(mockClient); - final CompactSegments compactSegments = new CompactSegments(statusTracker, mockClient); + final CompactSegments compactSegments = new CompactSegments(statusTracker, mockClient, compactionStateManager); final List compactionConfigs = new ArrayList<>(); final String dataSource = DATA_SOURCE_PREFIX + 0; compactionConfigs.add( @@ -1327,7 +1331,7 @@ public void testCompactWithMetricsSpec() AggregatorFactory[] aggregatorFactories = new AggregatorFactory[] {new CountAggregatorFactory("cnt")}; final OverlordClient mockClient = Mockito.mock(OverlordClient.class); final ArgumentCaptor payloadCaptor = setUpMockClient(mockClient); - final CompactSegments compactSegments = new CompactSegments(statusTracker, mockClient); + final CompactSegments compactSegments = new CompactSegments(statusTracker, mockClient, compactionStateManager); final List compactionConfigs = new ArrayList<>(); final String dataSource = DATA_SOURCE_PREFIX + 0; compactionConfigs.add( @@ -1383,7 +1387,7 @@ public void testDetermineSegmentGranularityFromSegmentsToCompact() final OverlordClient mockClient = Mockito.mock(OverlordClient.class); final ArgumentCaptor payloadCaptor = setUpMockClient(mockClient); - final CompactSegments compactSegments = new CompactSegments(statusTracker, mockClient); + final CompactSegments compactSegments = new CompactSegments(statusTracker, mockClient, compactionStateManager); final List compactionConfigs = new ArrayList<>(); compactionConfigs.add( InlineSchemaDataSourceCompactionConfig.builder() @@ -1443,7 +1447,7 @@ public void testDetermineSegmentGranularityFromSegmentGranularityInCompactionCon final OverlordClient mockClient = Mockito.mock(OverlordClient.class); final ArgumentCaptor payloadCaptor = setUpMockClient(mockClient); - final CompactSegments compactSegments = new CompactSegments(statusTracker, mockClient); + final CompactSegments compactSegments = new CompactSegments(statusTracker, mockClient, compactionStateManager); final List compactionConfigs = new ArrayList<>(); compactionConfigs.add( InlineSchemaDataSourceCompactionConfig.builder() @@ -1476,7 +1480,7 @@ public void testCompactWithMetricsSpecShouldSetPreserveExistingMetricsTrue() { final OverlordClient mockClient = Mockito.mock(OverlordClient.class); final ArgumentCaptor payloadCaptor = setUpMockClient(mockClient); - final CompactSegments compactSegments = new CompactSegments(statusTracker, mockClient); + final CompactSegments compactSegments = new CompactSegments(statusTracker, mockClient, compactionStateManager); final List compactionConfigs = new ArrayList<>(); final String dataSource = DATA_SOURCE_PREFIX + 0; compactionConfigs.add( @@ -1503,7 +1507,7 @@ public void testCompactWithoutMetricsSpecShouldSetPreserveExistingMetricsFalse() { final OverlordClient mockClient = Mockito.mock(OverlordClient.class); final ArgumentCaptor payloadCaptor = setUpMockClient(mockClient); - final CompactSegments compactSegments = new CompactSegments(statusTracker, mockClient); + final CompactSegments compactSegments = new CompactSegments(statusTracker, mockClient, compactionStateManager); final List compactionConfigs = new ArrayList<>(); final String dataSource = DATA_SOURCE_PREFIX + 0; compactionConfigs.add( @@ -1657,6 +1661,7 @@ private CoordinatorRunStats doCompactSegments( numCompactionTaskSlots, policy, null, + null, null ) ) diff --git a/server/src/test/java/org/apache/druid/server/coordinator/duty/KillUnreferencedCompactionStateTest.java b/server/src/test/java/org/apache/druid/server/coordinator/duty/KillUnreferencedCompactionStateTest.java new file mode 100644 index 000000000000..49fffbde3346 --- /dev/null +++ b/server/src/test/java/org/apache/druid/server/coordinator/duty/KillUnreferencedCompactionStateTest.java @@ -0,0 +1,262 @@ +/* + * 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.coordinator.duty; + +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; +import org.apache.druid.jackson.DefaultObjectMapper; +import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.metadata.MetadataStorageTablesConfig; +import org.apache.druid.metadata.TestDerbyConnector; +import org.apache.druid.segment.IndexSpec; +import org.apache.druid.segment.metadata.CompactionStateManager; +import org.apache.druid.segment.metadata.CompactionStateManagerConfig; +import org.apache.druid.server.coordinator.DruidCoordinatorRuntimeParams; +import org.apache.druid.server.coordinator.config.MetadataCleanupConfig; +import org.apache.druid.server.coordinator.stats.CoordinatorRunStats; +import org.apache.druid.timeline.CompactionState; +import org.easymock.EasyMock; +import org.joda.time.DateTime; +import org.joda.time.Period; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.RegisterExtension; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertNull; + +public class KillUnreferencedCompactionStateTest +{ + @RegisterExtension + public static final TestDerbyConnector.DerbyConnectorRule5 DERBY_CONNECTOR_RULE = + new TestDerbyConnector.DerbyConnectorRule5(); + + private final ObjectMapper jsonMapper = new DefaultObjectMapper(); + + private TestDerbyConnector derbyConnector; + private MetadataStorageTablesConfig tablesConfig; + private CompactionStateManager compactionStateManager; + private DruidCoordinatorRuntimeParams mockParams; + + @BeforeEach + public void setUp() + { + derbyConnector = DERBY_CONNECTOR_RULE.getConnector(); + tablesConfig = DERBY_CONNECTOR_RULE.metadataTablesConfigSupplier().get(); + + derbyConnector.createCompactionStatesTable(); + derbyConnector.createSegmentTable(); + + compactionStateManager = new CompactionStateManager(tablesConfig, jsonMapper, derbyConnector, new CompactionStateManagerConfig()); + + mockParams = EasyMock.createMock(DruidCoordinatorRuntimeParams.class); + CoordinatorRunStats runStats = new CoordinatorRunStats(); + EasyMock.expect(mockParams.getCoordinatorStats()).andReturn(runStats).anyTimes(); + EasyMock.replay(mockParams); + } + + @Test + public void testKillUnreferencedCompactionState_lifecycle() + { + // Setup time progression: now, +1hr, +7hrs (past cleanup period and retention) + List dateTimes = new ArrayList<>(); + DateTime now = DateTimes.nowUtc(); + dateTimes.add(now); // Run 1: Mark as unused + dateTimes.add(now.plusMinutes(61)); // Run 2: Still in retention period + dateTimes.add(now.plusMinutes(6 * 60 + 1)); // Run 3: Past retention, delete + + MetadataCleanupConfig cleanupConfig = new MetadataCleanupConfig( + true, + Period.parse("PT1H").toStandardDuration(), // cleanup period + Period.parse("PT6H").toStandardDuration() // retention duration + ); + + KillUnreferencedCompactionState duty = + new TestKillUnreferencedCompactionState(cleanupConfig, compactionStateManager, dateTimes); + + // Insert a compaction state (initially marked as used) + String fingerprint = "test_fingerprint"; + CompactionState state = createTestCompactionState(); + + derbyConnector.retryWithHandle(handle -> { + Map map = new HashMap<>(); + map.put(fingerprint, state); + compactionStateManager.persistCompactionState("test-ds", map, DateTimes.nowUtc()); + return null; + }); + + assertEquals(Boolean.TRUE, getCompactionStateUsedStatus(fingerprint)); + + // Run 1: Should mark as unused (no segments reference it) + duty.run(mockParams); + assertEquals(Boolean.FALSE, getCompactionStateUsedStatus(fingerprint)); + + // Run 2: Still unused, but within retention period - should not delete + duty.run(mockParams); + assertNotNull(getCompactionStateUsedStatus(fingerprint)); + + // Run 3: Past retention period - should delete + duty.run(mockParams); + assertNull(getCompactionStateUsedStatus(fingerprint)); + } + + @Test + public void testKillUnreferencedCompactionState_repair() + { + List dateTimes = new ArrayList<>(); + DateTime now = DateTimes.nowUtc(); + dateTimes.add(now); + dateTimes.add(now.plusMinutes(61)); + + MetadataCleanupConfig cleanupConfig = new MetadataCleanupConfig( + true, + Period.parse("PT1H").toStandardDuration(), + Period.parse("PT6H").toStandardDuration() + ); + + KillUnreferencedCompactionState duty = + new TestKillUnreferencedCompactionState(cleanupConfig, compactionStateManager, dateTimes); + + // Insert compaction state + String fingerprint = "repair_fingerprint"; + CompactionState state = createTestCompactionState(); + + derbyConnector.retryWithHandle(handle -> { + Map map = new HashMap<>(); + map.put(fingerprint, state); + compactionStateManager.persistCompactionState("test-ds", map, DateTimes.nowUtc()); + return null; + }); + + // Run 1: Mark as unused + duty.run(mockParams); + assertEquals(Boolean.FALSE, getCompactionStateUsedStatus(fingerprint)); + + // Now insert a used segment that references this fingerprint + derbyConnector.retryWithHandle(handle -> { + handle.createStatement( + "INSERT INTO " + tablesConfig.getSegmentsTable() + " " + + "(id, dataSource, created_date, start, \"end\", partitioned, version, used, payload, " + + "used_status_last_updated, compaction_state_fingerprint) " + + "VALUES (:id, :dataSource, :created_date, :start, :end, :partitioned, :version, :used, :payload, " + + ":used_status_last_updated, :compaction_state_fingerprint)" + ) + .bind("id", "testSegment_2024-01-01_2024-01-02_v1_0") + .bind("dataSource", "test-ds") + .bind("created_date", DateTimes.nowUtc().toString()) + .bind("start", "2024-01-01T00:00:00.000Z") + .bind("end", "2024-01-02T00:00:00.000Z") + .bind("partitioned", 0) + .bind("version", "v1") + .bind("used", true) + .bind("payload", new byte[]{}) + .bind("used_status_last_updated", DateTimes.nowUtc().toString()) + .bind("compaction_state_fingerprint", fingerprint) + .execute(); + return null; + }); + + // Run 2: Repair - should mark it back as used + duty.run(mockParams); + assertEquals(Boolean.TRUE, getCompactionStateUsedStatus(fingerprint)); + } + + @Test + public void testKillUnreferencedCompactionState_disabled() + { + MetadataCleanupConfig cleanupConfig = new MetadataCleanupConfig( + false, // disabled + Period.parse("PT1H").toStandardDuration(), + Period.parse("PT6H").toStandardDuration() + ); + + KillUnreferencedCompactionState duty = + new KillUnreferencedCompactionState(cleanupConfig, compactionStateManager); + + // Insert compaction state + String fingerprint = "disabled_fingerprint"; + derbyConnector.retryWithHandle(handle -> { + Map map = new HashMap<>(); + map.put(fingerprint, createTestCompactionState()); + compactionStateManager.persistCompactionState("test-ds", map, DateTimes.nowUtc()); + return null; + }); + + // Run duty - should do nothing + duty.run(mockParams); + + // Should still be used (not marked as unused) + assertEquals(Boolean.TRUE, getCompactionStateUsedStatus(fingerprint)); + } + + private static class TestKillUnreferencedCompactionState extends KillUnreferencedCompactionState + { + private final List dateTimes; + private int index = -1; + + public TestKillUnreferencedCompactionState( + MetadataCleanupConfig config, + CompactionStateManager compactionStateManager, + List dateTimes + ) + { + super(config, compactionStateManager); + this.dateTimes = dateTimes; + } + + @Override + protected DateTime getCurrentTime() + { + index++; + return dateTimes.get(index); + } + } + + private CompactionState createTestCompactionState() + { + return new CompactionState( + new DynamicPartitionsSpec(100, null), + null, null, null, + IndexSpec.getDefault(), + null, null + ); + } + + private Boolean getCompactionStateUsedStatus(String fingerprint) + { + List usedStatus = derbyConnector.retryWithHandle( + handle -> handle.createQuery( + "SELECT used FROM " + tablesConfig.getCompactionStatesTable() + + " WHERE fingerprint = :fp" + ) + .bind("fp", fingerprint) + .mapTo(Boolean.class) + .list() + ); + + return usedStatus.isEmpty() ? null : usedStatus.get(0); + } +} 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 0838b41402d5..8f36cd51232d 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 @@ -42,6 +42,7 @@ import org.apache.druid.rpc.indexing.NoopOverlordClient; import org.apache.druid.rpc.indexing.SegmentUpdateResponse; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; +import org.apache.druid.segment.metadata.HeapMemoryCompactionStateManager; import org.apache.druid.server.compaction.CompactionStatusTracker; import org.apache.druid.server.coordinator.CloneStatusManager; import org.apache.druid.server.coordinator.CoordinatorConfigManager; @@ -225,7 +226,8 @@ public CoordinatorSimulation build() CentralizedDatasourceSchemaConfig.create(), new CompactionStatusTracker(), env.configSyncer, - env.cloneStatusManager + env.cloneStatusManager, + new HeapMemoryCompactionStateManager() ); return new SimulationImpl(coordinator, env); @@ -514,7 +516,8 @@ private Environment( ruleManager, null, null, - NoopSegmentMetadataCache.instance() + NoopSegmentMetadataCache.instance(), + new HeapMemoryCompactionStateManager() ); this.configSyncer = EasyMock.niceMock(CoordinatorDynamicConfigSyncer.class); diff --git a/server/src/test/java/org/apache/druid/server/http/DataSegmentPlusTest.java b/server/src/test/java/org/apache/druid/server/http/DataSegmentPlusTest.java index d23777312657..8e3ce33b71a0 100644 --- a/server/src/test/java/org/apache/druid/server/http/DataSegmentPlusTest.java +++ b/server/src/test/java/org/apache/druid/server/http/DataSegmentPlusTest.java @@ -75,6 +75,7 @@ public void testSerde() throws JsonProcessingException final Interval interval = Intervals.of("2011-10-01/2011-10-02"); final ImmutableMap loadSpec = ImmutableMap.of("something", "or_other"); + String compactionStateFingerprint = "abc123"; String createdDateStr = "2024-01-20T00:00:00.701Z"; String usedStatusLastUpdatedDateStr = "2024-01-20T01:00:00.701Z"; DateTime createdDate = DateTimes.of(createdDateStr); @@ -87,6 +88,7 @@ public void testSerde() throws JsonProcessingException loadSpec, Arrays.asList("dim1", "dim2"), Arrays.asList("met1", "met2"), + null, new NumberedShardSpec(3, 0), new CompactionState( new HashedPartitionsSpec(100000, null, ImmutableList.of("dim1")), @@ -100,14 +102,17 @@ public void testSerde() throws JsonProcessingException null ), TEST_VERSION, - 1 + 1, + compactionStateFingerprint, + DataSegment.PruneSpecsHolder.DEFAULT ), createdDate, usedStatusLastUpdatedDate, null, null, null, - null + null, + compactionStateFingerprint ); final Map objectMap = MAPPER.readValue( @@ -115,14 +120,14 @@ public void testSerde() throws JsonProcessingException JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT ); - Assert.assertEquals(7, objectMap.size()); + Assert.assertEquals(8, objectMap.size()); final Map segmentObjectMap = MAPPER.readValue( MAPPER.writeValueAsString(segmentPlus.getDataSegment()), JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT ); // verify dataSegment - Assert.assertEquals(11, segmentObjectMap.size()); + Assert.assertEquals(12, segmentObjectMap.size()); Assert.assertEquals("something", segmentObjectMap.get("dataSource")); Assert.assertEquals(interval.toString(), segmentObjectMap.get("interval")); Assert.assertEquals("1", segmentObjectMap.get("version")); @@ -133,6 +138,7 @@ public void testSerde() throws JsonProcessingException Assert.assertEquals(TEST_VERSION, segmentObjectMap.get("binaryVersion")); Assert.assertEquals(1, segmentObjectMap.get("size")); Assert.assertEquals(6, ((Map) segmentObjectMap.get("lastCompactionState")).size()); + Assert.assertEquals("abc123", segmentObjectMap.get("compactionStateFingerprint")); // verify extra metadata Assert.assertEquals(createdDateStr, objectMap.get("createdDate")); @@ -151,6 +157,7 @@ public void testSerde() throws JsonProcessingException Assert.assertEquals(segmentPlus.getDataSegment().getSize(), deserializedSegmentPlus.getDataSegment().getSize()); Assert.assertEquals(segmentPlus.getDataSegment().getId(), deserializedSegmentPlus.getDataSegment().getId()); Assert.assertEquals(segmentPlus.getDataSegment().getLastCompactionState(), deserializedSegmentPlus.getDataSegment().getLastCompactionState()); + Assert.assertEquals(segmentPlus.getDataSegment().getCompactionStateFingerprint(), deserializedSegmentPlus.getDataSegment().getCompactionStateFingerprint()); // verify extra metadata Assert.assertEquals(segmentPlus.getCreatedDate(), deserializedSegmentPlus.getCreatedDate()); diff --git a/server/src/test/java/org/apache/druid/server/http/MetadataResourceTest.java b/server/src/test/java/org/apache/druid/server/http/MetadataResourceTest.java index 4458fd44f6c8..16e315fc05d1 100644 --- a/server/src/test/java/org/apache/druid/server/http/MetadataResourceTest.java +++ b/server/src/test/java/org/apache/druid/server/http/MetadataResourceTest.java @@ -78,7 +78,7 @@ public class MetadataResourceTest .toArray(new DataSegment[0]); private final List segmentsPlus = Arrays.stream(segments) - .map(s -> new DataSegmentPlus(s, DateTimes.nowUtc(), DateTimes.nowUtc(), null, null, null, null)) + .map(s -> new DataSegmentPlus(s, DateTimes.nowUtc(), DateTimes.nowUtc(), null, null, null, null, null)) .collect(Collectors.toList()); private HttpServletRequest request; private SegmentsMetadataManager segmentsMetadataManager; diff --git a/services/src/main/java/org/apache/druid/guice/MetadataManagerModule.java b/services/src/main/java/org/apache/druid/guice/MetadataManagerModule.java index 6d501f9f768c..08b9af422f65 100644 --- a/services/src/main/java/org/apache/druid/guice/MetadataManagerModule.java +++ b/services/src/main/java/org/apache/druid/guice/MetadataManagerModule.java @@ -41,6 +41,8 @@ import org.apache.druid.metadata.segment.SqlSegmentMetadataTransactionFactory; import org.apache.druid.metadata.segment.cache.HeapMemorySegmentMetadataCache; import org.apache.druid.metadata.segment.cache.SegmentMetadataCache; +import org.apache.druid.segment.metadata.CompactionStateManager; +import org.apache.druid.segment.metadata.CompactionStateManagerConfig; import org.apache.druid.segment.metadata.NoopSegmentSchemaCache; import org.apache.druid.segment.metadata.SegmentSchemaCache; import org.apache.druid.server.coordinator.CoordinatorConfigManager; @@ -60,6 +62,7 @@ *
  • {@link CoordinatorConfigManager}
  • *
  • {@link SegmentMetadataCache}
  • *
  • {@link SegmentSchemaCache} - Coordinator only
  • + *
  • {@link CompactionStateManager}
  • * */ public class MetadataManagerModule implements Module @@ -102,6 +105,10 @@ public void configure(Binder binder) .to(HeapMemorySegmentMetadataCache.class) .in(LazySingleton.class); + JsonConfigProvider.bind(binder, "druid.manager.compactionState", CompactionStateManagerConfig.class); + binder.bind(CompactionStateManager.class) + .in(ManageLifecycle.class); + // Coordinator-only dependencies if (nodeRoles.contains(NodeRole.COORDINATOR)) { JsonConfigProvider.bind(binder, "druid.manager.rules", MetadataRuleManagerConfig.class); diff --git a/web-console/src/dialogs/compaction-dynamic-config-dialog/compaction-dynamic-config-completions.ts b/web-console/src/dialogs/compaction-dynamic-config-dialog/compaction-dynamic-config-completions.ts index 4dd7f7fc2650..4021b8e1bebc 100644 --- a/web-console/src/dialogs/compaction-dynamic-config-dialog/compaction-dynamic-config-completions.ts +++ b/web-console/src/dialogs/compaction-dynamic-config-dialog/compaction-dynamic-config-completions.ts @@ -44,6 +44,11 @@ export const COMPACTION_DYNAMIC_CONFIG_COMPLETIONS: JsonCompletionRule[] = [ value: 'engine', documentation: 'Engine used for running compaction tasks (native or msq)', }, + { + value: 'legacyPersistLastCompactionStateInSegments', + documentation: + 'Whether to persist the full compaction state in segment metadata (default: true)', + }, ], }, // compactionTaskSlotRatio values @@ -116,4 +121,20 @@ export const COMPACTION_DYNAMIC_CONFIG_COMPLETIONS: JsonCompletionRule[] = [ condition: obj => !obj.useSupervisors, completions: [{ value: 'native', documentation: 'Native indexing engine (default)' }], }, + // legacyPersistLastCompactionStateInSegments values + { + path: '$.legacyPersistLastCompactionStateInSegments', + completions: [ + { + value: 'true', + documentation: + 'Store full compaction state in segment metadata (legacy behavior, default)', + }, + { + value: 'false', + documentation: + 'Store only fingerprint reference in segment metadata (reduces storage overhead)', + }, + ], + }, ]; diff --git a/web-console/src/druid-models/compaction-dynamic-config/compaction-dynamic-config.tsx b/web-console/src/druid-models/compaction-dynamic-config/compaction-dynamic-config.tsx index 047f3f84f738..7f6d6ac30917 100644 --- a/web-console/src/druid-models/compaction-dynamic-config/compaction-dynamic-config.tsx +++ b/web-console/src/druid-models/compaction-dynamic-config/compaction-dynamic-config.tsx @@ -27,6 +27,7 @@ export interface CompactionDynamicConfig { compactionPolicy: { type: 'newestSegmentFirst'; priorityDatasource?: string | null }; useSupervisors: boolean; engine: 'native' | 'msq'; + legacyPersistLastCompactionStateInSegments: boolean; } export const COMPACTION_DYNAMIC_CONFIG_DEFAULT_RATIO = 0.1; @@ -94,4 +95,29 @@ export const COMPACTION_DYNAMIC_CONFIG_FIELDS: Field[] ), }, + { + name: 'legacyPersistLastCompactionStateInSegments', + label: 'Legacy: Persist last compaction state in segments', + type: 'boolean', + defaultValue: true, + info: ( + <> +

    + Whether to persist the full compaction state in segment metadata. When{' '} + true (default), compaction state is stored in both the segment metadata and + the compaction states table. +

    +

    + When false, only a fingerprint reference is stored in the segment metadata, + reducing storage overhead in the segments table. The actual compaction state is stored in + the compaction states table. +

    +

    + Note: Eventually this configuration will be removed and all compaction + will use the fingerprint method only. This configuration exists for operators to opt into + this future pattern early. +

    + + ), + }, ]; diff --git a/website/.spelling b/website/.spelling index a6e7cf573889..ce5bd93e78e9 100644 --- a/website/.spelling +++ b/website/.spelling @@ -483,6 +483,7 @@ pre-computation pre-compute pre-computed pre-computing +pre-dates preconfigured pre-existing pre-filtered