From d8490b0d1f85dd34a5c71d58719058a4e43b61c0 Mon Sep 17 00:00:00 2001 From: capistrant Date: Fri, 12 Dec 2025 19:46:02 -0600 Subject: [PATCH 01/72] meatadata store bits part 1 --- .../MetadataStorageUpdaterJobSpec.java | 1 + .../input/table/DataSegmentWithLocation.java | 3 + .../metadata/MetadataStorageConnector.java | 5 + .../metadata/MetadataStorageTablesConfig.java | 14 ++- .../apache/druid/timeline/DataSegment.java | 38 ++++++ .../TestMetadataStorageConnector.java | 6 + .../TestMetadataStorageTablesConfig.java | 1 + .../druid/timeline/DataSegmentTest.java | 114 ++++++++++++++++++ .../timeline/SegmentStatusInClusterTest.java | 4 +- .../test/resources/test.runtime.properties | 1 + .../IndexerSQLMetadataStorageCoordinator.java | 9 +- .../druid/metadata/SQLMetadataConnector.java | 56 ++++++++- .../SqlSegmentsMetadataManagerProvider.java | 1 + .../metadata/SqlSegmentsMetadataQuery.java | 11 +- .../SqlSegmentMetadataTransaction.java | 11 +- .../cache/HeapMemorySegmentMetadataCache.java | 11 +- .../coordination/LoadableDataSegment.java | 4 +- .../druid/server/http/DataSegmentPlus.java | 21 +++- ...SqlMetadataStorageCoordinatorTestBase.java | 3 +- .../metadata/SQLMetadataConnectorTest.java | 31 ++++- .../HeapMemoryDatasourceSegmentCacheTest.java | 4 +- .../HeapMemorySegmentMetadataCacheTest.java | 2 + .../coordinator/CreateDataSegments.java | 17 ++- .../server/http/DataSegmentPlusTest.java | 6 +- .../server/http/MetadataResourceTest.java | 2 +- 25 files changed, 338 insertions(+), 38 deletions(-) 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/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..77816128940b 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("compactionStatesTable") 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/DataSegment.java b/processing/src/main/java/org/apache/druid/timeline/DataSegment.java index ef13b15f773b..5a661983cb32 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/DataSegmentTest.java b/processing/src/test/java/org/apache/druid/timeline/DataSegmentTest.java index 54caacdf48b5..e46f69b38314 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,120 @@ 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"; + + DataSegment segment = DataSegment.builder() + .dataSource("something") + .interval(interval) + .version("1") + .loadSpec(loadSpec) + .dimensions(Arrays.asList("dim1", "dim2")) + .metrics(Arrays.asList("met1", "met2")) + .shardSpec(new NumberedShardSpec(3, 0)) + .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"); + + DataSegment segment = DataSegment.builder() + .dataSource("something") + .interval(interval) + .version("1") + .loadSpec(loadSpec) + .dimensions(Arrays.asList("dim1", "dim2")) + .metrics(Arrays.asList("met1", "met2")) + .shardSpec(new NumberedShardSpec(3, 0)) + .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 DataSegment segment1 = DataSegment.builder() + .dataSource("foo") + .interval(Intervals.of("2012-01-01/2012-01-02")) + .version(DateTimes.of("2012-01-01T11:22:33.444Z").toString()) + .shardSpec(getShardSpec(7)) + .size(0) + .compactionStateFingerprint(fingerprint) + .build(); + final DataSegment segment2 = DataSegment.builder() + .dataSource("foo") + .interval(Intervals.of("2012-01-01/2012-01-02")) + .version(DateTimes.of("2012-01-01T11:22:33.444Z").toString()) + .shardSpec(getShardSpec(7)) + .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..9811521eb45e 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() // TODO this might look different eventually ); }).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/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/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..005fd5723912 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/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/http/DataSegmentPlusTest.java b/server/src/test/java/org/apache/druid/server/http/DataSegmentPlusTest.java index d23777312657..9d05b8832c47 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); @@ -107,7 +108,8 @@ public void testSerde() throws JsonProcessingException null, null, null, - null + null, + compactionStateFingerprint ); final Map objectMap = MAPPER.readValue( @@ -115,7 +117,7 @@ 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 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; From 3d2d4231ee71bbce3e898f3986d765f6fee432da Mon Sep 17 00:00:00 2001 From: capistrant Date: Fri, 12 Dec 2025 20:34:58 -0600 Subject: [PATCH 02/72] annotate segments with compaction fingerprint before persist --- .../common/task/AbstractBatchIndexTask.java | 15 +++++++++++++ .../druid/indexing/common/task/IndexTask.java | 8 +++++++ .../druid/indexing/common/task/Tasks.java | 9 ++++++++ .../parallel/ParallelIndexSupervisorTask.java | 16 ++++++++++---- .../apache/druid/msq/exec/ControllerImpl.java | 22 +++++++++++++++++++ .../apache/druid/timeline/DataSegment.java | 2 +- .../coordinator/duty/CompactSegments.java | 7 +++++- .../HeapMemorySegmentMetadataCacheTest.java | 2 +- 8 files changed, 74 insertions(+), 7 deletions(-) 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/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/processing/src/main/java/org/apache/druid/timeline/DataSegment.java b/processing/src/main/java/org/apache/druid/timeline/DataSegment.java index 5a661983cb32..19f3a817f6e0 100644 --- a/processing/src/main/java/org/apache/druid/timeline/DataSegment.java +++ b/processing/src/main/java/org/apache/druid/timeline/DataSegment.java @@ -258,7 +258,7 @@ public DataSegment( this.size = size; this.compactionStateFingerprint = compactionStateFingerprint != null ? STRING_INTERNER.intern(compactionStateFingerprint) - : null;; + : null; } /** 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..ad2323dd0dd6 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java @@ -76,7 +76,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); 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 005fd5723912..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,7 +89,7 @@ public void setup() derbyConnector.createSegmentTable(); derbyConnector.createSegmentSchemasTable(); derbyConnector.createPendingSegmentsTable(); - derbyConnector.createCompactionStatesTable();; + derbyConnector.createCompactionStatesTable(); schemaTestUtils = new SegmentSchemaTestUtils(derbyConnectorRule, derbyConnector, TestHelper.JSON_MAPPER); EmittingLogger.registerEmitter(serviceEmitter); From 48854f472dba856212038f3ac6cc48a380e6862a Mon Sep 17 00:00:00 2001 From: capistrant Date: Fri, 12 Dec 2025 20:59:03 -0600 Subject: [PATCH 03/72] Add ability to generate compaction state fingerprint --- .../druid/timeline/CompactionState.java | 64 +++++++ .../druid/timeline/CompactionStateTest.java | 157 ++++++++++++++++++ 2 files changed, 221 insertions(+) create mode 100644 processing/src/test/java/org/apache/druid/timeline/CompactionStateTest.java 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/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..a99eeec10850 --- /dev/null +++ b/processing/src/test/java/org/apache/druid/timeline/CompactionStateTest.java @@ -0,0 +1,157 @@ +/* + * 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 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_listOrderDoesNotInfluenceFingerprint() + { + 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 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 + ); + } +} From c6a336784750bb2ae12a0eb86125420204064704 Mon Sep 17 00:00:00 2001 From: capistrant Date: Fri, 12 Dec 2025 21:39:32 -0600 Subject: [PATCH 04/72] add fingerprint to task context and make legacy last compaction state storage configurable --- .../embedded/compact/AutoCompactionTest.java | 11 +-- .../compact/CompactionSupervisorTest.java | 2 +- .../indexing/KafkaClusterMetricsTest.java | 4 +- .../compact/CatalogCompactionTest.java | 2 +- .../CompactionConfigBasedJobTemplate.java | 15 +++- .../OverlordCompactionSchedulerTest.java | 10 +-- .../http/OverlordCompactionResourceTest.java | 4 +- .../compaction/CompactionRunSimulator.java | 3 +- .../server/compaction/CompactionStatus.java | 2 +- .../coordinator/ClusterCompactionConfig.java | 28 +++++- .../coordinator/CoordinatorConfigManager.java | 3 +- .../coordinator/DruidCompactionConfig.java | 14 ++- .../coordinator/duty/CompactSegments.java | 89 ++++++++++++++++++- .../rpc/indexing/OverlordClientImplTest.java | 5 +- .../CompactionRunSimulatorTest.java | 2 +- ...aSourceCompactionConfigAuditEntryTest.java | 12 +-- ...DataSourceCompactionConfigHistoryTest.java | 2 +- .../DruidCompactionConfigTest.java | 5 +- .../coordinator/DruidCoordinatorTest.java | 2 +- .../coordinator/duty/CompactSegmentsTest.java | 1 + 20 files changed, 173 insertions(+), 43 deletions(-) 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..e4f5edd42b4d 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 @@ -111,7 +111,7 @@ 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()); } 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-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..49bf8bd4c5c4 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionConfigBasedJobTemplate.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionConfigBasedJobTemplate.java @@ -31,6 +31,7 @@ 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; @@ -70,12 +71,22 @@ public List createCompactionJobs( final List jobs = new ArrayList<>(); + String compactionStateFingerprint = CompactionState.generateCompactionStateFingerprint( + CompactSegments.createCompactionStateFromConfig(config), + config.getDataSource() + ); + // 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, 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..d67a01d55913 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 @@ -173,7 +173,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() @@ -444,7 +444,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 +469,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 +536,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/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..80abb0a85112 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 @@ -138,7 +138,8 @@ 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(); 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..bec9dcfe4773 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 @@ -256,7 +256,7 @@ static CompactionStatus compute( } @Nullable - static PartitionsSpec findPartitionsSpecFromConfig(ClientCompactionTaskQueryTuningConfig tuningConfig) + public static PartitionsSpec findPartitionsSpecFromConfig(ClientCompactionTaskQueryTuningConfig tuningConfig) { final PartitionsSpec partitionsSpecFromTuningConfig = tuningConfig.getPartitionsSpec(); if (partitionsSpecFromTuningConfig == null) { 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/duty/CompactSegments.java b/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java index ad2323dd0dd6..a581e250620f 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,17 @@ 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.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.transform.CompactionTransformSpec; import org.apache.druid.server.compaction.CompactionCandidate; import org.apache.druid.server.compaction.CompactionCandidateSearchPolicy; @@ -53,14 +58,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; @@ -192,7 +200,8 @@ public void run( slotManager, iterator, policy, - defaultEngine + defaultEngine, + dynamicConfig.clusterConfig().isLegacyPersistLastCompactionStateInSegments() ); stats.add(Stats.Compaction.SUBMITTED_TASKS, numSubmittedCompactionTasks); @@ -228,7 +237,8 @@ private int submitCompactionTasks( CompactionSlotManager slotManager, CompactionSegmentIterator iterator, CompactionCandidateSearchPolicy policy, - CompactionEngine defaultEngine + CompactionEngine defaultEngine, + boolean persistLastCompactionStateInSegments ) { if (slotManager.getNumAvailableTaskSlots() <= 0) { @@ -259,7 +269,18 @@ private int submitCompactionTasks( snapshotBuilder.addToComplete(entry); } - final ClientCompactionTaskQuery taskPayload = createCompactionTask(entry, config, defaultEngine); + String compactionStateFingerprint = CompactionState.generateCompactionStateFingerprint( + createCompactionStateFromConfig(config), + config.getDataSource() + ); + + final ClientCompactionTaskQuery taskPayload = createCompactionTask( + entry, + config, + defaultEngine, + compactionStateFingerprint, + persistLastCompactionStateInSegments + ); final String taskId = taskPayload.getId(); FutureUtils.getUnchecked(overlordClient.runTask(taskId, taskPayload), true); @@ -285,7 +306,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(); @@ -363,6 +386,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(), @@ -466,4 +492,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/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/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/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/DruidCoordinatorTest.java b/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorTest.java index 39213ff8616d..f47cb9fd9f8e 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 @@ -829,7 +829,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..cd92e8f1999a 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 @@ -1657,6 +1657,7 @@ private CoordinatorRunStats doCompactSegments( numCompactionTaskSlots, policy, null, + null, null ) ) From f3b706e4b9904c9b4b81f1d421f70a86fbc5475d Mon Sep 17 00:00:00 2001 From: capistrant Date: Fri, 12 Dec 2025 22:29:45 -0600 Subject: [PATCH 05/72] update embedded tests for compaction supervisors to flex fingerprints --- .../compact/CompactionSupervisorTest.java | 167 +++++++++++++++++- .../metadata/MetadataStorageTablesConfig.java | 2 +- 2 files changed, 165 insertions(+), 4 deletions(-) diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionSupervisorTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionSupervisorTest.java index e4f5edd42b4d..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; @@ -118,7 +120,7 @@ private void configureCompaction(CompactionEngine compactionEngine) @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/processing/src/main/java/org/apache/druid/metadata/MetadataStorageTablesConfig.java b/processing/src/main/java/org/apache/druid/metadata/MetadataStorageTablesConfig.java index 77816128940b..0303da10a610 100644 --- a/processing/src/main/java/org/apache/druid/metadata/MetadataStorageTablesConfig.java +++ b/processing/src/main/java/org/apache/druid/metadata/MetadataStorageTablesConfig.java @@ -94,7 +94,7 @@ public MetadataStorageTablesConfig( @JsonProperty("upgradeSegments") String upgradeSegmentsTable, @JsonProperty("segmentSchemas") String segmentSchemasTable, @JsonProperty("useShortIndexNames") Boolean useShortIndexNames, - @JsonProperty("compactionStatesTable") String compactionStatesTable + @JsonProperty("compactionStates") String compactionStatesTable ) { this.base = (base == null) ? DEFAULT_BASE : base; From 46fb807d24e019625b083ab88d1aee78b587516e Mon Sep 17 00:00:00 2001 From: capistrant Date: Fri, 12 Dec 2025 23:51:57 -0600 Subject: [PATCH 06/72] checkpoint with persisting compaction states --- .../CompactionConfigBasedJobTemplate.java | 14 +- .../indexing/compact/CompactionJobParams.java | 11 +- .../indexing/compact/CompactionJobQueue.java | 7 +- .../compact/OverlordCompactionScheduler.java | 10 +- .../OverlordCompactionSchedulerTest.java | 3 +- .../metadata/CompactionStateManager.java | 518 ++++++++++++++++++ .../compaction/CompactionRunSimulator.java | 3 +- .../server/coordinator/DruidCoordinator.java | 8 +- .../coordinator/duty/CompactSegments.java | 19 +- .../coordinator/DruidCoordinatorTest.java | 20 +- .../coordinator/duty/CompactSegmentsTest.java | 70 ++- .../CoordinatorSimulationBuilder.java | 3 +- .../druid/guice/MetadataManagerModule.java | 4 + 13 files changed, 642 insertions(+), 48 deletions(-) create mode 100644 server/src/main/java/org/apache/druid/segment/metadata/CompactionStateManager.java diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionConfigBasedJobTemplate.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionConfigBasedJobTemplate.java index 49bf8bd4c5c4..0db9ce30716e 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; @@ -38,6 +39,7 @@ import javax.annotation.Nullable; import java.util.ArrayList; import java.util.List; +import java.util.Map; import java.util.Objects; /** @@ -71,11 +73,21 @@ public List createCompactionJobs( final List jobs = new ArrayList<>(); + CompactionState compactionState = CompactSegments.createCompactionStateFromConfig(config); + String compactionStateFingerprint = CompactionState.generateCompactionStateFingerprint( - CompactSegments.createCompactionStateFromConfig(config), + compactionState, config.getDataSource() ); + if (segmentIterator.hasNext()) { + 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(); 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 d67a01d55913..e715890f4fcc 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 @@ -231,7 +231,8 @@ private void initScheduler() (nameFormat, numThreads) -> new WrappingScheduledExecutorService("test", executor, false), brokerClient, serviceEmitter, - OBJECT_MAPPER + OBJECT_MAPPER, + null // TODO is there any legit testing needed here for compaction state manager ); } 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..85f1a640418f --- /dev/null +++ b/server/src/main/java/org/apache/druid/segment/metadata/CompactionStateManager.java @@ -0,0 +1,518 @@ +/* + * 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.collect.Lists; +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; + +/** + * Handles compaction state persistence on the Coordinator. + */ +@ManageLifecycle +public class CompactionStateManager +{ + private static final EmittingLogger log = new EmittingLogger(CompactionStateManager.class); + private static final int DB_ACTION_PARTITION_SIZE = 100; + private static final int DEFAULT_CACHE_SIZE = 100; + private static final int DEFAULT_PREWARM_SIZE = 100; + + private final MetadataStorageTablesConfig dbTables; + private final ObjectMapper jsonMapper; + private final SQLMetadataConnector connector; + private final Cache fingerprintCache; + + @Inject + public CompactionStateManager( + @Nonnull MetadataStorageTablesConfig dbTables, + @Nonnull ObjectMapper jsonMapper, + @Nonnull SQLMetadataConnector connector + ) + { + this.dbTables = dbTables; + this.jsonMapper = jsonMapper; + this.connector = connector; + + this.fingerprintCache = CacheBuilder.newBuilder() + .maximumSize(DEFAULT_CACHE_SIZE) + .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(DEFAULT_PREWARM_SIZE); + } 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 + protected CompactionStateManager() + { + this.dbTables = null; + this.jsonMapper = null; + this.connector = null; + this.fingerprintCache = null; + } + + /** + * Persist unique compaction state fingerprints in the DB. + */ + public void persistCompactionState( + final String dataSource, + final Map fingerprintToStateMap, + final DateTime updateTime + ) + { + if (fingerprintToStateMap.isEmpty()) { + return; + } + + 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; + }); + } + + /** + * 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) + { + // Check cache first + CompactionState cached = fingerprintCache.getIfPresent(fingerprint); + if (cached != null) { + return cached; + } + + // Cache miss - load from database + CompactionState fromDb = loadCompactionStateFromDatabase(fingerprint); + if (fromDb != null) { + fingerprintCache.put(fingerprint, fromDb); + } + + return fromDb; + } + + /** + * 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/server/compaction/CompactionRunSimulator.java b/server/src/main/java/org/apache/druid/server/compaction/CompactionRunSimulator.java index 80abb0a85112..1b8493e7af01 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 @@ -143,7 +143,8 @@ public void onTaskSubmitted(String taskId, CompactionCandidate candidateSegments ); final CoordinatorRunStats stats = new CoordinatorRunStats(); - new CompactSegments(simulationStatusTracker, readOnlyOverlordClient).run( + // TODO probably need something not null here + new CompactSegments(simulationStatusTracker, readOnlyOverlordClient, null).run( compactionConfig.withClusterConfig(configWithUnlimitedTaskSlots), dataSourcesSnapshot, defaultEngine, 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..62cf8e96a4a0 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; @@ -144,6 +145,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 +192,8 @@ public DruidCoordinator( CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig, CompactionStatusTracker compactionStatusTracker, CoordinatorDynamicConfigSyncer coordinatorDynamicConfigSyncer, - CloneStatusManager cloneStatusManager + CloneStatusManager cloneStatusManager, + CompactionStateManager compactionStateManager ) { this.config = config; @@ -216,6 +219,7 @@ public DruidCoordinator( this.cloneStatusManager = cloneStatusManager; this.compactSegments = initializeCompactSegmentsDuty(this.compactionStatusTracker); + this.compactionStateManager = compactionStateManager; } public boolean isLeader() @@ -619,7 +623,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/duty/CompactSegments.java b/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java index a581e250620f..0fe0dd4dec4b 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 @@ -39,12 +39,14 @@ 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; @@ -103,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(); } @@ -269,11 +275,20 @@ private int submitCompactionTasks( snapshotBuilder.addToComplete(entry); } + CompactionState compactionState = + createCompactionStateFromConfig(config); + String compactionStateFingerprint = CompactionState.generateCompactionStateFingerprint( - createCompactionStateFromConfig(config), + compactionState, config.getDataSource() ); + compactionStateManager.persistCompactionState( + config.getDataSource(), + Map.of(compactionStateFingerprint, compactionState), + DateTimes.nowUtc() + ); + final ClientCompactionTaskQuery taskPayload = createCompactionTask( entry, config, 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 f47cb9fd9f8e..429c2c26819b 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 ); } @@ -482,7 +486,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 +539,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 +571,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 +592,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 +703,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(); 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 cd92e8f1999a..fb1168d13746 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,7 @@ 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.transform.CompactionTransformSpec; import org.apache.druid.server.compaction.CompactionCandidate; import org.apache.druid.server.compaction.CompactionCandidateSearchPolicy; @@ -109,6 +110,7 @@ import org.apache.druid.timeline.partition.ShardSpec; import org.apache.druid.timeline.partition.SingleDimensionShardSpec; import org.apache.druid.utils.Streams; +import org.joda.time.DateTime; import org.joda.time.Interval; import org.joda.time.Period; import org.junit.Assert; @@ -197,6 +199,16 @@ public static Collection constructorFeeder() private DataSourcesSnapshot dataSources; private CompactionStatusTracker statusTracker; private final Map> datasourceToSegments = new HashMap<>(); + private final CompactionStateManager compactionStateManager = new CompactionStateManager() { + @Override + public void persistCompactionState( + final String dataSource, + final Map fingerprintToStateMap, + final DateTime updateTime + ) + { + } + }; public CompactSegmentsTest( PartitionsSpec partitionsSpec, @@ -277,7 +289,7 @@ public void testSerde() throws Exception .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 +301,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 +384,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 +508,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 +570,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 +662,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 +721,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 +741,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 +761,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 +783,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 +814,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 +838,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 +861,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 +898,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 +929,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 +952,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 +1027,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 +1053,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 +1144,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 +1183,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 +1229,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 +1266,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 +1285,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 +1314,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 +1339,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 +1395,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 +1455,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 +1488,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 +1515,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( 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..95ea38dee135 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 @@ -225,7 +225,8 @@ public CoordinatorSimulation build() CentralizedDatasourceSchemaConfig.create(), new CompactionStatusTracker(), env.configSyncer, - env.cloneStatusManager + env.cloneStatusManager, + null // TODO this probably needs some type of impl ); return new SimulationImpl(coordinator, env); 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..7dc95d7ff0bc 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,7 @@ 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.NoopSegmentSchemaCache; import org.apache.druid.segment.metadata.SegmentSchemaCache; import org.apache.druid.server.coordinator.CoordinatorConfigManager; @@ -60,6 +61,7 @@ *
  • {@link CoordinatorConfigManager}
  • *
  • {@link SegmentMetadataCache}
  • *
  • {@link SegmentSchemaCache} - Coordinator only
  • + *
  • {@link CompactionStateManager}
  • * */ public class MetadataManagerModule implements Module @@ -101,6 +103,8 @@ public void configure(Binder binder) binder.bind(SegmentMetadataCache.class) .to(HeapMemorySegmentMetadataCache.class) .in(LazySingleton.class); + binder.bind(CompactionStateManager.class) + .in(ManageLifecycle.class); // Coordinator-only dependencies if (nodeRoles.contains(NodeRole.COORDINATOR)) { From 0fef3581e6d90ef0f703469ef4a3faf3d633fb24 Mon Sep 17 00:00:00 2001 From: capistrant Date: Sun, 14 Dec 2025 15:52:21 -0600 Subject: [PATCH 07/72] add duty to clean up unused compaction states --- .../server/coordinator/DruidCoordinator.java | 2 + .../server/coordinator/MetadataManager.java | 11 +- .../config/CoordinatorKillConfigs.java | 12 +- .../duty/KillUnreferencedCompactionState.java | 61 ++++ .../druid/server/coordinator/stats/Stats.java | 2 + .../DruidCoordinatorConfigTest.java | 34 +++ .../coordinator/DruidCoordinatorTest.java | 3 +- .../KillUnreferencedCompactionStateTest.java | 261 ++++++++++++++++++ .../CoordinatorSimulationBuilder.java | 3 +- 9 files changed, 385 insertions(+), 4 deletions(-) create mode 100644 server/src/main/java/org/apache/druid/server/coordinator/duty/KillUnreferencedCompactionState.java create mode 100644 server/src/test/java/org/apache/druid/server/coordinator/duty/KillUnreferencedCompactionStateTest.java 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 62cf8e96a4a0..7a2434b5141b 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 @@ -77,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; @@ -613,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())); } 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..46f1af29e66b 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("compactionStates") + 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("segmentSchema") 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/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/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 429c2c26819b..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 @@ -188,7 +188,8 @@ private MetadataManager createMetadataManager(JacksonConfigManager configManager metadataRuleManager, null, null, - NoopSegmentMetadataCache.instance() + NoopSegmentMetadataCache.instance(), + compactionStateManager ); } 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..cdc2c45a6da3 --- /dev/null +++ b/server/src/test/java/org/apache/druid/server/coordinator/duty/KillUnreferencedCompactionStateTest.java @@ -0,0 +1,261 @@ +/* + * 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.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); + + 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 95ea38dee135..19650b6e83fe 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 @@ -515,7 +515,8 @@ private Environment( ruleManager, null, null, - NoopSegmentMetadataCache.instance() + NoopSegmentMetadataCache.instance(), + null // TODO does this need impl ); this.configSyncer = EasyMock.niceMock(CoordinatorDynamicConfigSyncer.class); From edeaf304f0dbc43b387b00cfeaaa1142246a7b00 Mon Sep 17 00:00:00 2001 From: capistrant Date: Sun, 14 Dec 2025 16:46:12 -0600 Subject: [PATCH 08/72] take fingerprints into account in CompactionStatus --- .../NewestSegmentFirstPolicyBenchmark.java | 3 +- .../CompactionConfigBasedJobTemplate.java | 3 +- .../server/compaction/CompactionStatus.java | 139 ++++++++++++++++-- .../DataSourceCompactibleSegmentIterator.java | 8 +- ...riorityBasedCompactionSegmentIterator.java | 7 +- .../coordinator/duty/CompactSegments.java | 5 +- .../compaction/CompactionStatusTest.java | 41 ++++-- .../NewestSegmentFirstPolicyTest.java | 29 ++-- 8 files changed, 193 insertions(+), 42 deletions(-) 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..ed0424127c8f 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 @@ -135,7 +135,8 @@ public void measureNewestSegmentFirstPolicy(Blackhole blackhole) policy, compactionConfigs, dataSources, - Collections.emptyMap() + Collections.emptyMap(), + null // TODO does this need impl ); for (int i = 0; i < numCompactionTaskSlots && iterator.hasNext(); i++) { blackhole.consume(iterator.next()); 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 0db9ce30716e..37f44e97b4ee 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 @@ -143,7 +143,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/server/src/main/java/org/apache/druid/server/compaction/CompactionStatus.java b/server/src/main/java/org/apache/druid/server/compaction/CompactionStatus.java index bec9dcfe4773..695943238817 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,10 +260,13 @@ 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 @@ -330,18 +344,27 @@ private static class Evaluator private final ClientCompactionTaskQueryTuningConfig tuningConfig; private final UserCompactionTaskGranularityConfig configuredGranularitySpec; + private final List fingerprintedSegments = 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,12 +374,29 @@ 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 @@ -376,15 +416,92 @@ private CompactionStatus evaluate() } } + /** + * 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); + } + } + + 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/coordinator/duty/CompactSegments.java b/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java index 0fe0dd4dec4b..7807137d4fff 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 @@ -196,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( 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..6f970cbb4ab3 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 @@ -37,6 +37,7 @@ 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; @@ -46,7 +47,9 @@ 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; @@ -59,6 +62,14 @@ public class CompactionStatusTest .size(100_000_000L) .build(); + private CompactionStateManager compactionStateManager; + + @Before + public void setUp() + { + compactionStateManager = EasyMock.createMock(CompactionStateManager.class); + } + @Test public void testFindPartitionsSpecWhenGivenIsNull() { @@ -326,8 +337,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 +387,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 +442,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 +496,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,8 +550,9 @@ 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()); } @@ -552,8 +568,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()); 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..1a24c8705654 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,7 @@ 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.transform.CompactionTransformSpec; import org.apache.druid.server.coordinator.CreateDataSegments; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; @@ -81,6 +82,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 = null; // TODO does this need impl? @Test public void testLargeOffsetAndSmallSegmentInterval() @@ -276,8 +278,9 @@ public void testSkipDataSourceWithNoSegments() .withNumPartitions(4) ) ), - Collections.emptyMap() - ); + Collections.emptyMap(), + compactionStateManager + ); assertCompactSegmentIntervals( iterator, @@ -508,8 +511,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 +551,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 +1407,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 +2057,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 +2079,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( From 97daf3f9da6e1d4bb9c14eb28ee02835cbedf9a7 Mon Sep 17 00:00:00 2001 From: capistrant Date: Mon, 15 Dec 2025 12:35:54 -0600 Subject: [PATCH 09/72] Add and improve tests --- .../OverlordCompactionSchedulerTest.java | 3 +- .../HeapMemoryCompactionStateManager.java | 81 ++++++ .../compaction/CompactionRunSimulator.java | 7 +- .../config/CoordinatorKillConfigs.java | 4 +- .../config/DruidCoordinatorConfig.java | 1 + .../compaction/CompactionStatusTest.java | 272 ++++++++++++++++++ .../NewestSegmentFirstPolicyTest.java | 3 +- .../coordinator/duty/CompactSegmentsTest.java | 12 +- .../server/http/DataSegmentPlusTest.java | 9 +- 9 files changed, 374 insertions(+), 18 deletions(-) create mode 100644 server/src/main/java/org/apache/druid/segment/metadata/HeapMemoryCompactionStateManager.java 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 e715890f4fcc..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; @@ -232,7 +233,7 @@ private void initScheduler() brokerClient, serviceEmitter, OBJECT_MAPPER, - null // TODO is there any legit testing needed here for compaction state manager + new HeapMemoryCompactionStateManager() ); } 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 1b8493e7af01..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; @@ -143,8 +145,9 @@ public void onTaskSubmitted(String taskId, CompactionCandidate candidateSegments ); final CoordinatorRunStats stats = new CoordinatorRunStats(); - // TODO probably need something not null here - new CompactSegments(simulationStatusTracker, readOnlyOverlordClient, null).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/coordinator/config/CoordinatorKillConfigs.java b/server/src/main/java/org/apache/druid/server/coordinator/config/CoordinatorKillConfigs.java index 46f1af29e66b..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 @@ -74,7 +74,7 @@ public class CoordinatorKillConfigs @JsonProperty("maxInterval") private final Period killUnusedMaxInterval; - @JsonProperty("compactionStates") + @JsonProperty("compactionState") private final MetadataCleanupConfig compactionStates; @JsonCreator @@ -86,7 +86,7 @@ public CoordinatorKillConfigs( @JsonProperty("rule") MetadataCleanupConfig rules, @JsonProperty("compaction") MetadataCleanupConfig compactionConfigs, @JsonProperty("segmentSchema") MetadataCleanupConfig segmentSchemas, - @JsonProperty("segmentSchema") MetadataCleanupConfig compactionStates, + @JsonProperty("compactionState") MetadataCleanupConfig compactionStates, // Configs for cleanup of unused segments @JsonProperty("on") Boolean killUnusedEnabled, @JsonProperty("period") Duration killUnusedPeriod, 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/test/java/org/apache/druid/server/compaction/CompactionStatusTest.java b/server/src/test/java/org/apache/druid/server/compaction/CompactionStatusTest.java index 6f970cbb4ab3..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,6 +32,7 @@ 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; @@ -44,6 +45,7 @@ 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; @@ -61,6 +63,10 @@ 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; @@ -556,6 +562,256 @@ public void testStatusWhenAutoSchemaMismatch() 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( CompactionState lastCompactionState, DataSourceCompactionConfig compactionConfig, @@ -597,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 1a24c8705654..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 @@ -48,6 +48,7 @@ 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; @@ -82,7 +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 = null; // TODO does this need impl? + private final CompactionStateManager compactionStateManager = new HeapMemoryCompactionStateManager(); @Test public void testLargeOffsetAndSmallSegmentInterval() 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 fb1168d13746..850ccb4f037a 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/duty/CompactSegmentsTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/duty/CompactSegmentsTest.java @@ -80,6 +80,7 @@ import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.segment.indexing.BatchIOConfig; import org.apache.druid.segment.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; @@ -199,16 +200,7 @@ public static Collection constructorFeeder() private DataSourcesSnapshot dataSources; private CompactionStatusTracker statusTracker; private final Map> datasourceToSegments = new HashMap<>(); - private final CompactionStateManager compactionStateManager = new CompactionStateManager() { - @Override - public void persistCompactionState( - final String dataSource, - final Map fingerprintToStateMap, - final DateTime updateTime - ) - { - } - }; + private final CompactionStateManager compactionStateManager = new HeapMemoryCompactionStateManager(); public CompactSegmentsTest( PartitionsSpec partitionsSpec, 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 9d05b8832c47..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 @@ -88,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")), @@ -101,7 +102,9 @@ public void testSerde() throws JsonProcessingException null ), TEST_VERSION, - 1 + 1, + compactionStateFingerprint, + DataSegment.PruneSpecsHolder.DEFAULT ), createdDate, usedStatusLastUpdatedDate, @@ -124,7 +127,7 @@ public void testSerde() throws JsonProcessingException ); // 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")); @@ -135,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")); @@ -153,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()); From dbcdfcf7d2c6c2f9aa2288900963f9d05328c4da Mon Sep 17 00:00:00 2001 From: capistrant Date: Mon, 15 Dec 2025 12:39:16 -0600 Subject: [PATCH 10/72] get rid of some todo comments --- .../coordinator/simulate/CoordinatorSimulationBuilder.java | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) 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 19650b6e83fe..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; @@ -226,7 +227,7 @@ public CoordinatorSimulation build() new CompactionStatusTracker(), env.configSyncer, env.cloneStatusManager, - null // TODO this probably needs some type of impl + new HeapMemoryCompactionStateManager() ); return new SimulationImpl(coordinator, env); @@ -516,7 +517,7 @@ private Environment( null, null, NoopSegmentMetadataCache.instance(), - null // TODO does this need impl + new HeapMemoryCompactionStateManager() ); this.configSyncer = EasyMock.niceMock(CoordinatorDynamicConfigSyncer.class); From 38f6d1501d50c35538a65a331bda27957f252fef Mon Sep 17 00:00:00 2001 From: capistrant Date: Mon, 15 Dec 2025 12:43:23 -0600 Subject: [PATCH 11/72] fix checkstyle --- .../druid/server/coordinator/duty/CompactSegmentsTest.java | 1 - 1 file changed, 1 deletion(-) 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 850ccb4f037a..6c15f46a5231 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 @@ -111,7 +111,6 @@ import org.apache.druid.timeline.partition.ShardSpec; import org.apache.druid.timeline.partition.SingleDimensionShardSpec; import org.apache.druid.utils.Streams; -import org.joda.time.DateTime; import org.joda.time.Interval; import org.joda.time.Period; import org.junit.Assert; From 4cf119774c8e264578a5f19e0c21196121b6ffe3 Mon Sep 17 00:00:00 2001 From: capistrant Date: Mon, 15 Dec 2025 12:47:31 -0600 Subject: [PATCH 12/72] cleanup some more TODO --- .../server/coordinator/NewestSegmentFirstPolicyBenchmark.java | 3 ++- .../druid/metadata/IndexerSQLMetadataStorageCoordinator.java | 2 +- 2 files changed, 3 insertions(+), 2 deletions(-) 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 ed0424127c8f..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; @@ -136,7 +137,7 @@ public void measureNewestSegmentFirstPolicy(Blackhole blackhole) compactionConfigs, dataSources, Collections.emptyMap(), - null // TODO does this need impl + new HeapMemoryCompactionStateManager() ); for (int i = 0; i < numCompactionTaskSlots && iterator.hasNext(); i++) { blackhole.consume(iterator.next()); 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 9811521eb45e..14cd95fd7ed7 100644 --- a/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java +++ b/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java @@ -2024,7 +2024,7 @@ private Set insertSegments( segmentMetadata == null ? null : segmentMetadata.getSchemaFingerprint(), segmentMetadata == null ? null : segmentMetadata.getNumRows(), upgradedFromSegmentIdMap.get(segment.getId().toString()), - segment.getCompactionStateFingerprint() // TODO this might look different eventually + segment.getCompactionStateFingerprint() ); }).collect(Collectors.toSet()); From ba269bd2151c92f46cd5a97f0d75648f7499d649 Mon Sep 17 00:00:00 2001 From: capistrant Date: Mon, 15 Dec 2025 13:00:34 -0600 Subject: [PATCH 13/72] Add some docs --- docs/api-reference/automatic-compaction-api.md | 1 + docs/configuration/index.md | 1 + 2 files changed, 2 insertions(+) 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..6296c4055483 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`| From f168bc9d95e5c7ac06c56a4b1b6ab4001d34a5ad Mon Sep 17 00:00:00 2001 From: capistrant Date: Mon, 15 Dec 2025 13:49:32 -0600 Subject: [PATCH 14/72] update web console --- .../compaction-dynamic-config-completions.ts | 21 +++++++++++++++ .../compaction-dynamic-config.tsx | 26 +++++++++++++++++++ 2 files changed, 47 insertions(+) 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. +

    + + ), + }, ]; From 2292b1569ef79a3a0ca0882c4a0e55e1756ba762 Mon Sep 17 00:00:00 2001 From: capistrant Date: Mon, 15 Dec 2025 15:19:02 -0600 Subject: [PATCH 15/72] make cache size configurable and fix some spelling --- .../metadata/CompactionStateManager.java | 49 ++++++++++++------- .../CompactionStateManagerConfig.java | 33 +++++++++++++ .../CompactionStateManagerConfigTest.java | 49 +++++++++++++++++++ .../druid/guice/MetadataManagerModule.java | 3 ++ website/.spelling | 1 + 5 files changed, 118 insertions(+), 17 deletions(-) create mode 100644 server/src/main/java/org/apache/druid/segment/metadata/CompactionStateManagerConfig.java create mode 100644 server/src/test/java/org/apache/druid/segment/metadata/CompactionStateManagerConfigTest.java 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 index 85f1a640418f..a9731859f098 100644 --- a/server/src/main/java/org/apache/druid/segment/metadata/CompactionStateManager.java +++ b/server/src/main/java/org/apache/druid/segment/metadata/CompactionStateManager.java @@ -24,7 +24,9 @@ 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; @@ -54,6 +56,8 @@ import java.util.List; import java.util.Map; import java.util.Set; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.locks.Lock; /** * Handles compaction state persistence on the Coordinator. @@ -63,27 +67,30 @@ public class CompactionStateManager { private static final EmittingLogger log = new EmittingLogger(CompactionStateManager.class); private static final int DB_ACTION_PARTITION_SIZE = 100; - private static final int DEFAULT_CACHE_SIZE = 100; private static final int DEFAULT_PREWARM_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 SQLMetadataConnector connector, + @Nonnull CompactionStateManagerConfig config ) { this.dbTables = dbTables; this.jsonMapper = jsonMapper; this.connector = connector; + this.config = config; this.fingerprintCache = CacheBuilder.newBuilder() - .maximumSize(DEFAULT_CACHE_SIZE) + .maximumSize(config.getCacheSize()) .build(); } @@ -115,16 +122,20 @@ public void stop() } @VisibleForTesting - protected CompactionStateManager() + 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, @@ -136,7 +147,10 @@ public void persistCompactionState( return; } - connector.retryWithHandle(handle -> { + final Lock lock = datasourceLocks.get(dataSource); + lock.lock(); + try { + connector.retryWithHandle(handle -> { // Fetch already existing compaction state fingerprints final Set existingFingerprints = getExistingFingerprints( handle, @@ -235,6 +249,9 @@ public void persistCompactionState( warmCache(fingerprintToStateMap); return null; }); + } finally { + lock.unlock(); + } } /** @@ -321,19 +338,17 @@ public int deleteUnusedCompactionStatesOlderThan(long timestamp) @Nullable public CompactionState getCompactionStateByFingerprint(String fingerprint) { - // Check cache first - CompactionState cached = fingerprintCache.getIfPresent(fingerprint); - if (cached != null) { - return cached; - } - - // Cache miss - load from database - CompactionState fromDb = loadCompactionStateFromDatabase(fingerprint); - if (fromDb != null) { - fingerprintCache.put(fingerprint, fromDb); + 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 (ExecutionException | CacheLoader.InvalidCacheLoadException e) { + return null; } - - return fromDb; } /** 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..f4481b981acc --- /dev/null +++ b/server/src/main/java/org/apache/druid/segment/metadata/CompactionStateManagerConfig.java @@ -0,0 +1,33 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.segment.metadata; + +import com.fasterxml.jackson.annotation.JsonProperty; + +public class CompactionStateManagerConfig +{ + @JsonProperty + private int cacheSize = 100; + + public int getCacheSize() + { + return cacheSize; + } +} \ No newline at end of file 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..aa989c558613 --- /dev/null +++ b/server/src/test/java/org/apache/druid/segment/metadata/CompactionStateManagerConfigTest.java @@ -0,0 +1,49 @@ +/* + * 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 nl.jqno.equalsverifier.EqualsVerifier; +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()); + } + + @Test + public void testSerdeRoundTripWithOverrides() throws Exception + { + String json = "{\"cacheSize\": 1000}"; + CompactionStateManagerConfig config = mapper.readValue(json, CompactionStateManagerConfig.class); + String serialized = mapper.writeValueAsString(config); + CompactionStateManagerConfig deserialized = mapper.readValue(serialized, CompactionStateManagerConfig.class); + Assert.assertEquals(1000, deserialized.getCacheSize()); + } +} \ No newline at end of file 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 7dc95d7ff0bc..08b9af422f65 100644 --- a/services/src/main/java/org/apache/druid/guice/MetadataManagerModule.java +++ b/services/src/main/java/org/apache/druid/guice/MetadataManagerModule.java @@ -42,6 +42,7 @@ 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; @@ -103,6 +104,8 @@ public void configure(Binder binder) binder.bind(SegmentMetadataCache.class) .to(HeapMemorySegmentMetadataCache.class) .in(LazySingleton.class); + + JsonConfigProvider.bind(binder, "druid.manager.compactionState", CompactionStateManagerConfig.class); binder.bind(CompactionStateManager.class) .in(ManageLifecycle.class); 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 From 74c8ebc3fbe5992a10fcc165c0008ce945e3b876 Mon Sep 17 00:00:00 2001 From: capistrant Date: Mon, 15 Dec 2025 15:38:54 -0600 Subject: [PATCH 16/72] fixup use of deprecated builder --- .../druid/timeline/DataSegmentTest.java | 32 +++++++------------ 1 file changed, 12 insertions(+), 20 deletions(-) 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 e46f69b38314..415117d729a5 100644 --- a/processing/src/test/java/org/apache/druid/timeline/DataSegmentTest.java +++ b/processing/src/test/java/org/apache/druid/timeline/DataSegmentTest.java @@ -41,6 +41,7 @@ import org.apache.druid.timeline.DataSegment.PruneSpecsHolder; import org.apache.druid.timeline.partition.NoneShardSpec; import org.apache.druid.timeline.partition.NumberedShardSpec; +import org.apache.druid.timeline.SegmentId; import org.apache.druid.timeline.partition.PartitionChunk; import org.apache.druid.timeline.partition.ShardSpec; import org.apache.druid.timeline.partition.ShardSpecLookup; @@ -519,15 +520,12 @@ 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() - .dataSource("something") - .interval(interval) - .version("1") + DataSegment segment = DataSegment.builder(segmentId) .loadSpec(loadSpec) .dimensions(Arrays.asList("dim1", "dim2")) .metrics(Arrays.asList("met1", "met2")) - .shardSpec(new NumberedShardSpec(3, 0)) .compactionStateFingerprint(fingerprint) .binaryVersion(TEST_VERSION) .size(1) @@ -551,15 +549,12 @@ public void testSerializationWithNullCompactionStateFingerprint() throws Excepti { 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() - .dataSource("something") - .interval(interval) - .version("1") + DataSegment segment = DataSegment.builder(segmentId) .loadSpec(loadSpec) .dimensions(Arrays.asList("dim1", "dim2")) .metrics(Arrays.asList("met1", "met2")) - .shardSpec(new NumberedShardSpec(3, 0)) .compactionStateFingerprint(null) .binaryVersion(TEST_VERSION) .size(1) @@ -606,19 +601,16 @@ public void testDeserializationBackwardCompatibility_missingCompactionStateFinge public void testWithCompactionStateFingerprint() { final String fingerprint = "test_fingerprint_12345"; - final DataSegment segment1 = DataSegment.builder() - .dataSource("foo") - .interval(Intervals.of("2012-01-01/2012-01-02")) - .version(DateTimes.of("2012-01-01T11:22:33.444Z").toString()) - .shardSpec(getShardSpec(7)) + 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() - .dataSource("foo") - .interval(Intervals.of("2012-01-01/2012-01-02")) - .version(DateTimes.of("2012-01-01T11:22:33.444Z").toString()) - .shardSpec(getShardSpec(7)) + final DataSegment segment2 = DataSegment.builder(segmentId) .size(0) .build(); From adac5ec61ae44c46ead883dc068b36335f3c3a5c Mon Sep 17 00:00:00 2001 From: capistrant Date: Mon, 15 Dec 2025 15:44:38 -0600 Subject: [PATCH 17/72] fix checktyle --- .../druid/timeline/DataSegmentTest.java | 1 - .../metadata/CompactionStateManager.java | 201 +++++++++--------- .../CompactionStateManagerConfig.java | 2 +- .../CompactionStateManagerConfigTest.java | 3 +- .../KillUnreferencedCompactionStateTest.java | 3 +- 5 files changed, 107 insertions(+), 103 deletions(-) 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 415117d729a5..f4ced3dfd38f 100644 --- a/processing/src/test/java/org/apache/druid/timeline/DataSegmentTest.java +++ b/processing/src/test/java/org/apache/druid/timeline/DataSegmentTest.java @@ -41,7 +41,6 @@ import org.apache.druid.timeline.DataSegment.PruneSpecsHolder; import org.apache.druid.timeline.partition.NoneShardSpec; import org.apache.druid.timeline.partition.NumberedShardSpec; -import org.apache.druid.timeline.SegmentId; import org.apache.druid.timeline.partition.PartitionChunk; import org.apache.druid.timeline.partition.ShardSpec; import org.apache.druid.timeline.partition.ShardSpecLookup; 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 index a9731859f098..b938d2ed55a7 100644 --- a/server/src/main/java/org/apache/druid/segment/metadata/CompactionStateManager.java +++ b/server/src/main/java/org/apache/druid/segment/metadata/CompactionStateManager.java @@ -133,7 +133,7 @@ public void stop() /** * 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. */ @@ -151,105 +151,106 @@ public void persistCompactionState( 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() + // Fetch already existing compaction state fingerprints + final Set existingFingerprints = getExistingFingerprints( + handle, + fingerprintToStateMap.keySet() ); - 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); + + 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(); } - markUsedBatch.execute(); - } - Map statesToPersist = new HashMap<>(); + Map statesToPersist = new HashMap<>(); - for (Map.Entry entry : fingerprintToStateMap.entrySet()) { - if (!existingFingerprints.contains(entry.getKey())) { - statesToPersist.put(entry.getKey(), entry.getValue()); + 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; - } + 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 - ); + 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() - ); + 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); + // 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 + ); + } } - 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)); + } } - } - 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 + ); } } - 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 { + warmCache(fingerprintToStateMap); + return null; + }); + } + finally { lock.unlock(); } } @@ -339,14 +340,18 @@ public int deleteUnusedCompactionStatesOlderThan(long timestamp) 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 (ExecutionException | CacheLoader.InvalidCacheLoadException e) { + 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 (ExecutionException | CacheLoader.InvalidCacheLoadException e) { return null; } } @@ -498,7 +503,7 @@ protected boolean isCached(String fingerprint) * 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 + * @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) @@ -516,9 +521,9 @@ private static String buildParameterizedInClause(String parameterPrefix, int val /** * Binds values to a parameterized IN clause in a SQL query. * - * @param values list of values to bind + * @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 + * @param query the SQL statement to bind values to */ private static void bindValuesToInClause( List values, 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 index f4481b981acc..cfe3d4fdce08 100644 --- a/server/src/main/java/org/apache/druid/segment/metadata/CompactionStateManagerConfig.java +++ b/server/src/main/java/org/apache/druid/segment/metadata/CompactionStateManagerConfig.java @@ -30,4 +30,4 @@ public int getCacheSize() { return cacheSize; } -} \ No newline at end of file +} 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 index aa989c558613..37297798b323 100644 --- a/server/src/test/java/org/apache/druid/segment/metadata/CompactionStateManagerConfigTest.java +++ b/server/src/test/java/org/apache/druid/segment/metadata/CompactionStateManagerConfigTest.java @@ -20,7 +20,6 @@ package org.apache.druid.segment.metadata; import com.fasterxml.jackson.databind.ObjectMapper; -import nl.jqno.equalsverifier.EqualsVerifier; import org.apache.druid.jackson.DefaultObjectMapper; import org.junit.Assert; import org.junit.Test; @@ -46,4 +45,4 @@ public void testSerdeRoundTripWithOverrides() throws Exception CompactionStateManagerConfig deserialized = mapper.readValue(serialized, CompactionStateManagerConfig.class); Assert.assertEquals(1000, deserialized.getCacheSize()); } -} \ No newline at end of file +} 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 index cdc2c45a6da3..49fffbde3346 100644 --- 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 @@ -27,6 +27,7 @@ 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; @@ -69,7 +70,7 @@ public void setUp() derbyConnector.createCompactionStatesTable(); derbyConnector.createSegmentTable(); - compactionStateManager = new CompactionStateManager(tablesConfig, jsonMapper, derbyConnector); + compactionStateManager = new CompactionStateManager(tablesConfig, jsonMapper, derbyConnector, new CompactionStateManagerConfig()); mockParams = EasyMock.createMock(DruidCoordinatorRuntimeParams.class); CoordinatorRunStats runStats = new CoordinatorRunStats(); From 4fb3a9c4861ce13149976d9dbdbe4012065c2427 Mon Sep 17 00:00:00 2001 From: capistrant Date: Mon, 15 Dec 2025 17:20:43 -0600 Subject: [PATCH 18/72] fix coordinator compactsegments duty and respond to self review comments --- docs/configuration/index.md | 2 + .../CompactionConfigBasedJobTemplate.java | 2 + .../druid/timeline/CompactionStateTest.java | 44 +- .../metadata/CompactionStateManager.java | 17 +- .../CompactionStateManagerConfig.java | 14 + .../server/compaction/CompactionStatus.java | 20 +- .../server/coordinator/DruidCoordinator.java | 2 +- .../coordinator/duty/CompactSegments.java | 2 + .../CompactionStateManagerConfigTest.java | 4 +- .../metadata/CompactionStateManagerTest.java | 417 ++++++++++++++++++ .../coordinator/duty/CompactSegmentsTest.java | 1 + 11 files changed, 511 insertions(+), 14 deletions(-) create mode 100644 server/src/test/java/org/apache/druid/segment/metadata/CompactionStateManagerTest.java diff --git a/docs/configuration/index.md b/docs/configuration/index.md index 6296c4055483..071ebe18872e 100644 --- a/docs/configuration/index.md +++ b/docs/configuration/index.md @@ -811,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/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 37f44e97b4ee..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 @@ -81,6 +81,8 @@ public List createCompactionJobs( ); 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), diff --git a/processing/src/test/java/org/apache/druid/timeline/CompactionStateTest.java b/processing/src/test/java/org/apache/druid/timeline/CompactionStateTest.java index a99eeec10850..13aa0aabcc3d 100644 --- a/processing/src/test/java/org/apache/druid/timeline/CompactionStateTest.java +++ b/processing/src/test/java/org/apache/druid/timeline/CompactionStateTest.java @@ -19,6 +19,7 @@ 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; @@ -67,7 +68,7 @@ public void test_generateCompactionStateFingerprint_differentDatasourcesWithSame } @Test - public void test_generateCompactionStateFingerprint_listOrderDoesNotInfluenceFingerprint() + public void test_generateCompactionStateFingerprint_metricsListOrderDifferenceResultsInNewFingerprint() { List metrics1 = Arrays.asList( new CountAggregatorFactory("count"), @@ -109,6 +110,47 @@ public void test_generateCompactionStateFingerprint_listOrderDoesNotInfluenceFin ); } + @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() { 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 index b938d2ed55a7..b3f57734814b 100644 --- a/server/src/main/java/org/apache/druid/segment/metadata/CompactionStateManager.java +++ b/server/src/main/java/org/apache/druid/segment/metadata/CompactionStateManager.java @@ -56,18 +56,23 @@ import java.util.List; import java.util.Map; import java.util.Set; -import java.util.concurrent.ExecutionException; import java.util.concurrent.locks.Lock; /** - * Handles compaction state persistence on the Coordinator. + * 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 static final int DEFAULT_PREWARM_SIZE = 100; private final MetadataStorageTablesConfig dbTables; private final ObjectMapper jsonMapper; @@ -105,7 +110,7 @@ public void start() ); if (tableExists) { log.info("Pre-warming compaction state cache"); - prewarmCache(DEFAULT_PREWARM_SIZE); + prewarmCache(config.getPrewarmFingerprintCount()); } else { log.info("Compaction states table does not exist, skipping pre-warm"); } @@ -351,7 +356,9 @@ public CompactionState getCompactionStateByFingerprint(String fingerprint) } ); } - catch (ExecutionException | CacheLoader.InvalidCacheLoadException e) { + 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; } } 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 index cfe3d4fdce08..04006931b8e5 100644 --- a/server/src/main/java/org/apache/druid/segment/metadata/CompactionStateManagerConfig.java +++ b/server/src/main/java/org/apache/druid/segment/metadata/CompactionStateManagerConfig.java @@ -23,11 +23,25 @@ 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/server/compaction/CompactionStatus.java b/server/src/main/java/org/apache/druid/server/compaction/CompactionStatus.java index 695943238817..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 @@ -345,6 +345,7 @@ private static class Evaluator 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<>(); @@ -399,17 +400,21 @@ private CompactionStatus evaluate() ); // 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) ); @@ -433,6 +438,9 @@ private CompactionStatus allFingerprintedCandidatesHaveExpectedFingerprint() 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); } } 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 7a2434b5141b..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 @@ -219,8 +219,8 @@ public DruidCoordinator( this.coordinatorDynamicConfigSyncer = coordinatorDynamicConfigSyncer; this.cloneStatusManager = cloneStatusManager; - this.compactSegments = initializeCompactSegmentsDuty(this.compactionStatusTracker); this.compactionStateManager = compactionStateManager; + this.compactSegments = initializeCompactSegmentsDuty(this.compactionStatusTracker); } public boolean isLeader() 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 7807137d4fff..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 @@ -284,6 +284,8 @@ private int submitCompactionTasks( 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), 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 index 37297798b323..edd7cac699b6 100644 --- a/server/src/test/java/org/apache/druid/segment/metadata/CompactionStateManagerConfigTest.java +++ b/server/src/test/java/org/apache/druid/segment/metadata/CompactionStateManagerConfigTest.java @@ -34,15 +34,17 @@ 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}"; + 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/coordinator/duty/CompactSegmentsTest.java b/server/src/test/java/org/apache/druid/server/coordinator/duty/CompactSegmentsTest.java index 6c15f46a5231..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 @@ -276,6 +276,7 @@ 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) ); From 708c6f895bba016a042ada5fcf1591d517dd242c Mon Sep 17 00:00:00 2001 From: capistrant Date: Mon, 15 Dec 2025 17:41:04 -0600 Subject: [PATCH 19/72] fix spellchecker --- website/.spelling | 1 + 1 file changed, 1 insertion(+) diff --git a/website/.spelling b/website/.spelling index ce5bd93e78e9..840d24b3df64 100644 --- a/website/.spelling +++ b/website/.spelling @@ -484,6 +484,7 @@ pre-compute pre-computed pre-computing pre-dates +pre-warms preconfigured pre-existing pre-filtered From 03bb14ae6cc5048453d50d9fb47185645ce7db59 Mon Sep 17 00:00:00 2001 From: capistrant Date: Tue, 16 Dec 2025 09:58:17 -0600 Subject: [PATCH 20/72] predates is a word --- docs/api-reference/automatic-compaction-api.md | 2 +- website/.spelling | 1 - 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/docs/api-reference/automatic-compaction-api.md b/docs/api-reference/automatic-compaction-api.md index 670fbff222a3..992b2b5ab805 100644 --- a/docs/api-reference/automatic-compaction-api.md +++ b/docs/api-reference/automatic-compaction-api.md @@ -889,7 +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`| +|`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 predates 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/website/.spelling b/website/.spelling index 840d24b3df64..ba35fa25a1a0 100644 --- a/website/.spelling +++ b/website/.spelling @@ -483,7 +483,6 @@ pre-computation pre-compute pre-computed pre-computing -pre-dates pre-warms preconfigured pre-existing From a262f79a1432cf5c56648843fd5276b259221acb Mon Sep 17 00:00:00 2001 From: capistrant Date: Tue, 16 Dec 2025 10:03:36 -0600 Subject: [PATCH 21/72] improve some javadocs --- .../druid/metadata/MetadataStorageConnector.java | 5 ++++- .../java/org/apache/druid/timeline/DataSegment.java | 11 +++++++++++ 2 files changed, 15 insertions(+), 1 deletion(-) 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 dfe30ec4af6d..221fb6b5ee0d 100644 --- a/processing/src/main/java/org/apache/druid/metadata/MetadataStorageConnector.java +++ b/processing/src/main/java/org/apache/druid/metadata/MetadataStorageConnector.java @@ -97,7 +97,10 @@ default void exportTable( void createSegmentSchemasTable(); /** - * + * CompactionStates table is centralized store for {@link org.apache.druid.timeline.CompactionState} objects. + *

    + * N segments can refer to the same compaction state via its unique fingerprint + *

    */ void createCompactionStatesTable(); } 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 19f3a817f6e0..b1d413950e65 100644 --- a/processing/src/main/java/org/apache/druid/timeline/DataSegment.java +++ b/processing/src/main/java/org/apache/druid/timeline/DataSegment.java @@ -116,6 +116,10 @@ public static class PruneSpecsHolder /** * SHA-256 fingerprint representation of the CompactionState. + *

    + * A null fingerprint indicates that this segment either has not been compacted, or was compacted before compaction + * fingerprinting existed. In the latter case, the segment would have a non-null {@link #lastCompactionState}. + *

    */ @Nullable private final String compactionStateFingerprint; @@ -353,6 +357,13 @@ public boolean isTombstone() return getShardSpec().getType().equals(ShardSpec.Type.TOMBSTONE); } + /** + * Get the compaction state fingerprint associated with this segment. + *

    + * A null fingerprint indicates that this segment either has not been compacted, or was compacted before compaction + * fingerprinting existed. In the latter case, the segment would have a non-null {@link #lastCompactionState}. + *

    + */ @Nullable @JsonProperty @JsonInclude(JsonInclude.Include.NON_NULL) From 6126e2276ec755d3acf2bb333b91a8b029aa860e Mon Sep 17 00:00:00 2001 From: capistrant Date: Tue, 16 Dec 2025 10:05:56 -0600 Subject: [PATCH 22/72] simplify some test assertions based on review --- .../compact/CompactionSupervisorTest.java | 17 +---------------- 1 file changed, 1 insertion(+), 16 deletions(-) diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionSupervisorTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionSupervisorTest.java index 52a7d04eaa63..a4579b4a5b15 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 @@ -310,24 +310,9 @@ private void verifyCompactedSegmentsHaveFingerprints(DataSourceCompactionConfig .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.getCompactionStateFingerprint(), "Segment " + segment.getId() + " fingerprint should match expected fingerprint" ); }); From b78ec137954d56b811e68ea22f34bc500298f305 Mon Sep 17 00:00:00 2001 From: capistrant Date: Tue, 16 Dec 2025 10:08:15 -0600 Subject: [PATCH 23/72] better naming --- .../testing/embedded/compact/CompactionSupervisorTest.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionSupervisorTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionSupervisorTest.java index a4579b4a5b15..85c1fe61e6f0 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 @@ -134,7 +134,7 @@ public void test_ingestDayGranularity_andCompactToMonthGranularity_andCompactToY Assertions.assertEquals(3, getNumSegmentsWith(Granularities.DAY)); // Create a compaction config with MONTH granularity - InlineSchemaDataSourceCompactionConfig monthGranConfig = + InlineSchemaDataSourceCompactionConfig monthGranularityConfig = InlineSchemaDataSourceCompactionConfig .builder() .forDataSource(dataSource) @@ -167,13 +167,13 @@ public void test_ingestDayGranularity_andCompactToMonthGranularity_andCompactToY ) .build(); - runCompactionWithSpec(monthGranConfig); + runCompactionWithSpec(monthGranularityConfig); waitForAllCompactionTasksToFinish(); Assertions.assertEquals(0, getNumSegmentsWith(Granularities.DAY)); Assertions.assertEquals(1, getNumSegmentsWith(Granularities.MONTH)); - verifyCompactedSegmentsHaveFingerprints(monthGranConfig); + verifyCompactedSegmentsHaveFingerprints(monthGranularityConfig); InlineSchemaDataSourceCompactionConfig yearGranConfig = InlineSchemaDataSourceCompactionConfig From 78f115e369c38edda1973748b351bcb9634c14eb Mon Sep 17 00:00:00 2001 From: capistrant Date: Tue, 16 Dec 2025 10:24:03 -0600 Subject: [PATCH 24/72] controller impl cleanup --- .../apache/druid/msq/exec/ControllerImpl.java | 19 ++++--------------- 1 file changed, 4 insertions(+), 15 deletions(-) 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 399a8e98ae3b..7d51eda543de 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 @@ -73,6 +73,7 @@ import org.apache.druid.indexing.common.actions.SegmentTransactionalReplaceAction; import org.apache.druid.indexing.common.actions.TaskAction; import org.apache.druid.indexing.common.actions.TaskActionClient; +import org.apache.druid.indexing.common.task.AbstractBatchIndexTask; import org.apache.druid.indexing.common.task.Tasks; import org.apache.druid.indexing.common.task.batch.TooManyBucketsException; import org.apache.druid.indexing.common.task.batch.parallel.TombstoneHelper; @@ -1696,7 +1697,7 @@ private void handleQueryResults( Tasks.DEFAULT_STORE_COMPACTION_STATE ); - String compactionStateFingerprint = querySpec.getContext() + final String compactionStateFingerprint = querySpec.getContext() .getString( Tasks.COMPACTION_STATE_FINGERPRINT_KEY, null @@ -1728,7 +1729,8 @@ private void handleQueryResults( } } if (compactionStateFingerprint != null) { - compactionStateAnnotateFunction = compactionStateAnnotateFunction.andThen(addCompactionStateFingerprintToSegments(compactionStateFingerprint)); + compactionStateAnnotateFunction = compactionStateAnnotateFunction.andThen( + AbstractBatchIndexTask.addCompactionStateFingerprintToSegments(compactionStateFingerprint)); } log.info("Query [%s] publishing %d segments.", queryDef.getQueryId(), segments.size()); publishAllSegments(segments, compactionStateAnnotateFunction); @@ -1760,19 +1762,6 @@ 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, From f06d7154bc9fdd74bc7239e9dc14294838b61467 Mon Sep 17 00:00:00 2001 From: capistrant Date: Tue, 16 Dec 2025 11:00:29 -0600 Subject: [PATCH 25/72] For compaction supervisors, take persisting pending compaction states out of hot path --- .../CompactionConfigBasedJobTemplate.java | 16 +++-------- .../druid/indexing/compact/CompactionJob.java | 27 +++++++++++++++++-- .../indexing/compact/CompactionJobQueue.java | 16 +++++++++++ 3 files changed, 44 insertions(+), 15 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionConfigBasedJobTemplate.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionConfigBasedJobTemplate.java index 5f5305df5e65..3e8be209de19 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionConfigBasedJobTemplate.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionConfigBasedJobTemplate.java @@ -23,7 +23,6 @@ import org.apache.druid.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; @@ -39,7 +38,6 @@ import javax.annotation.Nullable; import java.util.ArrayList; import java.util.List; -import java.util.Map; import java.util.Objects; /** @@ -80,16 +78,6 @@ public List createCompactionJobs( 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(); @@ -105,7 +93,9 @@ public List createCompactionJobs( new CompactionJob( taskPayload, candidate, - CompactionSlotManager.computeSlotsRequiredForTask(taskPayload) + CompactionSlotManager.computeSlotsRequiredForTask(taskPayload), + compactionStateFingerprint, + compactionState ) ); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionJob.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionJob.java index 7a7e7fdc1eab..0a29c0521689 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionJob.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionJob.java @@ -23,6 +23,7 @@ import org.apache.druid.indexing.template.BatchIndexingJob; import org.apache.druid.query.http.ClientSqlQuery; import org.apache.druid.server.compaction.CompactionCandidate; +import org.apache.druid.timeline.CompactionState; /** * {@link BatchIndexingJob} to compact an interval of a datasource. @@ -31,27 +32,37 @@ public class CompactionJob extends BatchIndexingJob { private final CompactionCandidate candidate; private final int maxRequiredTaskSlots; + private final String compactionStateFingerprint; + private final CompactionState compactionState; public CompactionJob( ClientCompactionTaskQuery task, CompactionCandidate candidate, - int maxRequiredTaskSlots + int maxRequiredTaskSlots, + String compactionStateFingerprint, + CompactionState compactionState ) { super(task, null); this.candidate = candidate; this.maxRequiredTaskSlots = maxRequiredTaskSlots; + this.compactionStateFingerprint = compactionStateFingerprint; + this.compactionState = compactionState; } public CompactionJob( ClientSqlQuery msqQuery, CompactionCandidate candidate, - int maxRequiredTaskSlots + int maxRequiredTaskSlots, + String compactionStateFingerprint, + CompactionState compactionState ) { super(null, msqQuery); this.candidate = candidate; this.maxRequiredTaskSlots = maxRequiredTaskSlots; + this.compactionStateFingerprint = compactionStateFingerprint; + this.compactionState = compactionState; } public String getDataSource() @@ -69,6 +80,16 @@ public int getMaxRequiredTaskSlots() return maxRequiredTaskSlots; } + public String getCompactionStateFingerprint() + { + return compactionStateFingerprint; + } + + public CompactionState getCompactionState() + { + return compactionState; + } + @Override public String toString() { @@ -76,6 +97,8 @@ public String toString() super.toString() + ", candidate=" + candidate + ", maxRequiredTaskSlots=" + maxRequiredTaskSlots + + ", compactionStateFingerprint='" + compactionStateFingerprint + '\'' + + ", compactionState=" + compactionState + '}'; } } 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 78648883e2df..2d11491f69b6 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 @@ -318,6 +318,7 @@ private String startTaskIfReady(CompactionJob job) // Assume MSQ jobs to be always ready if (job.isMsq()) { try { + persistPendingCompactionState(job); return FutureUtils.getUnchecked(brokerClient.submitSqlTask(job.getNonNullMsqQuery()), true) .getTaskId(); } @@ -336,6 +337,7 @@ private String startTaskIfReady(CompactionJob job) try { taskLockbox.add(task); if (task.isReady(taskActionClientFactory.create(task))) { + persistPendingCompactionState(job); // Hold the locks acquired by task.isReady() as we will reacquire them anyway FutureUtils.getUnchecked(overlordClient.runTask(task.getId(), task), true); return task.getId(); @@ -351,6 +353,20 @@ private String startTaskIfReady(CompactionJob job) } } + /** + * Persist the compaction state associated with the given job with {@link CompactionStateManager}. + */ + private void persistPendingCompactionState(CompactionJob job) + { + if (job.getCompactionState() != null && job.getCompactionStateFingerprint() != null) { + jobParams.getCompactionStateManager().persistCompactionState( + job.getDataSource(), + Map.of(job.getCompactionStateFingerprint(), job.getCompactionState()), + DateTimes.nowUtc() + ); + } + } + public CompactionStatus getCurrentStatusForJob(CompactionJob job, CompactionCandidateSearchPolicy policy) { final CompactionStatus compactionStatus = statusTracker.computeCompactionStatus(job.getCandidate(), policy); From d571e436c99619bc4b95d493c65e476b36945a04 Mon Sep 17 00:00:00 2001 From: capistrant Date: Tue, 16 Dec 2025 11:38:53 -0600 Subject: [PATCH 26/72] use Configs.valueOrDefault helper in data segment --- .../main/java/org/apache/druid/timeline/DataSegment.java | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) 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 b1d413950e65..6284039628b5 100644 --- a/processing/src/main/java/org/apache/druid/timeline/DataSegment.java +++ b/processing/src/main/java/org/apache/druid/timeline/DataSegment.java @@ -34,6 +34,7 @@ import com.google.common.collect.Interners; import com.google.inject.Inject; import it.unimi.dsi.fastutil.objects.Object2ObjectArrayMap; +import org.apache.druid.common.config.Configs; import org.apache.druid.guice.annotations.PublicApi; import org.apache.druid.jackson.CommaListJoinDeserializer; import org.apache.druid.jackson.CommaListJoinSerializer; @@ -260,9 +261,10 @@ public DataSegment( this.binaryVersion = binaryVersion; Preconditions.checkArgument(size >= 0); this.size = size; - this.compactionStateFingerprint = compactionStateFingerprint != null - ? STRING_INTERNER.intern(compactionStateFingerprint) - : null; + this.compactionStateFingerprint = Configs.valueOrDefault( + STRING_INTERNER.intern(compactionStateFingerprint), + null + ); } /** From 07afc2f078c4df516d04b4bae56f6d5ad40c2386 Mon Sep 17 00:00:00 2001 From: capistrant Date: Tue, 16 Dec 2025 13:08:14 -0600 Subject: [PATCH 27/72] Refactor where fingerprinting happens and how the object mapper is wired up --- .../compact/CompactionSupervisorTest.java | 8 +- .../CompactionConfigBasedJobTemplate.java | 2 +- .../druid/guice/DruidSecondaryModule.java | 17 ++ .../guice/annotations/Deterministic.java | 38 ++++ .../druid/timeline/CompactionState.java | 63 ------ .../druid/timeline/CompactionStateTest.java | 199 ------------------ .../metadata/CompactionStateManager.java | 39 ++++ .../HeapMemoryCompactionStateManager.java | 46 +++- .../server/compaction/CompactionStatus.java | 2 +- .../coordinator/duty/CompactSegments.java | 2 +- .../metadata/CompactionStateManagerTest.java | 185 +++++++++++++++- .../compaction/CompactionStatusTest.java | 11 +- .../KillUnreferencedCompactionStateTest.java | 15 +- 13 files changed, 352 insertions(+), 275 deletions(-) create mode 100644 processing/src/main/java/org/apache/druid/guice/annotations/Deterministic.java delete mode 100644 processing/src/test/java/org/apache/druid/timeline/CompactionStateTest.java diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionSupervisorTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionSupervisorTest.java index 85c1fe61e6f0..df83c473a63a 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 @@ -36,6 +36,7 @@ import org.apache.druid.msq.guice.SqlTaskModule; import org.apache.druid.query.DruidMetrics; import org.apache.druid.rpc.UpdateResponse; +import org.apache.druid.segment.metadata.CompactionStateManager; import org.apache.druid.server.coordinator.ClusterCompactionConfig; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; import org.apache.druid.server.coordinator.InlineSchemaDataSourceCompactionConfig; @@ -51,7 +52,6 @@ import org.apache.druid.testing.embedded.EmbeddedRouter; import org.apache.druid.testing.embedded.indexing.MoreResources; import org.apache.druid.testing.embedded.junit5.EmbeddedClusterTestBase; -import org.apache.druid.timeline.CompactionState; import org.hamcrest.Matcher; import org.hamcrest.Matchers; import org.joda.time.Period; @@ -300,7 +300,11 @@ private void verifySegmentsHaveNullLastCompactionStateAndNonNullFingerprint() private void verifyCompactedSegmentsHaveFingerprints(DataSourceCompactionConfig compactionConfig) { - String expectedFingerprint = CompactionState.generateCompactionStateFingerprint( + CompactionStateManager compactionStateManager = overlord + .bindings() + .getInstance(CompactionStateManager.class); + + String expectedFingerprint = compactionStateManager.generateCompactionStateFingerprint( CompactSegments.createCompactionStateFromConfig(compactionConfig), dataSource ); 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 3e8be209de19..c209b90418e0 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 @@ -73,7 +73,7 @@ public List createCompactionJobs( CompactionState compactionState = CompactSegments.createCompactionStateFromConfig(config); - String compactionStateFingerprint = CompactionState.generateCompactionStateFingerprint( + String compactionStateFingerprint = params.getCompactionStateManager().generateCompactionStateFingerprint( compactionState, config.getDataSource() ); diff --git a/processing/src/main/java/org/apache/druid/guice/DruidSecondaryModule.java b/processing/src/main/java/org/apache/druid/guice/DruidSecondaryModule.java index be0ea786f58e..6ff6fda3c0b8 100644 --- a/processing/src/main/java/org/apache/druid/guice/DruidSecondaryModule.java +++ b/processing/src/main/java/org/apache/druid/guice/DruidSecondaryModule.java @@ -20,7 +20,9 @@ package org.apache.druid.guice; import com.fasterxml.jackson.databind.AnnotationIntrospector; +import com.fasterxml.jackson.databind.MapperFeature; import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.SerializationFeature; import com.fasterxml.jackson.databind.introspect.AnnotationIntrospectorPair; import com.google.inject.Binder; import com.google.inject.Inject; @@ -28,9 +30,11 @@ import com.google.inject.Key; import com.google.inject.Module; import com.google.inject.Provides; +import org.apache.druid.guice.annotations.Deterministic; import org.apache.druid.guice.annotations.Json; import org.apache.druid.guice.annotations.JsonNonNull; import org.apache.druid.guice.annotations.Smile; +import org.apache.druid.jackson.DefaultObjectMapper; import javax.validation.Validator; import java.util.Properties; @@ -91,6 +95,19 @@ public ObjectMapper getSmileMapper(Injector injector) return smileMapper; } + @Provides + @LazySingleton + @Deterministic + public ObjectMapper getSortedMapper(Injector injector) + { + final ObjectMapper sortedMapper = new DefaultObjectMapper(); + sortedMapper.configure(SerializationFeature.ORDER_MAP_ENTRIES_BY_KEYS, true); + sortedMapper.configure(MapperFeature.SORT_PROPERTIES_ALPHABETICALLY, true); + setupJackson(injector, sortedMapper); + return sortedMapper; + } + + public static void setupJackson(Injector injector, final ObjectMapper mapper) { mapper.setInjectableValues(new GuiceInjectableValues(injector)); diff --git a/processing/src/main/java/org/apache/druid/guice/annotations/Deterministic.java b/processing/src/main/java/org/apache/druid/guice/annotations/Deterministic.java new file mode 100644 index 000000000000..12206e263e92 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/guice/annotations/Deterministic.java @@ -0,0 +1,38 @@ +/* + * 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.guice.annotations; + +import com.google.inject.BindingAnnotation; + +import java.lang.annotation.ElementType; +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; +import java.lang.annotation.Target; + +/** + * ObjectMapper with deterministic serialization for fingerprinting. + * Serializes with sorted map keys and alphabetically ordered properties. + */ +@Target({ElementType.FIELD, ElementType.PARAMETER, ElementType.METHOD}) +@Retention(RetentionPolicy.RUNTIME) +@BindingAnnotation +public @interface Deterministic +{ +} 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 e4542e553d6a..14c100736e7d 100644 --- a/processing/src/main/java/org/apache/druid/timeline/CompactionState.java +++ b/processing/src/main/java/org/apache/druid/timeline/CompactionState.java @@ -22,19 +22,10 @@ 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; @@ -60,25 +51,6 @@ 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; @@ -152,23 +124,6 @@ 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) { @@ -252,22 +207,4 @@ 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/test/java/org/apache/druid/timeline/CompactionStateTest.java b/processing/src/test/java/org/apache/druid/timeline/CompactionStateTest.java deleted file mode 100644 index 13aa0aabcc3d..000000000000 --- a/processing/src/test/java/org/apache/druid/timeline/CompactionStateTest.java +++ /dev/null @@ -1,199 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.druid.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/server/src/main/java/org/apache/druid/segment/metadata/CompactionStateManager.java b/server/src/main/java/org/apache/druid/segment/metadata/CompactionStateManager.java index b3f57734814b..4d03e055cfa5 100644 --- a/server/src/main/java/org/apache/druid/segment/metadata/CompactionStateManager.java +++ b/server/src/main/java/org/apache/druid/segment/metadata/CompactionStateManager.java @@ -26,10 +26,14 @@ import com.google.common.cache.CacheBuilder; import com.google.common.cache.CacheLoader; import com.google.common.collect.Lists; +import com.google.common.hash.Hasher; +import com.google.common.hash.Hashing; +import com.google.common.io.BaseEncoding; 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.guice.annotations.Deterministic; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.StringUtils; @@ -76,6 +80,7 @@ public class CompactionStateManager private final MetadataStorageTablesConfig dbTables; private final ObjectMapper jsonMapper; + private final ObjectMapper deterministicMapper; private final SQLMetadataConnector connector; private final CompactionStateManagerConfig config; private final Cache fingerprintCache; @@ -85,12 +90,14 @@ public class CompactionStateManager public CompactionStateManager( @Nonnull MetadataStorageTablesConfig dbTables, @Nonnull ObjectMapper jsonMapper, + @Deterministic @Nonnull ObjectMapper deterministicMapper, @Nonnull SQLMetadataConnector connector, @Nonnull CompactionStateManagerConfig config ) { this.dbTables = dbTables; this.jsonMapper = jsonMapper; + this.deterministicMapper = deterministicMapper; this.connector = connector; this.config = config; @@ -131,6 +138,7 @@ public void stop() { this.dbTables = null; this.jsonMapper = null; + this.deterministicMapper = null; this.connector = null; this.config = null; this.fingerprintCache = null; @@ -363,6 +371,37 @@ public CompactionState getCompactionStateByFingerprint(String fingerprint) } } + /** + * Generates a deterministic fingerprint for the given compaction state and datasource. + * The fingerprint is a SHA-256 hash of the datasource name and serialized compaction state. + * + * @param compactionState The compaction configuration to fingerprint + * @param dataSource The datasource name + * @return A hex-encoded SHA-256 fingerprint string + */ + @SuppressWarnings("UnstableApiUsage") + public String generateCompactionStateFingerprint( + final CompactionState compactionState, + final String dataSource + ) + { + final Hasher hasher = Hashing.sha256().newHasher(); + + hasher.putBytes(StringUtils.toUtf8(dataSource)); + hasher.putByte((byte) 0xff); + + try { + hasher.putBytes(deterministicMapper.writeValueAsBytes(compactionState)); + } + catch (JsonProcessingException e) { + throw new RuntimeException("Failed to serialize CompactionState for fingerprinting", e); + } + hasher.putByte((byte) 0xff); + + return BaseEncoding.base16().encode(hasher.hash().asBytes()); + } + + /** * Warms cache with specific states (after persisting). */ 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 index e0d8fdcf545c..c8b70c920712 100644 --- a/server/src/main/java/org/apache/druid/segment/metadata/HeapMemoryCompactionStateManager.java +++ b/server/src/main/java/org/apache/druid/segment/metadata/HeapMemoryCompactionStateManager.java @@ -19,12 +19,18 @@ package org.apache.druid.segment.metadata; +import com.fasterxml.jackson.databind.MapperFeature; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.SerializationFeature; +import org.apache.druid.jackson.DefaultObjectMapper; +import org.apache.druid.metadata.MetadataStorageTablesConfig; import org.apache.druid.timeline.CompactionState; import org.joda.time.DateTime; import javax.annotation.Nullable; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; /** * In-memory implementation of {@link CompactionStateManager} that stores @@ -34,7 +40,45 @@ */ public class HeapMemoryCompactionStateManager extends CompactionStateManager { - private final Map fingerprintToStateMap = new ConcurrentHashMap<>(); + private final ConcurrentMap fingerprintToStateMap = new ConcurrentHashMap<>(); + + /** + * Creates an in-memory compaction state manager with a default deterministic mapper. + * This is a convenience constructor for tests and simulations. + */ + public HeapMemoryCompactionStateManager() + { + this(createDeterministicMapper()); + } + + /** + * Creates an in-memory compaction state manager with the provided deterministic mapper + * for fingerprint generation. + * + * @param deterministicMapper ObjectMapper configured for deterministic serialization + */ + public HeapMemoryCompactionStateManager(ObjectMapper deterministicMapper) + { + super( + new MetadataStorageTablesConfig(null, null, null, null, null, null, null, null, null, null, null, null, null, null), + new DefaultObjectMapper(), + deterministicMapper, + null, + new CompactionStateManagerConfig() + ); + } + + /** + * Creates an ObjectMapper configured for deterministic serialization. + * Used for generating consistent fingerprints. + */ + private static ObjectMapper createDeterministicMapper() + { + ObjectMapper mapper = new DefaultObjectMapper(); + mapper.configure(SerializationFeature.ORDER_MAP_ENTRIES_BY_KEYS, true); + mapper.configure(MapperFeature.SORT_PROPERTIES_ALPHABETICALLY, true); + return mapper; + } @Override public void persistCompactionState( 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 6761c2529f82..ea4ce2de6494 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 @@ -265,7 +265,7 @@ static CompactionStatus compute( ) { final CompactionState expectedState = CompactSegments.createCompactionStateFromConfig(config); - final String expectedFingerprint = CompactionState.generateCompactionStateFingerprint(expectedState, config.getDataSource()); + final String expectedFingerprint = compactionStateManager.generateCompactionStateFingerprint(expectedState, config.getDataSource()); return new Evaluator(candidateSegments, config, expectedFingerprint, compactionStateManager).evaluate(); } 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 348119f57400..b4f1c59db4a2 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 @@ -279,7 +279,7 @@ private int submitCompactionTasks( CompactionState compactionState = createCompactionStateFromConfig(config); - String compactionStateFingerprint = CompactionState.generateCompactionStateFingerprint( + String compactionStateFingerprint = compactionStateManager.generateCompactionStateFingerprint( compactionState, config.getDataSource() ); 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 index 3aad61e43226..145caa99355c 100644 --- a/server/src/test/java/org/apache/druid/segment/metadata/CompactionStateManagerTest.java +++ b/server/src/test/java/org/apache/druid/segment/metadata/CompactionStateManagerTest.java @@ -19,12 +19,20 @@ package org.apache.druid.segment.metadata; +import com.fasterxml.jackson.databind.MapperFeature; import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.SerializationFeature; +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.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.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.apache.druid.timeline.CompactionState; import org.joda.time.DateTime; @@ -33,12 +41,15 @@ import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.RegisterExtension; +import java.util.Arrays; +import java.util.Collections; 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.assertNotEquals; import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.junit.jupiter.api.Assertions.assertNull; import static org.junit.jupiter.api.Assertions.assertTrue; @@ -50,6 +61,7 @@ public class CompactionStateManagerTest new TestDerbyConnector.DerbyConnectorRule5(); private final ObjectMapper jsonMapper = new DefaultObjectMapper(); + private final ObjectMapper deterministicMapper = createDeterministicMapper(); private static TestDerbyConnector derbyConnector; private static MetadataStorageTablesConfig tablesConfig; @@ -73,7 +85,7 @@ public void setUp() return null; }); - manager = new CompactionStateManager(tablesConfig, jsonMapper, derbyConnector, new CompactionStateManagerConfig()); + manager = new CompactionStateManager(tablesConfig, jsonMapper, deterministicMapper, derbyConnector, new CompactionStateManagerConfig()); } @Test @@ -282,6 +294,7 @@ public void test_prewarmCache_onModuleLifecycleStart() throws Exception CompactionStateManager newManager = new CompactionStateManager( tablesConfig, jsonMapper, + deterministicMapper, derbyConnector, new CompactionStateManagerConfig() ); @@ -401,6 +414,176 @@ public void test_markCompactionStateAsUsed_withEmptyList_returnsZero() assertEquals(0, manager.markCompactionStatesAsUsed(List.of())); } + // ===== Fingerprint Generation Tests ===== + + @Test + public void test_generateCompactionStateFingerprint_deterministicFingerprinting() + { + CompactionState compactionState1 = createBasicCompactionState(); + CompactionState compactionState2 = createBasicCompactionState(); + + String fingerprint1 = manager.generateCompactionStateFingerprint(compactionState1, "test-ds"); + String fingerprint2 = manager.generateCompactionStateFingerprint(compactionState2, "test-ds"); + + 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 = manager.generateCompactionStateFingerprint(compactionState, "ds1"); + String fingerprint2 = manager.generateCompactionStateFingerprint(compactionState, "ds2"); + + 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 = manager.generateCompactionStateFingerprint(state1, "test-ds"); + String fingerprint2 = manager.generateCompactionStateFingerprint(state2, "test-ds"); + + 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 = manager.generateCompactionStateFingerprint(state1, "test-ds"); + String fingerprint2 = manager.generateCompactionStateFingerprint(state2, "test-ds"); + + 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 = manager.generateCompactionStateFingerprint(state1, "test-ds"); + String fingerprint2 = manager.generateCompactionStateFingerprint(state2, "test-ds"); + + assertNotEquals( + fingerprint1, + fingerprint2, + "Different PartitionsSpec should produce different fingerprints" + ); + } + + private static ObjectMapper createDeterministicMapper() + { + ObjectMapper mapper = new DefaultObjectMapper(); + mapper.configure(SerializationFeature.ORDER_MAP_ENTRIES_BY_KEYS, true); + mapper.configure(MapperFeature.SORT_PROPERTIES_ALPHABETICALLY, true); + return mapper; + } + + private CompactionState createBasicCompactionState() + { + return new CompactionState( + new DynamicPartitionsSpec(5000000, null), + DimensionsSpec.EMPTY, + Collections.singletonList(new CountAggregatorFactory("count")), + null, + IndexSpec.getDefault(), + null, + null + ); + } private CompactionState createTestCompactionState() { 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 2f03be6ffa3e..159536c2502c 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 @@ -39,6 +39,7 @@ 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.metadata.HeapMemoryCompactionStateManager; import org.apache.druid.segment.nested.NestedCommonFormatColumnFormatSpec; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; import org.apache.druid.server.coordinator.InlineSchemaDataSourceCompactionConfig; @@ -73,7 +74,7 @@ public class CompactionStatusTest @Before public void setUp() { - compactionStateManager = EasyMock.createMock(CompactionStateManager.class); + compactionStateManager = new HeapMemoryCompactionStateManager(); } @Test @@ -597,7 +598,7 @@ public void test_evaluate_needsCompactionWhenSomeSegmentsHaveUnexpectedCompactio CompactionState expectedState = CompactSegments.createCompactionStateFromConfig(compactionConfig); - String expectedFingerprint = CompactionState.generateCompactionStateFingerprint(expectedState, TestDataSource.WIKI); + String expectedFingerprint = compactionStateManager.generateCompactionStateFingerprint(expectedState, TestDataSource.WIKI); List segments = List.of( DataSegment.builder(WIKI_SEGMENT).compactionStateFingerprint(expectedFingerprint).build(), @@ -689,7 +690,7 @@ public void test_evaluate_noCompactionWhenAllSegmentsHaveExpectedCompactionState CompactionState expectedState = CompactSegments.createCompactionStateFromConfig(compactionConfig); - String expectedFingerprint = CompactionState.generateCompactionStateFingerprint(expectedState, TestDataSource.WIKI); + String expectedFingerprint = compactionStateManager.generateCompactionStateFingerprint(expectedState, TestDataSource.WIKI); List segments = List.of( DataSegment.builder(WIKI_SEGMENT).compactionStateFingerprint(expectedFingerprint).build(), @@ -715,7 +716,7 @@ public void test_evaluate_needsCompactionWhenNonFingerprintedSegmentsFailChecksO CompactionState expectedState = CompactSegments.createCompactionStateFromConfig(compactionConfig); - String expectedFingerprint = CompactionState.generateCompactionStateFingerprint(expectedState, TestDataSource.WIKI); + String expectedFingerprint = compactionStateManager.generateCompactionStateFingerprint(expectedState, TestDataSource.WIKI); List segments = List.of( DataSegment.builder(WIKI_SEGMENT).compactionStateFingerprint(expectedFingerprint).build(), @@ -741,7 +742,7 @@ public void test_evaluate_noCompactionWhenNonFingerprintedSegmentsPassChecksOnLa CompactionState expectedState = CompactSegments.createCompactionStateFromConfig(compactionConfig); - String expectedFingerprint = CompactionState.generateCompactionStateFingerprint(expectedState, TestDataSource.WIKI); + String expectedFingerprint = compactionStateManager.generateCompactionStateFingerprint(expectedState, TestDataSource.WIKI); List segments = List.of( DataSegment.builder(WIKI_SEGMENT).compactionStateFingerprint(expectedFingerprint).build(), 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 index 49fffbde3346..b3921db0cbcb 100644 --- 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 @@ -19,7 +19,9 @@ package org.apache.druid.server.coordinator.duty; +import com.fasterxml.jackson.databind.MapperFeature; import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.SerializationFeature; import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.DateTimes; @@ -70,7 +72,10 @@ public void setUp() derbyConnector.createCompactionStatesTable(); derbyConnector.createSegmentTable(); - compactionStateManager = new CompactionStateManager(tablesConfig, jsonMapper, derbyConnector, new CompactionStateManagerConfig()); + compactionStateManager = new CompactionStateManager(tablesConfig, jsonMapper, createDeterministicMapper(), + derbyConnector, + new CompactionStateManagerConfig() + ); mockParams = EasyMock.createMock(DruidCoordinatorRuntimeParams.class); CoordinatorRunStats runStats = new CoordinatorRunStats(); @@ -259,4 +264,12 @@ private Boolean getCompactionStateUsedStatus(String fingerprint) return usedStatus.isEmpty() ? null : usedStatus.get(0); } + + private static ObjectMapper createDeterministicMapper() + { + ObjectMapper mapper = new DefaultObjectMapper(); + mapper.configure(SerializationFeature.ORDER_MAP_ENTRIES_BY_KEYS, true); + mapper.configure(MapperFeature.SORT_PROPERTIES_ALPHABETICALLY, true); + return mapper; + } } From 12ea7417953f0057eff34a085fbe0281054ba17d Mon Sep 17 00:00:00 2001 From: capistrant Date: Tue, 16 Dec 2025 15:12:24 -0600 Subject: [PATCH 28/72] refactor CompactionStateManager into an interface with a persisted and heap impl --- .../apache/druid/timeline/DataSegment.java | 8 +- .../metadata/CompactionStateManager.java | 561 ++--------------- .../HeapMemoryCompactionStateManager.java | 41 +- .../PersistedCompactionStateManager.java | 564 ++++++++++++++++++ .../compaction/CompactionRunSimulator.java | 2 +- ...stedCompactionStateManagerConfigTest.java} | 2 +- ... PersistedCompactionStateManagerTest.java} | 8 +- .../compaction/CompactionStatusTest.java | 34 +- .../coordinator/DruidCoordinatorTest.java | 20 +- .../KillUnreferencedCompactionStateTest.java | 6 +- .../druid/guice/MetadataManagerModule.java | 4 +- 11 files changed, 691 insertions(+), 559 deletions(-) create mode 100644 server/src/main/java/org/apache/druid/segment/metadata/PersistedCompactionStateManager.java rename server/src/test/java/org/apache/druid/segment/metadata/{CompactionStateManagerConfigTest.java => PersistedCompactionStateManagerConfigTest.java} (97%) rename server/src/test/java/org/apache/druid/segment/metadata/{CompactionStateManagerTest.java => PersistedCompactionStateManagerTest.java} (98%) 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 6284039628b5..45e6f0bc9a43 100644 --- a/processing/src/main/java/org/apache/druid/timeline/DataSegment.java +++ b/processing/src/main/java/org/apache/druid/timeline/DataSegment.java @@ -34,7 +34,6 @@ import com.google.common.collect.Interners; import com.google.inject.Inject; import it.unimi.dsi.fastutil.objects.Object2ObjectArrayMap; -import org.apache.druid.common.config.Configs; import org.apache.druid.guice.annotations.PublicApi; import org.apache.druid.jackson.CommaListJoinDeserializer; import org.apache.druid.jackson.CommaListJoinSerializer; @@ -261,10 +260,9 @@ public DataSegment( this.binaryVersion = binaryVersion; Preconditions.checkArgument(size >= 0); this.size = size; - this.compactionStateFingerprint = Configs.valueOrDefault( - STRING_INTERNER.intern(compactionStateFingerprint), - null - ); + this.compactionStateFingerprint = compactionStateFingerprint == null ? + null : + STRING_INTERNER.intern(compactionStateFingerprint); } /** 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 index 4d03e055cfa5..2ef592c36c8a 100644 --- a/server/src/main/java/org/apache/druid/segment/metadata/CompactionStateManager.java +++ b/server/src/main/java/org/apache/druid/segment/metadata/CompactionStateManager.java @@ -19,358 +19,21 @@ 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.hash.Hasher; -import com.google.common.hash.Hashing; -import com.google.common.io.BaseEncoding; -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.guice.annotations.Deterministic; -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. + * Manages compaction state persistence and fingerprint generation. *

    - * 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. - *

    + * Implementations may be backed by a database (like {@link PersistedCompactionStateManager}) or + * use in-memory storage (like {@link HeapMemoryCompactionStateManager}). */ -@ManageLifecycle -public class CompactionStateManager +public interface 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 ObjectMapper deterministicMapper; - 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, - @Deterministic @Nonnull ObjectMapper deterministicMapper, - @Nonnull SQLMetadataConnector connector, - @Nonnull CompactionStateManagerConfig config - ) - { - this.dbTables = dbTables; - this.jsonMapper = jsonMapper; - this.deterministicMapper = deterministicMapper; - 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.deterministicMapper = 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; - } - } - /** * Generates a deterministic fingerprint for the given compaction state and datasource. * The fingerprint is a SHA-256 hash of the datasource name and serialized compaction state. @@ -379,206 +42,74 @@ public CompactionState getCompactionStateByFingerprint(String fingerprint) * @param dataSource The datasource name * @return A hex-encoded SHA-256 fingerprint string */ - @SuppressWarnings("UnstableApiUsage") - public String generateCompactionStateFingerprint( - final CompactionState compactionState, - final String dataSource - ) - { - final Hasher hasher = Hashing.sha256().newHasher(); - - hasher.putBytes(StringUtils.toUtf8(dataSource)); - hasher.putByte((byte) 0xff); - - try { - hasher.putBytes(deterministicMapper.writeValueAsBytes(compactionState)); - } - catch (JsonProcessingException e) { - throw new RuntimeException("Failed to serialize CompactionState for fingerprinting", e); - } - hasher.putByte((byte) 0xff); - - return BaseEncoding.base16().encode(hasher.hash().asBytes()); - } - + String generateCompactionStateFingerprint(CompactionState compactionState, String dataSource); /** - * Warms cache with specific states (after persisting). + * Retrieves a compaction state by its fingerprint. + * + * @param fingerprint The fingerprint to look up + * @return The compaction state, or null if not found */ - private void warmCache(Map fingerprintToStateMap) - { - fingerprintCache.putAll(fingerprintToStateMap); - log.debug("Warmed cache with [%d] compaction states", fingerprintToStateMap.size()); - } + @Nullable + CompactionState getCompactionStateByFingerprint(String fingerprint); /** - * Pre-warms the cache by loading the N most recently used fingerprints. + * Persists compaction states to storage. + * + * @param dataSource The datasource name + * @param fingerprintToStateMap Map of fingerprints to their compaction states + * @param updateTime The timestamp for this update */ - 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 - ); - - } + void persistCompactionState( + String dataSource, + Map fingerprintToStateMap, + DateTime updateTime + ); /** - * Invalidates a fingerprint from cache. + * Marks compaction states as unused if they are not referenced by any used segments. + * This is used for cleanup operations. Implementations may choose to no-op this. + * + * @return Number of rows updated, or 0 if not applicable */ - public void invalidateFingerprint(String fingerprint) + default int markUnreferencedCompactionStatesAsUnused() { - fingerprintCache.invalidate(fingerprint); + return 0; } /** - * Loads from database. Returns null if not found or unused. + * Finds all compaction state fingerprints which have been marked as unused but are + * still referenced by some used segments. This is used for validation/reconciliation. + * Implementations may return an empty list if not applicable. + * + * @return List of fingerprints, or empty list */ - @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) + default List findReferencedCompactionStateMarkedAsUnused() { - return fingerprintCache.getIfPresent(fingerprint) != null; + return List.of(); } /** - * Builds a parameterized IN clause for the specified column with placeholders. - * Must be followed by a call to {@link #bindValuesToInClause(List, String, SQLStatement)}. + * Marks compaction states as used. This is used for reconciliation operations. + * Implementations may choose to no-op this. * - * @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 + * @param stateFingerprints List of fingerprints to mark as used + * @return Number of rows updated, or 0 if not applicable */ - private static String buildParameterizedInClause(String parameterPrefix, int valueCount) + default int markCompactionStatesAsUsed(List stateFingerprints) { - 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(); + return 0; } /** - * Binds values to a parameterized IN clause in a SQL query. + * Deletes unused compaction states older than the given timestamp. + * This is used for cleanup operations. Implementations may choose to no-op this. * - * @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 + * @param timestamp The cutoff timestamp in milliseconds + * @return Number of rows deleted, or 0 if not applicable */ - private static void bindValuesToInClause( - List values, - String parameterPrefix, - SQLStatement query - ) + default int deleteUnusedCompactionStatesOlderThan(long timestamp) { - for (int i = 0; i < values.size(); i++) { - query.bind(parameterPrefix + i, values.get(i)); - } + return 0; } } 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 index c8b70c920712..c71b09934e73 100644 --- a/server/src/main/java/org/apache/druid/segment/metadata/HeapMemoryCompactionStateManager.java +++ b/server/src/main/java/org/apache/druid/segment/metadata/HeapMemoryCompactionStateManager.java @@ -19,11 +19,15 @@ package org.apache.druid.segment.metadata; +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.jackson.DefaultObjectMapper; -import org.apache.druid.metadata.MetadataStorageTablesConfig; +import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.timeline.CompactionState; import org.joda.time.DateTime; @@ -37,10 +41,12 @@ * compaction state fingerprints in heap memory without requiring a database. *

    * Useful for simulations and unit tests where database persistence is not needed. + * Database-specific operations (cleanup, unused marking) are no-ops in this implementation. */ -public class HeapMemoryCompactionStateManager extends CompactionStateManager +public class HeapMemoryCompactionStateManager implements CompactionStateManager { private final ConcurrentMap fingerprintToStateMap = new ConcurrentHashMap<>(); + private final ObjectMapper deterministicMapper; /** * Creates an in-memory compaction state manager with a default deterministic mapper. @@ -59,13 +65,7 @@ public HeapMemoryCompactionStateManager() */ public HeapMemoryCompactionStateManager(ObjectMapper deterministicMapper) { - super( - new MetadataStorageTablesConfig(null, null, null, null, null, null, null, null, null, null, null, null, null, null), - new DefaultObjectMapper(), - deterministicMapper, - null, - new CompactionStateManagerConfig() - ); + this.deterministicMapper = deterministicMapper; } /** @@ -80,6 +80,29 @@ private static ObjectMapper createDeterministicMapper() return mapper; } + @Override + @SuppressWarnings("UnstableApiUsage") + public String generateCompactionStateFingerprint( + final CompactionState compactionState, + final String dataSource + ) + { + final Hasher hasher = Hashing.sha256().newHasher(); + + hasher.putBytes(StringUtils.toUtf8(dataSource)); + hasher.putByte((byte) 0xff); + + try { + hasher.putBytes(deterministicMapper.writeValueAsBytes(compactionState)); + } + catch (JsonProcessingException e) { + throw new RuntimeException("Failed to serialize CompactionState for fingerprinting", e); + } + hasher.putByte((byte) 0xff); + + return BaseEncoding.base16().encode(hasher.hash().asBytes()); + } + @Override public void persistCompactionState( final String dataSource, diff --git a/server/src/main/java/org/apache/druid/segment/metadata/PersistedCompactionStateManager.java b/server/src/main/java/org/apache/druid/segment/metadata/PersistedCompactionStateManager.java new file mode 100644 index 000000000000..e7d320c06eae --- /dev/null +++ b/server/src/main/java/org/apache/druid/segment/metadata/PersistedCompactionStateManager.java @@ -0,0 +1,564 @@ +/* + * 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.hash.Hasher; +import com.google.common.hash.Hashing; +import com.google.common.io.BaseEncoding; +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.guice.annotations.Deterministic; +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; + +/** + * Database-backed implementation of {@link CompactionStateManager}. + *

    + * 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 PersistedCompactionStateManager implements CompactionStateManager +{ + private static final EmittingLogger log = new EmittingLogger(PersistedCompactionStateManager.class); + private static final int DB_ACTION_PARTITION_SIZE = 100; + + private final MetadataStorageTablesConfig dbTables; + private final ObjectMapper jsonMapper; + private final ObjectMapper deterministicMapper; + private final SQLMetadataConnector connector; + private final CompactionStateManagerConfig config; + private final Cache fingerprintCache; + private final Striped datasourceLocks = Striped.lock(128); + + @Inject + public PersistedCompactionStateManager( + @Nonnull MetadataStorageTablesConfig dbTables, + @Nonnull ObjectMapper jsonMapper, + @Deterministic @Nonnull ObjectMapper deterministicMapper, + @Nonnull SQLMetadataConnector connector, + @Nonnull CompactionStateManagerConfig config + ) + { + this.dbTables = dbTables; + this.jsonMapper = jsonMapper; + this.deterministicMapper = deterministicMapper; + 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 + PersistedCompactionStateManager() + { + this.dbTables = null; + this.jsonMapper = null; + this.deterministicMapper = null; + this.connector = null; + this.config = null; + this.fingerprintCache = null; + } + + @Override + 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(); + } + } + + @Override + 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()); + } + + @Override + 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() + ); + } + + @Override + 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(); + } + ); + } + + @Override + 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()); + } + + @Override + @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; + } + } + + @Override + @SuppressWarnings("UnstableApiUsage") + public String generateCompactionStateFingerprint( + final CompactionState compactionState, + final String dataSource + ) + { + final Hasher hasher = Hashing.sha256().newHasher(); + + hasher.putBytes(StringUtils.toUtf8(dataSource)); + hasher.putByte((byte) 0xff); + + try { + hasher.putBytes(deterministicMapper.writeValueAsBytes(compactionState)); + } + catch (JsonProcessingException e) { + throw new RuntimeException("Failed to serialize CompactionState for fingerprinting", e); + } + hasher.putByte((byte) 0xff); + + return BaseEncoding.base16().encode(hasher.hash().asBytes()); + } + + + /** + * 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/server/compaction/CompactionRunSimulator.java b/server/src/main/java/org/apache/druid/server/compaction/CompactionRunSimulator.java index 60c97e4766a3..4f988a1c879c 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 @@ -145,7 +145,7 @@ public void onTaskSubmitted(String taskId, CompactionCandidate candidateSegments ); final CoordinatorRunStats stats = new CoordinatorRunStats(); - // Use an in-memory CompactionStateManager for simulations + // Use an in-memory PersistedCompactionStateManager for simulations final CompactionStateManager inMemoryStateManager = new HeapMemoryCompactionStateManager(); new CompactSegments(simulationStatusTracker, readOnlyOverlordClient, inMemoryStateManager).run( compactionConfig.withClusterConfig(configWithUnlimitedTaskSlots), diff --git a/server/src/test/java/org/apache/druid/segment/metadata/CompactionStateManagerConfigTest.java b/server/src/test/java/org/apache/druid/segment/metadata/PersistedCompactionStateManagerConfigTest.java similarity index 97% rename from server/src/test/java/org/apache/druid/segment/metadata/CompactionStateManagerConfigTest.java rename to server/src/test/java/org/apache/druid/segment/metadata/PersistedCompactionStateManagerConfigTest.java index edd7cac699b6..5106a48ad1d8 100644 --- a/server/src/test/java/org/apache/druid/segment/metadata/CompactionStateManagerConfigTest.java +++ b/server/src/test/java/org/apache/druid/segment/metadata/PersistedCompactionStateManagerConfigTest.java @@ -24,7 +24,7 @@ import org.junit.Assert; import org.junit.Test; -public class CompactionStateManagerConfigTest +public class PersistedCompactionStateManagerConfigTest { private final ObjectMapper mapper = new DefaultObjectMapper(); diff --git a/server/src/test/java/org/apache/druid/segment/metadata/CompactionStateManagerTest.java b/server/src/test/java/org/apache/druid/segment/metadata/PersistedCompactionStateManagerTest.java similarity index 98% rename from server/src/test/java/org/apache/druid/segment/metadata/CompactionStateManagerTest.java rename to server/src/test/java/org/apache/druid/segment/metadata/PersistedCompactionStateManagerTest.java index 145caa99355c..d631fae26cd6 100644 --- a/server/src/test/java/org/apache/druid/segment/metadata/CompactionStateManagerTest.java +++ b/server/src/test/java/org/apache/druid/segment/metadata/PersistedCompactionStateManagerTest.java @@ -54,7 +54,7 @@ import static org.junit.jupiter.api.Assertions.assertNull; import static org.junit.jupiter.api.Assertions.assertTrue; -public class CompactionStateManagerTest +public class PersistedCompactionStateManagerTest { @RegisterExtension public static final TestDerbyConnector.DerbyConnectorRule5 DERBY_CONNECTOR_RULE = @@ -65,7 +65,7 @@ public class CompactionStateManagerTest private static TestDerbyConnector derbyConnector; private static MetadataStorageTablesConfig tablesConfig; - private CompactionStateManager manager; + private PersistedCompactionStateManager manager; @BeforeAll public static void setUpClass() @@ -85,7 +85,7 @@ public void setUp() return null; }); - manager = new CompactionStateManager(tablesConfig, jsonMapper, deterministicMapper, derbyConnector, new CompactionStateManagerConfig()); + manager = new PersistedCompactionStateManager(tablesConfig, jsonMapper, deterministicMapper, derbyConnector, new CompactionStateManagerConfig()); } @Test @@ -291,7 +291,7 @@ public void test_prewarmCache_onModuleLifecycleStart() throws Exception }); // Create a NEW manager (not the shared one) - should prewarm cache in constructor - CompactionStateManager newManager = new CompactionStateManager( + PersistedCompactionStateManager newManager = new PersistedCompactionStateManager( tablesConfig, jsonMapper, deterministicMapper, 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 159536c2502c..858b8f67b250 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 @@ -30,6 +30,7 @@ import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; import org.apache.druid.indexer.partitions.HashedPartitionsSpec; import org.apache.druid.indexer.partitions.PartitionsSpec; +import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.granularity.Granularity; @@ -50,13 +51,13 @@ 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; import java.util.List; +import java.util.Map; public class CompactionStatusTest { @@ -570,14 +571,23 @@ public void test_evaluate_needsCompactionWhenAllSegmentsHaveUnexpectedCompaction DataSegment.builder(WIKI_SEGMENT).compactionStateFingerprint("wrongFingerprint").build(), DataSegment.builder(WIKI_SEGMENT_2).compactionStateFingerprint("wrongFingerprint").build() ); + + final DataSourceCompactionConfig oldCompactionConfig = InlineSchemaDataSourceCompactionConfig + .builder() + .forDataSource(TestDataSource.WIKI) + .withGranularitySpec(new UserCompactionTaskGranularityConfig(Granularities.HOUR, null, null)) + .build(); + CompactionState wrongState = CompactSegments.createCompactionStateFromConfig(oldCompactionConfig); + 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); + CompactionState expectedState = CompactSegments.createCompactionStateFromConfig(compactionConfig); + + compactionStateManager.persistCompactionState(TestDataSource.WIKI, Map.of("wrongFingerprint", wrongState), DateTimes.nowUtc()); verifyEvaluationNeedsCompactionBecauseWithCustomSegments( CompactionCandidate.from(segments, null), @@ -590,6 +600,13 @@ public void test_evaluate_needsCompactionWhenAllSegmentsHaveUnexpectedCompaction @Test public void test_evaluate_needsCompactionWhenSomeSegmentsHaveUnexpectedCompactionStateFingerprint() { + final DataSourceCompactionConfig oldCompactionConfig = InlineSchemaDataSourceCompactionConfig + .builder() + .forDataSource(TestDataSource.WIKI) + .withGranularitySpec(new UserCompactionTaskGranularityConfig(Granularities.HOUR, null, null)) + .build(); + CompactionState wrongState = CompactSegments.createCompactionStateFromConfig(oldCompactionConfig); + final DataSourceCompactionConfig compactionConfig = InlineSchemaDataSourceCompactionConfig .builder() .forDataSource(TestDataSource.WIKI) @@ -605,8 +622,7 @@ public void test_evaluate_needsCompactionWhenSomeSegmentsHaveUnexpectedCompactio DataSegment.builder(WIKI_SEGMENT_2).compactionStateFingerprint("wrongFingerprint").build() ); - EasyMock.expect(compactionStateManager.getCompactionStateByFingerprint("wrongFingerprint")).andReturn(createCompactionStateWithGranularity(Granularities.HOUR)); - EasyMock.replay(compactionStateManager); + compactionStateManager.persistCompactionState(TestDataSource.WIKI, Map.of("wrongFingerprint", wrongState), DateTimes.nowUtc()); verifyEvaluationNeedsCompactionBecauseWithCustomSegments( CompactionCandidate.from(segments, null), @@ -625,11 +641,11 @@ public void test_evaluate_noCompacationIfUnexpectedFingerprintHasExpectedCompact final DataSourceCompactionConfig compactionConfig = InlineSchemaDataSourceCompactionConfig .builder() .forDataSource(TestDataSource.WIKI) - .withGranularitySpec(new UserCompactionTaskGranularityConfig(Granularities.DAY, null, null)) + .withGranularitySpec(new UserCompactionTaskGranularityConfig(Granularities.HOUR, null, null)) .build(); - EasyMock.expect(compactionStateManager.getCompactionStateByFingerprint("wrongFingerprint")).andReturn(createCompactionStateWithGranularity(Granularities.DAY)); - EasyMock.replay(compactionStateManager); + CompactionState expectedState = CompactSegments.createCompactionStateFromConfig(compactionConfig); + compactionStateManager.persistCompactionState(TestDataSource.WIKI, Map.of("wrongFingerprint", expectedState), DateTimes.nowUtc()); final CompactionStatus status = CompactionStatus.compute( CompactionCandidate.from(segments, null), @@ -655,7 +671,7 @@ public void test_evaluate_needsCompactionWhenUnexpectedFingerprintAndNullCompact CompactionCandidate.from(segments, null), compactionConfig, "At least one segment has a mismatched fingerprint and needs compaction", - null + compactionStateManager ); } 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 391f5f66c784..b167c452b093 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,7 +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.segment.metadata.PersistedCompactionStateManager; import org.apache.druid.server.DruidNode; import org.apache.druid.server.compaction.CompactionSimulateResult; import org.apache.druid.server.compaction.CompactionStatusTracker; @@ -113,7 +113,7 @@ public class DruidCoordinatorTest private OverlordClient overlordClient; private CompactionStatusTracker statusTracker; private LatchableServiceEmitter serviceEmitter; - private CompactionStateManager compactionStateManager; + private PersistedCompactionStateManager persistedCompactionStateManager; @Before public void setUp() throws Exception @@ -124,7 +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); + persistedCompactionStateManager = EasyMock.createMock(PersistedCompactionStateManager.class); JacksonConfigManager configManager = EasyMock.createNiceMock(JacksonConfigManager.class); EasyMock.expect( @@ -174,7 +174,7 @@ public void setUp() throws Exception new CompactionStatusTracker(), EasyMock.niceMock(CoordinatorDynamicConfigSyncer.class), EasyMock.niceMock(CloneStatusManager.class), - compactionStateManager + persistedCompactionStateManager ); } @@ -189,7 +189,7 @@ private MetadataManager createMetadataManager(JacksonConfigManager configManager null, null, NoopSegmentMetadataCache.instance(), - compactionStateManager + persistedCompactionStateManager ); } @@ -488,7 +488,7 @@ public void testCompactSegmentsDutyWhenCustomDutyGroupEmpty() new CompactionStatusTracker(), EasyMock.niceMock(CoordinatorDynamicConfigSyncer.class), EasyMock.niceMock(CloneStatusManager.class), - compactionStateManager + persistedCompactionStateManager ); coordinator.start(); @@ -541,7 +541,7 @@ public void testInitializeCompactSegmentsDutyWhenCustomDutyGroupDoesNotContainsC new CompactionStatusTracker(), EasyMock.niceMock(CoordinatorDynamicConfigSyncer.class), EasyMock.niceMock(CloneStatusManager.class), - compactionStateManager + persistedCompactionStateManager ); coordinator.start(); // Since CompactSegments is not enabled in Custom Duty Group, then CompactSegments must be created in IndexingServiceDuties @@ -572,7 +572,7 @@ public void testInitializeCompactSegmentsDutyWhenCustomDutyGroupContainsCompactS CoordinatorCustomDutyGroup compactSegmentCustomGroup = new CoordinatorCustomDutyGroup( "group1", Duration.standardSeconds(1), - ImmutableList.of(new CompactSegments(statusTracker, null, compactionStateManager)) + ImmutableList.of(new CompactSegments(statusTracker, null, persistedCompactionStateManager)) ); CoordinatorCustomDutyGroups customDutyGroups = new CoordinatorCustomDutyGroups(ImmutableSet.of(compactSegmentCustomGroup)); coordinator = new DruidCoordinator( @@ -594,7 +594,7 @@ public void testInitializeCompactSegmentsDutyWhenCustomDutyGroupContainsCompactS new CompactionStatusTracker(), EasyMock.niceMock(CoordinatorDynamicConfigSyncer.class), EasyMock.niceMock(CloneStatusManager.class), - compactionStateManager + persistedCompactionStateManager ); coordinator.start(); @@ -705,7 +705,7 @@ public void testCoordinatorCustomDutyGroupsRunAsExpected() throws Exception new CompactionStatusTracker(), EasyMock.niceMock(CoordinatorDynamicConfigSyncer.class), EasyMock.niceMock(CloneStatusManager.class), - compactionStateManager + persistedCompactionStateManager ); coordinator.start(); 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 index b3921db0cbcb..25e6953a0c4c 100644 --- 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 @@ -30,6 +30,7 @@ import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.metadata.CompactionStateManager; import org.apache.druid.segment.metadata.CompactionStateManagerConfig; +import org.apache.druid.segment.metadata.PersistedCompactionStateManager; import org.apache.druid.server.coordinator.DruidCoordinatorRuntimeParams; import org.apache.druid.server.coordinator.config.MetadataCleanupConfig; import org.apache.druid.server.coordinator.stats.CoordinatorRunStats; @@ -72,10 +73,7 @@ public void setUp() derbyConnector.createCompactionStatesTable(); derbyConnector.createSegmentTable(); - compactionStateManager = new CompactionStateManager(tablesConfig, jsonMapper, createDeterministicMapper(), - derbyConnector, - new CompactionStateManagerConfig() - ); + compactionStateManager = new PersistedCompactionStateManager(tablesConfig, jsonMapper, createDeterministicMapper(), derbyConnector, new CompactionStateManagerConfig()); mockParams = EasyMock.createMock(DruidCoordinatorRuntimeParams.class); CoordinatorRunStats runStats = new CoordinatorRunStats(); 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 08b9af422f65..9e179b2567cf 100644 --- a/services/src/main/java/org/apache/druid/guice/MetadataManagerModule.java +++ b/services/src/main/java/org/apache/druid/guice/MetadataManagerModule.java @@ -44,6 +44,7 @@ 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.PersistedCompactionStateManager; import org.apache.druid.segment.metadata.SegmentSchemaCache; import org.apache.druid.server.coordinator.CoordinatorConfigManager; import org.apache.druid.server.coordinator.MetadataManager; @@ -62,7 +63,7 @@ *
  • {@link CoordinatorConfigManager}
  • *
  • {@link SegmentMetadataCache}
  • *
  • {@link SegmentSchemaCache} - Coordinator only
  • - *
  • {@link CompactionStateManager}
  • + *
  • {@link PersistedCompactionStateManager}
  • * */ public class MetadataManagerModule implements Module @@ -107,6 +108,7 @@ public void configure(Binder binder) JsonConfigProvider.bind(binder, "druid.manager.compactionState", CompactionStateManagerConfig.class); binder.bind(CompactionStateManager.class) + .to(PersistedCompactionStateManager.class) .in(ManageLifecycle.class); // Coordinator-only dependencies From 858cbd3f763d0ec442b8523d9ed163634a62caed Mon Sep 17 00:00:00 2001 From: capistrant Date: Mon, 22 Dec 2025 15:47:08 -0600 Subject: [PATCH 29/72] remove fingerprinting support from the coordinator compact segments duty --- .../compaction/CompactionRunSimulator.java | 2 +- .../server/compaction/CompactionStatus.java | 30 ++++++---- .../server/coordinator/DruidCoordinator.java | 2 +- .../coordinator/duty/CompactSegments.java | 36 ++---------- .../coordinator/DruidCoordinatorTest.java | 2 +- .../coordinator/duty/CompactSegmentsTest.java | 58 +++++++++---------- 6 files changed, 58 insertions(+), 72 deletions(-) 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 4f988a1c879c..8b88d807694a 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 @@ -147,7 +147,7 @@ public void onTaskSubmitted(String taskId, CompactionCandidate candidateSegments final CoordinatorRunStats stats = new CoordinatorRunStats(); // Use an in-memory PersistedCompactionStateManager for simulations final CompactionStateManager inMemoryStateManager = new HeapMemoryCompactionStateManager(); - new CompactSegments(simulationStatusTracker, readOnlyOverlordClient, inMemoryStateManager).run( + new CompactSegments(simulationStatusTracker, readOnlyOverlordClient).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 ea4ce2de6494..ae2cd57d990d 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 @@ -261,11 +261,19 @@ public static CompactionStatus running(String message) static CompactionStatus compute( CompactionCandidate candidateSegments, DataSourceCompactionConfig config, - CompactionStateManager compactionStateManager + @Nullable CompactionStateManager compactionStateManager ) { final CompactionState expectedState = CompactSegments.createCompactionStateFromConfig(config); - final String expectedFingerprint = compactionStateManager.generateCompactionStateFingerprint(expectedState, config.getDataSource()); + String expectedFingerprint; + if (compactionStateManager == null) { + expectedFingerprint = null; + } else { + expectedFingerprint = compactionStateManager.generateCompactionStateFingerprint( + expectedState, + config.getDataSource() + ); + } return new Evaluator(candidateSegments, config, expectedFingerprint, compactionStateManager).evaluate(); } @@ -357,7 +365,7 @@ private Evaluator( CompactionCandidate candidateSegments, DataSourceCompactionConfig compactionConfig, @Nullable String targetFingerprint, - CompactionStateManager compactionStateManager + @Nullable CompactionStateManager compactionStateManager ) { this.candidateSegments = candidateSegments; @@ -381,14 +389,16 @@ private CompactionStatus evaluate() 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 (compactionStateManager != null && targetFingerprint != null) { + // First try fingerprint-based evaluation (fast path) + CompactionStatus fingerprintStatus = FINGERPRINT_CHECKS.stream() + .map(f -> f.apply(this)) + .filter(status -> !status.isComplete()) + .findFirst().orElse(COMPLETE); - if (!fingerprintStatus.isComplete()) { - reasonsForCompaction.add(fingerprintStatus.getReason()); + if (!fingerprintStatus.isComplete()) { + reasonsForCompaction.add(fingerprintStatus.getReason()); + } } reasonsForCompaction.addAll( 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 20028b60e586..cb686fc01b4f 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 @@ -625,7 +625,7 @@ private CompactSegments initializeCompactSegmentsDuty(CompactionStatusTracker st { List compactSegmentsDutyFromCustomGroups = getCompactSegmentsDutyFromCustomGroups(); if (compactSegmentsDutyFromCustomGroups.isEmpty()) { - return new CompactSegments(statusTracker, overlordClient, compactionStateManager); + return new CompactSegments(statusTracker, overlordClient); } else { if (compactSegmentsDutyFromCustomGroups.size() > 1) { log.warn( 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 b4f1c59db4a2..6efa913e2357 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 @@ -39,14 +39,12 @@ 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; @@ -105,18 +103,14 @@ 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 CompactionStateManager compactionStateManager + @JacksonInject OverlordClient overlordClient ) { this.overlordClient = overlordClient; this.statusTracker = statusTracker; - this.compactionStateManager = compactionStateManager; resetCompactionSnapshot(); } @@ -197,7 +191,7 @@ public void run( compactionConfigs, dataSources.getUsedSegmentsTimelinesPerDataSource(), slotManager.getDatasourceIntervalsToSkipCompaction(), - compactionStateManager + null ); final CompactionSnapshotBuilder compactionSnapshotBuilder = new CompactionSnapshotBuilder(stats); @@ -207,8 +201,7 @@ public void run( slotManager, iterator, policy, - defaultEngine, - dynamicConfig.clusterConfig().isLegacyPersistLastCompactionStateInSegments() + defaultEngine ); stats.add(Stats.Compaction.SUBMITTED_TASKS, numSubmittedCompactionTasks); @@ -244,8 +237,7 @@ private int submitCompactionTasks( CompactionSlotManager slotManager, CompactionSegmentIterator iterator, CompactionCandidateSearchPolicy policy, - CompactionEngine defaultEngine, - boolean persistLastCompactionStateInSegments + CompactionEngine defaultEngine ) { if (slotManager.getNumAvailableTaskSlots() <= 0) { @@ -276,28 +268,12 @@ private int submitCompactionTasks( snapshotBuilder.addToComplete(entry); } - CompactionState compactionState = - createCompactionStateFromConfig(config); - - String compactionStateFingerprint = compactionStateManager.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 + null, + true ); final String taskId = taskPayload.getId(); 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 b167c452b093..d38f2d868872 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 @@ -572,7 +572,7 @@ public void testInitializeCompactSegmentsDutyWhenCustomDutyGroupContainsCompactS CoordinatorCustomDutyGroup compactSegmentCustomGroup = new CoordinatorCustomDutyGroup( "group1", Duration.standardSeconds(1), - ImmutableList.of(new CompactSegments(statusTracker, null, persistedCompactionStateManager)) + ImmutableList.of(new CompactSegments(statusTracker, null)) ); CoordinatorCustomDutyGroups customDutyGroups = new CoordinatorCustomDutyGroups(ImmutableSet.of(compactSegmentCustomGroup)); coordinator = new DruidCoordinator( 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 d39f3400b309..a2562240c028 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 @@ -281,7 +281,7 @@ public void testSerde() throws Exception .addValue(ExprMacroTable.class, TestExprMacroTable.INSTANCE) ); - final CompactSegments compactSegments = new CompactSegments(statusTracker, overlordClient, compactionStateManager); + final CompactSegments compactSegments = new CompactSegments(statusTracker, overlordClient); String compactSegmentString = JSON_MAPPER.writeValueAsString(compactSegments); CompactSegments serdeCompactSegments = JSON_MAPPER.readValue(compactSegmentString, CompactSegments.class); @@ -293,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, compactionStateManager); + final CompactSegments compactSegments = new CompactSegments(statusTracker, overlordClient); final Supplier expectedVersionSupplier = new Supplier<>() { @@ -376,7 +376,7 @@ public void testRun_withFixedIntervalOrderPolicy() public void testMakeStats() { final TestOverlordClient overlordClient = new TestOverlordClient(JSON_MAPPER); - final CompactSegments compactSegments = new CompactSegments(statusTracker, overlordClient, compactionStateManager); + final CompactSegments compactSegments = new CompactSegments(statusTracker, overlordClient); // Before any compaction, we do not have any snapshot of compactions Map autoCompactionSnapshots = compactSegments.getAutoCompactionSnapshot(); @@ -500,7 +500,7 @@ public void testMakeStatsForDataSourceWithCompactedIntervalBetweenNonCompactedIn dataSources = DataSourcesSnapshot.fromUsedSegments(segments); final TestOverlordClient overlordClient = new TestOverlordClient(JSON_MAPPER); - final CompactSegments compactSegments = new CompactSegments(statusTracker, overlordClient, compactionStateManager); + final CompactSegments compactSegments = new CompactSegments(statusTracker, overlordClient); // Before any compaction, we do not have any snapshot of compactions Map autoCompactionSnapshots = compactSegments.getAutoCompactionSnapshot(); @@ -562,7 +562,7 @@ public void testMakeStatsForDataSourceWithCompactedIntervalBetweenNonCompactedIn public void testMakeStatsWithDeactivatedDatasource() { final TestOverlordClient overlordClient = new TestOverlordClient(JSON_MAPPER); - final CompactSegments compactSegments = new CompactSegments(statusTracker, overlordClient, compactionStateManager); + final CompactSegments compactSegments = new CompactSegments(statusTracker, overlordClient); // Before any compaction, we do not have any snapshot of compactions Map autoCompactionSnapshots = compactSegments.getAutoCompactionSnapshot(); @@ -654,7 +654,7 @@ public void testMakeStatsForDataSourceWithSkipped() dataSources = DataSourcesSnapshot.fromUsedSegments(segments); final TestOverlordClient overlordClient = new TestOverlordClient(JSON_MAPPER); - final CompactSegments compactSegments = new CompactSegments(statusTracker, overlordClient, compactionStateManager); + final CompactSegments compactSegments = new CompactSegments(statusTracker, overlordClient); // Before any compaction, we do not have any snapshot of compactions Map autoCompactionSnapshots = compactSegments.getAutoCompactionSnapshot(); @@ -713,7 +713,7 @@ public void testMakeStatsForDataSourceWithSkipped() public void testRunMultipleCompactionTaskSlots() { final TestOverlordClient overlordClient = new TestOverlordClient(JSON_MAPPER); - final CompactSegments compactSegments = new CompactSegments(statusTracker, overlordClient, compactionStateManager); + final CompactSegments compactSegments = new CompactSegments(statusTracker, overlordClient); final CoordinatorRunStats stats = doCompactSegments(compactSegments, 3); Assert.assertEquals(3, stats.get(Stats.Compaction.AVAILABLE_SLOTS)); @@ -733,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, compactionStateManager); + final CompactSegments compactSegments = new CompactSegments(statusTracker, overlordClient); final CoordinatorRunStats stats = doCompactSegments(compactSegments, createCompactionConfigs(), maxCompactionSlot); Assert.assertEquals(maxCompactionSlot, stats.get(Stats.Compaction.AVAILABLE_SLOTS)); @@ -753,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, compactionStateManager); + final CompactSegments compactSegments = new CompactSegments(statusTracker, overlordClient); final CoordinatorRunStats stats = doCompactSegments(compactSegments, createCompactionConfigs(), maxCompactionSlot); Assert.assertEquals(MAXIMUM_CAPACITY_WITH_AUTO_SCALE, stats.get(Stats.Compaction.AVAILABLE_SLOTS)); @@ -775,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, compactionStateManager); + final CompactSegments compactSegments = new CompactSegments(statusTracker, mockClient); final List compactionConfigs = new ArrayList<>(); final String dataSource = DATA_SOURCE_PREFIX + 0; compactionConfigs.add( @@ -806,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, compactionStateManager); + final CompactSegments compactSegments = new CompactSegments(statusTracker, mockClient); final List compactionConfigs = new ArrayList<>(); final String dataSource = DATA_SOURCE_PREFIX + 0; compactionConfigs.add( @@ -830,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, compactionStateManager); + final CompactSegments compactSegments = new CompactSegments(statusTracker, mockClient); final List compactionConfigs = new ArrayList<>(); final String dataSource = DATA_SOURCE_PREFIX + 0; compactionConfigs.add( @@ -853,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, compactionStateManager); + final CompactSegments compactSegments = new CompactSegments(statusTracker, mockClient); final List compactionConfigs = new ArrayList<>(); final String dataSource = DATA_SOURCE_PREFIX + 0; compactionConfigs.add( @@ -890,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, compactionStateManager); + final CompactSegments compactSegments = new CompactSegments(statusTracker, mockClient); final List compactionConfigs = new ArrayList<>(); final String dataSource = DATA_SOURCE_PREFIX + 0; compactionConfigs.add( @@ -921,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, compactionStateManager); + final CompactSegments compactSegments = new CompactSegments(statusTracker, mockClient); final List compactionConfigs = new ArrayList<>(); final String dataSource = DATA_SOURCE_PREFIX + 0; compactionConfigs.add( @@ -944,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, compactionStateManager); + final CompactSegments compactSegments = new CompactSegments(statusTracker, mockClient); final List compactionConfigs = new ArrayList<>(); final String dataSource = DATA_SOURCE_PREFIX + 0; final List projections = List.of( @@ -1019,7 +1019,7 @@ public void testCompactWithCatalogProjections() ) .buildSpec() ); - final CompactSegments compactSegments = new CompactSegments(statusTracker, mockClient, compactionStateManager); + final CompactSegments compactSegments = new CompactSegments(statusTracker, mockClient); final List compactionConfigs = new ArrayList<>(); compactionConfigs.add( new CatalogDataSourceCompactionConfig( @@ -1045,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, compactionStateManager); + final CompactSegments compactSegments = new CompactSegments(statusTracker, mockClient); final List compactionConfigs = new ArrayList<>(); final String dataSource = DATA_SOURCE_PREFIX + 0; compactionConfigs.add( @@ -1136,7 +1136,7 @@ public void testCompactWithGranularitySpecConflictWithActiveCompactionTask() Mockito.when(mockClient.taskPayload(ArgumentMatchers.eq(conflictTaskId))) .thenReturn(Futures.immediateFuture(runningConflictCompactionTaskPayload)); - final CompactSegments compactSegments = new CompactSegments(statusTracker, mockClient, compactionStateManager); + final CompactSegments compactSegments = new CompactSegments(statusTracker, mockClient); final List compactionConfigs = new ArrayList<>(); compactionConfigs.add( InlineSchemaDataSourceCompactionConfig.builder() @@ -1175,7 +1175,7 @@ public void testCompactWithGranularitySpecConflictWithActiveCompactionTask() public void testIntervalIsCompactedAgainWhenSegmentIsAdded() { final TestOverlordClient overlordClient = new TestOverlordClient(JSON_MAPPER); - final CompactSegments compactSegments = new CompactSegments(statusTracker, overlordClient, compactionStateManager); + final CompactSegments compactSegments = new CompactSegments(statusTracker, overlordClient); final String dataSource = DATA_SOURCE_PREFIX + 0; final DataSourceCompactionConfig compactionConfig = InlineSchemaDataSourceCompactionConfig @@ -1221,7 +1221,7 @@ public void testIntervalIsCompactedAgainWhenSegmentIsAdded() public void testRunParallelCompactionMultipleCompactionTaskSlots() { final TestOverlordClient overlordClient = new TestOverlordClient(JSON_MAPPER); - final CompactSegments compactSegments = new CompactSegments(statusTracker, overlordClient, compactionStateManager); + final CompactSegments compactSegments = new CompactSegments(statusTracker, overlordClient); final CoordinatorRunStats stats; // Native uses maxNumConcurrentSubTasks for task slots whereas MSQ uses maxNumTasks. if (engine == CompactionEngine.NATIVE) { @@ -1258,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, compactionStateManager); + CompactSegments compactSegments = new CompactSegments(statusTracker, overlordClient); final CoordinatorRunStats stats = doCompactSegments(compactSegments, createcompactionConfigsForNative(2), 4); Assert.assertEquals(1, stats.get(Stats.Compaction.SUBMITTED_TASKS)); @@ -1277,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, compactionStateManager); + final CompactSegments compactSegments = new CompactSegments(statusTracker, mockClient); final List compactionConfigs = new ArrayList<>(); final String dataSource = DATA_SOURCE_PREFIX + 0; compactionConfigs.add( @@ -1306,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, compactionStateManager); + final CompactSegments compactSegments = new CompactSegments(statusTracker, mockClient); final List compactionConfigs = new ArrayList<>(); final String dataSource = DATA_SOURCE_PREFIX + 0; compactionConfigs.add( @@ -1331,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, compactionStateManager); + final CompactSegments compactSegments = new CompactSegments(statusTracker, mockClient); final List compactionConfigs = new ArrayList<>(); final String dataSource = DATA_SOURCE_PREFIX + 0; compactionConfigs.add( @@ -1387,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, compactionStateManager); + final CompactSegments compactSegments = new CompactSegments(statusTracker, mockClient); final List compactionConfigs = new ArrayList<>(); compactionConfigs.add( InlineSchemaDataSourceCompactionConfig.builder() @@ -1447,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, compactionStateManager); + final CompactSegments compactSegments = new CompactSegments(statusTracker, mockClient); final List compactionConfigs = new ArrayList<>(); compactionConfigs.add( InlineSchemaDataSourceCompactionConfig.builder() @@ -1480,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, compactionStateManager); + final CompactSegments compactSegments = new CompactSegments(statusTracker, mockClient); final List compactionConfigs = new ArrayList<>(); final String dataSource = DATA_SOURCE_PREFIX + 0; compactionConfigs.add( @@ -1507,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, compactionStateManager); + final CompactSegments compactSegments = new CompactSegments(statusTracker, mockClient); final List compactionConfigs = new ArrayList<>(); final String dataSource = DATA_SOURCE_PREFIX + 0; compactionConfigs.add( From 9afab2f02ea7dfae78fc5ad12aa4b5b4ad7765eb Mon Sep 17 00:00:00 2001 From: capistrant Date: Mon, 22 Dec 2025 16:40:38 -0600 Subject: [PATCH 30/72] Move on heap compaction state manager to test sources --- .../druid/server/compaction/CompactionRunSimulator.java | 4 ---- .../segment/metadata/HeapMemoryCompactionStateManager.java | 0 2 files changed, 4 deletions(-) rename server/src/{main => test}/java/org/apache/druid/segment/metadata/HeapMemoryCompactionStateManager.java (100%) 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 8b88d807694a..80abb0a85112 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,8 +32,6 @@ 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; @@ -145,8 +143,6 @@ public void onTaskSubmitted(String taskId, CompactionCandidate candidateSegments ); final CoordinatorRunStats stats = new CoordinatorRunStats(); - // Use an in-memory PersistedCompactionStateManager for simulations - final CompactionStateManager inMemoryStateManager = new HeapMemoryCompactionStateManager(); new CompactSegments(simulationStatusTracker, readOnlyOverlordClient).run( compactionConfig.withClusterConfig(configWithUnlimitedTaskSlots), dataSourcesSnapshot, diff --git a/server/src/main/java/org/apache/druid/segment/metadata/HeapMemoryCompactionStateManager.java b/server/src/test/java/org/apache/druid/segment/metadata/HeapMemoryCompactionStateManager.java similarity index 100% rename from server/src/main/java/org/apache/druid/segment/metadata/HeapMemoryCompactionStateManager.java rename to server/src/test/java/org/apache/druid/segment/metadata/HeapMemoryCompactionStateManager.java From 34a8a11785d54bfd65f3cb96131483e5dae2b313 Mon Sep 17 00:00:00 2001 From: capistrant Date: Mon, 22 Dec 2025 16:56:41 -0600 Subject: [PATCH 31/72] CompactionStateManager is now overlord only --- .../org/apache/druid/guice/MetadataManagerModule.java | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) 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 9e179b2567cf..c5ea7050b6f6 100644 --- a/services/src/main/java/org/apache/druid/guice/MetadataManagerModule.java +++ b/services/src/main/java/org/apache/druid/guice/MetadataManagerModule.java @@ -106,11 +106,6 @@ public void configure(Binder binder) .to(HeapMemorySegmentMetadataCache.class) .in(LazySingleton.class); - JsonConfigProvider.bind(binder, "druid.manager.compactionState", CompactionStateManagerConfig.class); - binder.bind(CompactionStateManager.class) - .to(PersistedCompactionStateManager.class) - .in(ManageLifecycle.class); - // Coordinator-only dependencies if (nodeRoles.contains(NodeRole.COORDINATOR)) { JsonConfigProvider.bind(binder, "druid.manager.rules", MetadataRuleManagerConfig.class); @@ -137,6 +132,11 @@ public void configure(Binder binder) binder.bind(SegmentMetadataTransactionFactory.class) .to(SqlSegmentMetadataTransactionFactory.class) .in(LazySingleton.class); + + JsonConfigProvider.bind(binder, "druid.manager.compactionState", CompactionStateManagerConfig.class); + binder.bind(CompactionStateManager.class) + .to(PersistedCompactionStateManager.class) + .in(ManageLifecycle.class); } else { binder.bind(SegmentMetadataTransactionFactory.class) .to(SqlSegmentMetadataReadOnlyTransactionFactory.class) From 7214418c9d99ba3a63629ec7fa11aa5ef85b590e Mon Sep 17 00:00:00 2001 From: capistrant Date: Fri, 2 Jan 2026 14:48:28 -0600 Subject: [PATCH 32/72] Refactor how the compaction state fingerprint cache is wired up --- .../NewestSegmentFirstPolicyBenchmark.java | 4 +- docs/configuration/index.md | 2 - .../CompactionConfigBasedJobTemplate.java | 3 +- .../indexing/compact/CompactionJobParams.java | 11 +- .../indexing/compact/CompactionJobQueue.java | 7 +- .../compact/OverlordCompactionScheduler.java | 9 +- .../common/actions/TaskActionTestKit.java | 3 + .../common/task/IngestionTestBase.java | 5 + .../OverlordCompactionSchedulerTest.java | 4 +- .../metadata/SqlSegmentsMetadataQuery.java | 119 +++++++++ .../segment/cache/CompactionStateRecord.java | 72 ++++++ .../cache/HeapMemorySegmentMetadataCache.java | 98 +++++++ .../druid/metadata/segment/cache/Metric.java | 10 + .../metadata/CompactionStateCache.java | 147 +++++++++++ .../metadata/CompactionStateManager.java | 10 - .../CompactionStateManagerConfig.java | 47 ---- .../apache/druid/segment/metadata/Metric.java | 8 +- .../metadata/NoopCompactionStateCache.java | 69 +++++ .../PersistedCompactionStateManager.java | 165 +----------- .../server/compaction/CompactionStatus.java | 18 +- .../DataSourceCompactibleSegmentIterator.java | 8 +- ...riorityBasedCompactionSegmentIterator.java | 7 +- .../server/coordinator/DruidCoordinator.java | 6 +- .../coordinator/duty/CompactSegments.java | 1 + ...etadataStorageCoordinatorReadOnlyTest.java | 2 + ...exerSQLMetadataStorageCoordinatorTest.java | 2 + .../SqlSegmentsMetadataManagerV2Test.java | 3 + .../HeapMemorySegmentMetadataCacheTest.java | 7 +- .../metadata/CompactionStateCacheTest.java | 239 ++++++++++++++++++ .../HeapMemoryCompactionStateManager.java | 19 +- ...istedCompactionStateManagerConfigTest.java | 50 ---- .../PersistedCompactionStateManagerTest.java | 100 +------- .../compaction/CompactionStatusTest.java | 49 +++- .../NewestSegmentFirstPolicyTest.java | 18 +- .../coordinator/DruidCoordinatorTest.java | 15 +- .../KillUnreferencedCompactionStateTest.java | 3 +- .../CoordinatorSimulationBuilder.java | 3 +- .../druid/guice/MetadataManagerModule.java | 21 +- 38 files changed, 931 insertions(+), 433 deletions(-) create mode 100644 server/src/main/java/org/apache/druid/metadata/segment/cache/CompactionStateRecord.java create mode 100644 server/src/main/java/org/apache/druid/segment/metadata/CompactionStateCache.java delete mode 100644 server/src/main/java/org/apache/druid/segment/metadata/CompactionStateManagerConfig.java create mode 100644 server/src/main/java/org/apache/druid/segment/metadata/NoopCompactionStateCache.java create mode 100644 server/src/test/java/org/apache/druid/segment/metadata/CompactionStateCacheTest.java delete mode 100644 server/src/test/java/org/apache/druid/segment/metadata/PersistedCompactionStateManagerConfigTest.java diff --git a/benchmarks/src/test/java/org/apache/druid/server/coordinator/NewestSegmentFirstPolicyBenchmark.java b/benchmarks/src/test/java/org/apache/druid/server/coordinator/NewestSegmentFirstPolicyBenchmark.java index 89adac0ce09f..2565cacdc282 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 @@ -23,6 +23,7 @@ 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.segment.metadata.NoopCompactionStateCache; import org.apache.druid.server.compaction.CompactionCandidateSearchPolicy; import org.apache.druid.server.compaction.CompactionSegmentIterator; import org.apache.druid.server.compaction.NewestSegmentFirstPolicy; @@ -137,7 +138,8 @@ public void measureNewestSegmentFirstPolicy(Blackhole blackhole) compactionConfigs, dataSources, Collections.emptyMap(), - new HeapMemoryCompactionStateManager() + new HeapMemoryCompactionStateManager(), + new NoopCompactionStateCache() ); for (int i = 0; i < numCompactionTaskSlots && iterator.hasNext(); i++) { blackhole.consume(iterator.next()); diff --git a/docs/configuration/index.md b/docs/configuration/index.md index 071ebe18872e..6296c4055483 100644 --- a/docs/configuration/index.md +++ b/docs/configuration/index.md @@ -811,8 +811,6 @@ 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/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 c209b90418e0..127831b046cd 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 @@ -136,7 +136,8 @@ DataSourceCompactibleSegmentIterator getCompactibleCandidates( // This policy is used only while creating jobs // The actual order of jobs is determined by the policy used in CompactionJobQueue new NewestSegmentFirstPolicy(null), - params.getCompactionStateManager() + params.getCompactionStateManager(), + params.getCompactionStateCache() ); // 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 5b82633b0c85..f9cbe64e7448 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.CompactionStateCache; import org.apache.druid.segment.metadata.CompactionStateManager; import org.apache.druid.server.compaction.CompactionSnapshotBuilder; import org.apache.druid.server.coordinator.ClusterCompactionConfig; @@ -35,13 +36,15 @@ public class CompactionJobParams private final ClusterCompactionConfig clusterCompactionConfig; private final CompactionSnapshotBuilder snapshotBuilder; private final CompactionStateManager compactionStateManager; + private final CompactionStateCache compactionStateCache; public CompactionJobParams( DateTime scheduleStartTime, ClusterCompactionConfig clusterCompactionConfig, TimelineProvider timelineProvider, CompactionSnapshotBuilder snapshotBuilder, - CompactionStateManager compactionStateManager + CompactionStateManager compactionStateManager, + CompactionStateCache compactionStateCache ) { this.scheduleStartTime = scheduleStartTime; @@ -49,6 +52,7 @@ public CompactionJobParams( this.timelineProvider = timelineProvider; this.snapshotBuilder = snapshotBuilder; this.compactionStateManager = compactionStateManager; + this.compactionStateCache = compactionStateCache; } /** @@ -97,6 +101,11 @@ public CompactionStateManager getCompactionStateManager() return compactionStateManager; } + public CompactionStateCache getCompactionStateCache() + { + return compactionStateCache; + } + @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 2d11491f69b6..962eece9ad21 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.CompactionStateCache; import org.apache.druid.segment.metadata.CompactionStateManager; import org.apache.druid.server.compaction.CompactionCandidate; import org.apache.druid.server.compaction.CompactionCandidateSearchPolicy; @@ -106,7 +107,8 @@ public CompactionJobQueue( OverlordClient overlordClient, BrokerClient brokerClient, ObjectMapper objectMapper, - CompactionStateManager compactionStateManager + CompactionStateManager compactionStateManager, + CompactionStateCache compactionStateCache ) { this.runStats = new CoordinatorRunStats(); @@ -123,7 +125,8 @@ public CompactionJobQueue( clusterCompactionConfig, dataSourcesSnapshot.getUsedSegmentsTimelinesPerDataSource()::get, snapshotBuilder, - compactionStateManager + compactionStateManager, + compactionStateCache ); 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 7e8caa3b8806..c42c97611ba6 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.CompactionStateCache; import org.apache.druid.segment.metadata.CompactionStateManager; import org.apache.druid.server.compaction.CompactionRunSimulator; import org.apache.druid.server.compaction.CompactionSimulateResult; @@ -141,6 +142,7 @@ public class OverlordCompactionScheduler implements CompactionScheduler private final long schedulePeriodMillis; private final CompactionStateManager compactionStateManager; + private final CompactionStateCache compactionStateCache; @Inject public OverlordCompactionScheduler( @@ -158,7 +160,8 @@ public OverlordCompactionScheduler( BrokerClient brokerClient, ServiceEmitter emitter, ObjectMapper objectMapper, - CompactionStateManager compactionStateManager + CompactionStateManager compactionStateManager, + CompactionStateCache compactionStateCache ) { final long segmentPollPeriodMillis = @@ -185,6 +188,7 @@ public OverlordCompactionScheduler( this.taskActionClientFactory = taskActionClientFactory; this.druidInputSourceFactory = druidInputSourceFactory; this.compactionStateManager = compactionStateManager; + this.compactionStateCache = compactionStateCache; this.taskRunnerListener = new TaskRunnerListener() { @Override @@ -372,7 +376,8 @@ private synchronized void resetCompactionJobQueue() overlordClient, brokerClient, objectMapper, - compactionStateManager + compactionStateManager, + compactionStateCache ); latestJobQueue.set(queue); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/TaskActionTestKit.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/TaskActionTestKit.java index b7c47a60a7dd..58d16d964bbf 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/TaskActionTestKit.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/TaskActionTestKit.java @@ -41,6 +41,7 @@ import org.apache.druid.metadata.segment.cache.HeapMemorySegmentMetadataCache; import org.apache.druid.metadata.segment.cache.SegmentMetadataCache; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; +import org.apache.druid.segment.metadata.CompactionStateCache; import org.apache.druid.segment.metadata.NoopSegmentSchemaCache; import org.apache.druid.segment.metadata.SegmentSchemaManager; import org.apache.druid.server.coordinator.simulate.BlockingExecutorService; @@ -176,6 +177,7 @@ public boolean isBatchAllocationReduceMetadataIO() testDerbyConnector.createConfigTable(); testDerbyConnector.createTaskTables(); testDerbyConnector.createAuditTable(); + testDerbyConnector.createCompactionStatesTable(); segmentMetadataCache.start(); segmentMetadataCache.becomeLeader(); @@ -198,6 +200,7 @@ private SqlSegmentMetadataTransactionFactory setupTransactionFactory( Suppliers.ofInstance(new SegmentsMetadataManagerConfig(Period.seconds(1), cacheMode, null)), Suppliers.ofInstance(metadataStorageTablesConfig), new NoopSegmentSchemaCache(), + new CompactionStateCache(), testDerbyConnector, (poolSize, name) -> new WrappingScheduledExecutorService(name, metadataCachePollExec, false), emitter diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IngestionTestBase.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IngestionTestBase.java index acdff02ba805..fc656bda3f67 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IngestionTestBase.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IngestionTestBase.java @@ -87,6 +87,7 @@ import org.apache.druid.segment.loading.LocalDataSegmentPusherConfig; import org.apache.druid.segment.loading.SegmentCacheManager; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; +import org.apache.druid.segment.metadata.CompactionStateCache; import org.apache.druid.segment.metadata.SegmentSchemaCache; import org.apache.druid.segment.metadata.SegmentSchemaManager; import org.apache.druid.segment.realtime.NoopChatHandlerProvider; @@ -139,6 +140,7 @@ public abstract class IngestionTestBase extends InitializedNullHandlingTest private TestDataSegmentKiller dataSegmentKiller; private SegmentMetadataCache segmentMetadataCache; private SegmentSchemaCache segmentSchemaCache; + private CompactionStateCache compactionStateCache; protected File reportsFile; protected IngestionTestBase() @@ -164,6 +166,7 @@ public void setUpIngestionTestBase() throws IOException connector.createSegmentSchemasTable(); connector.createSegmentTable(); connector.createPendingSegmentsTable(); + connector.createCompactionStatesTable(); taskStorage = new HeapMemoryTaskStorage(new TaskStorageConfig(null)); SegmentSchemaManager segmentSchemaManager = new SegmentSchemaManager( derbyConnectorRule.metadataTablesConfigSupplier().get(), @@ -172,6 +175,7 @@ public void setUpIngestionTestBase() throws IOException ); segmentSchemaCache = new SegmentSchemaCache(); + compactionStateCache = new CompactionStateCache(); storageCoordinator = new IndexerSQLMetadataStorageCoordinator( createTransactionFactory(), objectMapper, @@ -337,6 +341,7 @@ private SqlSegmentMetadataTransactionFactory createTransactionFactory() Suppliers.ofInstance(new SegmentsMetadataManagerConfig(Period.millis(10), cacheMode, null)), derbyConnectorRule.metadataTablesConfigSupplier(), segmentSchemaCache, + compactionStateCache, derbyConnectorRule.getConnector(), ScheduledExecutors::fixed, NoopServiceEmitter.instance() 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 82a8f5db3387..75f151e33588 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.CompactionStateCache; import org.apache.druid.segment.metadata.HeapMemoryCompactionStateManager; import org.apache.druid.server.compaction.CompactionSimulateResult; import org.apache.druid.server.compaction.CompactionStatistics; @@ -233,7 +234,8 @@ private void initScheduler() brokerClient, serviceEmitter, OBJECT_MAPPER, - new HeapMemoryCompactionStateManager() + new HeapMemoryCompactionStateManager(), + new CompactionStateCache() ); } 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 6a05bbc41cb0..09a4501a070f 100644 --- a/server/src/main/java/org/apache/druid/metadata/SqlSegmentsMetadataQuery.java +++ b/server/src/main/java/org/apache/druid/metadata/SqlSegmentsMetadataQuery.java @@ -40,11 +40,13 @@ import org.apache.druid.java.util.common.jackson.JacksonUtils; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.java.util.common.parsers.CloseableIterator; +import org.apache.druid.metadata.segment.cache.CompactionStateRecord; import org.apache.druid.metadata.segment.cache.SegmentSchemaRecord; import org.apache.druid.segment.SchemaPayload; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; import org.apache.druid.server.http.DataSegmentPlus; +import org.apache.druid.timeline.CompactionState; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.SegmentId; import org.apache.druid.timeline.SegmentTimeline; @@ -1703,6 +1705,123 @@ private SegmentSchemaRecord mapToSchemaRecord(ResultSet resultSet) } } + /** + * Retrieves all unique compaction state fingerprints currently referenced by used segments. + * This is used for delta syncs to determine which fingerprints are still active. + * + * @return Set of compaction state fingerprints + */ + public Set retrieveAllUsedCompactionStateFingerprints() + { + final String sql = StringUtils.format( + "SELECT DISTINCT compaction_state_fingerprint FROM %s " + + "WHERE used = true AND compaction_state_fingerprint IS NOT NULL", + dbTables.getSegmentsTable() + ); + + return Set.copyOf( + handle.createQuery(sql) + .setFetchSize(connector.getStreamingFetchSize()) + .mapTo(String.class) + .list() + ); + } + + /** + * Retrieves all compaction states for used segments (full sync). + * Fetches from compaction_states table where the fingerprint is referenced by used segments. + * + * @return List of CompactionStateRecord objects + */ + public List retrieveAllUsedCompactionStates() + { + final String sql = StringUtils.format( + "SELECT cs.fingerprint, cs.payload FROM %s cs " + + "WHERE cs.used = true " + + "AND cs.fingerprint IN (" + + " SELECT DISTINCT compaction_state_fingerprint FROM %s " + + " WHERE used = true AND compaction_state_fingerprint IS NOT NULL" + + ")", + dbTables.getCompactionStatesTable(), + dbTables.getSegmentsTable() + ); + + return retrieveValidCompactionStateRecordsWithQuery(handle.createQuery(sql)); + } + + /** + * Retrieves compaction states for specific fingerprints (delta sync). + * Used to fetch only newly added compaction states. + * + * @param fingerprints Set of fingerprints to retrieve + * @return List of CompactionStateRecord objects + */ + public List retrieveCompactionStatesForFingerprints( + Set fingerprints + ) + { + final List> fingerprintBatches = Lists.partition( + List.copyOf(fingerprints), + MAX_INTERVALS_PER_BATCH + ); + + final List records = new ArrayList<>(); + for (List fingerprintBatch : fingerprintBatches) { + records.addAll( + retrieveBatchOfCompactionStates(fingerprintBatch) + ); + } + + return records; + } + + /** + * Retrieves a batch of compaction state records for the given fingerprints. + */ + private List retrieveBatchOfCompactionStates(List fingerprints) + { + final String sql = StringUtils.format( + "SELECT fingerprint, payload FROM %s" + + " WHERE used = true" + + " %s", + dbTables.getCompactionStatesTable(), + getParameterizedInConditionForColumn("fingerprint", fingerprints) + ); + + final Query> query = handle.createQuery(sql); + bindColumnValuesToQueryWithInCondition("fingerprint", fingerprints, query); + + return retrieveValidCompactionStateRecordsWithQuery(query); + } + + private List retrieveValidCompactionStateRecordsWithQuery( + Query> query + ) + { + return query.setFetchSize(connector.getStreamingFetchSize()) + .map((index, r, ctx) -> mapToCompactionStateRecord(r)) + .list() + .stream() + .filter(Objects::nonNull) + .collect(Collectors.toList()); + } + + private CompactionStateRecord mapToCompactionStateRecord(ResultSet resultSet) + { + String fingerprint = null; + try { + fingerprint = resultSet.getString("fingerprint"); + return new CompactionStateRecord( + fingerprint, + jsonMapper.readValue(resultSet.getBytes("payload"), CompactionState.class) + ); + } + catch (Throwable t) { + log.error(t, "Could not read compaction state with fingerprint[%s]", fingerprint); + return null; + } + } + private ResultIterator getDataSegmentResultIterator(Query> sql) { return sql.map((index, r, ctx) -> JacksonUtils.readValue(jsonMapper, r.getBytes(2), DataSegment.class)) diff --git a/server/src/main/java/org/apache/druid/metadata/segment/cache/CompactionStateRecord.java b/server/src/main/java/org/apache/druid/metadata/segment/cache/CompactionStateRecord.java new file mode 100644 index 000000000000..bb70fc76602a --- /dev/null +++ b/server/src/main/java/org/apache/druid/metadata/segment/cache/CompactionStateRecord.java @@ -0,0 +1,72 @@ +/* + * 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.metadata.segment.cache; + +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.timeline.CompactionState; + +import java.sql.ResultSet; + +/** + * Represents a single record in the compaction_states table. + */ +public class CompactionStateRecord +{ + private final String fingerprint; + private final CompactionState state; + + public CompactionStateRecord(String fingerprint, CompactionState state) + { + this.fingerprint = fingerprint; + this.state = state; + } + + public String getFingerprint() + { + return fingerprint; + } + + public CompactionState getState() + { + return state; + } + + /** + * Creates a CompactionStateRecord from a ResultSet row. + * Expected columns: fingerprint (String), payload (byte[]) + * + * @param rs ResultSet positioned at the row to read + * @param jsonMapper ObjectMapper for deserializing the payload + * @return CompactionStateRecord or null if deserialization fails + */ + public static CompactionStateRecord fromResultSet(ResultSet rs, ObjectMapper jsonMapper) + throws Exception + { + String fingerprint = rs.getString("fingerprint"); + byte[] payload = rs.getBytes("payload"); + + if (fingerprint == null || payload == null) { + return null; + } + + CompactionState state = jsonMapper.readValue(payload, CompactionState.class); + return new CompactionStateRecord(fingerprint, state); + } +} 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 97d834da4322..6e33edf16ef1 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 @@ -51,8 +51,10 @@ import org.apache.druid.query.DruidMetrics; import org.apache.druid.segment.SchemaPayload; import org.apache.druid.segment.SegmentMetadata; +import org.apache.druid.segment.metadata.CompactionStateCache; import org.apache.druid.segment.metadata.SegmentSchemaCache; import org.apache.druid.server.http.DataSegmentPlus; +import org.apache.druid.timeline.CompactionState; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.SegmentId; import org.joda.time.DateTime; @@ -137,6 +139,9 @@ private enum CacheState private final boolean useSchemaCache; private final SegmentSchemaCache segmentSchemaCache; + private final boolean useCompactionStateCache; + private final CompactionStateCache compactionStateCache; + private final ListeningScheduledExecutorService pollExecutor; private final ServiceEmitter emitter; @@ -168,6 +173,7 @@ public HeapMemorySegmentMetadataCache( Supplier config, Supplier tablesConfig, SegmentSchemaCache segmentSchemaCache, + CompactionStateCache compactionStateCache, SQLMetadataConnector connector, ScheduledExecutorFactory executorFactory, ServiceEmitter emitter @@ -179,6 +185,8 @@ public HeapMemorySegmentMetadataCache( this.tablesConfig = tablesConfig.get(); this.useSchemaCache = segmentSchemaCache.isEnabled(); this.segmentSchemaCache = segmentSchemaCache; + this.useCompactionStateCache = compactionStateCache.isEnabled(); + this.compactionStateCache = compactionStateCache; this.connector = connector; this.pollExecutor = isEnabled() ? MoreExecutors.listeningDecorator(executorFactory.create(1, "SegmentMetadataCache-%s")) @@ -265,6 +273,9 @@ public void stopBeingLeader() { synchronized (cacheStateLock) { if (isEnabled()) { + if (useCompactionStateCache) { + compactionStateCache.clear(); + } updateCacheState(CacheState.FOLLOWER, "Not leader anymore"); } } @@ -576,6 +587,10 @@ private long syncWithMetadataStore() retrieveAndResetUsedSegmentSchemas(datasourceToSummary); } + if (useCompactionStateCache) { + retrieveAndResetUsedCompactionStates(); + } + markCacheSynced(syncStartTime); syncFinishTime.set(DateTimes.nowUtc()); @@ -1107,6 +1122,89 @@ private void emitMetric(String datasource, String metric, long value) ); } + /** + * Retrieves required used compaction states from the metadata store and resets + * them in the {@link CompactionStateCache}. If this is the first sync, all used + * compaction states are retrieved from the metadata store. If this is a delta sync, + * first only the fingerprints of all used compaction states are retrieved. Payloads are + * then fetched for only the fingerprints which are not present in the cache. + */ + private void retrieveAndResetUsedCompactionStates() + { + final Stopwatch compactionStateSyncDuration = Stopwatch.createStarted(); + + // Reset the CompactionStateCache with latest compaction states + final Map fingerprintToStateMap; + if (syncFinishTime.get() == null) { + fingerprintToStateMap = buildFingerprintToStateMapForFullSync(); + } else { + fingerprintToStateMap = buildFingerprintToStateMapForDeltaSync(); + } + + compactionStateCache.resetCompactionStatesForPublishedSegments(fingerprintToStateMap); + + // Emit metrics for the current contents of the cache + compactionStateCache.getStats().forEach(this::emitMetric); + emitMetric(Metric.RETRIEVE_COMPACTION_STATES_DURATION_MILLIS, compactionStateSyncDuration.millisElapsed()); + } + + /** + * Retrieves all used compaction states from the metadata store and builds a + * fresh map from compaction state fingerprint to state. + */ + private Map buildFingerprintToStateMapForFullSync() + { + final List records = query( + SqlSegmentsMetadataQuery::retrieveAllUsedCompactionStates + ); + + return records.stream().collect( + Collectors.toMap( + CompactionStateRecord::getFingerprint, + CompactionStateRecord::getState + ) + ); + } + + /** + * Retrieves compaction states from the metadata store if they are not present + * in the cache or have been recently updated in the metadata store. These + * compaction states along with those already present in the cache are used to + * build a complete updated map from compaction state fingerprint to state. + * + * @return Complete updated map from compaction state fingerprint to state for all + * used compaction states currently persisted in the metadata store. + */ + private Map buildFingerprintToStateMapForDeltaSync() + { + // Identify fingerprints in the cache and in the metadata store + final Map fingerprintToStateMap = new HashMap<>( + compactionStateCache.getPublishedCompactionStateMap() + ); + final Set cachedFingerprints = Set.copyOf(fingerprintToStateMap.keySet()); + final Set persistedFingerprints = query( + SqlSegmentsMetadataQuery::retrieveAllUsedCompactionStateFingerprints + ); + + // Remove entry for compaction states that have been deleted from the metadata store + final Set deletedFingerprints = Sets.difference(cachedFingerprints, persistedFingerprints); + deletedFingerprints.forEach(fingerprintToStateMap::remove); + + // Retrieve and add entry for compaction states that have been added to the metadata store + final Set addedFingerprints = Sets.difference(persistedFingerprints, cachedFingerprints); + final List addedCompactionStateRecords = query( + sql -> sql.retrieveCompactionStatesForFingerprints(addedFingerprints) + ); + if (addedCompactionStateRecords.size() < addedFingerprints.size()) { + emitMetric(Metric.SKIPPED_COMPACTION_STATES, addedFingerprints.size() - addedCompactionStateRecords.size()); + } + addedCompactionStateRecords.forEach( + record -> fingerprintToStateMap.put(record.getFingerprint(), record.getState()) + ); + + return fingerprintToStateMap; + } + /** * Summary of segments currently present in the metadata store for a single * datasource. diff --git a/server/src/main/java/org/apache/druid/metadata/segment/cache/Metric.java b/server/src/main/java/org/apache/druid/metadata/segment/cache/Metric.java index c8e87ca4d55d..0335e5116416 100644 --- a/server/src/main/java/org/apache/druid/metadata/segment/cache/Metric.java +++ b/server/src/main/java/org/apache/druid/metadata/segment/cache/Metric.java @@ -110,6 +110,11 @@ private Metric() */ public static final String RETRIEVE_SEGMENT_SCHEMAS_DURATION_MILLIS = METRIC_NAME_PREFIX + "fetchSchemas/time"; + /** + * Time taken in milliseconds to fetch all compaction states from the metadata store. + */ + public static final String RETRIEVE_COMPACTION_STATES_DURATION_MILLIS = METRIC_NAME_PREFIX + "fetchCompactionStates/time"; + /** * Time taken to update the datasource snapshot in the cache. */ @@ -158,6 +163,11 @@ private Metric() */ public static final String SKIPPED_SEGMENT_SCHEMAS = METRIC_NAME_PREFIX + "schema/skipped"; + /** + * Number of unparseable compaction state records skipped while refreshing the cache. + */ + public static final String SKIPPED_COMPACTION_STATES = METRIC_NAME_PREFIX + "compactionState/skipped"; + /** * Number of unparseable pending segment records skipped while refreshing the cache. */ diff --git a/server/src/main/java/org/apache/druid/segment/metadata/CompactionStateCache.java b/server/src/main/java/org/apache/druid/segment/metadata/CompactionStateCache.java new file mode 100644 index 000000000000..e3c614f252e5 --- /dev/null +++ b/server/src/main/java/org/apache/druid/segment/metadata/CompactionStateCache.java @@ -0,0 +1,147 @@ +/* + * 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.guice.LazySingleton; +import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.timeline.CompactionState; + +import java.util.Map; +import java.util.Optional; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; + +/** + * In-memory cache of compaction states used by {@link org.apache.druid.metadata.segment.cache.HeapMemorySegmentMetadataCache}. + *

    + * This cache stores compaction states for published segments polled from the metadata store. + * It is the PRIMARY way to read compaction states in production. + *

    + * The cache is populated during segment metadata cache sync operations and provides fast lookups + * without hitting the database. + */ +@LazySingleton +public class CompactionStateCache +{ + private static final Logger log = new Logger(CompactionStateCache.class); + + /** + * Atomically updated reference to published compaction states. + */ + private final AtomicReference publishedCompactionStates + = new AtomicReference<>(PublishedCompactionStates.EMPTY); + + private final AtomicInteger cacheMissCount = new AtomicInteger(0); + private final AtomicInteger cacheHitCount = new AtomicInteger(0); + + public boolean isEnabled() + { + // Always enabled when this implementation is bound + return true; + } + + /** + * Resets the cache with compaction states polled from the metadata store. + * Called after each successful poll in HeapMemorySegmentMetadataCache. + * + * @param fingerprintToStateMap Complete map of all active compaction state fingerprints + */ + public void resetCompactionStatesForPublishedSegments( + Map fingerprintToStateMap + ) + { + this.publishedCompactionStates.set( + new PublishedCompactionStates(fingerprintToStateMap) + ); + log.debug("Reset compaction state cache with [%d] fingerprints", fingerprintToStateMap.size()); + } + + /** + * Retrieves a compaction state by its fingerprint. + * This is the PRIMARY method for reading compaction states. + * + * @param fingerprint The fingerprint to look up + * @return The compaction state, or Optional.empty() if not cached + */ + public Optional getCompactionStateByFingerprint(String fingerprint) + { + if (fingerprint == null) { + return Optional.empty(); + } + + CompactionState state = publishedCompactionStates.get() + .fingerprintToStateMap + .get(fingerprint); + if (state != null) { + cacheHitCount.incrementAndGet(); + return Optional.of(state); + } + + cacheMissCount.incrementAndGet(); + return Optional.empty(); + } + + /** + * Gets the full cached map (immutable copy). + * Used by HeapMemorySegmentMetadataCache for delta sync calculations. + */ + public Map getPublishedCompactionStateMap() + { + return publishedCompactionStates.get().fingerprintToStateMap; + } + + /** + * Clears the cache. Called when node stops being leader. + */ + public void clear() + { + publishedCompactionStates.set(PublishedCompactionStates.EMPTY); + log.info("Cleared compaction state cache"); + } + + /** + * @return Summary stats for metric emission + */ + public Map getStats() + { + return Map.of( + Metric.COMPACTION_STATE_CACHE_HITS, cacheHitCount.getAndSet(0), + Metric.COMPACTION_STATE_CACHE_MISSES, cacheMissCount.getAndSet(0), + Metric.COMPACTION_STATE_CACHE_FINGERPRINTS, + publishedCompactionStates.get().fingerprintToStateMap.size() + ); + } + + /** + * Immutable snapshot of compaction states polled from DB. + */ + private static class PublishedCompactionStates + { + private static final PublishedCompactionStates EMPTY = + new PublishedCompactionStates(Map.of()); + + private final Map fingerprintToStateMap; + + private PublishedCompactionStates(Map fingerprintToStateMap) + { + this.fingerprintToStateMap = Map.copyOf(fingerprintToStateMap); + } + } +} 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 index 2ef592c36c8a..8962074fc2ef 100644 --- a/server/src/main/java/org/apache/druid/segment/metadata/CompactionStateManager.java +++ b/server/src/main/java/org/apache/druid/segment/metadata/CompactionStateManager.java @@ -22,7 +22,6 @@ import org.apache.druid.timeline.CompactionState; import org.joda.time.DateTime; -import javax.annotation.Nullable; import java.util.List; import java.util.Map; @@ -44,15 +43,6 @@ public interface CompactionStateManager */ String generateCompactionStateFingerprint(CompactionState compactionState, String dataSource); - /** - * Retrieves a compaction state by its fingerprint. - * - * @param fingerprint The fingerprint to look up - * @return The compaction state, or null if not found - */ - @Nullable - CompactionState getCompactionStateByFingerprint(String fingerprint); - /** * Persists compaction states to storage. * 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 deleted file mode 100644 index 04006931b8e5..000000000000 --- a/server/src/main/java/org/apache/druid/segment/metadata/CompactionStateManagerConfig.java +++ /dev/null @@ -1,47 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.druid.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/Metric.java b/server/src/main/java/org/apache/druid/segment/metadata/Metric.java index 3f1babbbde4a..4a566bc9c476 100644 --- a/server/src/main/java/org/apache/druid/segment/metadata/Metric.java +++ b/server/src/main/java/org/apache/druid/segment/metadata/Metric.java @@ -20,11 +20,12 @@ package org.apache.druid.segment.metadata; /** - * Metrics related to {@link SegmentSchemaCache} and {@link SegmentSchemaManager}. + * Metrics related to {@link SegmentSchemaCache}, {@link SegmentSchemaManager}, and {@link CompactionStateCache}. */ public class Metric { private static final String PREFIX = "segment/schemaCache/"; + private static final String COMPACTION_STATE_PREFIX = "segment/compactionStateCache/"; public static final String CACHE_MISSES = "miss/count"; @@ -57,4 +58,9 @@ public class Metric * Number of used cold segments in the metadata store. */ public static final String USED_COLD_SEGMENTS = "segment/used/deepStorageOnly/count"; + + // Compaction state cache metrics + public static final String COMPACTION_STATE_CACHE_HITS = COMPACTION_STATE_PREFIX + "hit/count"; + public static final String COMPACTION_STATE_CACHE_MISSES = COMPACTION_STATE_PREFIX + "miss/count"; + public static final String COMPACTION_STATE_CACHE_FINGERPRINTS = COMPACTION_STATE_PREFIX + "fingerprint/count"; } diff --git a/server/src/main/java/org/apache/druid/segment/metadata/NoopCompactionStateCache.java b/server/src/main/java/org/apache/druid/segment/metadata/NoopCompactionStateCache.java new file mode 100644 index 000000000000..c5247d5e710f --- /dev/null +++ b/server/src/main/java/org/apache/druid/segment/metadata/NoopCompactionStateCache.java @@ -0,0 +1,69 @@ +/* + * 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 java.util.Collections; +import java.util.Map; +import java.util.Optional; + +/** + * No-op implementation of {@link CompactionStateCache} used on non-Overlord nodes + * where compaction state caching is not needed. + */ +public class NoopCompactionStateCache extends CompactionStateCache +{ + @Override + public boolean isEnabled() + { + return false; + } + + @Override + public void resetCompactionStatesForPublishedSegments(Map fingerprintToStateMap) + { + // No-op + } + + @Override + public Optional getCompactionStateByFingerprint(String fingerprint) + { + return Optional.empty(); + } + + @Override + public Map getPublishedCompactionStateMap() + { + return Collections.emptyMap(); + } + + @Override + public void clear() + { + // No-op + } + + @Override + public Map getStats() + { + return Collections.emptyMap(); + } +} diff --git a/server/src/main/java/org/apache/druid/segment/metadata/PersistedCompactionStateManager.java b/server/src/main/java/org/apache/druid/segment/metadata/PersistedCompactionStateManager.java index e7d320c06eae..ed12ea220613 100644 --- a/server/src/main/java/org/apache/druid/segment/metadata/PersistedCompactionStateManager.java +++ b/server/src/main/java/org/apache/druid/segment/metadata/PersistedCompactionStateManager.java @@ -22,9 +22,6 @@ 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.hash.Hasher; import com.google.common.hash.Hashing; @@ -51,8 +48,6 @@ 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; @@ -83,8 +78,6 @@ public class PersistedCompactionStateManager implements CompactionStateManager private final ObjectMapper jsonMapper; private final ObjectMapper deterministicMapper; private final SQLMetadataConnector connector; - private final CompactionStateManagerConfig config; - private final Cache fingerprintCache; private final Striped datasourceLocks = Striped.lock(128); @Inject @@ -92,46 +85,23 @@ public PersistedCompactionStateManager( @Nonnull MetadataStorageTablesConfig dbTables, @Nonnull ObjectMapper jsonMapper, @Deterministic @Nonnull ObjectMapper deterministicMapper, - @Nonnull SQLMetadataConnector connector, - @Nonnull CompactionStateManagerConfig config + @Nonnull SQLMetadataConnector connector ) { this.dbTables = dbTables; this.jsonMapper = jsonMapper; this.deterministicMapper = deterministicMapper; 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 @@ -141,8 +111,6 @@ public void stop() this.jsonMapper = null; this.deterministicMapper = null; this.connector = null; - this.config = null; - this.fingerprintCache = null; } @Override @@ -255,7 +223,6 @@ public void persistCompactionState( ); } } - warmCache(fingerprintToStateMap); return null; }); } @@ -335,29 +302,6 @@ public int deleteUnusedCompactionStatesOlderThan(long timestamp) .execute()); } - @Override - @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; - } - } - @Override @SuppressWarnings("UnstableApiUsage") public String generateCompactionStateFingerprint( @@ -382,107 +326,6 @@ public String generateCompactionStateFingerprint( } - /** - * 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. **/ @@ -518,12 +361,6 @@ private Set getExistingFingerprints( 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)}. 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 ae2cd57d990d..4cbbdd63f54a 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 @@ -33,6 +33,7 @@ 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.CompactionStateCache; import org.apache.druid.segment.metadata.CompactionStateManager; import org.apache.druid.segment.transform.CompactionTransformSpec; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; @@ -261,7 +262,8 @@ public static CompactionStatus running(String message) static CompactionStatus compute( CompactionCandidate candidateSegments, DataSourceCompactionConfig config, - @Nullable CompactionStateManager compactionStateManager + @Nullable CompactionStateManager compactionStateManager, + @Nullable CompactionStateCache compactionStateCache ) { final CompactionState expectedState = CompactSegments.createCompactionStateFromConfig(config); @@ -274,7 +276,7 @@ static CompactionStatus compute( config.getDataSource() ); } - return new Evaluator(candidateSegments, config, expectedFingerprint, compactionStateManager).evaluate(); + return new Evaluator(candidateSegments, config, expectedFingerprint, compactionStateCache).evaluate(); } @Nullable @@ -359,13 +361,13 @@ private static class Evaluator @Nullable private final String targetFingerprint; - private final CompactionStateManager compactionStateManager; + private final CompactionStateCache compactionStateCache; private Evaluator( CompactionCandidate candidateSegments, DataSourceCompactionConfig compactionConfig, @Nullable String targetFingerprint, - @Nullable CompactionStateManager compactionStateManager + @Nullable CompactionStateCache compactionStateCache ) { this.candidateSegments = candidateSegments; @@ -373,7 +375,7 @@ private Evaluator( this.tuningConfig = ClientCompactionTaskQueryTuningConfig.from(compactionConfig); this.configuredGranularitySpec = compactionConfig.getGranularitySpec(); this.targetFingerprint = targetFingerprint; - this.compactionStateManager = compactionStateManager; + this.compactionStateCache = compactionStateCache; } private CompactionStatus evaluate() @@ -389,7 +391,7 @@ private CompactionStatus evaluate() reasonsForCompaction.add(compactedOnceCheck.getReason()); } - if (compactionStateManager != null && targetFingerprint != null) { + if (compactionStateCache != null && targetFingerprint != null) { // First try fingerprint-based evaluation (fast path) CompactionStatus fingerprintStatus = FINGERPRINT_CHECKS.stream() .map(f -> f.apply(this)) @@ -460,10 +462,10 @@ private CompactionStatus allFingerprintedCandidatesHaveExpectedFingerprint() boolean fingerprintedSegmentNeedingCompactionFound = false; - if (compactionStateManager != null) { + if (compactionStateCache != null) { for (Map.Entry> e : mismatchedFingerprintToSegmentMap.entrySet()) { String fingerprint = e.getKey(); - CompactionState stateToValidate = compactionStateManager.getCompactionStateByFingerprint(fingerprint); + CompactionState stateToValidate = compactionStateCache.getCompactionStateByFingerprint(fingerprint).orElse(null); if (stateToValidate == null) { log.warn("No compaction state found for fingerprint[%s]", fingerprint); fingerprintedSegmentNeedingCompactionFound = true; 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 c88fd264f82b..5b6f070ee38f 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.CompactionStateCache; import org.apache.druid.segment.metadata.CompactionStateManager; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; import org.apache.druid.timeline.DataSegment; @@ -70,6 +71,7 @@ public class DataSourceCompactibleSegmentIterator implements CompactionSegmentIt private final String dataSource; private final DataSourceCompactionConfig config; private final CompactionStateManager compactionStateManager; + private final CompactionStateCache compactionStateCache; private final List compactedSegments = new ArrayList<>(); private final List skippedSegments = new ArrayList<>(); @@ -87,13 +89,15 @@ public DataSourceCompactibleSegmentIterator( SegmentTimeline timeline, List skipIntervals, CompactionCandidateSearchPolicy searchPolicy, - CompactionStateManager compactionStateManager + CompactionStateManager compactionStateManager, + CompactionStateCache compactionStateCache ) { this.config = config; this.dataSource = config.getDataSource(); this.queue = new PriorityQueue<>(searchPolicy::compareCandidates); this.compactionStateManager = compactionStateManager; + this.compactionStateCache = compactionStateCache; populateQueue(timeline, skipIntervals); } @@ -330,7 +334,7 @@ private void findAndEnqueueSegmentsToCompact(CompactibleSegmentIterator compacti } final CompactionCandidate candidates = CompactionCandidate.from(segments, config.getSegmentGranularity()); - final CompactionStatus compactionStatus = CompactionStatus.compute(candidates, config, compactionStateManager); + final CompactionStatus compactionStatus = CompactionStatus.compute(candidates, config, compactionStateManager, compactionStateCache); 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 bdf945a10716..3a7feda8667f 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.CompactionStateCache; import org.apache.druid.segment.metadata.CompactionStateManager; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; import org.apache.druid.timeline.SegmentTimeline; @@ -50,7 +51,8 @@ public PriorityBasedCompactionSegmentIterator( Map compactionConfigs, Map datasourceToTimeline, Map> skipIntervals, - CompactionStateManager compactionStateManager + CompactionStateManager compactionStateManager, + CompactionStateCache compactionStateCache ) { this.queue = new PriorityQueue<>(searchPolicy::compareCandidates); @@ -72,7 +74,8 @@ public PriorityBasedCompactionSegmentIterator( timeline, skipIntervals.getOrDefault(datasource, Collections.emptyList()), searchPolicy, - compactionStateManager + compactionStateManager, + compactionStateCache ) ); addNextItemForDatasourceToQueue(datasource); 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 cb686fc01b4f..c2d67fb1fc9c 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,7 +53,6 @@ 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; @@ -146,7 +145,6 @@ public class DruidCoordinator private final CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig; private final CoordinatorDynamicConfigSyncer coordinatorDynamicConfigSyncer; private final CloneStatusManager cloneStatusManager; - private final CompactionStateManager compactionStateManager; private volatile boolean started = false; @@ -193,8 +191,7 @@ public DruidCoordinator( CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig, CompactionStatusTracker compactionStatusTracker, CoordinatorDynamicConfigSyncer coordinatorDynamicConfigSyncer, - CloneStatusManager cloneStatusManager, - CompactionStateManager compactionStateManager + CloneStatusManager cloneStatusManager ) { this.config = config; @@ -219,7 +216,6 @@ public DruidCoordinator( this.coordinatorDynamicConfigSyncer = coordinatorDynamicConfigSyncer; this.cloneStatusManager = cloneStatusManager; - this.compactionStateManager = compactionStateManager; this.compactSegments = initializeCompactSegmentsDuty(this.compactionStatusTracker); } 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 6efa913e2357..a65dd6d8e283 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 @@ -191,6 +191,7 @@ public void run( compactionConfigs, dataSources.getUsedSegmentsTimelinesPerDataSource(), slotManager.getDatasourceIntervalsToSkipCompaction(), + null, null ); diff --git a/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorReadOnlyTest.java b/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorReadOnlyTest.java index e0e6898df8b2..b0a9398b4a9c 100644 --- a/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorReadOnlyTest.java +++ b/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorReadOnlyTest.java @@ -36,6 +36,7 @@ import org.apache.druid.segment.TestDataSource; import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; +import org.apache.druid.segment.metadata.NoopCompactionStateCache; import org.apache.druid.segment.metadata.NoopSegmentSchemaCache; import org.apache.druid.server.coordinator.simulate.BlockingExecutorService; import org.apache.druid.server.coordinator.simulate.TestDruidLeaderSelector; @@ -104,6 +105,7 @@ public void setup() () -> new SegmentsMetadataManagerConfig(null, cacheMode, null), derbyConnectorRule.metadataTablesConfigSupplier(), new NoopSegmentSchemaCache(), + new NoopCompactionStateCache(), derbyConnector, (corePoolSize, nameFormat) -> new WrappingScheduledExecutorService( nameFormat, diff --git a/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java b/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java index dd9508c3c19b..5368cc005e29 100644 --- a/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java +++ b/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java @@ -46,6 +46,7 @@ import org.apache.druid.segment.TestDataSource; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; import org.apache.druid.segment.metadata.FingerprintGenerator; +import org.apache.druid.segment.metadata.NoopCompactionStateCache; import org.apache.druid.segment.metadata.NoopSegmentSchemaCache; import org.apache.druid.segment.metadata.SegmentSchemaManager; import org.apache.druid.segment.metadata.SegmentSchemaTestUtils; @@ -158,6 +159,7 @@ public void setUp() () -> new SegmentsMetadataManagerConfig(null, cacheMode, null), derbyConnectorRule.metadataTablesConfigSupplier(), new NoopSegmentSchemaCache(), + new NoopCompactionStateCache(), derbyConnector, (corePoolSize, nameFormat) -> new WrappingScheduledExecutorService( nameFormat, diff --git a/server/src/test/java/org/apache/druid/metadata/segment/SqlSegmentsMetadataManagerV2Test.java b/server/src/test/java/org/apache/druid/metadata/segment/SqlSegmentsMetadataManagerV2Test.java index 08098decdab1..a54831ec957e 100644 --- a/server/src/test/java/org/apache/druid/metadata/segment/SqlSegmentsMetadataManagerV2Test.java +++ b/server/src/test/java/org/apache/druid/metadata/segment/SqlSegmentsMetadataManagerV2Test.java @@ -33,6 +33,7 @@ import org.apache.druid.metadata.segment.cache.SegmentMetadataCache; import org.apache.druid.segment.TestDataSource; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; +import org.apache.druid.segment.metadata.CompactionStateCache; import org.apache.druid.segment.metadata.NoopSegmentSchemaCache; import org.apache.druid.segment.metadata.SegmentSchemaCache; import org.apache.druid.server.coordinator.CreateDataSegments; @@ -74,6 +75,7 @@ public void setup() throws Exception { setUp(derbyConnectorRule); connector.createPendingSegmentsTable(); + connector.createCompactionStatesTable(); emitter = new StubServiceEmitter(); @@ -91,6 +93,7 @@ private void initManager( Suppliers.ofInstance(new SegmentsMetadataManagerConfig(Period.seconds(1), cacheMode, null)), Suppliers.ofInstance(storageConfig), useSchemaCache ? new SegmentSchemaCache() : new NoopSegmentSchemaCache(), + new CompactionStateCache(), connector, (poolSize, name) -> new WrappingScheduledExecutorService(name, segmentMetadataCacheExec, false), emitter 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 99ebf7083ac8..fcfa34d17a5b 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 @@ -38,6 +38,7 @@ import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; +import org.apache.druid.segment.metadata.CompactionStateCache; import org.apache.druid.segment.metadata.FingerprintGenerator; import org.apache.druid.segment.metadata.NoopSegmentSchemaCache; import org.apache.druid.segment.metadata.SegmentSchemaCache; @@ -76,6 +77,7 @@ public class HeapMemorySegmentMetadataCacheTest private HeapMemorySegmentMetadataCache cache; private SegmentSchemaCache schemaCache; + private CompactionStateCache compactionStateCache; private SegmentSchemaTestUtils schemaTestUtils; @Before @@ -112,7 +114,8 @@ private void setupTargetWithCaching(SegmentMetadataCache.UsageMode cacheMode) /** * Creates the target {@link #cache} to be tested in the current test. */ - private void setupTargetWithCaching(SegmentMetadataCache.UsageMode cacheMode, boolean useSchemaCache) + private void setupTargetWithCaching(SegmentMetadataCache.UsageMode cacheMode, boolean useSchemaCache + ) { if (cache != null) { throw new ISE("Test target has already been initialized with caching[%s]", cache.isEnabled()); @@ -120,11 +123,13 @@ private void setupTargetWithCaching(SegmentMetadataCache.UsageMode cacheMode, bo final SegmentsMetadataManagerConfig metadataManagerConfig = new SegmentsMetadataManagerConfig(null, cacheMode, null); schemaCache = useSchemaCache ? new SegmentSchemaCache() : new NoopSegmentSchemaCache(); + compactionStateCache = new CompactionStateCache(); cache = new HeapMemorySegmentMetadataCache( TestHelper.JSON_MAPPER, () -> metadataManagerConfig, derbyConnectorRule.metadataTablesConfigSupplier(), schemaCache, + compactionStateCache, derbyConnector, executorFactory, serviceEmitter diff --git a/server/src/test/java/org/apache/druid/segment/metadata/CompactionStateCacheTest.java b/server/src/test/java/org/apache/druid/segment/metadata/CompactionStateCacheTest.java new file mode 100644 index 000000000000..5238b2a4bf54 --- /dev/null +++ b/server/src/test/java/org/apache/druid/segment/metadata/CompactionStateCacheTest.java @@ -0,0 +1,239 @@ +/* + * 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.data.input.impl.DimensionsSpec; +import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; +import org.apache.druid.segment.IndexSpec; +import org.apache.druid.timeline.CompactionState; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import java.util.Optional; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertTrue; + +public class CompactionStateCacheTest +{ + private CompactionStateCache cache; + + @BeforeEach + public void setUp() + { + cache = new CompactionStateCache(); + } + + @Test + public void test_isEnabled_returnsTrue() + { + assertTrue(cache.isEnabled()); + } + + @Test + public void test_getCompactionStateByFingerprint_emptyCache_returnsEmpty() + { + Optional result = cache.getCompactionStateByFingerprint("nonexistent"); + assertFalse(result.isPresent()); + } + + @Test + public void test_getCompactionStateByFingerprint_nullFingerprint_returnsEmpty() + { + Optional result = cache.getCompactionStateByFingerprint(null); + assertFalse(result.isPresent()); + } + + @Test + public void test_resetCompactionStatesForPublishedSegments_andThen_getCompactionStateByFingerprint() + { + CompactionState state1 = createTestCompactionState(); + CompactionState state2 = createTestCompactionState(); + + Map stateMap = new HashMap<>(); + stateMap.put("fingerprint1", state1); + stateMap.put("fingerprint2", state2); + + cache.resetCompactionStatesForPublishedSegments(stateMap); + + Optional result1 = cache.getCompactionStateByFingerprint("fingerprint1"); + assertTrue(result1.isPresent()); + assertEquals(state1, result1.get()); + + Optional result2 = cache.getCompactionStateByFingerprint("fingerprint2"); + assertTrue(result2.isPresent()); + assertEquals(state2, result2.get()); + + Optional result3 = cache.getCompactionStateByFingerprint("nonexistent"); + assertFalse(result3.isPresent()); + } + + @Test + public void test_getPublishedCompactionStateMap_returnsImmutableSnapshot() + { + CompactionState state1 = createTestCompactionState(); + Map stateMap = new HashMap<>(); + stateMap.put("fingerprint1", state1); + + cache.resetCompactionStatesForPublishedSegments(stateMap); + + Map retrieved = cache.getPublishedCompactionStateMap(); + assertEquals(1, retrieved.size()); + assertEquals(state1, retrieved.get("fingerprint1")); + } + + @Test + public void test_clear_emptiesCache() + { + CompactionState state1 = createTestCompactionState(); + Map stateMap = new HashMap<>(); + stateMap.put("fingerprint1", state1); + + cache.resetCompactionStatesForPublishedSegments(stateMap); + + Optional beforeClear = cache.getCompactionStateByFingerprint("fingerprint1"); + assertTrue(beforeClear.isPresent()); + + cache.clear(); + + Optional afterClear = cache.getCompactionStateByFingerprint("fingerprint1"); + assertFalse(afterClear.isPresent()); + + Map mapAfterClear = cache.getPublishedCompactionStateMap(); + assertEquals(0, mapAfterClear.size()); + } + + @Test + public void test_stats_trackHitsAndMisses() + { + CompactionState state1 = createTestCompactionState(); + Map stateMap = new HashMap<>(); + stateMap.put("fingerprint1", state1); + + cache.resetCompactionStatesForPublishedSegments(stateMap); + + // Generate 3 hits + cache.getCompactionStateByFingerprint("fingerprint1"); + cache.getCompactionStateByFingerprint("fingerprint1"); + cache.getCompactionStateByFingerprint("fingerprint1"); + + // Generate 2 misses + cache.getCompactionStateByFingerprint("nonexistent1"); + cache.getCompactionStateByFingerprint("nonexistent2"); + + Map stats = cache.getStats(); + assertEquals(3, stats.get(Metric.COMPACTION_STATE_CACHE_HITS)); + assertEquals(2, stats.get(Metric.COMPACTION_STATE_CACHE_MISSES)); + assertEquals(1, stats.get(Metric.COMPACTION_STATE_CACHE_FINGERPRINTS)); + } + + @Test + public void test_stats_resetAfterReading() + { + CompactionState state1 = createTestCompactionState(); + Map stateMap = new HashMap<>(); + stateMap.put("fingerprint1", state1); + + cache.resetCompactionStatesForPublishedSegments(stateMap); + + // Generate hits and misses + cache.getCompactionStateByFingerprint("fingerprint1"); + cache.getCompactionStateByFingerprint("nonexistent"); + + Map stats1 = cache.getStats(); + assertEquals(1, stats1.get(Metric.COMPACTION_STATE_CACHE_HITS)); + assertEquals(1, stats1.get(Metric.COMPACTION_STATE_CACHE_MISSES)); + + // Stats should be reset after reading + Map stats2 = cache.getStats(); + assertEquals(0, stats2.get(Metric.COMPACTION_STATE_CACHE_HITS)); + assertEquals(0, stats2.get(Metric.COMPACTION_STATE_CACHE_MISSES)); + assertEquals(1, stats2.get(Metric.COMPACTION_STATE_CACHE_FINGERPRINTS)); // Fingerprints count doesn't reset + } + + @Test + public void test_multipleResets_replacesCache() + { + CompactionState state1 = createTestCompactionState(); + CompactionState state2 = createTestCompactionState(); + + // First reset + Map firstMap = new HashMap<>(); + firstMap.put("fingerprint1", state1); + cache.resetCompactionStatesForPublishedSegments(firstMap); + + Optional result1 = cache.getCompactionStateByFingerprint("fingerprint1"); + assertTrue(result1.isPresent()); + assertEquals(state1, result1.get()); + + // Second reset with different data + Map secondMap = new HashMap<>(); + secondMap.put("fingerprint2", state2); + cache.resetCompactionStatesForPublishedSegments(secondMap); + + // Old fingerprint should be gone + Optional oldResult = cache.getCompactionStateByFingerprint("fingerprint1"); + assertFalse(oldResult.isPresent()); + + // New fingerprint should exist + Optional newResult = cache.getCompactionStateByFingerprint("fingerprint2"); + assertTrue(newResult.isPresent()); + assertEquals(state2, newResult.get()); + } + + @Test + public void test_resetWithEmptyMap() + { + CompactionState state1 = createTestCompactionState(); + Map stateMap = new HashMap<>(); + stateMap.put("fingerprint1", state1); + + cache.resetCompactionStatesForPublishedSegments(stateMap); + + Optional beforeReset = cache.getCompactionStateByFingerprint("fingerprint1"); + assertTrue(beforeReset.isPresent()); + + // Reset with empty map + cache.resetCompactionStatesForPublishedSegments(Collections.emptyMap()); + + Optional afterReset = cache.getCompactionStateByFingerprint("fingerprint1"); + assertFalse(afterReset.isPresent()); + + Map stats = cache.getStats(); + assertEquals(0, stats.get(Metric.COMPACTION_STATE_CACHE_FINGERPRINTS)); + } + + private CompactionState createTestCompactionState() + { + return new CompactionState( + new DynamicPartitionsSpec(100, null), + DimensionsSpec.EMPTY, + null, + null, + IndexSpec.getDefault(), + null, + null + ); + } +} diff --git a/server/src/test/java/org/apache/druid/segment/metadata/HeapMemoryCompactionStateManager.java b/server/src/test/java/org/apache/druid/segment/metadata/HeapMemoryCompactionStateManager.java index c71b09934e73..7171ec1e5a0c 100644 --- a/server/src/test/java/org/apache/druid/segment/metadata/HeapMemoryCompactionStateManager.java +++ b/server/src/test/java/org/apache/druid/segment/metadata/HeapMemoryCompactionStateManager.java @@ -114,13 +114,23 @@ public void persistCompactionState( this.fingerprintToStateMap.putAll(fingerprintToStateMap); } - @Override + /** + * Gets a compaction state by fingerprint. For test verification only. + */ @Nullable public CompactionState getCompactionStateByFingerprint(String fingerprint) { return fingerprintToStateMap.get(fingerprint); } + /** + * Gets all stored compaction states. For test verification only. + */ + public Map getAllStoredStates() + { + return Map.copyOf(fingerprintToStateMap); + } + /** * Clears all stored compaction states. Useful for test cleanup or resetting * state between test runs. @@ -138,11 +148,4 @@ 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/test/java/org/apache/druid/segment/metadata/PersistedCompactionStateManagerConfigTest.java b/server/src/test/java/org/apache/druid/segment/metadata/PersistedCompactionStateManagerConfigTest.java deleted file mode 100644 index 5106a48ad1d8..000000000000 --- a/server/src/test/java/org/apache/druid/segment/metadata/PersistedCompactionStateManagerConfigTest.java +++ /dev/null @@ -1,50 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.druid.segment.metadata; - -import com.fasterxml.jackson.databind.ObjectMapper; -import org.apache.druid.jackson.DefaultObjectMapper; -import org.junit.Assert; -import org.junit.Test; - -public class PersistedCompactionStateManagerConfigTest -{ - 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/PersistedCompactionStateManagerTest.java b/server/src/test/java/org/apache/druid/segment/metadata/PersistedCompactionStateManagerTest.java index d631fae26cd6..187d80d41d38 100644 --- a/server/src/test/java/org/apache/druid/segment/metadata/PersistedCompactionStateManagerTest.java +++ b/server/src/test/java/org/apache/druid/segment/metadata/PersistedCompactionStateManagerTest.java @@ -48,10 +48,7 @@ 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.assertNotEquals; -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 PersistedCompactionStateManagerTest @@ -85,11 +82,11 @@ public void setUp() return null; }); - manager = new PersistedCompactionStateManager(tablesConfig, jsonMapper, deterministicMapper, derbyConnector, new CompactionStateManagerConfig()); + manager = new PersistedCompactionStateManager(tablesConfig, jsonMapper, deterministicMapper, derbyConnector); } @Test - public void test_persistCompactionState_andThen_getCompactionStateByFingerprint() + public void test_persistCompactionState_successfullyInsertsIntoDatabase() { CompactionState state1 = createTestCompactionState(); String fingerprint = "fingerprint_abc123"; @@ -106,31 +103,16 @@ public void test_persistCompactionState_andThen_getCompactionStateByFingerprint( 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); + // Verify the state was inserted into database by checking count + Integer count = derbyConnector.retryWithHandle(handle -> + handle.createQuery( + "SELECT COUNT(*) FROM " + tablesConfig.getCompactionStatesTable() + + " WHERE fingerprint = :fp" + ).bind("fp", fingerprint) + .map((i, r, ctx) -> r.getInt(1)) + .first() + ); + assertEquals(1, count); } @Test @@ -254,10 +236,6 @@ public void test_deleteCompactionStatesOlderThan_deletesOnlyOldUnusedStates() th 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()) @@ -267,48 +245,6 @@ public void test_deleteCompactionStatesOlderThan_deletesOnlyOldUnusedStates() th 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 - PersistedCompactionStateManager newManager = new PersistedCompactionStateManager( - tablesConfig, - jsonMapper, - deterministicMapper, - 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() { @@ -335,18 +271,6 @@ public void test_persistCompactionState_withEmptyMap_doesNothing() 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 { 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 858b8f67b250..33eec17a0be4 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 @@ -39,6 +39,7 @@ 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.CompactionStateCache; import org.apache.druid.segment.metadata.CompactionStateManager; import org.apache.druid.segment.metadata.HeapMemoryCompactionStateManager; import org.apache.druid.segment.nested.NestedCommonFormatColumnFormatSpec; @@ -70,12 +71,22 @@ public class CompactionStatusTest .size(100_000_000L) .build(); - private CompactionStateManager compactionStateManager; + private HeapMemoryCompactionStateManager compactionStateManager; + private CompactionStateCache compactionStateCache; @Before public void setUp() { compactionStateManager = new HeapMemoryCompactionStateManager(); + compactionStateCache = new CompactionStateCache(); + } + + /** + * Helper to sync the cache with states stored in the manager (for tests that persist states). + */ + private void syncCacheFromManager() + { + compactionStateCache.resetCompactionStatesForPublishedSegments(compactionStateManager.getAllStoredStates()); } @Test @@ -346,7 +357,8 @@ public void testStatusWhenLastCompactionStateSameAsRequired() final CompactionStatus status = CompactionStatus.compute( CompactionCandidate.from(List.of(segment), Granularities.HOUR), compactionConfig, - compactionStateManager + compactionStateManager, + compactionStateCache ); Assert.assertTrue(status.isComplete()); } @@ -396,7 +408,8 @@ public void testStatusWhenProjectionsMatch() final CompactionStatus status = CompactionStatus.compute( CompactionCandidate.from(List.of(segment), Granularities.HOUR), compactionConfig, - compactionStateManager + compactionStateManager, + compactionStateCache ); Assert.assertTrue(status.isComplete()); } @@ -451,7 +464,8 @@ public void testStatusWhenProjectionsMismatch() final CompactionStatus status = CompactionStatus.compute( CompactionCandidate.from(List.of(segment), Granularities.HOUR), compactionConfig, - compactionStateManager + compactionStateManager, + compactionStateCache ); Assert.assertFalse(status.isComplete()); } @@ -505,7 +519,8 @@ public void testStatusWhenAutoSchemaMatch() final CompactionStatus status = CompactionStatus.compute( CompactionCandidate.from(List.of(segment), null), compactionConfig, - compactionStateManager + compactionStateManager, + compactionStateCache ); Assert.assertTrue(status.isComplete()); } @@ -559,7 +574,8 @@ public void testStatusWhenAutoSchemaMismatch() final CompactionStatus status = CompactionStatus.compute( CompactionCandidate.from(List.of(segment), null), compactionConfig, - compactionStateManager + compactionStateManager, + compactionStateCache ); Assert.assertFalse(status.isComplete()); } @@ -588,6 +604,7 @@ public void test_evaluate_needsCompactionWhenAllSegmentsHaveUnexpectedCompaction CompactionState expectedState = CompactSegments.createCompactionStateFromConfig(compactionConfig); compactionStateManager.persistCompactionState(TestDataSource.WIKI, Map.of("wrongFingerprint", wrongState), DateTimes.nowUtc()); + syncCacheFromManager(); verifyEvaluationNeedsCompactionBecauseWithCustomSegments( CompactionCandidate.from(segments, null), @@ -623,6 +640,7 @@ public void test_evaluate_needsCompactionWhenSomeSegmentsHaveUnexpectedCompactio ); compactionStateManager.persistCompactionState(TestDataSource.WIKI, Map.of("wrongFingerprint", wrongState), DateTimes.nowUtc()); + syncCacheFromManager(); verifyEvaluationNeedsCompactionBecauseWithCustomSegments( CompactionCandidate.from(segments, null), @@ -646,11 +664,13 @@ public void test_evaluate_noCompacationIfUnexpectedFingerprintHasExpectedCompact CompactionState expectedState = CompactSegments.createCompactionStateFromConfig(compactionConfig); compactionStateManager.persistCompactionState(TestDataSource.WIKI, Map.of("wrongFingerprint", expectedState), DateTimes.nowUtc()); + syncCacheFromManager(); final CompactionStatus status = CompactionStatus.compute( CompactionCandidate.from(segments, null), compactionConfig, - compactionStateManager + compactionStateManager, + compactionStateCache ); Assert.assertTrue(status.isComplete()); } @@ -716,7 +736,8 @@ public void test_evaluate_noCompactionWhenAllSegmentsHaveExpectedCompactionState final CompactionStatus status = CompactionStatus.compute( CompactionCandidate.from(segments, null), compactionConfig, - compactionStateManager + compactionStateManager, + compactionStateCache ); Assert.assertTrue(status.isComplete()); } @@ -768,7 +789,8 @@ public void test_evaluate_noCompactionWhenNonFingerprintedSegmentsPassChecksOnLa final CompactionStatus status = CompactionStatus.compute( CompactionCandidate.from(segments, null), compactionConfig, - compactionStateManager + compactionStateManager, + compactionStateCache ); Assert.assertTrue(status.isComplete()); } @@ -798,7 +820,8 @@ public void test_evaluate_isSkippedWhenInputBytesExceedLimit() final CompactionStatus status = CompactionStatus.compute( CompactionCandidate.from(segments, null), compactionConfig, - compactionStateManager + compactionStateManager, + compactionStateCache ); Assert.assertFalse(status.isComplete()); @@ -822,7 +845,8 @@ private void verifyEvaluationNeedsCompactionBecauseWithCustomSegments( final CompactionStatus status = CompactionStatus.compute( candidate, compactionConfig, - compactionStateManager + compactionStateManager, + compactionStateCache ); Assert.assertFalse(status.isComplete()); @@ -842,7 +866,8 @@ private void verifyCompactionStatusIsPendingBecause( final CompactionStatus status = CompactionStatus.compute( CompactionCandidate.from(List.of(segment), null), compactionConfig, - compactionStateManager + compactionStateManager, + compactionStateCache ); Assert.assertFalse(status.isComplete()); 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 23853a9ca6ba..412b6041a982 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,8 +47,10 @@ 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.CompactionStateCache; import org.apache.druid.segment.metadata.CompactionStateManager; import org.apache.druid.segment.metadata.HeapMemoryCompactionStateManager; +import org.apache.druid.segment.metadata.NoopCompactionStateCache; import org.apache.druid.segment.transform.CompactionTransformSpec; import org.apache.druid.server.coordinator.CreateDataSegments; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; @@ -84,6 +86,7 @@ public class NewestSegmentFirstPolicyTest private final ObjectMapper mapper = new DefaultObjectMapper(); private final NewestSegmentFirstPolicy policy = new NewestSegmentFirstPolicy(null); private final CompactionStateManager compactionStateManager = new HeapMemoryCompactionStateManager(); + private final CompactionStateCache compactionStateCache = new NoopCompactionStateCache(); @Test public void testLargeOffsetAndSmallSegmentInterval() @@ -280,7 +283,8 @@ public void testSkipDataSourceWithNoSegments() ) ), Collections.emptyMap(), - compactionStateManager + compactionStateManager, + compactionStateCache ); assertCompactSegmentIntervals( @@ -513,7 +517,8 @@ public void testWithSkipIntervals() Intervals.of("2017-11-13T00:00:00/2017-11-14T01:00:00") ) ), - compactionStateManager + compactionStateManager, + compactionStateCache ); assertCompactSegmentIntervals( @@ -553,7 +558,8 @@ public void testHoleInSearchInterval() Intervals.of("2017-11-16T14:00:00/2017-11-16T20:00:00") ) ), - compactionStateManager + compactionStateManager, + compactionStateCache ); assertCompactSegmentIntervals( @@ -2059,7 +2065,8 @@ TestDataSource.KOALA, configBuilder().forDataSource(TestDataSource.KOALA).build( TestDataSource.KOALA, SegmentTimeline.forSegments(koalaSegments) ), Collections.emptyMap(), - compactionStateManager + compactionStateManager, + compactionStateCache ); // Verify that the segments of WIKI are preferred even though they are older @@ -2081,7 +2088,8 @@ private CompactionSegmentIterator createIterator(DataSourceCompactionConfig conf Collections.singletonMap(TestDataSource.WIKI, config), Collections.singletonMap(TestDataSource.WIKI, timeline), Collections.emptyMap(), - compactionStateManager + compactionStateManager, + compactionStateCache ); } 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 d38f2d868872..9c3de8c3b2a2 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 @@ -173,8 +173,7 @@ public void setUp() throws Exception CentralizedDatasourceSchemaConfig.create(), new CompactionStatusTracker(), EasyMock.niceMock(CoordinatorDynamicConfigSyncer.class), - EasyMock.niceMock(CloneStatusManager.class), - persistedCompactionStateManager + EasyMock.niceMock(CloneStatusManager.class) ); } @@ -487,8 +486,7 @@ public void testCompactSegmentsDutyWhenCustomDutyGroupEmpty() CentralizedDatasourceSchemaConfig.create(), new CompactionStatusTracker(), EasyMock.niceMock(CoordinatorDynamicConfigSyncer.class), - EasyMock.niceMock(CloneStatusManager.class), - persistedCompactionStateManager + EasyMock.niceMock(CloneStatusManager.class) ); coordinator.start(); @@ -540,8 +538,7 @@ public void testInitializeCompactSegmentsDutyWhenCustomDutyGroupDoesNotContainsC CentralizedDatasourceSchemaConfig.create(), new CompactionStatusTracker(), EasyMock.niceMock(CoordinatorDynamicConfigSyncer.class), - EasyMock.niceMock(CloneStatusManager.class), - persistedCompactionStateManager + EasyMock.niceMock(CloneStatusManager.class) ); coordinator.start(); // Since CompactSegments is not enabled in Custom Duty Group, then CompactSegments must be created in IndexingServiceDuties @@ -593,8 +590,7 @@ public void testInitializeCompactSegmentsDutyWhenCustomDutyGroupContainsCompactS CentralizedDatasourceSchemaConfig.create(), new CompactionStatusTracker(), EasyMock.niceMock(CoordinatorDynamicConfigSyncer.class), - EasyMock.niceMock(CloneStatusManager.class), - persistedCompactionStateManager + EasyMock.niceMock(CloneStatusManager.class) ); coordinator.start(); @@ -704,8 +700,7 @@ public void testCoordinatorCustomDutyGroupsRunAsExpected() throws Exception CentralizedDatasourceSchemaConfig.create(), new CompactionStatusTracker(), EasyMock.niceMock(CoordinatorDynamicConfigSyncer.class), - EasyMock.niceMock(CloneStatusManager.class), - persistedCompactionStateManager + EasyMock.niceMock(CloneStatusManager.class) ); coordinator.start(); 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 index 25e6953a0c4c..daf1a259c3a1 100644 --- 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 @@ -29,7 +29,6 @@ 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.segment.metadata.PersistedCompactionStateManager; import org.apache.druid.server.coordinator.DruidCoordinatorRuntimeParams; import org.apache.druid.server.coordinator.config.MetadataCleanupConfig; @@ -73,7 +72,7 @@ public void setUp() derbyConnector.createCompactionStatesTable(); derbyConnector.createSegmentTable(); - compactionStateManager = new PersistedCompactionStateManager(tablesConfig, jsonMapper, createDeterministicMapper(), derbyConnector, new CompactionStateManagerConfig()); + compactionStateManager = new PersistedCompactionStateManager(tablesConfig, jsonMapper, createDeterministicMapper(), derbyConnector); mockParams = EasyMock.createMock(DruidCoordinatorRuntimeParams.class); CoordinatorRunStats runStats = new CoordinatorRunStats(); 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 8f36cd51232d..0c07ef75da9e 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 @@ -226,8 +226,7 @@ public CoordinatorSimulation build() CentralizedDatasourceSchemaConfig.create(), new CompactionStatusTracker(), env.configSyncer, - env.cloneStatusManager, - new HeapMemoryCompactionStateManager() + env.cloneStatusManager ); return new SimulationImpl(coordinator, env); 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 c5ea7050b6f6..726025dde149 100644 --- a/services/src/main/java/org/apache/druid/guice/MetadataManagerModule.java +++ b/services/src/main/java/org/apache/druid/guice/MetadataManagerModule.java @@ -41,8 +41,9 @@ 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.CompactionStateCache; import org.apache.druid.segment.metadata.CompactionStateManager; -import org.apache.druid.segment.metadata.CompactionStateManagerConfig; +import org.apache.druid.segment.metadata.NoopCompactionStateCache; import org.apache.druid.segment.metadata.NoopSegmentSchemaCache; import org.apache.druid.segment.metadata.PersistedCompactionStateManager; import org.apache.druid.segment.metadata.SegmentSchemaCache; @@ -62,6 +63,7 @@ *

  • {@link IndexerMetadataStorageCoordinator}
  • *
  • {@link CoordinatorConfigManager}
  • *
  • {@link SegmentMetadataCache}
  • + *
  • {@link CompactionStateCache} - Overlord only
  • *
  • {@link SegmentSchemaCache} - Coordinator only
  • *
  • {@link PersistedCompactionStateManager}
  • * @@ -105,6 +107,9 @@ public void configure(Binder binder) binder.bind(SegmentMetadataCache.class) .to(HeapMemorySegmentMetadataCache.class) .in(LazySingleton.class); + binder.bind(CompactionStateManager.class) + .to(PersistedCompactionStateManager.class) + .in(ManageLifecycle.class); // Coordinator-only dependencies if (nodeRoles.contains(NodeRole.COORDINATOR)) { @@ -127,16 +132,20 @@ public void configure(Binder binder) .in(LazySingleton.class); } + // Overlord-only compaction state dependencies + if (nodeRoles.contains(NodeRole.OVERLORD)) { + binder.bind(CompactionStateCache.class).in(LazySingleton.class); + } else { + binder.bind(CompactionStateCache.class) + .to(NoopCompactionStateCache.class) + .in(LazySingleton.class); + } + // Overlord-only dependencies if (nodeRoles.contains(NodeRole.OVERLORD)) { binder.bind(SegmentMetadataTransactionFactory.class) .to(SqlSegmentMetadataTransactionFactory.class) .in(LazySingleton.class); - - JsonConfigProvider.bind(binder, "druid.manager.compactionState", CompactionStateManagerConfig.class); - binder.bind(CompactionStateManager.class) - .to(PersistedCompactionStateManager.class) - .in(ManageLifecycle.class); } else { binder.bind(SegmentMetadataTransactionFactory.class) .to(SqlSegmentMetadataReadOnlyTransactionFactory.class) From cd55b0e2cf6f9a868e72c31db30724521957b9a8 Mon Sep 17 00:00:00 2001 From: capistrant Date: Sun, 4 Jan 2026 10:26:21 -0600 Subject: [PATCH 33/72] prettify --- .../compaction-dynamic-config-completions.ts | 3 +-- .../compaction-dynamic-config/compaction-dynamic-config.tsx | 6 +++--- 2 files changed, 4 insertions(+), 5 deletions(-) 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 4021b8e1bebc..688934fad885 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 @@ -127,8 +127,7 @@ export const COMPACTION_DYNAMIC_CONFIG_COMPLETIONS: JsonCompletionRule[] = [ completions: [ { value: 'true', - documentation: - 'Store full compaction state in segment metadata (legacy behavior, default)', + documentation: 'Store full compaction state in segment metadata (legacy behavior, default)', }, { value: 'false', 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 7f6d6ac30917..1eff27e51d6e 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 @@ -103,9 +103,9 @@ export const COMPACTION_DYNAMIC_CONFIG_FIELDS: Field[] 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. + 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, From 58724cc1839d637b6304fcc3222bd18adacb000a Mon Sep 17 00:00:00 2001 From: capistrant Date: Sun, 4 Jan 2026 12:33:57 -0600 Subject: [PATCH 34/72] small changes after self-review --- .../compact/CompactionSupervisorTest.java | 4 +- .../common/task/AbstractBatchIndexTask.java | 4 + .../CompactionConfigBasedJobTemplate.java | 3 +- .../druid/timeline/CompactionState.java | 1 - .../metadata/SqlSegmentsMetadataQuery.java | 9 + .../segment/cache/CompactionStateRecord.java | 25 -- .../cache/HeapMemorySegmentMetadataCache.java | 6 +- .../server/compaction/CompactionStatus.java | 62 ++- .../coordinator/duty/CompactSegments.java | 63 --- .../duty/KillUnreferencedCompactionState.java | 13 + .../SqlSegmentsMetadataQueryTest.java | 363 ++++++++++++++++++ .../HeapMemorySegmentMetadataCacheTest.java | 3 +- .../compaction/CompactionStatusTest.java | 17 +- .../druid/guice/MetadataManagerModule.java | 13 +- website/.spelling | 1 - 15 files changed, 469 insertions(+), 118 deletions(-) diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionSupervisorTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionSupervisorTest.java index ce29dcecd695..fe602b5e3463 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionSupervisorTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionSupervisorTest.java @@ -32,12 +32,12 @@ import org.apache.druid.query.DruidMetrics; import org.apache.druid.rpc.UpdateResponse; import org.apache.druid.segment.metadata.CompactionStateManager; +import org.apache.druid.server.compaction.CompactionStatus; import org.apache.druid.server.coordinator.ClusterCompactionConfig; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; import org.apache.druid.server.coordinator.InlineSchemaDataSourceCompactionConfig; import org.apache.druid.server.coordinator.UserCompactionTaskGranularityConfig; import org.apache.druid.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; @@ -292,7 +292,7 @@ private void verifyCompactedSegmentsHaveFingerprints(DataSourceCompactionConfig .getInstance(CompactionStateManager.class); String expectedFingerprint = compactionStateManager.generateCompactionStateFingerprint( - CompactSegments.createCompactionStateFromConfig(compactionConfig), + CompactionStatus.createCompactionStateFromConfig(compactionConfig), dataSource ); 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 fb4a9903fb13..aa3f580a01c3 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,10 @@ public static boolean isGuaranteedRollup( return tuningConfig.isForceGuaranteedRollup(); } + /** + * Returns a function that adds the given compaction state fingerprint to all segments. + * If the fingerprint is null, returns an identity function that leaves segments unchanged. + */ public static Function, Set> addCompactionStateFingerprintToSegments( String compactionStateFingerprint ) 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 127831b046cd..c0b74e811ed9 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 @@ -27,6 +27,7 @@ import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.server.compaction.CompactionCandidate; import org.apache.druid.server.compaction.CompactionSlotManager; +import org.apache.druid.server.compaction.CompactionStatus; import org.apache.druid.server.compaction.DataSourceCompactibleSegmentIterator; import org.apache.druid.server.compaction.NewestSegmentFirstPolicy; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; @@ -71,7 +72,7 @@ public List createCompactionJobs( final List jobs = new ArrayList<>(); - CompactionState compactionState = CompactSegments.createCompactionStateFromConfig(config); + CompactionState compactionState = CompactionStatus.createCompactionStateFromConfig(config); String compactionStateFingerprint = params.getCompactionStateManager().generateCompactionStateFingerprint( compactionState, 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 14c100736e7d..f99d996a6b73 100644 --- a/processing/src/main/java/org/apache/druid/timeline/CompactionState.java +++ b/processing/src/main/java/org/apache/druid/timeline/CompactionState.java @@ -50,7 +50,6 @@ */ public class CompactionState { - private final PartitionsSpec partitionsSpec; private final DimensionsSpec dimensionsSpec; private final CompactionTransformSpec transformSpec; 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 09a4501a070f..e31c453bb76f 100644 --- a/server/src/main/java/org/apache/druid/metadata/SqlSegmentsMetadataQuery.java +++ b/server/src/main/java/org/apache/druid/metadata/SqlSegmentsMetadataQuery.java @@ -1794,6 +1794,10 @@ private List retrieveBatchOfCompactionStates(List return retrieveValidCompactionStateRecordsWithQuery(query); } + /** + * Executes the given query and maps results to valid CompactionStateRecord objects. + * Records that fail to parse are filtered out. + */ private List retrieveValidCompactionStateRecordsWithQuery( Query> query ) @@ -1806,6 +1810,11 @@ private List retrieveValidCompactionStateRecordsWithQuery .collect(Collectors.toList()); } + /** + * Tries to parse the fields of the result set into a {@link CompactionStateRecord}. + * + * @return null if an error occurred while parsing the result + */ private CompactionStateRecord mapToCompactionStateRecord(ResultSet resultSet) { String fingerprint = null; diff --git a/server/src/main/java/org/apache/druid/metadata/segment/cache/CompactionStateRecord.java b/server/src/main/java/org/apache/druid/metadata/segment/cache/CompactionStateRecord.java index bb70fc76602a..5dd6e20e06e2 100644 --- a/server/src/main/java/org/apache/druid/metadata/segment/cache/CompactionStateRecord.java +++ b/server/src/main/java/org/apache/druid/metadata/segment/cache/CompactionStateRecord.java @@ -19,11 +19,8 @@ package org.apache.druid.metadata.segment.cache; -import com.fasterxml.jackson.databind.ObjectMapper; import org.apache.druid.timeline.CompactionState; -import java.sql.ResultSet; - /** * Represents a single record in the compaction_states table. */ @@ -47,26 +44,4 @@ public CompactionState getState() { return state; } - - /** - * Creates a CompactionStateRecord from a ResultSet row. - * Expected columns: fingerprint (String), payload (byte[]) - * - * @param rs ResultSet positioned at the row to read - * @param jsonMapper ObjectMapper for deserializing the payload - * @return CompactionStateRecord or null if deserialization fails - */ - public static CompactionStateRecord fromResultSet(ResultSet rs, ObjectMapper jsonMapper) - throws Exception - { - String fingerprint = rs.getString("fingerprint"); - byte[] payload = rs.getBytes("payload"); - - if (fingerprint == null || payload == null) { - return null; - } - - CompactionState state = jsonMapper.readValue(payload, CompactionState.class); - return new CompactionStateRecord(fingerprint, state); - } } 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 6e33edf16ef1..f2fa483aca0d 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 @@ -240,6 +240,9 @@ public void stop() datasourceToSegmentCache.forEach((datasource, cache) -> cache.stop()); datasourceToSegmentCache.clear(); datasourcesSnapshot.set(null); + if (useCompactionStateCache) { + compactionStateCache.clear(); + } syncFinishTime.set(null); updateCacheState(CacheState.STOPPED, "Stopped sync with metadata store"); @@ -273,9 +276,6 @@ public void stopBeingLeader() { synchronized (cacheStateLock) { if (isEnabled()) { - if (useCompactionStateCache) { - compactionStateCache.clear(); - } updateCacheState(CacheState.FOLLOWER, "Not leader anymore"); } } 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 4cbbdd63f54a..68484d0f107a 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 @@ -22,7 +22,11 @@ import org.apache.commons.lang3.ArrayUtils; import org.apache.druid.client.indexing.ClientCompactionTaskQueryTuningConfig; import org.apache.druid.common.config.Configs; +import org.apache.druid.data.input.impl.AggregateProjectionSpec; import org.apache.druid.data.input.impl.DimensionSchema; +import org.apache.druid.data.input.impl.DimensionsSpec; +import org.apache.druid.indexer.granularity.GranularitySpec; +import org.apache.druid.indexer.granularity.UniformGranularitySpec; import org.apache.druid.indexer.partitions.DimensionRangePartitionsSpec; import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; import org.apache.druid.indexer.partitions.HashedPartitionsSpec; @@ -38,7 +42,6 @@ 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; @@ -266,7 +269,7 @@ static CompactionStatus compute( @Nullable CompactionStateCache compactionStateCache ) { - final CompactionState expectedState = CompactSegments.createCompactionStateFromConfig(config); + final CompactionState expectedState = createCompactionStateFromConfig(config); String expectedFingerprint; if (compactionStateManager == null) { expectedFingerprint = null; @@ -811,4 +814,59 @@ private static CompactionStatistics createStats(List segments) return CompactionStatistics.create(totalBytes, segments.size(), segmentIntervals.size()); } } + + /** + * 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/CompactSegments.java b/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java index a65dd6d8e283..c3140ce59ae2 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,17 +34,12 @@ 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.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.transform.CompactionTransformSpec; import org.apache.druid.server.compaction.CompactionCandidate; import org.apache.druid.server.compaction.CompactionCandidateSearchPolicy; @@ -58,17 +53,14 @@ 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; @@ -487,59 +479,4 @@ 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 index 90a03ba3dfc9..8183e4c453df 100644 --- 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 @@ -27,6 +27,19 @@ import java.util.List; +/** + * Coordinator duty that cleans up old, unused compaction state entries from the database. + *

    + * This duty performs a three-step cleanup process: + *

      + *
    1. Marks compaction states not referenced by any segments as unused
    2. + *
    3. Repairs any incorrectly marked unused states that are still referenced by used segments
    4. + *
    5. Deletes unused compaction states older than the configured retention period
    6. + *
    + *

    + * This prevents unbounded growth of the compaction states table while ensuring that + * states referenced by active segments are preserved. + */ public class KillUnreferencedCompactionState extends MetadataCleanupDuty { private static final Logger log = new Logger(KillUnreferencedCompactionState.class); diff --git a/server/src/test/java/org/apache/druid/metadata/SqlSegmentsMetadataQueryTest.java b/server/src/test/java/org/apache/druid/metadata/SqlSegmentsMetadataQueryTest.java index 1084c888e01e..bf74eac5ea00 100644 --- a/server/src/test/java/org/apache/druid/metadata/SqlSegmentsMetadataQueryTest.java +++ b/server/src/test/java/org/apache/druid/metadata/SqlSegmentsMetadataQueryTest.java @@ -19,15 +19,22 @@ package org.apache.druid.metadata; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableSet; +import org.apache.druid.data.input.impl.DimensionsSpec; +import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.parsers.CloseableIterator; +import org.apache.druid.metadata.segment.cache.CompactionStateRecord; import org.apache.druid.metadata.storage.derby.DerbyConnector; +import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.TestDataSource; import org.apache.druid.segment.TestHelper; +import org.apache.druid.segment.metadata.PersistedCompactionStateManager; import org.apache.druid.server.coordinator.CreateDataSegments; +import org.apache.druid.timeline.CompactionState; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.SegmentId; import org.joda.time.DateTime; @@ -38,6 +45,8 @@ import org.junit.Rule; import org.junit.Test; +import java.util.HashMap; +import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; @@ -365,4 +374,358 @@ private static Set getIds(Set segments) { return segments.stream().map(DataSegment::getId).collect(Collectors.toSet()); } + + // ==================== Compaction State Tests ==================== + + @Test + public void test_retrieveAllUsedCompactionStateFingerprints_emptyDatabase() + { + derbyConnectorRule.getConnector().createCompactionStatesTable(); + + Set fingerprints = read(SqlSegmentsMetadataQuery::retrieveAllUsedCompactionStateFingerprints); + + Assert.assertTrue("Should return empty set when no segments have compaction states", fingerprints.isEmpty()); + } + + @Test + public void test_retrieveAllUsedCompactionStateFingerprints_onlyUsedSegments() + { + derbyConnectorRule.getConnector().createCompactionStatesTable(); + + // Insert compaction states + Map compactionStates = new HashMap<>(); + compactionStates.put("fp1", createTestCompactionState()); + compactionStates.put("fp2", createTestCompactionState()); + compactionStates.put("fp3", createTestCompactionState()); + insertCompactionStates(compactionStates); + + // Insert segments referencing compaction states + insertSegmentWithCompactionState("seg1", "fp1", true); + insertSegmentWithCompactionState("seg2", "fp2", true); + insertSegmentWithCompactionState("seg3", "fp1", true); // Duplicate fingerprint + insertSegmentWithCompactionState("seg4", "fp3", false); // Unused segment + + Set fingerprints = read(SqlSegmentsMetadataQuery::retrieveAllUsedCompactionStateFingerprints); + + Assert.assertEquals("Should return only fingerprints from used segments", Set.of("fp1", "fp2"), fingerprints); + } + + @Test + public void test_retrieveAllUsedCompactionStateFingerprints_ignoresNullFingerprints() + { + derbyConnectorRule.getConnector().createCompactionStatesTable(); + + Map compactionStates = new HashMap<>(); + compactionStates.put("fp1", createTestCompactionState()); + insertCompactionStates(compactionStates); + + insertSegmentWithCompactionState("seg1", "fp1", true); + insertSegmentWithCompactionState("seg2", null, true); // No compaction state + + Set fingerprints = read(SqlSegmentsMetadataQuery::retrieveAllUsedCompactionStateFingerprints); + + Assert.assertEquals("Should ignore segments without compaction states", Set.of("fp1"), fingerprints); + } + + @Test + public void test_retrieveAllUsedCompactionStates_emptyDatabase() + { + derbyConnectorRule.getConnector().createCompactionStatesTable(); + + List records = read(SqlSegmentsMetadataQuery::retrieveAllUsedCompactionStates); + + Assert.assertTrue("Should return empty list when no compaction states exist", records.isEmpty()); + } + + @Test + public void test_retrieveAllUsedCompactionStates_fullSync() + { + derbyConnectorRule.getConnector().createCompactionStatesTable(); + + // Create distinct compaction states + CompactionState state1 = createTestCompactionState(); + CompactionState state2 = new CompactionState( + new DynamicPartitionsSpec(200, null), + DimensionsSpec.EMPTY, + null, + null, + IndexSpec.getDefault(), + null, + null + ); + + Map compactionStates = new HashMap<>(); + compactionStates.put("fp1", state1); + compactionStates.put("fp2", state2); + compactionStates.put("fp3", createTestCompactionState()); // Unreferenced state + insertCompactionStates(compactionStates); + + // Only reference fp1 and fp2 + insertSegmentWithCompactionState("seg1", "fp1", true); + insertSegmentWithCompactionState("seg2", "fp2", true); + + List records = read(SqlSegmentsMetadataQuery::retrieveAllUsedCompactionStates); + + Assert.assertEquals("Should return only referenced compaction states", 2, records.size()); + + Set retrievedFingerprints = records.stream() + .map(CompactionStateRecord::getFingerprint) + .collect(Collectors.toSet()); + Assert.assertEquals("Should contain fp1 and fp2", Set.of("fp1", "fp2"), retrievedFingerprints); + + // Verify payloads + Map retrievedStates = records.stream() + .collect(Collectors.toMap( + CompactionStateRecord::getFingerprint, + CompactionStateRecord::getState + )); + Assert.assertEquals("fp1 state should match", state1, retrievedStates.get("fp1")); + Assert.assertEquals("fp2 state should match", state2, retrievedStates.get("fp2")); + } + + @Test + public void test_retrieveAllUsedCompactionStates_onlyFromUsedSegments() + { + derbyConnectorRule.getConnector().createCompactionStatesTable(); + + Map compactionStates = new HashMap<>(); + compactionStates.put("fp1", createTestCompactionState()); + compactionStates.put("fp2", createTestCompactionState()); + insertCompactionStates(compactionStates); + + insertSegmentWithCompactionState("seg1", "fp1", true); // Used + insertSegmentWithCompactionState("seg2", "fp2", false); // Unused + + List records = read(SqlSegmentsMetadataQuery::retrieveAllUsedCompactionStates); + + Assert.assertEquals("Should only return states from used segments", 1, records.size()); + Assert.assertEquals("Should return fp1", "fp1", records.get(0).getFingerprint()); + } + + @Test + public void test_retrieveAllUsedCompactionStates_ignoresUnusedCompactionStates() + { + derbyConnectorRule.getConnector().createCompactionStatesTable(); + + Map compactionStates = new HashMap<>(); + compactionStates.put("fp1", createTestCompactionState()); + insertCompactionStates(compactionStates); + + insertSegmentWithCompactionState("seg1", "fp1", true); + + // Mark compaction state as unused + markCompactionStateAsUnused("fp1"); + + List records = read(SqlSegmentsMetadataQuery::retrieveAllUsedCompactionStates); + + Assert.assertTrue("Should not return unused compaction states", records.isEmpty()); + } + + @Test + public void test_retrieveCompactionStatesForFingerprints_emptyInput() + { + derbyConnectorRule.getConnector().createCompactionStatesTable(); + + List records = read( + sql -> sql.retrieveCompactionStatesForFingerprints(Set.of()) + ); + + Assert.assertTrue("Should return empty list for empty input", records.isEmpty()); + } + + @Test + public void test_retrieveCompactionStatesForFingerprints_deltaSync() + { + derbyConnectorRule.getConnector().createCompactionStatesTable(); + + // Insert multiple compaction states + Map compactionStates = new HashMap<>(); + compactionStates.put("fp1", createTestCompactionState()); + compactionStates.put("fp2", createTestCompactionState()); + compactionStates.put("fp3", createTestCompactionState()); + insertCompactionStates(compactionStates); + + // Request specific fingerprints (delta sync scenario) + List records = read( + sql -> sql.retrieveCompactionStatesForFingerprints(Set.of("fp1", "fp3")) + ); + + Assert.assertEquals("Should return requested fingerprints", 2, records.size()); + + Set retrievedFingerprints = records.stream() + .map(CompactionStateRecord::getFingerprint) + .collect(Collectors.toSet()); + Assert.assertEquals("Should contain only requested fingerprints", Set.of("fp1", "fp3"), retrievedFingerprints); + } + + @Test + public void test_retrieveCompactionStatesForFingerprints_largeBatch() + { + derbyConnectorRule.getConnector().createCompactionStatesTable(); + + // Insert 150 compaction states (exceeds batching threshold of 100) + Map compactionStates = new HashMap<>(); + Set expectedFingerprints = new HashSet<>(); + for (int i = 0; i < 150; i++) { + String fingerprint = "fp" + i; + compactionStates.put(fingerprint, createTestCompactionState()); + expectedFingerprints.add(fingerprint); + } + insertCompactionStates(compactionStates); + + // Request all fingerprints + List records = read( + sql -> sql.retrieveCompactionStatesForFingerprints(expectedFingerprints) + ); + + Assert.assertEquals("Should return all fingerprints across multiple batches", 150, records.size()); + + Set retrievedFingerprints = records.stream() + .map(CompactionStateRecord::getFingerprint) + .collect(Collectors.toSet()); + Assert.assertEquals("Should contain all requested fingerprints", expectedFingerprints, retrievedFingerprints); + } + + @Test + public void test_retrieveCompactionStatesForFingerprints_nonexistentFingerprints() + { + derbyConnectorRule.getConnector().createCompactionStatesTable(); + + Map compactionStates = new HashMap<>(); + compactionStates.put("fp1", createTestCompactionState()); + insertCompactionStates(compactionStates); + + // Request fingerprints that don't exist + List records = read( + sql -> sql.retrieveCompactionStatesForFingerprints(Set.of("fp999", "fp888")) + ); + + Assert.assertTrue("Should return empty list when fingerprints don't exist", records.isEmpty()); + } + + @Test + public void test_retrieveCompactionStatesForFingerprints_mixedExistingAndNonexistent() + { + derbyConnectorRule.getConnector().createCompactionStatesTable(); + + Map compactionStates = new HashMap<>(); + compactionStates.put("fp1", createTestCompactionState()); + compactionStates.put("fp2", createTestCompactionState()); + insertCompactionStates(compactionStates); + + // Mix existing and non-existing fingerprints + List records = read( + sql -> sql.retrieveCompactionStatesForFingerprints(Set.of("fp1", "fp999", "fp2", "fp888")) + ); + + Assert.assertEquals("Should return only existing fingerprints", 2, records.size()); + + Set retrievedFingerprints = records.stream() + .map(CompactionStateRecord::getFingerprint) + .collect(Collectors.toSet()); + Assert.assertEquals("Should contain only existing fingerprints", Set.of("fp1", "fp2"), retrievedFingerprints); + } + + @Test + public void test_retrieveCompactionStatesForFingerprints_onlyReturnsUsedStates() + { + derbyConnectorRule.getConnector().createCompactionStatesTable(); + + Map compactionStates = new HashMap<>(); + compactionStates.put("fp1", createTestCompactionState()); + compactionStates.put("fp2", createTestCompactionState()); + insertCompactionStates(compactionStates); + + // Mark fp2 as unused + markCompactionStateAsUnused("fp2"); + + List records = read( + sql -> sql.retrieveCompactionStatesForFingerprints(Set.of("fp1", "fp2")) + ); + + Assert.assertEquals("Should only return used compaction states", 1, records.size()); + Assert.assertEquals("Should return fp1", "fp1", records.get(0).getFingerprint()); + } + + // ==================== Helper Methods for Compaction State Tests ==================== + + private CompactionState createTestCompactionState() + { + return new CompactionState( + new DynamicPartitionsSpec(100, null), + DimensionsSpec.EMPTY, + null, + null, + IndexSpec.getDefault(), + null, + null + ); + } + + private void insertCompactionStates(Map compactionStates) + { + ObjectMapper mapper = TestHelper.JSON_MAPPER; + MetadataStorageTablesConfig tablesConfig = derbyConnectorRule.metadataTablesConfigSupplier().get(); + PersistedCompactionStateManager manager = new PersistedCompactionStateManager( + tablesConfig, + mapper, + mapper, + derbyConnectorRule.getConnector() + ); + + derbyConnectorRule.getConnector().retryWithHandle(handle -> { + manager.persistCompactionState(TestDataSource.WIKI, compactionStates, DateTimes.nowUtc()); + return null; + }); + } + + private void insertSegmentWithCompactionState( + String segmentId, + String compactionStateFingerprint, + boolean used + ) + { + MetadataStorageTablesConfig tablesConfig = derbyConnectorRule.metadataTablesConfigSupplier().get(); + DerbyConnector connector = derbyConnectorRule.getConnector(); + + connector.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", segmentId) + .bind("dataSource", TestDataSource.WIKI) + .bind("created_date", DateTimes.nowUtc().toString()) + .bind("start", JAN_1.toString()) + .bind("end", JAN_1.plusDays(1).toString()) + .bind("partitioned", false) + .bind("version", V1) + .bind("used", used) + .bind("payload", TestHelper.JSON_MAPPER.writeValueAsBytes(WIKI_SEGMENTS_2X5D.get(0))) + .bind("used_status_last_updated", DateTimes.nowUtc().toString()) + .bind("compaction_state_fingerprint", compactionStateFingerprint) + .execute(); + return null; + }); + } + + private void markCompactionStateAsUnused(String fingerprint) + { + MetadataStorageTablesConfig tablesConfig = derbyConnectorRule.metadataTablesConfigSupplier().get(); + DerbyConnector connector = derbyConnectorRule.getConnector(); + + connector.retryWithHandle(handle -> { + handle.createStatement( + "UPDATE " + tablesConfig.getCompactionStatesTable() + " " + + "SET used = false " + + "WHERE fingerprint = :fingerprint" + ) + .bind("fingerprint", fingerprint) + .execute(); + return null; + }); + } } 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 fcfa34d17a5b..deb22e45fd7b 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 @@ -114,8 +114,7 @@ private void setupTargetWithCaching(SegmentMetadataCache.UsageMode cacheMode) /** * Creates the target {@link #cache} to be tested in the current test. */ - private void setupTargetWithCaching(SegmentMetadataCache.UsageMode cacheMode, boolean useSchemaCache - ) + private void setupTargetWithCaching(SegmentMetadataCache.UsageMode cacheMode, boolean useSchemaCache) { if (cache != null) { throw new ISE("Test target has already been initialized with caching[%s]", cache.isEnabled()); 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 33eec17a0be4..0cc5c3f094ba 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 @@ -48,7 +48,6 @@ 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; @@ -593,7 +592,7 @@ public void test_evaluate_needsCompactionWhenAllSegmentsHaveUnexpectedCompaction .forDataSource(TestDataSource.WIKI) .withGranularitySpec(new UserCompactionTaskGranularityConfig(Granularities.HOUR, null, null)) .build(); - CompactionState wrongState = CompactSegments.createCompactionStateFromConfig(oldCompactionConfig); + CompactionState wrongState = CompactionStatus.createCompactionStateFromConfig(oldCompactionConfig); final DataSourceCompactionConfig compactionConfig = InlineSchemaDataSourceCompactionConfig .builder() @@ -601,7 +600,7 @@ public void test_evaluate_needsCompactionWhenAllSegmentsHaveUnexpectedCompaction .withGranularitySpec(new UserCompactionTaskGranularityConfig(Granularities.DAY, null, null)) .build(); - CompactionState expectedState = CompactSegments.createCompactionStateFromConfig(compactionConfig); + CompactionState expectedState = CompactionStatus.createCompactionStateFromConfig(compactionConfig); compactionStateManager.persistCompactionState(TestDataSource.WIKI, Map.of("wrongFingerprint", wrongState), DateTimes.nowUtc()); syncCacheFromManager(); @@ -622,7 +621,7 @@ public void test_evaluate_needsCompactionWhenSomeSegmentsHaveUnexpectedCompactio .forDataSource(TestDataSource.WIKI) .withGranularitySpec(new UserCompactionTaskGranularityConfig(Granularities.HOUR, null, null)) .build(); - CompactionState wrongState = CompactSegments.createCompactionStateFromConfig(oldCompactionConfig); + CompactionState wrongState = CompactionStatus.createCompactionStateFromConfig(oldCompactionConfig); final DataSourceCompactionConfig compactionConfig = InlineSchemaDataSourceCompactionConfig .builder() @@ -630,7 +629,7 @@ public void test_evaluate_needsCompactionWhenSomeSegmentsHaveUnexpectedCompactio .withGranularitySpec(new UserCompactionTaskGranularityConfig(Granularities.DAY, null, null)) .build(); - CompactionState expectedState = CompactSegments.createCompactionStateFromConfig(compactionConfig); + CompactionState expectedState = CompactionStatus.createCompactionStateFromConfig(compactionConfig); String expectedFingerprint = compactionStateManager.generateCompactionStateFingerprint(expectedState, TestDataSource.WIKI); @@ -662,7 +661,7 @@ public void test_evaluate_noCompacationIfUnexpectedFingerprintHasExpectedCompact .withGranularitySpec(new UserCompactionTaskGranularityConfig(Granularities.HOUR, null, null)) .build(); - CompactionState expectedState = CompactSegments.createCompactionStateFromConfig(compactionConfig); + CompactionState expectedState = CompactionStatus.createCompactionStateFromConfig(compactionConfig); compactionStateManager.persistCompactionState(TestDataSource.WIKI, Map.of("wrongFingerprint", expectedState), DateTimes.nowUtc()); syncCacheFromManager(); @@ -724,7 +723,7 @@ public void test_evaluate_noCompactionWhenAllSegmentsHaveExpectedCompactionState .withGranularitySpec(new UserCompactionTaskGranularityConfig(Granularities.DAY, null, null)) .build(); - CompactionState expectedState = CompactSegments.createCompactionStateFromConfig(compactionConfig); + CompactionState expectedState = CompactionStatus.createCompactionStateFromConfig(compactionConfig); String expectedFingerprint = compactionStateManager.generateCompactionStateFingerprint(expectedState, TestDataSource.WIKI); @@ -751,7 +750,7 @@ public void test_evaluate_needsCompactionWhenNonFingerprintedSegmentsFailChecksO .withGranularitySpec(new UserCompactionTaskGranularityConfig(Granularities.DAY, null, null)) .build(); - CompactionState expectedState = CompactSegments.createCompactionStateFromConfig(compactionConfig); + CompactionState expectedState = CompactionStatus.createCompactionStateFromConfig(compactionConfig); String expectedFingerprint = compactionStateManager.generateCompactionStateFingerprint(expectedState, TestDataSource.WIKI); @@ -777,7 +776,7 @@ public void test_evaluate_noCompactionWhenNonFingerprintedSegmentsPassChecksOnLa .withGranularitySpec(new UserCompactionTaskGranularityConfig(Granularities.DAY, null, null)) .build(); - CompactionState expectedState = CompactSegments.createCompactionStateFromConfig(compactionConfig); + CompactionState expectedState = CompactionStatus.createCompactionStateFromConfig(compactionConfig); String expectedFingerprint = compactionStateManager.generateCompactionStateFingerprint(expectedState, TestDataSource.WIKI); 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 726025dde149..395d64ea3f17 100644 --- a/services/src/main/java/org/apache/druid/guice/MetadataManagerModule.java +++ b/services/src/main/java/org/apache/druid/guice/MetadataManagerModule.java @@ -132,24 +132,19 @@ public void configure(Binder binder) .in(LazySingleton.class); } - // Overlord-only compaction state dependencies - if (nodeRoles.contains(NodeRole.OVERLORD)) { - binder.bind(CompactionStateCache.class).in(LazySingleton.class); - } else { - binder.bind(CompactionStateCache.class) - .to(NoopCompactionStateCache.class) - .in(LazySingleton.class); - } - // Overlord-only dependencies if (nodeRoles.contains(NodeRole.OVERLORD)) { binder.bind(SegmentMetadataTransactionFactory.class) .to(SqlSegmentMetadataTransactionFactory.class) .in(LazySingleton.class); + binder.bind(CompactionStateCache.class).in(LazySingleton.class); } else { binder.bind(SegmentMetadataTransactionFactory.class) .to(SqlSegmentMetadataReadOnlyTransactionFactory.class) .in(LazySingleton.class); + binder.bind(CompactionStateCache.class) + .to(NoopCompactionStateCache.class) + .in(LazySingleton.class); } } } diff --git a/website/.spelling b/website/.spelling index ba35fa25a1a0..a6e7cf573889 100644 --- a/website/.spelling +++ b/website/.spelling @@ -483,7 +483,6 @@ pre-computation pre-compute pre-computed pre-computing -pre-warms preconfigured pre-existing pre-filtered From 0c1c6ed7be68665b45aa4d4723b9493a5b30d53a Mon Sep 17 00:00:00 2001 From: capistrant Date: Fri, 9 Jan 2026 13:02:11 -0600 Subject: [PATCH 35/72] Cleanup CompactionStateCache per review --- .../cache/HeapMemorySegmentMetadataCache.java | 2 +- .../metadata/CompactionStateCache.java | 21 +++++++++++++------ .../metadata/NoopCompactionStateCache.java | 2 +- .../metadata/CompactionStateCacheTest.java | 8 +++---- 4 files changed, 21 insertions(+), 12 deletions(-) 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 f2fa483aca0d..79c3de3ab0e2 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 @@ -1144,7 +1144,7 @@ private void retrieveAndResetUsedCompactionStates() compactionStateCache.resetCompactionStatesForPublishedSegments(fingerprintToStateMap); // Emit metrics for the current contents of the cache - compactionStateCache.getStats().forEach(this::emitMetric); + compactionStateCache.getAndResetStats().forEach(this::emitMetric); emitMetric(Metric.RETRIEVE_COMPACTION_STATES_DURATION_MILLIS, compactionStateSyncDuration.millisElapsed()); } diff --git a/server/src/main/java/org/apache/druid/segment/metadata/CompactionStateCache.java b/server/src/main/java/org/apache/druid/segment/metadata/CompactionStateCache.java index e3c614f252e5..0decfd3c68c0 100644 --- a/server/src/main/java/org/apache/druid/segment/metadata/CompactionStateCache.java +++ b/server/src/main/java/org/apache/druid/segment/metadata/CompactionStateCache.java @@ -89,13 +89,13 @@ public Optional getCompactionStateByFingerprint(String fingerpr CompactionState state = publishedCompactionStates.get() .fingerprintToStateMap .get(fingerprint); - if (state != null) { + if (state == null) { + cacheMissCount.incrementAndGet(); + return Optional.empty(); + } else { cacheHitCount.incrementAndGet(); return Optional.of(state); } - - cacheMissCount.incrementAndGet(); - return Optional.empty(); } /** @@ -113,13 +113,13 @@ public Map getPublishedCompactionStateMap() public void clear() { publishedCompactionStates.set(PublishedCompactionStates.EMPTY); - log.info("Cleared compaction state cache"); + resetStats(); } /** * @return Summary stats for metric emission */ - public Map getStats() + public Map getAndResetStats() { return Map.of( Metric.COMPACTION_STATE_CACHE_HITS, cacheHitCount.getAndSet(0), @@ -129,6 +129,15 @@ public Map getStats() ); } + /** + * Resets hit/miss stats. + */ + private void resetStats() + { + cacheHitCount.set(0); + cacheMissCount.set(0); + } + /** * Immutable snapshot of compaction states polled from DB. */ diff --git a/server/src/main/java/org/apache/druid/segment/metadata/NoopCompactionStateCache.java b/server/src/main/java/org/apache/druid/segment/metadata/NoopCompactionStateCache.java index c5247d5e710f..e7e013b2addb 100644 --- a/server/src/main/java/org/apache/druid/segment/metadata/NoopCompactionStateCache.java +++ b/server/src/main/java/org/apache/druid/segment/metadata/NoopCompactionStateCache.java @@ -62,7 +62,7 @@ public void clear() } @Override - public Map getStats() + public Map getAndResetStats() { return Collections.emptyMap(); } diff --git a/server/src/test/java/org/apache/druid/segment/metadata/CompactionStateCacheTest.java b/server/src/test/java/org/apache/druid/segment/metadata/CompactionStateCacheTest.java index 5238b2a4bf54..89603685e924 100644 --- a/server/src/test/java/org/apache/druid/segment/metadata/CompactionStateCacheTest.java +++ b/server/src/test/java/org/apache/druid/segment/metadata/CompactionStateCacheTest.java @@ -142,7 +142,7 @@ public void test_stats_trackHitsAndMisses() cache.getCompactionStateByFingerprint("nonexistent1"); cache.getCompactionStateByFingerprint("nonexistent2"); - Map stats = cache.getStats(); + Map stats = cache.getAndResetStats(); assertEquals(3, stats.get(Metric.COMPACTION_STATE_CACHE_HITS)); assertEquals(2, stats.get(Metric.COMPACTION_STATE_CACHE_MISSES)); assertEquals(1, stats.get(Metric.COMPACTION_STATE_CACHE_FINGERPRINTS)); @@ -161,12 +161,12 @@ public void test_stats_resetAfterReading() cache.getCompactionStateByFingerprint("fingerprint1"); cache.getCompactionStateByFingerprint("nonexistent"); - Map stats1 = cache.getStats(); + Map stats1 = cache.getAndResetStats(); assertEquals(1, stats1.get(Metric.COMPACTION_STATE_CACHE_HITS)); assertEquals(1, stats1.get(Metric.COMPACTION_STATE_CACHE_MISSES)); // Stats should be reset after reading - Map stats2 = cache.getStats(); + Map stats2 = cache.getAndResetStats(); assertEquals(0, stats2.get(Metric.COMPACTION_STATE_CACHE_HITS)); assertEquals(0, stats2.get(Metric.COMPACTION_STATE_CACHE_MISSES)); assertEquals(1, stats2.get(Metric.COMPACTION_STATE_CACHE_FINGERPRINTS)); // Fingerprints count doesn't reset @@ -220,7 +220,7 @@ public void test_resetWithEmptyMap() Optional afterReset = cache.getCompactionStateByFingerprint("fingerprint1"); assertFalse(afterReset.isPresent()); - Map stats = cache.getStats(); + Map stats = cache.getAndResetStats(); assertEquals(0, stats.get(Metric.COMPACTION_STATE_CACHE_FINGERPRINTS)); } From 6bd1875c4188eed9c7a5ca6326c1d5f59a92a164 Mon Sep 17 00:00:00 2001 From: capistrant Date: Fri, 9 Jan 2026 15:37:20 -0600 Subject: [PATCH 36/72] compactionstatemanager to compactionstatestorage plus refactor --- .../NewestSegmentFirstPolicyBenchmark.java | 4 +- .../compact/CompactionSupervisorTest.java | 8 +- .../CompactionConfigBasedJobTemplate.java | 4 +- .../indexing/compact/CompactionJobParams.java | 12 +- .../indexing/compact/CompactionJobQueue.java | 13 +- .../compact/OverlordCompactionScheduler.java | 10 +- .../OverlordCompactionSchedulerTest.java | 4 +- ...nager.java => CompactionStateStorage.java} | 56 ++-- ...er.java => SqlCompactionStateStorage.java} | 274 ++++++++---------- .../server/compaction/CompactionStatus.java | 8 +- .../DataSourceCompactibleSegmentIterator.java | 11 +- ...riorityBasedCompactionSegmentIterator.java | 6 +- .../server/coordinator/MetadataManager.java | 12 +- .../duty/KillUnreferencedCompactionState.java | 16 +- .../SqlSegmentsMetadataQueryTest.java | 8 +- ... => HeapMemoryCompactionStateStorage.java} | 40 ++- ...ava => SqlCompactionStateStorageTest.java} | 78 +++-- .../compaction/CompactionStatusTest.java | 61 ++-- .../NewestSegmentFirstPolicyTest.java | 16 +- .../coordinator/DruidCoordinatorTest.java | 8 +- .../coordinator/duty/CompactSegmentsTest.java | 8 +- .../KillUnreferencedCompactionStateTest.java | 32 +- .../CoordinatorSimulationBuilder.java | 4 +- .../druid/guice/MetadataManagerModule.java | 10 +- 24 files changed, 351 insertions(+), 352 deletions(-) rename server/src/main/java/org/apache/druid/segment/metadata/{CompactionStateManager.java => CompactionStateStorage.java} (66%) rename server/src/main/java/org/apache/druid/segment/metadata/{PersistedCompactionStateManager.java => SqlCompactionStateStorage.java} (54%) rename server/src/test/java/org/apache/druid/segment/metadata/{HeapMemoryCompactionStateManager.java => HeapMemoryCompactionStateStorage.java} (83%) rename server/src/test/java/org/apache/druid/segment/metadata/{PersistedCompactionStateManagerTest.java => SqlCompactionStateStorageTest.java} (89%) 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 2565cacdc282..02b3ebd49f1b 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,7 +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.segment.metadata.HeapMemoryCompactionStateStorage; import org.apache.druid.segment.metadata.NoopCompactionStateCache; import org.apache.druid.server.compaction.CompactionCandidateSearchPolicy; import org.apache.druid.server.compaction.CompactionSegmentIterator; @@ -138,7 +138,7 @@ public void measureNewestSegmentFirstPolicy(Blackhole blackhole) compactionConfigs, dataSources, Collections.emptyMap(), - new HeapMemoryCompactionStateManager(), + new HeapMemoryCompactionStateStorage(), new NoopCompactionStateCache() ); for (int i = 0; i < numCompactionTaskSlots && iterator.hasNext(); i++) { 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 fe602b5e3463..76893aa1d50e 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 @@ -31,7 +31,7 @@ import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.query.DruidMetrics; import org.apache.druid.rpc.UpdateResponse; -import org.apache.druid.segment.metadata.CompactionStateManager; +import org.apache.druid.segment.metadata.CompactionStateStorage; import org.apache.druid.server.compaction.CompactionStatus; import org.apache.druid.server.coordinator.ClusterCompactionConfig; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; @@ -287,11 +287,11 @@ private void verifySegmentsHaveNullLastCompactionStateAndNonNullFingerprint() private void verifyCompactedSegmentsHaveFingerprints(DataSourceCompactionConfig compactionConfig) { - CompactionStateManager compactionStateManager = overlord + CompactionStateStorage compactionStateStorage = overlord .bindings() - .getInstance(CompactionStateManager.class); + .getInstance(CompactionStateStorage.class); - String expectedFingerprint = compactionStateManager.generateCompactionStateFingerprint( + String expectedFingerprint = compactionStateStorage.generateCompactionStateFingerprint( CompactionStatus.createCompactionStateFromConfig(compactionConfig), dataSource ); 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 c0b74e811ed9..786b3f1bf3fc 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 @@ -74,7 +74,7 @@ public List createCompactionJobs( CompactionState compactionState = CompactionStatus.createCompactionStateFromConfig(config); - String compactionStateFingerprint = params.getCompactionStateManager().generateCompactionStateFingerprint( + String compactionStateFingerprint = params.getCompactionStateStorageImpl().generateCompactionStateFingerprint( compactionState, config.getDataSource() ); @@ -137,7 +137,7 @@ DataSourceCompactibleSegmentIterator getCompactibleCandidates( // This policy is used only while creating jobs // The actual order of jobs is determined by the policy used in CompactionJobQueue new NewestSegmentFirstPolicy(null), - params.getCompactionStateManager(), + params.getCompactionStateStorageImpl(), params.getCompactionStateCache() ); 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 f9cbe64e7448..bd9fec817615 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionJobParams.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionJobParams.java @@ -20,7 +20,7 @@ package org.apache.druid.indexing.compact; import org.apache.druid.segment.metadata.CompactionStateCache; -import org.apache.druid.segment.metadata.CompactionStateManager; +import org.apache.druid.segment.metadata.CompactionStateStorage; import org.apache.druid.server.compaction.CompactionSnapshotBuilder; import org.apache.druid.server.coordinator.ClusterCompactionConfig; import org.apache.druid.timeline.SegmentTimeline; @@ -35,7 +35,7 @@ public class CompactionJobParams private final TimelineProvider timelineProvider; private final ClusterCompactionConfig clusterCompactionConfig; private final CompactionSnapshotBuilder snapshotBuilder; - private final CompactionStateManager compactionStateManager; + private final CompactionStateStorage compactionStateStorage; private final CompactionStateCache compactionStateCache; public CompactionJobParams( @@ -43,7 +43,7 @@ public CompactionJobParams( ClusterCompactionConfig clusterCompactionConfig, TimelineProvider timelineProvider, CompactionSnapshotBuilder snapshotBuilder, - CompactionStateManager compactionStateManager, + CompactionStateStorage compactionStateStorage, CompactionStateCache compactionStateCache ) { @@ -51,7 +51,7 @@ public CompactionJobParams( this.clusterCompactionConfig = clusterCompactionConfig; this.timelineProvider = timelineProvider; this.snapshotBuilder = snapshotBuilder; - this.compactionStateManager = compactionStateManager; + this.compactionStateStorage = compactionStateStorage; this.compactionStateCache = compactionStateCache; } @@ -96,9 +96,9 @@ public CompactionSnapshotBuilder getSnapshotBuilder() return snapshotBuilder; } - public CompactionStateManager getCompactionStateManager() + public CompactionStateStorage getCompactionStateStorageImpl() { - return compactionStateManager; + return compactionStateStorage; } public CompactionStateCache getCompactionStateCache() 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 962eece9ad21..3f64b8d10fa7 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 @@ -37,7 +37,7 @@ import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.rpc.indexing.OverlordClient; import org.apache.druid.segment.metadata.CompactionStateCache; -import org.apache.druid.segment.metadata.CompactionStateManager; +import org.apache.druid.segment.metadata.CompactionStateStorage; import org.apache.druid.server.compaction.CompactionCandidate; import org.apache.druid.server.compaction.CompactionCandidateSearchPolicy; import org.apache.druid.server.compaction.CompactionSlotManager; @@ -107,7 +107,7 @@ public CompactionJobQueue( OverlordClient overlordClient, BrokerClient brokerClient, ObjectMapper objectMapper, - CompactionStateManager compactionStateManager, + CompactionStateStorage compactionStateStorage, CompactionStateCache compactionStateCache ) { @@ -125,7 +125,7 @@ public CompactionJobQueue( clusterCompactionConfig, dataSourcesSnapshot.getUsedSegmentsTimelinesPerDataSource()::get, snapshotBuilder, - compactionStateManager, + compactionStateStorage, compactionStateCache ); @@ -357,14 +357,15 @@ private String startTaskIfReady(CompactionJob job) } /** - * Persist the compaction state associated with the given job with {@link CompactionStateManager}. + * Persist the compaction state associated with the given job with {@link CompactionStateStorage}. */ private void persistPendingCompactionState(CompactionJob job) { if (job.getCompactionState() != null && job.getCompactionStateFingerprint() != null) { - jobParams.getCompactionStateManager().persistCompactionState( + jobParams.getCompactionStateStorageImpl().upsertCompactionState( job.getDataSource(), - Map.of(job.getCompactionStateFingerprint(), job.getCompactionState()), + job.getCompactionStateFingerprint(), + job.getCompactionState(), DateTimes.nowUtc() ); } 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 c42c97611ba6..d7a352f8fe0d 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/compact/OverlordCompactionScheduler.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/OverlordCompactionScheduler.java @@ -45,7 +45,7 @@ import org.apache.druid.metadata.SegmentsMetadataManager; import org.apache.druid.metadata.SegmentsMetadataManagerConfig; import org.apache.druid.segment.metadata.CompactionStateCache; -import org.apache.druid.segment.metadata.CompactionStateManager; +import org.apache.druid.segment.metadata.CompactionStateStorage; import org.apache.druid.server.compaction.CompactionRunSimulator; import org.apache.druid.server.compaction.CompactionSimulateResult; import org.apache.druid.server.compaction.CompactionStatusTracker; @@ -141,7 +141,7 @@ public class OverlordCompactionScheduler implements CompactionScheduler private final boolean shouldPollSegments; private final long schedulePeriodMillis; - private final CompactionStateManager compactionStateManager; + private final CompactionStateStorage compactionStateStorage; private final CompactionStateCache compactionStateCache; @Inject @@ -160,7 +160,7 @@ public OverlordCompactionScheduler( BrokerClient brokerClient, ServiceEmitter emitter, ObjectMapper objectMapper, - CompactionStateManager compactionStateManager, + CompactionStateStorage compactionStateStorage, CompactionStateCache compactionStateCache ) { @@ -187,7 +187,7 @@ public OverlordCompactionScheduler( this.taskActionClientFactory = taskActionClientFactory; this.druidInputSourceFactory = druidInputSourceFactory; - this.compactionStateManager = compactionStateManager; + this.compactionStateStorage = compactionStateStorage; this.compactionStateCache = compactionStateCache; this.taskRunnerListener = new TaskRunnerListener() { @@ -376,7 +376,7 @@ private synchronized void resetCompactionJobQueue() overlordClient, brokerClient, objectMapper, - compactionStateManager, + compactionStateStorage, compactionStateCache ); 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 75f151e33588..bda95b539f38 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 @@ -65,7 +65,7 @@ import org.apache.druid.query.http.SqlTaskStatus; import org.apache.druid.segment.TestIndex; import org.apache.druid.segment.metadata.CompactionStateCache; -import org.apache.druid.segment.metadata.HeapMemoryCompactionStateManager; +import org.apache.druid.segment.metadata.HeapMemoryCompactionStateStorage; import org.apache.druid.server.compaction.CompactionSimulateResult; import org.apache.druid.server.compaction.CompactionStatistics; import org.apache.druid.server.compaction.CompactionStatus; @@ -234,7 +234,7 @@ private void initScheduler() brokerClient, serviceEmitter, OBJECT_MAPPER, - new HeapMemoryCompactionStateManager(), + new HeapMemoryCompactionStateStorage(), new CompactionStateCache() ); } diff --git a/server/src/main/java/org/apache/druid/segment/metadata/CompactionStateManager.java b/server/src/main/java/org/apache/druid/segment/metadata/CompactionStateStorage.java similarity index 66% rename from server/src/main/java/org/apache/druid/segment/metadata/CompactionStateManager.java rename to server/src/main/java/org/apache/druid/segment/metadata/CompactionStateStorage.java index 8962074fc2ef..9dc4dcf132d7 100644 --- a/server/src/main/java/org/apache/druid/segment/metadata/CompactionStateManager.java +++ b/server/src/main/java/org/apache/druid/segment/metadata/CompactionStateStorage.java @@ -23,15 +23,14 @@ import org.joda.time.DateTime; import java.util.List; -import java.util.Map; /** * Manages compaction state persistence and fingerprint generation. *

    - * Implementations may be backed by a database (like {@link PersistedCompactionStateManager}) or - * use in-memory storage (like {@link HeapMemoryCompactionStateManager}). + * Implementations may be backed by a database (like {@link SqlCompactionStateStorage}) or + * use in-memory storage (like {@link HeapMemoryCompactionStateStorage}). */ -public interface CompactionStateManager +public interface CompactionStateStorage { /** * Generates a deterministic fingerprint for the given compaction state and datasource. @@ -44,28 +43,32 @@ public interface CompactionStateManager String generateCompactionStateFingerprint(CompactionState compactionState, String dataSource); /** - * Persists compaction states to storage. + * Upserts a compaction state to storage. + *

    + * If a fingerprint already exists, marks it as used and updates the timestamp. + * If a fingerprint doesn't exist, inserts a new row with the full state payload. * - * @param dataSource The datasource name - * @param fingerprintToStateMap Map of fingerprints to their compaction states - * @param updateTime The timestamp for this update + * @param dataSource The datasource name + * @param fingerprint The fingerprint of the compaction state + * @param compactionState The compaction state to upsert + * @param updateTime The timestamp for this update */ - void persistCompactionState( + + void upsertCompactionState( String dataSource, - Map fingerprintToStateMap, + String fingerprint, + CompactionState compactionState, DateTime updateTime ); /** * Marks compaction states as unused if they are not referenced by any used segments. - * This is used for cleanup operations. Implementations may choose to no-op this. + *

    + * This is used for cleanup operations. * * @return Number of rows updated, or 0 if not applicable */ - default int markUnreferencedCompactionStatesAsUnused() - { - return 0; - } + int markUnreferencedCompactionStatesAsUnused(); /** * Finds all compaction state fingerprints which have been marked as unused but are @@ -74,32 +77,25 @@ default int markUnreferencedCompactionStatesAsUnused() * * @return List of fingerprints, or empty list */ - default List findReferencedCompactionStateMarkedAsUnused() - { - return List.of(); - } + List findReferencedCompactionStateMarkedAsUnused(); /** - * Marks compaction states as used. This is used for reconciliation operations. - * Implementations may choose to no-op this. + * Marks compaction states as used. + *

    + * This is used for reconciliation operations to avoid deleting states that are still in use. * * @param stateFingerprints List of fingerprints to mark as used * @return Number of rows updated, or 0 if not applicable */ - default int markCompactionStatesAsUsed(List stateFingerprints) - { - return 0; - } + int markCompactionStatesAsUsed(List stateFingerprints); /** * Deletes unused compaction states older than the given timestamp. - * This is used for cleanup operations. Implementations may choose to no-op this. + *

    + * This is used for cleanup operations. * * @param timestamp The cutoff timestamp in milliseconds * @return Number of rows deleted, or 0 if not applicable */ - default int deleteUnusedCompactionStatesOlderThan(long timestamp) - { - return 0; - } + int deleteUnusedCompactionStatesOlderThan(long timestamp); } diff --git a/server/src/main/java/org/apache/druid/segment/metadata/PersistedCompactionStateManager.java b/server/src/main/java/org/apache/druid/segment/metadata/SqlCompactionStateStorage.java similarity index 54% rename from server/src/main/java/org/apache/druid/segment/metadata/PersistedCompactionStateManager.java rename to server/src/main/java/org/apache/druid/segment/metadata/SqlCompactionStateStorage.java index ed12ea220613..0b36bc3f530a 100644 --- a/server/src/main/java/org/apache/druid/segment/metadata/PersistedCompactionStateManager.java +++ b/server/src/main/java/org/apache/druid/segment/metadata/SqlCompactionStateStorage.java @@ -21,67 +21,52 @@ import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.annotations.VisibleForTesting; -import com.google.common.collect.Lists; import com.google.common.hash.Hasher; import com.google.common.hash.Hashing; import com.google.common.io.BaseEncoding; -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.guice.LazySingleton; import org.apache.druid.guice.annotations.Deterministic; 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 java.util.ArrayList; -import java.util.Collections; -import java.util.HashMap; -import java.util.HashSet; +import java.sql.SQLException; import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.concurrent.locks.Lock; /** - * Database-backed implementation of {@link CompactionStateManager}. + * Database-backed implementation of {@link CompactionStateStorage}. *

    * 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. + * Compaction states are uniquely identified by their fingerprints, which are SHA-256 hashes of their content. *

    *

    - * A striped locking mechanism is used to ensure thread-safe persistence of compaction states on a per-datasource basis. + * This implementation is designed to be called from a single thread (CompactionJobQueue) and relies on + * database constraints and the retry mechanism to handle any conflicts. Operations are idempotent - concurrent + * upserts for the same fingerprint will either succeed or fail with a constraint violation that is safely ignored. *

    */ -@ManageLifecycle -public class PersistedCompactionStateManager implements CompactionStateManager +@LazySingleton +public class SqlCompactionStateStorage implements CompactionStateStorage { - private static final EmittingLogger log = new EmittingLogger(PersistedCompactionStateManager.class); - private static final int DB_ACTION_PARTITION_SIZE = 100; + private static final EmittingLogger log = new EmittingLogger(SqlCompactionStateStorage.class); private final MetadataStorageTablesConfig dbTables; private final ObjectMapper jsonMapper; private final ObjectMapper deterministicMapper; private final SQLMetadataConnector connector; - private final Striped datasourceLocks = Striped.lock(128); @Inject - public PersistedCompactionStateManager( + public SqlCompactionStateStorage( @Nonnull MetadataStorageTablesConfig dbTables, @Nonnull ObjectMapper jsonMapper, @Deterministic @Nonnull ObjectMapper deterministicMapper, @@ -94,140 +79,93 @@ public PersistedCompactionStateManager( this.connector = connector; } - @LifecycleStart - public void start() - { - } - - @LifecycleStop - public void stop() - { - } - - @VisibleForTesting - PersistedCompactionStateManager() - { - this.dbTables = null; - this.jsonMapper = null; - this.deterministicMapper = null; - this.connector = null; - } - @Override - public void persistCompactionState( + public void upsertCompactionState( final String dataSource, - final Map fingerprintToStateMap, + final String fingerprint, + final CompactionState compactionState, final DateTime updateTime ) { - if (fingerprintToStateMap.isEmpty()) { + if (compactionState == null || fingerprint == null || fingerprint.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() - ); + // Check if the fingerprint already exists + final boolean fingerprintExists = isExistingFingerprint(handle, fingerprint); + final String now = updateTime.toString(); - if (!existingFingerprints.isEmpty()) { + if (fingerprintExists) { + // Fingerprint exists - update the used flag log.info( - "Found already existing compaction state in the DB for dataSource[%s]. Fingerprints: %s.", - dataSource, - existingFingerprints + "Found already existing compaction state in DB for fingerprint[%s] in dataSource[%s].", + fingerprint, + dataSource ); - String setFingerprintsUsedSql = StringUtils.format( + String updateSql = 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<>(); + handle.createStatement(updateSql) + .bind("used", true) + .bind("used_status_last_updated", now) + .bind("fingerprint", fingerprint) + .execute(); - for (Map.Entry entry : fingerprintToStateMap.entrySet()) { - if (!existingFingerprints.contains(entry.getKey())) { - statesToPersist.put(entry.getKey(), entry.getValue()); - } - } + log.info("Updated existing compaction state for datasource[%s].", dataSource); + } else { - if (statesToPersist.isEmpty()) { - log.info("No compaction state to persist for dataSource [%s].", dataSource); - return null; - } + // Fingerprint doesn't exist - insert new state + log.info("Inserting new compaction state for fingerprint[%s] in dataSource[%s].", fingerprint, dataSource); - 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() + ); - 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() - ); + try { + handle.createStatement(insertSql) + .bind("created_date", now) + .bind("datasource", dataSource) + .bind("fingerprint", fingerprint) + .bind("payload", jsonMapper.writeValueAsBytes(compactionState)) + .bind("used", true) + .bind("used_status_last_updated", now) + .execute(); - // 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, + "Published compaction state for fingerprint[%s] to DB for datasource[%s].", + fingerprint, dataSource ); - } else { - throw new ISE( - "Failed to publish compaction states[%s] to DB for datasource[%s]", - failedInserts, - dataSource + } + catch (JsonProcessingException e) { + throw InternalServerError.exception( + e, + "Failed to serialize compaction state for fingerprint[%s]", + fingerprint ); } } return null; }); } - finally { - lock.unlock(); + catch (Exception e) { + if (isUniqueConstraintViolation(e)) { + log.info( + "Fingerprints already exist for datasource[%s] (likely concurrent insert). " + + "Treating as success since operation is idempotent.", + dataSource + ); + // Swallow exception - another thread already persisted the same data + return; + } + // For other exceptions, let them propagate + throw e; } } @@ -327,38 +265,32 @@ public String generateCompactionStateFingerprint( /** - * Query the metadata DB to filter the fingerprints that already exist. - **/ - private Set getExistingFingerprints( + * Checks if a fingerprint already exists in the metadata DB. + * + * @param handle Database handle + * @param fingerprintToCheck The fingerprint to check + * @return true if the fingerprint exists, false otherwise + */ + private boolean isExistingFingerprint( final Handle handle, - final Set fingerprintsToInsert + @Nonnull final String fingerprintToCheck ) { - if (fingerprintsToInsert.isEmpty()) { - return Collections.emptySet(); + if (fingerprintToCheck.isEmpty()) { + return false; } - List> partitionedFingerprints = Lists.partition( - new ArrayList<>(fingerprintsToInsert), - DB_ACTION_PARTITION_SIZE + String sql = StringUtils.format( + "SELECT COUNT(*) FROM %s WHERE fingerprint = :fingerprint", + dbTables.getCompactionStatesTable() ); - 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); + Integer count = handle.createQuery(sql) + .bind("fingerprint", fingerprintToCheck) + .mapTo(Integer.class) + .first(); - query.map((index, r, ctx) -> existingFingerprints.add(r.getString(1))) - .list(); - } - return existingFingerprints; + return count != null && count > 0; } /** @@ -398,4 +330,40 @@ private static void bindValuesToInClause( query.bind(parameterPrefix + i, values.get(i)); } } + + /** + * Checks if an exception is a unique constraint violation. + * This is expected when multiple threads try to insert the same fingerprint concurrently. + * Since operations are idempotent, these violations can be safely ignored. + */ + private boolean isUniqueConstraintViolation(Exception e) + { + // Look for SQLException in the cause chain + Throwable cause = e; + while (cause != null) { + if (cause instanceof SQLException) { + SQLException sqlException = (SQLException) cause; + String sqlState = sqlException.getSQLState(); + + // SQL standard unique constraint violation codes + // 23505 = unique_violation (PostgreSQL, Derby) + // 23000 = integrity_constraint_violation (MySQL and others) + if ("23505".equals(sqlState) || "23000".equals(sqlState)) { + return true; + } + } + cause = cause.getCause(); + } + + // Also check exception message as fallback + String message = e.getMessage(); + if (message != null) { + String lowerMessage = message.toLowerCase(); + return lowerMessage.contains("unique constraint") + || lowerMessage.contains("duplicate key") + || lowerMessage.contains("duplicate entry"); + } + + return false; + } } 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 68484d0f107a..b5415460d47d 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 @@ -38,7 +38,7 @@ import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.metadata.CompactionStateCache; -import org.apache.druid.segment.metadata.CompactionStateManager; +import org.apache.druid.segment.metadata.CompactionStateStorage; import org.apache.druid.segment.transform.CompactionTransformSpec; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; import org.apache.druid.server.coordinator.UserCompactionTaskGranularityConfig; @@ -265,16 +265,16 @@ public static CompactionStatus running(String message) static CompactionStatus compute( CompactionCandidate candidateSegments, DataSourceCompactionConfig config, - @Nullable CompactionStateManager compactionStateManager, + @Nullable CompactionStateStorage compactionStateStorage, @Nullable CompactionStateCache compactionStateCache ) { final CompactionState expectedState = createCompactionStateFromConfig(config); String expectedFingerprint; - if (compactionStateManager == null) { + if (compactionStateStorage == null) { expectedFingerprint = null; } else { - expectedFingerprint = compactionStateManager.generateCompactionStateFingerprint( + expectedFingerprint = compactionStateStorage.generateCompactionStateFingerprint( expectedState, config.getDataSource() ); 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 5b6f070ee38f..fac7be005893 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 @@ -31,7 +31,7 @@ import org.apache.druid.java.util.common.guava.Comparators; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.segment.metadata.CompactionStateCache; -import org.apache.druid.segment.metadata.CompactionStateManager; +import org.apache.druid.segment.metadata.CompactionStateStorage; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.Partitions; @@ -70,7 +70,7 @@ public class DataSourceCompactibleSegmentIterator implements CompactionSegmentIt private final String dataSource; private final DataSourceCompactionConfig config; - private final CompactionStateManager compactionStateManager; + private final CompactionStateStorage compactionStateStorage; private final CompactionStateCache compactionStateCache; private final List compactedSegments = new ArrayList<>(); @@ -89,14 +89,14 @@ public DataSourceCompactibleSegmentIterator( SegmentTimeline timeline, List skipIntervals, CompactionCandidateSearchPolicy searchPolicy, - CompactionStateManager compactionStateManager, + CompactionStateStorage compactionStateStorage, CompactionStateCache compactionStateCache ) { this.config = config; this.dataSource = config.getDataSource(); this.queue = new PriorityQueue<>(searchPolicy::compareCandidates); - this.compactionStateManager = compactionStateManager; + this.compactionStateStorage = compactionStateStorage; this.compactionStateCache = compactionStateCache; populateQueue(timeline, skipIntervals); @@ -334,7 +334,8 @@ private void findAndEnqueueSegmentsToCompact(CompactibleSegmentIterator compacti } final CompactionCandidate candidates = CompactionCandidate.from(segments, config.getSegmentGranularity()); - final CompactionStatus compactionStatus = CompactionStatus.compute(candidates, config, compactionStateManager, compactionStateCache); + final CompactionStatus compactionStatus = CompactionStatus.compute(candidates, config, + compactionStateStorage, compactionStateCache); 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 3a7feda8667f..3f740ee3d8ee 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 @@ -23,7 +23,7 @@ import org.apache.druid.error.DruidException; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.segment.metadata.CompactionStateCache; -import org.apache.druid.segment.metadata.CompactionStateManager; +import org.apache.druid.segment.metadata.CompactionStateStorage; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; import org.apache.druid.timeline.SegmentTimeline; import org.joda.time.Interval; @@ -51,7 +51,7 @@ public PriorityBasedCompactionSegmentIterator( Map compactionConfigs, Map datasourceToTimeline, Map> skipIntervals, - CompactionStateManager compactionStateManager, + CompactionStateStorage compactionStateStorage, CompactionStateCache compactionStateCache ) { @@ -74,7 +74,7 @@ public PriorityBasedCompactionSegmentIterator( timeline, skipIntervals.getOrDefault(datasource, Collections.emptyList()), searchPolicy, - compactionStateManager, + compactionStateStorage, compactionStateCache ) ); 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 be7eea2f3079..15c7b1f6db90 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,7 +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.CompactionStateStorage; import org.apache.druid.segment.metadata.SegmentSchemaManager; import org.apache.druid.timeline.DataSegment; @@ -43,7 +43,7 @@ public class MetadataManager private final IndexerMetadataStorageCoordinator storageCoordinator; private final SegmentSchemaManager segmentSchemaManager; private final SegmentMetadataCache segmentMetadataCache; - private final CompactionStateManager compactionStateManager; + private final CompactionStateStorage compactionStateStorage; @Inject public MetadataManager( @@ -55,7 +55,7 @@ public MetadataManager( IndexerMetadataStorageCoordinator storageCoordinator, SegmentSchemaManager segmentSchemaManager, SegmentMetadataCache segmentMetadataCache, - CompactionStateManager compactionStateManager + CompactionStateStorage compactionStateStorage ) { this.auditManager = auditManager; @@ -66,7 +66,7 @@ public MetadataManager( this.storageCoordinator = storageCoordinator; this.segmentSchemaManager = segmentSchemaManager; this.segmentMetadataCache = segmentMetadataCache; - this.compactionStateManager = compactionStateManager; + this.compactionStateStorage = compactionStateStorage; } public void onLeaderStart() @@ -135,9 +135,9 @@ public SegmentSchemaManager schemas() return segmentSchemaManager; } - public CompactionStateManager compactionStates() + public CompactionStateStorage compactionStates() { - return compactionStateManager; + return compactionStateStorage; } /** 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 index 8183e4c453df..35ac3085e3f0 100644 --- 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 @@ -20,7 +20,7 @@ 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.segment.metadata.CompactionStateStorage; import org.apache.druid.server.coordinator.config.MetadataCleanupConfig; import org.apache.druid.server.coordinator.stats.Stats; import org.joda.time.DateTime; @@ -43,32 +43,32 @@ public class KillUnreferencedCompactionState extends MetadataCleanupDuty { private static final Logger log = new Logger(KillUnreferencedCompactionState.class); - private final CompactionStateManager compactionStateManager; + private final CompactionStateStorage compactionStateStorage; public KillUnreferencedCompactionState( MetadataCleanupConfig config, - CompactionStateManager compactionStateManager + CompactionStateStorage compactionStateStorage ) { super("compactionState", config, Stats.Kill.COMPACTION_STATE); - this.compactionStateManager = compactionStateManager; + this.compactionStateStorage = compactionStateStorage; } @Override protected int cleanupEntriesCreatedBefore(DateTime minCreatedTime) { // 1: Mark unreferenced states as unused - int unused = compactionStateManager.markUnreferencedCompactionStatesAsUnused(); + int unused = compactionStateStorage.markUnreferencedCompactionStatesAsUnused(); log.info("Marked [%s] unreferenced compaction states as unused.", unused); // 2: Repair - find unused states still referenced by segments - List stateFingerprints = compactionStateManager.findReferencedCompactionStateMarkedAsUnused(); + List stateFingerprints = compactionStateStorage.findReferencedCompactionStateMarkedAsUnused(); if (!stateFingerprints.isEmpty()) { - int numUpdated = compactionStateManager.markCompactionStatesAsUsed(stateFingerprints); + int numUpdated = compactionStateStorage.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()); + return compactionStateStorage.deleteUnusedCompactionStatesOlderThan(minCreatedTime.getMillis()); } } diff --git a/server/src/test/java/org/apache/druid/metadata/SqlSegmentsMetadataQueryTest.java b/server/src/test/java/org/apache/druid/metadata/SqlSegmentsMetadataQueryTest.java index bf74eac5ea00..acc055455c40 100644 --- a/server/src/test/java/org/apache/druid/metadata/SqlSegmentsMetadataQueryTest.java +++ b/server/src/test/java/org/apache/druid/metadata/SqlSegmentsMetadataQueryTest.java @@ -32,7 +32,7 @@ import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.TestDataSource; import org.apache.druid.segment.TestHelper; -import org.apache.druid.segment.metadata.PersistedCompactionStateManager; +import org.apache.druid.segment.metadata.SqlCompactionStateStorage; import org.apache.druid.server.coordinator.CreateDataSegments; import org.apache.druid.timeline.CompactionState; import org.apache.druid.timeline.DataSegment; @@ -666,7 +666,7 @@ private void insertCompactionStates(Map compactionState { ObjectMapper mapper = TestHelper.JSON_MAPPER; MetadataStorageTablesConfig tablesConfig = derbyConnectorRule.metadataTablesConfigSupplier().get(); - PersistedCompactionStateManager manager = new PersistedCompactionStateManager( + SqlCompactionStateStorage manager = new SqlCompactionStateStorage( tablesConfig, mapper, mapper, @@ -674,7 +674,9 @@ private void insertCompactionStates(Map compactionState ); derbyConnectorRule.getConnector().retryWithHandle(handle -> { - manager.persistCompactionState(TestDataSource.WIKI, compactionStates, DateTimes.nowUtc()); + for (Map.Entry entry : compactionStates.entrySet()) { + manager.upsertCompactionState(TestDataSource.WIKI, entry.getKey(), entry.getValue(), DateTimes.nowUtc()); + } return null; }); } diff --git a/server/src/test/java/org/apache/druid/segment/metadata/HeapMemoryCompactionStateManager.java b/server/src/test/java/org/apache/druid/segment/metadata/HeapMemoryCompactionStateStorage.java similarity index 83% rename from server/src/test/java/org/apache/druid/segment/metadata/HeapMemoryCompactionStateManager.java rename to server/src/test/java/org/apache/druid/segment/metadata/HeapMemoryCompactionStateStorage.java index 7171ec1e5a0c..12d2f2712ff3 100644 --- a/server/src/test/java/org/apache/druid/segment/metadata/HeapMemoryCompactionStateManager.java +++ b/server/src/test/java/org/apache/druid/segment/metadata/HeapMemoryCompactionStateStorage.java @@ -32,18 +32,19 @@ import org.joda.time.DateTime; import javax.annotation.Nullable; +import java.util.List; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; /** - * In-memory implementation of {@link CompactionStateManager} that stores + * In-memory implementation of {@link CompactionStateStorage} that stores * compaction state fingerprints in heap memory without requiring a database. *

    * Useful for simulations and unit tests where database persistence is not needed. * Database-specific operations (cleanup, unused marking) are no-ops in this implementation. */ -public class HeapMemoryCompactionStateManager implements CompactionStateManager +public class HeapMemoryCompactionStateStorage implements CompactionStateStorage { private final ConcurrentMap fingerprintToStateMap = new ConcurrentHashMap<>(); private final ObjectMapper deterministicMapper; @@ -52,7 +53,7 @@ public class HeapMemoryCompactionStateManager implements CompactionStateManager * Creates an in-memory compaction state manager with a default deterministic mapper. * This is a convenience constructor for tests and simulations. */ - public HeapMemoryCompactionStateManager() + public HeapMemoryCompactionStateStorage() { this(createDeterministicMapper()); } @@ -63,7 +64,7 @@ public HeapMemoryCompactionStateManager() * * @param deterministicMapper ObjectMapper configured for deterministic serialization */ - public HeapMemoryCompactionStateManager(ObjectMapper deterministicMapper) + public HeapMemoryCompactionStateStorage(ObjectMapper deterministicMapper) { this.deterministicMapper = deterministicMapper; } @@ -104,14 +105,39 @@ public String generateCompactionStateFingerprint( } @Override - public void persistCompactionState( + public void upsertCompactionState( final String dataSource, - final Map fingerprintToStateMap, + final String fingerprint, + final CompactionState compactionState, final DateTime updateTime ) { // Store in memory for lookup during simulations/tests - this.fingerprintToStateMap.putAll(fingerprintToStateMap); + this.fingerprintToStateMap.put(fingerprint, compactionState); + } + + @Override + public int markUnreferencedCompactionStatesAsUnused() + { + return 0; + } + + @Override + public List findReferencedCompactionStateMarkedAsUnused() + { + return List.of(); + } + + @Override + public int markCompactionStatesAsUsed(List stateFingerprints) + { + return 0; + } + + @Override + public int deleteUnusedCompactionStatesOlderThan(long timestamp) + { + return 0; } /** diff --git a/server/src/test/java/org/apache/druid/segment/metadata/PersistedCompactionStateManagerTest.java b/server/src/test/java/org/apache/druid/segment/metadata/SqlCompactionStateStorageTest.java similarity index 89% rename from server/src/test/java/org/apache/druid/segment/metadata/PersistedCompactionStateManagerTest.java rename to server/src/test/java/org/apache/druid/segment/metadata/SqlCompactionStateStorageTest.java index 187d80d41d38..58ea4ea181da 100644 --- a/server/src/test/java/org/apache/druid/segment/metadata/PersistedCompactionStateManagerTest.java +++ b/server/src/test/java/org/apache/druid/segment/metadata/SqlCompactionStateStorageTest.java @@ -43,15 +43,13 @@ import java.util.Arrays; import java.util.Collections; -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.assertNotEquals; import static org.junit.jupiter.api.Assertions.assertTrue; -public class PersistedCompactionStateManagerTest +public class SqlCompactionStateStorageTest { @RegisterExtension public static final TestDerbyConnector.DerbyConnectorRule5 DERBY_CONNECTOR_RULE = @@ -62,7 +60,7 @@ public class PersistedCompactionStateManagerTest private static TestDerbyConnector derbyConnector; private static MetadataStorageTablesConfig tablesConfig; - private PersistedCompactionStateManager manager; + private SqlCompactionStateStorage manager; @BeforeAll public static void setUpClass() @@ -82,22 +80,20 @@ public void setUp() return null; }); - manager = new PersistedCompactionStateManager(tablesConfig, jsonMapper, deterministicMapper, derbyConnector); + manager = new SqlCompactionStateStorage(tablesConfig, jsonMapper, deterministicMapper, derbyConnector); } @Test - public void test_persistCompactionState_successfullyInsertsIntoDatabase() + public void test_upsertCompactionState_successfullyInsertsIntoDatabase() { CompactionState state1 = createTestCompactionState(); String fingerprint = "fingerprint_abc123"; - Map fingerprintMap = new HashMap<>(); - fingerprintMap.put(fingerprint, state1); - derbyConnector.retryWithHandle(handle -> { - manager.persistCompactionState( + manager.upsertCompactionState( "testDatasource", - fingerprintMap, + fingerprint, + state1, DateTimes.nowUtc() ); return null; @@ -116,18 +112,16 @@ public void test_persistCompactionState_successfullyInsertsIntoDatabase() } @Test - public void test_persistCompactionState_andThen_markUnreferencedCompactionStateAsUnused_andThen_markCompactionStatesAsUsed() + public void test_upsertCompactionState_andThen_markUnreferencedCompactionStateAsUnused_andThen_markCompactionStatesAsUsed() { CompactionState state1 = createTestCompactionState(); String fingerprint = "fingerprint_abc123"; - Map fingerprintMap = new HashMap<>(); - fingerprintMap.put(fingerprint, state1); - derbyConnector.retryWithHandle(handle -> { - manager.persistCompactionState( + manager.upsertCompactionState( "testDatasource", - fingerprintMap, + fingerprint, + state1, DateTimes.nowUtc() ); return null; @@ -142,13 +136,11 @@ public void test_findReferencedCompactionStateMarkedAsUnused() CompactionState state1 = createTestCompactionState(); String fingerprint = "fingerprint_abc123"; - Map fingerprintMap = new HashMap<>(); - fingerprintMap.put(fingerprint, state1); - derbyConnector.retryWithHandle(handle -> { - manager.persistCompactionState( + manager.upsertCompactionState( "testDatasource", - fingerprintMap, + fingerprint, + state1, DateTimes.nowUtc() ); return null; @@ -185,7 +177,7 @@ public void test_findReferencedCompactionStateMarkedAsUnused() } @Test - public void test_deleteCompactionStatesOlderThan_deletesOnlyOldUnusedStates() throws Exception + public void test_deleteCompactionStatesOlderThan_deletesOnlyOldUnusedStates() { DateTime now = DateTimes.nowUtc(); DateTime oldTime = now.minusDays(60); @@ -246,7 +238,33 @@ public void test_deleteCompactionStatesOlderThan_deletesOnlyOldUnusedStates() th } @Test - public void test_persistCompactionState_withEmptyMap_doesNothing() + public void test_upsertCompactionState_withNullState_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.upsertCompactionState("ds", "somePrint", null, 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_upsertCompactionState_withEmptyPrint_doesNothing() { // Get initial count Integer beforeCount = derbyConnector.retryWithHandle(handle -> @@ -257,7 +275,7 @@ public void test_persistCompactionState_withEmptyMap_doesNothing() // Persist empty map derbyConnector.retryWithHandle(handle -> { - manager.persistCompactionState("ds", new HashMap<>(), DateTimes.nowUtc()); + manager.upsertCompactionState("ds", "", createBasicCompactionState(), DateTimes.nowUtc()); return null; }); @@ -272,16 +290,14 @@ public void test_persistCompactionState_withEmptyMap_doesNothing() } @Test - public void test_persistCompactionState_verifyExistingFingerprintMarkedUsed() throws Exception + public void test_upsertCompactionState_verifyExistingFingerprintMarkedUsed() { 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()); + manager.upsertCompactionState("ds1", fingerprint, state, DateTimes.nowUtc()); return null; }); @@ -306,9 +322,7 @@ public void test_persistCompactionState_verifyExistingFingerprintMarkedUsed() th // 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()); + manager.upsertCompactionState("ds1", fingerprint, state, DateTimes.nowUtc()); return null; }); 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 0cc5c3f094ba..619946298179 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 @@ -40,8 +40,8 @@ import org.apache.druid.segment.TestDataSource; import org.apache.druid.segment.data.CompressionStrategy; import org.apache.druid.segment.metadata.CompactionStateCache; -import org.apache.druid.segment.metadata.CompactionStateManager; -import org.apache.druid.segment.metadata.HeapMemoryCompactionStateManager; +import org.apache.druid.segment.metadata.CompactionStateStorage; +import org.apache.druid.segment.metadata.HeapMemoryCompactionStateStorage; import org.apache.druid.segment.nested.NestedCommonFormatColumnFormatSpec; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; import org.apache.druid.server.coordinator.InlineSchemaDataSourceCompactionConfig; @@ -57,7 +57,6 @@ import java.util.Collections; import java.util.List; -import java.util.Map; public class CompactionStatusTest { @@ -70,13 +69,13 @@ public class CompactionStatusTest .size(100_000_000L) .build(); - private HeapMemoryCompactionStateManager compactionStateManager; + private HeapMemoryCompactionStateStorage compactionStateStorage; private CompactionStateCache compactionStateCache; @Before public void setUp() { - compactionStateManager = new HeapMemoryCompactionStateManager(); + compactionStateStorage = new HeapMemoryCompactionStateStorage(); compactionStateCache = new CompactionStateCache(); } @@ -85,7 +84,7 @@ public void setUp() */ private void syncCacheFromManager() { - compactionStateCache.resetCompactionStatesForPublishedSegments(compactionStateManager.getAllStoredStates()); + compactionStateCache.resetCompactionStatesForPublishedSegments(compactionStateStorage.getAllStoredStates()); } @Test @@ -356,7 +355,7 @@ public void testStatusWhenLastCompactionStateSameAsRequired() final CompactionStatus status = CompactionStatus.compute( CompactionCandidate.from(List.of(segment), Granularities.HOUR), compactionConfig, - compactionStateManager, + compactionStateStorage, compactionStateCache ); Assert.assertTrue(status.isComplete()); @@ -407,7 +406,7 @@ public void testStatusWhenProjectionsMatch() final CompactionStatus status = CompactionStatus.compute( CompactionCandidate.from(List.of(segment), Granularities.HOUR), compactionConfig, - compactionStateManager, + compactionStateStorage, compactionStateCache ); Assert.assertTrue(status.isComplete()); @@ -463,7 +462,7 @@ public void testStatusWhenProjectionsMismatch() final CompactionStatus status = CompactionStatus.compute( CompactionCandidate.from(List.of(segment), Granularities.HOUR), compactionConfig, - compactionStateManager, + compactionStateStorage, compactionStateCache ); Assert.assertFalse(status.isComplete()); @@ -518,7 +517,7 @@ public void testStatusWhenAutoSchemaMatch() final CompactionStatus status = CompactionStatus.compute( CompactionCandidate.from(List.of(segment), null), compactionConfig, - compactionStateManager, + compactionStateStorage, compactionStateCache ); Assert.assertTrue(status.isComplete()); @@ -573,7 +572,7 @@ public void testStatusWhenAutoSchemaMismatch() final CompactionStatus status = CompactionStatus.compute( CompactionCandidate.from(List.of(segment), null), compactionConfig, - compactionStateManager, + compactionStateStorage, compactionStateCache ); Assert.assertFalse(status.isComplete()); @@ -602,14 +601,14 @@ public void test_evaluate_needsCompactionWhenAllSegmentsHaveUnexpectedCompaction CompactionState expectedState = CompactionStatus.createCompactionStateFromConfig(compactionConfig); - compactionStateManager.persistCompactionState(TestDataSource.WIKI, Map.of("wrongFingerprint", wrongState), DateTimes.nowUtc()); + compactionStateStorage.upsertCompactionState(TestDataSource.WIKI, "wrongFingerprint", wrongState, DateTimes.nowUtc()); syncCacheFromManager(); verifyEvaluationNeedsCompactionBecauseWithCustomSegments( CompactionCandidate.from(segments, null), compactionConfig, "'segmentGranularity' mismatch: required[DAY], current[HOUR]", - compactionStateManager + compactionStateStorage ); } @@ -631,21 +630,21 @@ public void test_evaluate_needsCompactionWhenSomeSegmentsHaveUnexpectedCompactio CompactionState expectedState = CompactionStatus.createCompactionStateFromConfig(compactionConfig); - String expectedFingerprint = compactionStateManager.generateCompactionStateFingerprint(expectedState, TestDataSource.WIKI); + String expectedFingerprint = compactionStateStorage.generateCompactionStateFingerprint(expectedState, TestDataSource.WIKI); List segments = List.of( DataSegment.builder(WIKI_SEGMENT).compactionStateFingerprint(expectedFingerprint).build(), DataSegment.builder(WIKI_SEGMENT_2).compactionStateFingerprint("wrongFingerprint").build() ); - compactionStateManager.persistCompactionState(TestDataSource.WIKI, Map.of("wrongFingerprint", wrongState), DateTimes.nowUtc()); + compactionStateStorage.upsertCompactionState(TestDataSource.WIKI, "wrongFingerprint", wrongState, DateTimes.nowUtc()); syncCacheFromManager(); verifyEvaluationNeedsCompactionBecauseWithCustomSegments( CompactionCandidate.from(segments, null), compactionConfig, "'segmentGranularity' mismatch: required[DAY], current[HOUR]", - compactionStateManager + compactionStateStorage ); } @@ -662,20 +661,20 @@ public void test_evaluate_noCompacationIfUnexpectedFingerprintHasExpectedCompact .build(); CompactionState expectedState = CompactionStatus.createCompactionStateFromConfig(compactionConfig); - compactionStateManager.persistCompactionState(TestDataSource.WIKI, Map.of("wrongFingerprint", expectedState), DateTimes.nowUtc()); + compactionStateStorage.upsertCompactionState(TestDataSource.WIKI, "wrongFingerprint", expectedState, DateTimes.nowUtc()); syncCacheFromManager(); final CompactionStatus status = CompactionStatus.compute( CompactionCandidate.from(segments, null), compactionConfig, - compactionStateManager, + compactionStateStorage, compactionStateCache ); Assert.assertTrue(status.isComplete()); } @Test - public void test_evaluate_needsCompactionWhenUnexpectedFingerprintAndNullCompactionStateManager() + public void test_evaluate_needsCompactionWhenUnexpectedFingerprintAndNullCompactionStateStorage() { List segments = List.of( DataSegment.builder(WIKI_SEGMENT).compactionStateFingerprint("wrongFingerprint").build() @@ -690,7 +689,7 @@ public void test_evaluate_needsCompactionWhenUnexpectedFingerprintAndNullCompact CompactionCandidate.from(segments, null), compactionConfig, "At least one segment has a mismatched fingerprint and needs compaction", - compactionStateManager + compactionStateStorage ); } @@ -710,7 +709,7 @@ public void test_evaluate_needsCompactionWhenUnexpectedFingerprintAndNoFingerpri CompactionCandidate.from(segments, null), compactionConfig, "At least one segment has a mismatched fingerprint and needs compaction", - compactionStateManager + compactionStateStorage ); } @@ -725,7 +724,7 @@ public void test_evaluate_noCompactionWhenAllSegmentsHaveExpectedCompactionState CompactionState expectedState = CompactionStatus.createCompactionStateFromConfig(compactionConfig); - String expectedFingerprint = compactionStateManager.generateCompactionStateFingerprint(expectedState, TestDataSource.WIKI); + String expectedFingerprint = compactionStateStorage.generateCompactionStateFingerprint(expectedState, TestDataSource.WIKI); List segments = List.of( DataSegment.builder(WIKI_SEGMENT).compactionStateFingerprint(expectedFingerprint).build(), @@ -735,7 +734,7 @@ public void test_evaluate_noCompactionWhenAllSegmentsHaveExpectedCompactionState final CompactionStatus status = CompactionStatus.compute( CompactionCandidate.from(segments, null), compactionConfig, - compactionStateManager, + compactionStateStorage, compactionStateCache ); Assert.assertTrue(status.isComplete()); @@ -752,7 +751,7 @@ public void test_evaluate_needsCompactionWhenNonFingerprintedSegmentsFailChecksO CompactionState expectedState = CompactionStatus.createCompactionStateFromConfig(compactionConfig); - String expectedFingerprint = compactionStateManager.generateCompactionStateFingerprint(expectedState, TestDataSource.WIKI); + String expectedFingerprint = compactionStateStorage.generateCompactionStateFingerprint(expectedState, TestDataSource.WIKI); List segments = List.of( DataSegment.builder(WIKI_SEGMENT).compactionStateFingerprint(expectedFingerprint).build(), @@ -763,7 +762,7 @@ public void test_evaluate_needsCompactionWhenNonFingerprintedSegmentsFailChecksO CompactionCandidate.from(segments, null), compactionConfig, "'segmentGranularity' mismatch: required[DAY], current[HOUR]", - compactionStateManager + compactionStateStorage ); } @@ -778,7 +777,7 @@ public void test_evaluate_noCompactionWhenNonFingerprintedSegmentsPassChecksOnLa CompactionState expectedState = CompactionStatus.createCompactionStateFromConfig(compactionConfig); - String expectedFingerprint = compactionStateManager.generateCompactionStateFingerprint(expectedState, TestDataSource.WIKI); + String expectedFingerprint = compactionStateStorage.generateCompactionStateFingerprint(expectedState, TestDataSource.WIKI); List segments = List.of( DataSegment.builder(WIKI_SEGMENT).compactionStateFingerprint(expectedFingerprint).build(), @@ -788,7 +787,7 @@ public void test_evaluate_noCompactionWhenNonFingerprintedSegmentsPassChecksOnLa final CompactionStatus status = CompactionStatus.compute( CompactionCandidate.from(segments, null), compactionConfig, - compactionStateManager, + compactionStateStorage, compactionStateCache ); Assert.assertTrue(status.isComplete()); @@ -819,7 +818,7 @@ public void test_evaluate_isSkippedWhenInputBytesExceedLimit() final CompactionStatus status = CompactionStatus.compute( CompactionCandidate.from(segments, null), compactionConfig, - compactionStateManager, + compactionStateStorage, compactionStateCache ); @@ -838,13 +837,13 @@ private void verifyEvaluationNeedsCompactionBecauseWithCustomSegments( CompactionCandidate candidate, DataSourceCompactionConfig compactionConfig, String expectedReason, - CompactionStateManager compactionStateManager + CompactionStateStorage compactionStateStorage ) { final CompactionStatus status = CompactionStatus.compute( candidate, compactionConfig, - compactionStateManager, + compactionStateStorage, compactionStateCache ); @@ -865,7 +864,7 @@ private void verifyCompactionStatusIsPendingBecause( final CompactionStatus status = CompactionStatus.compute( CompactionCandidate.from(List.of(segment), null), compactionConfig, - compactionStateManager, + compactionStateStorage, compactionStateCache ); 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 412b6041a982..bb132a302dc6 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 @@ -48,8 +48,8 @@ import org.apache.druid.segment.data.ConciseBitmapSerdeFactory; import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.segment.metadata.CompactionStateCache; -import org.apache.druid.segment.metadata.CompactionStateManager; -import org.apache.druid.segment.metadata.HeapMemoryCompactionStateManager; +import org.apache.druid.segment.metadata.CompactionStateStorage; +import org.apache.druid.segment.metadata.HeapMemoryCompactionStateStorage; import org.apache.druid.segment.metadata.NoopCompactionStateCache; import org.apache.druid.segment.transform.CompactionTransformSpec; import org.apache.druid.server.coordinator.CreateDataSegments; @@ -85,7 +85,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(); + private final CompactionStateStorage compactionStateStorage = new HeapMemoryCompactionStateStorage(); private final CompactionStateCache compactionStateCache = new NoopCompactionStateCache(); @Test @@ -283,7 +283,7 @@ public void testSkipDataSourceWithNoSegments() ) ), Collections.emptyMap(), - compactionStateManager, + compactionStateStorage, compactionStateCache ); @@ -517,7 +517,7 @@ public void testWithSkipIntervals() Intervals.of("2017-11-13T00:00:00/2017-11-14T01:00:00") ) ), - compactionStateManager, + compactionStateStorage, compactionStateCache ); @@ -558,7 +558,7 @@ public void testHoleInSearchInterval() Intervals.of("2017-11-16T14:00:00/2017-11-16T20:00:00") ) ), - compactionStateManager, + compactionStateStorage, compactionStateCache ); @@ -2065,7 +2065,7 @@ TestDataSource.KOALA, configBuilder().forDataSource(TestDataSource.KOALA).build( TestDataSource.KOALA, SegmentTimeline.forSegments(koalaSegments) ), Collections.emptyMap(), - compactionStateManager, + compactionStateStorage, compactionStateCache ); @@ -2088,7 +2088,7 @@ private CompactionSegmentIterator createIterator(DataSourceCompactionConfig conf Collections.singletonMap(TestDataSource.WIKI, config), Collections.singletonMap(TestDataSource.WIKI, timeline), Collections.emptyMap(), - compactionStateManager, + compactionStateStorage, compactionStateCache ); } 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 9c3de8c3b2a2..90a3fdd1ff11 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,7 +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.PersistedCompactionStateManager; +import org.apache.druid.segment.metadata.SqlCompactionStateStorage; import org.apache.druid.server.DruidNode; import org.apache.druid.server.compaction.CompactionSimulateResult; import org.apache.druid.server.compaction.CompactionStatusTracker; @@ -113,7 +113,7 @@ public class DruidCoordinatorTest private OverlordClient overlordClient; private CompactionStatusTracker statusTracker; private LatchableServiceEmitter serviceEmitter; - private PersistedCompactionStateManager persistedCompactionStateManager; + private SqlCompactionStateStorage sqlCompactionStateStorage; @Before public void setUp() throws Exception @@ -124,7 +124,7 @@ public void setUp() throws Exception metadataRuleManager = EasyMock.createNiceMock(MetadataRuleManager.class); loadQueueTaskMaster = EasyMock.createMock(LoadQueueTaskMaster.class); overlordClient = EasyMock.createMock(OverlordClient.class); - persistedCompactionStateManager = EasyMock.createMock(PersistedCompactionStateManager.class); + sqlCompactionStateStorage = EasyMock.createMock(SqlCompactionStateStorage.class); JacksonConfigManager configManager = EasyMock.createNiceMock(JacksonConfigManager.class); EasyMock.expect( @@ -188,7 +188,7 @@ private MetadataManager createMetadataManager(JacksonConfigManager configManager null, null, NoopSegmentMetadataCache.instance(), - persistedCompactionStateManager + sqlCompactionStateStorage ); } 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 a2562240c028..b8ac5ad111ef 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,8 +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.metadata.CompactionStateStorage; +import org.apache.druid.segment.metadata.HeapMemoryCompactionStateStorage; import org.apache.druid.segment.transform.CompactionTransformSpec; import org.apache.druid.server.compaction.CompactionCandidate; import org.apache.druid.server.compaction.CompactionCandidateSearchPolicy; @@ -199,7 +199,7 @@ public static Collection constructorFeeder() private DataSourcesSnapshot dataSources; private CompactionStatusTracker statusTracker; private final Map> datasourceToSegments = new HashMap<>(); - private final CompactionStateManager compactionStateManager = new HeapMemoryCompactionStateManager(); + private final CompactionStateStorage compactionStateStorage = new HeapMemoryCompactionStateStorage(); public CompactSegmentsTest( PartitionsSpec partitionsSpec, @@ -276,7 +276,7 @@ public void testSerde() throws Exception .addValue(DruidCoordinatorConfig.class, COORDINATOR_CONFIG) .addValue(OverlordClient.class, overlordClient) .addValue(CompactionStatusTracker.class, statusTracker) - .addValue(CompactionStateManager.class, compactionStateManager) + .addValue(CompactionStateStorage.class, compactionStateStorage) .addValue(MetadataCatalog.class, NullMetadataCatalog.INSTANCE) .addValue(ExprMacroTable.class, TestExprMacroTable.INSTANCE) ); 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 index daf1a259c3a1..3be3fd216eb5 100644 --- 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 @@ -28,8 +28,8 @@ 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.PersistedCompactionStateManager; +import org.apache.druid.segment.metadata.CompactionStateStorage; +import org.apache.druid.segment.metadata.SqlCompactionStateStorage; import org.apache.druid.server.coordinator.DruidCoordinatorRuntimeParams; import org.apache.druid.server.coordinator.config.MetadataCleanupConfig; import org.apache.druid.server.coordinator.stats.CoordinatorRunStats; @@ -42,9 +42,7 @@ 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; @@ -60,7 +58,7 @@ public class KillUnreferencedCompactionStateTest private TestDerbyConnector derbyConnector; private MetadataStorageTablesConfig tablesConfig; - private CompactionStateManager compactionStateManager; + private CompactionStateStorage compactionStateStorage; private DruidCoordinatorRuntimeParams mockParams; @BeforeEach @@ -72,7 +70,7 @@ public void setUp() derbyConnector.createCompactionStatesTable(); derbyConnector.createSegmentTable(); - compactionStateManager = new PersistedCompactionStateManager(tablesConfig, jsonMapper, createDeterministicMapper(), derbyConnector); + compactionStateStorage = new SqlCompactionStateStorage(tablesConfig, jsonMapper, createDeterministicMapper(), derbyConnector); mockParams = EasyMock.createMock(DruidCoordinatorRuntimeParams.class); CoordinatorRunStats runStats = new CoordinatorRunStats(); @@ -97,16 +95,14 @@ public void testKillUnreferencedCompactionState_lifecycle() ); KillUnreferencedCompactionState duty = - new TestKillUnreferencedCompactionState(cleanupConfig, compactionStateManager, dateTimes); + new TestKillUnreferencedCompactionState(cleanupConfig, compactionStateStorage, 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()); + compactionStateStorage.upsertCompactionState("test-ds", fingerprint, state, DateTimes.nowUtc()); return null; }); @@ -140,16 +136,14 @@ public void testKillUnreferencedCompactionState_repair() ); KillUnreferencedCompactionState duty = - new TestKillUnreferencedCompactionState(cleanupConfig, compactionStateManager, dateTimes); + new TestKillUnreferencedCompactionState(cleanupConfig, compactionStateStorage, 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()); + compactionStateStorage.upsertCompactionState("test-ds", fingerprint, state, DateTimes.nowUtc()); return null; }); @@ -196,14 +190,12 @@ public void testKillUnreferencedCompactionState_disabled() ); KillUnreferencedCompactionState duty = - new KillUnreferencedCompactionState(cleanupConfig, compactionStateManager); + new KillUnreferencedCompactionState(cleanupConfig, compactionStateStorage); // 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()); + compactionStateStorage.upsertCompactionState("test-ds", fingerprint, createTestCompactionState(), DateTimes.nowUtc()); return null; }); @@ -221,11 +213,11 @@ private static class TestKillUnreferencedCompactionState extends KillUnreference public TestKillUnreferencedCompactionState( MetadataCleanupConfig config, - CompactionStateManager compactionStateManager, + CompactionStateStorage compactionStateStorage, List dateTimes ) { - super(config, compactionStateManager); + super(config, compactionStateStorage); this.dateTimes = dateTimes; } 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 0c07ef75da9e..7b99b6dba988 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,7 +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.segment.metadata.HeapMemoryCompactionStateStorage; import org.apache.druid.server.compaction.CompactionStatusTracker; import org.apache.druid.server.coordinator.CloneStatusManager; import org.apache.druid.server.coordinator.CoordinatorConfigManager; @@ -516,7 +516,7 @@ private Environment( null, null, NoopSegmentMetadataCache.instance(), - new HeapMemoryCompactionStateManager() + new HeapMemoryCompactionStateStorage() ); this.configSyncer = EasyMock.niceMock(CoordinatorDynamicConfigSyncer.class); 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 395d64ea3f17..b93264681061 100644 --- a/services/src/main/java/org/apache/druid/guice/MetadataManagerModule.java +++ b/services/src/main/java/org/apache/druid/guice/MetadataManagerModule.java @@ -42,11 +42,11 @@ import org.apache.druid.metadata.segment.cache.HeapMemorySegmentMetadataCache; import org.apache.druid.metadata.segment.cache.SegmentMetadataCache; import org.apache.druid.segment.metadata.CompactionStateCache; -import org.apache.druid.segment.metadata.CompactionStateManager; +import org.apache.druid.segment.metadata.CompactionStateStorage; import org.apache.druid.segment.metadata.NoopCompactionStateCache; import org.apache.druid.segment.metadata.NoopSegmentSchemaCache; -import org.apache.druid.segment.metadata.PersistedCompactionStateManager; import org.apache.druid.segment.metadata.SegmentSchemaCache; +import org.apache.druid.segment.metadata.SqlCompactionStateStorage; import org.apache.druid.server.coordinator.CoordinatorConfigManager; import org.apache.druid.server.coordinator.MetadataManager; @@ -65,7 +65,7 @@ *

  • {@link SegmentMetadataCache}
  • *
  • {@link CompactionStateCache} - Overlord only
  • *
  • {@link SegmentSchemaCache} - Coordinator only
  • - *
  • {@link PersistedCompactionStateManager}
  • + *
  • {@link SqlCompactionStateStorage}
  • * */ public class MetadataManagerModule implements Module @@ -107,8 +107,8 @@ public void configure(Binder binder) binder.bind(SegmentMetadataCache.class) .to(HeapMemorySegmentMetadataCache.class) .in(LazySingleton.class); - binder.bind(CompactionStateManager.class) - .to(PersistedCompactionStateManager.class) + binder.bind(CompactionStateStorage.class) + .to(SqlCompactionStateStorage.class) .in(ManageLifecycle.class); // Coordinator-only dependencies From 79e3a5418ec0eed90807a968e1cf1579c89c188c Mon Sep 17 00:00:00 2001 From: capistrant Date: Fri, 9 Jan 2026 16:20:47 -0600 Subject: [PATCH 37/72] Add compaction state added and deleted metrics --- .../segment/cache/HeapMemorySegmentMetadataCache.java | 2 ++ .../apache/druid/metadata/segment/cache/Metric.java | 10 ++++++++++ 2 files changed, 12 insertions(+) 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 79c3de3ab0e2..0ac12c7b4901 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 @@ -1189,6 +1189,7 @@ private Map buildFingerprintToStateMapForDeltaSync() // Remove entry for compaction states that have been deleted from the metadata store final Set deletedFingerprints = Sets.difference(cachedFingerprints, persistedFingerprints); deletedFingerprints.forEach(fingerprintToStateMap::remove); + emitMetric(Metric.DELETED_COMPACTION_STATES, deletedFingerprints.size()); // Retrieve and add entry for compaction states that have been added to the metadata store final Set addedFingerprints = Sets.difference(persistedFingerprints, cachedFingerprints); @@ -1201,6 +1202,7 @@ private Map buildFingerprintToStateMapForDeltaSync() addedCompactionStateRecords.forEach( record -> fingerprintToStateMap.put(record.getFingerprint(), record.getState()) ); + emitMetric(Metric.ADDED_COMPACTION_STATES, addedCompactionStateRecords.size()); return fingerprintToStateMap; } diff --git a/server/src/main/java/org/apache/druid/metadata/segment/cache/Metric.java b/server/src/main/java/org/apache/druid/metadata/segment/cache/Metric.java index 0335e5116416..101ba575e1e3 100644 --- a/server/src/main/java/org/apache/druid/metadata/segment/cache/Metric.java +++ b/server/src/main/java/org/apache/druid/metadata/segment/cache/Metric.java @@ -168,6 +168,16 @@ private Metric() */ public static final String SKIPPED_COMPACTION_STATES = METRIC_NAME_PREFIX + "compactionState/skipped"; + /** + * Number of compaction states added to the cache in the latest sync. + */ + public static final String ADDED_COMPACTION_STATES = METRIC_NAME_PREFIX + "compactionState/added"; + + /** + * Number of compaction states deleted from the cache in the latest sync. + */ + public static final String DELETED_COMPACTION_STATES = METRIC_NAME_PREFIX + "compactionState/deleted"; + /** * Number of unparseable pending segment records skipped while refreshing the cache. */ From 6e23adc010e6c9ca47803f11214be692f602566a Mon Sep 17 00:00:00 2001 From: capistrant Date: Fri, 9 Jan 2026 16:36:45 -0600 Subject: [PATCH 38/72] improve queries for compaction state cache sync --- .../metadata/SqlSegmentsMetadataQuery.java | 20 ++++++------------- 1 file changed, 6 insertions(+), 14 deletions(-) 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 e31c453bb76f..b2ad9ae531e4 100644 --- a/server/src/main/java/org/apache/druid/metadata/SqlSegmentsMetadataQuery.java +++ b/server/src/main/java/org/apache/druid/metadata/SqlSegmentsMetadataQuery.java @@ -1706,7 +1706,7 @@ private SegmentSchemaRecord mapToSchemaRecord(ResultSet resultSet) } /** - * Retrieves all unique compaction state fingerprints currently referenced by used segments. + * Retrieves all unique compaction state fingerprints currently marked as used. * This is used for delta syncs to determine which fingerprints are still active. * * @return Set of compaction state fingerprints @@ -1714,9 +1714,8 @@ private SegmentSchemaRecord mapToSchemaRecord(ResultSet resultSet) public Set retrieveAllUsedCompactionStateFingerprints() { final String sql = StringUtils.format( - "SELECT DISTINCT compaction_state_fingerprint FROM %s " - + "WHERE used = true AND compaction_state_fingerprint IS NOT NULL", - dbTables.getSegmentsTable() + "SELECT fingerprint FROM %s WHERE used = true", + dbTables.getCompactionStatesTable() ); return Set.copyOf( @@ -1728,22 +1727,15 @@ public Set retrieveAllUsedCompactionStateFingerprints() } /** - * Retrieves all compaction states for used segments (full sync). - * Fetches from compaction_states table where the fingerprint is referenced by used segments. + * Retrieves all compaction states marked as used (full sync). * * @return List of CompactionStateRecord objects */ public List retrieveAllUsedCompactionStates() { final String sql = StringUtils.format( - "SELECT cs.fingerprint, cs.payload FROM %s cs " - + "WHERE cs.used = true " - + "AND cs.fingerprint IN (" - + " SELECT DISTINCT compaction_state_fingerprint FROM %s " - + " WHERE used = true AND compaction_state_fingerprint IS NOT NULL" - + ")", - dbTables.getCompactionStatesTable(), - dbTables.getSegmentsTable() + "SELECT fingerprint, payload FROM %s WHERE used = true", + dbTables.getCompactionStatesTable() ); return retrieveValidCompactionStateRecordsWithQuery(handle.createQuery(sql)); From 58149e64f4896b7271deea0c78b372ccea0a7c34 Mon Sep 17 00:00:00 2001 From: capistrant Date: Fri, 9 Jan 2026 16:37:00 -0600 Subject: [PATCH 39/72] clean up doc wording --- docs/configuration/index.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/configuration/index.md b/docs/configuration/index.md index 6296c4055483..f9f6b87c7e33 100644 --- a/docs/configuration/index.md +++ b/docs/configuration/index.md @@ -389,7 +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.compactionStates`|The table that stores compaction state payloads and 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`| From b5568d2b00cbafea73831570f4f6b6f15b216917 Mon Sep 17 00:00:00 2001 From: capistrant Date: Fri, 9 Jan 2026 16:50:17 -0600 Subject: [PATCH 40/72] Miscl. cleanup from review --- .../druid/metadata/MetadataStorageConnector.java | 4 ++-- .../metadata/segment/cache/CompactionStateRecord.java | 2 +- .../compaction/NewestSegmentFirstPolicyTest.java | 10 +++++----- .../server/coordinator/duty/CompactSegmentsTest.java | 4 ---- 4 files changed, 8 insertions(+), 12 deletions(-) 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 221fb6b5ee0d..c41dee70f0ba 100644 --- a/processing/src/main/java/org/apache/druid/metadata/MetadataStorageConnector.java +++ b/processing/src/main/java/org/apache/druid/metadata/MetadataStorageConnector.java @@ -97,9 +97,9 @@ default void exportTable( void createSegmentSchemasTable(); /** - * CompactionStates table is centralized store for {@link org.apache.druid.timeline.CompactionState} objects. + * This table stores {@link org.apache.druid.timeline.CompactionState} objects. *

    - * N segments can refer to the same compaction state via its unique fingerprint + * Multiple segments can refer to the same compaction state via its unique fingerprint *

    */ void createCompactionStatesTable(); diff --git a/server/src/main/java/org/apache/druid/metadata/segment/cache/CompactionStateRecord.java b/server/src/main/java/org/apache/druid/metadata/segment/cache/CompactionStateRecord.java index 5dd6e20e06e2..1595febec683 100644 --- a/server/src/main/java/org/apache/druid/metadata/segment/cache/CompactionStateRecord.java +++ b/server/src/main/java/org/apache/druid/metadata/segment/cache/CompactionStateRecord.java @@ -22,7 +22,7 @@ import org.apache.druid.timeline.CompactionState; /** - * Represents a single record in the compaction_states table. + * Represents a single record in the druid_compaction_states table. */ public class CompactionStateRecord { 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 bb132a302dc6..4a9a0603b002 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 @@ -285,7 +285,7 @@ public void testSkipDataSourceWithNoSegments() Collections.emptyMap(), compactionStateStorage, compactionStateCache - ); + ); assertCompactSegmentIntervals( iterator, @@ -519,7 +519,7 @@ public void testWithSkipIntervals() ), compactionStateStorage, compactionStateCache - ); + ); assertCompactSegmentIntervals( iterator, @@ -560,7 +560,7 @@ public void testHoleInSearchInterval() ), compactionStateStorage, compactionStateCache - ); + ); assertCompactSegmentIntervals( iterator, @@ -2067,7 +2067,7 @@ TestDataSource.KOALA, configBuilder().forDataSource(TestDataSource.KOALA).build( Collections.emptyMap(), compactionStateStorage, compactionStateCache - ); + ); // Verify that the segments of WIKI are preferred even though they are older Assert.assertTrue(iterator.hasNext()); @@ -2090,7 +2090,7 @@ private CompactionSegmentIterator createIterator(DataSourceCompactionConfig conf Collections.emptyMap(), compactionStateStorage, compactionStateCache - ); + ); } private static void assertCompactSegmentIntervals( 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 b8ac5ad111ef..cd92e8f1999a 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,8 +79,6 @@ 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.CompactionStateStorage; -import org.apache.druid.segment.metadata.HeapMemoryCompactionStateStorage; import org.apache.druid.segment.transform.CompactionTransformSpec; import org.apache.druid.server.compaction.CompactionCandidate; import org.apache.druid.server.compaction.CompactionCandidateSearchPolicy; @@ -199,7 +197,6 @@ public static Collection constructorFeeder() private DataSourcesSnapshot dataSources; private CompactionStatusTracker statusTracker; private final Map> datasourceToSegments = new HashMap<>(); - private final CompactionStateStorage compactionStateStorage = new HeapMemoryCompactionStateStorage(); public CompactSegmentsTest( PartitionsSpec partitionsSpec, @@ -276,7 +273,6 @@ public void testSerde() throws Exception .addValue(DruidCoordinatorConfig.class, COORDINATOR_CONFIG) .addValue(OverlordClient.class, overlordClient) .addValue(CompactionStatusTracker.class, statusTracker) - .addValue(CompactionStateStorage.class, compactionStateStorage) .addValue(MetadataCatalog.class, NullMetadataCatalog.INSTANCE) .addValue(ExprMacroTable.class, TestExprMacroTable.INSTANCE) ); From 0fab9f91bd0dc0f940a600659925add6b9a5b0b1 Mon Sep 17 00:00:00 2001 From: capistrant Date: Fri, 9 Jan 2026 17:06:28 -0600 Subject: [PATCH 41/72] some metadata store code cleanup --- .../org/apache/druid/metadata/SQLMetadataConnector.java | 3 ++- .../druid/segment/metadata/SqlCompactionStateStorage.java | 6 +++--- .../segment/metadata/SqlCompactionStateStorageTest.java | 4 ++-- 3 files changed, 7 insertions(+), 6 deletions(-) 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 9d19ad7f16af..943fd088cdaa 100644 --- a/server/src/main/java/org/apache/druid/metadata/SQLMetadataConnector.java +++ b/server/src/main/java/org/apache/druid/metadata/SQLMetadataConnector.java @@ -356,6 +356,7 @@ public void createSegmentTable(final String tableName) columns.add("payload %2$s NOT NULL"); columns.add("used_status_last_updated VARCHAR(255) NOT NULL"); columns.add("compaction_state_fingerprint VARCHAR(255)"); + columns.add("upgraded_from_segment_id VARCHAR(255)"); if (centralizedDatasourceSchemaConfig.isEnabled()) { columns.add("schema_fingerprint VARCHAR(255)"); @@ -1115,7 +1116,7 @@ public void createCompactionStatesTable(final String tableName) "CREATE TABLE %1$s (\n" + " id %2$s NOT NULL,\n" + " created_date VARCHAR(255) NOT NULL,\n" - + " datasource 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" diff --git a/server/src/main/java/org/apache/druid/segment/metadata/SqlCompactionStateStorage.java b/server/src/main/java/org/apache/druid/segment/metadata/SqlCompactionStateStorage.java index 0b36bc3f530a..0bf0e7d9db21 100644 --- a/server/src/main/java/org/apache/druid/segment/metadata/SqlCompactionStateStorage.java +++ b/server/src/main/java/org/apache/druid/segment/metadata/SqlCompactionStateStorage.java @@ -122,15 +122,15 @@ public void upsertCompactionState( log.info("Inserting new compaction state for fingerprint[%s] in dataSource[%s].", fingerprint, dataSource); 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)", + "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() ); try { handle.createStatement(insertSql) .bind("created_date", now) - .bind("datasource", dataSource) + .bind("dataSource", dataSource) .bind("fingerprint", fingerprint) .bind("payload", jsonMapper.writeValueAsBytes(compactionState)) .bind("used", true) diff --git a/server/src/test/java/org/apache/druid/segment/metadata/SqlCompactionStateStorageTest.java b/server/src/test/java/org/apache/druid/segment/metadata/SqlCompactionStateStorageTest.java index 58ea4ea181da..45b321575e2f 100644 --- a/server/src/test/java/org/apache/druid/segment/metadata/SqlCompactionStateStorageTest.java +++ b/server/src/test/java/org/apache/druid/segment/metadata/SqlCompactionStateStorageTest.java @@ -194,7 +194,7 @@ public void test_deleteCompactionStatesOlderThan_deletesOnlyOldUnusedStates() derbyConnector.retryWithHandle(handle -> { handle.createStatement( "INSERT INTO " + tablesConfig.getCompactionStatesTable() + " " - + "(created_date, datasource, fingerprint, payload, used, used_status_last_updated) " + + "(created_date, dataSource, fingerprint, payload, used, used_status_last_updated) " + "VALUES (:cd, :ds, :fp, :pl, :used, :updated)" ) .bind("cd", oldTime.toString()) @@ -211,7 +211,7 @@ public void test_deleteCompactionStatesOlderThan_deletesOnlyOldUnusedStates() derbyConnector.retryWithHandle(handle -> { handle.createStatement( "INSERT INTO " + tablesConfig.getCompactionStatesTable() + " " - + "(created_date, datasource, fingerprint, payload, used, used_status_last_updated) " + + "(created_date, dataSource, fingerprint, payload, used, used_status_last_updated) " + "VALUES (:cd, :ds, :fp, :pl, :used, :updated)" ) .bind("cd", recentTime.toString()) From 6f40f9ff559315526d586fc995b6799b3134be4f Mon Sep 17 00:00:00 2001 From: capistrant Date: Fri, 9 Jan 2026 17:18:27 -0600 Subject: [PATCH 42/72] refactor id out of the compaction states table as it is superflous --- .../druid/metadata/SQLMetadataConnector.java | 14 +++----------- 1 file changed, 3 insertions(+), 11 deletions(-) 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 943fd088cdaa..26fbd1443db2 100644 --- a/server/src/main/java/org/apache/druid/metadata/SQLMetadataConnector.java +++ b/server/src/main/java/org/apache/druid/metadata/SQLMetadataConnector.java @@ -1114,27 +1114,19 @@ public void createCompactionStatesTable(final String 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" + + " payload %2$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" + + " PRIMARY KEY (fingerprint)\n" + ")", - tableName, getSerialType(), getPayloadType() + tableName, getPayloadType() ) ) ); - createIndex( - tableName, - "IDX_%s_FINGERPRINT", - List.of("fingerprint") - ); - createIndex( tableName, "IDX_%s_USED", From 87137608e1e352a19ad20860524a8ab4bd06e71c Mon Sep 17 00:00:00 2001 From: capistrant Date: Fri, 9 Jan 2026 18:16:10 -0600 Subject: [PATCH 43/72] Some CompactionStatus cleanup --- .../server/compaction/CompactionStatus.java | 92 +++++++++---------- .../compaction/CompactionStatusTest.java | 37 +++----- 2 files changed, 55 insertions(+), 74 deletions(-) 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 b5415460d47d..637c94897275 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 @@ -449,82 +449,78 @@ private CompactionStatus allFingerprintedCandidatesHaveExpectedFingerprint() Map> mismatchedFingerprintToSegmentMap = new HashMap<>(); for (DataSegment segment : fingerprintedSegments) { String fingerprint = segment.getCompactionStateFingerprint(); - if (fingerprint != null && !fingerprint.equals(targetFingerprint)) { + if (fingerprint == null) { + // Should not happen since we are iterating over fingerprintedSegments + } else if (fingerprint.equals(targetFingerprint)) { + compactedSegments.add(segment); + } else { 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()) { + // All fingerprinted segments have the expected fingerprint - compaction is complete return COMPLETE; } - boolean fingerprintedSegmentNeedingCompactionFound = false; - - if (compactionStateCache != null) { - for (Map.Entry> e : mismatchedFingerprintToSegmentMap.entrySet()) { - String fingerprint = e.getKey(); - CompactionState stateToValidate = compactionStateCache.getCompactionStateByFingerprint(fingerprint).orElse(null); - 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()) { + if (compactionStateCache == null) { + // Cannot evaluate further without a compaction state cache + uncompactedSegments.addAll( + mismatchedFingerprintToSegmentMap.values() + .stream() + .flatMap(List::stream) + .collect(Collectors.toList()) + ); + return CompactionStatus.pending("Segments have a mismatched fingerprint and no compaction state cache is available"); + } + + boolean fingerprintedSegmentWithoutCachedStateFound = false; + + for (Map.Entry> e : mismatchedFingerprintToSegmentMap.entrySet()) { + String fingerprint = e.getKey(); + CompactionState stateToValidate = compactionStateCache.getCompactionStateByFingerprint(fingerprint).orElse(null); + if (stateToValidate == null) { + log.warn("No compaction state found for fingerprint[%s]", fingerprint); + fingerprintedSegmentWithoutCachedStateFound = true; uncompactedSegments.addAll(e.getValue()); - fingerprintedSegmentNeedingCompactionFound = true; + } 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; + }); } } - if (fingerprintedSegmentNeedingCompactionFound) { - return CompactionStatus.pending("At least one segment has a mismatched fingerprint and needs compaction"); + if (fingerprintedSegmentWithoutCachedStateFound) { + return CompactionStatus.pending("One or more fingerprinted segments do not have a cached compaction state"); } 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
    • - *
    - *

    + * Checks if all the segments have been compacted at least once and groups them into uncompacted, fingerprinted, or + * non-fingerprinted. */ private CompactionStatus segmentsHaveBeenCompactedAtLeastOnce() { for (DataSegment segment : candidateSegments.getSegments()) { final String fingerprint = segment.getCompactionStateFingerprint(); + final CompactionState segmentState = segment.getLastCompactionState(); if (fingerprint != null) { fingerprintedSegments.add(segment); + } else if (segmentState == null) { + uncompactedSegments.add(segment); } else { - final CompactionState segmentState = segment.getLastCompactionState(); - if (segmentState == null) { - uncompactedSegments.add(segment); - } else { - unknownStateToSegments.computeIfAbsent(segmentState, k -> new ArrayList<>()).add(segment); - } + unknownStateToSegments.computeIfAbsent(segmentState, k -> new ArrayList<>()).add(segment); } } 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 619946298179..93c3d65c9826 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 @@ -608,7 +608,8 @@ public void test_evaluate_needsCompactionWhenAllSegmentsHaveUnexpectedCompaction CompactionCandidate.from(segments, null), compactionConfig, "'segmentGranularity' mismatch: required[DAY], current[HOUR]", - compactionStateStorage + compactionStateStorage, + compactionStateCache ); } @@ -644,7 +645,8 @@ public void test_evaluate_needsCompactionWhenSomeSegmentsHaveUnexpectedCompactio CompactionCandidate.from(segments, null), compactionConfig, "'segmentGranularity' mismatch: required[DAY], current[HOUR]", - compactionStateStorage + compactionStateStorage, + compactionStateCache ); } @@ -673,26 +675,6 @@ public void test_evaluate_noCompacationIfUnexpectedFingerprintHasExpectedCompact Assert.assertTrue(status.isComplete()); } - @Test - public void test_evaluate_needsCompactionWhenUnexpectedFingerprintAndNullCompactionStateStorage() - { - 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", - compactionStateStorage - ); - } - @Test public void test_evaluate_needsCompactionWhenUnexpectedFingerprintAndNoFingerprintInMetadataStore() { @@ -708,8 +690,9 @@ public void test_evaluate_needsCompactionWhenUnexpectedFingerprintAndNoFingerpri verifyEvaluationNeedsCompactionBecauseWithCustomSegments( CompactionCandidate.from(segments, null), compactionConfig, - "At least one segment has a mismatched fingerprint and needs compaction", - compactionStateStorage + "One or more fingerprinted segments do not have a cached compaction state", + compactionStateStorage, + compactionStateCache ); } @@ -762,7 +745,8 @@ public void test_evaluate_needsCompactionWhenNonFingerprintedSegmentsFailChecksO CompactionCandidate.from(segments, null), compactionConfig, "'segmentGranularity' mismatch: required[DAY], current[HOUR]", - compactionStateStorage + compactionStateStorage, + compactionStateCache ); } @@ -837,7 +821,8 @@ private void verifyEvaluationNeedsCompactionBecauseWithCustomSegments( CompactionCandidate candidate, DataSourceCompactionConfig compactionConfig, String expectedReason, - CompactionStateStorage compactionStateStorage + CompactionStateStorage compactionStateStorage, + CompactionStateCache compactionStateCache ) { final CompactionStatus status = CompactionStatus.compute( From a4bf6c8af4ed564d7a5dfe29c3b828fd5316265a Mon Sep 17 00:00:00 2001 From: capistrant Date: Fri, 9 Jan 2026 18:29:29 -0600 Subject: [PATCH 44/72] Migrate the location of creating a compaction state from config --- .../compact/CompactionSupervisorTest.java | 3 +- .../CompactionConfigBasedJobTemplate.java | 3 +- .../server/compaction/CompactionStatus.java | 60 +---------------- .../DataSourceCompactionConfig.java | 64 +++++++++++++++++++ .../compaction/CompactionStatusTest.java | 16 ++--- 5 files changed, 75 insertions(+), 71 deletions(-) diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionSupervisorTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionSupervisorTest.java index 76893aa1d50e..4f018fbcffed 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionSupervisorTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionSupervisorTest.java @@ -32,7 +32,6 @@ import org.apache.druid.query.DruidMetrics; import org.apache.druid.rpc.UpdateResponse; import org.apache.druid.segment.metadata.CompactionStateStorage; -import org.apache.druid.server.compaction.CompactionStatus; import org.apache.druid.server.coordinator.ClusterCompactionConfig; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; import org.apache.druid.server.coordinator.InlineSchemaDataSourceCompactionConfig; @@ -292,7 +291,7 @@ private void verifyCompactedSegmentsHaveFingerprints(DataSourceCompactionConfig .getInstance(CompactionStateStorage.class); String expectedFingerprint = compactionStateStorage.generateCompactionStateFingerprint( - CompactionStatus.createCompactionStateFromConfig(compactionConfig), + compactionConfig.toCompactionState(), dataSource ); 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 786b3f1bf3fc..255614b0bf7a 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 @@ -27,7 +27,6 @@ import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.server.compaction.CompactionCandidate; import org.apache.druid.server.compaction.CompactionSlotManager; -import org.apache.druid.server.compaction.CompactionStatus; import org.apache.druid.server.compaction.DataSourceCompactibleSegmentIterator; import org.apache.druid.server.compaction.NewestSegmentFirstPolicy; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; @@ -72,7 +71,7 @@ public List createCompactionJobs( final List jobs = new ArrayList<>(); - CompactionState compactionState = CompactionStatus.createCompactionStateFromConfig(config); + CompactionState compactionState = config.toCompactionState(); String compactionStateFingerprint = params.getCompactionStateStorageImpl().generateCompactionStateFingerprint( compactionState, 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 637c94897275..79933fff2d96 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 @@ -22,11 +22,7 @@ import org.apache.commons.lang3.ArrayUtils; import org.apache.druid.client.indexing.ClientCompactionTaskQueryTuningConfig; import org.apache.druid.common.config.Configs; -import org.apache.druid.data.input.impl.AggregateProjectionSpec; import org.apache.druid.data.input.impl.DimensionSchema; -import org.apache.druid.data.input.impl.DimensionsSpec; -import org.apache.druid.indexer.granularity.GranularitySpec; -import org.apache.druid.indexer.granularity.UniformGranularitySpec; import org.apache.druid.indexer.partitions.DimensionRangePartitionsSpec; import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; import org.apache.druid.indexer.partitions.HashedPartitionsSpec; @@ -269,7 +265,7 @@ static CompactionStatus compute( @Nullable CompactionStateCache compactionStateCache ) { - final CompactionState expectedState = createCompactionStateFromConfig(config); + final CompactionState expectedState = config.toCompactionState(); String expectedFingerprint; if (compactionStateStorage == null) { expectedFingerprint = null; @@ -811,58 +807,4 @@ private static CompactionStatistics createStats(List segments) } } - /** - * 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/DataSourceCompactionConfig.java b/server/src/main/java/org/apache/druid/server/coordinator/DataSourceCompactionConfig.java index 8d9b861b5713..727637e7bee4 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/DataSourceCompactionConfig.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/DataSourceCompactionConfig.java @@ -21,14 +21,23 @@ import com.fasterxml.jackson.annotation.JsonSubTypes; import com.fasterxml.jackson.annotation.JsonTypeInfo; +import org.apache.druid.client.indexing.ClientCompactionTaskQueryTuningConfig; 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.granularity.Granularity; import org.apache.druid.query.aggregation.AggregatorFactory; +import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.transform.CompactionTransformSpec; +import org.apache.druid.server.compaction.CompactionStatus; +import org.apache.druid.timeline.CompactionState; import org.joda.time.Period; import javax.annotation.Nullable; +import java.util.Arrays; import java.util.List; import java.util.Map; @@ -89,4 +98,59 @@ public interface DataSourceCompactionConfig @Nullable AggregatorFactory[] getMetricsSpec(); + + /** + * Converts this compaction config to a {@link CompactionState}. + */ + default CompactionState toCompactionState() + { + ClientCompactionTaskQueryTuningConfig tuningConfig = ClientCompactionTaskQueryTuningConfig.from(this); + + // 1. PartitionsSpec - reuse existing method + PartitionsSpec partitionsSpec = CompactionStatus.findPartitionsSpecFromConfig(tuningConfig); + + // 2. DimensionsSpec + DimensionsSpec dimensionsSpec = null; + if (getDimensionsSpec() != null && getDimensionsSpec().getDimensions() != null) { + dimensionsSpec = new DimensionsSpec(getDimensionsSpec().getDimensions()); + } + + // 3. Metrics + List metricsSpec = getMetricsSpec() == null + ? null + : Arrays.asList(getMetricsSpec()); + + // 4. Transform + CompactionTransformSpec transformSpec = getTransformSpec(); + + // 5. IndexSpec + IndexSpec indexSpec = tuningConfig.getIndexSpec() == null + ? IndexSpec.getDefault() + : tuningConfig.getIndexSpec(); + + // 6. GranularitySpec + GranularitySpec granularitySpec = null; + if (getGranularitySpec() != null) { + UserCompactionTaskGranularityConfig userGranularityConfig = getGranularitySpec(); + granularitySpec = new UniformGranularitySpec( + userGranularityConfig.getSegmentGranularity(), + userGranularityConfig.getQueryGranularity(), + userGranularityConfig.isRollup(), + null // intervals + ); + } + + // 7. Projections + List projections = getProjections(); + + return new CompactionState( + partitionsSpec, + dimensionsSpec, + metricsSpec, + transformSpec, + indexSpec, + granularitySpec, + projections + ); + } } 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 93c3d65c9826..668af1cb3231 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 @@ -591,7 +591,7 @@ public void test_evaluate_needsCompactionWhenAllSegmentsHaveUnexpectedCompaction .forDataSource(TestDataSource.WIKI) .withGranularitySpec(new UserCompactionTaskGranularityConfig(Granularities.HOUR, null, null)) .build(); - CompactionState wrongState = CompactionStatus.createCompactionStateFromConfig(oldCompactionConfig); + CompactionState wrongState = oldCompactionConfig.toCompactionState(); final DataSourceCompactionConfig compactionConfig = InlineSchemaDataSourceCompactionConfig .builder() @@ -599,7 +599,7 @@ public void test_evaluate_needsCompactionWhenAllSegmentsHaveUnexpectedCompaction .withGranularitySpec(new UserCompactionTaskGranularityConfig(Granularities.DAY, null, null)) .build(); - CompactionState expectedState = CompactionStatus.createCompactionStateFromConfig(compactionConfig); + CompactionState expectedState = compactionConfig.toCompactionState(); compactionStateStorage.upsertCompactionState(TestDataSource.WIKI, "wrongFingerprint", wrongState, DateTimes.nowUtc()); syncCacheFromManager(); @@ -621,7 +621,7 @@ public void test_evaluate_needsCompactionWhenSomeSegmentsHaveUnexpectedCompactio .forDataSource(TestDataSource.WIKI) .withGranularitySpec(new UserCompactionTaskGranularityConfig(Granularities.HOUR, null, null)) .build(); - CompactionState wrongState = CompactionStatus.createCompactionStateFromConfig(oldCompactionConfig); + CompactionState wrongState = oldCompactionConfig.toCompactionState(); final DataSourceCompactionConfig compactionConfig = InlineSchemaDataSourceCompactionConfig .builder() @@ -629,7 +629,7 @@ public void test_evaluate_needsCompactionWhenSomeSegmentsHaveUnexpectedCompactio .withGranularitySpec(new UserCompactionTaskGranularityConfig(Granularities.DAY, null, null)) .build(); - CompactionState expectedState = CompactionStatus.createCompactionStateFromConfig(compactionConfig); + CompactionState expectedState = compactionConfig.toCompactionState(); String expectedFingerprint = compactionStateStorage.generateCompactionStateFingerprint(expectedState, TestDataSource.WIKI); @@ -662,7 +662,7 @@ public void test_evaluate_noCompacationIfUnexpectedFingerprintHasExpectedCompact .withGranularitySpec(new UserCompactionTaskGranularityConfig(Granularities.HOUR, null, null)) .build(); - CompactionState expectedState = CompactionStatus.createCompactionStateFromConfig(compactionConfig); + CompactionState expectedState = compactionConfig.toCompactionState(); compactionStateStorage.upsertCompactionState(TestDataSource.WIKI, "wrongFingerprint", expectedState, DateTimes.nowUtc()); syncCacheFromManager(); @@ -705,7 +705,7 @@ public void test_evaluate_noCompactionWhenAllSegmentsHaveExpectedCompactionState .withGranularitySpec(new UserCompactionTaskGranularityConfig(Granularities.DAY, null, null)) .build(); - CompactionState expectedState = CompactionStatus.createCompactionStateFromConfig(compactionConfig); + CompactionState expectedState = compactionConfig.toCompactionState(); String expectedFingerprint = compactionStateStorage.generateCompactionStateFingerprint(expectedState, TestDataSource.WIKI); @@ -732,7 +732,7 @@ public void test_evaluate_needsCompactionWhenNonFingerprintedSegmentsFailChecksO .withGranularitySpec(new UserCompactionTaskGranularityConfig(Granularities.DAY, null, null)) .build(); - CompactionState expectedState = CompactionStatus.createCompactionStateFromConfig(compactionConfig); + CompactionState expectedState = compactionConfig.toCompactionState(); String expectedFingerprint = compactionStateStorage.generateCompactionStateFingerprint(expectedState, TestDataSource.WIKI); @@ -759,7 +759,7 @@ public void test_evaluate_noCompactionWhenNonFingerprintedSegmentsPassChecksOnLa .withGranularitySpec(new UserCompactionTaskGranularityConfig(Granularities.DAY, null, null)) .build(); - CompactionState expectedState = CompactionStatus.createCompactionStateFromConfig(compactionConfig); + CompactionState expectedState = compactionConfig.toCompactionState(); String expectedFingerprint = compactionStateStorage.generateCompactionStateFingerprint(expectedState, TestDataSource.WIKI); From 21a10e6220a0164f9b7dfd1da3cce24ec4fd8147 Mon Sep 17 00:00:00 2001 From: capistrant Date: Fri, 9 Jan 2026 18:55:10 -0600 Subject: [PATCH 45/72] More refactoring per review --- .../embedded/compact/AutoCompactionTest.java | 2 +- .../CompactionConfigBasedJobTemplate.java | 2 +- .../druid/indexing/compact/CompactionJob.java | 32 +++++++++---------- .../indexing/compact/CompactionJobQueue.java | 6 ++-- .../apache/druid/msq/exec/ControllerImpl.java | 6 +--- .../metadata/CompactionStateStorage.java | 6 ++-- .../compaction/CompactionRunSimulator.java | 2 +- .../server/compaction/CompactionStatus.java | 5 ++- .../coordinator/ClusterCompactionConfig.java | 32 +++++++++---------- .../coordinator/CoordinatorConfigManager.java | 2 +- .../coordinator/DruidCompactionConfig.java | 2 +- 11 files changed, 47 insertions(+), 50 deletions(-) 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 868cc48fbfb7..4bc72870a1e5 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 @@ -1957,7 +1957,7 @@ private void updateCompactionTaskSlot(double compactionTaskSlotRatio, int maxCom oldConfig.getCompactionPolicy(), oldConfig.isUseSupervisors(), oldConfig.getEngine(), - oldConfig.isLegacyPersistLastCompactionStateInSegments() + oldConfig.isStoreCompactionStatePerSegment() ) ); 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 255614b0bf7a..be7864313733 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionConfigBasedJobTemplate.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionConfigBasedJobTemplate.java @@ -87,7 +87,7 @@ public List createCompactionJobs( config, params.getClusterCompactionConfig().getEngine(), compactionStateFingerprint, - params.getClusterCompactionConfig().isLegacyPersistLastCompactionStateInSegments() + params.getClusterCompactionConfig().isStoreCompactionStatePerSegment() ); jobs.add( new CompactionJob( diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionJob.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionJob.java index 0a29c0521689..4ceaec85b6ea 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionJob.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionJob.java @@ -32,37 +32,37 @@ public class CompactionJob extends BatchIndexingJob { private final CompactionCandidate candidate; private final int maxRequiredTaskSlots; - private final String compactionStateFingerprint; - private final CompactionState compactionState; + private final String targetCompactionStateFingerprint; + private final CompactionState targetCompactionState; public CompactionJob( ClientCompactionTaskQuery task, CompactionCandidate candidate, int maxRequiredTaskSlots, - String compactionStateFingerprint, - CompactionState compactionState + String targetCompactionStateFingerprint, + CompactionState targetCompactionState ) { super(task, null); this.candidate = candidate; this.maxRequiredTaskSlots = maxRequiredTaskSlots; - this.compactionStateFingerprint = compactionStateFingerprint; - this.compactionState = compactionState; + this.targetCompactionStateFingerprint = targetCompactionStateFingerprint; + this.targetCompactionState = targetCompactionState; } public CompactionJob( ClientSqlQuery msqQuery, CompactionCandidate candidate, int maxRequiredTaskSlots, - String compactionStateFingerprint, - CompactionState compactionState + String targetCompactionStateFingerprint, + CompactionState targetCompactionState ) { super(null, msqQuery); this.candidate = candidate; this.maxRequiredTaskSlots = maxRequiredTaskSlots; - this.compactionStateFingerprint = compactionStateFingerprint; - this.compactionState = compactionState; + this.targetCompactionStateFingerprint = targetCompactionStateFingerprint; + this.targetCompactionState = targetCompactionState; } public String getDataSource() @@ -80,14 +80,14 @@ public int getMaxRequiredTaskSlots() return maxRequiredTaskSlots; } - public String getCompactionStateFingerprint() + public String getTargetCompactionStateFingerprint() { - return compactionStateFingerprint; + return targetCompactionStateFingerprint; } - public CompactionState getCompactionState() + public CompactionState getTargetCompactionState() { - return compactionState; + return targetCompactionState; } @Override @@ -97,8 +97,8 @@ public String toString() super.toString() + ", candidate=" + candidate + ", maxRequiredTaskSlots=" + maxRequiredTaskSlots + - ", compactionStateFingerprint='" + compactionStateFingerprint + '\'' + - ", compactionState=" + compactionState + + ", compactionStateFingerprint='" + targetCompactionStateFingerprint + '\'' + + ", compactionState=" + targetCompactionState + '}'; } } 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 3f64b8d10fa7..aefc2f17202f 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 @@ -361,11 +361,11 @@ private String startTaskIfReady(CompactionJob job) */ private void persistPendingCompactionState(CompactionJob job) { - if (job.getCompactionState() != null && job.getCompactionStateFingerprint() != null) { + if (job.getTargetCompactionState() != null && job.getTargetCompactionStateFingerprint() != null) { jobParams.getCompactionStateStorageImpl().upsertCompactionState( job.getDataSource(), - job.getCompactionStateFingerprint(), - job.getCompactionState(), + job.getTargetCompactionStateFingerprint(), + job.getTargetCompactionState(), DateTimes.nowUtc() ); } 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 0eeaefe5e728..be72dc588bdc 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 @@ -1700,11 +1700,7 @@ private void handleQueryResults( Tasks.DEFAULT_STORE_COMPACTION_STATE ); - final String compactionStateFingerprint = querySpec.getContext() - .getString( - Tasks.COMPACTION_STATE_FINGERPRINT_KEY, - null - ); + final String compactionStateFingerprint = querySpec.getContext().getString(Tasks.COMPACTION_STATE_FINGERPRINT_KEY); if (storeCompactionState) { DataSourceMSQDestination destination = (DataSourceMSQDestination) querySpec.getDestination(); diff --git a/server/src/main/java/org/apache/druid/segment/metadata/CompactionStateStorage.java b/server/src/main/java/org/apache/druid/segment/metadata/CompactionStateStorage.java index 9dc4dcf132d7..78185f3912a0 100644 --- a/server/src/main/java/org/apache/druid/segment/metadata/CompactionStateStorage.java +++ b/server/src/main/java/org/apache/druid/segment/metadata/CompactionStateStorage.java @@ -34,11 +34,13 @@ public interface CompactionStateStorage { /** * Generates a deterministic fingerprint for the given compaction state and datasource. - * The fingerprint is a SHA-256 hash of the datasource name and serialized compaction state. + *

    + * The fingerprint is a SHA-256 hash of the datasource name and serialized compaction state that is globally unique in + * the segment space. * * @param compactionState The compaction configuration to fingerprint * @param dataSource The datasource name - * @return A hex-encoded SHA-256 fingerprint string + * @return A hex-encoded SHA-256 fingerprint string that is globally unique */ String generateCompactionStateFingerprint(CompactionState compactionState, String dataSource); 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 80abb0a85112..bab65f90bf94 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 @@ -139,7 +139,7 @@ public void onTaskSubmitted(String taskId, CompactionCandidate candidateSegments clusterConfig.getCompactionPolicy(), clusterConfig.isUseSupervisors(), clusterConfig.getEngine(), - clusterConfig.isLegacyPersistLastCompactionStateInSegments() + clusterConfig.isStoreCompactionStatePerSegment() ); final CoordinatorRunStats stats = new CoordinatorRunStats(); 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 79933fff2d96..cf86acbd49e3 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 @@ -412,8 +412,7 @@ private CompactionStatus evaluate() // Consider segments which have passed all checks to be compacted // Includes segments with correct fingerprints and segments that passed all state checks - final List allCompactedSegments = new ArrayList<>(this.compactedSegments); - allCompactedSegments.addAll( + this.compactedSegments.addAll( unknownStateToSegments .values() .stream() @@ -425,7 +424,7 @@ private CompactionStatus evaluate() return COMPLETE; } else { return CompactionStatus.pending( - createStats(allCompactedSegments), + createStats(this.compactedSegments), createStats(uncompactedSegments), reasonsForCompaction.get(0) ); 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 7e608e6f6a3e..daaf98d67511 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,14 +45,7 @@ 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; + private final boolean storeCompactionStatePerSegment; @JsonCreator public ClusterCompactionConfig( @@ -61,7 +54,7 @@ public ClusterCompactionConfig( @JsonProperty("compactionPolicy") @Nullable CompactionCandidateSearchPolicy compactionPolicy, @JsonProperty("useSupervisors") @Nullable Boolean useSupervisors, @JsonProperty("engine") @Nullable CompactionEngine engine, - @JsonProperty("legacyPersistLastCompactionStateInSegments") Boolean legacyPersistLastCompactionStateInSegments + @JsonProperty("storeCompactionStatePerSegment") Boolean storeCompactionStatePerSegment ) { this.compactionTaskSlotRatio = Configs.valueOrDefault(compactionTaskSlotRatio, 0.1); @@ -69,8 +62,8 @@ 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, + this.storeCompactionStatePerSegment = Configs.valueOrDefault( + storeCompactionStatePerSegment, true ); @@ -109,10 +102,17 @@ public CompactionEngine getEngine() return engine; } + /** + * 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. + */ @JsonProperty - public boolean isLegacyPersistLastCompactionStateInSegments() + public boolean isStoreCompactionStatePerSegment() { - return legacyPersistLastCompactionStateInSegments; + return storeCompactionStatePerSegment; } @Override @@ -130,7 +130,7 @@ public boolean equals(Object o) && Objects.equals(compactionPolicy, that.compactionPolicy) && Objects.equals(useSupervisors, that.useSupervisors) && Objects.equals(engine, that.engine) - && Objects.equals(legacyPersistLastCompactionStateInSegments, that.legacyPersistLastCompactionStateInSegments); + && Objects.equals(storeCompactionStatePerSegment, that.storeCompactionStatePerSegment); } @Override @@ -142,7 +142,7 @@ public int hashCode() compactionPolicy, useSupervisors, engine, - legacyPersistLastCompactionStateInSegments + storeCompactionStatePerSegment ); } @@ -155,7 +155,7 @@ public String toString() ", useSupervisors=" + useSupervisors + ", engine=" + engine + ", compactionPolicy=" + compactionPolicy + - ", legacyPersistLastCompactionStateInSegments=" + legacyPersistLastCompactionStateInSegments + + ", legacyPersistLastCompactionStateInSegments=" + storeCompactionStatePerSegment + '}'; } } 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 65397332b0c9..a047bf99070d 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 @@ -167,7 +167,7 @@ public boolean updateCompactionTaskSlots( currentClusterConfig.getCompactionPolicy(), currentClusterConfig.isUseSupervisors(), currentClusterConfig.getEngine(), - currentClusterConfig.isLegacyPersistLastCompactionStateInSegments() + currentClusterConfig.isStoreCompactionStatePerSegment() ); 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 9059ab82add6..2119fb36c4e3 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 @@ -145,7 +145,7 @@ public CompactionEngine getEngine() @JsonProperty public boolean isLegacyPersistLastCompactionStateInSegments() { - return clusterConfig.isLegacyPersistLastCompactionStateInSegments(); + return clusterConfig.isStoreCompactionStatePerSegment(); } /** From 5ffb59effaff5ae98b9bb4f461911b4c6b6e276d Mon Sep 17 00:00:00 2001 From: capistrant Date: Fri, 9 Jan 2026 20:57:16 -0600 Subject: [PATCH 46/72] refactor to remove duplicate fingerprint generator code --- .../metadata/CompactionStateFingerprints.java | 72 +++++++++++++++++++ .../metadata/SqlCompactionStateStorage.java | 19 +---- .../HeapMemoryCompactionStateStorage.java | 21 +----- 3 files changed, 74 insertions(+), 38 deletions(-) create mode 100644 server/src/main/java/org/apache/druid/segment/metadata/CompactionStateFingerprints.java diff --git a/server/src/main/java/org/apache/druid/segment/metadata/CompactionStateFingerprints.java b/server/src/main/java/org/apache/druid/segment/metadata/CompactionStateFingerprints.java new file mode 100644 index 000000000000..80de1a4331bf --- /dev/null +++ b/server/src/main/java/org/apache/druid/segment/metadata/CompactionStateFingerprints.java @@ -0,0 +1,72 @@ +/* + * 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.hash.Hasher; +import com.google.common.hash.Hashing; +import com.google.common.io.BaseEncoding; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.timeline.CompactionState; + +/** + * Utility class for generating deterministic fingerprints for compaction states. + */ +public final class CompactionStateFingerprints +{ + private CompactionStateFingerprints() + { + // Prevent instantiation + } + + /** + * Generates a deterministic fingerprint for the given compaction state and datasource. + *

    + * The fingerprint is a SHA-256 hash of the datasource name and serialized compaction state. + * + * @param compactionState The compaction configuration to fingerprint + * @param dataSource The datasource name + * @param deterministicMapper ObjectMapper configured for deterministic serialization + * @return A hex-encoded SHA-256 fingerprint string + */ + @SuppressWarnings("UnstableApiUsage") + public static String generate( + final CompactionState compactionState, + final String dataSource, + final ObjectMapper deterministicMapper + ) + { + final Hasher hasher = Hashing.sha256().newHasher(); + + hasher.putBytes(StringUtils.toUtf8(dataSource)); + hasher.putByte((byte) 0xff); + + try { + hasher.putBytes(deterministicMapper.writeValueAsBytes(compactionState)); + } + catch (JsonProcessingException e) { + throw new RuntimeException("Failed to serialize CompactionState for fingerprinting", e); + } + hasher.putByte((byte) 0xff); + + return BaseEncoding.base16().encode(hasher.hash().asBytes()); + } +} diff --git a/server/src/main/java/org/apache/druid/segment/metadata/SqlCompactionStateStorage.java b/server/src/main/java/org/apache/druid/segment/metadata/SqlCompactionStateStorage.java index 0bf0e7d9db21..486ca3980fb7 100644 --- a/server/src/main/java/org/apache/druid/segment/metadata/SqlCompactionStateStorage.java +++ b/server/src/main/java/org/apache/druid/segment/metadata/SqlCompactionStateStorage.java @@ -21,9 +21,6 @@ import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.hash.Hasher; -import com.google.common.hash.Hashing; -import com.google.common.io.BaseEncoding; import com.google.inject.Inject; import org.apache.druid.error.InternalServerError; import org.apache.druid.guice.LazySingleton; @@ -241,26 +238,12 @@ public int deleteUnusedCompactionStatesOlderThan(long timestamp) } @Override - @SuppressWarnings("UnstableApiUsage") public String generateCompactionStateFingerprint( final CompactionState compactionState, final String dataSource ) { - final Hasher hasher = Hashing.sha256().newHasher(); - - hasher.putBytes(StringUtils.toUtf8(dataSource)); - hasher.putByte((byte) 0xff); - - try { - hasher.putBytes(deterministicMapper.writeValueAsBytes(compactionState)); - } - catch (JsonProcessingException e) { - throw new RuntimeException("Failed to serialize CompactionState for fingerprinting", e); - } - hasher.putByte((byte) 0xff); - - return BaseEncoding.base16().encode(hasher.hash().asBytes()); + return CompactionStateFingerprints.generate(compactionState, dataSource, deterministicMapper); } diff --git a/server/src/test/java/org/apache/druid/segment/metadata/HeapMemoryCompactionStateStorage.java b/server/src/test/java/org/apache/druid/segment/metadata/HeapMemoryCompactionStateStorage.java index 12d2f2712ff3..beb95c2f9ebc 100644 --- a/server/src/test/java/org/apache/druid/segment/metadata/HeapMemoryCompactionStateStorage.java +++ b/server/src/test/java/org/apache/druid/segment/metadata/HeapMemoryCompactionStateStorage.java @@ -19,15 +19,10 @@ package org.apache.druid.segment.metadata; -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.jackson.DefaultObjectMapper; -import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.timeline.CompactionState; import org.joda.time.DateTime; @@ -82,26 +77,12 @@ private static ObjectMapper createDeterministicMapper() } @Override - @SuppressWarnings("UnstableApiUsage") public String generateCompactionStateFingerprint( final CompactionState compactionState, final String dataSource ) { - final Hasher hasher = Hashing.sha256().newHasher(); - - hasher.putBytes(StringUtils.toUtf8(dataSource)); - hasher.putByte((byte) 0xff); - - try { - hasher.putBytes(deterministicMapper.writeValueAsBytes(compactionState)); - } - catch (JsonProcessingException e) { - throw new RuntimeException("Failed to serialize CompactionState for fingerprinting", e); - } - hasher.putByte((byte) 0xff); - - return BaseEncoding.base16().encode(hasher.hash().asBytes()); + return CompactionStateFingerprints.generate(compactionState, dataSource, deterministicMapper); } @Override From 0c38c768c5601fca2e7695a3642c242e8a718ea9 Mon Sep 17 00:00:00 2001 From: capistrant Date: Fri, 9 Jan 2026 22:48:06 -0600 Subject: [PATCH 47/72] Do some consolidation of fingerprint related classes to clean up code --- .../NewestSegmentFirstPolicyBenchmark.java | 4 +- .../CompactionConfigBasedJobTemplate.java | 3 +- .../indexing/compact/CompactionJobParams.java | 18 +++--- .../indexing/compact/CompactionJobQueue.java | 10 +-- .../compact/OverlordCompactionScheduler.java | 10 +-- .../metadata/CompactionFingerprintMapper.java | 50 +++++++++++++++ .../DefaultCompactionFingerprintMapper.java | 64 +++++++++++++++++++ .../server/compaction/CompactionStatus.java | 32 +++++----- .../DataSourceCompactibleSegmentIterator.java | 15 ++--- ...riorityBasedCompactionSegmentIterator.java | 9 +-- .../coordinator/duty/CompactSegments.java | 3 +- .../SqlSegmentsMetadataQueryTest.java | 15 +++-- .../compaction/CompactionStatusTest.java | 52 +++++++-------- .../NewestSegmentFirstPolicyTest.java | 22 +++---- 14 files changed, 201 insertions(+), 106 deletions(-) create mode 100644 server/src/main/java/org/apache/druid/segment/metadata/CompactionFingerprintMapper.java create mode 100644 server/src/main/java/org/apache/druid/segment/metadata/DefaultCompactionFingerprintMapper.java diff --git a/benchmarks/src/test/java/org/apache/druid/server/coordinator/NewestSegmentFirstPolicyBenchmark.java b/benchmarks/src/test/java/org/apache/druid/server/coordinator/NewestSegmentFirstPolicyBenchmark.java index 02b3ebd49f1b..f6f0e366f9aa 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.DefaultCompactionFingerprintMapper; import org.apache.druid.segment.metadata.HeapMemoryCompactionStateStorage; import org.apache.druid.segment.metadata.NoopCompactionStateCache; import org.apache.druid.server.compaction.CompactionCandidateSearchPolicy; @@ -138,8 +139,7 @@ public void measureNewestSegmentFirstPolicy(Blackhole blackhole) compactionConfigs, dataSources, Collections.emptyMap(), - new HeapMemoryCompactionStateStorage(), - new NoopCompactionStateCache() + new DefaultCompactionFingerprintMapper(new HeapMemoryCompactionStateStorage(), new NoopCompactionStateCache()) ); for (int i = 0; i < numCompactionTaskSlots && iterator.hasNext(); i++) { blackhole.consume(iterator.next()); 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 be7864313733..a08b83522774 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 @@ -136,8 +136,7 @@ DataSourceCompactibleSegmentIterator getCompactibleCandidates( // This policy is used only while creating jobs // The actual order of jobs is determined by the policy used in CompactionJobQueue new NewestSegmentFirstPolicy(null), - params.getCompactionStateStorageImpl(), - params.getCompactionStateCache() + params.getFingerprintMapper() ); // 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 bd9fec817615..0bcb10c163b7 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionJobParams.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionJobParams.java @@ -19,7 +19,7 @@ package org.apache.druid.indexing.compact; -import org.apache.druid.segment.metadata.CompactionStateCache; +import org.apache.druid.segment.metadata.CompactionFingerprintMapper; import org.apache.druid.segment.metadata.CompactionStateStorage; import org.apache.druid.server.compaction.CompactionSnapshotBuilder; import org.apache.druid.server.coordinator.ClusterCompactionConfig; @@ -35,24 +35,24 @@ public class CompactionJobParams private final TimelineProvider timelineProvider; private final ClusterCompactionConfig clusterCompactionConfig; private final CompactionSnapshotBuilder snapshotBuilder; + private final CompactionFingerprintMapper fingerprintMapper; private final CompactionStateStorage compactionStateStorage; - private final CompactionStateCache compactionStateCache; public CompactionJobParams( DateTime scheduleStartTime, ClusterCompactionConfig clusterCompactionConfig, TimelineProvider timelineProvider, CompactionSnapshotBuilder snapshotBuilder, - CompactionStateStorage compactionStateStorage, - CompactionStateCache compactionStateCache + CompactionFingerprintMapper fingerprintMapper, + CompactionStateStorage compactionStateStorage ) { this.scheduleStartTime = scheduleStartTime; this.clusterCompactionConfig = clusterCompactionConfig; this.timelineProvider = timelineProvider; this.snapshotBuilder = snapshotBuilder; + this.fingerprintMapper = fingerprintMapper; this.compactionStateStorage = compactionStateStorage; - this.compactionStateCache = compactionStateCache; } /** @@ -96,14 +96,14 @@ public CompactionSnapshotBuilder getSnapshotBuilder() return snapshotBuilder; } - public CompactionStateStorage getCompactionStateStorageImpl() + public CompactionFingerprintMapper getFingerprintMapper() { - return compactionStateStorage; + return fingerprintMapper; } - public CompactionStateCache getCompactionStateCache() + public CompactionStateStorage getCompactionStateStorageImpl() { - return compactionStateCache; + return compactionStateStorage; } @FunctionalInterface 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 aefc2f17202f..de14b6a36f58 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,7 +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.CompactionStateCache; +import org.apache.druid.segment.metadata.CompactionFingerprintMapper; import org.apache.druid.segment.metadata.CompactionStateStorage; import org.apache.druid.server.compaction.CompactionCandidate; import org.apache.druid.server.compaction.CompactionCandidateSearchPolicy; @@ -107,8 +107,8 @@ public CompactionJobQueue( OverlordClient overlordClient, BrokerClient brokerClient, ObjectMapper objectMapper, - CompactionStateStorage compactionStateStorage, - CompactionStateCache compactionStateCache + CompactionFingerprintMapper fingerprintMapper, + CompactionStateStorage compactionStateStorage ) { this.runStats = new CoordinatorRunStats(); @@ -125,8 +125,8 @@ public CompactionJobQueue( clusterCompactionConfig, dataSourcesSnapshot.getUsedSegmentsTimelinesPerDataSource()::get, snapshotBuilder, - compactionStateStorage, - compactionStateCache + fingerprintMapper, + compactionStateStorage ); 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 d7a352f8fe0d..572ac29eb526 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,8 +44,10 @@ 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.CompactionFingerprintMapper; import org.apache.druid.segment.metadata.CompactionStateCache; import org.apache.druid.segment.metadata.CompactionStateStorage; +import org.apache.druid.segment.metadata.DefaultCompactionFingerprintMapper; import org.apache.druid.server.compaction.CompactionRunSimulator; import org.apache.druid.server.compaction.CompactionSimulateResult; import org.apache.druid.server.compaction.CompactionStatusTracker; @@ -142,7 +144,7 @@ public class OverlordCompactionScheduler implements CompactionScheduler private final long schedulePeriodMillis; private final CompactionStateStorage compactionStateStorage; - private final CompactionStateCache compactionStateCache; + private final CompactionFingerprintMapper fingerprintMapper; @Inject public OverlordCompactionScheduler( @@ -188,7 +190,7 @@ public OverlordCompactionScheduler( this.taskActionClientFactory = taskActionClientFactory; this.druidInputSourceFactory = druidInputSourceFactory; this.compactionStateStorage = compactionStateStorage; - this.compactionStateCache = compactionStateCache; + this.fingerprintMapper = new DefaultCompactionFingerprintMapper(compactionStateStorage, compactionStateCache); this.taskRunnerListener = new TaskRunnerListener() { @Override @@ -376,8 +378,8 @@ private synchronized void resetCompactionJobQueue() overlordClient, brokerClient, objectMapper, - compactionStateStorage, - compactionStateCache + fingerprintMapper, + compactionStateStorage ); latestJobQueue.set(queue); diff --git a/server/src/main/java/org/apache/druid/segment/metadata/CompactionFingerprintMapper.java b/server/src/main/java/org/apache/druid/segment/metadata/CompactionFingerprintMapper.java new file mode 100644 index 000000000000..17f36d930998 --- /dev/null +++ b/server/src/main/java/org/apache/druid/segment/metadata/CompactionFingerprintMapper.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 org.apache.druid.timeline.CompactionState; + +import java.util.Optional; + +/** + * Provides operations for mapping between compaction state fingerprints and their corresponding states. + *

    + * This interface abstracts the fingerprint generation and lookup operations, simplifying + * dependencies and improving testability for classes that need both operations. + */ +public interface CompactionFingerprintMapper +{ + /** + * Generates a deterministic fingerprint for the given compaction state and datasource. + * + * @param dataSource The datasource name + * @param compactionState The compaction configuration to fingerprint + * @return A hex-encoded SHA-256 fingerprint string + */ + String generateFingerprint(String dataSource, CompactionState compactionState); + + /** + * Retrieves a compaction state by its fingerprint. + * + * @param fingerprint The fingerprint to look up + * @return The compaction state, or Optional.empty() if not found + */ + Optional getStateForFingerprint(String fingerprint); +} diff --git a/server/src/main/java/org/apache/druid/segment/metadata/DefaultCompactionFingerprintMapper.java b/server/src/main/java/org/apache/druid/segment/metadata/DefaultCompactionFingerprintMapper.java new file mode 100644 index 000000000000..8b06cbf7d714 --- /dev/null +++ b/server/src/main/java/org/apache/druid/segment/metadata/DefaultCompactionFingerprintMapper.java @@ -0,0 +1,64 @@ +/* + * 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.google.inject.Inject; +import org.apache.druid.guice.LazySingleton; +import org.apache.druid.timeline.CompactionState; + +import javax.annotation.Nullable; +import java.util.Optional; + +/** + * Default implementation of {@link CompactionFingerprintMapper} that delegates to + * {@link CompactionStateStorage} for fingerprint generation and {@link CompactionStateCache} + * for state lookups. + */ +@LazySingleton +public class DefaultCompactionFingerprintMapper implements CompactionFingerprintMapper +{ + private final CompactionStateStorage compactionStateStorage; + private final CompactionStateCache compactionStateCache; + + @Inject + public DefaultCompactionFingerprintMapper( + CompactionStateStorage compactionStateStorage, + @Nullable CompactionStateCache compactionStateCache + ) + { + this.compactionStateStorage = compactionStateStorage; + this.compactionStateCache = compactionStateCache; + } + + @Override + public String generateFingerprint(String dataSource, CompactionState compactionState) + { + return compactionStateStorage.generateCompactionStateFingerprint(compactionState, dataSource); + } + + @Override + public Optional getStateForFingerprint(String fingerprint) + { + if (compactionStateCache == null) { + return Optional.empty(); + } + return compactionStateCache.getCompactionStateByFingerprint(fingerprint); + } +} 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 cf86acbd49e3..cdff901aa192 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 @@ -33,8 +33,7 @@ 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.CompactionStateCache; -import org.apache.druid.segment.metadata.CompactionStateStorage; +import org.apache.druid.segment.metadata.CompactionFingerprintMapper; import org.apache.druid.segment.transform.CompactionTransformSpec; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; import org.apache.druid.server.coordinator.UserCompactionTaskGranularityConfig; @@ -261,21 +260,20 @@ public static CompactionStatus running(String message) static CompactionStatus compute( CompactionCandidate candidateSegments, DataSourceCompactionConfig config, - @Nullable CompactionStateStorage compactionStateStorage, - @Nullable CompactionStateCache compactionStateCache + @Nullable CompactionFingerprintMapper fingerprintMapper ) { final CompactionState expectedState = config.toCompactionState(); String expectedFingerprint; - if (compactionStateStorage == null) { + if (fingerprintMapper == null) { expectedFingerprint = null; } else { - expectedFingerprint = compactionStateStorage.generateCompactionStateFingerprint( - expectedState, - config.getDataSource() + expectedFingerprint = fingerprintMapper.generateFingerprint( + config.getDataSource(), + expectedState ); } - return new Evaluator(candidateSegments, config, expectedFingerprint, compactionStateCache).evaluate(); + return new Evaluator(candidateSegments, config, expectedFingerprint, fingerprintMapper).evaluate(); } @Nullable @@ -360,13 +358,13 @@ private static class Evaluator @Nullable private final String targetFingerprint; - private final CompactionStateCache compactionStateCache; + private final CompactionFingerprintMapper fingerprintMapper; private Evaluator( CompactionCandidate candidateSegments, DataSourceCompactionConfig compactionConfig, @Nullable String targetFingerprint, - @Nullable CompactionStateCache compactionStateCache + @Nullable CompactionFingerprintMapper fingerprintMapper ) { this.candidateSegments = candidateSegments; @@ -374,7 +372,7 @@ private Evaluator( this.tuningConfig = ClientCompactionTaskQueryTuningConfig.from(compactionConfig); this.configuredGranularitySpec = compactionConfig.getGranularitySpec(); this.targetFingerprint = targetFingerprint; - this.compactionStateCache = compactionStateCache; + this.fingerprintMapper = fingerprintMapper; } private CompactionStatus evaluate() @@ -390,7 +388,7 @@ private CompactionStatus evaluate() reasonsForCompaction.add(compactedOnceCheck.getReason()); } - if (compactionStateCache != null && targetFingerprint != null) { + if (fingerprintMapper != null && targetFingerprint != null) { // First try fingerprint-based evaluation (fast path) CompactionStatus fingerprintStatus = FINGERPRINT_CHECKS.stream() .map(f -> f.apply(this)) @@ -460,22 +458,22 @@ private CompactionStatus allFingerprintedCandidatesHaveExpectedFingerprint() return COMPLETE; } - if (compactionStateCache == null) { - // Cannot evaluate further without a compaction state cache + if (fingerprintMapper == null) { + // Cannot evaluate further without a fingerprint mapper uncompactedSegments.addAll( mismatchedFingerprintToSegmentMap.values() .stream() .flatMap(List::stream) .collect(Collectors.toList()) ); - return CompactionStatus.pending("Segments have a mismatched fingerprint and no compaction state cache is available"); + return CompactionStatus.pending("Segments have a mismatched fingerprint and no fingerprint mapper is available"); } boolean fingerprintedSegmentWithoutCachedStateFound = false; for (Map.Entry> e : mismatchedFingerprintToSegmentMap.entrySet()) { String fingerprint = e.getKey(); - CompactionState stateToValidate = compactionStateCache.getCompactionStateByFingerprint(fingerprint).orElse(null); + CompactionState stateToValidate = fingerprintMapper.getStateForFingerprint(fingerprint).orElse(null); if (stateToValidate == null) { log.warn("No compaction state found for fingerprint[%s]", fingerprint); fingerprintedSegmentWithoutCachedStateFound = true; 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 fac7be005893..1453463d0206 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,8 +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.CompactionStateCache; -import org.apache.druid.segment.metadata.CompactionStateStorage; +import org.apache.druid.segment.metadata.CompactionFingerprintMapper; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.Partitions; @@ -70,8 +69,7 @@ public class DataSourceCompactibleSegmentIterator implements CompactionSegmentIt private final String dataSource; private final DataSourceCompactionConfig config; - private final CompactionStateStorage compactionStateStorage; - private final CompactionStateCache compactionStateCache; + private final CompactionFingerprintMapper fingerprintMapper; private final List compactedSegments = new ArrayList<>(); private final List skippedSegments = new ArrayList<>(); @@ -89,15 +87,13 @@ public DataSourceCompactibleSegmentIterator( SegmentTimeline timeline, List skipIntervals, CompactionCandidateSearchPolicy searchPolicy, - CompactionStateStorage compactionStateStorage, - CompactionStateCache compactionStateCache + CompactionFingerprintMapper compactionFingerprintMapper ) { this.config = config; this.dataSource = config.getDataSource(); this.queue = new PriorityQueue<>(searchPolicy::compareCandidates); - this.compactionStateStorage = compactionStateStorage; - this.compactionStateCache = compactionStateCache; + this.fingerprintMapper = compactionFingerprintMapper; populateQueue(timeline, skipIntervals); } @@ -334,8 +330,7 @@ private void findAndEnqueueSegmentsToCompact(CompactibleSegmentIterator compacti } final CompactionCandidate candidates = CompactionCandidate.from(segments, config.getSegmentGranularity()); - final CompactionStatus compactionStatus = CompactionStatus.compute(candidates, config, - compactionStateStorage, compactionStateCache); + final CompactionStatus compactionStatus = CompactionStatus.compute(candidates, config, fingerprintMapper); 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 3f740ee3d8ee..4b0410975095 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,8 +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.CompactionStateCache; -import org.apache.druid.segment.metadata.CompactionStateStorage; +import org.apache.druid.segment.metadata.CompactionFingerprintMapper; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; import org.apache.druid.timeline.SegmentTimeline; import org.joda.time.Interval; @@ -51,8 +50,7 @@ public PriorityBasedCompactionSegmentIterator( Map compactionConfigs, Map datasourceToTimeline, Map> skipIntervals, - CompactionStateStorage compactionStateStorage, - CompactionStateCache compactionStateCache + CompactionFingerprintMapper compactionFingerprintMapper ) { this.queue = new PriorityQueue<>(searchPolicy::compareCandidates); @@ -74,8 +72,7 @@ public PriorityBasedCompactionSegmentIterator( timeline, skipIntervals.getOrDefault(datasource, Collections.emptyList()), searchPolicy, - compactionStateStorage, - compactionStateCache + compactionFingerprintMapper ) ); addNextItemForDatasourceToQueue(datasource); diff --git a/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java b/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java index c3140ce59ae2..38ff8de92619 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 @@ -183,9 +183,8 @@ public void run( compactionConfigs, dataSources.getUsedSegmentsTimelinesPerDataSource(), slotManager.getDatasourceIntervalsToSkipCompaction(), - null, null - ); + ); final CompactionSnapshotBuilder compactionSnapshotBuilder = new CompactionSnapshotBuilder(stats); final int numSubmittedCompactionTasks = submitCompactionTasks( diff --git a/server/src/test/java/org/apache/druid/metadata/SqlSegmentsMetadataQueryTest.java b/server/src/test/java/org/apache/druid/metadata/SqlSegmentsMetadataQueryTest.java index acc055455c40..a143e6c4b9c4 100644 --- a/server/src/test/java/org/apache/druid/metadata/SqlSegmentsMetadataQueryTest.java +++ b/server/src/test/java/org/apache/druid/metadata/SqlSegmentsMetadataQueryTest.java @@ -388,7 +388,7 @@ public void test_retrieveAllUsedCompactionStateFingerprints_emptyDatabase() } @Test - public void test_retrieveAllUsedCompactionStateFingerprints_onlyUsedSegments() + public void test_retrieveAllUsedCompactionStateFingerprints() { derbyConnectorRule.getConnector().createCompactionStatesTable(); @@ -407,7 +407,7 @@ public void test_retrieveAllUsedCompactionStateFingerprints_onlyUsedSegments() Set fingerprints = read(SqlSegmentsMetadataQuery::retrieveAllUsedCompactionStateFingerprints); - Assert.assertEquals("Should return only fingerprints from used segments", Set.of("fp1", "fp2"), fingerprints); + Assert.assertEquals("Should return all fingerprints in the cache", Set.of("fp1", "fp2", "fp3"), fingerprints); } @Test @@ -453,11 +453,12 @@ public void test_retrieveAllUsedCompactionStates_fullSync() null, null ); + CompactionState state3 = createTestCompactionState(); Map compactionStates = new HashMap<>(); compactionStates.put("fp1", state1); compactionStates.put("fp2", state2); - compactionStates.put("fp3", createTestCompactionState()); // Unreferenced state + compactionStates.put("fp3", state3); // Unreferenced state insertCompactionStates(compactionStates); // Only reference fp1 and fp2 @@ -466,12 +467,12 @@ public void test_retrieveAllUsedCompactionStates_fullSync() List records = read(SqlSegmentsMetadataQuery::retrieveAllUsedCompactionStates); - Assert.assertEquals("Should return only referenced compaction states", 2, records.size()); + Assert.assertEquals("Should return all compaction states", 3, records.size()); Set retrievedFingerprints = records.stream() .map(CompactionStateRecord::getFingerprint) .collect(Collectors.toSet()); - Assert.assertEquals("Should contain fp1 and fp2", Set.of("fp1", "fp2"), retrievedFingerprints); + Assert.assertEquals("Should contain all fps", Set.of("fp1", "fp2", "fp3"), retrievedFingerprints); // Verify payloads Map retrievedStates = records.stream() @@ -481,6 +482,7 @@ public void test_retrieveAllUsedCompactionStates_fullSync() )); Assert.assertEquals("fp1 state should match", state1, retrievedStates.get("fp1")); Assert.assertEquals("fp2 state should match", state2, retrievedStates.get("fp2")); + Assert.assertEquals("fp3 state should match", state3, retrievedStates.get("fp3")); } @Test @@ -498,8 +500,7 @@ public void test_retrieveAllUsedCompactionStates_onlyFromUsedSegments() List records = read(SqlSegmentsMetadataQuery::retrieveAllUsedCompactionStates); - Assert.assertEquals("Should only return states from used segments", 1, records.size()); - Assert.assertEquals("Should return fp1", "fp1", records.get(0).getFingerprint()); + Assert.assertEquals("Should only return all compaction states", 2, records.size()); } @Test 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 668af1cb3231..b6f645e12a9d 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 @@ -39,8 +39,10 @@ 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.CompactionFingerprintMapper; import org.apache.druid.segment.metadata.CompactionStateCache; import org.apache.druid.segment.metadata.CompactionStateStorage; +import org.apache.druid.segment.metadata.DefaultCompactionFingerprintMapper; import org.apache.druid.segment.metadata.HeapMemoryCompactionStateStorage; import org.apache.druid.segment.nested.NestedCommonFormatColumnFormatSpec; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; @@ -71,12 +73,17 @@ public class CompactionStatusTest private HeapMemoryCompactionStateStorage compactionStateStorage; private CompactionStateCache compactionStateCache; + private CompactionFingerprintMapper fingerprintMapper; @Before public void setUp() { compactionStateStorage = new HeapMemoryCompactionStateStorage(); compactionStateCache = new CompactionStateCache(); + fingerprintMapper = new DefaultCompactionFingerprintMapper( + compactionStateStorage, + compactionStateCache + ); } /** @@ -355,9 +362,8 @@ public void testStatusWhenLastCompactionStateSameAsRequired() final CompactionStatus status = CompactionStatus.compute( CompactionCandidate.from(List.of(segment), Granularities.HOUR), compactionConfig, - compactionStateStorage, - compactionStateCache - ); + fingerprintMapper + ); Assert.assertTrue(status.isComplete()); } @@ -406,9 +412,8 @@ public void testStatusWhenProjectionsMatch() final CompactionStatus status = CompactionStatus.compute( CompactionCandidate.from(List.of(segment), Granularities.HOUR), compactionConfig, - compactionStateStorage, - compactionStateCache - ); + fingerprintMapper + ); Assert.assertTrue(status.isComplete()); } @@ -462,9 +467,8 @@ public void testStatusWhenProjectionsMismatch() final CompactionStatus status = CompactionStatus.compute( CompactionCandidate.from(List.of(segment), Granularities.HOUR), compactionConfig, - compactionStateStorage, - compactionStateCache - ); + fingerprintMapper + ); Assert.assertFalse(status.isComplete()); } @@ -517,9 +521,8 @@ public void testStatusWhenAutoSchemaMatch() final CompactionStatus status = CompactionStatus.compute( CompactionCandidate.from(List.of(segment), null), compactionConfig, - compactionStateStorage, - compactionStateCache - ); + fingerprintMapper + ); Assert.assertTrue(status.isComplete()); } @@ -572,9 +575,8 @@ public void testStatusWhenAutoSchemaMismatch() final CompactionStatus status = CompactionStatus.compute( CompactionCandidate.from(List.of(segment), null), compactionConfig, - compactionStateStorage, - compactionStateCache - ); + fingerprintMapper + ); Assert.assertFalse(status.isComplete()); } @@ -669,8 +671,7 @@ public void test_evaluate_noCompacationIfUnexpectedFingerprintHasExpectedCompact final CompactionStatus status = CompactionStatus.compute( CompactionCandidate.from(segments, null), compactionConfig, - compactionStateStorage, - compactionStateCache + fingerprintMapper ); Assert.assertTrue(status.isComplete()); } @@ -717,8 +718,7 @@ public void test_evaluate_noCompactionWhenAllSegmentsHaveExpectedCompactionState final CompactionStatus status = CompactionStatus.compute( CompactionCandidate.from(segments, null), compactionConfig, - compactionStateStorage, - compactionStateCache + fingerprintMapper ); Assert.assertTrue(status.isComplete()); } @@ -771,8 +771,7 @@ public void test_evaluate_noCompactionWhenNonFingerprintedSegmentsPassChecksOnLa final CompactionStatus status = CompactionStatus.compute( CompactionCandidate.from(segments, null), compactionConfig, - compactionStateStorage, - compactionStateCache + fingerprintMapper ); Assert.assertTrue(status.isComplete()); } @@ -802,8 +801,7 @@ public void test_evaluate_isSkippedWhenInputBytesExceedLimit() final CompactionStatus status = CompactionStatus.compute( CompactionCandidate.from(segments, null), compactionConfig, - compactionStateStorage, - compactionStateCache + fingerprintMapper ); Assert.assertFalse(status.isComplete()); @@ -828,8 +826,7 @@ private void verifyEvaluationNeedsCompactionBecauseWithCustomSegments( final CompactionStatus status = CompactionStatus.compute( candidate, compactionConfig, - compactionStateStorage, - compactionStateCache + new DefaultCompactionFingerprintMapper(compactionStateStorage, compactionStateCache) ); Assert.assertFalse(status.isComplete()); @@ -849,9 +846,8 @@ private void verifyCompactionStatusIsPendingBecause( final CompactionStatus status = CompactionStatus.compute( CompactionCandidate.from(List.of(segment), null), compactionConfig, - compactionStateStorage, - compactionStateCache - ); + fingerprintMapper + ); Assert.assertFalse(status.isComplete()); Assert.assertEquals(expectedReason, status.getReason()); diff --git a/server/src/test/java/org/apache/druid/server/compaction/NewestSegmentFirstPolicyTest.java b/server/src/test/java/org/apache/druid/server/compaction/NewestSegmentFirstPolicyTest.java index 4a9a0603b002..087d9aea9031 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,8 +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.CompactionStateCache; -import org.apache.druid.segment.metadata.CompactionStateStorage; +import org.apache.druid.segment.metadata.CompactionFingerprintMapper; +import org.apache.druid.segment.metadata.DefaultCompactionFingerprintMapper; import org.apache.druid.segment.metadata.HeapMemoryCompactionStateStorage; import org.apache.druid.segment.metadata.NoopCompactionStateCache; import org.apache.druid.segment.transform.CompactionTransformSpec; @@ -85,8 +85,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 CompactionStateStorage compactionStateStorage = new HeapMemoryCompactionStateStorage(); - private final CompactionStateCache compactionStateCache = new NoopCompactionStateCache(); + private final CompactionFingerprintMapper fingerprintMapper = new DefaultCompactionFingerprintMapper(new HeapMemoryCompactionStateStorage(), new NoopCompactionStateCache()); @Test public void testLargeOffsetAndSmallSegmentInterval() @@ -283,8 +282,7 @@ public void testSkipDataSourceWithNoSegments() ) ), Collections.emptyMap(), - compactionStateStorage, - compactionStateCache + fingerprintMapper ); assertCompactSegmentIntervals( @@ -517,8 +515,7 @@ public void testWithSkipIntervals() Intervals.of("2017-11-13T00:00:00/2017-11-14T01:00:00") ) ), - compactionStateStorage, - compactionStateCache + fingerprintMapper ); assertCompactSegmentIntervals( @@ -558,8 +555,7 @@ public void testHoleInSearchInterval() Intervals.of("2017-11-16T14:00:00/2017-11-16T20:00:00") ) ), - compactionStateStorage, - compactionStateCache + fingerprintMapper ); assertCompactSegmentIntervals( @@ -2065,8 +2061,7 @@ TestDataSource.KOALA, configBuilder().forDataSource(TestDataSource.KOALA).build( TestDataSource.KOALA, SegmentTimeline.forSegments(koalaSegments) ), Collections.emptyMap(), - compactionStateStorage, - compactionStateCache + fingerprintMapper ); // Verify that the segments of WIKI are preferred even though they are older @@ -2088,8 +2083,7 @@ private CompactionSegmentIterator createIterator(DataSourceCompactionConfig conf Collections.singletonMap(TestDataSource.WIKI, config), Collections.singletonMap(TestDataSource.WIKI, timeline), Collections.emptyMap(), - compactionStateStorage, - compactionStateCache + fingerprintMapper ); } From f62e2ea28fa438d0b177f6e44938aa4f2aa83f5c Mon Sep 17 00:00:00 2001 From: capistrant Date: Fri, 9 Jan 2026 23:10:08 -0600 Subject: [PATCH 48/72] minor cleanup --- .../org/apache/druid/indexing/compact/CompactionJob.java | 4 ++-- .../druid/metadata/segment/cache/CompactionStateRecord.java | 2 +- .../compaction-dynamic-config-completions.ts | 5 ++--- .../compaction-dynamic-config/compaction-dynamic-config.tsx | 4 ++-- 4 files changed, 7 insertions(+), 8 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionJob.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionJob.java index 4ceaec85b6ea..93fba80a0e4d 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionJob.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionJob.java @@ -97,8 +97,8 @@ public String toString() super.toString() + ", candidate=" + candidate + ", maxRequiredTaskSlots=" + maxRequiredTaskSlots + - ", compactionStateFingerprint='" + targetCompactionStateFingerprint + '\'' + - ", compactionState=" + targetCompactionState + + ", targetCompactionStateFingerprint='" + targetCompactionStateFingerprint + '\'' + + ", targetCompactionState=" + targetCompactionState + '}'; } } diff --git a/server/src/main/java/org/apache/druid/metadata/segment/cache/CompactionStateRecord.java b/server/src/main/java/org/apache/druid/metadata/segment/cache/CompactionStateRecord.java index 1595febec683..d6bbd268c2d2 100644 --- a/server/src/main/java/org/apache/druid/metadata/segment/cache/CompactionStateRecord.java +++ b/server/src/main/java/org/apache/druid/metadata/segment/cache/CompactionStateRecord.java @@ -22,7 +22,7 @@ import org.apache.druid.timeline.CompactionState; /** - * Represents a single record in the druid_compaction_states table. + * Represents a single record in the druid_compactionStates table. */ public class CompactionStateRecord { 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 688934fad885..1fa4b7b24160 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 @@ -45,7 +45,7 @@ export const COMPACTION_DYNAMIC_CONFIG_COMPLETIONS: JsonCompletionRule[] = [ documentation: 'Engine used for running compaction tasks (native or msq)', }, { - value: 'legacyPersistLastCompactionStateInSegments', + value: 'storeCompactionStatePerSegment', documentation: 'Whether to persist the full compaction state in segment metadata (default: true)', }, @@ -121,9 +121,8 @@ export const COMPACTION_DYNAMIC_CONFIG_COMPLETIONS: JsonCompletionRule[] = [ condition: obj => !obj.useSupervisors, completions: [{ value: 'native', documentation: 'Native indexing engine (default)' }], }, - // legacyPersistLastCompactionStateInSegments values { - path: '$.legacyPersistLastCompactionStateInSegments', + path: '$.storeCompactionStatePerSegment', completions: [ { value: 'true', 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 1eff27e51d6e..7d3c766c8aad 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,7 +27,7 @@ export interface CompactionDynamicConfig { compactionPolicy: { type: 'newestSegmentFirst'; priorityDatasource?: string | null }; useSupervisors: boolean; engine: 'native' | 'msq'; - legacyPersistLastCompactionStateInSegments: boolean; + storeCompactionStatePerSegment: boolean; } export const COMPACTION_DYNAMIC_CONFIG_DEFAULT_RATIO = 0.1; @@ -96,7 +96,7 @@ export const COMPACTION_DYNAMIC_CONFIG_FIELDS: Field[] ), }, { - name: 'legacyPersistLastCompactionStateInSegments', + name: 'storeCompactionStatePerSegment', label: 'Legacy: Persist last compaction state in segments', type: 'boolean', defaultValue: true, From c81f242d75c60538082f0678ed0217b7ccc1becb Mon Sep 17 00:00:00 2001 From: capistrant Date: Sun, 11 Jan 2026 10:00:00 -0600 Subject: [PATCH 49/72] fix fobidden api use --- .../druid/segment/metadata/SqlCompactionStateStorage.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/src/main/java/org/apache/druid/segment/metadata/SqlCompactionStateStorage.java b/server/src/main/java/org/apache/druid/segment/metadata/SqlCompactionStateStorage.java index 486ca3980fb7..9b2ef20fddab 100644 --- a/server/src/main/java/org/apache/druid/segment/metadata/SqlCompactionStateStorage.java +++ b/server/src/main/java/org/apache/druid/segment/metadata/SqlCompactionStateStorage.java @@ -341,7 +341,7 @@ private boolean isUniqueConstraintViolation(Exception e) // Also check exception message as fallback String message = e.getMessage(); if (message != null) { - String lowerMessage = message.toLowerCase(); + String lowerMessage = StringUtils.toLowerCase(message); return lowerMessage.contains("unique constraint") || lowerMessage.contains("duplicate key") || lowerMessage.contains("duplicate entry"); From ed2c06c51398ac7900104831efb86731217d5940 Mon Sep 17 00:00:00 2001 From: capistrant Date: Mon, 12 Jan 2026 10:18:55 -0600 Subject: [PATCH 50/72] Improvements and cleanup to the fingerprint and state persist + cache --- .../CompactionConfigBasedJobTemplate.java | 6 +- .../indexing/compact/CompactionJobParams.java | 13 +- .../indexing/compact/CompactionJobQueue.java | 19 ++- .../compact/OverlordCompactionScheduler.java | 10 +- .../metadata/CompactionStateCache.java | 42 +++++ .../DefaultCompactionFingerprintMapper.java | 4 - .../metadata/SqlCompactionStateStorage.java | 154 +++++++++++------- .../metadata/CompactionStateCacheTest.java | 62 +++++++ .../SqlCompactionStateStorageTest.java | 57 +++++++ 9 files changed, 277 insertions(+), 90 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionConfigBasedJobTemplate.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionConfigBasedJobTemplate.java index a08b83522774..b41e13d43f2f 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 @@ -73,9 +73,9 @@ public List createCompactionJobs( CompactionState compactionState = config.toCompactionState(); - String compactionStateFingerprint = params.getCompactionStateStorageImpl().generateCompactionStateFingerprint( - compactionState, - config.getDataSource() + String compactionStateFingerprint = params.getFingerprintMapper().generateFingerprint( + config.getDataSource(), + compactionState ); // Create a job for each CompactionCandidate 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 0bcb10c163b7..d0e8219e742c 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionJobParams.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionJobParams.java @@ -20,7 +20,6 @@ package org.apache.druid.indexing.compact; import org.apache.druid.segment.metadata.CompactionFingerprintMapper; -import org.apache.druid.segment.metadata.CompactionStateStorage; import org.apache.druid.server.compaction.CompactionSnapshotBuilder; import org.apache.druid.server.coordinator.ClusterCompactionConfig; import org.apache.druid.timeline.SegmentTimeline; @@ -36,23 +35,20 @@ public class CompactionJobParams private final ClusterCompactionConfig clusterCompactionConfig; private final CompactionSnapshotBuilder snapshotBuilder; private final CompactionFingerprintMapper fingerprintMapper; - private final CompactionStateStorage compactionStateStorage; public CompactionJobParams( DateTime scheduleStartTime, ClusterCompactionConfig clusterCompactionConfig, TimelineProvider timelineProvider, CompactionSnapshotBuilder snapshotBuilder, - CompactionFingerprintMapper fingerprintMapper, - CompactionStateStorage compactionStateStorage + CompactionFingerprintMapper compactionFingerprintMapper ) { this.scheduleStartTime = scheduleStartTime; this.clusterCompactionConfig = clusterCompactionConfig; this.timelineProvider = timelineProvider; this.snapshotBuilder = snapshotBuilder; - this.fingerprintMapper = fingerprintMapper; - this.compactionStateStorage = compactionStateStorage; + this.fingerprintMapper = compactionFingerprintMapper; } /** @@ -101,11 +97,6 @@ public CompactionFingerprintMapper getFingerprintMapper() return fingerprintMapper; } - public CompactionStateStorage getCompactionStateStorageImpl() - { - return compactionStateStorage; - } - @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 de14b6a36f58..6f094d35fb51 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,8 +36,9 @@ 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.CompactionFingerprintMapper; +import org.apache.druid.segment.metadata.CompactionStateCache; import org.apache.druid.segment.metadata.CompactionStateStorage; +import org.apache.druid.segment.metadata.DefaultCompactionFingerprintMapper; import org.apache.druid.server.compaction.CompactionCandidate; import org.apache.druid.server.compaction.CompactionCandidateSearchPolicy; import org.apache.druid.server.compaction.CompactionSlotManager; @@ -98,6 +99,9 @@ public class CompactionJobQueue private final Set activeSupervisors; private final Map submittedTaskIdToJob; + private final CompactionStateStorage compactionStateStorage; + private final CompactionStateCache compactionStateCache; + public CompactionJobQueue( DataSourcesSnapshot dataSourcesSnapshot, ClusterCompactionConfig clusterCompactionConfig, @@ -107,8 +111,8 @@ public CompactionJobQueue( OverlordClient overlordClient, BrokerClient brokerClient, ObjectMapper objectMapper, - CompactionFingerprintMapper fingerprintMapper, - CompactionStateStorage compactionStateStorage + CompactionStateStorage compactionStateStorage, + CompactionStateCache compactionStateCache ) { this.runStats = new CoordinatorRunStats(); @@ -125,10 +129,12 @@ public CompactionJobQueue( clusterCompactionConfig, dataSourcesSnapshot.getUsedSegmentsTimelinesPerDataSource()::get, snapshotBuilder, - fingerprintMapper, - compactionStateStorage + new DefaultCompactionFingerprintMapper(compactionStateStorage, compactionStateCache) ); + this.compactionStateStorage = compactionStateStorage; + this.compactionStateCache = compactionStateCache; + this.taskActionClientFactory = taskActionClientFactory; this.overlordClient = overlordClient; this.brokerClient = brokerClient; @@ -362,12 +368,13 @@ private String startTaskIfReady(CompactionJob job) private void persistPendingCompactionState(CompactionJob job) { if (job.getTargetCompactionState() != null && job.getTargetCompactionStateFingerprint() != null) { - jobParams.getCompactionStateStorageImpl().upsertCompactionState( + compactionStateStorage.upsertCompactionState( job.getDataSource(), job.getTargetCompactionStateFingerprint(), job.getTargetCompactionState(), DateTimes.nowUtc() ); + compactionStateCache.addCompactionState(job.getTargetCompactionStateFingerprint(), job.getTargetCompactionState()); } } 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 572ac29eb526..d7a352f8fe0d 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,10 +44,8 @@ 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.CompactionFingerprintMapper; import org.apache.druid.segment.metadata.CompactionStateCache; import org.apache.druid.segment.metadata.CompactionStateStorage; -import org.apache.druid.segment.metadata.DefaultCompactionFingerprintMapper; import org.apache.druid.server.compaction.CompactionRunSimulator; import org.apache.druid.server.compaction.CompactionSimulateResult; import org.apache.druid.server.compaction.CompactionStatusTracker; @@ -144,7 +142,7 @@ public class OverlordCompactionScheduler implements CompactionScheduler private final long schedulePeriodMillis; private final CompactionStateStorage compactionStateStorage; - private final CompactionFingerprintMapper fingerprintMapper; + private final CompactionStateCache compactionStateCache; @Inject public OverlordCompactionScheduler( @@ -190,7 +188,7 @@ public OverlordCompactionScheduler( this.taskActionClientFactory = taskActionClientFactory; this.druidInputSourceFactory = druidInputSourceFactory; this.compactionStateStorage = compactionStateStorage; - this.fingerprintMapper = new DefaultCompactionFingerprintMapper(compactionStateStorage, compactionStateCache); + this.compactionStateCache = compactionStateCache; this.taskRunnerListener = new TaskRunnerListener() { @Override @@ -378,8 +376,8 @@ private synchronized void resetCompactionJobQueue() overlordClient, brokerClient, objectMapper, - fingerprintMapper, - compactionStateStorage + compactionStateStorage, + compactionStateCache ); latestJobQueue.set(queue); diff --git a/server/src/main/java/org/apache/druid/segment/metadata/CompactionStateCache.java b/server/src/main/java/org/apache/druid/segment/metadata/CompactionStateCache.java index 0decfd3c68c0..99eedea4b172 100644 --- a/server/src/main/java/org/apache/druid/segment/metadata/CompactionStateCache.java +++ b/server/src/main/java/org/apache/druid/segment/metadata/CompactionStateCache.java @@ -23,6 +23,7 @@ import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.timeline.CompactionState; +import java.util.HashMap; import java.util.Map; import java.util.Optional; import java.util.concurrent.atomic.AtomicInteger; @@ -98,6 +99,47 @@ public Optional getCompactionStateByFingerprint(String fingerpr } } + /** + * Adds or updates a single compaction state in the cache. + *

    + * This is called when a new compaction state is persisted to the database via upsertCompactionState + * to ensure the cache is immediately consistent without waiting for the next sync. + *

    + * This method checks if the state is already cached before performing the atomic update. + * + * @param fingerprint The fingerprint key + * @param state The compaction state to cache + */ + public void addCompactionState(String fingerprint, CompactionState state) + { + if (fingerprint == null || state == null) { + return; + } + + // Check if the state is already cached - avoid expensive update if not needed + CompactionState existing = publishedCompactionStates.get() + .fingerprintToStateMap + .get(fingerprint); + if (state.equals(existing)) { + log.debug("Compaction state for fingerprint[%s] already cached, skipping update", fingerprint); + return; + } + + // State is not cached or different - perform atomic update + publishedCompactionStates.updateAndGet(current -> { + // Double-check in case another thread updated between our check and now + if (state.equals(current.fingerprintToStateMap.get(fingerprint))) { + return current; + } + + Map newMap = new HashMap<>(current.fingerprintToStateMap); + newMap.put(fingerprint, state); + return new PublishedCompactionStates(newMap); + }); + + log.debug("Added compaction state to cache for fingerprint[%s]", fingerprint); + } + /** * Gets the full cached map (immutable copy). * Used by HeapMemorySegmentMetadataCache for delta sync calculations. diff --git a/server/src/main/java/org/apache/druid/segment/metadata/DefaultCompactionFingerprintMapper.java b/server/src/main/java/org/apache/druid/segment/metadata/DefaultCompactionFingerprintMapper.java index 8b06cbf7d714..03066c159e88 100644 --- a/server/src/main/java/org/apache/druid/segment/metadata/DefaultCompactionFingerprintMapper.java +++ b/server/src/main/java/org/apache/druid/segment/metadata/DefaultCompactionFingerprintMapper.java @@ -19,8 +19,6 @@ package org.apache.druid.segment.metadata; -import com.google.inject.Inject; -import org.apache.druid.guice.LazySingleton; import org.apache.druid.timeline.CompactionState; import javax.annotation.Nullable; @@ -31,13 +29,11 @@ * {@link CompactionStateStorage} for fingerprint generation and {@link CompactionStateCache} * for state lookups. */ -@LazySingleton public class DefaultCompactionFingerprintMapper implements CompactionFingerprintMapper { private final CompactionStateStorage compactionStateStorage; private final CompactionStateCache compactionStateCache; - @Inject public DefaultCompactionFingerprintMapper( CompactionStateStorage compactionStateStorage, @Nullable CompactionStateCache compactionStateCache diff --git a/server/src/main/java/org/apache/druid/segment/metadata/SqlCompactionStateStorage.java b/server/src/main/java/org/apache/druid/segment/metadata/SqlCompactionStateStorage.java index 9b2ef20fddab..6aadc1882625 100644 --- a/server/src/main/java/org/apache/druid/segment/metadata/SqlCompactionStateStorage.java +++ b/server/src/main/java/org/apache/druid/segment/metadata/SqlCompactionStateStorage.java @@ -90,63 +90,78 @@ public void upsertCompactionState( try { connector.retryWithHandle(handle -> { - // Check if the fingerprint already exists - final boolean fingerprintExists = isExistingFingerprint(handle, fingerprint); + // Check if the fingerprint already exists and its used status + final FingerprintState state = getFingerprintState(handle, fingerprint); final String now = updateTime.toString(); - if (fingerprintExists) { - // Fingerprint exists - update the used flag - log.info( - "Found already existing compaction state in DB for fingerprint[%s] in dataSource[%s].", - fingerprint, - dataSource - ); - String updateSql = StringUtils.format( - "UPDATE %s SET used = :used, used_status_last_updated = :used_status_last_updated " - + "WHERE fingerprint = :fingerprint", - dbTables.getCompactionStatesTable() - ); - handle.createStatement(updateSql) - .bind("used", true) - .bind("used_status_last_updated", now) - .bind("fingerprint", fingerprint) - .execute(); - - log.info("Updated existing compaction state for datasource[%s].", dataSource); - } else { - - // Fingerprint doesn't exist - insert new state - log.info("Inserting new compaction state for fingerprint[%s] in dataSource[%s].", fingerprint, dataSource); - - 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() - ); - - try { - handle.createStatement(insertSql) - .bind("created_date", now) - .bind("dataSource", dataSource) - .bind("fingerprint", fingerprint) - .bind("payload", jsonMapper.writeValueAsBytes(compactionState)) - .bind("used", true) - .bind("used_status_last_updated", now) - .execute(); + switch (state) { + case EXISTS_AND_USED: + // Fingerprint exists and is already marked as used - no operation needed + log.debug( + "Compaction state for fingerprint[%s] in dataSource[%s] already exists and is marked as used. Skipping update.", + fingerprint, + dataSource + ); + break; + case EXISTS_AND_UNUSED: + // Fingerprint exists but is marked as unused - update the used flag log.info( - "Published compaction state for fingerprint[%s] to DB for datasource[%s].", + "Found existing compaction state in DB for fingerprint[%s] in dataSource[%s]. Marking as used.", fingerprint, dataSource ); - } - catch (JsonProcessingException e) { - throw InternalServerError.exception( - e, - "Failed to serialize compaction state for fingerprint[%s]", - fingerprint + String updateSql = StringUtils.format( + "UPDATE %s SET used = :used, used_status_last_updated = :used_status_last_updated " + + "WHERE fingerprint = :fingerprint", + dbTables.getCompactionStatesTable() ); - } + handle.createStatement(updateSql) + .bind("used", true) + .bind("used_status_last_updated", now) + .bind("fingerprint", fingerprint) + .execute(); + + log.info("Updated existing compaction state for datasource[%s].", dataSource); + break; + + case DOES_NOT_EXIST: + // Fingerprint doesn't exist - insert new state + log.info("Inserting new compaction state for fingerprint[%s] in dataSource[%s].", fingerprint, dataSource); + + 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() + ); + + try { + handle.createStatement(insertSql) + .bind("created_date", now) + .bind("dataSource", dataSource) + .bind("fingerprint", fingerprint) + .bind("payload", jsonMapper.writeValueAsBytes(compactionState)) + .bind("used", true) + .bind("used_status_last_updated", now) + .execute(); + + log.info( + "Published compaction state for fingerprint[%s] to DB for datasource[%s].", + fingerprint, + dataSource + ); + } + catch (JsonProcessingException e) { + throw InternalServerError.exception( + e, + "Failed to serialize compaction state for fingerprint[%s]", + fingerprint + ); + } + break; + + default: + throw new IllegalStateException("Unknown fingerprint state: " + state); } return null; }); @@ -248,32 +263,51 @@ public String generateCompactionStateFingerprint( /** - * Checks if a fingerprint already exists in the metadata DB. + * Represents the state of an indexing state fingerprint in the database. + *

    + * Intent is to help upsert logic decide whether to insert, update, or skip operations. + */ + private enum FingerprintState + { + /** Fingerprint does not exist in the database */ + DOES_NOT_EXIST, + /** Fingerprint exists and is marked as used */ + EXISTS_AND_USED, + /** Fingerprint exists but is marked as unused */ + EXISTS_AND_UNUSED + } + + /** + * Checks the state of a fingerprint in the metadata DB. * - * @param handle Database handle + * @param handle Database handle * @param fingerprintToCheck The fingerprint to check - * @return true if the fingerprint exists, false otherwise + * @return The state of the fingerprint (exists and used, exists and unused, or does not exist) */ - private boolean isExistingFingerprint( + private FingerprintState getFingerprintState( final Handle handle, @Nonnull final String fingerprintToCheck ) { if (fingerprintToCheck.isEmpty()) { - return false; + return FingerprintState.DOES_NOT_EXIST; } String sql = StringUtils.format( - "SELECT COUNT(*) FROM %s WHERE fingerprint = :fingerprint", + "SELECT used FROM %s WHERE fingerprint = :fingerprint", dbTables.getCompactionStatesTable() ); - Integer count = handle.createQuery(sql) - .bind("fingerprint", fingerprintToCheck) - .mapTo(Integer.class) - .first(); + Boolean used = handle.createQuery(sql) + .bind("fingerprint", fingerprintToCheck) + .mapTo(Boolean.class) + .first(); + + if (used == null) { + return FingerprintState.DOES_NOT_EXIST; + } - return count != null && count > 0; + return used ? FingerprintState.EXISTS_AND_USED : FingerprintState.EXISTS_AND_UNUSED; } /** diff --git a/server/src/test/java/org/apache/druid/segment/metadata/CompactionStateCacheTest.java b/server/src/test/java/org/apache/druid/segment/metadata/CompactionStateCacheTest.java index 89603685e924..233656855e31 100644 --- a/server/src/test/java/org/apache/druid/segment/metadata/CompactionStateCacheTest.java +++ b/server/src/test/java/org/apache/druid/segment/metadata/CompactionStateCacheTest.java @@ -224,6 +224,68 @@ public void test_resetWithEmptyMap() assertEquals(0, stats.get(Metric.COMPACTION_STATE_CACHE_FINGERPRINTS)); } + @Test + public void test_addCompactionState_addsNewStateToCache() + { + CompactionState state = createTestCompactionState(); + String fingerprint = "test_fingerprint_123"; + + // Initially, cache should not have the state + assertEquals(Optional.empty(), cache.getCompactionStateByFingerprint(fingerprint)); + + // Add the state to cache + cache.addCompactionState(fingerprint, state); + + // Now cache should have the state + assertEquals(Optional.of(state), cache.getCompactionStateByFingerprint(fingerprint)); + } + + @Test + public void test_addCompactionState_withDifferentStateForSameFingerprint_updatesCache() + { + CompactionState state1 = createTestCompactionState(); + CompactionState state2 = new CompactionState( + new DynamicPartitionsSpec(200, null), + DimensionsSpec.EMPTY, + null, + null, + IndexSpec.getDefault(), + null, + null + ); + String fingerprint = "same_fp"; + + // Add first state + cache.addCompactionState(fingerprint, state1); + assertEquals(Optional.of(state1), cache.getCompactionStateByFingerprint(fingerprint)); + + // Add different state with same fingerprint + cache.addCompactionState(fingerprint, state2); + + // Cache should now have the new state + assertEquals(Optional.of(state2), cache.getCompactionStateByFingerprint(fingerprint)); + } + + @Test + public void test_addCompactionState_withNullFingerprint_doesNothing() + { + CompactionState state = createTestCompactionState(); + + cache.addCompactionState(null, state); + + // Cache should remain empty + assertEquals(0, cache.getPublishedCompactionStateMap().size()); + } + + @Test + public void test_addCompactionState_withNullState_doesNothing() + { + cache.addCompactionState("some_fp", null); + + // Cache should remain empty + assertEquals(0, cache.getPublishedCompactionStateMap().size()); + } + private CompactionState createTestCompactionState() { return new CompactionState( diff --git a/server/src/test/java/org/apache/druid/segment/metadata/SqlCompactionStateStorageTest.java b/server/src/test/java/org/apache/druid/segment/metadata/SqlCompactionStateStorageTest.java index 45b321575e2f..4d6a0debac82 100644 --- a/server/src/test/java/org/apache/druid/segment/metadata/SqlCompactionStateStorageTest.java +++ b/server/src/test/java/org/apache/druid/segment/metadata/SqlCompactionStateStorageTest.java @@ -346,6 +346,63 @@ public void test_upsertCompactionState_verifyExistingFingerprintMarkedUsed() assertEquals(1, count); } + @Test + public void test_upsertCompactionState_whenAlreadyUsed_skipsUpdate() + { + String fingerprint = "already_used_fingerprint"; + CompactionState state = createTestCompactionState(); + DateTime initialTime = DateTimes.of("2024-01-01T00:00:00.000Z"); + + // Insert fingerprint as used initially + derbyConnector.retryWithHandle(handle -> { + manager.upsertCompactionState("ds1", fingerprint, state, initialTime); + return null; + }); + + // Verify it's marked as used with the initial timestamp + DateTime usedStatusBeforeUpdate = derbyConnector.retryWithHandle(handle -> + handle.createQuery( + "SELECT used_status_last_updated FROM " + tablesConfig.getCompactionStatesTable() + + " WHERE fingerprint = :fp" + ).bind("fp", fingerprint) + .map((i, r, ctx) -> DateTimes.of(r.getString("used_status_last_updated"))) + .first() + ); + assertEquals(initialTime, usedStatusBeforeUpdate); + + // Call upsert again with a different timestamp + // Since the fingerprint is already used, this should skip the UPDATE + DateTime laterTime = DateTimes.of("2024-01-02T00:00:00.000Z"); + derbyConnector.retryWithHandle(handle -> { + manager.upsertCompactionState("ds1", fingerprint, state, laterTime); + return null; + }); + + // Verify the used_status_last_updated timestamp DID NOT change + DateTime usedStatusAfterUpdate = derbyConnector.retryWithHandle(handle -> + handle.createQuery( + "SELECT used_status_last_updated FROM " + tablesConfig.getCompactionStatesTable() + + " WHERE fingerprint = :fp" + ).bind("fp", fingerprint) + .map((i, r, ctx) -> DateTimes.of(r.getString("used_status_last_updated"))) + .first() + ); + + assertEquals( + initialTime, + usedStatusAfterUpdate, + "used_status_last_updated should not change when upserting an already-used fingerprint" + ); + + // Verify still only 1 row + 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() { From f0632f966909d90275c945ceb5c3cca71c214267 Mon Sep 17 00:00:00 2001 From: capistrant Date: Mon, 12 Jan 2026 12:59:51 -0600 Subject: [PATCH 51/72] Refactor where in the code compaction fingerprints are generated --- .../NewestSegmentFirstPolicyBenchmark.java | 4 +- .../compact/CompactionSupervisorTest.java | 20 +++--- .../indexing/compact/CompactionJobQueue.java | 5 +- .../compact/OverlordCompactionScheduler.java | 9 ++- .../OverlordCompactionSchedulerTest.java | 3 +- .../metadata/CompactionStateFingerprints.java | 72 ------------------- .../metadata/CompactionStateStorage.java | 12 ---- .../DefaultCompactionFingerprintMapper.java | 45 +++++++++--- .../metadata/SqlCompactionStateStorage.java | 14 ---- .../SqlSegmentsMetadataQueryTest.java | 1 - .../segment/metadata/CompactionTestUtils.java | 51 +++++++++++++ .../HeapMemoryCompactionStateStorage.java | 51 +------------ .../SqlCompactionStateStorageTest.java | 40 +++++------ .../compaction/CompactionStatusTest.java | 20 ++++-- .../NewestSegmentFirstPolicyTest.java | 3 +- .../KillUnreferencedCompactionStateTest.java | 12 +--- 16 files changed, 148 insertions(+), 214 deletions(-) delete mode 100644 server/src/main/java/org/apache/druid/segment/metadata/CompactionStateFingerprints.java create mode 100644 server/src/test/java/org/apache/druid/segment/metadata/CompactionTestUtils.java diff --git a/benchmarks/src/test/java/org/apache/druid/server/coordinator/NewestSegmentFirstPolicyBenchmark.java b/benchmarks/src/test/java/org/apache/druid/server/coordinator/NewestSegmentFirstPolicyBenchmark.java index f6f0e366f9aa..686ebef64883 100644 --- a/benchmarks/src/test/java/org/apache/druid/server/coordinator/NewestSegmentFirstPolicyBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/server/coordinator/NewestSegmentFirstPolicyBenchmark.java @@ -21,9 +21,9 @@ import com.google.common.collect.ImmutableList; import org.apache.druid.client.DataSourcesSnapshot; +import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.segment.metadata.DefaultCompactionFingerprintMapper; -import org.apache.druid.segment.metadata.HeapMemoryCompactionStateStorage; import org.apache.druid.segment.metadata.NoopCompactionStateCache; import org.apache.druid.server.compaction.CompactionCandidateSearchPolicy; import org.apache.druid.server.compaction.CompactionSegmentIterator; @@ -139,7 +139,7 @@ public void measureNewestSegmentFirstPolicy(Blackhole blackhole) compactionConfigs, dataSources, Collections.emptyMap(), - new DefaultCompactionFingerprintMapper(new HeapMemoryCompactionStateStorage(), new NoopCompactionStateCache()) + new DefaultCompactionFingerprintMapper(new NoopCompactionStateCache(), new DefaultObjectMapper()) // TODO fix ); for (int i = 0; i < numCompactionTaskSlots && iterator.hasNext(); i++) { blackhole.consume(iterator.next()); diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionSupervisorTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionSupervisorTest.java index 4f018fbcffed..15e16746973b 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 @@ -31,7 +31,10 @@ import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.query.DruidMetrics; import org.apache.druid.rpc.UpdateResponse; -import org.apache.druid.segment.metadata.CompactionStateStorage; +import org.apache.druid.segment.metadata.CompactionFingerprintMapper; +import org.apache.druid.segment.metadata.CompactionStateCache; +import org.apache.druid.segment.metadata.CompactionTestUtils; +import org.apache.druid.segment.metadata.DefaultCompactionFingerprintMapper; import org.apache.druid.server.coordinator.ClusterCompactionConfig; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; import org.apache.druid.server.coordinator.InlineSchemaDataSourceCompactionConfig; @@ -286,13 +289,14 @@ private void verifySegmentsHaveNullLastCompactionStateAndNonNullFingerprint() private void verifyCompactedSegmentsHaveFingerprints(DataSourceCompactionConfig compactionConfig) { - CompactionStateStorage compactionStateStorage = overlord - .bindings() - .getInstance(CompactionStateStorage.class); - - String expectedFingerprint = compactionStateStorage.generateCompactionStateFingerprint( - compactionConfig.toCompactionState(), - dataSource + CompactionStateCache cache = overlord.bindings().getInstance(CompactionStateCache.class); + CompactionFingerprintMapper fingerprintMapper = new DefaultCompactionFingerprintMapper( + cache, + CompactionTestUtils.createDeterministicMapper() + ); + String expectedFingerprint = fingerprintMapper.generateFingerprint( + dataSource, + compactionConfig.toCompactionState() ); overlord 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 6f094d35fb51..a1417d066c83 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 @@ -112,7 +112,8 @@ public CompactionJobQueue( BrokerClient brokerClient, ObjectMapper objectMapper, CompactionStateStorage compactionStateStorage, - CompactionStateCache compactionStateCache + CompactionStateCache compactionStateCache, + ObjectMapper deterministicCompactionStateMapper ) { this.runStats = new CoordinatorRunStats(); @@ -129,7 +130,7 @@ public CompactionJobQueue( clusterCompactionConfig, dataSourcesSnapshot.getUsedSegmentsTimelinesPerDataSource()::get, snapshotBuilder, - new DefaultCompactionFingerprintMapper(compactionStateStorage, compactionStateCache) + new DefaultCompactionFingerprintMapper(compactionStateCache, deterministicCompactionStateMapper) ); this.compactionStateStorage = compactionStateStorage; 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 d7a352f8fe0d..a96143c756b0 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 @@ -26,6 +26,7 @@ import org.apache.druid.client.DataSourcesSnapshot; import org.apache.druid.client.broker.BrokerClient; import org.apache.druid.client.indexing.ClientCompactionRunnerInfo; +import org.apache.druid.guice.annotations.Deterministic; import org.apache.druid.indexer.TaskLocation; import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexing.common.actions.TaskActionClientFactory; @@ -143,6 +144,7 @@ public class OverlordCompactionScheduler implements CompactionScheduler private final CompactionStateStorage compactionStateStorage; private final CompactionStateCache compactionStateCache; + private final ObjectMapper deterministicMapper; @Inject public OverlordCompactionScheduler( @@ -161,7 +163,8 @@ public OverlordCompactionScheduler( ServiceEmitter emitter, ObjectMapper objectMapper, CompactionStateStorage compactionStateStorage, - CompactionStateCache compactionStateCache + CompactionStateCache compactionStateCache, + @Deterministic ObjectMapper deterministicMapper ) { final long segmentPollPeriodMillis = @@ -189,6 +192,7 @@ public OverlordCompactionScheduler( this.druidInputSourceFactory = druidInputSourceFactory; this.compactionStateStorage = compactionStateStorage; this.compactionStateCache = compactionStateCache; + this.deterministicMapper = deterministicMapper; this.taskRunnerListener = new TaskRunnerListener() { @Override @@ -377,7 +381,8 @@ private synchronized void resetCompactionJobQueue() brokerClient, objectMapper, compactionStateStorage, - compactionStateCache + compactionStateCache, + deterministicMapper ); 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 bda95b539f38..3e2b6c43eea8 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 @@ -235,7 +235,8 @@ private void initScheduler() serviceEmitter, OBJECT_MAPPER, new HeapMemoryCompactionStateStorage(), - new CompactionStateCache() + new CompactionStateCache(), + OBJECT_MAPPER // TODO fix ); } diff --git a/server/src/main/java/org/apache/druid/segment/metadata/CompactionStateFingerprints.java b/server/src/main/java/org/apache/druid/segment/metadata/CompactionStateFingerprints.java deleted file mode 100644 index 80de1a4331bf..000000000000 --- a/server/src/main/java/org/apache/druid/segment/metadata/CompactionStateFingerprints.java +++ /dev/null @@ -1,72 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.druid.segment.metadata; - -import com.fasterxml.jackson.core.JsonProcessingException; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.hash.Hasher; -import com.google.common.hash.Hashing; -import com.google.common.io.BaseEncoding; -import org.apache.druid.java.util.common.StringUtils; -import org.apache.druid.timeline.CompactionState; - -/** - * Utility class for generating deterministic fingerprints for compaction states. - */ -public final class CompactionStateFingerprints -{ - private CompactionStateFingerprints() - { - // Prevent instantiation - } - - /** - * Generates a deterministic fingerprint for the given compaction state and datasource. - *

    - * The fingerprint is a SHA-256 hash of the datasource name and serialized compaction state. - * - * @param compactionState The compaction configuration to fingerprint - * @param dataSource The datasource name - * @param deterministicMapper ObjectMapper configured for deterministic serialization - * @return A hex-encoded SHA-256 fingerprint string - */ - @SuppressWarnings("UnstableApiUsage") - public static String generate( - final CompactionState compactionState, - final String dataSource, - final ObjectMapper deterministicMapper - ) - { - final Hasher hasher = Hashing.sha256().newHasher(); - - hasher.putBytes(StringUtils.toUtf8(dataSource)); - hasher.putByte((byte) 0xff); - - try { - hasher.putBytes(deterministicMapper.writeValueAsBytes(compactionState)); - } - catch (JsonProcessingException e) { - throw new RuntimeException("Failed to serialize CompactionState for fingerprinting", e); - } - hasher.putByte((byte) 0xff); - - return BaseEncoding.base16().encode(hasher.hash().asBytes()); - } -} diff --git a/server/src/main/java/org/apache/druid/segment/metadata/CompactionStateStorage.java b/server/src/main/java/org/apache/druid/segment/metadata/CompactionStateStorage.java index 78185f3912a0..fbe8c42577c2 100644 --- a/server/src/main/java/org/apache/druid/segment/metadata/CompactionStateStorage.java +++ b/server/src/main/java/org/apache/druid/segment/metadata/CompactionStateStorage.java @@ -32,18 +32,6 @@ */ public interface CompactionStateStorage { - /** - * Generates a deterministic fingerprint for the given compaction state and datasource. - *

    - * The fingerprint is a SHA-256 hash of the datasource name and serialized compaction state that is globally unique in - * the segment space. - * - * @param compactionState The compaction configuration to fingerprint - * @param dataSource The datasource name - * @return A hex-encoded SHA-256 fingerprint string that is globally unique - */ - String generateCompactionStateFingerprint(CompactionState compactionState, String dataSource); - /** * Upserts a compaction state to storage. *

    diff --git a/server/src/main/java/org/apache/druid/segment/metadata/DefaultCompactionFingerprintMapper.java b/server/src/main/java/org/apache/druid/segment/metadata/DefaultCompactionFingerprintMapper.java index 03066c159e88..96c7a9871437 100644 --- a/server/src/main/java/org/apache/druid/segment/metadata/DefaultCompactionFingerprintMapper.java +++ b/server/src/main/java/org/apache/druid/segment/metadata/DefaultCompactionFingerprintMapper.java @@ -19,9 +19,15 @@ package org.apache.druid.segment.metadata; +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.hash.Hasher; +import com.google.common.hash.Hashing; +import com.google.common.io.BaseEncoding; +import org.apache.druid.guice.annotations.Deterministic; +import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.timeline.CompactionState; -import javax.annotation.Nullable; import java.util.Optional; /** @@ -31,30 +37,51 @@ */ public class DefaultCompactionFingerprintMapper implements CompactionFingerprintMapper { - private final CompactionStateStorage compactionStateStorage; private final CompactionStateCache compactionStateCache; + private final ObjectMapper deterministicMapper; public DefaultCompactionFingerprintMapper( - CompactionStateStorage compactionStateStorage, - @Nullable CompactionStateCache compactionStateCache + CompactionStateCache compactionStateCache, + @Deterministic ObjectMapper deterministicMapper ) { - this.compactionStateStorage = compactionStateStorage; this.compactionStateCache = compactionStateCache; + this.deterministicMapper = deterministicMapper; } + /** + * Generates a deterministic fingerprint for the given compaction state and datasource. + *

    + * The fingerprint is a SHA-256 hash of the datasource name and serialized compaction state that is globally unique in + * the segment space. + * + * @param compactionState The compaction configuration to fingerprint + * @param dataSource The datasource name + * @return A hex-encoded SHA-256 fingerprint string + */ + @SuppressWarnings("UnstableApiUsage") @Override public String generateFingerprint(String dataSource, CompactionState compactionState) { - return compactionStateStorage.generateCompactionStateFingerprint(compactionState, dataSource); + final Hasher hasher = Hashing.sha256().newHasher(); + + hasher.putBytes(StringUtils.toUtf8(dataSource)); + hasher.putByte((byte) 0xff); + + try { + hasher.putBytes(deterministicMapper.writeValueAsBytes(compactionState)); + } + catch (JsonProcessingException e) { + throw new RuntimeException("Failed to serialize CompactionState for fingerprinting", e); + } + hasher.putByte((byte) 0xff); + + return BaseEncoding.base16().encode(hasher.hash().asBytes()); } @Override public Optional getStateForFingerprint(String fingerprint) { - if (compactionStateCache == null) { - return Optional.empty(); - } return compactionStateCache.getCompactionStateByFingerprint(fingerprint); } } diff --git a/server/src/main/java/org/apache/druid/segment/metadata/SqlCompactionStateStorage.java b/server/src/main/java/org/apache/druid/segment/metadata/SqlCompactionStateStorage.java index 6aadc1882625..250cd217c9e0 100644 --- a/server/src/main/java/org/apache/druid/segment/metadata/SqlCompactionStateStorage.java +++ b/server/src/main/java/org/apache/druid/segment/metadata/SqlCompactionStateStorage.java @@ -24,7 +24,6 @@ import com.google.inject.Inject; import org.apache.druid.error.InternalServerError; import org.apache.druid.guice.LazySingleton; -import org.apache.druid.guice.annotations.Deterministic; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.emitter.EmittingLogger; @@ -59,20 +58,17 @@ public class SqlCompactionStateStorage implements CompactionStateStorage private final MetadataStorageTablesConfig dbTables; private final ObjectMapper jsonMapper; - private final ObjectMapper deterministicMapper; private final SQLMetadataConnector connector; @Inject public SqlCompactionStateStorage( @Nonnull MetadataStorageTablesConfig dbTables, @Nonnull ObjectMapper jsonMapper, - @Deterministic @Nonnull ObjectMapper deterministicMapper, @Nonnull SQLMetadataConnector connector ) { this.dbTables = dbTables; this.jsonMapper = jsonMapper; - this.deterministicMapper = deterministicMapper; this.connector = connector; } @@ -252,16 +248,6 @@ public int deleteUnusedCompactionStatesOlderThan(long timestamp) .execute()); } - @Override - public String generateCompactionStateFingerprint( - final CompactionState compactionState, - final String dataSource - ) - { - return CompactionStateFingerprints.generate(compactionState, dataSource, deterministicMapper); - } - - /** * Represents the state of an indexing state fingerprint in the database. *

    diff --git a/server/src/test/java/org/apache/druid/metadata/SqlSegmentsMetadataQueryTest.java b/server/src/test/java/org/apache/druid/metadata/SqlSegmentsMetadataQueryTest.java index a143e6c4b9c4..f865bc337e8d 100644 --- a/server/src/test/java/org/apache/druid/metadata/SqlSegmentsMetadataQueryTest.java +++ b/server/src/test/java/org/apache/druid/metadata/SqlSegmentsMetadataQueryTest.java @@ -670,7 +670,6 @@ private void insertCompactionStates(Map compactionState SqlCompactionStateStorage manager = new SqlCompactionStateStorage( tablesConfig, mapper, - mapper, derbyConnectorRule.getConnector() ); diff --git a/server/src/test/java/org/apache/druid/segment/metadata/CompactionTestUtils.java b/server/src/test/java/org/apache/druid/segment/metadata/CompactionTestUtils.java new file mode 100644 index 000000000000..c20a9015c05a --- /dev/null +++ b/server/src/test/java/org/apache/druid/segment/metadata/CompactionTestUtils.java @@ -0,0 +1,51 @@ +/* + * 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.MapperFeature; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.SerializationFeature; +import org.apache.druid.jackson.DefaultObjectMapper; + +/** + * Test utilities for compaction-related tests. + */ +public class CompactionTestUtils +{ + /** + * Creates a deterministic ObjectMapper for fingerprinting tests. + * This mapper is configured to serialize with sorted map keys and alphabetically ordered properties, + * ensuring consistent fingerprints across test runs. + * + * @return A deterministic ObjectMapper instance + */ + public static ObjectMapper createDeterministicMapper() + { + ObjectMapper mapper = new DefaultObjectMapper(); + mapper.configure(SerializationFeature.ORDER_MAP_ENTRIES_BY_KEYS, true); + mapper.configure(MapperFeature.SORT_PROPERTIES_ALPHABETICALLY, true); + return mapper; + } + + private CompactionTestUtils() + { + // Utility class + } +} diff --git a/server/src/test/java/org/apache/druid/segment/metadata/HeapMemoryCompactionStateStorage.java b/server/src/test/java/org/apache/druid/segment/metadata/HeapMemoryCompactionStateStorage.java index beb95c2f9ebc..de4686114fa4 100644 --- a/server/src/test/java/org/apache/druid/segment/metadata/HeapMemoryCompactionStateStorage.java +++ b/server/src/test/java/org/apache/druid/segment/metadata/HeapMemoryCompactionStateStorage.java @@ -19,14 +19,9 @@ package org.apache.druid.segment.metadata; -import com.fasterxml.jackson.databind.MapperFeature; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.fasterxml.jackson.databind.SerializationFeature; -import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.timeline.CompactionState; import org.joda.time.DateTime; -import javax.annotation.Nullable; import java.util.List; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; @@ -41,8 +36,7 @@ */ public class HeapMemoryCompactionStateStorage implements CompactionStateStorage { - private final ConcurrentMap fingerprintToStateMap = new ConcurrentHashMap<>(); - private final ObjectMapper deterministicMapper; + private final ConcurrentMap fingerprintToStateMap; /** * Creates an in-memory compaction state manager with a default deterministic mapper. @@ -50,39 +44,7 @@ public class HeapMemoryCompactionStateStorage implements CompactionStateStorage */ public HeapMemoryCompactionStateStorage() { - this(createDeterministicMapper()); - } - - /** - * Creates an in-memory compaction state manager with the provided deterministic mapper - * for fingerprint generation. - * - * @param deterministicMapper ObjectMapper configured for deterministic serialization - */ - public HeapMemoryCompactionStateStorage(ObjectMapper deterministicMapper) - { - this.deterministicMapper = deterministicMapper; - } - - /** - * Creates an ObjectMapper configured for deterministic serialization. - * Used for generating consistent fingerprints. - */ - private static ObjectMapper createDeterministicMapper() - { - ObjectMapper mapper = new DefaultObjectMapper(); - mapper.configure(SerializationFeature.ORDER_MAP_ENTRIES_BY_KEYS, true); - mapper.configure(MapperFeature.SORT_PROPERTIES_ALPHABETICALLY, true); - return mapper; - } - - @Override - public String generateCompactionStateFingerprint( - final CompactionState compactionState, - final String dataSource - ) - { - return CompactionStateFingerprints.generate(compactionState, dataSource, deterministicMapper); + this.fingerprintToStateMap = new ConcurrentHashMap<>(); } @Override @@ -121,15 +83,6 @@ public int deleteUnusedCompactionStatesOlderThan(long timestamp) return 0; } - /** - * Gets a compaction state by fingerprint. For test verification only. - */ - @Nullable - public CompactionState getCompactionStateByFingerprint(String fingerprint) - { - return fingerprintToStateMap.get(fingerprint); - } - /** * Gets all stored compaction states. For test verification only. */ diff --git a/server/src/test/java/org/apache/druid/segment/metadata/SqlCompactionStateStorageTest.java b/server/src/test/java/org/apache/druid/segment/metadata/SqlCompactionStateStorageTest.java index 4d6a0debac82..89109450e6f8 100644 --- a/server/src/test/java/org/apache/druid/segment/metadata/SqlCompactionStateStorageTest.java +++ b/server/src/test/java/org/apache/druid/segment/metadata/SqlCompactionStateStorageTest.java @@ -19,9 +19,7 @@ package org.apache.druid.segment.metadata; -import com.fasterxml.jackson.databind.MapperFeature; import com.fasterxml.jackson.databind.ObjectMapper; -import com.fasterxml.jackson.databind.SerializationFeature; import com.google.common.collect.ImmutableList; import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; @@ -54,14 +52,16 @@ public class SqlCompactionStateStorageTest @RegisterExtension public static final TestDerbyConnector.DerbyConnectorRule5 DERBY_CONNECTOR_RULE = new TestDerbyConnector.DerbyConnectorRule5(); + private static final ObjectMapper DETERMINISTIC_MAPPER = CompactionTestUtils.createDeterministicMapper(); private final ObjectMapper jsonMapper = new DefaultObjectMapper(); - private final ObjectMapper deterministicMapper = createDeterministicMapper(); private static TestDerbyConnector derbyConnector; private static MetadataStorageTablesConfig tablesConfig; private SqlCompactionStateStorage manager; + private static DefaultCompactionFingerprintMapper fingerprintMapper; + @BeforeAll public static void setUpClass() { @@ -69,6 +69,10 @@ public static void setUpClass() tablesConfig = DERBY_CONNECTOR_RULE.metadataTablesConfigSupplier().get(); derbyConnector.createCompactionStatesTable(); derbyConnector.createSegmentTable(); + fingerprintMapper = new DefaultCompactionFingerprintMapper( + new NoopCompactionStateCache(), + DETERMINISTIC_MAPPER + ); } @BeforeEach @@ -80,7 +84,7 @@ public void setUp() return null; }); - manager = new SqlCompactionStateStorage(tablesConfig, jsonMapper, deterministicMapper, derbyConnector); + manager = new SqlCompactionStateStorage(tablesConfig, jsonMapper, derbyConnector); } @Test @@ -417,8 +421,8 @@ public void test_generateCompactionStateFingerprint_deterministicFingerprinting( CompactionState compactionState1 = createBasicCompactionState(); CompactionState compactionState2 = createBasicCompactionState(); - String fingerprint1 = manager.generateCompactionStateFingerprint(compactionState1, "test-ds"); - String fingerprint2 = manager.generateCompactionStateFingerprint(compactionState2, "test-ds"); + String fingerprint1 = fingerprintMapper.generateFingerprint("test-ds", compactionState1); + String fingerprint2 = fingerprintMapper.generateFingerprint("test-ds", compactionState2); assertEquals( fingerprint1, @@ -432,8 +436,8 @@ public void test_generateCompactionStateFingerprint_differentDatasourcesWithSame { CompactionState compactionState = createBasicCompactionState(); - String fingerprint1 = manager.generateCompactionStateFingerprint(compactionState, "ds1"); - String fingerprint2 = manager.generateCompactionStateFingerprint(compactionState, "ds2"); + String fingerprint1 = fingerprintMapper.generateFingerprint("ds1", compactionState); + String fingerprint2 = fingerprintMapper.generateFingerprint("ds2", compactionState); assertNotEquals( fingerprint1, @@ -475,8 +479,8 @@ public void test_generateCompactionStateFingerprint_metricsListOrderDifferenceRe null ); - String fingerprint1 = manager.generateCompactionStateFingerprint(state1, "test-ds"); - String fingerprint2 = manager.generateCompactionStateFingerprint(state2, "test-ds"); + String fingerprint1 = fingerprintMapper.generateFingerprint("test-ds", state1); + String fingerprint2 = fingerprintMapper.generateFingerprint("test-ds", state2); assertNotEquals( fingerprint1, @@ -516,8 +520,8 @@ public void test_generateCompactionStateFingerprint_dimensionsListOrderDifferenc null ); - String fingerprint1 = manager.generateCompactionStateFingerprint(state1, "test-ds"); - String fingerprint2 = manager.generateCompactionStateFingerprint(state2, "test-ds"); + String fingerprint1 = fingerprintMapper.generateFingerprint("test-ds", state1); + String fingerprint2 = fingerprintMapper.generateFingerprint("test-ds", state2); assertNotEquals( fingerprint1, @@ -549,8 +553,8 @@ public void testGenerateCompactionStateFingerprint_differentPartitionsSpec() null ); - String fingerprint1 = manager.generateCompactionStateFingerprint(state1, "test-ds"); - String fingerprint2 = manager.generateCompactionStateFingerprint(state2, "test-ds"); + String fingerprint1 = fingerprintMapper.generateFingerprint("test-ds", state1); + String fingerprint2 = fingerprintMapper.generateFingerprint("test-ds", state2); assertNotEquals( fingerprint1, @@ -559,14 +563,6 @@ public void testGenerateCompactionStateFingerprint_differentPartitionsSpec() ); } - private static ObjectMapper createDeterministicMapper() - { - ObjectMapper mapper = new DefaultObjectMapper(); - mapper.configure(SerializationFeature.ORDER_MAP_ENTRIES_BY_KEYS, true); - mapper.configure(MapperFeature.SORT_PROPERTIES_ALPHABETICALLY, true); - return mapper; - } - private CompactionState createBasicCompactionState() { return new CompactionState( 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 b6f645e12a9d..d78646e89809 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 @@ -30,6 +30,7 @@ import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; import org.apache.druid.indexer.partitions.HashedPartitionsSpec; import org.apache.druid.indexer.partitions.PartitionsSpec; +import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.granularity.Granularities; @@ -42,6 +43,7 @@ import org.apache.druid.segment.metadata.CompactionFingerprintMapper; import org.apache.druid.segment.metadata.CompactionStateCache; import org.apache.druid.segment.metadata.CompactionStateStorage; +import org.apache.druid.segment.metadata.CompactionTestUtils; import org.apache.druid.segment.metadata.DefaultCompactionFingerprintMapper; import org.apache.druid.segment.metadata.HeapMemoryCompactionStateStorage; import org.apache.druid.segment.nested.NestedCommonFormatColumnFormatSpec; @@ -81,8 +83,8 @@ public void setUp() compactionStateStorage = new HeapMemoryCompactionStateStorage(); compactionStateCache = new CompactionStateCache(); fingerprintMapper = new DefaultCompactionFingerprintMapper( - compactionStateStorage, - compactionStateCache + compactionStateCache, + CompactionTestUtils.createDeterministicMapper() ); } @@ -633,13 +635,14 @@ public void test_evaluate_needsCompactionWhenSomeSegmentsHaveUnexpectedCompactio CompactionState expectedState = compactionConfig.toCompactionState(); - String expectedFingerprint = compactionStateStorage.generateCompactionStateFingerprint(expectedState, TestDataSource.WIKI); + String expectedFingerprint = fingerprintMapper.generateFingerprint(TestDataSource.WIKI, expectedState); List segments = List.of( DataSegment.builder(WIKI_SEGMENT).compactionStateFingerprint(expectedFingerprint).build(), DataSegment.builder(WIKI_SEGMENT_2).compactionStateFingerprint("wrongFingerprint").build() ); + compactionStateStorage.upsertCompactionState(TestDataSource.WIKI, expectedFingerprint, expectedState, DateTimes.nowUtc()); compactionStateStorage.upsertCompactionState(TestDataSource.WIKI, "wrongFingerprint", wrongState, DateTimes.nowUtc()); syncCacheFromManager(); @@ -708,7 +711,7 @@ public void test_evaluate_noCompactionWhenAllSegmentsHaveExpectedCompactionState CompactionState expectedState = compactionConfig.toCompactionState(); - String expectedFingerprint = compactionStateStorage.generateCompactionStateFingerprint(expectedState, TestDataSource.WIKI); + String expectedFingerprint = fingerprintMapper.generateFingerprint(TestDataSource.WIKI, expectedState); List segments = List.of( DataSegment.builder(WIKI_SEGMENT).compactionStateFingerprint(expectedFingerprint).build(), @@ -733,14 +736,17 @@ public void test_evaluate_needsCompactionWhenNonFingerprintedSegmentsFailChecksO .build(); CompactionState expectedState = compactionConfig.toCompactionState(); + String expectedFingerprint = fingerprintMapper.generateFingerprint(TestDataSource.WIKI, expectedState); - String expectedFingerprint = compactionStateStorage.generateCompactionStateFingerprint(expectedState, TestDataSource.WIKI); + compactionStateStorage.upsertCompactionState(TestDataSource.WIKI, expectedFingerprint, expectedState, DateTimes.nowUtc()); + syncCacheFromManager(); 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, @@ -761,7 +767,7 @@ public void test_evaluate_noCompactionWhenNonFingerprintedSegmentsPassChecksOnLa CompactionState expectedState = compactionConfig.toCompactionState(); - String expectedFingerprint = compactionStateStorage.generateCompactionStateFingerprint(expectedState, TestDataSource.WIKI); + String expectedFingerprint = fingerprintMapper.generateFingerprint(TestDataSource.WIKI, expectedState); List segments = List.of( DataSegment.builder(WIKI_SEGMENT).compactionStateFingerprint(expectedFingerprint).build(), @@ -826,7 +832,7 @@ private void verifyEvaluationNeedsCompactionBecauseWithCustomSegments( final CompactionStatus status = CompactionStatus.compute( candidate, compactionConfig, - new DefaultCompactionFingerprintMapper(compactionStateStorage, compactionStateCache) + new DefaultCompactionFingerprintMapper(compactionStateCache, new DefaultObjectMapper()) // TODO fix ); Assert.assertFalse(status.isComplete()); 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 087d9aea9031..7669aae3224c 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 @@ -49,7 +49,6 @@ import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.segment.metadata.CompactionFingerprintMapper; import org.apache.druid.segment.metadata.DefaultCompactionFingerprintMapper; -import org.apache.druid.segment.metadata.HeapMemoryCompactionStateStorage; import org.apache.druid.segment.metadata.NoopCompactionStateCache; import org.apache.druid.segment.transform.CompactionTransformSpec; import org.apache.druid.server.coordinator.CreateDataSegments; @@ -85,7 +84,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 CompactionFingerprintMapper fingerprintMapper = new DefaultCompactionFingerprintMapper(new HeapMemoryCompactionStateStorage(), new NoopCompactionStateCache()); + private final CompactionFingerprintMapper fingerprintMapper = new DefaultCompactionFingerprintMapper(new NoopCompactionStateCache(), mapper); // TODO fix @Test public void testLargeOffsetAndSmallSegmentInterval() 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 index 3be3fd216eb5..c5341a73ac32 100644 --- 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 @@ -19,9 +19,7 @@ package org.apache.druid.server.coordinator.duty; -import com.fasterxml.jackson.databind.MapperFeature; import com.fasterxml.jackson.databind.ObjectMapper; -import com.fasterxml.jackson.databind.SerializationFeature; import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.DateTimes; @@ -70,7 +68,7 @@ public void setUp() derbyConnector.createCompactionStatesTable(); derbyConnector.createSegmentTable(); - compactionStateStorage = new SqlCompactionStateStorage(tablesConfig, jsonMapper, createDeterministicMapper(), derbyConnector); + compactionStateStorage = new SqlCompactionStateStorage(tablesConfig, jsonMapper, derbyConnector); mockParams = EasyMock.createMock(DruidCoordinatorRuntimeParams.class); CoordinatorRunStats runStats = new CoordinatorRunStats(); @@ -253,12 +251,4 @@ private Boolean getCompactionStateUsedStatus(String fingerprint) return usedStatus.isEmpty() ? null : usedStatus.get(0); } - - private static ObjectMapper createDeterministicMapper() - { - ObjectMapper mapper = new DefaultObjectMapper(); - mapper.configure(SerializationFeature.ORDER_MAP_ENTRIES_BY_KEYS, true); - mapper.configure(MapperFeature.SORT_PROPERTIES_ALPHABETICALLY, true); - return mapper; - } } From b081727db195f0dafa3ff3029a54f54121cc1aa1 Mon Sep 17 00:00:00 2001 From: capistrant Date: Mon, 12 Jan 2026 13:53:29 -0600 Subject: [PATCH 52/72] Formalize unique constraint exception check in sqlmetadataconnector and db specific impls --- .../storage/sqlserver/SQLServerConnector.java | 19 +++++++ .../sqlserver/SQLServerConnectorTest.java | 30 ++++++++++++ .../storage/mysql/MySQLConnector.java | 19 +++++++ .../storage/mysql/MySQLConnectorTest.java | 30 ++++++++++++ .../postgresql/PostgreSQLConnector.java | 19 +++++++ .../postgresql/PostgreSQLConnectorTest.java | 30 ++++++++++++ .../druid/metadata/SQLMetadataConnector.java | 11 +++++ .../storage/derby/DerbyConnector.java | 19 +++++++ .../metadata/SqlCompactionStateStorage.java | 49 +++---------------- 9 files changed, 183 insertions(+), 43 deletions(-) diff --git a/extensions-contrib/sqlserver-metadata-storage/src/main/java/org/apache/druid/metadata/storage/sqlserver/SQLServerConnector.java b/extensions-contrib/sqlserver-metadata-storage/src/main/java/org/apache/druid/metadata/storage/sqlserver/SQLServerConnector.java index 7b3c94f49926..be7147cc5d23 100644 --- a/extensions-contrib/sqlserver-metadata-storage/src/main/java/org/apache/druid/metadata/storage/sqlserver/SQLServerConnector.java +++ b/extensions-contrib/sqlserver-metadata-storage/src/main/java/org/apache/druid/metadata/storage/sqlserver/SQLServerConnector.java @@ -294,4 +294,23 @@ protected boolean connectorIsTransientException(Throwable e) } return false; } + + @Override + public boolean isUniqueConstraintViolation(Throwable t) + { + Throwable cause = t; + while (cause != null) { + if (cause instanceof SQLException) { + SQLException sqlException = (SQLException) cause; + String sqlState = sqlException.getSQLState(); + + // SQL standard unique constraint violation code is 23000 for Sql Server + if ("23000".equals(sqlState)) { + return true; + } + } + cause = cause.getCause(); + } + return false; + } } diff --git a/extensions-contrib/sqlserver-metadata-storage/src/test/java/org/apache/druid/metadata/storage/sqlserver/SQLServerConnectorTest.java b/extensions-contrib/sqlserver-metadata-storage/src/test/java/org/apache/druid/metadata/storage/sqlserver/SQLServerConnectorTest.java index 1c44edd18fe5..9903871b573e 100644 --- a/extensions-contrib/sqlserver-metadata-storage/src/test/java/org/apache/druid/metadata/storage/sqlserver/SQLServerConnectorTest.java +++ b/extensions-contrib/sqlserver-metadata-storage/src/test/java/org/apache/druid/metadata/storage/sqlserver/SQLServerConnectorTest.java @@ -54,6 +54,36 @@ public void testIsTransientException() Assert.assertFalse(connector.isTransientException(new Throwable("Throwable with reason only"))); } + @Test + public void testIsUniqueConstraintViolation() + { + SQLServerConnector connector = new SQLServerConnector( + Suppliers.ofInstance(new MetadataStorageConnectorConfig()), + Suppliers.ofInstance( + MetadataStorageTablesConfig.fromBase(null) + ), + CentralizedDatasourceSchemaConfig.create() + ); + + // SQL Server integrity_constraint_violation SQL state (23000) + Assert.assertTrue(connector.isUniqueConstraintViolation( + new SQLException("Violation of UNIQUE KEY constraint", "23000") + )); + + // Different SQL state should return false + Assert.assertFalse(connector.isUniqueConstraintViolation( + new SQLException("some other error", "42000") + )); + + // SQLException wrapped in another exception (tests cause chain traversal) + Assert.assertTrue(connector.isUniqueConstraintViolation( + new RuntimeException(new SQLException("Duplicate key", "23000")) + )); + + // Non-SQLException exception + Assert.assertFalse(connector.isUniqueConstraintViolation(new Exception("not a SQLException"))); + } + @Test public void testLimitClause() { diff --git a/extensions-core/mysql-metadata-storage/src/main/java/org/apache/druid/metadata/storage/mysql/MySQLConnector.java b/extensions-core/mysql-metadata-storage/src/main/java/org/apache/druid/metadata/storage/mysql/MySQLConnector.java index b6a05b300f03..8c4f928be76b 100644 --- a/extensions-core/mysql-metadata-storage/src/main/java/org/apache/druid/metadata/storage/mysql/MySQLConnector.java +++ b/extensions-core/mysql-metadata-storage/src/main/java/org/apache/druid/metadata/storage/mysql/MySQLConnector.java @@ -271,6 +271,25 @@ public DBI getDBI() return dbi; } + @Override + public boolean isUniqueConstraintViolation(Throwable t) + { + Throwable cause = t; + while (cause != null) { + if (cause instanceof SQLException) { + SQLException sqlException = (SQLException) cause; + String sqlState = sqlException.getSQLState(); + + // SQL standard unique constraint violation code is 23000 for MySQL + if ("23000".equals(sqlState)) { + return true; + } + } + cause = cause.getCause(); + } + return false; + } + @Nullable private Class tryLoadDriverClass(String className, boolean failIfNotFound) { diff --git a/extensions-core/mysql-metadata-storage/src/test/java/org/apache/druid/metadata/storage/mysql/MySQLConnectorTest.java b/extensions-core/mysql-metadata-storage/src/test/java/org/apache/druid/metadata/storage/mysql/MySQLConnectorTest.java index da628020d0f5..4f5984303802 100644 --- a/extensions-core/mysql-metadata-storage/src/test/java/org/apache/druid/metadata/storage/mysql/MySQLConnectorTest.java +++ b/extensions-core/mysql-metadata-storage/src/test/java/org/apache/druid/metadata/storage/mysql/MySQLConnectorTest.java @@ -137,6 +137,36 @@ public void testIsRootCausePacketTooBigException() ); } + @Test + public void testIsUniqueConstraintViolation() + { + MySQLConnector connector = new MySQLConnector( + CONNECTOR_CONFIG_SUPPLIER, + TABLES_CONFIG_SUPPLIER, + new MySQLConnectorSslConfig(), + MYSQL_DRIVER_CONFIG, + centralizedDatasourceSchemaConfig + ); + + // MySQL integrity_constraint_violation SQL state (23000) + Assert.assertTrue(connector.isUniqueConstraintViolation( + new SQLException("Duplicate entry 'value' for key 'PRIMARY'", "23000") + )); + + // Different SQL state should return false + Assert.assertFalse(connector.isUniqueConstraintViolation( + new SQLException("some other error", "42S02") + )); + + // SQLException wrapped in another exception (tests cause chain traversal) + Assert.assertTrue(connector.isUniqueConstraintViolation( + new RuntimeException(new SQLException("Duplicate entry", "23000")) + )); + + // Non-SQLException exception + Assert.assertFalse(connector.isUniqueConstraintViolation(new Exception("not a SQLException"))); + } + @Test public void testLimitClause() { diff --git a/extensions-core/postgresql-metadata-storage/src/main/java/org/apache/druid/metadata/storage/postgresql/PostgreSQLConnector.java b/extensions-core/postgresql-metadata-storage/src/main/java/org/apache/druid/metadata/storage/postgresql/PostgreSQLConnector.java index 765e12aa39a8..48e0bd4e8c39 100644 --- a/extensions-core/postgresql-metadata-storage/src/main/java/org/apache/druid/metadata/storage/postgresql/PostgreSQLConnector.java +++ b/extensions-core/postgresql-metadata-storage/src/main/java/org/apache/druid/metadata/storage/postgresql/PostgreSQLConnector.java @@ -308,4 +308,23 @@ public Set getIndexOnTable(String tableName) { return super.getIndexOnTable(StringUtils.toLowerCase(tableName)); } + + @Override + public boolean isUniqueConstraintViolation(Throwable t) + { + Throwable cause = t; + while (cause != null) { + if (cause instanceof SQLException) { + SQLException sqlException = (SQLException) cause; + String sqlState = sqlException.getSQLState(); + + // SQL standard unique constraint violation code is 23505 for PostgreSQL + if ("23505".equals(sqlState)) { + return true; + } + } + cause = cause.getCause(); + } + return false; + } } diff --git a/extensions-core/postgresql-metadata-storage/src/test/java/org/apache/druid/metadata/storage/postgresql/PostgreSQLConnectorTest.java b/extensions-core/postgresql-metadata-storage/src/test/java/org/apache/druid/metadata/storage/postgresql/PostgreSQLConnectorTest.java index bffe53b693b6..3fdf1c5e0693 100644 --- a/extensions-core/postgresql-metadata-storage/src/test/java/org/apache/druid/metadata/storage/postgresql/PostgreSQLConnectorTest.java +++ b/extensions-core/postgresql-metadata-storage/src/test/java/org/apache/druid/metadata/storage/postgresql/PostgreSQLConnectorTest.java @@ -69,6 +69,36 @@ public void testIsTransientException() Assert.assertFalse(connector.isTransientException(new Throwable("I give up"))); } + @Test + public void testIsUniqueConstraintViolation() + { + PostgreSQLConnector connector = new PostgreSQLConnector( + Suppliers.ofInstance(new MetadataStorageConnectorConfig()), + Suppliers.ofInstance(MetadataStorageTablesConfig.fromBase(null)), + new PostgreSQLConnectorConfig(), + new PostgreSQLTablesConfig(), + centralizedDatasourceSchemaConfig + ); + + // PostgreSQL unique_violation SQL state (23505) + Assert.assertTrue(connector.isUniqueConstraintViolation( + new SQLException("duplicate key value violates unique constraint", "23505") + )); + + // Different SQL state should return false + Assert.assertFalse(connector.isUniqueConstraintViolation( + new SQLException("some other error", "42P01") + )); + + // SQLException wrapped in another exception (tests cause chain traversal) + Assert.assertTrue(connector.isUniqueConstraintViolation( + new RuntimeException(new SQLException("duplicate key", "23505")) + )); + + // Non-SQLException exception + Assert.assertFalse(connector.isUniqueConstraintViolation(new Exception("not a SQLException"))); + } + @Test public void testLimitClause() { 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 26fbd1443db2..95e0b5ced84a 100644 --- a/server/src/main/java/org/apache/druid/metadata/SQLMetadataConnector.java +++ b/server/src/main/java/org/apache/druid/metadata/SQLMetadataConnector.java @@ -229,6 +229,17 @@ protected boolean isRootCausePacketTooBigException(Throwable t) return false; } + /** + * Checks if the root cause of the given exception is a unique constraint violation. + * + * @return false by default. Specific implementations should override this method + * to correctly classify their unique constraint violation exceptions. + */ + public boolean isUniqueConstraintViolation(Throwable t) + { + return false; + } + /** * Creates the given table and indexes if the table doesn't already exist. */ diff --git a/server/src/main/java/org/apache/druid/metadata/storage/derby/DerbyConnector.java b/server/src/main/java/org/apache/druid/metadata/storage/derby/DerbyConnector.java index 97081d47ba3e..cb4815d2313e 100644 --- a/server/src/main/java/org/apache/druid/metadata/storage/derby/DerbyConnector.java +++ b/server/src/main/java/org/apache/druid/metadata/storage/derby/DerbyConnector.java @@ -212,6 +212,25 @@ public Boolean withHandle(Handle handle) ); } + @Override + public boolean isUniqueConstraintViolation(Throwable t) + { + Throwable cause = t; + while (cause != null) { + if (cause instanceof SQLException) { + SQLException sqlException = (SQLException) cause; + String sqlState = sqlException.getSQLState(); + + // SQL standard unique constraint violation code is 23505 for Derby + if ("23505".equals(sqlState)) { + return true; + } + } + cause = cause.getCause(); + } + return false; + } + @LifecycleStart public void start() { diff --git a/server/src/main/java/org/apache/druid/segment/metadata/SqlCompactionStateStorage.java b/server/src/main/java/org/apache/druid/segment/metadata/SqlCompactionStateStorage.java index 250cd217c9e0..3dbb630bccf3 100644 --- a/server/src/main/java/org/apache/druid/segment/metadata/SqlCompactionStateStorage.java +++ b/server/src/main/java/org/apache/druid/segment/metadata/SqlCompactionStateStorage.java @@ -36,7 +36,6 @@ import org.skife.jdbi.v2.Update; import javax.annotation.Nonnull; -import java.sql.SQLException; import java.util.List; /** @@ -162,18 +161,18 @@ public void upsertCompactionState( return null; }); } - catch (Exception e) { - if (isUniqueConstraintViolation(e)) { + catch (Throwable e) { + if (connector.isUniqueConstraintViolation(e)) { + // Swallow exception - another thread already persisted the same data log.info( "Fingerprints already exist for datasource[%s] (likely concurrent insert). " + "Treating as success since operation is idempotent.", dataSource ); - // Swallow exception - another thread already persisted the same data - return; + } else { + // For other exceptions, let them propagate + throw e; } - // For other exceptions, let them propagate - throw e; } } @@ -333,40 +332,4 @@ private static void bindValuesToInClause( query.bind(parameterPrefix + i, values.get(i)); } } - - /** - * Checks if an exception is a unique constraint violation. - * This is expected when multiple threads try to insert the same fingerprint concurrently. - * Since operations are idempotent, these violations can be safely ignored. - */ - private boolean isUniqueConstraintViolation(Exception e) - { - // Look for SQLException in the cause chain - Throwable cause = e; - while (cause != null) { - if (cause instanceof SQLException) { - SQLException sqlException = (SQLException) cause; - String sqlState = sqlException.getSQLState(); - - // SQL standard unique constraint violation codes - // 23505 = unique_violation (PostgreSQL, Derby) - // 23000 = integrity_constraint_violation (MySQL and others) - if ("23505".equals(sqlState) || "23000".equals(sqlState)) { - return true; - } - } - cause = cause.getCause(); - } - - // Also check exception message as fallback - String message = e.getMessage(); - if (message != null) { - String lowerMessage = StringUtils.toLowerCase(message); - return lowerMessage.contains("unique constraint") - || lowerMessage.contains("duplicate key") - || lowerMessage.contains("duplicate entry"); - } - - return false; - } } From 0103d8eeb32eccc5280057ad9eac8f5dbb33b0c3 Mon Sep 17 00:00:00 2001 From: capistrant Date: Mon, 12 Jan 2026 14:02:57 -0600 Subject: [PATCH 53/72] some naming cleanup --- docs/api-reference/automatic-compaction-api.md | 2 +- .../segment/cache/HeapMemorySegmentMetadataCache.java | 4 +--- .../java/org/apache/druid/metadata/segment/cache/Metric.java | 5 ----- .../druid/server/coordinator/DruidCompactionConfig.java | 4 ++-- .../druid/server/coordinator/duty/CompactSegments.java | 4 ++-- 5 files changed, 6 insertions(+), 13 deletions(-) diff --git a/docs/api-reference/automatic-compaction-api.md b/docs/api-reference/automatic-compaction-api.md index 992b2b5ab805..f98e19890f33 100644 --- a/docs/api-reference/automatic-compaction-api.md +++ b/docs/api-reference/automatic-compaction-api.md @@ -889,7 +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 predates compaction state fingerprinting (< Druid 36) will result in missing compaction states and trigger compaction on segments that may already be compacted.**|`true`| +|`storeCompactionStatePerSegment`|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 predates 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/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 0ac12c7b4901..862e33c5e554 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 @@ -1196,9 +1196,7 @@ private Map buildFingerprintToStateMapForDeltaSync() final List addedCompactionStateRecords = query( sql -> sql.retrieveCompactionStatesForFingerprints(addedFingerprints) ); - if (addedCompactionStateRecords.size() < addedFingerprints.size()) { - emitMetric(Metric.SKIPPED_COMPACTION_STATES, addedFingerprints.size() - addedCompactionStateRecords.size()); - } + addedCompactionStateRecords.forEach( record -> fingerprintToStateMap.put(record.getFingerprint(), record.getState()) ); diff --git a/server/src/main/java/org/apache/druid/metadata/segment/cache/Metric.java b/server/src/main/java/org/apache/druid/metadata/segment/cache/Metric.java index 101ba575e1e3..625b2f43a242 100644 --- a/server/src/main/java/org/apache/druid/metadata/segment/cache/Metric.java +++ b/server/src/main/java/org/apache/druid/metadata/segment/cache/Metric.java @@ -163,11 +163,6 @@ private Metric() */ public static final String SKIPPED_SEGMENT_SCHEMAS = METRIC_NAME_PREFIX + "schema/skipped"; - /** - * Number of unparseable compaction state records skipped while refreshing the cache. - */ - public static final String SKIPPED_COMPACTION_STATES = METRIC_NAME_PREFIX + "compactionState/skipped"; - /** * Number of compaction states added to the cache in the latest sync. */ 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 2119fb36c4e3..1cc73dc06eea 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 @@ -87,7 +87,7 @@ public DruidCompactionConfig( @JsonProperty("compactionPolicy") @Nullable CompactionCandidateSearchPolicy compactionPolicy, @JsonProperty("useSupervisors") @Nullable Boolean useSupervisors, @JsonProperty("engine") @Nullable CompactionEngine engine, - @JsonProperty("legacyPersistLastCompactionStateInSegments") @Nullable Boolean legacyPersistLastCompactionStateInSegments + @JsonProperty("storeCompactionStatePerSegment") @Nullable Boolean storeCompactionStatePerSegment ) { this( @@ -98,7 +98,7 @@ public DruidCompactionConfig( compactionPolicy, useSupervisors, engine, - legacyPersistLastCompactionStateInSegments + storeCompactionStatePerSegment ) ); } 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 38ff8de92619..3e4de17a51c3 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 @@ -294,7 +294,7 @@ public static ClientCompactionTaskQuery createCompactionTask( DataSourceCompactionConfig config, CompactionEngine defaultEngine, String compactionStateFingerprint, - boolean persistLastCompactionStateInSegments + boolean storeCompactionStatePerSegment ) { final List segmentsToCompact = candidate.getSegments(); @@ -372,7 +372,7 @@ public static ClientCompactionTaskQuery createCompactionTask( autoCompactionContext.put(COMPACTION_REASON_KEY, candidate.getCurrentStatus().getReason()); } - autoCompactionContext.put(STORE_COMPACTION_STATE_KEY, persistLastCompactionStateInSegments); + autoCompactionContext.put(STORE_COMPACTION_STATE_KEY, storeCompactionStatePerSegment); autoCompactionContext.put(COMPACTION_STATE_FINGERPRINT_KEY, compactionStateFingerprint); return compactSegments( From 7aefe3805a8522f89686303ea5e33a14a16be11e Mon Sep 17 00:00:00 2001 From: capistrant Date: Mon, 12 Jan 2026 20:18:28 -0600 Subject: [PATCH 54/72] Migrate the compaction state cleanup duty to the overlord --- .../compact/CompactionSupervisorTest.java | 3 +- .../overlord/config/OverlordKillConfigs.java | 43 +++++++ .../config/OverlordMetadataCleanupConfig.java | 89 ++++++++++++++ .../duty/KillUnreferencedCompactionState.java | 26 ++-- .../duty/OverlordMetadataCleanupDuty.java | 113 ++++++++++++++++++ .../KillUnreferencedCompactionStateTest.java | 70 +++++------ .../coordinator/ClusterCompactionConfig.java | 4 +- .../coordinator/DruidCompactionConfig.java | 2 +- .../server/coordinator/DruidCoordinator.java | 2 - .../server/coordinator/MetadataManager.java | 11 +- .../config/CoordinatorKillConfigs.java | 12 +- .../config/DruidCoordinatorConfig.java | 1 - .../DruidCompactionConfigTest.java | 2 +- .../DruidCoordinatorConfigTest.java | 34 ------ .../coordinator/DruidCoordinatorTest.java | 6 +- .../CoordinatorSimulationBuilder.java | 4 +- .../org/apache/druid/cli/CliOverlord.java | 12 ++ .../druid/guice/MetadataManagerModule.java | 10 +- 18 files changed, 305 insertions(+), 139 deletions(-) create mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/overlord/config/OverlordKillConfigs.java create mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/overlord/config/OverlordMetadataCleanupConfig.java rename {server/src/main/java/org/apache/druid/server/coordinator => indexing-service/src/main/java/org/apache/druid/indexing/overlord}/duty/KillUnreferencedCompactionState.java (69%) create mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/overlord/duty/OverlordMetadataCleanupDuty.java rename {server/src/test/java/org/apache/druid/server/coordinator => indexing-service/src/test/java/org/apache/druid/indexing/overlord}/duty/KillUnreferencedCompactionStateTest.java (77%) 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 15e16746973b..3d1731ced68f 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 @@ -211,9 +211,8 @@ public void test_ingestDayGranularity_andCompactToMonthGranularity_andCompactToY @MethodSource("getEngine") @ParameterizedTest(name = "compactionEngine={0}") public void test_compaction_withPersistLastCompactionStateFalse_storesOnlyFingerprint(CompactionEngine compactionEngine) - throws InterruptedException { - // Configure cluster with persistLastCompactionState=false + // Configure cluster with storeCompactionStatePerSegment=false final UpdateResponse updateResponse = cluster.callApi().onLeaderOverlord( o -> o.updateClusterCompactionConfig( new ClusterCompactionConfig(1.0, 100, null, true, compactionEngine, false) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/config/OverlordKillConfigs.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/config/OverlordKillConfigs.java new file mode 100644 index 000000000000..00fad85fea5e --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/config/OverlordKillConfigs.java @@ -0,0 +1,43 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.overlord.config; + +import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.druid.common.config.Configs; + +public class OverlordKillConfigs +{ + public static OverlordKillConfigs DEFAULT = new OverlordKillConfigs(null); + + @JsonProperty("compactionStates") + private final OverlordMetadataCleanupConfig compactionStates; + + public OverlordKillConfigs( + @JsonProperty("compactionStates") OverlordMetadataCleanupConfig compactionStates + ) + { + this.compactionStates = Configs.valueOrDefault(compactionStates, OverlordMetadataCleanupConfig.DEFAULT); + } + + public OverlordMetadataCleanupConfig compactionStates() + { + return compactionStates; + } +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/config/OverlordMetadataCleanupConfig.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/config/OverlordMetadataCleanupConfig.java new file mode 100644 index 000000000000..b21e7f89784a --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/config/OverlordMetadataCleanupConfig.java @@ -0,0 +1,89 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.overlord.config; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.druid.common.config.Configs; +import org.joda.time.Duration; + +import java.util.Objects; + +public class OverlordMetadataCleanupConfig +{ + public static final OverlordMetadataCleanupConfig DEFAULT = new OverlordMetadataCleanupConfig(null, null, null); + + @JsonProperty("on") + private final boolean cleanupEnabled; + + @JsonProperty("period") + private final Duration cleanupPeriod; + + @JsonProperty("durationToRetain") + private final Duration durationToRetain; + + @JsonCreator + public OverlordMetadataCleanupConfig( + @JsonProperty("on") Boolean cleanupEnabled, + @JsonProperty("period") Duration cleanupPeriod, + @JsonProperty("durationToRetain") Duration durationToRetain + ) + { + this.cleanupEnabled = Configs.valueOrDefault(cleanupEnabled, true); + this.cleanupPeriod = Configs.valueOrDefault(cleanupPeriod, Duration.standardDays(1)); + this.durationToRetain = Configs.valueOrDefault(durationToRetain, Duration.standardDays(90)); + } + + public Duration getCleanupPeriod() + { + return cleanupPeriod; + } + + public Duration getDurationToRetain() + { + return durationToRetain; + } + + public boolean isCleanupEnabled() + { + return cleanupEnabled; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + OverlordMetadataCleanupConfig that = (OverlordMetadataCleanupConfig) o; + return cleanupEnabled == that.cleanupEnabled + && Objects.equals(cleanupPeriod, that.cleanupPeriod) + && Objects.equals(durationToRetain, that.durationToRetain); + } + + @Override + public int hashCode() + { + return Objects.hash(cleanupEnabled, cleanupPeriod, durationToRetain); + } +} diff --git a/server/src/main/java/org/apache/druid/server/coordinator/duty/KillUnreferencedCompactionState.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/duty/KillUnreferencedCompactionState.java similarity index 69% rename from server/src/main/java/org/apache/druid/server/coordinator/duty/KillUnreferencedCompactionState.java rename to indexing-service/src/main/java/org/apache/druid/indexing/overlord/duty/KillUnreferencedCompactionState.java index 35ac3085e3f0..e59dc76968c9 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/duty/KillUnreferencedCompactionState.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/duty/KillUnreferencedCompactionState.java @@ -17,40 +17,28 @@ * under the License. */ -package org.apache.druid.server.coordinator.duty; +package org.apache.druid.indexing.overlord.duty; +import org.apache.druid.indexing.overlord.config.OverlordMetadataCleanupConfig; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.segment.metadata.CompactionStateStorage; -import org.apache.druid.server.coordinator.config.MetadataCleanupConfig; -import org.apache.druid.server.coordinator.stats.Stats; import org.joda.time.DateTime; +import javax.inject.Inject; import java.util.List; -/** - * Coordinator duty that cleans up old, unused compaction state entries from the database. - *

    - * This duty performs a three-step cleanup process: - *

      - *
    1. Marks compaction states not referenced by any segments as unused
    2. - *
    3. Repairs any incorrectly marked unused states that are still referenced by used segments
    4. - *
    5. Deletes unused compaction states older than the configured retention period
    6. - *
    - *

    - * This prevents unbounded growth of the compaction states table while ensuring that - * states referenced by active segments are preserved. - */ -public class KillUnreferencedCompactionState extends MetadataCleanupDuty +public class KillUnreferencedCompactionState extends OverlordMetadataCleanupDuty { private static final Logger log = new Logger(KillUnreferencedCompactionState.class); private final CompactionStateStorage compactionStateStorage; + @Inject public KillUnreferencedCompactionState( - MetadataCleanupConfig config, + OverlordMetadataCleanupConfig config, CompactionStateStorage compactionStateStorage ) { - super("compactionState", config, Stats.Kill.COMPACTION_STATE); + super("compactionState", config); this.compactionStateStorage = compactionStateStorage; } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/duty/OverlordMetadataCleanupDuty.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/duty/OverlordMetadataCleanupDuty.java new file mode 100644 index 000000000000..9092ee00c639 --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/duty/OverlordMetadataCleanupDuty.java @@ -0,0 +1,113 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.overlord.duty; + +import org.apache.druid.indexing.overlord.config.OverlordMetadataCleanupConfig; +import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.java.util.common.logger.Logger; +import org.joda.time.DateTime; + +/** + * Performs cleanup of stale metadata entries created before a configured retain duration. + *

    + * In every invocation of {@link #run}, the duty checks if the {@code cleanupPeriod} + * has elapsed since the {@link #lastCleanupTime}. If it has, then the method + * {@link #cleanupEntriesCreatedBefore(DateTime)} is invoked. Otherwise, the duty + * completes immediately without making any changes. + */ +public abstract class OverlordMetadataCleanupDuty implements OverlordDuty +{ + private static final Logger log = new Logger(OverlordMetadataCleanupDuty.class); + + private final String entryType; + private final OverlordMetadataCleanupConfig cleanupConfig; + + private DateTime lastCleanupTime = DateTimes.utc(0); + + protected OverlordMetadataCleanupDuty(String entryType, OverlordMetadataCleanupConfig cleanupConfig) + { + this.entryType = entryType; + this.cleanupConfig = cleanupConfig; + + if (cleanupConfig.isCleanupEnabled()) { + log.debug( + "Enabled cleanup of [%s] with period [%s] and durationToRetain [%s].", + entryType, cleanupConfig.getCleanupPeriod(), cleanupConfig.getDurationToRetain() + ); + } + } + + @Override + public void run() + { + if (!cleanupConfig.isCleanupEnabled()) { + return; + } + + final DateTime now = getCurrentTime(); + + // Perform cleanup only if cleanup period has elapsed + if (lastCleanupTime.plus(cleanupConfig.getCleanupPeriod()).isBefore(now)) { + lastCleanupTime = now; + + try { + DateTime minCreatedTime = now.minus(cleanupConfig.getDurationToRetain()); + int deletedEntries = cleanupEntriesCreatedBefore(minCreatedTime); + if (deletedEntries > 0) { + log.info("Removed [%,d] [%s] created before [%s].", deletedEntries, entryType, minCreatedTime); + } + } + catch (Exception e) { + log.error(e, "Failed to perform cleanup of [%s]", entryType); + } + } + } + + @Override + public boolean isEnabled() + { + return cleanupConfig.isCleanupEnabled(); + } + + @Override + public DutySchedule getSchedule() + { + if (isEnabled()) { + return new DutySchedule(cleanupConfig.getCleanupPeriod().getMillis(), 0); + } else { + return new DutySchedule(0, 0); + } + } + + /** + * Cleans up metadata entries created before the {@code minCreatedTime}. + *

    + * This method is not invoked if the {@code cleanupPeriod} has not elapsed + * since the {@link #lastCleanupTime}. + * + * @return Number of deleted metadata entries + */ + protected abstract int cleanupEntriesCreatedBefore(DateTime minCreatedTime); + + protected DateTime getCurrentTime() + { + return DateTimes.nowUtc(); + } +} diff --git a/server/src/test/java/org/apache/druid/server/coordinator/duty/KillUnreferencedCompactionStateTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/duty/KillUnreferencedCompactionStateTest.java similarity index 77% rename from server/src/test/java/org/apache/druid/server/coordinator/duty/KillUnreferencedCompactionStateTest.java rename to indexing-service/src/test/java/org/apache/druid/indexing/overlord/duty/KillUnreferencedCompactionStateTest.java index c5341a73ac32..a0c182fdf280 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/duty/KillUnreferencedCompactionStateTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/duty/KillUnreferencedCompactionStateTest.java @@ -17,10 +17,11 @@ * under the License. */ -package org.apache.druid.server.coordinator.duty; +package org.apache.druid.indexing.overlord.duty; import com.fasterxml.jackson.databind.ObjectMapper; import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; +import org.apache.druid.indexing.overlord.config.OverlordMetadataCleanupConfig; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.metadata.MetadataStorageTablesConfig; @@ -28,52 +29,39 @@ import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.metadata.CompactionStateStorage; import org.apache.druid.segment.metadata.SqlCompactionStateStorage; -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 org.junit.Assert; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; import java.util.ArrayList; import java.util.List; -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(); + @Rule + public final TestDerbyConnector.DerbyConnectorRule derbyConnectorRule = + new TestDerbyConnector.DerbyConnectorRule(); private final ObjectMapper jsonMapper = new DefaultObjectMapper(); private TestDerbyConnector derbyConnector; private MetadataStorageTablesConfig tablesConfig; private CompactionStateStorage compactionStateStorage; - private DruidCoordinatorRuntimeParams mockParams; - @BeforeEach + @Before public void setUp() { - derbyConnector = DERBY_CONNECTOR_RULE.getConnector(); - tablesConfig = DERBY_CONNECTOR_RULE.metadataTablesConfigSupplier().get(); + derbyConnector = derbyConnectorRule.getConnector(); + tablesConfig = derbyConnectorRule.metadataTablesConfigSupplier().get(); derbyConnector.createCompactionStatesTable(); derbyConnector.createSegmentTable(); compactionStateStorage = new SqlCompactionStateStorage(tablesConfig, jsonMapper, derbyConnector); - - mockParams = EasyMock.createMock(DruidCoordinatorRuntimeParams.class); - CoordinatorRunStats runStats = new CoordinatorRunStats(); - EasyMock.expect(mockParams.getCoordinatorStats()).andReturn(runStats).anyTimes(); - EasyMock.replay(mockParams); } @Test @@ -86,7 +74,7 @@ public void testKillUnreferencedCompactionState_lifecycle() 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( + OverlordMetadataCleanupConfig cleanupConfig = new OverlordMetadataCleanupConfig( true, Period.parse("PT1H").toStandardDuration(), // cleanup period Period.parse("PT6H").toStandardDuration() // retention duration @@ -104,19 +92,19 @@ public void testKillUnreferencedCompactionState_lifecycle() return null; }); - assertEquals(Boolean.TRUE, getCompactionStateUsedStatus(fingerprint)); + Assert.assertEquals(Boolean.TRUE, getCompactionStateUsedStatus(fingerprint)); // Run 1: Should mark as unused (no segments reference it) - duty.run(mockParams); - assertEquals(Boolean.FALSE, getCompactionStateUsedStatus(fingerprint)); + duty.run(); + Assert.assertEquals(Boolean.FALSE, getCompactionStateUsedStatus(fingerprint)); // Run 2: Still unused, but within retention period - should not delete - duty.run(mockParams); - assertNotNull(getCompactionStateUsedStatus(fingerprint)); + duty.run(); + Assert.assertNotNull(getCompactionStateUsedStatus(fingerprint)); // Run 3: Past retention period - should delete - duty.run(mockParams); - assertNull(getCompactionStateUsedStatus(fingerprint)); + duty.run(); + Assert.assertNull(getCompactionStateUsedStatus(fingerprint)); } @Test @@ -127,7 +115,7 @@ public void testKillUnreferencedCompactionState_repair() dateTimes.add(now); dateTimes.add(now.plusMinutes(61)); - MetadataCleanupConfig cleanupConfig = new MetadataCleanupConfig( + OverlordMetadataCleanupConfig cleanupConfig = new OverlordMetadataCleanupConfig( true, Period.parse("PT1H").toStandardDuration(), Period.parse("PT6H").toStandardDuration() @@ -146,8 +134,8 @@ public void testKillUnreferencedCompactionState_repair() }); // Run 1: Mark as unused - duty.run(mockParams); - assertEquals(Boolean.FALSE, getCompactionStateUsedStatus(fingerprint)); + duty.run(); + Assert.assertEquals(Boolean.FALSE, getCompactionStateUsedStatus(fingerprint)); // Now insert a used segment that references this fingerprint derbyConnector.retryWithHandle(handle -> { @@ -174,14 +162,14 @@ public void testKillUnreferencedCompactionState_repair() }); // Run 2: Repair - should mark it back as used - duty.run(mockParams); - assertEquals(Boolean.TRUE, getCompactionStateUsedStatus(fingerprint)); + duty.run(); + Assert.assertEquals(Boolean.TRUE, getCompactionStateUsedStatus(fingerprint)); } @Test public void testKillUnreferencedCompactionState_disabled() { - MetadataCleanupConfig cleanupConfig = new MetadataCleanupConfig( + OverlordMetadataCleanupConfig cleanupConfig = new OverlordMetadataCleanupConfig( false, // disabled Period.parse("PT1H").toStandardDuration(), Period.parse("PT6H").toStandardDuration() @@ -198,10 +186,10 @@ public void testKillUnreferencedCompactionState_disabled() }); // Run duty - should do nothing - duty.run(mockParams); + duty.run(); // Should still be used (not marked as unused) - assertEquals(Boolean.TRUE, getCompactionStateUsedStatus(fingerprint)); + Assert.assertEquals(Boolean.TRUE, getCompactionStateUsedStatus(fingerprint)); } private static class TestKillUnreferencedCompactionState extends KillUnreferencedCompactionState @@ -210,7 +198,7 @@ private static class TestKillUnreferencedCompactionState extends KillUnreference private int index = -1; public TestKillUnreferencedCompactionState( - MetadataCleanupConfig config, + OverlordMetadataCleanupConfig config, CompactionStateStorage compactionStateStorage, List dateTimes ) 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 daaf98d67511..08e543504a6a 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 @@ -54,7 +54,7 @@ public ClusterCompactionConfig( @JsonProperty("compactionPolicy") @Nullable CompactionCandidateSearchPolicy compactionPolicy, @JsonProperty("useSupervisors") @Nullable Boolean useSupervisors, @JsonProperty("engine") @Nullable CompactionEngine engine, - @JsonProperty("storeCompactionStatePerSegment") Boolean storeCompactionStatePerSegment + @JsonProperty("storeCompactionStatePerSegment") @Nullable Boolean storeCompactionStatePerSegment ) { this.compactionTaskSlotRatio = Configs.valueOrDefault(compactionTaskSlotRatio, 0.1); @@ -155,7 +155,7 @@ public String toString() ", useSupervisors=" + useSupervisors + ", engine=" + engine + ", compactionPolicy=" + compactionPolicy + - ", legacyPersistLastCompactionStateInSegments=" + storeCompactionStatePerSegment + + ", storeCompactionStatePerSegment=" + storeCompactionStatePerSegment + '}'; } } 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 1cc73dc06eea..3af188d9a03a 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 @@ -143,7 +143,7 @@ public CompactionEngine getEngine() } @JsonProperty - public boolean isLegacyPersistLastCompactionStateInSegments() + public boolean isStoreCompactionStatePerSegment() { return clusterConfig.isStoreCompactionStatePerSegment(); } 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 c2d67fb1fc9c..de6ef45a0533 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 @@ -76,7 +76,6 @@ 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; @@ -610,7 +609,6 @@ 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())); } 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 15c7b1f6db90..91df6ae428f8 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,7 +26,6 @@ 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.CompactionStateStorage; import org.apache.druid.segment.metadata.SegmentSchemaManager; import org.apache.druid.timeline.DataSegment; @@ -43,7 +42,6 @@ public class MetadataManager private final IndexerMetadataStorageCoordinator storageCoordinator; private final SegmentSchemaManager segmentSchemaManager; private final SegmentMetadataCache segmentMetadataCache; - private final CompactionStateStorage compactionStateStorage; @Inject public MetadataManager( @@ -54,8 +52,7 @@ public MetadataManager( MetadataRuleManager metadataRuleManager, IndexerMetadataStorageCoordinator storageCoordinator, SegmentSchemaManager segmentSchemaManager, - SegmentMetadataCache segmentMetadataCache, - CompactionStateStorage compactionStateStorage + SegmentMetadataCache segmentMetadataCache ) { this.auditManager = auditManager; @@ -66,7 +63,6 @@ public MetadataManager( this.storageCoordinator = storageCoordinator; this.segmentSchemaManager = segmentSchemaManager; this.segmentMetadataCache = segmentMetadataCache; - this.compactionStateStorage = compactionStateStorage; } public void onLeaderStart() @@ -135,11 +131,6 @@ public SegmentSchemaManager schemas() return segmentSchemaManager; } - public CompactionStateStorage compactionStates() - { - return compactionStateStorage; - } - /** * 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 a7454a7878fa..c1f5f8bca67b 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, null); + = new CoordinatorKillConfigs(null, null, null, null, null, null, null, null, null, null, null, null, null, null); @JsonProperty("supervisor") private final MetadataCleanupConfig supervisors; @@ -74,9 +74,6 @@ public class CoordinatorKillConfigs @JsonProperty("maxInterval") private final Period killUnusedMaxInterval; - @JsonProperty("compactionState") - private final MetadataCleanupConfig compactionStates; - @JsonCreator public CoordinatorKillConfigs( @JsonProperty("pendingSegments") MetadataCleanupConfig pendingSegments, @@ -86,7 +83,6 @@ 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, @@ -104,7 +100,6 @@ 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; @@ -150,11 +145,6 @@ 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 29f43014bf06..6004c5b1ba47 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,7 +96,6 @@ 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/test/java/org/apache/druid/server/coordinator/DruidCompactionConfigTest.java b/server/src/test/java/org/apache/druid/server/coordinator/DruidCompactionConfigTest.java index 2f22ed66e6fb..8fae5623db41 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 @@ -120,6 +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()); + Assert.assertTrue(config.isStoreCompactionStatePerSegment()); } } 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 9390ffaa20d7..d91cb62050a1 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,10 +256,6 @@ 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); @@ -287,10 +283,6 @@ 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()); } @@ -338,12 +330,6 @@ 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 @@ -383,11 +369,6 @@ 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 @@ -433,13 +414,6 @@ 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 @@ -511,7 +485,6 @@ private static class KillConfigBuilder MetadataCleanupConfig pendingSegments; MetadataCleanupConfig segmentSchema; KillUnusedSegmentsConfig unusedSegments; - MetadataCleanupConfig compactionState; KillConfigBuilder audit(MetadataCleanupConfig config) { @@ -549,12 +522,6 @@ KillConfigBuilder segmentSchema(MetadataCleanupConfig config) return this; } - KillConfigBuilder compactionState(MetadataCleanupConfig config) - { - this.compactionState = config; - return this; - } - KillConfigBuilder unusedSegments(KillUnusedSegmentsConfig config) { this.unusedSegments = config; @@ -571,7 +538,6 @@ 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 90a3fdd1ff11..f47cb9fd9f8e 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,7 +46,6 @@ 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.SqlCompactionStateStorage; import org.apache.druid.server.DruidNode; import org.apache.druid.server.compaction.CompactionSimulateResult; import org.apache.druid.server.compaction.CompactionStatusTracker; @@ -113,7 +112,6 @@ public class DruidCoordinatorTest private OverlordClient overlordClient; private CompactionStatusTracker statusTracker; private LatchableServiceEmitter serviceEmitter; - private SqlCompactionStateStorage sqlCompactionStateStorage; @Before public void setUp() throws Exception @@ -124,7 +122,6 @@ public void setUp() throws Exception metadataRuleManager = EasyMock.createNiceMock(MetadataRuleManager.class); loadQueueTaskMaster = EasyMock.createMock(LoadQueueTaskMaster.class); overlordClient = EasyMock.createMock(OverlordClient.class); - sqlCompactionStateStorage = EasyMock.createMock(SqlCompactionStateStorage.class); JacksonConfigManager configManager = EasyMock.createNiceMock(JacksonConfigManager.class); EasyMock.expect( @@ -187,8 +184,7 @@ private MetadataManager createMetadataManager(JacksonConfigManager configManager metadataRuleManager, null, null, - NoopSegmentMetadataCache.instance(), - sqlCompactionStateStorage + NoopSegmentMetadataCache.instance() ); } 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 7b99b6dba988..0838b41402d5 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/simulate/CoordinatorSimulationBuilder.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/simulate/CoordinatorSimulationBuilder.java @@ -42,7 +42,6 @@ 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.HeapMemoryCompactionStateStorage; import org.apache.druid.server.compaction.CompactionStatusTracker; import org.apache.druid.server.coordinator.CloneStatusManager; import org.apache.druid.server.coordinator.CoordinatorConfigManager; @@ -515,8 +514,7 @@ private Environment( ruleManager, null, null, - NoopSegmentMetadataCache.instance(), - new HeapMemoryCompactionStateStorage() + NoopSegmentMetadataCache.instance() ); this.configSyncer = EasyMock.niceMock(CoordinatorDynamicConfigSyncer.class); diff --git a/services/src/main/java/org/apache/druid/cli/CliOverlord.java b/services/src/main/java/org/apache/druid/cli/CliOverlord.java index b23878453efd..3737721b5c76 100644 --- a/services/src/main/java/org/apache/druid/cli/CliOverlord.java +++ b/services/src/main/java/org/apache/druid/cli/CliOverlord.java @@ -90,8 +90,11 @@ import org.apache.druid.indexing.overlord.autoscaling.SimpleWorkerProvisioningConfig; import org.apache.druid.indexing.overlord.autoscaling.SimpleWorkerProvisioningStrategy; import org.apache.druid.indexing.overlord.config.DefaultTaskConfig; +import org.apache.druid.indexing.overlord.config.OverlordKillConfigs; +import org.apache.druid.indexing.overlord.config.OverlordMetadataCleanupConfig; import org.apache.druid.indexing.overlord.config.TaskLockConfig; import org.apache.druid.indexing.overlord.config.TaskQueueConfig; +import org.apache.druid.indexing.overlord.duty.KillUnreferencedCompactionState; import org.apache.druid.indexing.overlord.duty.OverlordDuty; import org.apache.druid.indexing.overlord.duty.TaskLogAutoCleaner; import org.apache.druid.indexing.overlord.duty.TaskLogAutoCleanerConfig; @@ -409,6 +412,13 @@ public TaskStorageDirTracker getTaskStorageDirTracker(WorkerConfig workerConfig, return TaskStorageDirTracker.fromConfigs(workerConfig, taskConfig); } + @Provides + @LazySingleton + public OverlordMetadataCleanupConfig provideCompactionStateCleanupConfig(OverlordKillConfigs killConfigs) + { + return killConfigs.compactionStates(); + } + @Provides @LazySingleton @Named(ServiceStatusMonitor.HEARTBEAT_TAGS_BINDING) @@ -451,9 +461,11 @@ private void configureAutoscale(Binder binder) private void configureOverlordHelpers(Binder binder) { JsonConfigProvider.bind(binder, "druid.indexer.logs.kill", TaskLogAutoCleanerConfig.class); + JsonConfigProvider.bind(binder, "druid.overlord.kill", OverlordKillConfigs.class); final Multibinder dutyBinder = Multibinder.newSetBinder(binder, OverlordDuty.class); dutyBinder.addBinding().to(TaskLogAutoCleaner.class); dutyBinder.addBinding().to(UnusedSegmentsKiller.class).in(LazySingleton.class); + dutyBinder.addBinding().to(KillUnreferencedCompactionState.class); } /** 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 b93264681061..49a19e35a1a7 100644 --- a/services/src/main/java/org/apache/druid/guice/MetadataManagerModule.java +++ b/services/src/main/java/org/apache/druid/guice/MetadataManagerModule.java @@ -43,7 +43,6 @@ import org.apache.druid.metadata.segment.cache.SegmentMetadataCache; import org.apache.druid.segment.metadata.CompactionStateCache; import org.apache.druid.segment.metadata.CompactionStateStorage; -import org.apache.druid.segment.metadata.NoopCompactionStateCache; import org.apache.druid.segment.metadata.NoopSegmentSchemaCache; import org.apache.druid.segment.metadata.SegmentSchemaCache; import org.apache.druid.segment.metadata.SqlCompactionStateStorage; @@ -107,9 +106,6 @@ public void configure(Binder binder) binder.bind(SegmentMetadataCache.class) .to(HeapMemorySegmentMetadataCache.class) .in(LazySingleton.class); - binder.bind(CompactionStateStorage.class) - .to(SqlCompactionStateStorage.class) - .in(ManageLifecycle.class); // Coordinator-only dependencies if (nodeRoles.contains(NodeRole.COORDINATOR)) { @@ -138,13 +134,13 @@ public void configure(Binder binder) .to(SqlSegmentMetadataTransactionFactory.class) .in(LazySingleton.class); binder.bind(CompactionStateCache.class).in(LazySingleton.class); + binder.bind(CompactionStateStorage.class) + .to(SqlCompactionStateStorage.class) + .in(ManageLifecycle.class); } else { binder.bind(SegmentMetadataTransactionFactory.class) .to(SqlSegmentMetadataReadOnlyTransactionFactory.class) .in(LazySingleton.class); - binder.bind(CompactionStateCache.class) - .to(NoopCompactionStateCache.class) - .in(LazySingleton.class); } } } From 5c4fd980e6734491553e713f651845fe3b954b6a Mon Sep 17 00:00:00 2001 From: capistrant Date: Mon, 12 Jan 2026 21:10:19 -0600 Subject: [PATCH 55/72] Blow up the compaction supervisor scheduler if incremental caching is disabled --- .../compact/OverlordCompactionScheduler.java | 14 +++++++++++++- 1 file changed, 13 insertions(+), 1 deletion(-) 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 a96143c756b0..4b19b9c2c141 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 @@ -26,6 +26,7 @@ import org.apache.druid.client.DataSourcesSnapshot; import org.apache.druid.client.broker.BrokerClient; import org.apache.druid.client.indexing.ClientCompactionRunnerInfo; +import org.apache.druid.error.DruidException; import org.apache.druid.guice.annotations.Deterministic; import org.apache.druid.indexer.TaskLocation; import org.apache.druid.indexer.TaskStatus; @@ -221,7 +222,18 @@ public void statusChanged(String taskId, TaskStatus status) @LifecycleStart public synchronized void start() { - // Do nothing + // Validate that if compaction supervisors are enabled, the segment metadata incremental cache must be enabled + if (compactionConfigSupplier.get().isUseSupervisors()) { + if (segmentManager != null && !segmentManager.isPollingDatabasePeriodically()) { + throw DruidException + .forPersona(DruidException.Persona.OPERATOR) + .ofCategory(DruidException.Category.INVALID_INPUT) + .build( + "Compaction supervisors require segment metadata cache to be enabled. " + + "Set 'druid.manager.segments.useIncrementalCache=always' or 'ifSynced'" + ); + } + } } @LifecycleStop From 74cb69b78e88d9a7c8a3586c660c095d5e77be72 Mon Sep 17 00:00:00 2001 From: capistrant Date: Mon, 12 Jan 2026 22:02:38 -0600 Subject: [PATCH 56/72] add some strict input sanitization in upserting compaction fingerprints --- .../metadata/SqlCompactionStateStorage.java | 30 ++++++++-- .../SqlCompactionStateStorageTest.java | 60 +++++++------------ 2 files changed, 46 insertions(+), 44 deletions(-) diff --git a/server/src/main/java/org/apache/druid/segment/metadata/SqlCompactionStateStorage.java b/server/src/main/java/org/apache/druid/segment/metadata/SqlCompactionStateStorage.java index 3dbb630bccf3..0d3076f9d5b5 100644 --- a/server/src/main/java/org/apache/druid/segment/metadata/SqlCompactionStateStorage.java +++ b/server/src/main/java/org/apache/druid/segment/metadata/SqlCompactionStateStorage.java @@ -22,6 +22,7 @@ import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.inject.Inject; +import org.apache.druid.error.DruidException; import org.apache.druid.error.InternalServerError; import org.apache.druid.guice.LazySingleton; import org.apache.druid.java.util.common.DateTimes; @@ -36,6 +37,7 @@ import org.skife.jdbi.v2.Update; import javax.annotation.Nonnull; +import javax.validation.constraints.NotEmpty; import java.util.List; /** @@ -73,14 +75,30 @@ public SqlCompactionStateStorage( @Override public void upsertCompactionState( - final String dataSource, - final String fingerprint, - final CompactionState compactionState, - final DateTime updateTime + @NotEmpty final String dataSource, + @NotEmpty final String fingerprint, + @Nonnull final CompactionState compactionState, + @Nonnull final DateTime updateTime ) { - if (compactionState == null || fingerprint == null || fingerprint.isEmpty()) { - return; + // Strictly sanitize inputs to avoid writing junk data to the rdbms + StringBuilder errors = new StringBuilder(); + if (dataSource == null || dataSource.isEmpty()) { + errors.append("dataSource cannot be empty; "); + } + if (fingerprint == null || fingerprint.isEmpty()) { + errors.append("fingerprint cannot be empty; "); + } + if (compactionState == null) { + errors.append("compactionState cannot be null; "); + } + if (updateTime == null) { + errors.append("updateTime cannot be null; "); + } + if (errors.length() > 0) { + throw DruidException.forPersona(DruidException.Persona.DEVELOPER) + .ofCategory(DruidException.Category.INVALID_INPUT) + .build(errors.toString().trim()); } try { diff --git a/server/src/test/java/org/apache/druid/segment/metadata/SqlCompactionStateStorageTest.java b/server/src/test/java/org/apache/druid/segment/metadata/SqlCompactionStateStorageTest.java index 89109450e6f8..3ea12611cb1b 100644 --- a/server/src/test/java/org/apache/druid/segment/metadata/SqlCompactionStateStorageTest.java +++ b/server/src/test/java/org/apache/druid/segment/metadata/SqlCompactionStateStorageTest.java @@ -45,6 +45,7 @@ import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertNotEquals; +import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; public class SqlCompactionStateStorageTest @@ -242,55 +243,38 @@ public void test_deleteCompactionStatesOlderThan_deletesOnlyOldUnusedStates() } @Test - public void test_upsertCompactionState_withNullState_doesNothing() + public void test_upsertCompactionState_withNullState_throwsException() { - // Get initial count - Integer beforeCount = derbyConnector.retryWithHandle(handle -> - handle.createQuery("SELECT COUNT(*) FROM " + tablesConfig.getCompactionStatesTable()) - .map((i, r, ctx) -> r.getInt(1)) - .first() + Exception exception = assertThrows( + Exception.class, + () -> derbyConnector.retryWithHandle(handle -> { + manager.upsertCompactionState("ds", "somePrint", null, DateTimes.nowUtc()); + return null; + }) ); - // Persist empty map - derbyConnector.retryWithHandle(handle -> { - manager.upsertCompactionState("ds", "somePrint", null, 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() + assertTrue( + exception.getMessage().contains("compactionState cannot be null"), + "Exception message should contain 'compactionState cannot be null'" ); - - assertEquals(beforeCount, afterCount); } @Test - public void test_upsertCompactionState_withEmptyPrint_doesNothing() + public void test_upsertCompactionState_withEmptyFingerprint_throwsException() { - // Get initial count - Integer beforeCount = derbyConnector.retryWithHandle(handle -> - handle.createQuery("SELECT COUNT(*) FROM " + tablesConfig.getCompactionStatesTable()) - .map((i, r, ctx) -> r.getInt(1)) - .first() + // The exception ends up wrapped in a sql exception doe to the retryWithHandle so we will just check the message + Exception exception = assertThrows( + Exception.class, + () -> derbyConnector.retryWithHandle(handle -> { + manager.upsertCompactionState("ds", "", createBasicCompactionState(), DateTimes.nowUtc()); + return null; + }) ); - // Persist empty map - derbyConnector.retryWithHandle(handle -> { - manager.upsertCompactionState("ds", "", createBasicCompactionState(), 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() + assertTrue( + exception.getMessage().contains("fingerprint cannot be empty"), + "Exception message should contain 'fingerprint cannot be empty'" ); - - assertEquals(beforeCount, afterCount); } @Test From fa72c3843a23db567717e68a7b1cb0f06a4cf130 Mon Sep 17 00:00:00 2001 From: capistrant Date: Tue, 13 Jan 2026 09:42:17 -0600 Subject: [PATCH 57/72] cleanup test class --- .../compaction/CompactionStatusTest.java | 23 +++++-------------- 1 file changed, 6 insertions(+), 17 deletions(-) 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 d78646e89809..95df260de8fb 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 @@ -42,7 +42,6 @@ import org.apache.druid.segment.data.CompressionStrategy; import org.apache.druid.segment.metadata.CompactionFingerprintMapper; import org.apache.druid.segment.metadata.CompactionStateCache; -import org.apache.druid.segment.metadata.CompactionStateStorage; import org.apache.druid.segment.metadata.CompactionTestUtils; import org.apache.druid.segment.metadata.DefaultCompactionFingerprintMapper; import org.apache.druid.segment.metadata.HeapMemoryCompactionStateStorage; @@ -611,9 +610,7 @@ public void test_evaluate_needsCompactionWhenAllSegmentsHaveUnexpectedCompaction verifyEvaluationNeedsCompactionBecauseWithCustomSegments( CompactionCandidate.from(segments, null), compactionConfig, - "'segmentGranularity' mismatch: required[DAY], current[HOUR]", - compactionStateStorage, - compactionStateCache + "'segmentGranularity' mismatch: required[DAY], current[HOUR]" ); } @@ -649,9 +646,7 @@ public void test_evaluate_needsCompactionWhenSomeSegmentsHaveUnexpectedCompactio verifyEvaluationNeedsCompactionBecauseWithCustomSegments( CompactionCandidate.from(segments, null), compactionConfig, - "'segmentGranularity' mismatch: required[DAY], current[HOUR]", - compactionStateStorage, - compactionStateCache + "'segmentGranularity' mismatch: required[DAY], current[HOUR]" ); } @@ -694,9 +689,7 @@ public void test_evaluate_needsCompactionWhenUnexpectedFingerprintAndNoFingerpri verifyEvaluationNeedsCompactionBecauseWithCustomSegments( CompactionCandidate.from(segments, null), compactionConfig, - "One or more fingerprinted segments do not have a cached compaction state", - compactionStateStorage, - compactionStateCache + "One or more fingerprinted segments do not have a cached compaction state" ); } @@ -750,9 +743,7 @@ public void test_evaluate_needsCompactionWhenNonFingerprintedSegmentsFailChecksO verifyEvaluationNeedsCompactionBecauseWithCustomSegments( CompactionCandidate.from(segments, null), compactionConfig, - "'segmentGranularity' mismatch: required[DAY], current[HOUR]", - compactionStateStorage, - compactionStateCache + "'segmentGranularity' mismatch: required[DAY], current[HOUR]" ); } @@ -824,15 +815,13 @@ public void test_evaluate_isSkippedWhenInputBytesExceedLimit() private void verifyEvaluationNeedsCompactionBecauseWithCustomSegments( CompactionCandidate candidate, DataSourceCompactionConfig compactionConfig, - String expectedReason, - CompactionStateStorage compactionStateStorage, - CompactionStateCache compactionStateCache + String expectedReason ) { final CompactionStatus status = CompactionStatus.compute( candidate, compactionConfig, - new DefaultCompactionFingerprintMapper(compactionStateCache, new DefaultObjectMapper()) // TODO fix + fingerprintMapper ); Assert.assertFalse(status.isComplete()); From c467a395f31c65205abc68c7de43b5df42685cd4 Mon Sep 17 00:00:00 2001 From: capistrant Date: Tue, 13 Jan 2026 16:13:27 -0600 Subject: [PATCH 58/72] Add pending flag to compaction state to prevent potentially destructive early cleanup --- .../MaterializedViewSupervisorTest.java | 4 +- .../DatasourceOptimizerTest.java | 4 +- .../config/OverlordMetadataCleanupConfig.java | 19 +- .../duty/KillUnreferencedCompactionState.java | 8 +- .../duty/OverlordMetadataCleanupDuty.java | 25 +- .../common/actions/TaskActionTestKit.java | 4 +- .../common/task/IngestionTestBase.java | 4 +- .../overlord/GlobalTaskLockboxTest.java | 7 +- .../overlord/TaskLockBoxConcurrencyTest.java | 4 +- .../indexing/overlord/TaskQueueScaleTest.java | 4 +- .../KillUnreferencedCompactionStateTest.java | 213 ++++++++++++++---- .../SeekableStreamIndexTaskTestBase.java | 4 +- .../IndexerSQLMetadataStorageCoordinator.java | 79 ++++++- .../druid/metadata/SQLMetadataConnector.java | 1 + .../metadata/CompactionStateStorage.java | 15 ++ .../metadata/SqlCompactionStateStorage.java | 59 ++++- ...etadataStorageCoordinatorMarkUsedTest.java | 4 +- ...etadataStorageCoordinatorReadOnlyTest.java | 4 +- ...exerSQLMetadataStorageCoordinatorTest.java | 112 ++++++++- ...orageCoordinatorSchemaPersistenceTest.java | 4 +- .../HeapMemoryCompactionStateStorage.java | 12 + .../SqlCompactionStateStorageTest.java | 73 +++++- .../compaction/CompactionStatusTest.java | 1 - .../duty/KillUnusedSegmentsTest.java | 4 +- .../druid/guice/MetadataManagerModule.java | 6 +- 25 files changed, 584 insertions(+), 90 deletions(-) diff --git a/extensions-contrib/materialized-view-maintenance/src/test/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisorTest.java b/extensions-contrib/materialized-view-maintenance/src/test/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisorTest.java index 0e225bf92fa5..2552f6092e7b 100644 --- a/extensions-contrib/materialized-view-maintenance/src/test/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisorTest.java +++ b/extensions-contrib/materialized-view-maintenance/src/test/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisorTest.java @@ -53,6 +53,7 @@ import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; +import org.apache.druid.segment.metadata.HeapMemoryCompactionStateStorage; import org.apache.druid.segment.metadata.SegmentSchemaManager; import org.apache.druid.segment.realtime.ChatHandlerProvider; import org.apache.druid.server.coordinator.simulate.TestDruidLeaderSelector; @@ -120,7 +121,8 @@ public void setUp() derbyConnectorRule.metadataTablesConfigSupplier().get(), derbyConnector, segmentSchemaManager, - CentralizedDatasourceSchemaConfig.create() + CentralizedDatasourceSchemaConfig.create(), + new HeapMemoryCompactionStateStorage() ); metadataSupervisorManager = EasyMock.createMock(MetadataSupervisorManager.class); taskQueue = EasyMock.createMock(TaskQueue.class); diff --git a/extensions-contrib/materialized-view-selection/src/test/java/org/apache/druid/query/materializedview/DatasourceOptimizerTest.java b/extensions-contrib/materialized-view-selection/src/test/java/org/apache/druid/query/materializedview/DatasourceOptimizerTest.java index 1ec98359db79..dce71b491520 100644 --- a/extensions-contrib/materialized-view-selection/src/test/java/org/apache/druid/query/materializedview/DatasourceOptimizerTest.java +++ b/extensions-contrib/materialized-view-selection/src/test/java/org/apache/druid/query/materializedview/DatasourceOptimizerTest.java @@ -55,6 +55,7 @@ import org.apache.druid.query.topn.TopNQueryBuilder; import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; +import org.apache.druid.segment.metadata.HeapMemoryCompactionStateStorage; import org.apache.druid.segment.metadata.SegmentSchemaManager; import org.apache.druid.segment.realtime.appenderator.SegmentSchemas; import org.apache.druid.server.coordination.DruidServerMetadata; @@ -129,7 +130,8 @@ public void setUp() throws Exception derbyConnectorRule.metadataTablesConfigSupplier().get(), derbyConnector, segmentSchemaManager, - CentralizedDatasourceSchemaConfig.create() + CentralizedDatasourceSchemaConfig.create(), + new HeapMemoryCompactionStateStorage() ); setupServerAndCurator(); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/config/OverlordMetadataCleanupConfig.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/config/OverlordMetadataCleanupConfig.java index b21e7f89784a..1f28922d2e0d 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/config/OverlordMetadataCleanupConfig.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/config/OverlordMetadataCleanupConfig.java @@ -28,7 +28,7 @@ public class OverlordMetadataCleanupConfig { - public static final OverlordMetadataCleanupConfig DEFAULT = new OverlordMetadataCleanupConfig(null, null, null); + public static final OverlordMetadataCleanupConfig DEFAULT = new OverlordMetadataCleanupConfig(null, null, null, null); @JsonProperty("on") private final boolean cleanupEnabled; @@ -39,16 +39,21 @@ public class OverlordMetadataCleanupConfig @JsonProperty("durationToRetain") private final Duration durationToRetain; + @JsonProperty("pendingDurationToRetain") + private final Duration pendingDurationToRetain; + @JsonCreator public OverlordMetadataCleanupConfig( @JsonProperty("on") Boolean cleanupEnabled, @JsonProperty("period") Duration cleanupPeriod, - @JsonProperty("durationToRetain") Duration durationToRetain + @JsonProperty("durationToRetain") Duration durationToRetain, + @JsonProperty("pendingDurationToRetain") Duration pendingDurationToRetain ) { this.cleanupEnabled = Configs.valueOrDefault(cleanupEnabled, true); this.cleanupPeriod = Configs.valueOrDefault(cleanupPeriod, Duration.standardDays(1)); this.durationToRetain = Configs.valueOrDefault(durationToRetain, Duration.standardDays(90)); + this.pendingDurationToRetain = Configs.valueOrDefault(pendingDurationToRetain, Duration.standardDays(7)); } public Duration getCleanupPeriod() @@ -61,6 +66,11 @@ public Duration getDurationToRetain() return durationToRetain; } + public Duration getPendingDurationToRetain() + { + return pendingDurationToRetain; + } + public boolean isCleanupEnabled() { return cleanupEnabled; @@ -78,12 +88,13 @@ public boolean equals(Object o) OverlordMetadataCleanupConfig that = (OverlordMetadataCleanupConfig) o; return cleanupEnabled == that.cleanupEnabled && Objects.equals(cleanupPeriod, that.cleanupPeriod) - && Objects.equals(durationToRetain, that.durationToRetain); + && Objects.equals(durationToRetain, that.durationToRetain) + && Objects.equals(pendingDurationToRetain, that.pendingDurationToRetain); } @Override public int hashCode() { - return Objects.hash(cleanupEnabled, cleanupPeriod, durationToRetain); + return Objects.hash(cleanupEnabled, cleanupPeriod, durationToRetain, pendingDurationToRetain); } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/duty/KillUnreferencedCompactionState.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/duty/KillUnreferencedCompactionState.java index e59dc76968c9..804cf9c0faef 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/duty/KillUnreferencedCompactionState.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/duty/KillUnreferencedCompactionState.java @@ -43,7 +43,7 @@ public KillUnreferencedCompactionState( } @Override - protected int cleanupEntriesCreatedBefore(DateTime minCreatedTime) + protected int cleanupEntriesCreatedBeforeDurationToRetain(DateTime minCreatedTime) { // 1: Mark unreferenced states as unused int unused = compactionStateStorage.markUnreferencedCompactionStatesAsUnused(); @@ -59,4 +59,10 @@ protected int cleanupEntriesCreatedBefore(DateTime minCreatedTime) // 3: Delete unused states older than threshold return compactionStateStorage.deleteUnusedCompactionStatesOlderThan(minCreatedTime.getMillis()); } + + @Override + protected int cleanupEntriesCreatedBeforePendingDurationToRetain(DateTime minCreatedTime) + { + return compactionStateStorage.deletePendingCompactionStatesOlderThan(minCreatedTime.getMillis()); + } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/duty/OverlordMetadataCleanupDuty.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/duty/OverlordMetadataCleanupDuty.java index 9092ee00c639..57da62e7de4b 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/duty/OverlordMetadataCleanupDuty.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/duty/OverlordMetadataCleanupDuty.java @@ -29,7 +29,7 @@ *

    * In every invocation of {@link #run}, the duty checks if the {@code cleanupPeriod} * has elapsed since the {@link #lastCleanupTime}. If it has, then the method - * {@link #cleanupEntriesCreatedBefore(DateTime)} is invoked. Otherwise, the duty + * {@link #cleanupEntriesCreatedBeforeDurationToRetain(DateTime)} is invoked. Otherwise, the duty * completes immediately without making any changes. */ public abstract class OverlordMetadataCleanupDuty implements OverlordDuty @@ -69,10 +69,15 @@ public void run() try { DateTime minCreatedTime = now.minus(cleanupConfig.getDurationToRetain()); - int deletedEntries = cleanupEntriesCreatedBefore(minCreatedTime); + int deletedEntries = cleanupEntriesCreatedBeforeDurationToRetain(minCreatedTime); if (deletedEntries > 0) { log.info("Removed [%,d] [%s] created before [%s].", deletedEntries, entryType, minCreatedTime); } + DateTime pendingMinCreatedTime = now.minus(cleanupConfig.getPendingDurationToRetain()); + int deletedPendingEntries = cleanupEntriesCreatedBeforePendingDurationToRetain(pendingMinCreatedTime); + if (deletedPendingEntries > 0) { + log.info("Removed [%,d] pending entries [%s] created before [%s].", deletedPendingEntries, entryType, pendingMinCreatedTime); + } } catch (Exception e) { log.error(e, "Failed to perform cleanup of [%s]", entryType); @@ -97,14 +102,22 @@ public DutySchedule getSchedule() } /** - * Cleans up metadata entries created before the {@code minCreatedTime}. + * Cleans up metadata entries created before the {@code minCreatedTime} calculated with {@link OverlordMetadataCleanupConfig#durationToRetain}. *

    - * This method is not invoked if the {@code cleanupPeriod} has not elapsed - * since the {@link #lastCleanupTime}. + * This method is not invoked if the {@code cleanupPeriod} has not elapsed since the {@link #lastCleanupTime}. * * @return Number of deleted metadata entries */ - protected abstract int cleanupEntriesCreatedBefore(DateTime minCreatedTime); + protected abstract int cleanupEntriesCreatedBeforeDurationToRetain(DateTime minCreatedTime); + + /** + * Cleans up pending metadata entries created before the {@code minCreatedTime} calculated with {@link OverlordMetadataCleanupConfig#pendingDurationToRetain}. + *

    + * This method is not invoked if the {@code cleanupPeriod} has not elapsed since the {@link #lastCleanupTime}. + * + * @return Number of deleted pending metadata entries + */ + protected abstract int cleanupEntriesCreatedBeforePendingDurationToRetain(DateTime minCreatedTime); protected DateTime getCurrentTime() { diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/TaskActionTestKit.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/TaskActionTestKit.java index 58d16d964bbf..5a1a2093270e 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/TaskActionTestKit.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/TaskActionTestKit.java @@ -42,6 +42,7 @@ import org.apache.druid.metadata.segment.cache.SegmentMetadataCache; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; import org.apache.druid.segment.metadata.CompactionStateCache; +import org.apache.druid.segment.metadata.HeapMemoryCompactionStateStorage; import org.apache.druid.segment.metadata.NoopSegmentSchemaCache; import org.apache.druid.segment.metadata.SegmentSchemaManager; import org.apache.druid.server.coordinator.simulate.BlockingExecutorService; @@ -135,7 +136,8 @@ public void before() metadataStorageTablesConfig, testDerbyConnector, segmentSchemaManager, - CentralizedDatasourceSchemaConfig.create() + CentralizedDatasourceSchemaConfig.create(), + new HeapMemoryCompactionStateStorage() ); taskLockbox = new GlobalTaskLockbox(taskStorage, metadataStorageCoordinator); taskLockbox.syncFromStorage(); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IngestionTestBase.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IngestionTestBase.java index fc656bda3f67..59933d9d67db 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IngestionTestBase.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IngestionTestBase.java @@ -88,6 +88,7 @@ import org.apache.druid.segment.loading.SegmentCacheManager; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; import org.apache.druid.segment.metadata.CompactionStateCache; +import org.apache.druid.segment.metadata.HeapMemoryCompactionStateStorage; import org.apache.druid.segment.metadata.SegmentSchemaCache; import org.apache.druid.segment.metadata.SegmentSchemaManager; import org.apache.druid.segment.realtime.NoopChatHandlerProvider; @@ -182,7 +183,8 @@ public void setUpIngestionTestBase() throws IOException derbyConnectorRule.metadataTablesConfigSupplier().get(), derbyConnectorRule.getConnector(), segmentSchemaManager, - CentralizedDatasourceSchemaConfig.create() + CentralizedDatasourceSchemaConfig.create(), + new HeapMemoryCompactionStateStorage() ); segmentsMetadataManager = new SqlSegmentsMetadataManagerV2( segmentMetadataCache, diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/GlobalTaskLockboxTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/GlobalTaskLockboxTest.java index 2c5390e568b9..c3313c20cd6e 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/GlobalTaskLockboxTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/GlobalTaskLockboxTest.java @@ -60,6 +60,7 @@ import org.apache.druid.metadata.segment.cache.NoopSegmentMetadataCache; import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; +import org.apache.druid.segment.metadata.HeapMemoryCompactionStateStorage; import org.apache.druid.segment.metadata.SegmentSchemaManager; import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; import org.apache.druid.server.coordinator.simulate.TestDruidLeaderSelector; @@ -145,7 +146,8 @@ public void setup() tablesConfig, derbyConnector, segmentSchemaManager, - CentralizedDatasourceSchemaConfig.create() + CentralizedDatasourceSchemaConfig.create(), + new HeapMemoryCompactionStateStorage() ); lockbox = new GlobalTaskLockbox(taskStorage, metadataStorageCoordinator); @@ -491,7 +493,8 @@ public void testSyncWithUnknownTaskTypesFromModuleNotLoaded() derby.metadataTablesConfigSupplier().get(), derbyConnector, segmentSchemaManager, - CentralizedDatasourceSchemaConfig.create() + CentralizedDatasourceSchemaConfig.create(), + new HeapMemoryCompactionStateStorage() ); GlobalTaskLockbox theBox = new GlobalTaskLockbox(taskStorage, metadataStorageCoordinator); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLockBoxConcurrencyTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLockBoxConcurrencyTest.java index 723af0e721d0..e96a6b0723b1 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLockBoxConcurrencyTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLockBoxConcurrencyTest.java @@ -41,6 +41,7 @@ import org.apache.druid.metadata.segment.SqlSegmentMetadataTransactionFactory; import org.apache.druid.metadata.segment.cache.NoopSegmentMetadataCache; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; +import org.apache.druid.segment.metadata.HeapMemoryCompactionStateStorage; import org.apache.druid.segment.metadata.SegmentSchemaManager; import org.apache.druid.server.coordinator.simulate.TestDruidLeaderSelector; import org.apache.druid.server.metrics.NoopServiceEmitter; @@ -106,7 +107,8 @@ public void setup() derby.metadataTablesConfigSupplier().get(), derbyConnector, segmentSchemaManager, - CentralizedDatasourceSchemaConfig.create() + CentralizedDatasourceSchemaConfig.create(), + new HeapMemoryCompactionStateStorage() ) ); lockbox.syncFromStorage(); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskQueueScaleTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskQueueScaleTest.java index 447a51f44b11..3a76f415940f 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskQueueScaleTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskQueueScaleTest.java @@ -51,6 +51,7 @@ import org.apache.druid.metadata.segment.cache.NoopSegmentMetadataCache; import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; +import org.apache.druid.segment.metadata.HeapMemoryCompactionStateStorage; import org.apache.druid.segment.metadata.SegmentSchemaManager; import org.apache.druid.server.coordinator.simulate.TestDruidLeaderSelector; import org.apache.druid.server.metrics.NoopServiceEmitter; @@ -118,7 +119,8 @@ public void setUp() derbyConnectorRule.metadataTablesConfigSupplier().get(), derbyConnectorRule.getConnector(), segmentSchemaManager, - CentralizedDatasourceSchemaConfig.create() + CentralizedDatasourceSchemaConfig.create(), + new HeapMemoryCompactionStateStorage() ); final TaskActionClientFactory unsupportedTaskActionFactory = diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/duty/KillUnreferencedCompactionStateTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/duty/KillUnreferencedCompactionStateTest.java index a0c182fdf280..df145d4b4945 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/duty/KillUnreferencedCompactionStateTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/duty/KillUnreferencedCompactionStateTest.java @@ -50,7 +50,7 @@ public class KillUnreferencedCompactionStateTest private TestDerbyConnector derbyConnector; private MetadataStorageTablesConfig tablesConfig; - private CompactionStateStorage compactionStateStorage; + private SqlCompactionStateStorage compactionStateStorage; @Before public void setUp() @@ -65,32 +65,30 @@ public void setUp() } @Test - public void testKillUnreferencedCompactionState_lifecycle() + public void test_killUnreferencedCompactionState_validateLifecycleOfActiveCompactionState() { - // Setup time progression: now, +1hr, +7hrs (past cleanup period and retention) + // Setup time progression: now, +1hr, +7hrs 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 + dateTimes.add(now); + dateTimes.add(now.plusMinutes(61)); + dateTimes.add(now.plusMinutes(6 * 60 + 1)); OverlordMetadataCleanupConfig cleanupConfig = new OverlordMetadataCleanupConfig( true, - Period.parse("PT1H").toStandardDuration(), // cleanup period - Period.parse("PT6H").toStandardDuration() // retention duration + Period.parse("PT1H").toStandardDuration(), + Period.parse("PT6H").toStandardDuration(), // Unused and over 6 hours old should be deleted + Period.parse("P8D").toStandardDuration() ); KillUnreferencedCompactionState duty = new TestKillUnreferencedCompactionState(cleanupConfig, compactionStateStorage, dateTimes); - // Insert a compaction state (initially marked as used) String fingerprint = "test_fingerprint"; CompactionState state = createTestCompactionState(); - derbyConnector.retryWithHandle(handle -> { - compactionStateStorage.upsertCompactionState("test-ds", fingerprint, state, DateTimes.nowUtc()); - return null; - }); + compactionStateStorage.upsertCompactionState("test-ds", fingerprint, state, DateTimes.nowUtc()); + compactionStateStorage.markCompactionStatesAsActive(fingerprint); Assert.assertEquals(Boolean.TRUE, getCompactionStateUsedStatus(fingerprint)); @@ -108,7 +106,7 @@ public void testKillUnreferencedCompactionState_lifecycle() } @Test - public void testKillUnreferencedCompactionState_repair() + public void test_killUnreferencedCompactionState_validateRepair() { List dateTimes = new ArrayList<>(); DateTime now = DateTimes.nowUtc(); @@ -118,7 +116,8 @@ public void testKillUnreferencedCompactionState_repair() OverlordMetadataCleanupConfig cleanupConfig = new OverlordMetadataCleanupConfig( true, Period.parse("PT1H").toStandardDuration(), - Period.parse("PT6H").toStandardDuration() + Period.parse("PT6H").toStandardDuration(), + Period.parse("P8D").toStandardDuration() ); KillUnreferencedCompactionState duty = @@ -128,12 +127,10 @@ public void testKillUnreferencedCompactionState_repair() String fingerprint = "repair_fingerprint"; CompactionState state = createTestCompactionState(); - derbyConnector.retryWithHandle(handle -> { - compactionStateStorage.upsertCompactionState("test-ds", fingerprint, state, DateTimes.nowUtc()); - return null; - }); + compactionStateStorage.upsertCompactionState("test-ds", fingerprint, state, DateTimes.nowUtc()); + compactionStateStorage.markCompactionStatesAsActive(fingerprint); - // Run 1: Mark as unused + Assert.assertEquals(Boolean.TRUE, getCompactionStateUsedStatus(fingerprint)); duty.run(); Assert.assertEquals(Boolean.FALSE, getCompactionStateUsedStatus(fingerprint)); @@ -161,18 +158,19 @@ public void testKillUnreferencedCompactionState_repair() return null; }); - // Run 2: Repair - should mark it back as used + // Confirm that the state is "repaired" now that it is referenced duty.run(); Assert.assertEquals(Boolean.TRUE, getCompactionStateUsedStatus(fingerprint)); } @Test - public void testKillUnreferencedCompactionState_disabled() + public void test_killUnreferencedCompactionState_disabled() { OverlordMetadataCleanupConfig cleanupConfig = new OverlordMetadataCleanupConfig( - false, // disabled + false, // cleanup disabled Period.parse("PT1H").toStandardDuration(), - Period.parse("PT6H").toStandardDuration() + Period.parse("PT6H").toStandardDuration(), + Period.parse("P8D").toStandardDuration() ); KillUnreferencedCompactionState duty = @@ -180,18 +178,162 @@ public void testKillUnreferencedCompactionState_disabled() // Insert compaction state String fingerprint = "disabled_fingerprint"; + compactionStateStorage.upsertCompactionState("test-ds", fingerprint, createTestCompactionState(), DateTimes.nowUtc()); + compactionStateStorage.markCompactionStatesAsActive(fingerprint); + + // Run duty - should do nothing + duty.run(); + + // Should still be used (not marked as unused since cleanup is disabled) + Assert.assertEquals(Boolean.TRUE, getCompactionStateUsedStatus(fingerprint)); + } + + @Test + public void test_killUnreferencedCompactionState_validateLifecycleOfPendingCompactionState() + { + List dateTimes = new ArrayList<>(); + DateTime now = DateTimes.nowUtc(); + dateTimes.add(now.plusDays(8)); + dateTimes.add(now.plusDays(15)); + + OverlordMetadataCleanupConfig cleanupConfig = new OverlordMetadataCleanupConfig( + true, + Period.parse("PT1H").toStandardDuration(), + Period.parse("P7D").toStandardDuration(), + Period.parse("P10D").toStandardDuration() // Pending states older than 10 days should be deleted + ); + + KillUnreferencedCompactionState duty = + new TestKillUnreferencedCompactionState(cleanupConfig, compactionStateStorage, dateTimes); + + String fingerprint = "pending_fingerprint"; + CompactionState state = createTestCompactionState(); + compactionStateStorage.upsertCompactionState("test-ds", fingerprint, state, DateTimes.nowUtc()); + + Assert.assertEquals(Boolean.TRUE, compactionStateStorage.isCompactionStatePending(fingerprint)); + + duty.run(); + Assert.assertNotNull(compactionStateStorage.isCompactionStatePending(fingerprint)); + + duty.run(); + Assert.assertNull(compactionStateStorage.isCompactionStatePending(fingerprint)); + } + + /** + * Validate multiple states cleaned up as per their individual retention policies. + */ + @Test + public void test_killUnreferencedCompactionState_validateMixedPendingAndActiveCompactionStateCleanup() + { + List dateTimes = new ArrayList<>(); + DateTime now = DateTimes.nowUtc(); + dateTimes.add(now.plusDays(8)); + dateTimes.add(now.plusDays(31)); + + OverlordMetadataCleanupConfig cleanupConfig = new OverlordMetadataCleanupConfig( + true, + Period.parse("PT1H").toStandardDuration(), + Period.parse("P7D").toStandardDuration(), + Period.parse("P30D").toStandardDuration() + ); + + KillUnreferencedCompactionState duty = + new TestKillUnreferencedCompactionState(cleanupConfig, compactionStateStorage, dateTimes); + + String pendingFingerprint = "pending_fp"; + String nonPendingFingerprint = "non_pending_fp"; + CompactionState state = createTestCompactionState(); + + compactionStateStorage.upsertCompactionState("test-ds", pendingFingerprint, state, DateTimes.nowUtc()); + compactionStateStorage.upsertCompactionState("test-ds", nonPendingFingerprint, state, DateTimes.nowUtc()); + compactionStateStorage.markCompactionStatesAsActive(nonPendingFingerprint); + + Assert.assertEquals(Boolean.TRUE, compactionStateStorage.isCompactionStatePending(pendingFingerprint)); + Assert.assertNotNull(getCompactionStateUsedStatus(nonPendingFingerprint)); + + duty.run(); + Assert.assertNotNull(compactionStateStorage.isCompactionStatePending(pendingFingerprint)); + Assert.assertNull(getCompactionStateUsedStatus(nonPendingFingerprint)); + + duty.run(); + Assert.assertNull(getCompactionStateUsedStatus(nonPendingFingerprint)); + Assert.assertNull(compactionStateStorage.isCompactionStatePending(pendingFingerprint)); + } + + @Test + public void test_killUnreferencedCompactionState_pendingStateMarkedActiveNotDeleted() + { + List dateTimes = new ArrayList<>(); + DateTime now = DateTimes.nowUtc(); + dateTimes.add(now.plusDays(31)); // The state would be removed if it was still pending + + OverlordMetadataCleanupConfig cleanupConfig = new OverlordMetadataCleanupConfig( + true, + Period.parse("PT1H").toStandardDuration(), + Period.parse("P7D").toStandardDuration(), + Period.parse("P30D").toStandardDuration() + ); + + KillUnreferencedCompactionState duty = + new TestKillUnreferencedCompactionState(cleanupConfig, compactionStateStorage, dateTimes); + + String fingerprint = "pending_marked_active_fp"; + CompactionState state = createTestCompactionState(); + + compactionStateStorage.upsertCompactionState("test-ds", fingerprint, state, DateTimes.nowUtc()); + Assert.assertEquals(Boolean.TRUE, compactionStateStorage.isCompactionStatePending(fingerprint)); + + // Now insert a used segment that references this fingerprint derbyConnector.retryWithHandle(handle -> { - compactionStateStorage.upsertCompactionState("test-ds", fingerprint, createTestCompactionState(), DateTimes.nowUtc()); + 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 duty - should do nothing + compactionStateStorage.markCompactionStatesAsActive(fingerprint); + Assert.assertNotEquals(Boolean.TRUE, compactionStateStorage.isCompactionStatePending(fingerprint)); + duty.run(); + Assert.assertNotNull(compactionStateStorage.isCompactionStatePending(fingerprint)); + } - // Should still be used (not marked as unused) - Assert.assertEquals(Boolean.TRUE, getCompactionStateUsedStatus(fingerprint)); + 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); } + /** + * Extension of KillUnreferencedCompactionState that allows controlling the reference time used for cleanup decisions. + *

    + * Allowing time control enables realistic testing of time-based retention logic. + */ private static class TestKillUnreferencedCompactionState extends KillUnreferencedCompactionState { private final List dateTimes; @@ -224,19 +366,4 @@ private CompactionState createTestCompactionState() 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/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskTestBase.java b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskTestBase.java index f0c273baa904..4bf36fbaecb1 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskTestBase.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskTestBase.java @@ -120,6 +120,7 @@ import org.apache.druid.segment.loading.LocalDataSegmentPusher; import org.apache.druid.segment.loading.LocalDataSegmentPusherConfig; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; +import org.apache.druid.segment.metadata.HeapMemoryCompactionStateStorage; import org.apache.druid.segment.metadata.SegmentSchemaManager; import org.apache.druid.segment.realtime.NoopChatHandlerProvider; import org.apache.druid.segment.realtime.SegmentGenerationMetrics; @@ -636,7 +637,8 @@ protected void makeToolboxFactory(TestUtils testUtils, ServiceEmitter emitter, b derby.metadataTablesConfigSupplier().get(), derbyConnector, segmentSchemaManager, - CentralizedDatasourceSchemaConfig.create() + CentralizedDatasourceSchemaConfig.create(), + new HeapMemoryCompactionStateStorage() ); taskLockbox = new GlobalTaskLockbox(taskStorage, metadataStorageCoordinator); final TaskActionToolbox taskActionToolbox = new TaskActionToolbox( 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 14cd95fd7ed7..c4b7e5ac6778 100644 --- a/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java +++ b/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java @@ -56,6 +56,7 @@ import org.apache.druid.segment.SegmentSchemaMapping; import org.apache.druid.segment.SegmentUtils; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; +import org.apache.druid.segment.metadata.CompactionStateStorage; import org.apache.druid.segment.metadata.SegmentSchemaManager; import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; import org.apache.druid.server.http.DataSegmentPlus; @@ -111,6 +112,7 @@ public class IndexerSQLMetadataStorageCoordinator implements IndexerMetadataStor private final SegmentSchemaManager segmentSchemaManager; private final CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig; private final boolean schemaPersistEnabled; + private final CompactionStateStorage compactionStateStorage; private final SegmentMetadataTransactionFactory transactionFactory; @@ -121,7 +123,8 @@ public IndexerSQLMetadataStorageCoordinator( MetadataStorageTablesConfig dbTables, SQLMetadataConnector connector, SegmentSchemaManager segmentSchemaManager, - CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig + CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig, + CompactionStateStorage compactionStateStorage ) { this.transactionFactory = transactionFactory; @@ -133,6 +136,7 @@ public IndexerSQLMetadataStorageCoordinator( this.schemaPersistEnabled = centralizedDatasourceSchemaConfig.isEnabled() && !centralizedDatasourceSchemaConfig.isTaskSchemaPublishDisabled(); + this.compactionStateStorage = compactionStateStorage; } @LifecycleStart @@ -438,12 +442,12 @@ public SegmentPublishResult commitSegmentsAndMetadata( final String dataSource = segments.iterator().next().getDataSource(); try { - return inReadWriteDatasourceTransaction( + final SegmentPublishResult result = inReadWriteDatasourceTransaction( dataSource, transaction -> { // Try to update datasource metadata first if (startMetadata != null) { - final SegmentPublishResult result = updateDataSourceMetadataInTransaction( + final SegmentPublishResult metadataResult = updateDataSourceMetadataInTransaction( transaction, supervisorId, dataSource, @@ -452,8 +456,8 @@ public SegmentPublishResult commitSegmentsAndMetadata( ); // Do not proceed if the datasource metadata update failed - if (!result.isSuccess()) { - return result; + if (!metadataResult.isSuccess()) { + return metadataResult; } } @@ -462,6 +466,13 @@ public SegmentPublishResult commitSegmentsAndMetadata( ); } ); + + // Mark compaction state fingerprints as active after successful publish + if (result.isSuccess()) { + markCompactionFingerprintsAsActive(result.getSegments()); + } + + return result; } catch (CallbackFailedException e) { throw e; @@ -478,7 +489,7 @@ public SegmentPublishResult commitReplaceSegments( final String dataSource = verifySegmentsToCommit(replaceSegments); try { - return inReadWriteDatasourceTransaction( + final SegmentPublishResult result = inReadWriteDatasourceTransaction( dataSource, transaction -> { final Set segmentsToInsert = new HashSet<>(replaceSegments); @@ -520,6 +531,13 @@ public SegmentPublishResult commitReplaceSegments( ); } ); + + // Mark compaction state fingerprints as active after successful publish + if (result.isSuccess()) { + markCompactionFingerprintsAsActive(result.getSegments()); + } + + return result; } catch (CallbackFailedException e) { return SegmentPublishResult.fail(e.getMessage()); @@ -1213,7 +1231,7 @@ private SegmentPublishResult commitAppendSegmentsAndMetadataInTransaction( ); try { - return inReadWriteDatasourceTransaction( + final SegmentPublishResult result = inReadWriteDatasourceTransaction( dataSource, transaction -> { // Try to update datasource metadata first @@ -1254,6 +1272,13 @@ private SegmentPublishResult commitAppendSegmentsAndMetadataInTransaction( ); } ); + + // Mark compaction state fingerprints as active after successful publish + if (result.isSuccess()) { + markCompactionFingerprintsAsActive(result.getSegments()); + } + + return result; } catch (CallbackFailedException e) { throw e; @@ -2688,6 +2713,46 @@ public Map> retrieveUpgradedToSegmentIds( return upgradedToSegmentIds; } + /** + * Marks compaction state fingerprints as active (non-pending) for successfully published segments. + *

    + * Extracts unique compaction state fingerprints from the given segments and marks them as active + * in the compaction state storage. This is called after successful segment publishing to indicate + * that the compaction state is no longer pending and can be retained with the regular grace period. + *

    + * Most calls result in 0-row updates since fingerprints are stable and the first task already + * marked them as active. + * + * @param segments The segments that were successfully published + */ + private void markCompactionFingerprintsAsActive(Set segments) + { + if (segments == null || segments.isEmpty()) { + return; + } + + // Collect unique non-null compaction state fingerprints + final Set fingerprints = segments.stream() + .map(DataSegment::getCompactionStateFingerprint) + .filter(fp -> fp != null && !fp.isEmpty()) + .collect(Collectors.toSet()); + + // Mark each fingerprint as active + for (String fingerprint : fingerprints) { + try { + int rowsUpdated = compactionStateStorage.markCompactionStatesAsActive(fingerprint); + if (rowsUpdated > 0) { + log.info("Marked compaction state fingerprint[%s] as active (non-pending).", fingerprint); + } + } + catch (Exception e) { + // Log but don't fail the overall operation - the fingerprint will stay pending + // and be cleaned up by the pending grace period + log.warn(e, "Failed to mark compaction state fingerprint[%s] as active. Will retry on next publish.", fingerprint); + } + } + } + /** * Performs a read-write transaction using the {@link SegmentMetadataTransactionFactory}, * which may use the segment metadata cache, if enabled and ready. 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 95e0b5ced84a..c8106cf99219 100644 --- a/server/src/main/java/org/apache/druid/metadata/SQLMetadataConnector.java +++ b/server/src/main/java/org/apache/druid/metadata/SQLMetadataConnector.java @@ -1130,6 +1130,7 @@ public void createCompactionStatesTable(final String tableName) + " fingerprint VARCHAR(255) NOT NULL,\n" + " payload %2$s NOT NULL,\n" + " used BOOLEAN NOT NULL,\n" + + " pending BOOLEAN NOT NULL,\n" + " used_status_last_updated VARCHAR(255) NOT NULL,\n" + " PRIMARY KEY (fingerprint)\n" + ")", diff --git a/server/src/main/java/org/apache/druid/segment/metadata/CompactionStateStorage.java b/server/src/main/java/org/apache/druid/segment/metadata/CompactionStateStorage.java index fbe8c42577c2..223e8c05be34 100644 --- a/server/src/main/java/org/apache/druid/segment/metadata/CompactionStateStorage.java +++ b/server/src/main/java/org/apache/druid/segment/metadata/CompactionStateStorage.java @@ -79,6 +79,21 @@ void upsertCompactionState( */ int markCompactionStatesAsUsed(List stateFingerprints); + /** + * Marks compaction states as active for a given fingerprint. + * + * @param stateFingerprint The fingerprint to mark as active + * @return Number of rows updated, or 0 if not applicable + */ + int markCompactionStatesAsActive(String stateFingerprint); + + /** + * Deletes pending compaction states older than the given timestamp. + * @param timestamp The cutoff timestamp in milliseconds + * @return Number of rows deleted, or 0 if not applicable + */ + int deletePendingCompactionStatesOlderThan(long timestamp); + /** * Deletes unused compaction states older than the given timestamp. *

    diff --git a/server/src/main/java/org/apache/druid/segment/metadata/SqlCompactionStateStorage.java b/server/src/main/java/org/apache/druid/segment/metadata/SqlCompactionStateStorage.java index 0d3076f9d5b5..050bb06b81dd 100644 --- a/server/src/main/java/org/apache/druid/segment/metadata/SqlCompactionStateStorage.java +++ b/server/src/main/java/org/apache/druid/segment/metadata/SqlCompactionStateStorage.java @@ -21,6 +21,7 @@ import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.annotations.VisibleForTesting; import com.google.inject.Inject; import org.apache.druid.error.DruidException; import org.apache.druid.error.InternalServerError; @@ -37,6 +38,7 @@ import org.skife.jdbi.v2.Update; import javax.annotation.Nonnull; +import javax.annotation.Nullable; import javax.validation.constraints.NotEmpty; import java.util.List; @@ -143,8 +145,8 @@ public void upsertCompactionState( log.info("Inserting new compaction state for fingerprint[%s] in dataSource[%s].", fingerprint, dataSource); 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)", + "INSERT INTO %s (created_date, dataSource, fingerprint, payload, used, pending, used_status_last_updated) " + + "VALUES (:created_date, :dataSource, :fingerprint, :payload, :used, :pending, :used_status_last_updated)", dbTables.getCompactionStatesTable() ); @@ -155,6 +157,7 @@ public void upsertCompactionState( .bind("fingerprint", fingerprint) .bind("payload", jsonMapper.writeValueAsBytes(compactionState)) .bind("used", true) + .bind("pending", true) .bind("used_status_last_updated", now) .execute(); @@ -201,7 +204,7 @@ public int markUnreferencedCompactionStatesAsUnused() handle -> handle.createStatement( StringUtils.format( - "UPDATE %s SET used = false, used_status_last_updated = :now WHERE used = true " + "UPDATE %s SET used = false, used_status_last_updated = :now WHERE used = true AND pending = false " + "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() @@ -238,7 +241,7 @@ public int markCompactionStatesAsUsed(List stateFingerprints) handle -> { Update statement = handle.createStatement( StringUtils.format( - "UPDATE %s SET used = true, used_status_last_updated = :now" + "UPDATE %s SET used = true, pending = false, used_status_last_updated = :now" + " WHERE fingerprint IN (%s)", dbTables.getCompactionStatesTable(), buildParameterizedInClause("fp", stateFingerprints.size()) @@ -252,19 +255,65 @@ public int markCompactionStatesAsUsed(List stateFingerprints) ); } + @Override + public int markCompactionStatesAsActive(String stateFingerprint) + { + return connector.retryWithHandle( + handle -> handle.createStatement( + StringUtils.format( + "UPDATE %s SET pending = false WHERE fingerprint = :fingerprint AND pending = true", + dbTables.getCompactionStatesTable() + )) + .bind("fingerprint", stateFingerprint) + .execute() + ); + } + @Override 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", + "DELETE FROM %s WHERE used = false AND pending = false AND used_status_last_updated < :maxUpdateTime", dbTables.getCompactionStatesTable() )) .bind("maxUpdateTime", DateTimes.utc(timestamp).toString()) .execute()); } + @Override + public int deletePendingCompactionStatesOlderThan(long timestamp) + { + return connector.retryWithHandle( + handle -> handle.createStatement( + StringUtils.format( + "DELETE FROM %s WHERE pending = true AND used_status_last_updated < :maxUpdateTime", + dbTables.getCompactionStatesTable() + )) + .bind("maxUpdateTime", DateTimes.utc(timestamp).toString()) + .execute()); + } + + @Nullable + @VisibleForTesting + public Boolean isCompactionStatePending(final String fingerprint) + { + return connector.retryWithHandle( + handle -> { + String sql = StringUtils.format( + "SELECT pending FROM %s WHERE fingerprint = :fingerprint", + dbTables.getCompactionStatesTable() + ); + + return handle.createQuery(sql) + .bind("fingerprint", fingerprint) + .mapTo(Boolean.class) + .first(); + } + ); + } + /** * Represents the state of an indexing state fingerprint in the database. *

    diff --git a/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorMarkUsedTest.java b/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorMarkUsedTest.java index 2406af6f12b6..ae65f26f5378 100644 --- a/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorMarkUsedTest.java +++ b/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorMarkUsedTest.java @@ -30,6 +30,7 @@ import org.apache.druid.metadata.segment.cache.NoopSegmentMetadataCache; import org.apache.druid.segment.TestDataSource; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; +import org.apache.druid.segment.metadata.HeapMemoryCompactionStateStorage; import org.apache.druid.server.coordinator.CreateDataSegments; import org.apache.druid.server.coordinator.simulate.TestDruidLeaderSelector; import org.apache.druid.server.metrics.NoopServiceEmitter; @@ -95,7 +96,8 @@ public int getMaxRetries() derbyConnectorRule.metadataTablesConfigSupplier().get(), derbyConnector, null, - CentralizedDatasourceSchemaConfig.create() + CentralizedDatasourceSchemaConfig.create(), + new HeapMemoryCompactionStateStorage() ); derbyConnector.createSegmentTable(); diff --git a/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorReadOnlyTest.java b/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorReadOnlyTest.java index b0a9398b4a9c..22d3ab7c9b39 100644 --- a/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorReadOnlyTest.java +++ b/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorReadOnlyTest.java @@ -36,6 +36,7 @@ import org.apache.druid.segment.TestDataSource; import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; +import org.apache.druid.segment.metadata.HeapMemoryCompactionStateStorage; import org.apache.druid.segment.metadata.NoopCompactionStateCache; import org.apache.druid.segment.metadata.NoopSegmentSchemaCache; import org.apache.druid.server.coordinator.simulate.BlockingExecutorService; @@ -180,7 +181,8 @@ private IndexerSQLMetadataStorageCoordinator createStorageCoordinator( derbyConnectorRule.metadataTablesConfigSupplier().get(), derbyConnector, null, - CentralizedDatasourceSchemaConfig.enabled(false) + CentralizedDatasourceSchemaConfig.enabled(false), + new HeapMemoryCompactionStateStorage() ); } diff --git a/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java b/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java index 5368cc005e29..b98c043826a4 100644 --- a/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java +++ b/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java @@ -27,6 +27,7 @@ import org.apache.druid.data.input.StringTuple; import org.apache.druid.error.DruidExceptionMatcher; import org.apache.druid.error.ExceptionMatcher; +import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; import org.apache.druid.indexing.overlord.DataSourceMetadata; import org.apache.druid.indexing.overlord.ObjectMetadata; import org.apache.druid.indexing.overlord.SegmentCreateRequest; @@ -42,20 +43,24 @@ import org.apache.druid.metadata.segment.cache.HeapMemorySegmentMetadataCache; import org.apache.druid.metadata.segment.cache.Metric; import org.apache.druid.metadata.segment.cache.SegmentMetadataCache; +import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.SegmentSchemaMapping; import org.apache.druid.segment.TestDataSource; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; import org.apache.druid.segment.metadata.FingerprintGenerator; +import org.apache.druid.segment.metadata.HeapMemoryCompactionStateStorage; import org.apache.druid.segment.metadata.NoopCompactionStateCache; import org.apache.druid.segment.metadata.NoopSegmentSchemaCache; import org.apache.druid.segment.metadata.SegmentSchemaManager; import org.apache.druid.segment.metadata.SegmentSchemaTestUtils; +import org.apache.druid.segment.metadata.SqlCompactionStateStorage; import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; import org.apache.druid.server.coordinator.CreateDataSegments; import org.apache.druid.server.coordinator.simulate.BlockingExecutorService; import org.apache.druid.server.coordinator.simulate.TestDruidLeaderSelector; import org.apache.druid.server.coordinator.simulate.WrappingScheduledExecutorService; import org.apache.druid.server.http.DataSegmentPlus; +import org.apache.druid.timeline.CompactionState; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.SegmentId; import org.apache.druid.timeline.SegmentTimeline; @@ -113,6 +118,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest extends IndexerSqlMetadata private StubServiceEmitter emitter; private SqlSegmentMetadataTransactionFactory transactionFactory; private BlockingExecutorService cachePollExecutor; + private SqlCompactionStateStorage compactionStateStorage; private final SegmentMetadataCache.UsageMode cacheMode; @@ -142,12 +148,18 @@ public void setUp() derbyConnector.createSegmentTable(); derbyConnector.createUpgradeSegmentsTable(); derbyConnector.createPendingSegmentsTable(); + derbyConnector.createCompactionStatesTable(); metadataUpdateCounter.set(0); segmentTableDropUpdateCounter.set(0); fingerprintGenerator = new FingerprintGenerator(mapper); segmentSchemaManager = new SegmentSchemaManager(derbyConnectorRule.metadataTablesConfigSupplier().get(), mapper, derbyConnector); segmentSchemaTestUtils = new SegmentSchemaTestUtils(derbyConnectorRule, derbyConnector, mapper); + compactionStateStorage = new SqlCompactionStateStorage( + derbyConnectorRule.metadataTablesConfigSupplier().get(), + mapper, + derbyConnector + ); emitter = new StubServiceEmitter(); leaderSelector = new TestDruidLeaderSelector(); @@ -200,7 +212,8 @@ public int getMaxRetries() derbyConnectorRule.metadataTablesConfigSupplier().get(), derbyConnector, segmentSchemaManager, - CentralizedDatasourceSchemaConfig.create() + CentralizedDatasourceSchemaConfig.create(), + compactionStateStorage ) { @Override @@ -796,7 +809,8 @@ public void testTransactionalAnnounceRetryAndSuccess() throws IOException derbyConnectorRule.metadataTablesConfigSupplier().get(), derbyConnector, segmentSchemaManager, - CentralizedDatasourceSchemaConfig.create() + CentralizedDatasourceSchemaConfig.create(), + new HeapMemoryCompactionStateStorage() ) { @Override @@ -955,7 +969,8 @@ public void test_commitSegmentsAndMetadata_isAtomic() derbyConnectorRule.metadataTablesConfigSupplier().get(), derbyConnector, segmentSchemaManager, - CentralizedDatasourceSchemaConfig.create() + CentralizedDatasourceSchemaConfig.create(), + new HeapMemoryCompactionStateStorage() ) { @Override @@ -4317,6 +4332,97 @@ public void testWriteOperation_alwaysUsesCache_inModeIfSynced() emitter.verifyValue(Metric.READ_WRITE_TRANSACTIONS, 1L); } + @Test + public void testCommitSegmentsAndMetadata_marksPendingCompactionStateAsActive() + { + String fingerprint = "vanillaFingerprint"; + CompactionState state = createTestCompactionState(); + compactionStateStorage.upsertCompactionState(TestDataSource.WIKI, fingerprint, state, DateTimes.nowUtc()); + Assert.assertEquals(Boolean.TRUE, compactionStateStorage.isCompactionStatePending(fingerprint)); + + final DataSegment segment = CreateDataSegments.ofDatasource(TestDataSource.WIKI) + .startingAt("2023-01-01") + .withCompactionStateFingerprint(fingerprint) + .eachOfSizeInMb(500) + .get(0); + + coordinator.commitSegmentsAndMetadata( + ImmutableSet.of(segment), + SUPERVISOR_ID, + new ObjectMetadata(null), + new ObjectMetadata(ImmutableMap.of("foo", "bar")), + null + ); + + Assert.assertEquals(Boolean.FALSE, compactionStateStorage.isCompactionStatePending(fingerprint)); + } + + @Test + public void testCommitReplaceSegments_marksPendingCompactionStateAsActive() + { + String fingerprint = "replaceFingerprint"; + CompactionState state = createTestCompactionState(); + compactionStateStorage.upsertCompactionState(TestDataSource.WIKI, fingerprint, state, DateTimes.nowUtc()); + Assert.assertEquals(Boolean.TRUE, compactionStateStorage.isCompactionStatePending(fingerprint)); + + final DataSegment segment = CreateDataSegments.ofDatasource(TestDataSource.WIKI) + .startingAt("2023-01-01") + .withCompactionStateFingerprint(fingerprint) + .eachOfSizeInMb(500) + .get(0); + + final String replaceTaskId = "replaceTask"; + final ReplaceTaskLock replaceLock = new ReplaceTaskLock( + replaceTaskId, + Intervals.of("2023-01-01/2023-01-02"), + "2024-01-01" + ); + + coordinator.commitReplaceSegments( + ImmutableSet.of(segment), + ImmutableSet.of(replaceLock), + null + ); + + Assert.assertEquals(Boolean.FALSE, compactionStateStorage.isCompactionStatePending(fingerprint)); + } + + @Test + public void testCCommitAppendSegments_marksPendingCompactionStateAsActive() + { + String fingerprint = "appendFingerprint"; + CompactionState state = createTestCompactionState(); + compactionStateStorage.upsertCompactionState(TestDataSource.WIKI, fingerprint, state, DateTimes.nowUtc()); + Assert.assertEquals(Boolean.TRUE, compactionStateStorage.isCompactionStatePending(fingerprint)); + + final DataSegment segment = CreateDataSegments.ofDatasource(TestDataSource.WIKI) + .startingAt("2023-01-01") + .withCompactionStateFingerprint(fingerprint) + .eachOfSizeInMb(500) + .get(0); + + final String taskAllocatorId = "appendTask"; + + coordinator.commitAppendSegments( + ImmutableSet.of(segment), + Map.of(), + taskAllocatorId, + null + ); + + Assert.assertEquals(Boolean.FALSE, compactionStateStorage.isCompactionStatePending(fingerprint)); + } + + private CompactionState createTestCompactionState() + { + return new CompactionState( + new DynamicPartitionsSpec(100, null), + null, null, null, + IndexSpec.getDefault(), + null, null + ); + } + private SegmentIdWithShardSpec allocatePendingSegment( String datasource, String sequenceName, diff --git a/server/src/test/java/org/apache/druid/metadata/IndexerSqlMetadataStorageCoordinatorSchemaPersistenceTest.java b/server/src/test/java/org/apache/druid/metadata/IndexerSqlMetadataStorageCoordinatorSchemaPersistenceTest.java index 642f14f9824a..d649c1b8cd1b 100644 --- a/server/src/test/java/org/apache/druid/metadata/IndexerSqlMetadataStorageCoordinatorSchemaPersistenceTest.java +++ b/server/src/test/java/org/apache/druid/metadata/IndexerSqlMetadataStorageCoordinatorSchemaPersistenceTest.java @@ -39,6 +39,7 @@ import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; import org.apache.druid.segment.metadata.FingerprintGenerator; +import org.apache.druid.segment.metadata.HeapMemoryCompactionStateStorage; import org.apache.druid.segment.metadata.SegmentSchemaManager; import org.apache.druid.segment.metadata.SegmentSchemaTestUtils; import org.apache.druid.server.coordinator.simulate.TestDruidLeaderSelector; @@ -108,7 +109,8 @@ public void setUp() derbyConnectorRule.metadataTablesConfigSupplier().get(), derbyConnector, segmentSchemaManager, - centralizedDatasourceSchemaConfig + centralizedDatasourceSchemaConfig, + new HeapMemoryCompactionStateStorage() ) { @Override diff --git a/server/src/test/java/org/apache/druid/segment/metadata/HeapMemoryCompactionStateStorage.java b/server/src/test/java/org/apache/druid/segment/metadata/HeapMemoryCompactionStateStorage.java index de4686114fa4..9cd891927527 100644 --- a/server/src/test/java/org/apache/druid/segment/metadata/HeapMemoryCompactionStateStorage.java +++ b/server/src/test/java/org/apache/druid/segment/metadata/HeapMemoryCompactionStateStorage.java @@ -77,6 +77,18 @@ public int markCompactionStatesAsUsed(List stateFingerprints) return 0; } + @Override + public int markCompactionStatesAsActive(String stateFingerprint) + { + return 0; + } + + @Override + public int deletePendingCompactionStatesOlderThan(long timestamp) + { + return 0; + } + @Override public int deleteUnusedCompactionStatesOlderThan(long timestamp) { diff --git a/server/src/test/java/org/apache/druid/segment/metadata/SqlCompactionStateStorageTest.java b/server/src/test/java/org/apache/druid/segment/metadata/SqlCompactionStateStorageTest.java index 3ea12611cb1b..a1167d8acc76 100644 --- a/server/src/test/java/org/apache/druid/segment/metadata/SqlCompactionStateStorageTest.java +++ b/server/src/test/java/org/apache/druid/segment/metadata/SqlCompactionStateStorageTest.java @@ -199,14 +199,15 @@ public void test_deleteCompactionStatesOlderThan_deletesOnlyOldUnusedStates() 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)" + + "(created_date, dataSource, fingerprint, payload, used, pending, used_status_last_updated) " + + "VALUES (:cd, :ds, :fp, :pl, :used, :pending, :updated)" ) .bind("cd", oldTime.toString()) .bind("ds", "testDatasource") .bind("fp", oldFingerprint) .bind("pl", jsonMapper.writeValueAsBytes(oldState)) .bind("used", false) + .bind("pending", false) .bind("updated", oldTime.toString()) .execute(); return null; @@ -216,14 +217,15 @@ public void test_deleteCompactionStatesOlderThan_deletesOnlyOldUnusedStates() 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)" + + "(created_date, dataSource, fingerprint, payload, used, pending, used_status_last_updated) " + + "VALUES (:cd, :ds, :fp, :pl, :used, :pending, :updated)" ) .bind("cd", recentTime.toString()) .bind("ds", "testDatasource") .bind("fp", recentFingerprint) .bind("pl", jsonMapper.writeValueAsBytes(recentState)) .bind("used", false) + .bind("pending", false) .bind("updated", recentTime.toString()) .execute(); return null; @@ -397,6 +399,69 @@ public void test_markCompactionStateAsUsed_withEmptyList_returnsZero() assertEquals(0, manager.markCompactionStatesAsUsed(List.of())); } + @Test + public void test_markCompactionStatesAsActive_marksPendingStateAsActive() + { + String fingerprint = "pending_fingerprint"; + CompactionState state = createTestCompactionState(); + + derbyConnector.retryWithHandle(handle -> { + manager.upsertCompactionState("ds1", fingerprint, state, DateTimes.nowUtc()); + return null; + }); + + Boolean pendingBefore = derbyConnector.retryWithHandle(handle -> + handle.createQuery("SELECT pending FROM " + tablesConfig.getCompactionStatesTable() + " WHERE fingerprint = :fp") + .bind("fp", fingerprint) + .map((i, r, ctx) -> r.getBoolean("pending")) + .first() + ); + assertTrue(pendingBefore); + + int rowsUpdated = manager.markCompactionStatesAsActive(fingerprint); + assertEquals(1, rowsUpdated); + + Boolean pendingAfter = derbyConnector.retryWithHandle(handle -> + handle.createQuery("SELECT pending FROM " + tablesConfig.getCompactionStatesTable() + " WHERE fingerprint = :fp") + .bind("fp", fingerprint) + .map((i, r, ctx) -> r.getBoolean("pending")) + .first() + ); + assertEquals(false, pendingAfter); + } + + @Test + public void test_markCompactionStatesAsActive_idempotent_returnsZeroWhenAlreadyActive() + { + String fingerprint = "already_active_fingerprint"; + CompactionState state = createTestCompactionState(); + + derbyConnector.retryWithHandle(handle -> { + manager.upsertCompactionState("ds1", fingerprint, state, DateTimes.nowUtc()); + return null; + }); + int firstUpdate = manager.markCompactionStatesAsActive(fingerprint); + assertEquals(1, firstUpdate); + + int secondUpdate = manager.markCompactionStatesAsActive(fingerprint); + assertEquals(0, secondUpdate); + + Boolean pending = derbyConnector.retryWithHandle(handle -> + handle.createQuery("SELECT pending FROM " + tablesConfig.getCompactionStatesTable() + " WHERE fingerprint = :fp") + .bind("fp", fingerprint) + .map((i, r, ctx) -> r.getBoolean("pending")) + .first() + ); + assertEquals(false, pending); + } + + @Test + public void test_markCompactionStatesAsActive_nonExistentFingerprint_returnsZero() + { + int rowsUpdated = manager.markCompactionStatesAsActive("does_not_exist"); + assertEquals(0, rowsUpdated); + } + // ===== Fingerprint Generation Tests ===== @Test 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 95df260de8fb..a9714a080267 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 @@ -30,7 +30,6 @@ import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; import org.apache.druid.indexer.partitions.HashedPartitionsSpec; import org.apache.druid.indexer.partitions.PartitionsSpec; -import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.granularity.Granularities; diff --git a/server/src/test/java/org/apache/druid/server/coordinator/duty/KillUnusedSegmentsTest.java b/server/src/test/java/org/apache/druid/server/coordinator/duty/KillUnusedSegmentsTest.java index f8ea67308385..64587fe8af3b 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/duty/KillUnusedSegmentsTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/duty/KillUnusedSegmentsTest.java @@ -44,6 +44,7 @@ import org.apache.druid.rpc.indexing.NoopOverlordClient; import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; +import org.apache.druid.segment.metadata.HeapMemoryCompactionStateStorage; import org.apache.druid.server.coordinator.CoordinatorDynamicConfig; import org.apache.druid.server.coordinator.DruidCoordinatorRuntimeParams; import org.apache.druid.server.coordinator.config.KillUnusedSegmentsConfig; @@ -117,7 +118,8 @@ public void setup() derbyConnectorRule.metadataTablesConfigSupplier().get(), connector, null, - CentralizedDatasourceSchemaConfig.create() + CentralizedDatasourceSchemaConfig.create(), + new HeapMemoryCompactionStateStorage() ); this.config = derbyConnectorRule.metadataTablesConfigSupplier().get(); 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 49a19e35a1a7..205de59a27ff 100644 --- a/services/src/main/java/org/apache/druid/guice/MetadataManagerModule.java +++ b/services/src/main/java/org/apache/druid/guice/MetadataManagerModule.java @@ -106,6 +106,9 @@ public void configure(Binder binder) binder.bind(SegmentMetadataCache.class) .to(HeapMemorySegmentMetadataCache.class) .in(LazySingleton.class); + binder.bind(CompactionStateStorage.class) + .to(SqlCompactionStateStorage.class) + .in(ManageLifecycle.class); // Coordinator-only dependencies if (nodeRoles.contains(NodeRole.COORDINATOR)) { @@ -134,9 +137,6 @@ public void configure(Binder binder) .to(SqlSegmentMetadataTransactionFactory.class) .in(LazySingleton.class); binder.bind(CompactionStateCache.class).in(LazySingleton.class); - binder.bind(CompactionStateStorage.class) - .to(SqlCompactionStateStorage.class) - .in(ManageLifecycle.class); } else { binder.bind(SegmentMetadataTransactionFactory.class) .to(SqlSegmentMetadataReadOnlyTransactionFactory.class) From 79ac0c5df82c567bfa43b02605fda2557c448afd Mon Sep 17 00:00:00 2001 From: capistrant Date: Tue, 13 Jan 2026 16:28:51 -0600 Subject: [PATCH 59/72] Refactor database naming to use indexingState instead of compactionState --- .../common/actions/TaskActionTestKit.java | 2 +- .../common/task/IngestionTestBase.java | 2 +- .../KillUnreferencedCompactionStateTest.java | 16 ++++---- .../metadata/MetadataStorageConnector.java | 2 +- .../metadata/MetadataStorageTablesConfig.java | 12 +++--- .../TestMetadataStorageConnector.java | 2 +- .../druid/metadata/SQLMetadataConnector.java | 18 ++++----- .../SqlSegmentsMetadataManagerProvider.java | 2 +- .../metadata/SqlSegmentsMetadataQuery.java | 10 ++--- .../SqlSegmentMetadataTransaction.java | 10 ++--- .../cache/HeapMemorySegmentMetadataCache.java | 4 +- .../metadata/SqlCompactionStateStorage.java | 24 ++++++------ ...exerSQLMetadataStorageCoordinatorTest.java | 2 +- .../metadata/SQLMetadataConnectorTest.java | 4 +- .../SqlSegmentsMetadataQueryTest.java | 34 ++++++++--------- .../SqlSegmentsMetadataManagerV2Test.java | 2 +- .../HeapMemorySegmentMetadataCacheTest.java | 2 +- .../SqlCompactionStateStorageTest.java | 38 +++++++++---------- 18 files changed, 93 insertions(+), 93 deletions(-) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/TaskActionTestKit.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/TaskActionTestKit.java index 5a1a2093270e..256d2ed2de3b 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/TaskActionTestKit.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/TaskActionTestKit.java @@ -179,7 +179,7 @@ public boolean isBatchAllocationReduceMetadataIO() testDerbyConnector.createConfigTable(); testDerbyConnector.createTaskTables(); testDerbyConnector.createAuditTable(); - testDerbyConnector.createCompactionStatesTable(); + testDerbyConnector.createIndexingStatesTable(); segmentMetadataCache.start(); segmentMetadataCache.becomeLeader(); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IngestionTestBase.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IngestionTestBase.java index 59933d9d67db..ad6ec2bd4019 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IngestionTestBase.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IngestionTestBase.java @@ -167,7 +167,7 @@ public void setUpIngestionTestBase() throws IOException connector.createSegmentSchemasTable(); connector.createSegmentTable(); connector.createPendingSegmentsTable(); - connector.createCompactionStatesTable(); + connector.createIndexingStatesTable(); taskStorage = new HeapMemoryTaskStorage(new TaskStorageConfig(null)); SegmentSchemaManager segmentSchemaManager = new SegmentSchemaManager( derbyConnectorRule.metadataTablesConfigSupplier().get(), diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/duty/KillUnreferencedCompactionStateTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/duty/KillUnreferencedCompactionStateTest.java index df145d4b4945..358ab355f48f 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/duty/KillUnreferencedCompactionStateTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/duty/KillUnreferencedCompactionStateTest.java @@ -58,7 +58,7 @@ public void setUp() derbyConnector = derbyConnectorRule.getConnector(); tablesConfig = derbyConnectorRule.metadataTablesConfigSupplier().get(); - derbyConnector.createCompactionStatesTable(); + derbyConnector.createIndexingStatesTable(); derbyConnector.createSegmentTable(); compactionStateStorage = new SqlCompactionStateStorage(tablesConfig, jsonMapper, derbyConnector); @@ -139,9 +139,9 @@ public void test_killUnreferencedCompactionState_validateRepair() handle.createStatement( "INSERT INTO " + tablesConfig.getSegmentsTable() + " " + "(id, dataSource, created_date, start, \"end\", partitioned, version, used, payload, " - + "used_status_last_updated, compaction_state_fingerprint) " + + "used_status_last_updated, indexing_state_fingerprint) " + "VALUES (:id, :dataSource, :created_date, :start, :end, :partitioned, :version, :used, :payload, " - + ":used_status_last_updated, :compaction_state_fingerprint)" + + ":used_status_last_updated, :indexing_state_fingerprint)" ) .bind("id", "testSegment_2024-01-01_2024-01-02_v1_0") .bind("dataSource", "test-ds") @@ -153,7 +153,7 @@ public void test_killUnreferencedCompactionState_validateRepair() .bind("used", true) .bind("payload", new byte[]{}) .bind("used_status_last_updated", DateTimes.nowUtc().toString()) - .bind("compaction_state_fingerprint", fingerprint) + .bind("indexing_state_fingerprint", fingerprint) .execute(); return null; }); @@ -288,9 +288,9 @@ public void test_killUnreferencedCompactionState_pendingStateMarkedActiveNotDele handle.createStatement( "INSERT INTO " + tablesConfig.getSegmentsTable() + " " + "(id, dataSource, created_date, start, \"end\", partitioned, version, used, payload, " - + "used_status_last_updated, compaction_state_fingerprint) " + + "used_status_last_updated, indexing_state_fingerprint) " + "VALUES (:id, :dataSource, :created_date, :start, :end, :partitioned, :version, :used, :payload, " - + ":used_status_last_updated, :compaction_state_fingerprint)" + + ":used_status_last_updated, :indexing_state_fingerprint)" ) .bind("id", "testSegment_2024-01-01_2024-01-02_v1_0") .bind("dataSource", "test-ds") @@ -302,7 +302,7 @@ public void test_killUnreferencedCompactionState_pendingStateMarkedActiveNotDele .bind("used", true) .bind("payload", new byte[]{}) .bind("used_status_last_updated", DateTimes.nowUtc().toString()) - .bind("compaction_state_fingerprint", fingerprint) + .bind("indexing_state_fingerprint", fingerprint) .execute(); return null; }); @@ -318,7 +318,7 @@ private Boolean getCompactionStateUsedStatus(String fingerprint) { List usedStatus = derbyConnector.retryWithHandle( handle -> handle.createQuery( - "SELECT used FROM " + tablesConfig.getCompactionStatesTable() + "SELECT used FROM " + tablesConfig.getIndexingStatesTable() + " WHERE fingerprint = :fp" ) .bind("fp", fingerprint) 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 c41dee70f0ba..f40b36efa171 100644 --- a/processing/src/main/java/org/apache/druid/metadata/MetadataStorageConnector.java +++ b/processing/src/main/java/org/apache/druid/metadata/MetadataStorageConnector.java @@ -102,5 +102,5 @@ default void exportTable( * Multiple segments can refer to the same compaction state via its unique fingerprint *

    */ - void createCompactionStatesTable(); + void createIndexingStatesTable(); } 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 0303da10a610..819b876154e0 100644 --- a/processing/src/main/java/org/apache/druid/metadata/MetadataStorageTablesConfig.java +++ b/processing/src/main/java/org/apache/druid/metadata/MetadataStorageTablesConfig.java @@ -76,8 +76,8 @@ public static MetadataStorageTablesConfig fromBase(String base) @JsonProperty("useShortIndexNames") private final boolean useShortIndexNames; - @JsonProperty("compactionStates") - private final String compactionStatesTable; + @JsonProperty("indexingStates") + private final String indexingStatesTable; @JsonCreator public MetadataStorageTablesConfig( @@ -94,7 +94,7 @@ public MetadataStorageTablesConfig( @JsonProperty("upgradeSegments") String upgradeSegmentsTable, @JsonProperty("segmentSchemas") String segmentSchemasTable, @JsonProperty("useShortIndexNames") Boolean useShortIndexNames, - @JsonProperty("compactionStates") String compactionStatesTable + @JsonProperty("indexingStates") String indexingStatesTable ) { this.base = (base == null) ? DEFAULT_BASE : base; @@ -111,7 +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"); + this.indexingStatesTable = makeTableName(indexingStatesTable, "indexingStates"); } private String makeTableName(String explicitTableName, String defaultSuffix) @@ -186,9 +186,9 @@ public String getSegmentSchemasTable() return segmentSchemasTable; } - public String getCompactionStatesTable() + public String getIndexingStatesTable() { - return compactionStatesTable; + return indexingStatesTable; } /** 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 1251c4fcf6f7..28291821e086 100644 --- a/processing/src/test/java/org/apache/druid/metadata/TestMetadataStorageConnector.java +++ b/processing/src/test/java/org/apache/druid/metadata/TestMetadataStorageConnector.java @@ -103,7 +103,7 @@ public void createSegmentSchemasTable() } @Override - public void createCompactionStatesTable() + public void createIndexingStatesTable() { throw new UnsupportedOperationException(); } 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 c8106cf99219..d1321f5c9779 100644 --- a/server/src/main/java/org/apache/druid/metadata/SQLMetadataConnector.java +++ b/server/src/main/java/org/apache/druid/metadata/SQLMetadataConnector.java @@ -366,7 +366,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)"); + columns.add("indexing_state_fingerprint VARCHAR(255)"); columns.add("upgraded_from_segment_id VARCHAR(255)"); if (centralizedDatasourceSchemaConfig.isEnabled()) { @@ -627,7 +627,7 @@ protected void alterSegmentTable() columnNameTypes.put("upgraded_from_segment_id", "VARCHAR(255)"); - columnNameTypes.put("compaction_state_fingerprint", "VARCHAR(255)"); + columnNameTypes.put("indexing_state_fingerprint", "VARCHAR(255)"); if (centralizedDatasourceSchemaConfig.isEnabled()) { columnNameTypes.put("schema_fingerprint", "VARCHAR(255)"); @@ -1113,12 +1113,12 @@ public void createSegmentSchemasTable() } /** - * Creates the compaction states table for storing fingerprinted compaction states + * Creates the indexing states table for storing fingerprinted indexing states *

    - * This table stores unique compaction states that are referenced by + * This table stores unique indexing states that are referenced by * segments via fingerprints. */ - public void createCompactionStatesTable(final String tableName) + public void createIndexingStatesTable(final String tableName) { createTable( tableName, @@ -1147,10 +1147,10 @@ tableName, getPayloadType() } @Override - public void createCompactionStatesTable() + public void createIndexingStatesTable() { if (config.get().isCreateTables()) { - createCompactionStatesTable(tablesConfigSupplier.get().getCompactionStatesTable()); + createIndexingStatesTable(tablesConfigSupplier.get().getIndexingStatesTable()); } } @@ -1300,12 +1300,12 @@ private void validateSegmentsTable() (tableHasColumn(segmentsTables, "schema_fingerprint") && tableHasColumn(segmentsTables, "num_rows")); - if (tableHasColumn(segmentsTables, "used_status_last_updated") && schemaPersistenceRequirementMet && tableHasColumn(segmentsTables, "compaction_state_fingerprint")) { + if (tableHasColumn(segmentsTables, "used_status_last_updated") && schemaPersistenceRequirementMet && tableHasColumn(segmentsTables, "indexing_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, compaction_state_fingerprint] does not exist in table." + + " Reason: One or all of these columns [used_status_last_updated, schema_fingerprint, num_rows, indexing_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 9a632d7909b8..f3542cdc6d0f 100644 --- a/server/src/main/java/org/apache/druid/metadata/SqlSegmentsMetadataManagerProvider.java +++ b/server/src/main/java/org/apache/druid/metadata/SqlSegmentsMetadataManagerProvider.java @@ -77,7 +77,7 @@ public void start() connector.createSegmentSchemasTable(); connector.createSegmentTable(); connector.createUpgradeSegmentsTable(); - connector.createCompactionStatesTable(); + connector.createIndexingStatesTable(); } @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 b2ad9ae531e4..ed3159c7fbbd 100644 --- a/server/src/main/java/org/apache/druid/metadata/SqlSegmentsMetadataQuery.java +++ b/server/src/main/java/org/apache/druid/metadata/SqlSegmentsMetadataQuery.java @@ -587,7 +587,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, compaction_state_fingerprint" + + " upgraded_from_segment_id, used_status_last_updated, indexing_state_fingerprint" + " FROM %s WHERE dataSource = :dataSource %s", dbTables.getSegmentsTable(), getParameterizedInConditionForColumn("id", segmentIds) ) @@ -618,7 +618,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, compaction_state_fingerprint" + "SELECT payload, used, upgraded_from_segment_id, used_status_last_updated, created_date, indexing_state_fingerprint" + " FROM %s WHERE dataSource = :dataSource %s", dbTables.getSegmentsTable(), getParameterizedInConditionForColumn("id", segmentIds) ) @@ -1715,7 +1715,7 @@ public Set retrieveAllUsedCompactionStateFingerprints() { final String sql = StringUtils.format( "SELECT fingerprint FROM %s WHERE used = true", - dbTables.getCompactionStatesTable() + dbTables.getIndexingStatesTable() ); return Set.copyOf( @@ -1735,7 +1735,7 @@ public List retrieveAllUsedCompactionStates() { final String sql = StringUtils.format( "SELECT fingerprint, payload FROM %s WHERE used = true", - dbTables.getCompactionStatesTable() + dbTables.getIndexingStatesTable() ); return retrieveValidCompactionStateRecordsWithQuery(handle.createQuery(sql)); @@ -1776,7 +1776,7 @@ private List retrieveBatchOfCompactionStates(List "SELECT fingerprint, payload FROM %s" + " WHERE used = true" + " %s", - dbTables.getCompactionStatesTable(), + dbTables.getIndexingStatesTable(), getParameterizedInConditionForColumn("fingerprint", fingerprints) ); 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 ecfa21cc3835..bb06c0a0bf12 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, compaction_state_fingerprint) " + + "version, used, payload, used_status_last_updated, upgraded_from_segment_id, indexing_state_fingerprint) " + "VALUES " + "(:id, :dataSource, :created_date, :start, :end, :partitioned, " - + ":version, :used, :payload, :used_status_last_updated, :upgraded_from_segment_id, :compaction_state_fingerprint)" + + ":version, :used, :payload, :used_status_last_updated, :upgraded_from_segment_id, :indexing_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, compaction_state_fingerprint) " + + "schema_fingerprint, num_rows, indexing_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, :compaction_state_fingerprint)" + + ":schema_fingerprint, :num_rows, :indexing_state_fingerprint)" ); } @@ -533,7 +533,7 @@ private int insertSegmentsInBatches( .bind("payload", getJsonBytes(segment)) .bind("used_status_last_updated", toNonNullString(segmentPlus.getUsedStatusLastUpdatedDate())) .bind("upgraded_from_segment_id", segmentPlus.getUpgradedFromSegmentId()) - .bind("compaction_state_fingerprint", segmentPlus.getCompactionStateFingerprint()); + .bind("indexing_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 862e33c5e554..a260323265ff 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 @@ -800,13 +800,13 @@ private void retrieveAllUsedSegments( final String sql; if (useSchemaCache) { sql = StringUtils.format( - "SELECT id, payload, created_date, used_status_last_updated, compaction_state_fingerprint, schema_fingerprint, num_rows" + "SELECT id, payload, created_date, used_status_last_updated, indexing_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, compaction_state_fingerprint" + "SELECT id, payload, created_date, used_status_last_updated, indexing_state_fingerprint" + " FROM %s WHERE used = true", tablesConfig.getSegmentsTable() ); diff --git a/server/src/main/java/org/apache/druid/segment/metadata/SqlCompactionStateStorage.java b/server/src/main/java/org/apache/druid/segment/metadata/SqlCompactionStateStorage.java index 050bb06b81dd..15daa666181a 100644 --- a/server/src/main/java/org/apache/druid/segment/metadata/SqlCompactionStateStorage.java +++ b/server/src/main/java/org/apache/druid/segment/metadata/SqlCompactionStateStorage.java @@ -129,7 +129,7 @@ public void upsertCompactionState( String updateSql = StringUtils.format( "UPDATE %s SET used = :used, used_status_last_updated = :used_status_last_updated " + "WHERE fingerprint = :fingerprint", - dbTables.getCompactionStatesTable() + dbTables.getIndexingStatesTable() ); handle.createStatement(updateSql) .bind("used", true) @@ -147,7 +147,7 @@ public void upsertCompactionState( String insertSql = StringUtils.format( "INSERT INTO %s (created_date, dataSource, fingerprint, payload, used, pending, used_status_last_updated) " + "VALUES (:created_date, :dataSource, :fingerprint, :payload, :used, :pending, :used_status_last_updated)", - dbTables.getCompactionStatesTable() + dbTables.getIndexingStatesTable() ); try { @@ -205,8 +205,8 @@ public int markUnreferencedCompactionStatesAsUnused() handle.createStatement( StringUtils.format( "UPDATE %s SET used = false, used_status_last_updated = :now WHERE used = true AND pending = false " - + "AND fingerprint NOT IN (SELECT DISTINCT compaction_state_fingerprint FROM %s WHERE used = true AND compaction_state_fingerprint IS NOT NULL)", - dbTables.getCompactionStatesTable(), + + "AND fingerprint NOT IN (SELECT DISTINCT indexing_state_fingerprint FROM %s WHERE used = true AND indexing_state_fingerprint IS NOT NULL)", + dbTables.getIndexingStatesTable(), dbTables.getSegmentsTable() ) ) @@ -221,9 +221,9 @@ public List findReferencedCompactionStateMarkedAsUnused() 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)", + "SELECT DISTINCT indexing_state_fingerprint FROM %s WHERE used = true AND indexing_state_fingerprint IN (SELECT fingerprint FROM %s WHERE used = false)", dbTables.getSegmentsTable(), - dbTables.getCompactionStatesTable() + dbTables.getIndexingStatesTable() )) .mapTo(String.class) .list() @@ -243,7 +243,7 @@ public int markCompactionStatesAsUsed(List stateFingerprints) StringUtils.format( "UPDATE %s SET used = true, pending = false, used_status_last_updated = :now" + " WHERE fingerprint IN (%s)", - dbTables.getCompactionStatesTable(), + dbTables.getIndexingStatesTable(), buildParameterizedInClause("fp", stateFingerprints.size()) ) ).bind("now", DateTimes.nowUtc().toString()); @@ -262,7 +262,7 @@ public int markCompactionStatesAsActive(String stateFingerprint) handle -> handle.createStatement( StringUtils.format( "UPDATE %s SET pending = false WHERE fingerprint = :fingerprint AND pending = true", - dbTables.getCompactionStatesTable() + dbTables.getIndexingStatesTable() )) .bind("fingerprint", stateFingerprint) .execute() @@ -276,7 +276,7 @@ public int deleteUnusedCompactionStatesOlderThan(long timestamp) handle -> handle.createStatement( StringUtils.format( "DELETE FROM %s WHERE used = false AND pending = false AND used_status_last_updated < :maxUpdateTime", - dbTables.getCompactionStatesTable() + dbTables.getIndexingStatesTable() )) .bind("maxUpdateTime", DateTimes.utc(timestamp).toString()) .execute()); @@ -289,7 +289,7 @@ public int deletePendingCompactionStatesOlderThan(long timestamp) handle -> handle.createStatement( StringUtils.format( "DELETE FROM %s WHERE pending = true AND used_status_last_updated < :maxUpdateTime", - dbTables.getCompactionStatesTable() + dbTables.getIndexingStatesTable() )) .bind("maxUpdateTime", DateTimes.utc(timestamp).toString()) .execute()); @@ -303,7 +303,7 @@ public Boolean isCompactionStatePending(final String fingerprint) handle -> { String sql = StringUtils.format( "SELECT pending FROM %s WHERE fingerprint = :fingerprint", - dbTables.getCompactionStatesTable() + dbTables.getIndexingStatesTable() ); return handle.createQuery(sql) @@ -347,7 +347,7 @@ private FingerprintState getFingerprintState( String sql = StringUtils.format( "SELECT used FROM %s WHERE fingerprint = :fingerprint", - dbTables.getCompactionStatesTable() + dbTables.getIndexingStatesTable() ); Boolean used = handle.createQuery(sql) diff --git a/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java b/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java index b98c043826a4..27bba0c6821f 100644 --- a/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java +++ b/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java @@ -148,7 +148,7 @@ public void setUp() derbyConnector.createSegmentTable(); derbyConnector.createUpgradeSegmentsTable(); derbyConnector.createPendingSegmentsTable(); - derbyConnector.createCompactionStatesTable(); + derbyConnector.createIndexingStatesTable(); metadataUpdateCounter.set(0); segmentTableDropUpdateCounter.set(0); 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 523d7220e72d..6dd0a724b035 100644 --- a/server/src/test/java/org/apache/druid/metadata/SQLMetadataConnectorTest.java +++ b/server/src/test/java/org/apache/druid/metadata/SQLMetadataConnectorTest.java @@ -76,7 +76,7 @@ public void testCreateTables() tables.add(tablesConfig.getTasksTable()); tables.add(tablesConfig.getAuditTable()); tables.add(tablesConfig.getSupervisorTable()); - tables.add(tablesConfig.getCompactionStatesTable()); + tables.add(tablesConfig.getIndexingStatesTable()); connector.createSegmentTable(); connector.createConfigTable(); @@ -84,7 +84,7 @@ public void testCreateTables() connector.createTaskTables(); connector.createAuditTable(); connector.createSupervisorsTable(); - connector.createCompactionStatesTable(); + connector.createIndexingStatesTable(); connector.getDBI().withHandle( handle -> { diff --git a/server/src/test/java/org/apache/druid/metadata/SqlSegmentsMetadataQueryTest.java b/server/src/test/java/org/apache/druid/metadata/SqlSegmentsMetadataQueryTest.java index f865bc337e8d..bbcbc6273b22 100644 --- a/server/src/test/java/org/apache/druid/metadata/SqlSegmentsMetadataQueryTest.java +++ b/server/src/test/java/org/apache/druid/metadata/SqlSegmentsMetadataQueryTest.java @@ -380,7 +380,7 @@ private static Set getIds(Set segments) @Test public void test_retrieveAllUsedCompactionStateFingerprints_emptyDatabase() { - derbyConnectorRule.getConnector().createCompactionStatesTable(); + derbyConnectorRule.getConnector().createIndexingStatesTable(); Set fingerprints = read(SqlSegmentsMetadataQuery::retrieveAllUsedCompactionStateFingerprints); @@ -390,7 +390,7 @@ public void test_retrieveAllUsedCompactionStateFingerprints_emptyDatabase() @Test public void test_retrieveAllUsedCompactionStateFingerprints() { - derbyConnectorRule.getConnector().createCompactionStatesTable(); + derbyConnectorRule.getConnector().createIndexingStatesTable(); // Insert compaction states Map compactionStates = new HashMap<>(); @@ -413,7 +413,7 @@ public void test_retrieveAllUsedCompactionStateFingerprints() @Test public void test_retrieveAllUsedCompactionStateFingerprints_ignoresNullFingerprints() { - derbyConnectorRule.getConnector().createCompactionStatesTable(); + derbyConnectorRule.getConnector().createIndexingStatesTable(); Map compactionStates = new HashMap<>(); compactionStates.put("fp1", createTestCompactionState()); @@ -430,7 +430,7 @@ public void test_retrieveAllUsedCompactionStateFingerprints_ignoresNullFingerpri @Test public void test_retrieveAllUsedCompactionStates_emptyDatabase() { - derbyConnectorRule.getConnector().createCompactionStatesTable(); + derbyConnectorRule.getConnector().createIndexingStatesTable(); List records = read(SqlSegmentsMetadataQuery::retrieveAllUsedCompactionStates); @@ -440,7 +440,7 @@ public void test_retrieveAllUsedCompactionStates_emptyDatabase() @Test public void test_retrieveAllUsedCompactionStates_fullSync() { - derbyConnectorRule.getConnector().createCompactionStatesTable(); + derbyConnectorRule.getConnector().createIndexingStatesTable(); // Create distinct compaction states CompactionState state1 = createTestCompactionState(); @@ -488,7 +488,7 @@ public void test_retrieveAllUsedCompactionStates_fullSync() @Test public void test_retrieveAllUsedCompactionStates_onlyFromUsedSegments() { - derbyConnectorRule.getConnector().createCompactionStatesTable(); + derbyConnectorRule.getConnector().createIndexingStatesTable(); Map compactionStates = new HashMap<>(); compactionStates.put("fp1", createTestCompactionState()); @@ -506,7 +506,7 @@ public void test_retrieveAllUsedCompactionStates_onlyFromUsedSegments() @Test public void test_retrieveAllUsedCompactionStates_ignoresUnusedCompactionStates() { - derbyConnectorRule.getConnector().createCompactionStatesTable(); + derbyConnectorRule.getConnector().createIndexingStatesTable(); Map compactionStates = new HashMap<>(); compactionStates.put("fp1", createTestCompactionState()); @@ -525,7 +525,7 @@ public void test_retrieveAllUsedCompactionStates_ignoresUnusedCompactionStates() @Test public void test_retrieveCompactionStatesForFingerprints_emptyInput() { - derbyConnectorRule.getConnector().createCompactionStatesTable(); + derbyConnectorRule.getConnector().createIndexingStatesTable(); List records = read( sql -> sql.retrieveCompactionStatesForFingerprints(Set.of()) @@ -537,7 +537,7 @@ public void test_retrieveCompactionStatesForFingerprints_emptyInput() @Test public void test_retrieveCompactionStatesForFingerprints_deltaSync() { - derbyConnectorRule.getConnector().createCompactionStatesTable(); + derbyConnectorRule.getConnector().createIndexingStatesTable(); // Insert multiple compaction states Map compactionStates = new HashMap<>(); @@ -562,7 +562,7 @@ public void test_retrieveCompactionStatesForFingerprints_deltaSync() @Test public void test_retrieveCompactionStatesForFingerprints_largeBatch() { - derbyConnectorRule.getConnector().createCompactionStatesTable(); + derbyConnectorRule.getConnector().createIndexingStatesTable(); // Insert 150 compaction states (exceeds batching threshold of 100) Map compactionStates = new HashMap<>(); @@ -590,7 +590,7 @@ public void test_retrieveCompactionStatesForFingerprints_largeBatch() @Test public void test_retrieveCompactionStatesForFingerprints_nonexistentFingerprints() { - derbyConnectorRule.getConnector().createCompactionStatesTable(); + derbyConnectorRule.getConnector().createIndexingStatesTable(); Map compactionStates = new HashMap<>(); compactionStates.put("fp1", createTestCompactionState()); @@ -607,7 +607,7 @@ public void test_retrieveCompactionStatesForFingerprints_nonexistentFingerprints @Test public void test_retrieveCompactionStatesForFingerprints_mixedExistingAndNonexistent() { - derbyConnectorRule.getConnector().createCompactionStatesTable(); + derbyConnectorRule.getConnector().createIndexingStatesTable(); Map compactionStates = new HashMap<>(); compactionStates.put("fp1", createTestCompactionState()); @@ -630,7 +630,7 @@ public void test_retrieveCompactionStatesForFingerprints_mixedExistingAndNonexis @Test public void test_retrieveCompactionStatesForFingerprints_onlyReturnsUsedStates() { - derbyConnectorRule.getConnector().createCompactionStatesTable(); + derbyConnectorRule.getConnector().createIndexingStatesTable(); Map compactionStates = new HashMap<>(); compactionStates.put("fp1", createTestCompactionState()); @@ -694,9 +694,9 @@ private void insertSegmentWithCompactionState( handle.createStatement( "INSERT INTO " + tablesConfig.getSegmentsTable() + " " + "(id, dataSource, created_date, start, \"end\", partitioned, version, used, payload, " - + "used_status_last_updated, compaction_state_fingerprint) " + + "used_status_last_updated, indexing_state_fingerprint) " + "VALUES (:id, :dataSource, :created_date, :start, :end, :partitioned, :version, :used, :payload, " - + ":used_status_last_updated, :compaction_state_fingerprint)" + + ":used_status_last_updated, :indexing_state_fingerprint)" ) .bind("id", segmentId) .bind("dataSource", TestDataSource.WIKI) @@ -708,7 +708,7 @@ private void insertSegmentWithCompactionState( .bind("used", used) .bind("payload", TestHelper.JSON_MAPPER.writeValueAsBytes(WIKI_SEGMENTS_2X5D.get(0))) .bind("used_status_last_updated", DateTimes.nowUtc().toString()) - .bind("compaction_state_fingerprint", compactionStateFingerprint) + .bind("indexing_state_fingerprint", compactionStateFingerprint) .execute(); return null; }); @@ -721,7 +721,7 @@ private void markCompactionStateAsUnused(String fingerprint) connector.retryWithHandle(handle -> { handle.createStatement( - "UPDATE " + tablesConfig.getCompactionStatesTable() + " " + "UPDATE " + tablesConfig.getIndexingStatesTable() + " " + "SET used = false " + "WHERE fingerprint = :fingerprint" ) diff --git a/server/src/test/java/org/apache/druid/metadata/segment/SqlSegmentsMetadataManagerV2Test.java b/server/src/test/java/org/apache/druid/metadata/segment/SqlSegmentsMetadataManagerV2Test.java index a54831ec957e..8ccaf03b4254 100644 --- a/server/src/test/java/org/apache/druid/metadata/segment/SqlSegmentsMetadataManagerV2Test.java +++ b/server/src/test/java/org/apache/druid/metadata/segment/SqlSegmentsMetadataManagerV2Test.java @@ -75,7 +75,7 @@ public void setup() throws Exception { setUp(derbyConnectorRule); connector.createPendingSegmentsTable(); - connector.createCompactionStatesTable(); + connector.createIndexingStatesTable(); emitter = new StubServiceEmitter(); 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 deb22e45fd7b..ec837c9925d3 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 @@ -91,7 +91,7 @@ public void setup() derbyConnector.createSegmentTable(); derbyConnector.createSegmentSchemasTable(); derbyConnector.createPendingSegmentsTable(); - derbyConnector.createCompactionStatesTable(); + derbyConnector.createIndexingStatesTable(); schemaTestUtils = new SegmentSchemaTestUtils(derbyConnectorRule, derbyConnector, TestHelper.JSON_MAPPER); EmittingLogger.registerEmitter(serviceEmitter); diff --git a/server/src/test/java/org/apache/druid/segment/metadata/SqlCompactionStateStorageTest.java b/server/src/test/java/org/apache/druid/segment/metadata/SqlCompactionStateStorageTest.java index a1167d8acc76..8e440d7a14dd 100644 --- a/server/src/test/java/org/apache/druid/segment/metadata/SqlCompactionStateStorageTest.java +++ b/server/src/test/java/org/apache/druid/segment/metadata/SqlCompactionStateStorageTest.java @@ -68,7 +68,7 @@ public static void setUpClass() { derbyConnector = DERBY_CONNECTOR_RULE.getConnector(); tablesConfig = DERBY_CONNECTOR_RULE.metadataTablesConfigSupplier().get(); - derbyConnector.createCompactionStatesTable(); + derbyConnector.createIndexingStatesTable(); derbyConnector.createSegmentTable(); fingerprintMapper = new DefaultCompactionFingerprintMapper( new NoopCompactionStateCache(), @@ -81,7 +81,7 @@ public void setUp() { derbyConnector.retryWithHandle(handle -> { handle.createStatement("DELETE FROM " + tablesConfig.getSegmentsTable()).execute(); - handle.createStatement("DELETE FROM " + tablesConfig.getCompactionStatesTable()).execute(); + handle.createStatement("DELETE FROM " + tablesConfig.getIndexingStatesTable()).execute(); return null; }); @@ -107,7 +107,7 @@ public void test_upsertCompactionState_successfullyInsertsIntoDatabase() // Verify the state was inserted into database by checking count Integer count = derbyConnector.retryWithHandle(handle -> handle.createQuery( - "SELECT COUNT(*) FROM " + tablesConfig.getCompactionStatesTable() + "SELECT COUNT(*) FROM " + tablesConfig.getIndexingStatesTable() + " WHERE fingerprint = :fp" ).bind("fp", fingerprint) .map((i, r, ctx) -> r.getInt(1)) @@ -157,9 +157,9 @@ public void test_findReferencedCompactionStateMarkedAsUnused() handle.createStatement( "INSERT INTO " + tablesConfig.getSegmentsTable() + " " + "(id, dataSource, created_date, start, \"end\", partitioned, version, used, payload, " - + "used_status_last_updated, compaction_state_fingerprint) " + + "used_status_last_updated, indexing_state_fingerprint) " + "VALUES (:id, :dataSource, :created_date, :start, :end, :partitioned, :version, :used, :payload, " - + ":used_status_last_updated, :compaction_state_fingerprint)" + + ":used_status_last_updated, :indexing_state_fingerprint)" ) .bind("id", "testSegment_2024-01-01_2024-01-02_v1_0") .bind("dataSource", "testDatasource") @@ -171,7 +171,7 @@ public void test_findReferencedCompactionStateMarkedAsUnused() .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) + .bind("indexing_state_fingerprint", fingerprint) .execute(); return null; }); @@ -198,7 +198,7 @@ public void test_deleteCompactionStatesOlderThan_deletesOnlyOldUnusedStates() // Insert old unused state (60 days old) derbyConnector.retryWithHandle(handle -> { handle.createStatement( - "INSERT INTO " + tablesConfig.getCompactionStatesTable() + " " + "INSERT INTO " + tablesConfig.getIndexingStatesTable() + " " + "(created_date, dataSource, fingerprint, payload, used, pending, used_status_last_updated) " + "VALUES (:cd, :ds, :fp, :pl, :used, :pending, :updated)" ) @@ -216,7 +216,7 @@ public void test_deleteCompactionStatesOlderThan_deletesOnlyOldUnusedStates() // Insert recent unused state (15 days old) derbyConnector.retryWithHandle(handle -> { handle.createStatement( - "INSERT INTO " + tablesConfig.getCompactionStatesTable() + " " + "INSERT INTO " + tablesConfig.getIndexingStatesTable() + " " + "(created_date, dataSource, fingerprint, payload, used, pending, used_status_last_updated) " + "VALUES (:cd, :ds, :fp, :pl, :used, :pending, :updated)" ) @@ -237,7 +237,7 @@ public void test_deleteCompactionStatesOlderThan_deletesOnlyOldUnusedStates() // Verify only 1 state remains in the table Integer count = derbyConnector.retryWithHandle(handle -> - handle.createQuery("SELECT COUNT(*) FROM " + tablesConfig.getCompactionStatesTable()) + handle.createQuery("SELECT COUNT(*) FROM " + tablesConfig.getIndexingStatesTable()) .map((i, r, ctx) -> r.getInt(1)) .first() ); @@ -294,7 +294,7 @@ public void test_upsertCompactionState_verifyExistingFingerprintMarkedUsed() // Verify it's marked as used Boolean usedBefore = derbyConnector.retryWithHandle(handle -> handle.createQuery( - "SELECT used FROM " + tablesConfig.getCompactionStatesTable() + "SELECT used FROM " + tablesConfig.getIndexingStatesTable() + " WHERE fingerprint = :fp" ).bind("fp", fingerprint) .map((i, r, ctx) -> r.getBoolean("used")) @@ -305,7 +305,7 @@ public void test_upsertCompactionState_verifyExistingFingerprintMarkedUsed() // Manually mark it as unused derbyConnector.retryWithHandle(handle -> handle.createStatement( - "UPDATE " + tablesConfig.getCompactionStatesTable() + "UPDATE " + tablesConfig.getIndexingStatesTable() + " SET used = false WHERE fingerprint = :fp" ).bind("fp", fingerprint).execute() ); @@ -319,7 +319,7 @@ public void test_upsertCompactionState_verifyExistingFingerprintMarkedUsed() // Verify it's marked as used again Boolean usedAfter = derbyConnector.retryWithHandle(handle -> handle.createQuery( - "SELECT used FROM " + tablesConfig.getCompactionStatesTable() + "SELECT used FROM " + tablesConfig.getIndexingStatesTable() + " WHERE fingerprint = :fp" ).bind("fp", fingerprint) .map((i, r, ctx) -> r.getBoolean("used")) @@ -329,7 +329,7 @@ public void test_upsertCompactionState_verifyExistingFingerprintMarkedUsed() // Verify only 1 row exists (no duplicate insert) Integer count = derbyConnector.retryWithHandle(handle -> - handle.createQuery("SELECT COUNT(*) FROM " + tablesConfig.getCompactionStatesTable()) + handle.createQuery("SELECT COUNT(*) FROM " + tablesConfig.getIndexingStatesTable()) .map((i, r, ctx) -> r.getInt(1)) .first() ); @@ -352,7 +352,7 @@ public void test_upsertCompactionState_whenAlreadyUsed_skipsUpdate() // Verify it's marked as used with the initial timestamp DateTime usedStatusBeforeUpdate = derbyConnector.retryWithHandle(handle -> handle.createQuery( - "SELECT used_status_last_updated FROM " + tablesConfig.getCompactionStatesTable() + "SELECT used_status_last_updated FROM " + tablesConfig.getIndexingStatesTable() + " WHERE fingerprint = :fp" ).bind("fp", fingerprint) .map((i, r, ctx) -> DateTimes.of(r.getString("used_status_last_updated"))) @@ -371,7 +371,7 @@ public void test_upsertCompactionState_whenAlreadyUsed_skipsUpdate() // Verify the used_status_last_updated timestamp DID NOT change DateTime usedStatusAfterUpdate = derbyConnector.retryWithHandle(handle -> handle.createQuery( - "SELECT used_status_last_updated FROM " + tablesConfig.getCompactionStatesTable() + "SELECT used_status_last_updated FROM " + tablesConfig.getIndexingStatesTable() + " WHERE fingerprint = :fp" ).bind("fp", fingerprint) .map((i, r, ctx) -> DateTimes.of(r.getString("used_status_last_updated"))) @@ -386,7 +386,7 @@ public void test_upsertCompactionState_whenAlreadyUsed_skipsUpdate() // Verify still only 1 row Integer count = derbyConnector.retryWithHandle(handle -> - handle.createQuery("SELECT COUNT(*) FROM " + tablesConfig.getCompactionStatesTable()) + handle.createQuery("SELECT COUNT(*) FROM " + tablesConfig.getIndexingStatesTable()) .map((i, r, ctx) -> r.getInt(1)) .first() ); @@ -411,7 +411,7 @@ public void test_markCompactionStatesAsActive_marksPendingStateAsActive() }); Boolean pendingBefore = derbyConnector.retryWithHandle(handle -> - handle.createQuery("SELECT pending FROM " + tablesConfig.getCompactionStatesTable() + " WHERE fingerprint = :fp") + handle.createQuery("SELECT pending FROM " + tablesConfig.getIndexingStatesTable() + " WHERE fingerprint = :fp") .bind("fp", fingerprint) .map((i, r, ctx) -> r.getBoolean("pending")) .first() @@ -422,7 +422,7 @@ public void test_markCompactionStatesAsActive_marksPendingStateAsActive() assertEquals(1, rowsUpdated); Boolean pendingAfter = derbyConnector.retryWithHandle(handle -> - handle.createQuery("SELECT pending FROM " + tablesConfig.getCompactionStatesTable() + " WHERE fingerprint = :fp") + handle.createQuery("SELECT pending FROM " + tablesConfig.getIndexingStatesTable() + " WHERE fingerprint = :fp") .bind("fp", fingerprint) .map((i, r, ctx) -> r.getBoolean("pending")) .first() @@ -447,7 +447,7 @@ public void test_markCompactionStatesAsActive_idempotent_returnsZeroWhenAlreadyA assertEquals(0, secondUpdate); Boolean pending = derbyConnector.retryWithHandle(handle -> - handle.createQuery("SELECT pending FROM " + tablesConfig.getCompactionStatesTable() + " WHERE fingerprint = :fp") + handle.createQuery("SELECT pending FROM " + tablesConfig.getIndexingStatesTable() + " WHERE fingerprint = :fp") .bind("fp", fingerprint) .map((i, r, ctx) -> r.getBoolean("pending")) .first() From 0d7c73f4cfe6e2910f49691b2da7314f99718aec Mon Sep 17 00:00:00 2001 From: capistrant Date: Tue, 13 Jan 2026 18:04:44 -0600 Subject: [PATCH 60/72] Refactor naming to IndexingState for the metadata cleanup duty --- .../overlord/config/OverlordKillConfigs.java | 12 +++---- .../config/OverlordMetadataCleanupConfig.java | 2 +- ...ava => KillUnreferencedIndexingState.java} | 12 +++---- ...=> KillUnreferencedIndexingStateTest.java} | 32 +++++++++---------- .../org/apache/druid/cli/CliOverlord.java | 6 ++-- 5 files changed, 32 insertions(+), 32 deletions(-) rename indexing-service/src/main/java/org/apache/druid/indexing/overlord/duty/{KillUnreferencedCompactionState.java => KillUnreferencedIndexingState.java} (84%) rename indexing-service/src/test/java/org/apache/druid/indexing/overlord/duty/{KillUnreferencedCompactionStateTest.java => KillUnreferencedIndexingStateTest.java} (92%) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/config/OverlordKillConfigs.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/config/OverlordKillConfigs.java index 00fad85fea5e..76e30ebdcca6 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/config/OverlordKillConfigs.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/config/OverlordKillConfigs.java @@ -26,18 +26,18 @@ public class OverlordKillConfigs { public static OverlordKillConfigs DEFAULT = new OverlordKillConfigs(null); - @JsonProperty("compactionStates") - private final OverlordMetadataCleanupConfig compactionStates; + @JsonProperty("indexingStates") + private final OverlordMetadataCleanupConfig indexingStates; public OverlordKillConfigs( - @JsonProperty("compactionStates") OverlordMetadataCleanupConfig compactionStates + @JsonProperty("indexingStates") OverlordMetadataCleanupConfig indexingStates ) { - this.compactionStates = Configs.valueOrDefault(compactionStates, OverlordMetadataCleanupConfig.DEFAULT); + this.indexingStates = Configs.valueOrDefault(indexingStates, OverlordMetadataCleanupConfig.DEFAULT); } - public OverlordMetadataCleanupConfig compactionStates() + public OverlordMetadataCleanupConfig indexingStates() { - return compactionStates; + return indexingStates; } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/config/OverlordMetadataCleanupConfig.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/config/OverlordMetadataCleanupConfig.java index 1f28922d2e0d..4142bdc5cd01 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/config/OverlordMetadataCleanupConfig.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/config/OverlordMetadataCleanupConfig.java @@ -52,7 +52,7 @@ public OverlordMetadataCleanupConfig( { this.cleanupEnabled = Configs.valueOrDefault(cleanupEnabled, true); this.cleanupPeriod = Configs.valueOrDefault(cleanupPeriod, Duration.standardDays(1)); - this.durationToRetain = Configs.valueOrDefault(durationToRetain, Duration.standardDays(90)); + this.durationToRetain = Configs.valueOrDefault(durationToRetain, Duration.standardDays(7)); this.pendingDurationToRetain = Configs.valueOrDefault(pendingDurationToRetain, Duration.standardDays(7)); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/duty/KillUnreferencedCompactionState.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/duty/KillUnreferencedIndexingState.java similarity index 84% rename from indexing-service/src/main/java/org/apache/druid/indexing/overlord/duty/KillUnreferencedCompactionState.java rename to indexing-service/src/main/java/org/apache/druid/indexing/overlord/duty/KillUnreferencedIndexingState.java index 804cf9c0faef..cd38e206bf93 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/duty/KillUnreferencedCompactionState.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/duty/KillUnreferencedIndexingState.java @@ -27,18 +27,18 @@ import javax.inject.Inject; import java.util.List; -public class KillUnreferencedCompactionState extends OverlordMetadataCleanupDuty +public class KillUnreferencedIndexingState extends OverlordMetadataCleanupDuty { - private static final Logger log = new Logger(KillUnreferencedCompactionState.class); + private static final Logger log = new Logger(KillUnreferencedIndexingState.class); private final CompactionStateStorage compactionStateStorage; @Inject - public KillUnreferencedCompactionState( + public KillUnreferencedIndexingState( OverlordMetadataCleanupConfig config, CompactionStateStorage compactionStateStorage ) { - super("compactionState", config); + super("indexingStates", config); this.compactionStateStorage = compactionStateStorage; } @@ -47,13 +47,13 @@ protected int cleanupEntriesCreatedBeforeDurationToRetain(DateTime minCreatedTim { // 1: Mark unreferenced states as unused int unused = compactionStateStorage.markUnreferencedCompactionStatesAsUnused(); - log.info("Marked [%s] unreferenced compaction states as unused.", unused); + log.info("Marked [%s] unreferenced indexing states as unused.", unused); // 2: Repair - find unused states still referenced by segments List stateFingerprints = compactionStateStorage.findReferencedCompactionStateMarkedAsUnused(); if (!stateFingerprints.isEmpty()) { int numUpdated = compactionStateStorage.markCompactionStatesAsUsed(stateFingerprints); - log.info("Marked [%s] unused compaction states referenced by used segments as used.", numUpdated); + log.info("Marked [%s] unused indexing states referenced by used segments as used.", numUpdated); } // 3: Delete unused states older than threshold diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/duty/KillUnreferencedCompactionStateTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/duty/KillUnreferencedIndexingStateTest.java similarity index 92% rename from indexing-service/src/test/java/org/apache/druid/indexing/overlord/duty/KillUnreferencedCompactionStateTest.java rename to indexing-service/src/test/java/org/apache/druid/indexing/overlord/duty/KillUnreferencedIndexingStateTest.java index 358ab355f48f..7f2cdd5aa3d3 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/duty/KillUnreferencedCompactionStateTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/duty/KillUnreferencedIndexingStateTest.java @@ -40,7 +40,7 @@ import java.util.ArrayList; import java.util.List; -public class KillUnreferencedCompactionStateTest +public class KillUnreferencedIndexingStateTest { @Rule public final TestDerbyConnector.DerbyConnectorRule derbyConnectorRule = @@ -81,8 +81,8 @@ public void test_killUnreferencedCompactionState_validateLifecycleOfActiveCompac Period.parse("P8D").toStandardDuration() ); - KillUnreferencedCompactionState duty = - new TestKillUnreferencedCompactionState(cleanupConfig, compactionStateStorage, dateTimes); + KillUnreferencedIndexingState duty = + new TestKillUnreferencedIndexingState(cleanupConfig, compactionStateStorage, dateTimes); String fingerprint = "test_fingerprint"; CompactionState state = createTestCompactionState(); @@ -120,8 +120,8 @@ public void test_killUnreferencedCompactionState_validateRepair() Period.parse("P8D").toStandardDuration() ); - KillUnreferencedCompactionState duty = - new TestKillUnreferencedCompactionState(cleanupConfig, compactionStateStorage, dateTimes); + KillUnreferencedIndexingState duty = + new TestKillUnreferencedIndexingState(cleanupConfig, compactionStateStorage, dateTimes); // Insert compaction state String fingerprint = "repair_fingerprint"; @@ -173,8 +173,8 @@ public void test_killUnreferencedCompactionState_disabled() Period.parse("P8D").toStandardDuration() ); - KillUnreferencedCompactionState duty = - new KillUnreferencedCompactionState(cleanupConfig, compactionStateStorage); + KillUnreferencedIndexingState duty = + new KillUnreferencedIndexingState(cleanupConfig, compactionStateStorage); // Insert compaction state String fingerprint = "disabled_fingerprint"; @@ -203,8 +203,8 @@ public void test_killUnreferencedCompactionState_validateLifecycleOfPendingCompa Period.parse("P10D").toStandardDuration() // Pending states older than 10 days should be deleted ); - KillUnreferencedCompactionState duty = - new TestKillUnreferencedCompactionState(cleanupConfig, compactionStateStorage, dateTimes); + KillUnreferencedIndexingState duty = + new TestKillUnreferencedIndexingState(cleanupConfig, compactionStateStorage, dateTimes); String fingerprint = "pending_fingerprint"; CompactionState state = createTestCompactionState(); @@ -237,8 +237,8 @@ public void test_killUnreferencedCompactionState_validateMixedPendingAndActiveCo Period.parse("P30D").toStandardDuration() ); - KillUnreferencedCompactionState duty = - new TestKillUnreferencedCompactionState(cleanupConfig, compactionStateStorage, dateTimes); + KillUnreferencedIndexingState duty = + new TestKillUnreferencedIndexingState(cleanupConfig, compactionStateStorage, dateTimes); String pendingFingerprint = "pending_fp"; String nonPendingFingerprint = "non_pending_fp"; @@ -274,8 +274,8 @@ public void test_killUnreferencedCompactionState_pendingStateMarkedActiveNotDele Period.parse("P30D").toStandardDuration() ); - KillUnreferencedCompactionState duty = - new TestKillUnreferencedCompactionState(cleanupConfig, compactionStateStorage, dateTimes); + KillUnreferencedIndexingState duty = + new TestKillUnreferencedIndexingState(cleanupConfig, compactionStateStorage, dateTimes); String fingerprint = "pending_marked_active_fp"; CompactionState state = createTestCompactionState(); @@ -330,16 +330,16 @@ private Boolean getCompactionStateUsedStatus(String fingerprint) } /** - * Extension of KillUnreferencedCompactionState that allows controlling the reference time used for cleanup decisions. + * Extension of KillUnreferencedIndexingState that allows controlling the reference time used for cleanup decisions. *

    * Allowing time control enables realistic testing of time-based retention logic. */ - private static class TestKillUnreferencedCompactionState extends KillUnreferencedCompactionState + private static class TestKillUnreferencedIndexingState extends KillUnreferencedIndexingState { private final List dateTimes; private int index = -1; - public TestKillUnreferencedCompactionState( + public TestKillUnreferencedIndexingState( OverlordMetadataCleanupConfig config, CompactionStateStorage compactionStateStorage, List dateTimes diff --git a/services/src/main/java/org/apache/druid/cli/CliOverlord.java b/services/src/main/java/org/apache/druid/cli/CliOverlord.java index 3737721b5c76..7f3e0b5907d0 100644 --- a/services/src/main/java/org/apache/druid/cli/CliOverlord.java +++ b/services/src/main/java/org/apache/druid/cli/CliOverlord.java @@ -94,7 +94,7 @@ import org.apache.druid.indexing.overlord.config.OverlordMetadataCleanupConfig; import org.apache.druid.indexing.overlord.config.TaskLockConfig; import org.apache.druid.indexing.overlord.config.TaskQueueConfig; -import org.apache.druid.indexing.overlord.duty.KillUnreferencedCompactionState; +import org.apache.druid.indexing.overlord.duty.KillUnreferencedIndexingState; import org.apache.druid.indexing.overlord.duty.OverlordDuty; import org.apache.druid.indexing.overlord.duty.TaskLogAutoCleaner; import org.apache.druid.indexing.overlord.duty.TaskLogAutoCleanerConfig; @@ -416,7 +416,7 @@ public TaskStorageDirTracker getTaskStorageDirTracker(WorkerConfig workerConfig, @LazySingleton public OverlordMetadataCleanupConfig provideCompactionStateCleanupConfig(OverlordKillConfigs killConfigs) { - return killConfigs.compactionStates(); + return killConfigs.indexingStates(); } @Provides @@ -465,7 +465,7 @@ private void configureOverlordHelpers(Binder binder) final Multibinder dutyBinder = Multibinder.newSetBinder(binder, OverlordDuty.class); dutyBinder.addBinding().to(TaskLogAutoCleaner.class); dutyBinder.addBinding().to(UnusedSegmentsKiller.class).in(LazySingleton.class); - dutyBinder.addBinding().to(KillUnreferencedCompactionState.class); + dutyBinder.addBinding().to(KillUnreferencedIndexingState.class); } /** From d5105f3cb79f0b4df30444399be1247b5dbbf13d Mon Sep 17 00:00:00 2001 From: capistrant Date: Tue, 13 Jan 2026 18:04:54 -0600 Subject: [PATCH 61/72] refresh some docs --- .../api-reference/automatic-compaction-api.md | 2 +- docs/configuration/index.md | 2 +- docs/data-management/automatic-compaction.md | 12 +++-- docs/operations/clean-metadata-store.md | 48 ++++++++++++++++++- 4 files changed, 57 insertions(+), 7 deletions(-) diff --git a/docs/api-reference/automatic-compaction-api.md b/docs/api-reference/automatic-compaction-api.md index f98e19890f33..6864aae4735c 100644 --- a/docs/api-reference/automatic-compaction-api.md +++ b/docs/api-reference/automatic-compaction-api.md @@ -889,7 +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`| -|`storeCompactionStatePerSegment`|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 predates compaction state fingerprinting (< Druid 36) will result in missing compaction states and trigger compaction on segments that may already be compacted.**|`true`| +|`storeCompactionStatePerSegment`|**This configuration only takes effect if `useSupervisors` is `true`.** Whether to persist the full compaction state in segment metadata. When `true` (default), compaction state is stored in both the segment metadata and the indexing 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 indexing 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 predates indexing state fingerprinting (< Druid 37) 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 f9f6b87c7e33..6e0f34583c69 100644 --- a/docs/configuration/index.md +++ b/docs/configuration/index.md @@ -389,7 +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 that stores compaction state payloads and fingerprints.|`druid_compactionStates`| +|`druid.metadata.storage.tables.indexingStates`|The table that stores indexing state payloads and fingerprints.|`druid_indexingStates`| |`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`| diff --git a/docs/data-management/automatic-compaction.md b/docs/data-management/automatic-compaction.md index 97d7074586cb..af0e5beb8117 100644 --- a/docs/data-management/automatic-compaction.md +++ b/docs/data-management/automatic-compaction.md @@ -241,12 +241,18 @@ You can run automatic compaction using compaction supervisors on the Overlord ra * Can use either the native compaction engine or the [MSQ task engine](#use-msq-for-auto-compaction) * More reactive and submits tasks as soon as a compaction slot is available * Tracked compaction task status to avoid re-compacting an interval repeatedly +* Uses new Indexing State Fingerprinting mechanisms to store less data per segment in metadata storage -To use compaction supervisors, update the [compaction dynamic config](../api-reference/automatic-compaction-api.md#update-cluster-level-compaction-config) and set: +To use compaction supervisors, the following configuration requirements must be met: -* `useSupervisors` to `true` so that compaction tasks can be run as supervisor tasks -* `engine` to `msq` to use the MSQ task engine as the compaction engine or to `native` (default value) to use the native engine. +* You mut be using incremental segment metadata caching: + * `druid.manager.segments.useIncrementalCache` set to `always` or `ifSynced` in your Overlord runtime properties. + * See [Segment metadata caching](../configuration/index.md#metadata-retrieval) for full configuration documentation. + +* update the [compaction dynamic config](../api-reference/automatic-compaction-api.md#update-cluster-level-compaction-config) and set: + * `useSupervisors` to `true` so that compaction tasks can be run as supervisor tasks + * `engine` to `msq` to use the MSQ task engine as the compaction engine or to `native` (default value) to use the native engine. Compaction supervisors use the same syntax as auto-compaction using Coordinator duties with one key difference: you submit the auto-compaction as a supervisor spec. In the spec, set the `type` to `autocompact` and include the auto-compaction config in the `spec`. diff --git a/docs/operations/clean-metadata-store.md b/docs/operations/clean-metadata-store.md index 65de31123023..71f673d2a3aa 100644 --- a/docs/operations/clean-metadata-store.md +++ b/docs/operations/clean-metadata-store.md @@ -34,6 +34,7 @@ The metadata store includes the following: - Compaction configuration records - Datasource records created by supervisors - Indexer task logs +- Indexing State records When you delete some entities from Apache Druid, records related to the entity may remain in the metadata store. If you have a high datasource churn rate, meaning you frequently create and delete many short-lived datasources or other related entities like compaction configuration or rules, the leftover records can fill your metadata store and cause performance issues. @@ -59,7 +60,7 @@ If you have compliance requirements to keep audit records and you enable automat ## Configure automated metadata cleanup You can configure cleanup for each entity separately, as described in this section. -Define the properties in the `coordinator/runtime.properties` file. +Unless otherwise specified, define the properties in the `coordinator/runtime.properties` file. The cleanup of one entity may depend on the cleanup of another entity as follows: - You have to configure a [kill task for segment records](#segment-records-and-segments-in-deep-storage-kill-task) before you can configure automated cleanup for [rules](#rules-records) or [compaction configuration](#compaction-configuration-records). @@ -131,6 +132,27 @@ Compaction configuration cleanup uses the following configuration: If you already have an extremely large compaction configuration, you may not be able to delete compaction configuration due to size limits with the audit log. In this case you can set `druid.audit.manager.maxPayloadSizeBytes` and `druid.audit.manager.skipNullField` to avoid the auditing issue. See [Audit logging](../configuration/index.md#audit-logging). ::: +### Indexing State Records + +:::info +Indexing State Records are cleaned up by the overlord. Therefore, this section should be configured in the `overlord/runtime.properties` file. +::: + +:::info +Indexing State Records are only created if you are using automatic compaction supervisors. +::: + +Indexing State records become eligible for deletion in the following scenarios: +- When no `used` segments have an `indexing_state_fingerprint` that is equal to the `fingerprint` of the record. +- When a record has `pending` state set to `true` + +Indexing State cleanup uses the following configuration: + - `druid.overlord.kill.indexingStates.on`: When `true`, enables cleanup for indexing state records. + - `druid.overlord.kill.indexingStates.period`: Defines the frequency in [ISO 8601 format](https://en.wikipedia.org/wiki/ISO_8601#Durations) for the cleanup job to check for and delete eligible indexing state records. Defaults to `P1D`. + - `druid.overlord.kill.indexingStates.durationToRetain`: Defines the retention period in [ISO 8601 format](https://en.wikipedia.org/wiki/ISO_8601#Durations) after indexing state records are marked as `used=false` become eligible for deletion. Defaults to `P7D`. + - `druid.overlord.kill.indexingStates.pendingDurationToRetain`: Defines the retention period in [ISO 8601 format](https://en.wikipedia.org/wiki/ISO_8601#Durations) after creation that pending indexing state records become eligible for deletion. Defaults to `P7D`. + - It is recommended that this value be greater than the maximum expected duration of compaction tasks to avoid pending records being deleted prematurely. + ### Datasource records created by supervisors Datasource records created by supervisors become eligible for deletion when the supervisor is terminated or does not exist in the `druid_supervisors` table and the `durationToRetain` time has passed since their creation. @@ -160,7 +182,9 @@ For more detail, see [Task logging](../configuration/index.md#task-logging). ## Disable automated metadata cleanup Druid automatically cleans up metadata records, excluding compaction configuration records and indexer task logs. -To disable automated metadata cleanup, set the following properties in the `coordinator/runtime.properties` file: +To disable automated metadata cleanup + +set the following properties in the `coordinator/runtime.properties` file: ```properties # Keep unused segments @@ -178,11 +202,19 @@ druid.coordinator.kill.rule.on=false # Keep datasource records created by supervisors druid.coordinator.kill.datasource.on=false ``` +set the following properties in the `overlord/runtime.properties` file: + +```properties +# Keep indexing state records +druid.overlord.kill.indexingStates.on=false +``` ## Example configuration for automated metadata cleanup Consider a scenario where you have scripts to create and delete hundreds of datasources and related entities a day. You do not want to fill your metadata store with leftover records. The datasources and related entities tend to persist for only one or two days. Therefore, you want to run a cleanup job that identifies and removes leftover records that are at least four days old after a seven day buffer period in case you want to recover the data. The exception is for audit logs, which you need to retain for 30 days: +Coordinator configuration (`coordinator/runtime.properties`): + ```properties ... # Schedule the metadata management store task for every hour: @@ -226,6 +258,18 @@ druid.coordinator.kill.datasource.durationToRetain=P4D ... ``` +Overlord configuration - if using automatic compaction supervisors (`overlord/runtime.properties`): + +```properties +... +# Poll every day to delete pending or unreferenced indexing state records > 4 days old +druid.overlord.kill.indexingStates.on=true +druid.overlord.kill.indexingStates.period=P1D +druid.overlord.kill.indexingStates.durationToRetain=P4D +druid.overlord.kill.indexingStates.pendingDurationToRetain=P4D +... +``` + ## Learn more See the following topics for more information: - [Metadata management](../configuration/index.md#metadata-management) for metadata store configuration reference. From 5793cb0aa971c618ec3b342e4985522a2b34a416 Mon Sep 17 00:00:00 2001 From: capistrant Date: Tue, 13 Jan 2026 20:13:27 -0600 Subject: [PATCH 62/72] fixup tests --- .../metadata/SQLMetadataConnectorTest.java | 5 +- .../SqlCompactionStateStorageTest.java | 90 +++++++------------ 2 files changed, 33 insertions(+), 62 deletions(-) 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 6dd0a724b035..12b267838ef2 100644 --- a/server/src/test/java/org/apache/druid/metadata/SQLMetadataConnectorTest.java +++ b/server/src/test/java/org/apache/druid/metadata/SQLMetadataConnectorTest.java @@ -197,12 +197,11 @@ public void testAlterSegmentTableAddLastUsed() public void testAlterSegmentTableAddCompactionStateFingerprint() { connector.createSegmentTable(); - derbyConnectorRule.segments().update("ALTER TABLE %1$s DROP COLUMN COMPACTION_STATE_FINGERPRINT"); - + derbyConnectorRule.segments().update("ALTER TABLE %1$s DROP COLUMN INDEXING_STATE_FINGERPRINT"); connector.alterSegmentTable(); Assert.assertTrue(connector.tableHasColumn( derbyConnectorRule.metadataTablesConfigSupplier().get().getSegmentsTable(), - "COMPACTION_STATE_FINGERPRINT" + "INDEXING_STATE_FINGERPRINT" )); } diff --git a/server/src/test/java/org/apache/druid/segment/metadata/SqlCompactionStateStorageTest.java b/server/src/test/java/org/apache/druid/segment/metadata/SqlCompactionStateStorageTest.java index 8e440d7a14dd..a6764c7edf16 100644 --- a/server/src/test/java/org/apache/druid/segment/metadata/SqlCompactionStateStorageTest.java +++ b/server/src/test/java/org/apache/druid/segment/metadata/SqlCompactionStateStorageTest.java @@ -94,15 +94,12 @@ public void test_upsertCompactionState_successfullyInsertsIntoDatabase() CompactionState state1 = createTestCompactionState(); String fingerprint = "fingerprint_abc123"; - derbyConnector.retryWithHandle(handle -> { - manager.upsertCompactionState( - "testDatasource", - fingerprint, - state1, - DateTimes.nowUtc() - ); - return null; - }); + manager.upsertCompactionState( + "testDatasource", + fingerprint, + state1, + DateTimes.nowUtc() + ); // Verify the state was inserted into database by checking count Integer count = derbyConnector.retryWithHandle(handle -> @@ -122,15 +119,14 @@ public void test_upsertCompactionState_andThen_markUnreferencedCompactionStateAs CompactionState state1 = createTestCompactionState(); String fingerprint = "fingerprint_abc123"; - derbyConnector.retryWithHandle(handle -> { - manager.upsertCompactionState( - "testDatasource", - fingerprint, - state1, - DateTimes.nowUtc() - ); - return null; - }); + manager.upsertCompactionState( + "testDatasource", + fingerprint, + state1, + DateTimes.nowUtc() + ); + manager.markCompactionStatesAsActive(fingerprint); + assertEquals(1, manager.markUnreferencedCompactionStatesAsUnused()); assertEquals(1, manager.markCompactionStatesAsUsed(List.of(fingerprint))); } @@ -141,15 +137,14 @@ public void test_findReferencedCompactionStateMarkedAsUnused() CompactionState state1 = createTestCompactionState(); String fingerprint = "fingerprint_abc123"; - derbyConnector.retryWithHandle(handle -> { - manager.upsertCompactionState( - "testDatasource", - fingerprint, - state1, - DateTimes.nowUtc() - ); - return null; - }); + manager.upsertCompactionState( + "testDatasource", + fingerprint, + state1, + DateTimes.nowUtc() + ); + manager.markCompactionStatesAsActive(fingerprint); + manager.markUnreferencedCompactionStatesAsUnused(); assertEquals(0, manager.findReferencedCompactionStateMarkedAsUnused().size()); @@ -249,10 +244,7 @@ public void test_upsertCompactionState_withNullState_throwsException() { Exception exception = assertThrows( Exception.class, - () -> derbyConnector.retryWithHandle(handle -> { - manager.upsertCompactionState("ds", "somePrint", null, DateTimes.nowUtc()); - return null; - }) + () -> manager.upsertCompactionState("ds", "somePrint", null, DateTimes.nowUtc()) ); assertTrue( @@ -267,10 +259,7 @@ public void test_upsertCompactionState_withEmptyFingerprint_throwsException() // The exception ends up wrapped in a sql exception doe to the retryWithHandle so we will just check the message Exception exception = assertThrows( Exception.class, - () -> derbyConnector.retryWithHandle(handle -> { - manager.upsertCompactionState("ds", "", createBasicCompactionState(), DateTimes.nowUtc()); - return null; - }) + () -> manager.upsertCompactionState("ds", "", createBasicCompactionState(), DateTimes.nowUtc()) ); assertTrue( @@ -286,10 +275,7 @@ public void test_upsertCompactionState_verifyExistingFingerprintMarkedUsed() CompactionState state = createTestCompactionState(); // Persist initially - derbyConnector.retryWithHandle(handle -> { - manager.upsertCompactionState("ds1", fingerprint, state, DateTimes.nowUtc()); - return null; - }); + manager.upsertCompactionState("ds1", fingerprint, state, DateTimes.nowUtc()); // Verify it's marked as used Boolean usedBefore = derbyConnector.retryWithHandle(handle -> @@ -311,10 +297,7 @@ public void test_upsertCompactionState_verifyExistingFingerprintMarkedUsed() ); // Persist again with the same fingerprint (should UPDATE, not INSERT) - derbyConnector.retryWithHandle(handle -> { - manager.upsertCompactionState("ds1", fingerprint, state, DateTimes.nowUtc()); - return null; - }); + manager.upsertCompactionState("ds1", fingerprint, state, DateTimes.nowUtc()); // Verify it's marked as used again Boolean usedAfter = derbyConnector.retryWithHandle(handle -> @@ -344,10 +327,7 @@ public void test_upsertCompactionState_whenAlreadyUsed_skipsUpdate() DateTime initialTime = DateTimes.of("2024-01-01T00:00:00.000Z"); // Insert fingerprint as used initially - derbyConnector.retryWithHandle(handle -> { - manager.upsertCompactionState("ds1", fingerprint, state, initialTime); - return null; - }); + manager.upsertCompactionState("ds1", fingerprint, state, initialTime); // Verify it's marked as used with the initial timestamp DateTime usedStatusBeforeUpdate = derbyConnector.retryWithHandle(handle -> @@ -363,10 +343,7 @@ public void test_upsertCompactionState_whenAlreadyUsed_skipsUpdate() // Call upsert again with a different timestamp // Since the fingerprint is already used, this should skip the UPDATE DateTime laterTime = DateTimes.of("2024-01-02T00:00:00.000Z"); - derbyConnector.retryWithHandle(handle -> { - manager.upsertCompactionState("ds1", fingerprint, state, laterTime); - return null; - }); + manager.upsertCompactionState("ds1", fingerprint, state, laterTime); // Verify the used_status_last_updated timestamp DID NOT change DateTime usedStatusAfterUpdate = derbyConnector.retryWithHandle(handle -> @@ -405,10 +382,7 @@ public void test_markCompactionStatesAsActive_marksPendingStateAsActive() String fingerprint = "pending_fingerprint"; CompactionState state = createTestCompactionState(); - derbyConnector.retryWithHandle(handle -> { - manager.upsertCompactionState("ds1", fingerprint, state, DateTimes.nowUtc()); - return null; - }); + manager.upsertCompactionState("ds1", fingerprint, state, DateTimes.nowUtc()); Boolean pendingBefore = derbyConnector.retryWithHandle(handle -> handle.createQuery("SELECT pending FROM " + tablesConfig.getIndexingStatesTable() + " WHERE fingerprint = :fp") @@ -436,10 +410,8 @@ public void test_markCompactionStatesAsActive_idempotent_returnsZeroWhenAlreadyA String fingerprint = "already_active_fingerprint"; CompactionState state = createTestCompactionState(); - derbyConnector.retryWithHandle(handle -> { - manager.upsertCompactionState("ds1", fingerprint, state, DateTimes.nowUtc()); - return null; - }); + manager.upsertCompactionState("ds1", fingerprint, state, DateTimes.nowUtc()); + int firstUpdate = manager.markCompactionStatesAsActive(fingerprint); assertEquals(1, firstUpdate); From f87f99fa9c9d9a58e9cb4eebb9698a970dc5e50d Mon Sep 17 00:00:00 2001 From: capistrant Date: Wed, 14 Jan 2026 11:33:38 -0600 Subject: [PATCH 63/72] Refactoring name of CompactionStateCache to IndexingStateCache --- .../NewestSegmentFirstPolicyBenchmark.java | 6 +- .../compact/CompactionSupervisorTest.java | 4 +- .../indexing/compact/CompactionJobQueue.java | 12 +- .../compact/OverlordCompactionScheduler.java | 10 +- .../common/actions/TaskActionTestKit.java | 4 +- .../common/task/IngestionTestBase.java | 8 +- .../OverlordCompactionSchedulerTest.java | 4 +- .../cache/HeapMemorySegmentMetadataCache.java | 22 +-- .../DefaultCompactionFingerprintMapper.java | 10 +- ...tateCache.java => IndexingStateCache.java} | 78 ++++----- .../apache/druid/segment/metadata/Metric.java | 10 +- ...Cache.java => NoopIndexingStateCache.java} | 11 +- ...etadataStorageCoordinatorReadOnlyTest.java | 4 +- ...exerSQLMetadataStorageCoordinatorTest.java | 4 +- .../SqlSegmentsMetadataManagerV2Test.java | 4 +- .../HeapMemorySegmentMetadataCacheTest.java | 8 +- ...eTest.java => IndexingStateCacheTest.java} | 148 +++++++++--------- .../SqlCompactionStateStorageTest.java | 2 +- .../compaction/CompactionStatusTest.java | 10 +- .../NewestSegmentFirstPolicyTest.java | 8 +- .../druid/guice/MetadataManagerModule.java | 6 +- 21 files changed, 186 insertions(+), 187 deletions(-) rename server/src/main/java/org/apache/druid/segment/metadata/{CompactionStateCache.java => IndexingStateCache.java} (60%) rename server/src/main/java/org/apache/druid/segment/metadata/{NoopCompactionStateCache.java => NoopIndexingStateCache.java} (74%) rename server/src/test/java/org/apache/druid/segment/metadata/{CompactionStateCacheTest.java => IndexingStateCacheTest.java} (51%) 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 686ebef64883..485d0f2f9979 100644 --- a/benchmarks/src/test/java/org/apache/druid/server/coordinator/NewestSegmentFirstPolicyBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/server/coordinator/NewestSegmentFirstPolicyBenchmark.java @@ -21,10 +21,10 @@ import com.google.common.collect.ImmutableList; import org.apache.druid.client.DataSourcesSnapshot; -import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.segment.metadata.CompactionTestUtils; import org.apache.druid.segment.metadata.DefaultCompactionFingerprintMapper; -import org.apache.druid.segment.metadata.NoopCompactionStateCache; +import org.apache.druid.segment.metadata.NoopIndexingStateCache; import org.apache.druid.server.compaction.CompactionCandidateSearchPolicy; import org.apache.druid.server.compaction.CompactionSegmentIterator; import org.apache.druid.server.compaction.NewestSegmentFirstPolicy; @@ -139,7 +139,7 @@ public void measureNewestSegmentFirstPolicy(Blackhole blackhole) compactionConfigs, dataSources, Collections.emptyMap(), - new DefaultCompactionFingerprintMapper(new NoopCompactionStateCache(), new DefaultObjectMapper()) // TODO fix + new DefaultCompactionFingerprintMapper(new NoopIndexingStateCache(), CompactionTestUtils.createDeterministicMapper()) ); for (int i = 0; i < numCompactionTaskSlots && iterator.hasNext(); i++) { blackhole.consume(iterator.next()); diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionSupervisorTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionSupervisorTest.java index 3d1731ced68f..8e82cdb824cb 100644 --- a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionSupervisorTest.java +++ b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionSupervisorTest.java @@ -32,9 +32,9 @@ import org.apache.druid.query.DruidMetrics; import org.apache.druid.rpc.UpdateResponse; import org.apache.druid.segment.metadata.CompactionFingerprintMapper; -import org.apache.druid.segment.metadata.CompactionStateCache; import org.apache.druid.segment.metadata.CompactionTestUtils; import org.apache.druid.segment.metadata.DefaultCompactionFingerprintMapper; +import org.apache.druid.segment.metadata.IndexingStateCache; import org.apache.druid.server.coordinator.ClusterCompactionConfig; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; import org.apache.druid.server.coordinator.InlineSchemaDataSourceCompactionConfig; @@ -288,7 +288,7 @@ private void verifySegmentsHaveNullLastCompactionStateAndNonNullFingerprint() private void verifyCompactedSegmentsHaveFingerprints(DataSourceCompactionConfig compactionConfig) { - CompactionStateCache cache = overlord.bindings().getInstance(CompactionStateCache.class); + IndexingStateCache cache = overlord.bindings().getInstance(IndexingStateCache.class); CompactionFingerprintMapper fingerprintMapper = new DefaultCompactionFingerprintMapper( cache, CompactionTestUtils.createDeterministicMapper() 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 a1417d066c83..4b98282eae7e 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,9 +36,9 @@ 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.CompactionStateCache; import org.apache.druid.segment.metadata.CompactionStateStorage; import org.apache.druid.segment.metadata.DefaultCompactionFingerprintMapper; +import org.apache.druid.segment.metadata.IndexingStateCache; import org.apache.druid.server.compaction.CompactionCandidate; import org.apache.druid.server.compaction.CompactionCandidateSearchPolicy; import org.apache.druid.server.compaction.CompactionSlotManager; @@ -100,7 +100,7 @@ public class CompactionJobQueue private final Map submittedTaskIdToJob; private final CompactionStateStorage compactionStateStorage; - private final CompactionStateCache compactionStateCache; + private final IndexingStateCache indexingStateCache; public CompactionJobQueue( DataSourcesSnapshot dataSourcesSnapshot, @@ -112,7 +112,7 @@ public CompactionJobQueue( BrokerClient brokerClient, ObjectMapper objectMapper, CompactionStateStorage compactionStateStorage, - CompactionStateCache compactionStateCache, + IndexingStateCache indexingStateCache, ObjectMapper deterministicCompactionStateMapper ) { @@ -130,11 +130,11 @@ public CompactionJobQueue( clusterCompactionConfig, dataSourcesSnapshot.getUsedSegmentsTimelinesPerDataSource()::get, snapshotBuilder, - new DefaultCompactionFingerprintMapper(compactionStateCache, deterministicCompactionStateMapper) + new DefaultCompactionFingerprintMapper(indexingStateCache, deterministicCompactionStateMapper) ); this.compactionStateStorage = compactionStateStorage; - this.compactionStateCache = compactionStateCache; + this.indexingStateCache = indexingStateCache; this.taskActionClientFactory = taskActionClientFactory; this.overlordClient = overlordClient; @@ -375,7 +375,7 @@ private void persistPendingCompactionState(CompactionJob job) job.getTargetCompactionState(), DateTimes.nowUtc() ); - compactionStateCache.addCompactionState(job.getTargetCompactionStateFingerprint(), job.getTargetCompactionState()); + indexingStateCache.addIndexingState(job.getTargetCompactionStateFingerprint(), job.getTargetCompactionState()); } } 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 4b19b9c2c141..c7606d5e5608 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 @@ -46,8 +46,8 @@ 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.CompactionStateCache; import org.apache.druid.segment.metadata.CompactionStateStorage; +import org.apache.druid.segment.metadata.IndexingStateCache; import org.apache.druid.server.compaction.CompactionRunSimulator; import org.apache.druid.server.compaction.CompactionSimulateResult; import org.apache.druid.server.compaction.CompactionStatusTracker; @@ -144,7 +144,7 @@ public class OverlordCompactionScheduler implements CompactionScheduler private final long schedulePeriodMillis; private final CompactionStateStorage compactionStateStorage; - private final CompactionStateCache compactionStateCache; + private final IndexingStateCache indexingStateCache; private final ObjectMapper deterministicMapper; @Inject @@ -164,7 +164,7 @@ public OverlordCompactionScheduler( ServiceEmitter emitter, ObjectMapper objectMapper, CompactionStateStorage compactionStateStorage, - CompactionStateCache compactionStateCache, + IndexingStateCache indexingStateCache, @Deterministic ObjectMapper deterministicMapper ) { @@ -192,7 +192,7 @@ public OverlordCompactionScheduler( this.taskActionClientFactory = taskActionClientFactory; this.druidInputSourceFactory = druidInputSourceFactory; this.compactionStateStorage = compactionStateStorage; - this.compactionStateCache = compactionStateCache; + this.indexingStateCache = indexingStateCache; this.deterministicMapper = deterministicMapper; this.taskRunnerListener = new TaskRunnerListener() { @@ -393,7 +393,7 @@ private synchronized void resetCompactionJobQueue() brokerClient, objectMapper, compactionStateStorage, - compactionStateCache, + indexingStateCache, deterministicMapper ); latestJobQueue.set(queue); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/TaskActionTestKit.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/TaskActionTestKit.java index 256d2ed2de3b..59a59213b0fe 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/TaskActionTestKit.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/TaskActionTestKit.java @@ -41,8 +41,8 @@ import org.apache.druid.metadata.segment.cache.HeapMemorySegmentMetadataCache; import org.apache.druid.metadata.segment.cache.SegmentMetadataCache; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; -import org.apache.druid.segment.metadata.CompactionStateCache; import org.apache.druid.segment.metadata.HeapMemoryCompactionStateStorage; +import org.apache.druid.segment.metadata.IndexingStateCache; import org.apache.druid.segment.metadata.NoopSegmentSchemaCache; import org.apache.druid.segment.metadata.SegmentSchemaManager; import org.apache.druid.server.coordinator.simulate.BlockingExecutorService; @@ -202,7 +202,7 @@ private SqlSegmentMetadataTransactionFactory setupTransactionFactory( Suppliers.ofInstance(new SegmentsMetadataManagerConfig(Period.seconds(1), cacheMode, null)), Suppliers.ofInstance(metadataStorageTablesConfig), new NoopSegmentSchemaCache(), - new CompactionStateCache(), + new IndexingStateCache(), testDerbyConnector, (poolSize, name) -> new WrappingScheduledExecutorService(name, metadataCachePollExec, false), emitter diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IngestionTestBase.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IngestionTestBase.java index ad6ec2bd4019..5329fb7fd8ef 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IngestionTestBase.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IngestionTestBase.java @@ -87,8 +87,8 @@ import org.apache.druid.segment.loading.LocalDataSegmentPusherConfig; import org.apache.druid.segment.loading.SegmentCacheManager; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; -import org.apache.druid.segment.metadata.CompactionStateCache; import org.apache.druid.segment.metadata.HeapMemoryCompactionStateStorage; +import org.apache.druid.segment.metadata.IndexingStateCache; import org.apache.druid.segment.metadata.SegmentSchemaCache; import org.apache.druid.segment.metadata.SegmentSchemaManager; import org.apache.druid.segment.realtime.NoopChatHandlerProvider; @@ -141,7 +141,7 @@ public abstract class IngestionTestBase extends InitializedNullHandlingTest private TestDataSegmentKiller dataSegmentKiller; private SegmentMetadataCache segmentMetadataCache; private SegmentSchemaCache segmentSchemaCache; - private CompactionStateCache compactionStateCache; + private IndexingStateCache indexingStateCache; protected File reportsFile; protected IngestionTestBase() @@ -176,7 +176,7 @@ public void setUpIngestionTestBase() throws IOException ); segmentSchemaCache = new SegmentSchemaCache(); - compactionStateCache = new CompactionStateCache(); + indexingStateCache = new IndexingStateCache(); storageCoordinator = new IndexerSQLMetadataStorageCoordinator( createTransactionFactory(), objectMapper, @@ -343,7 +343,7 @@ private SqlSegmentMetadataTransactionFactory createTransactionFactory() Suppliers.ofInstance(new SegmentsMetadataManagerConfig(Period.millis(10), cacheMode, null)), derbyConnectorRule.metadataTablesConfigSupplier(), segmentSchemaCache, - compactionStateCache, + indexingStateCache, derbyConnectorRule.getConnector(), ScheduledExecutors::fixed, NoopServiceEmitter.instance() 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 3e2b6c43eea8..55d5b85539f8 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,8 +64,8 @@ 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.CompactionStateCache; import org.apache.druid.segment.metadata.HeapMemoryCompactionStateStorage; +import org.apache.druid.segment.metadata.IndexingStateCache; import org.apache.druid.server.compaction.CompactionSimulateResult; import org.apache.druid.server.compaction.CompactionStatistics; import org.apache.druid.server.compaction.CompactionStatus; @@ -235,7 +235,7 @@ private void initScheduler() serviceEmitter, OBJECT_MAPPER, new HeapMemoryCompactionStateStorage(), - new CompactionStateCache(), + new IndexingStateCache(), OBJECT_MAPPER // TODO fix ); } 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 a260323265ff..7f3440d20be4 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 @@ -51,7 +51,7 @@ import org.apache.druid.query.DruidMetrics; import org.apache.druid.segment.SchemaPayload; import org.apache.druid.segment.SegmentMetadata; -import org.apache.druid.segment.metadata.CompactionStateCache; +import org.apache.druid.segment.metadata.IndexingStateCache; import org.apache.druid.segment.metadata.SegmentSchemaCache; import org.apache.druid.server.http.DataSegmentPlus; import org.apache.druid.timeline.CompactionState; @@ -140,7 +140,7 @@ private enum CacheState private final SegmentSchemaCache segmentSchemaCache; private final boolean useCompactionStateCache; - private final CompactionStateCache compactionStateCache; + private final IndexingStateCache indexingStateCache; private final ListeningScheduledExecutorService pollExecutor; private final ServiceEmitter emitter; @@ -173,7 +173,7 @@ public HeapMemorySegmentMetadataCache( Supplier config, Supplier tablesConfig, SegmentSchemaCache segmentSchemaCache, - CompactionStateCache compactionStateCache, + IndexingStateCache indexingStateCache, SQLMetadataConnector connector, ScheduledExecutorFactory executorFactory, ServiceEmitter emitter @@ -185,8 +185,8 @@ public HeapMemorySegmentMetadataCache( this.tablesConfig = tablesConfig.get(); this.useSchemaCache = segmentSchemaCache.isEnabled(); this.segmentSchemaCache = segmentSchemaCache; - this.useCompactionStateCache = compactionStateCache.isEnabled(); - this.compactionStateCache = compactionStateCache; + this.useCompactionStateCache = indexingStateCache.isEnabled(); + this.indexingStateCache = indexingStateCache; this.connector = connector; this.pollExecutor = isEnabled() ? MoreExecutors.listeningDecorator(executorFactory.create(1, "SegmentMetadataCache-%s")) @@ -241,7 +241,7 @@ public void stop() datasourceToSegmentCache.clear(); datasourcesSnapshot.set(null); if (useCompactionStateCache) { - compactionStateCache.clear(); + indexingStateCache.clear(); } syncFinishTime.set(null); @@ -1124,7 +1124,7 @@ private void emitMetric(String datasource, String metric, long value) /** * Retrieves required used compaction states from the metadata store and resets - * them in the {@link CompactionStateCache}. If this is the first sync, all used + * them in the {@link IndexingStateCache}. If this is the first sync, all used * compaction states are retrieved from the metadata store. If this is a delta sync, * first only the fingerprints of all used compaction states are retrieved. Payloads are * then fetched for only the fingerprints which are not present in the cache. @@ -1133,7 +1133,7 @@ private void retrieveAndResetUsedCompactionStates() { final Stopwatch compactionStateSyncDuration = Stopwatch.createStarted(); - // Reset the CompactionStateCache with latest compaction states + // Reset the IndexingStateCache with latest compaction states final Map fingerprintToStateMap; if (syncFinishTime.get() == null) { fingerprintToStateMap = buildFingerprintToStateMapForFullSync(); @@ -1141,10 +1141,10 @@ private void retrieveAndResetUsedCompactionStates() fingerprintToStateMap = buildFingerprintToStateMapForDeltaSync(); } - compactionStateCache.resetCompactionStatesForPublishedSegments(fingerprintToStateMap); + indexingStateCache.resetIndexingStatesForPublishedSegments(fingerprintToStateMap); // Emit metrics for the current contents of the cache - compactionStateCache.getAndResetStats().forEach(this::emitMetric); + indexingStateCache.getAndResetStats().forEach(this::emitMetric); emitMetric(Metric.RETRIEVE_COMPACTION_STATES_DURATION_MILLIS, compactionStateSyncDuration.millisElapsed()); } @@ -1179,7 +1179,7 @@ private Map buildFingerprintToStateMapForDeltaSync() { // Identify fingerprints in the cache and in the metadata store final Map fingerprintToStateMap = new HashMap<>( - compactionStateCache.getPublishedCompactionStateMap() + indexingStateCache.getPublishedIndexingStateMap() ); final Set cachedFingerprints = Set.copyOf(fingerprintToStateMap.keySet()); final Set persistedFingerprints = query( diff --git a/server/src/main/java/org/apache/druid/segment/metadata/DefaultCompactionFingerprintMapper.java b/server/src/main/java/org/apache/druid/segment/metadata/DefaultCompactionFingerprintMapper.java index 96c7a9871437..dfeaec3fce13 100644 --- a/server/src/main/java/org/apache/druid/segment/metadata/DefaultCompactionFingerprintMapper.java +++ b/server/src/main/java/org/apache/druid/segment/metadata/DefaultCompactionFingerprintMapper.java @@ -32,20 +32,20 @@ /** * Default implementation of {@link CompactionFingerprintMapper} that delegates to - * {@link CompactionStateStorage} for fingerprint generation and {@link CompactionStateCache} + * {@link CompactionStateStorage} for fingerprint generation and {@link IndexingStateCache} * for state lookups. */ public class DefaultCompactionFingerprintMapper implements CompactionFingerprintMapper { - private final CompactionStateCache compactionStateCache; + private final IndexingStateCache indexingStateCache; private final ObjectMapper deterministicMapper; public DefaultCompactionFingerprintMapper( - CompactionStateCache compactionStateCache, + IndexingStateCache indexingStateCache, @Deterministic ObjectMapper deterministicMapper ) { - this.compactionStateCache = compactionStateCache; + this.indexingStateCache = indexingStateCache; this.deterministicMapper = deterministicMapper; } @@ -82,6 +82,6 @@ public String generateFingerprint(String dataSource, CompactionState compactionS @Override public Optional getStateForFingerprint(String fingerprint) { - return compactionStateCache.getCompactionStateByFingerprint(fingerprint); + return indexingStateCache.getIndexingStateByFingerprint(fingerprint); } } diff --git a/server/src/main/java/org/apache/druid/segment/metadata/CompactionStateCache.java b/server/src/main/java/org/apache/druid/segment/metadata/IndexingStateCache.java similarity index 60% rename from server/src/main/java/org/apache/druid/segment/metadata/CompactionStateCache.java rename to server/src/main/java/org/apache/druid/segment/metadata/IndexingStateCache.java index 99eedea4b172..9f11c7137734 100644 --- a/server/src/main/java/org/apache/druid/segment/metadata/CompactionStateCache.java +++ b/server/src/main/java/org/apache/druid/segment/metadata/IndexingStateCache.java @@ -30,24 +30,24 @@ import java.util.concurrent.atomic.AtomicReference; /** - * In-memory cache of compaction states used by {@link org.apache.druid.metadata.segment.cache.HeapMemorySegmentMetadataCache}. + * In-memory cache of indexing states used by {@link org.apache.druid.metadata.segment.cache.HeapMemorySegmentMetadataCache}. *

    - * This cache stores compaction states for published segments polled from the metadata store. - * It is the PRIMARY way to read compaction states in production. + * This cache stores indexing states for published segments polled from the metadata store. + * It is the primary way to read indexing states in production. *

    * The cache is populated during segment metadata cache sync operations and provides fast lookups * without hitting the database. */ @LazySingleton -public class CompactionStateCache +public class IndexingStateCache { - private static final Logger log = new Logger(CompactionStateCache.class); + private static final Logger log = new Logger(IndexingStateCache.class); /** - * Atomically updated reference to published compaction states. + * Atomically updated reference to published indexing states. */ - private final AtomicReference publishedCompactionStates - = new AtomicReference<>(PublishedCompactionStates.EMPTY); + private final AtomicReference publishedIndexingStates + = new AtomicReference<>(PublishedIndexingStates.EMPTY); private final AtomicInteger cacheMissCount = new AtomicInteger(0); private final AtomicInteger cacheHitCount = new AtomicInteger(0); @@ -59,35 +59,35 @@ public boolean isEnabled() } /** - * Resets the cache with compaction states polled from the metadata store. + * Resets the cache with indexing states polled from the metadata store. * Called after each successful poll in HeapMemorySegmentMetadataCache. * - * @param fingerprintToStateMap Complete map of all active compaction state fingerprints + * @param fingerprintToStateMap Complete fp:state map of all active indexing state fingerprints */ - public void resetCompactionStatesForPublishedSegments( + public void resetIndexingStatesForPublishedSegments( Map fingerprintToStateMap ) { - this.publishedCompactionStates.set( - new PublishedCompactionStates(fingerprintToStateMap) + this.publishedIndexingStates.set( + new PublishedIndexingStates(fingerprintToStateMap) ); - log.debug("Reset compaction state cache with [%d] fingerprints", fingerprintToStateMap.size()); + log.debug("Reset indexing state cache with [%d] fingerprints", fingerprintToStateMap.size()); } /** - * Retrieves a compaction state by its fingerprint. - * This is the PRIMARY method for reading compaction states. + * Retrieves an indexing state by its fingerprint. + * This is the indexing method for reading indexing states. * * @param fingerprint The fingerprint to look up - * @return The compaction state, or Optional.empty() if not cached + * @return The cached indexing state, or Optional.empty() if not cached */ - public Optional getCompactionStateByFingerprint(String fingerprint) + public Optional getIndexingStateByFingerprint(String fingerprint) { if (fingerprint == null) { return Optional.empty(); } - CompactionState state = publishedCompactionStates.get() + CompactionState state = publishedIndexingStates.get() .fingerprintToStateMap .get(fingerprint); if (state == null) { @@ -100,7 +100,7 @@ public Optional getCompactionStateByFingerprint(String fingerpr } /** - * Adds or updates a single compaction state in the cache. + * Adds or updates a single indexing state in the cache. *

    * This is called when a new compaction state is persisted to the database via upsertCompactionState * to ensure the cache is immediately consistent without waiting for the next sync. @@ -108,25 +108,25 @@ public Optional getCompactionStateByFingerprint(String fingerpr * This method checks if the state is already cached before performing the atomic update. * * @param fingerprint The fingerprint key - * @param state The compaction state to cache + * @param state The indexing state to cache */ - public void addCompactionState(String fingerprint, CompactionState state) + public void addIndexingState(String fingerprint, CompactionState state) { if (fingerprint == null || state == null) { return; } // Check if the state is already cached - avoid expensive update if not needed - CompactionState existing = publishedCompactionStates.get() + CompactionState existing = publishedIndexingStates.get() .fingerprintToStateMap .get(fingerprint); if (state.equals(existing)) { - log.debug("Compaction state for fingerprint[%s] already cached, skipping update", fingerprint); + log.debug("Indexing state for fingerprint[%s] already cached, skipping update", fingerprint); return; } // State is not cached or different - perform atomic update - publishedCompactionStates.updateAndGet(current -> { + publishedIndexingStates.updateAndGet(current -> { // Double-check in case another thread updated between our check and now if (state.equals(current.fingerprintToStateMap.get(fingerprint))) { return current; @@ -134,19 +134,19 @@ public void addCompactionState(String fingerprint, CompactionState state) Map newMap = new HashMap<>(current.fingerprintToStateMap); newMap.put(fingerprint, state); - return new PublishedCompactionStates(newMap); + return new PublishedIndexingStates(newMap); }); - log.debug("Added compaction state to cache for fingerprint[%s]", fingerprint); + log.debug("Added indexing state to cache for fingerprint[%s]", fingerprint); } /** * Gets the full cached map (immutable copy). * Used by HeapMemorySegmentMetadataCache for delta sync calculations. */ - public Map getPublishedCompactionStateMap() + public Map getPublishedIndexingStateMap() { - return publishedCompactionStates.get().fingerprintToStateMap; + return publishedIndexingStates.get().fingerprintToStateMap; } /** @@ -154,7 +154,7 @@ public Map getPublishedCompactionStateMap() */ public void clear() { - publishedCompactionStates.set(PublishedCompactionStates.EMPTY); + publishedIndexingStates.set(PublishedIndexingStates.EMPTY); resetStats(); } @@ -164,10 +164,10 @@ public void clear() public Map getAndResetStats() { return Map.of( - Metric.COMPACTION_STATE_CACHE_HITS, cacheHitCount.getAndSet(0), - Metric.COMPACTION_STATE_CACHE_MISSES, cacheMissCount.getAndSet(0), - Metric.COMPACTION_STATE_CACHE_FINGERPRINTS, - publishedCompactionStates.get().fingerprintToStateMap.size() + Metric.INDEXING_STATE_CACHE_HITS, cacheHitCount.getAndSet(0), + Metric.INDEXING_STATE_CACHE_MISSES, cacheMissCount.getAndSet(0), + Metric.INDEXING_STATE_CACHE_FINGERPRINTS, + publishedIndexingStates.get().fingerprintToStateMap.size() ); } @@ -181,16 +181,16 @@ private void resetStats() } /** - * Immutable snapshot of compaction states polled from DB. + * Immutable snapshot of indexing states polled from DB. */ - private static class PublishedCompactionStates + private static class PublishedIndexingStates { - private static final PublishedCompactionStates EMPTY = - new PublishedCompactionStates(Map.of()); + private static final PublishedIndexingStates EMPTY = + new PublishedIndexingStates(Map.of()); private final Map fingerprintToStateMap; - private PublishedCompactionStates(Map fingerprintToStateMap) + private PublishedIndexingStates(Map fingerprintToStateMap) { this.fingerprintToStateMap = Map.copyOf(fingerprintToStateMap); } diff --git a/server/src/main/java/org/apache/druid/segment/metadata/Metric.java b/server/src/main/java/org/apache/druid/segment/metadata/Metric.java index 4a566bc9c476..a29f4a7eafff 100644 --- a/server/src/main/java/org/apache/druid/segment/metadata/Metric.java +++ b/server/src/main/java/org/apache/druid/segment/metadata/Metric.java @@ -20,12 +20,12 @@ package org.apache.druid.segment.metadata; /** - * Metrics related to {@link SegmentSchemaCache}, {@link SegmentSchemaManager}, and {@link CompactionStateCache}. + * Metrics related to {@link SegmentSchemaCache}, {@link SegmentSchemaManager}, and {@link IndexingStateCache}. */ public class Metric { private static final String PREFIX = "segment/schemaCache/"; - private static final String COMPACTION_STATE_PREFIX = "segment/compactionStateCache/"; + private static final String INDEXING_STATE_PREFIX = "segment/indexingStateCache/"; public static final String CACHE_MISSES = "miss/count"; @@ -60,7 +60,7 @@ public class Metric public static final String USED_COLD_SEGMENTS = "segment/used/deepStorageOnly/count"; // Compaction state cache metrics - public static final String COMPACTION_STATE_CACHE_HITS = COMPACTION_STATE_PREFIX + "hit/count"; - public static final String COMPACTION_STATE_CACHE_MISSES = COMPACTION_STATE_PREFIX + "miss/count"; - public static final String COMPACTION_STATE_CACHE_FINGERPRINTS = COMPACTION_STATE_PREFIX + "fingerprint/count"; + public static final String INDEXING_STATE_CACHE_HITS = INDEXING_STATE_PREFIX + "hit/count"; + public static final String INDEXING_STATE_CACHE_MISSES = INDEXING_STATE_PREFIX + "miss/count"; + public static final String INDEXING_STATE_CACHE_FINGERPRINTS = INDEXING_STATE_PREFIX + "fingerprint/count"; } diff --git a/server/src/main/java/org/apache/druid/segment/metadata/NoopCompactionStateCache.java b/server/src/main/java/org/apache/druid/segment/metadata/NoopIndexingStateCache.java similarity index 74% rename from server/src/main/java/org/apache/druid/segment/metadata/NoopCompactionStateCache.java rename to server/src/main/java/org/apache/druid/segment/metadata/NoopIndexingStateCache.java index e7e013b2addb..36bf8b2f260c 100644 --- a/server/src/main/java/org/apache/druid/segment/metadata/NoopCompactionStateCache.java +++ b/server/src/main/java/org/apache/druid/segment/metadata/NoopIndexingStateCache.java @@ -26,10 +26,9 @@ import java.util.Optional; /** - * No-op implementation of {@link CompactionStateCache} used on non-Overlord nodes - * where compaction state caching is not needed. + * No-op implementation of {@link IndexingStateCache} */ -public class NoopCompactionStateCache extends CompactionStateCache +public class NoopIndexingStateCache extends IndexingStateCache { @Override public boolean isEnabled() @@ -38,19 +37,19 @@ public boolean isEnabled() } @Override - public void resetCompactionStatesForPublishedSegments(Map fingerprintToStateMap) + public void resetIndexingStatesForPublishedSegments(Map fingerprintToStateMap) { // No-op } @Override - public Optional getCompactionStateByFingerprint(String fingerprint) + public Optional getIndexingStateByFingerprint(String fingerprint) { return Optional.empty(); } @Override - public Map getPublishedCompactionStateMap() + public Map getPublishedIndexingStateMap() { return Collections.emptyMap(); } diff --git a/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorReadOnlyTest.java b/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorReadOnlyTest.java index 22d3ab7c9b39..1617d6134109 100644 --- a/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorReadOnlyTest.java +++ b/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorReadOnlyTest.java @@ -37,7 +37,7 @@ import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; import org.apache.druid.segment.metadata.HeapMemoryCompactionStateStorage; -import org.apache.druid.segment.metadata.NoopCompactionStateCache; +import org.apache.druid.segment.metadata.NoopIndexingStateCache; import org.apache.druid.segment.metadata.NoopSegmentSchemaCache; import org.apache.druid.server.coordinator.simulate.BlockingExecutorService; import org.apache.druid.server.coordinator.simulate.TestDruidLeaderSelector; @@ -106,7 +106,7 @@ public void setup() () -> new SegmentsMetadataManagerConfig(null, cacheMode, null), derbyConnectorRule.metadataTablesConfigSupplier(), new NoopSegmentSchemaCache(), - new NoopCompactionStateCache(), + new NoopIndexingStateCache(), derbyConnector, (corePoolSize, nameFormat) -> new WrappingScheduledExecutorService( nameFormat, diff --git a/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java b/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java index 27bba0c6821f..e7e8d69ea259 100644 --- a/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java +++ b/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java @@ -49,7 +49,7 @@ import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; import org.apache.druid.segment.metadata.FingerprintGenerator; import org.apache.druid.segment.metadata.HeapMemoryCompactionStateStorage; -import org.apache.druid.segment.metadata.NoopCompactionStateCache; +import org.apache.druid.segment.metadata.NoopIndexingStateCache; import org.apache.druid.segment.metadata.NoopSegmentSchemaCache; import org.apache.druid.segment.metadata.SegmentSchemaManager; import org.apache.druid.segment.metadata.SegmentSchemaTestUtils; @@ -171,7 +171,7 @@ public void setUp() () -> new SegmentsMetadataManagerConfig(null, cacheMode, null), derbyConnectorRule.metadataTablesConfigSupplier(), new NoopSegmentSchemaCache(), - new NoopCompactionStateCache(), + new NoopIndexingStateCache(), derbyConnector, (corePoolSize, nameFormat) -> new WrappingScheduledExecutorService( nameFormat, diff --git a/server/src/test/java/org/apache/druid/metadata/segment/SqlSegmentsMetadataManagerV2Test.java b/server/src/test/java/org/apache/druid/metadata/segment/SqlSegmentsMetadataManagerV2Test.java index 8ccaf03b4254..293b076b1115 100644 --- a/server/src/test/java/org/apache/druid/metadata/segment/SqlSegmentsMetadataManagerV2Test.java +++ b/server/src/test/java/org/apache/druid/metadata/segment/SqlSegmentsMetadataManagerV2Test.java @@ -33,7 +33,7 @@ import org.apache.druid.metadata.segment.cache.SegmentMetadataCache; import org.apache.druid.segment.TestDataSource; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; -import org.apache.druid.segment.metadata.CompactionStateCache; +import org.apache.druid.segment.metadata.IndexingStateCache; import org.apache.druid.segment.metadata.NoopSegmentSchemaCache; import org.apache.druid.segment.metadata.SegmentSchemaCache; import org.apache.druid.server.coordinator.CreateDataSegments; @@ -93,7 +93,7 @@ private void initManager( Suppliers.ofInstance(new SegmentsMetadataManagerConfig(Period.seconds(1), cacheMode, null)), Suppliers.ofInstance(storageConfig), useSchemaCache ? new SegmentSchemaCache() : new NoopSegmentSchemaCache(), - new CompactionStateCache(), + new IndexingStateCache(), connector, (poolSize, name) -> new WrappingScheduledExecutorService(name, segmentMetadataCacheExec, false), emitter 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 ec837c9925d3..4b6e9d0e9bd3 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 @@ -38,8 +38,8 @@ import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; -import org.apache.druid.segment.metadata.CompactionStateCache; import org.apache.druid.segment.metadata.FingerprintGenerator; +import org.apache.druid.segment.metadata.IndexingStateCache; import org.apache.druid.segment.metadata.NoopSegmentSchemaCache; import org.apache.druid.segment.metadata.SegmentSchemaCache; import org.apache.druid.segment.metadata.SegmentSchemaTestUtils; @@ -77,7 +77,7 @@ public class HeapMemorySegmentMetadataCacheTest private HeapMemorySegmentMetadataCache cache; private SegmentSchemaCache schemaCache; - private CompactionStateCache compactionStateCache; + private IndexingStateCache indexingStateCache; private SegmentSchemaTestUtils schemaTestUtils; @Before @@ -122,13 +122,13 @@ private void setupTargetWithCaching(SegmentMetadataCache.UsageMode cacheMode, bo final SegmentsMetadataManagerConfig metadataManagerConfig = new SegmentsMetadataManagerConfig(null, cacheMode, null); schemaCache = useSchemaCache ? new SegmentSchemaCache() : new NoopSegmentSchemaCache(); - compactionStateCache = new CompactionStateCache(); + indexingStateCache = new IndexingStateCache(); cache = new HeapMemorySegmentMetadataCache( TestHelper.JSON_MAPPER, () -> metadataManagerConfig, derbyConnectorRule.metadataTablesConfigSupplier(), schemaCache, - compactionStateCache, + indexingStateCache, derbyConnector, executorFactory, serviceEmitter diff --git a/server/src/test/java/org/apache/druid/segment/metadata/CompactionStateCacheTest.java b/server/src/test/java/org/apache/druid/segment/metadata/IndexingStateCacheTest.java similarity index 51% rename from server/src/test/java/org/apache/druid/segment/metadata/CompactionStateCacheTest.java rename to server/src/test/java/org/apache/druid/segment/metadata/IndexingStateCacheTest.java index 233656855e31..4027edaf5f08 100644 --- a/server/src/test/java/org/apache/druid/segment/metadata/CompactionStateCacheTest.java +++ b/server/src/test/java/org/apache/druid/segment/metadata/IndexingStateCacheTest.java @@ -35,14 +35,14 @@ import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertTrue; -public class CompactionStateCacheTest +public class IndexingStateCacheTest { - private CompactionStateCache cache; + private IndexingStateCache cache; @BeforeEach public void setUp() { - cache = new CompactionStateCache(); + cache = new IndexingStateCache(); } @Test @@ -52,53 +52,53 @@ public void test_isEnabled_returnsTrue() } @Test - public void test_getCompactionStateByFingerprint_emptyCache_returnsEmpty() + public void test_getIndexingStateByFingerprint_emptyCache_returnsEmpty() { - Optional result = cache.getCompactionStateByFingerprint("nonexistent"); + Optional result = cache.getIndexingStateByFingerprint("nonexistent"); assertFalse(result.isPresent()); } @Test - public void test_getCompactionStateByFingerprint_nullFingerprint_returnsEmpty() + public void test_getIndexingStateByFingerprint_nullFingerprint_returnsEmpty() { - Optional result = cache.getCompactionStateByFingerprint(null); + Optional result = cache.getIndexingStateByFingerprint(null); assertFalse(result.isPresent()); } @Test - public void test_resetCompactionStatesForPublishedSegments_andThen_getCompactionStateByFingerprint() + public void test_resetIndexingStatesForPublishedSegments_andThen_getIndexingStateByFingerprint() { - CompactionState state1 = createTestCompactionState(); - CompactionState state2 = createTestCompactionState(); + CompactionState state1 = createTestIndexingState(); + CompactionState state2 = createTestIndexingState(); Map stateMap = new HashMap<>(); stateMap.put("fingerprint1", state1); stateMap.put("fingerprint2", state2); - cache.resetCompactionStatesForPublishedSegments(stateMap); + cache.resetIndexingStatesForPublishedSegments(stateMap); - Optional result1 = cache.getCompactionStateByFingerprint("fingerprint1"); + Optional result1 = cache.getIndexingStateByFingerprint("fingerprint1"); assertTrue(result1.isPresent()); assertEquals(state1, result1.get()); - Optional result2 = cache.getCompactionStateByFingerprint("fingerprint2"); + Optional result2 = cache.getIndexingStateByFingerprint("fingerprint2"); assertTrue(result2.isPresent()); assertEquals(state2, result2.get()); - Optional result3 = cache.getCompactionStateByFingerprint("nonexistent"); + Optional result3 = cache.getIndexingStateByFingerprint("nonexistent"); assertFalse(result3.isPresent()); } @Test - public void test_getPublishedCompactionStateMap_returnsImmutableSnapshot() + public void test_getPublishedIndexingStateMap_returnsImmutableSnapshot() { - CompactionState state1 = createTestCompactionState(); + CompactionState state1 = createTestIndexingState(); Map stateMap = new HashMap<>(); stateMap.put("fingerprint1", state1); - cache.resetCompactionStatesForPublishedSegments(stateMap); + cache.resetIndexingStatesForPublishedSegments(stateMap); - Map retrieved = cache.getPublishedCompactionStateMap(); + Map retrieved = cache.getPublishedIndexingStateMap(); assertEquals(1, retrieved.size()); assertEquals(state1, retrieved.get("fingerprint1")); } @@ -106,98 +106,98 @@ public void test_getPublishedCompactionStateMap_returnsImmutableSnapshot() @Test public void test_clear_emptiesCache() { - CompactionState state1 = createTestCompactionState(); + CompactionState state1 = createTestIndexingState(); Map stateMap = new HashMap<>(); stateMap.put("fingerprint1", state1); - cache.resetCompactionStatesForPublishedSegments(stateMap); + cache.resetIndexingStatesForPublishedSegments(stateMap); - Optional beforeClear = cache.getCompactionStateByFingerprint("fingerprint1"); + Optional beforeClear = cache.getIndexingStateByFingerprint("fingerprint1"); assertTrue(beforeClear.isPresent()); cache.clear(); - Optional afterClear = cache.getCompactionStateByFingerprint("fingerprint1"); + Optional afterClear = cache.getIndexingStateByFingerprint("fingerprint1"); assertFalse(afterClear.isPresent()); - Map mapAfterClear = cache.getPublishedCompactionStateMap(); + Map mapAfterClear = cache.getPublishedIndexingStateMap(); assertEquals(0, mapAfterClear.size()); } @Test public void test_stats_trackHitsAndMisses() { - CompactionState state1 = createTestCompactionState(); + CompactionState state1 = createTestIndexingState(); Map stateMap = new HashMap<>(); stateMap.put("fingerprint1", state1); - cache.resetCompactionStatesForPublishedSegments(stateMap); + cache.resetIndexingStatesForPublishedSegments(stateMap); // Generate 3 hits - cache.getCompactionStateByFingerprint("fingerprint1"); - cache.getCompactionStateByFingerprint("fingerprint1"); - cache.getCompactionStateByFingerprint("fingerprint1"); + cache.getIndexingStateByFingerprint("fingerprint1"); + cache.getIndexingStateByFingerprint("fingerprint1"); + cache.getIndexingStateByFingerprint("fingerprint1"); // Generate 2 misses - cache.getCompactionStateByFingerprint("nonexistent1"); - cache.getCompactionStateByFingerprint("nonexistent2"); + cache.getIndexingStateByFingerprint("nonexistent1"); + cache.getIndexingStateByFingerprint("nonexistent2"); Map stats = cache.getAndResetStats(); - assertEquals(3, stats.get(Metric.COMPACTION_STATE_CACHE_HITS)); - assertEquals(2, stats.get(Metric.COMPACTION_STATE_CACHE_MISSES)); - assertEquals(1, stats.get(Metric.COMPACTION_STATE_CACHE_FINGERPRINTS)); + assertEquals(3, stats.get(Metric.INDEXING_STATE_CACHE_HITS)); + assertEquals(2, stats.get(Metric.INDEXING_STATE_CACHE_MISSES)); + assertEquals(1, stats.get(Metric.INDEXING_STATE_CACHE_FINGERPRINTS)); } @Test public void test_stats_resetAfterReading() { - CompactionState state1 = createTestCompactionState(); + CompactionState state1 = createTestIndexingState(); Map stateMap = new HashMap<>(); stateMap.put("fingerprint1", state1); - cache.resetCompactionStatesForPublishedSegments(stateMap); + cache.resetIndexingStatesForPublishedSegments(stateMap); // Generate hits and misses - cache.getCompactionStateByFingerprint("fingerprint1"); - cache.getCompactionStateByFingerprint("nonexistent"); + cache.getIndexingStateByFingerprint("fingerprint1"); + cache.getIndexingStateByFingerprint("nonexistent"); Map stats1 = cache.getAndResetStats(); - assertEquals(1, stats1.get(Metric.COMPACTION_STATE_CACHE_HITS)); - assertEquals(1, stats1.get(Metric.COMPACTION_STATE_CACHE_MISSES)); + assertEquals(1, stats1.get(Metric.INDEXING_STATE_CACHE_HITS)); + assertEquals(1, stats1.get(Metric.INDEXING_STATE_CACHE_MISSES)); // Stats should be reset after reading Map stats2 = cache.getAndResetStats(); - assertEquals(0, stats2.get(Metric.COMPACTION_STATE_CACHE_HITS)); - assertEquals(0, stats2.get(Metric.COMPACTION_STATE_CACHE_MISSES)); - assertEquals(1, stats2.get(Metric.COMPACTION_STATE_CACHE_FINGERPRINTS)); // Fingerprints count doesn't reset + assertEquals(0, stats2.get(Metric.INDEXING_STATE_CACHE_HITS)); + assertEquals(0, stats2.get(Metric.INDEXING_STATE_CACHE_MISSES)); + assertEquals(1, stats2.get(Metric.INDEXING_STATE_CACHE_FINGERPRINTS)); // Fingerprints count doesn't reset } @Test public void test_multipleResets_replacesCache() { - CompactionState state1 = createTestCompactionState(); - CompactionState state2 = createTestCompactionState(); + CompactionState state1 = createTestIndexingState(); + CompactionState state2 = createTestIndexingState(); // First reset Map firstMap = new HashMap<>(); firstMap.put("fingerprint1", state1); - cache.resetCompactionStatesForPublishedSegments(firstMap); + cache.resetIndexingStatesForPublishedSegments(firstMap); - Optional result1 = cache.getCompactionStateByFingerprint("fingerprint1"); + Optional result1 = cache.getIndexingStateByFingerprint("fingerprint1"); assertTrue(result1.isPresent()); assertEquals(state1, result1.get()); // Second reset with different data Map secondMap = new HashMap<>(); secondMap.put("fingerprint2", state2); - cache.resetCompactionStatesForPublishedSegments(secondMap); + cache.resetIndexingStatesForPublishedSegments(secondMap); // Old fingerprint should be gone - Optional oldResult = cache.getCompactionStateByFingerprint("fingerprint1"); + Optional oldResult = cache.getIndexingStateByFingerprint("fingerprint1"); assertFalse(oldResult.isPresent()); // New fingerprint should exist - Optional newResult = cache.getCompactionStateByFingerprint("fingerprint2"); + Optional newResult = cache.getIndexingStateByFingerprint("fingerprint2"); assertTrue(newResult.isPresent()); assertEquals(state2, newResult.get()); } @@ -205,45 +205,45 @@ public void test_multipleResets_replacesCache() @Test public void test_resetWithEmptyMap() { - CompactionState state1 = createTestCompactionState(); + CompactionState state1 = createTestIndexingState(); Map stateMap = new HashMap<>(); stateMap.put("fingerprint1", state1); - cache.resetCompactionStatesForPublishedSegments(stateMap); + cache.resetIndexingStatesForPublishedSegments(stateMap); - Optional beforeReset = cache.getCompactionStateByFingerprint("fingerprint1"); + Optional beforeReset = cache.getIndexingStateByFingerprint("fingerprint1"); assertTrue(beforeReset.isPresent()); // Reset with empty map - cache.resetCompactionStatesForPublishedSegments(Collections.emptyMap()); + cache.resetIndexingStatesForPublishedSegments(Collections.emptyMap()); - Optional afterReset = cache.getCompactionStateByFingerprint("fingerprint1"); + Optional afterReset = cache.getIndexingStateByFingerprint("fingerprint1"); assertFalse(afterReset.isPresent()); Map stats = cache.getAndResetStats(); - assertEquals(0, stats.get(Metric.COMPACTION_STATE_CACHE_FINGERPRINTS)); + assertEquals(0, stats.get(Metric.INDEXING_STATE_CACHE_FINGERPRINTS)); } @Test - public void test_addCompactionState_addsNewStateToCache() + public void test_addIndexingState_addsNewStateToCache() { - CompactionState state = createTestCompactionState(); + CompactionState state = createTestIndexingState(); String fingerprint = "test_fingerprint_123"; // Initially, cache should not have the state - assertEquals(Optional.empty(), cache.getCompactionStateByFingerprint(fingerprint)); + assertEquals(Optional.empty(), cache.getIndexingStateByFingerprint(fingerprint)); // Add the state to cache - cache.addCompactionState(fingerprint, state); + cache.addIndexingState(fingerprint, state); // Now cache should have the state - assertEquals(Optional.of(state), cache.getCompactionStateByFingerprint(fingerprint)); + assertEquals(Optional.of(state), cache.getIndexingStateByFingerprint(fingerprint)); } @Test - public void test_addCompactionState_withDifferentStateForSameFingerprint_updatesCache() + public void test_addIndexingState_withDifferentStateForSameFingerprint_updatesCache() { - CompactionState state1 = createTestCompactionState(); + CompactionState state1 = createTestIndexingState(); CompactionState state2 = new CompactionState( new DynamicPartitionsSpec(200, null), DimensionsSpec.EMPTY, @@ -256,37 +256,37 @@ public void test_addCompactionState_withDifferentStateForSameFingerprint_updates String fingerprint = "same_fp"; // Add first state - cache.addCompactionState(fingerprint, state1); - assertEquals(Optional.of(state1), cache.getCompactionStateByFingerprint(fingerprint)); + cache.addIndexingState(fingerprint, state1); + assertEquals(Optional.of(state1), cache.getIndexingStateByFingerprint(fingerprint)); // Add different state with same fingerprint - cache.addCompactionState(fingerprint, state2); + cache.addIndexingState(fingerprint, state2); // Cache should now have the new state - assertEquals(Optional.of(state2), cache.getCompactionStateByFingerprint(fingerprint)); + assertEquals(Optional.of(state2), cache.getIndexingStateByFingerprint(fingerprint)); } @Test - public void test_addCompactionState_withNullFingerprint_doesNothing() + public void test_addIndexingState_withNullFingerprint_doesNothing() { - CompactionState state = createTestCompactionState(); + CompactionState state = createTestIndexingState(); - cache.addCompactionState(null, state); + cache.addIndexingState(null, state); // Cache should remain empty - assertEquals(0, cache.getPublishedCompactionStateMap().size()); + assertEquals(0, cache.getPublishedIndexingStateMap().size()); } @Test - public void test_addCompactionState_withNullState_doesNothing() + public void test_addIndexingState_withNullState_doesNothing() { - cache.addCompactionState("some_fp", null); + cache.addIndexingState("some_fp", null); // Cache should remain empty - assertEquals(0, cache.getPublishedCompactionStateMap().size()); + assertEquals(0, cache.getPublishedIndexingStateMap().size()); } - private CompactionState createTestCompactionState() + private CompactionState createTestIndexingState() { return new CompactionState( new DynamicPartitionsSpec(100, null), diff --git a/server/src/test/java/org/apache/druid/segment/metadata/SqlCompactionStateStorageTest.java b/server/src/test/java/org/apache/druid/segment/metadata/SqlCompactionStateStorageTest.java index a6764c7edf16..09b66b12923f 100644 --- a/server/src/test/java/org/apache/druid/segment/metadata/SqlCompactionStateStorageTest.java +++ b/server/src/test/java/org/apache/druid/segment/metadata/SqlCompactionStateStorageTest.java @@ -71,7 +71,7 @@ public static void setUpClass() derbyConnector.createIndexingStatesTable(); derbyConnector.createSegmentTable(); fingerprintMapper = new DefaultCompactionFingerprintMapper( - new NoopCompactionStateCache(), + new NoopIndexingStateCache(), DETERMINISTIC_MAPPER ); } 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 a9714a080267..aad3c9d15571 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 @@ -40,10 +40,10 @@ import org.apache.druid.segment.TestDataSource; import org.apache.druid.segment.data.CompressionStrategy; import org.apache.druid.segment.metadata.CompactionFingerprintMapper; -import org.apache.druid.segment.metadata.CompactionStateCache; import org.apache.druid.segment.metadata.CompactionTestUtils; import org.apache.druid.segment.metadata.DefaultCompactionFingerprintMapper; import org.apache.druid.segment.metadata.HeapMemoryCompactionStateStorage; +import org.apache.druid.segment.metadata.IndexingStateCache; import org.apache.druid.segment.nested.NestedCommonFormatColumnFormatSpec; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; import org.apache.druid.server.coordinator.InlineSchemaDataSourceCompactionConfig; @@ -72,16 +72,16 @@ public class CompactionStatusTest .build(); private HeapMemoryCompactionStateStorage compactionStateStorage; - private CompactionStateCache compactionStateCache; + private IndexingStateCache indexingStateCache; private CompactionFingerprintMapper fingerprintMapper; @Before public void setUp() { compactionStateStorage = new HeapMemoryCompactionStateStorage(); - compactionStateCache = new CompactionStateCache(); + indexingStateCache = new IndexingStateCache(); fingerprintMapper = new DefaultCompactionFingerprintMapper( - compactionStateCache, + indexingStateCache, CompactionTestUtils.createDeterministicMapper() ); } @@ -91,7 +91,7 @@ public void setUp() */ private void syncCacheFromManager() { - compactionStateCache.resetCompactionStatesForPublishedSegments(compactionStateStorage.getAllStoredStates()); + indexingStateCache.resetIndexingStatesForPublishedSegments(compactionStateStorage.getAllStoredStates()); } @Test 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 7669aae3224c..45da1b14a3ae 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 @@ -30,7 +30,6 @@ import org.apache.druid.indexer.granularity.UniformGranularitySpec; import org.apache.druid.indexer.partitions.DimensionRangePartitionsSpec; import org.apache.druid.indexer.partitions.PartitionsSpec; -import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.granularity.Granularities; @@ -48,8 +47,9 @@ import org.apache.druid.segment.data.ConciseBitmapSerdeFactory; import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.segment.metadata.CompactionFingerprintMapper; +import org.apache.druid.segment.metadata.CompactionTestUtils; import org.apache.druid.segment.metadata.DefaultCompactionFingerprintMapper; -import org.apache.druid.segment.metadata.NoopCompactionStateCache; +import org.apache.druid.segment.metadata.NoopIndexingStateCache; import org.apache.druid.segment.transform.CompactionTransformSpec; import org.apache.druid.server.coordinator.CreateDataSegments; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; @@ -82,9 +82,9 @@ public class NewestSegmentFirstPolicyTest { private static final int DEFAULT_NUM_SEGMENTS_PER_SHARD = 4; - private final ObjectMapper mapper = new DefaultObjectMapper(); + private final ObjectMapper mapper = CompactionTestUtils.createDeterministicMapper(); private final NewestSegmentFirstPolicy policy = new NewestSegmentFirstPolicy(null); - private final CompactionFingerprintMapper fingerprintMapper = new DefaultCompactionFingerprintMapper(new NoopCompactionStateCache(), mapper); // TODO fix + private final CompactionFingerprintMapper fingerprintMapper = new DefaultCompactionFingerprintMapper(new NoopIndexingStateCache(), mapper); @Test public void testLargeOffsetAndSmallSegmentInterval() 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 205de59a27ff..09c266222663 100644 --- a/services/src/main/java/org/apache/druid/guice/MetadataManagerModule.java +++ b/services/src/main/java/org/apache/druid/guice/MetadataManagerModule.java @@ -41,8 +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.CompactionStateCache; import org.apache.druid.segment.metadata.CompactionStateStorage; +import org.apache.druid.segment.metadata.IndexingStateCache; import org.apache.druid.segment.metadata.NoopSegmentSchemaCache; import org.apache.druid.segment.metadata.SegmentSchemaCache; import org.apache.druid.segment.metadata.SqlCompactionStateStorage; @@ -62,7 +62,7 @@ *

  • {@link IndexerMetadataStorageCoordinator}
  • *
  • {@link CoordinatorConfigManager}
  • *
  • {@link SegmentMetadataCache}
  • - *
  • {@link CompactionStateCache} - Overlord only
  • + *
  • {@link IndexingStateCache} - Overlord only
  • *
  • {@link SegmentSchemaCache} - Coordinator only
  • *
  • {@link SqlCompactionStateStorage}
  • * @@ -136,7 +136,7 @@ public void configure(Binder binder) binder.bind(SegmentMetadataTransactionFactory.class) .to(SqlSegmentMetadataTransactionFactory.class) .in(LazySingleton.class); - binder.bind(CompactionStateCache.class).in(LazySingleton.class); + binder.bind(IndexingStateCache.class).in(LazySingleton.class); } else { binder.bind(SegmentMetadataTransactionFactory.class) .to(SqlSegmentMetadataReadOnlyTransactionFactory.class) From 5af3a5b78e923813a78547040c2f07e3175521fb Mon Sep 17 00:00:00 2001 From: capistrant Date: Wed, 14 Jan 2026 12:32:17 -0600 Subject: [PATCH 64/72] Rename CompactionStateStorage to IndexingStateStorage --- .../MaterializedViewSupervisorTest.java | 4 +- .../DatasourceOptimizerTest.java | 4 +- .../indexing/compact/CompactionJobQueue.java | 18 ++--- .../compact/OverlordCompactionScheduler.java | 10 +-- .../duty/KillUnreferencedIndexingState.java | 18 ++--- .../common/actions/TaskActionTestKit.java | 4 +- .../common/task/IngestionTestBase.java | 4 +- .../OverlordCompactionSchedulerTest.java | 4 +- .../overlord/GlobalTaskLockboxTest.java | 6 +- .../overlord/TaskLockBoxConcurrencyTest.java | 4 +- .../indexing/overlord/TaskQueueScaleTest.java | 4 +- .../KillUnreferencedIndexingStateTest.java | 54 +++++++------- .../SeekableStreamIndexTaskTestBase.java | 4 +- .../IndexerSQLMetadataStorageCoordinator.java | 10 +-- .../DefaultCompactionFingerprintMapper.java | 2 +- .../segment/metadata/IndexingStateCache.java | 2 +- ...Storage.java => IndexingStateStorage.java} | 38 +++++----- ...rage.java => SqlIndexingStateStorage.java} | 48 ++++++------ ...etadataStorageCoordinatorMarkUsedTest.java | 4 +- ...etadataStorageCoordinatorReadOnlyTest.java | 4 +- ...exerSQLMetadataStorageCoordinatorTest.java | 30 ++++---- ...orageCoordinatorSchemaPersistenceTest.java | 4 +- .../SqlSegmentsMetadataQueryTest.java | 6 +- ...va => HeapMemoryIndexingStateStorage.java} | 30 ++++---- ....java => SqlIndexingStateStorageTest.java} | 74 +++++++++---------- .../compaction/CompactionStatusTest.java | 16 ++-- .../duty/KillUnusedSegmentsTest.java | 4 +- .../druid/guice/MetadataManagerModule.java | 10 +-- 28 files changed, 213 insertions(+), 207 deletions(-) rename server/src/main/java/org/apache/druid/segment/metadata/{CompactionStateStorage.java => IndexingStateStorage.java} (67%) rename server/src/main/java/org/apache/druid/segment/metadata/{SqlCompactionStateStorage.java => SqlIndexingStateStorage.java} (88%) rename server/src/test/java/org/apache/druid/segment/metadata/{HeapMemoryCompactionStateStorage.java => HeapMemoryIndexingStateStorage.java} (69%) rename server/src/test/java/org/apache/druid/segment/metadata/{SqlCompactionStateStorageTest.java => SqlIndexingStateStorageTest.java} (88%) diff --git a/extensions-contrib/materialized-view-maintenance/src/test/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisorTest.java b/extensions-contrib/materialized-view-maintenance/src/test/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisorTest.java index 2552f6092e7b..212a1e8529de 100644 --- a/extensions-contrib/materialized-view-maintenance/src/test/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisorTest.java +++ b/extensions-contrib/materialized-view-maintenance/src/test/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisorTest.java @@ -53,7 +53,7 @@ import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; -import org.apache.druid.segment.metadata.HeapMemoryCompactionStateStorage; +import org.apache.druid.segment.metadata.HeapMemoryIndexingStateStorage; import org.apache.druid.segment.metadata.SegmentSchemaManager; import org.apache.druid.segment.realtime.ChatHandlerProvider; import org.apache.druid.server.coordinator.simulate.TestDruidLeaderSelector; @@ -122,7 +122,7 @@ public void setUp() derbyConnector, segmentSchemaManager, CentralizedDatasourceSchemaConfig.create(), - new HeapMemoryCompactionStateStorage() + new HeapMemoryIndexingStateStorage() ); metadataSupervisorManager = EasyMock.createMock(MetadataSupervisorManager.class); taskQueue = EasyMock.createMock(TaskQueue.class); diff --git a/extensions-contrib/materialized-view-selection/src/test/java/org/apache/druid/query/materializedview/DatasourceOptimizerTest.java b/extensions-contrib/materialized-view-selection/src/test/java/org/apache/druid/query/materializedview/DatasourceOptimizerTest.java index dce71b491520..7ce32d1f1f1a 100644 --- a/extensions-contrib/materialized-view-selection/src/test/java/org/apache/druid/query/materializedview/DatasourceOptimizerTest.java +++ b/extensions-contrib/materialized-view-selection/src/test/java/org/apache/druid/query/materializedview/DatasourceOptimizerTest.java @@ -55,7 +55,7 @@ import org.apache.druid.query.topn.TopNQueryBuilder; import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; -import org.apache.druid.segment.metadata.HeapMemoryCompactionStateStorage; +import org.apache.druid.segment.metadata.HeapMemoryIndexingStateStorage; import org.apache.druid.segment.metadata.SegmentSchemaManager; import org.apache.druid.segment.realtime.appenderator.SegmentSchemas; import org.apache.druid.server.coordination.DruidServerMetadata; @@ -131,7 +131,7 @@ public void setUp() throws Exception derbyConnector, segmentSchemaManager, CentralizedDatasourceSchemaConfig.create(), - new HeapMemoryCompactionStateStorage() + new HeapMemoryIndexingStateStorage() ); setupServerAndCurator(); 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 4b98282eae7e..f4ef9a61a741 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,7 +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.CompactionStateStorage; +import org.apache.druid.segment.metadata.IndexingStateStorage; import org.apache.druid.segment.metadata.DefaultCompactionFingerprintMapper; import org.apache.druid.segment.metadata.IndexingStateCache; import org.apache.druid.server.compaction.CompactionCandidate; @@ -99,7 +99,7 @@ public class CompactionJobQueue private final Set activeSupervisors; private final Map submittedTaskIdToJob; - private final CompactionStateStorage compactionStateStorage; + private final IndexingStateStorage indexingStateStorage; private final IndexingStateCache indexingStateCache; public CompactionJobQueue( @@ -111,7 +111,7 @@ public CompactionJobQueue( OverlordClient overlordClient, BrokerClient brokerClient, ObjectMapper objectMapper, - CompactionStateStorage compactionStateStorage, + IndexingStateStorage indexingStateStorage, IndexingStateCache indexingStateCache, ObjectMapper deterministicCompactionStateMapper ) @@ -133,7 +133,7 @@ public CompactionJobQueue( new DefaultCompactionFingerprintMapper(indexingStateCache, deterministicCompactionStateMapper) ); - this.compactionStateStorage = compactionStateStorage; + this.indexingStateStorage = indexingStateStorage; this.indexingStateCache = indexingStateCache; this.taskActionClientFactory = taskActionClientFactory; @@ -328,7 +328,7 @@ private String startTaskIfReady(CompactionJob job) // Assume MSQ jobs to be always ready if (job.isMsq()) { try { - persistPendingCompactionState(job); + persistPendingIndexingState(job); return FutureUtils.getUnchecked(brokerClient.submitSqlTask(job.getNonNullMsqQuery()), true) .getTaskId(); } @@ -347,7 +347,7 @@ private String startTaskIfReady(CompactionJob job) try { taskLockbox.add(task); if (task.isReady(taskActionClientFactory.create(task))) { - persistPendingCompactionState(job); + persistPendingIndexingState(job); // Hold the locks acquired by task.isReady() as we will reacquire them anyway FutureUtils.getUnchecked(overlordClient.runTask(task.getId(), task), true); return task.getId(); @@ -364,12 +364,12 @@ private String startTaskIfReady(CompactionJob job) } /** - * Persist the compaction state associated with the given job with {@link CompactionStateStorage}. + * Persist the indexing state associated with the given job with {@link IndexingStateStorage}. */ - private void persistPendingCompactionState(CompactionJob job) + private void persistPendingIndexingState(CompactionJob job) { if (job.getTargetCompactionState() != null && job.getTargetCompactionStateFingerprint() != null) { - compactionStateStorage.upsertCompactionState( + indexingStateStorage.upsertIndexingState( job.getDataSource(), job.getTargetCompactionStateFingerprint(), job.getTargetCompactionState(), 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 c7606d5e5608..e2906a7fabef 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 @@ -46,7 +46,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.CompactionStateStorage; +import org.apache.druid.segment.metadata.IndexingStateStorage; import org.apache.druid.segment.metadata.IndexingStateCache; import org.apache.druid.server.compaction.CompactionRunSimulator; import org.apache.druid.server.compaction.CompactionSimulateResult; @@ -143,7 +143,7 @@ public class OverlordCompactionScheduler implements CompactionScheduler private final boolean shouldPollSegments; private final long schedulePeriodMillis; - private final CompactionStateStorage compactionStateStorage; + private final IndexingStateStorage indexingStateStorage; private final IndexingStateCache indexingStateCache; private final ObjectMapper deterministicMapper; @@ -163,7 +163,7 @@ public OverlordCompactionScheduler( BrokerClient brokerClient, ServiceEmitter emitter, ObjectMapper objectMapper, - CompactionStateStorage compactionStateStorage, + IndexingStateStorage indexingStateStorage, IndexingStateCache indexingStateCache, @Deterministic ObjectMapper deterministicMapper ) @@ -191,7 +191,7 @@ public OverlordCompactionScheduler( this.taskActionClientFactory = taskActionClientFactory; this.druidInputSourceFactory = druidInputSourceFactory; - this.compactionStateStorage = compactionStateStorage; + this.indexingStateStorage = indexingStateStorage; this.indexingStateCache = indexingStateCache; this.deterministicMapper = deterministicMapper; this.taskRunnerListener = new TaskRunnerListener() @@ -392,7 +392,7 @@ private synchronized void resetCompactionJobQueue() overlordClient, brokerClient, objectMapper, - compactionStateStorage, + indexingStateStorage, indexingStateCache, deterministicMapper ); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/duty/KillUnreferencedIndexingState.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/duty/KillUnreferencedIndexingState.java index cd38e206bf93..b1afb6df2911 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/duty/KillUnreferencedIndexingState.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/duty/KillUnreferencedIndexingState.java @@ -21,7 +21,7 @@ import org.apache.druid.indexing.overlord.config.OverlordMetadataCleanupConfig; import org.apache.druid.java.util.common.logger.Logger; -import org.apache.druid.segment.metadata.CompactionStateStorage; +import org.apache.druid.segment.metadata.IndexingStateStorage; import org.joda.time.DateTime; import javax.inject.Inject; @@ -30,39 +30,39 @@ public class KillUnreferencedIndexingState extends OverlordMetadataCleanupDuty { private static final Logger log = new Logger(KillUnreferencedIndexingState.class); - private final CompactionStateStorage compactionStateStorage; + private final IndexingStateStorage indexingStateStorage; @Inject public KillUnreferencedIndexingState( OverlordMetadataCleanupConfig config, - CompactionStateStorage compactionStateStorage + IndexingStateStorage indexingStateStorage ) { super("indexingStates", config); - this.compactionStateStorage = compactionStateStorage; + this.indexingStateStorage = indexingStateStorage; } @Override protected int cleanupEntriesCreatedBeforeDurationToRetain(DateTime minCreatedTime) { // 1: Mark unreferenced states as unused - int unused = compactionStateStorage.markUnreferencedCompactionStatesAsUnused(); + int unused = indexingStateStorage.markUnreferencedIndexingStatesAsUnused(); log.info("Marked [%s] unreferenced indexing states as unused.", unused); // 2: Repair - find unused states still referenced by segments - List stateFingerprints = compactionStateStorage.findReferencedCompactionStateMarkedAsUnused(); + List stateFingerprints = indexingStateStorage.findReferencedIndexingStateMarkedAsUnused(); if (!stateFingerprints.isEmpty()) { - int numUpdated = compactionStateStorage.markCompactionStatesAsUsed(stateFingerprints); + int numUpdated = indexingStateStorage.markIndexingStatesAsUsed(stateFingerprints); log.info("Marked [%s] unused indexing states referenced by used segments as used.", numUpdated); } // 3: Delete unused states older than threshold - return compactionStateStorage.deleteUnusedCompactionStatesOlderThan(minCreatedTime.getMillis()); + return indexingStateStorage.deleteUnusedIndexingStatesOlderThan(minCreatedTime.getMillis()); } @Override protected int cleanupEntriesCreatedBeforePendingDurationToRetain(DateTime minCreatedTime) { - return compactionStateStorage.deletePendingCompactionStatesOlderThan(minCreatedTime.getMillis()); + return indexingStateStorage.deletePendingIndexingStatesOlderThan(minCreatedTime.getMillis()); } } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/TaskActionTestKit.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/TaskActionTestKit.java index 59a59213b0fe..cb78e3bb33b5 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/TaskActionTestKit.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/TaskActionTestKit.java @@ -41,7 +41,7 @@ import org.apache.druid.metadata.segment.cache.HeapMemorySegmentMetadataCache; import org.apache.druid.metadata.segment.cache.SegmentMetadataCache; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; -import org.apache.druid.segment.metadata.HeapMemoryCompactionStateStorage; +import org.apache.druid.segment.metadata.HeapMemoryIndexingStateStorage; import org.apache.druid.segment.metadata.IndexingStateCache; import org.apache.druid.segment.metadata.NoopSegmentSchemaCache; import org.apache.druid.segment.metadata.SegmentSchemaManager; @@ -137,7 +137,7 @@ public void before() testDerbyConnector, segmentSchemaManager, CentralizedDatasourceSchemaConfig.create(), - new HeapMemoryCompactionStateStorage() + new HeapMemoryIndexingStateStorage() ); taskLockbox = new GlobalTaskLockbox(taskStorage, metadataStorageCoordinator); taskLockbox.syncFromStorage(); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IngestionTestBase.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IngestionTestBase.java index 5329fb7fd8ef..abe2d16e0942 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IngestionTestBase.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IngestionTestBase.java @@ -87,7 +87,7 @@ import org.apache.druid.segment.loading.LocalDataSegmentPusherConfig; import org.apache.druid.segment.loading.SegmentCacheManager; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; -import org.apache.druid.segment.metadata.HeapMemoryCompactionStateStorage; +import org.apache.druid.segment.metadata.HeapMemoryIndexingStateStorage; import org.apache.druid.segment.metadata.IndexingStateCache; import org.apache.druid.segment.metadata.SegmentSchemaCache; import org.apache.druid.segment.metadata.SegmentSchemaManager; @@ -184,7 +184,7 @@ public void setUpIngestionTestBase() throws IOException derbyConnectorRule.getConnector(), segmentSchemaManager, CentralizedDatasourceSchemaConfig.create(), - new HeapMemoryCompactionStateStorage() + new HeapMemoryIndexingStateStorage() ); segmentsMetadataManager = new SqlSegmentsMetadataManagerV2( segmentMetadataCache, 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 55d5b85539f8..4bb0a2df2a4f 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,7 +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.HeapMemoryCompactionStateStorage; +import org.apache.druid.segment.metadata.HeapMemoryIndexingStateStorage; import org.apache.druid.segment.metadata.IndexingStateCache; import org.apache.druid.server.compaction.CompactionSimulateResult; import org.apache.druid.server.compaction.CompactionStatistics; @@ -234,7 +234,7 @@ private void initScheduler() brokerClient, serviceEmitter, OBJECT_MAPPER, - new HeapMemoryCompactionStateStorage(), + new HeapMemoryIndexingStateStorage(), new IndexingStateCache(), OBJECT_MAPPER // TODO fix ); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/GlobalTaskLockboxTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/GlobalTaskLockboxTest.java index c3313c20cd6e..42e6e365b306 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/GlobalTaskLockboxTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/GlobalTaskLockboxTest.java @@ -60,7 +60,7 @@ import org.apache.druid.metadata.segment.cache.NoopSegmentMetadataCache; import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; -import org.apache.druid.segment.metadata.HeapMemoryCompactionStateStorage; +import org.apache.druid.segment.metadata.HeapMemoryIndexingStateStorage; import org.apache.druid.segment.metadata.SegmentSchemaManager; import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; import org.apache.druid.server.coordinator.simulate.TestDruidLeaderSelector; @@ -147,7 +147,7 @@ public void setup() derbyConnector, segmentSchemaManager, CentralizedDatasourceSchemaConfig.create(), - new HeapMemoryCompactionStateStorage() + new HeapMemoryIndexingStateStorage() ); lockbox = new GlobalTaskLockbox(taskStorage, metadataStorageCoordinator); @@ -494,7 +494,7 @@ public void testSyncWithUnknownTaskTypesFromModuleNotLoaded() derbyConnector, segmentSchemaManager, CentralizedDatasourceSchemaConfig.create(), - new HeapMemoryCompactionStateStorage() + new HeapMemoryIndexingStateStorage() ); GlobalTaskLockbox theBox = new GlobalTaskLockbox(taskStorage, metadataStorageCoordinator); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLockBoxConcurrencyTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLockBoxConcurrencyTest.java index e96a6b0723b1..3515790e4a2b 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLockBoxConcurrencyTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLockBoxConcurrencyTest.java @@ -41,7 +41,7 @@ import org.apache.druid.metadata.segment.SqlSegmentMetadataTransactionFactory; import org.apache.druid.metadata.segment.cache.NoopSegmentMetadataCache; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; -import org.apache.druid.segment.metadata.HeapMemoryCompactionStateStorage; +import org.apache.druid.segment.metadata.HeapMemoryIndexingStateStorage; import org.apache.druid.segment.metadata.SegmentSchemaManager; import org.apache.druid.server.coordinator.simulate.TestDruidLeaderSelector; import org.apache.druid.server.metrics.NoopServiceEmitter; @@ -108,7 +108,7 @@ public void setup() derbyConnector, segmentSchemaManager, CentralizedDatasourceSchemaConfig.create(), - new HeapMemoryCompactionStateStorage() + new HeapMemoryIndexingStateStorage() ) ); lockbox.syncFromStorage(); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskQueueScaleTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskQueueScaleTest.java index 3a76f415940f..190d78bb33c9 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskQueueScaleTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskQueueScaleTest.java @@ -51,7 +51,7 @@ import org.apache.druid.metadata.segment.cache.NoopSegmentMetadataCache; import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; -import org.apache.druid.segment.metadata.HeapMemoryCompactionStateStorage; +import org.apache.druid.segment.metadata.HeapMemoryIndexingStateStorage; import org.apache.druid.segment.metadata.SegmentSchemaManager; import org.apache.druid.server.coordinator.simulate.TestDruidLeaderSelector; import org.apache.druid.server.metrics.NoopServiceEmitter; @@ -120,7 +120,7 @@ public void setUp() derbyConnectorRule.getConnector(), segmentSchemaManager, CentralizedDatasourceSchemaConfig.create(), - new HeapMemoryCompactionStateStorage() + new HeapMemoryIndexingStateStorage() ); final TaskActionClientFactory unsupportedTaskActionFactory = diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/duty/KillUnreferencedIndexingStateTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/duty/KillUnreferencedIndexingStateTest.java index 7f2cdd5aa3d3..59ab2077d9c5 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/duty/KillUnreferencedIndexingStateTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/duty/KillUnreferencedIndexingStateTest.java @@ -27,8 +27,8 @@ import org.apache.druid.metadata.MetadataStorageTablesConfig; import org.apache.druid.metadata.TestDerbyConnector; import org.apache.druid.segment.IndexSpec; -import org.apache.druid.segment.metadata.CompactionStateStorage; -import org.apache.druid.segment.metadata.SqlCompactionStateStorage; +import org.apache.druid.segment.metadata.IndexingStateStorage; +import org.apache.druid.segment.metadata.SqlIndexingStateStorage; import org.apache.druid.timeline.CompactionState; import org.joda.time.DateTime; import org.joda.time.Period; @@ -50,7 +50,7 @@ public class KillUnreferencedIndexingStateTest private TestDerbyConnector derbyConnector; private MetadataStorageTablesConfig tablesConfig; - private SqlCompactionStateStorage compactionStateStorage; + private SqlIndexingStateStorage compactionStateStorage; @Before public void setUp() @@ -61,7 +61,7 @@ public void setUp() derbyConnector.createIndexingStatesTable(); derbyConnector.createSegmentTable(); - compactionStateStorage = new SqlCompactionStateStorage(tablesConfig, jsonMapper, derbyConnector); + compactionStateStorage = new SqlIndexingStateStorage(tablesConfig, jsonMapper, derbyConnector); } @Test @@ -87,8 +87,8 @@ public void test_killUnreferencedCompactionState_validateLifecycleOfActiveCompac String fingerprint = "test_fingerprint"; CompactionState state = createTestCompactionState(); - compactionStateStorage.upsertCompactionState("test-ds", fingerprint, state, DateTimes.nowUtc()); - compactionStateStorage.markCompactionStatesAsActive(fingerprint); + compactionStateStorage.upsertIndexingState("test-ds", fingerprint, state, DateTimes.nowUtc()); + compactionStateStorage.markIndexingStatesAsActive(fingerprint); Assert.assertEquals(Boolean.TRUE, getCompactionStateUsedStatus(fingerprint)); @@ -127,8 +127,8 @@ public void test_killUnreferencedCompactionState_validateRepair() String fingerprint = "repair_fingerprint"; CompactionState state = createTestCompactionState(); - compactionStateStorage.upsertCompactionState("test-ds", fingerprint, state, DateTimes.nowUtc()); - compactionStateStorage.markCompactionStatesAsActive(fingerprint); + compactionStateStorage.upsertIndexingState("test-ds", fingerprint, state, DateTimes.nowUtc()); + compactionStateStorage.markIndexingStatesAsActive(fingerprint); Assert.assertEquals(Boolean.TRUE, getCompactionStateUsedStatus(fingerprint)); duty.run(); @@ -178,8 +178,8 @@ public void test_killUnreferencedCompactionState_disabled() // Insert compaction state String fingerprint = "disabled_fingerprint"; - compactionStateStorage.upsertCompactionState("test-ds", fingerprint, createTestCompactionState(), DateTimes.nowUtc()); - compactionStateStorage.markCompactionStatesAsActive(fingerprint); + compactionStateStorage.upsertIndexingState("test-ds", fingerprint, createTestCompactionState(), DateTimes.nowUtc()); + compactionStateStorage.markIndexingStatesAsActive(fingerprint); // Run duty - should do nothing duty.run(); @@ -208,15 +208,15 @@ public void test_killUnreferencedCompactionState_validateLifecycleOfPendingCompa String fingerprint = "pending_fingerprint"; CompactionState state = createTestCompactionState(); - compactionStateStorage.upsertCompactionState("test-ds", fingerprint, state, DateTimes.nowUtc()); + compactionStateStorage.upsertIndexingState("test-ds", fingerprint, state, DateTimes.nowUtc()); - Assert.assertEquals(Boolean.TRUE, compactionStateStorage.isCompactionStatePending(fingerprint)); + Assert.assertEquals(Boolean.TRUE, compactionStateStorage.isIndexingStatePending(fingerprint)); duty.run(); - Assert.assertNotNull(compactionStateStorage.isCompactionStatePending(fingerprint)); + Assert.assertNotNull(compactionStateStorage.isIndexingStatePending(fingerprint)); duty.run(); - Assert.assertNull(compactionStateStorage.isCompactionStatePending(fingerprint)); + Assert.assertNull(compactionStateStorage.isIndexingStatePending(fingerprint)); } /** @@ -244,20 +244,20 @@ public void test_killUnreferencedCompactionState_validateMixedPendingAndActiveCo String nonPendingFingerprint = "non_pending_fp"; CompactionState state = createTestCompactionState(); - compactionStateStorage.upsertCompactionState("test-ds", pendingFingerprint, state, DateTimes.nowUtc()); - compactionStateStorage.upsertCompactionState("test-ds", nonPendingFingerprint, state, DateTimes.nowUtc()); - compactionStateStorage.markCompactionStatesAsActive(nonPendingFingerprint); + compactionStateStorage.upsertIndexingState("test-ds", pendingFingerprint, state, DateTimes.nowUtc()); + compactionStateStorage.upsertIndexingState("test-ds", nonPendingFingerprint, state, DateTimes.nowUtc()); + compactionStateStorage.markIndexingStatesAsActive(nonPendingFingerprint); - Assert.assertEquals(Boolean.TRUE, compactionStateStorage.isCompactionStatePending(pendingFingerprint)); + Assert.assertEquals(Boolean.TRUE, compactionStateStorage.isIndexingStatePending(pendingFingerprint)); Assert.assertNotNull(getCompactionStateUsedStatus(nonPendingFingerprint)); duty.run(); - Assert.assertNotNull(compactionStateStorage.isCompactionStatePending(pendingFingerprint)); + Assert.assertNotNull(compactionStateStorage.isIndexingStatePending(pendingFingerprint)); Assert.assertNull(getCompactionStateUsedStatus(nonPendingFingerprint)); duty.run(); Assert.assertNull(getCompactionStateUsedStatus(nonPendingFingerprint)); - Assert.assertNull(compactionStateStorage.isCompactionStatePending(pendingFingerprint)); + Assert.assertNull(compactionStateStorage.isIndexingStatePending(pendingFingerprint)); } @Test @@ -280,8 +280,8 @@ public void test_killUnreferencedCompactionState_pendingStateMarkedActiveNotDele String fingerprint = "pending_marked_active_fp"; CompactionState state = createTestCompactionState(); - compactionStateStorage.upsertCompactionState("test-ds", fingerprint, state, DateTimes.nowUtc()); - Assert.assertEquals(Boolean.TRUE, compactionStateStorage.isCompactionStatePending(fingerprint)); + compactionStateStorage.upsertIndexingState("test-ds", fingerprint, state, DateTimes.nowUtc()); + Assert.assertEquals(Boolean.TRUE, compactionStateStorage.isIndexingStatePending(fingerprint)); // Now insert a used segment that references this fingerprint derbyConnector.retryWithHandle(handle -> { @@ -307,11 +307,11 @@ public void test_killUnreferencedCompactionState_pendingStateMarkedActiveNotDele return null; }); - compactionStateStorage.markCompactionStatesAsActive(fingerprint); - Assert.assertNotEquals(Boolean.TRUE, compactionStateStorage.isCompactionStatePending(fingerprint)); + compactionStateStorage.markIndexingStatesAsActive(fingerprint); + Assert.assertNotEquals(Boolean.TRUE, compactionStateStorage.isIndexingStatePending(fingerprint)); duty.run(); - Assert.assertNotNull(compactionStateStorage.isCompactionStatePending(fingerprint)); + Assert.assertNotNull(compactionStateStorage.isIndexingStatePending(fingerprint)); } private Boolean getCompactionStateUsedStatus(String fingerprint) @@ -341,11 +341,11 @@ private static class TestKillUnreferencedIndexingState extends KillUnreferencedI public TestKillUnreferencedIndexingState( OverlordMetadataCleanupConfig config, - CompactionStateStorage compactionStateStorage, + IndexingStateStorage indexingStateStorage, List dateTimes ) { - super(config, compactionStateStorage); + super(config, indexingStateStorage); this.dateTimes = dateTimes; } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskTestBase.java b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskTestBase.java index 4bf36fbaecb1..ada47f04d07e 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskTestBase.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskTestBase.java @@ -120,7 +120,7 @@ import org.apache.druid.segment.loading.LocalDataSegmentPusher; import org.apache.druid.segment.loading.LocalDataSegmentPusherConfig; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; -import org.apache.druid.segment.metadata.HeapMemoryCompactionStateStorage; +import org.apache.druid.segment.metadata.HeapMemoryIndexingStateStorage; import org.apache.druid.segment.metadata.SegmentSchemaManager; import org.apache.druid.segment.realtime.NoopChatHandlerProvider; import org.apache.druid.segment.realtime.SegmentGenerationMetrics; @@ -638,7 +638,7 @@ protected void makeToolboxFactory(TestUtils testUtils, ServiceEmitter emitter, b derbyConnector, segmentSchemaManager, CentralizedDatasourceSchemaConfig.create(), - new HeapMemoryCompactionStateStorage() + new HeapMemoryIndexingStateStorage() ); taskLockbox = new GlobalTaskLockbox(taskStorage, metadataStorageCoordinator); final TaskActionToolbox taskActionToolbox = new TaskActionToolbox( 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 c4b7e5ac6778..508c59aa439b 100644 --- a/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java +++ b/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java @@ -56,7 +56,7 @@ import org.apache.druid.segment.SegmentSchemaMapping; import org.apache.druid.segment.SegmentUtils; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; -import org.apache.druid.segment.metadata.CompactionStateStorage; +import org.apache.druid.segment.metadata.IndexingStateStorage; import org.apache.druid.segment.metadata.SegmentSchemaManager; import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; import org.apache.druid.server.http.DataSegmentPlus; @@ -112,7 +112,7 @@ public class IndexerSQLMetadataStorageCoordinator implements IndexerMetadataStor private final SegmentSchemaManager segmentSchemaManager; private final CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig; private final boolean schemaPersistEnabled; - private final CompactionStateStorage compactionStateStorage; + private final IndexingStateStorage indexingStateStorage; private final SegmentMetadataTransactionFactory transactionFactory; @@ -124,7 +124,7 @@ public IndexerSQLMetadataStorageCoordinator( SQLMetadataConnector connector, SegmentSchemaManager segmentSchemaManager, CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig, - CompactionStateStorage compactionStateStorage + IndexingStateStorage indexingStateStorage ) { this.transactionFactory = transactionFactory; @@ -136,7 +136,7 @@ public IndexerSQLMetadataStorageCoordinator( this.schemaPersistEnabled = centralizedDatasourceSchemaConfig.isEnabled() && !centralizedDatasourceSchemaConfig.isTaskSchemaPublishDisabled(); - this.compactionStateStorage = compactionStateStorage; + this.indexingStateStorage = indexingStateStorage; } @LifecycleStart @@ -2740,7 +2740,7 @@ private void markCompactionFingerprintsAsActive(Set segments) // Mark each fingerprint as active for (String fingerprint : fingerprints) { try { - int rowsUpdated = compactionStateStorage.markCompactionStatesAsActive(fingerprint); + int rowsUpdated = indexingStateStorage.markIndexingStatesAsActive(fingerprint); if (rowsUpdated > 0) { log.info("Marked compaction state fingerprint[%s] as active (non-pending).", fingerprint); } diff --git a/server/src/main/java/org/apache/druid/segment/metadata/DefaultCompactionFingerprintMapper.java b/server/src/main/java/org/apache/druid/segment/metadata/DefaultCompactionFingerprintMapper.java index dfeaec3fce13..b6e79d1d98de 100644 --- a/server/src/main/java/org/apache/druid/segment/metadata/DefaultCompactionFingerprintMapper.java +++ b/server/src/main/java/org/apache/druid/segment/metadata/DefaultCompactionFingerprintMapper.java @@ -32,7 +32,7 @@ /** * Default implementation of {@link CompactionFingerprintMapper} that delegates to - * {@link CompactionStateStorage} for fingerprint generation and {@link IndexingStateCache} + * {@link IndexingStateStorage} for fingerprint generation and {@link IndexingStateCache} * for state lookups. */ public class DefaultCompactionFingerprintMapper implements CompactionFingerprintMapper diff --git a/server/src/main/java/org/apache/druid/segment/metadata/IndexingStateCache.java b/server/src/main/java/org/apache/druid/segment/metadata/IndexingStateCache.java index 9f11c7137734..be98766322fc 100644 --- a/server/src/main/java/org/apache/druid/segment/metadata/IndexingStateCache.java +++ b/server/src/main/java/org/apache/druid/segment/metadata/IndexingStateCache.java @@ -102,7 +102,7 @@ public Optional getIndexingStateByFingerprint(String fingerprin /** * Adds or updates a single indexing state in the cache. *

    - * This is called when a new compaction state is persisted to the database via upsertCompactionState + * This is called when a new compaction state is persisted to the database via upsertIndexingState * to ensure the cache is immediately consistent without waiting for the next sync. *

    * This method checks if the state is already cached before performing the atomic update. diff --git a/server/src/main/java/org/apache/druid/segment/metadata/CompactionStateStorage.java b/server/src/main/java/org/apache/druid/segment/metadata/IndexingStateStorage.java similarity index 67% rename from server/src/main/java/org/apache/druid/segment/metadata/CompactionStateStorage.java rename to server/src/main/java/org/apache/druid/segment/metadata/IndexingStateStorage.java index 223e8c05be34..529ee5b37d04 100644 --- a/server/src/main/java/org/apache/druid/segment/metadata/CompactionStateStorage.java +++ b/server/src/main/java/org/apache/druid/segment/metadata/IndexingStateStorage.java @@ -25,17 +25,17 @@ import java.util.List; /** - * Manages compaction state persistence and fingerprint generation. + * Manages indexing state persistence and fingerprint generation. *

    - * Implementations may be backed by a database (like {@link SqlCompactionStateStorage}) or - * use in-memory storage (like {@link HeapMemoryCompactionStateStorage}). + * Implementations may be backed by a database (like {@link SqlIndexingStateStorage}) or + * use in-memory storage (like {@link HeapMemoryIndexingStateStorage}). */ -public interface CompactionStateStorage +public interface IndexingStateStorage { /** - * Upserts a compaction state to storage. + * Upserts an indexing state to storage. *

    - * If a fingerprint already exists, marks it as used and updates the timestamp. + * If a fingerprint already exists, update to reflect proper used state and timestamp. * If a fingerprint doesn't exist, inserts a new row with the full state payload. * * @param dataSource The datasource name @@ -44,7 +44,7 @@ public interface CompactionStateStorage * @param updateTime The timestamp for this update */ - void upsertCompactionState( + void upsertIndexingState( String dataSource, String fingerprint, CompactionState compactionState, @@ -52,55 +52,55 @@ void upsertCompactionState( ); /** - * Marks compaction states as unused if they are not referenced by any used segments. + * Marks indexing states as unused if they are not referenced by any used segments. *

    * This is used for cleanup operations. * * @return Number of rows updated, or 0 if not applicable */ - int markUnreferencedCompactionStatesAsUnused(); + int markUnreferencedIndexingStatesAsUnused(); /** - * Finds all compaction state fingerprints which have been marked as unused but are + * Finds all indexing state fingerprints which have been marked as unused but are * still referenced by some used segments. This is used for validation/reconciliation. * Implementations may return an empty list if not applicable. * * @return List of fingerprints, or empty list */ - List findReferencedCompactionStateMarkedAsUnused(); + List findReferencedIndexingStateMarkedAsUnused(); /** - * Marks compaction states as used. + * Marks indexing states as used. *

    * This is used for reconciliation operations to avoid deleting states that are still in use. * * @param stateFingerprints List of fingerprints to mark as used * @return Number of rows updated, or 0 if not applicable */ - int markCompactionStatesAsUsed(List stateFingerprints); + int markIndexingStatesAsUsed(List stateFingerprints); /** - * Marks compaction states as active for a given fingerprint. + * Marks indexing states as active for a given fingerprint. * * @param stateFingerprint The fingerprint to mark as active * @return Number of rows updated, or 0 if not applicable */ - int markCompactionStatesAsActive(String stateFingerprint); + int markIndexingStatesAsActive(String stateFingerprint); /** - * Deletes pending compaction states older than the given timestamp. + * Deletes pending indexing states older than the given timestamp. * @param timestamp The cutoff timestamp in milliseconds * @return Number of rows deleted, or 0 if not applicable */ - int deletePendingCompactionStatesOlderThan(long timestamp); + int deletePendingIndexingStatesOlderThan(long timestamp); /** - * Deletes unused compaction states older than the given timestamp. + * Deletes unused indexing states older than the given timestamp. *

    * This is used for cleanup operations. * * @param timestamp The cutoff timestamp in milliseconds * @return Number of rows deleted, or 0 if not applicable */ - int deleteUnusedCompactionStatesOlderThan(long timestamp); + int deleteUnusedIndexingStatesOlderThan(long timestamp); } diff --git a/server/src/main/java/org/apache/druid/segment/metadata/SqlCompactionStateStorage.java b/server/src/main/java/org/apache/druid/segment/metadata/SqlIndexingStateStorage.java similarity index 88% rename from server/src/main/java/org/apache/druid/segment/metadata/SqlCompactionStateStorage.java rename to server/src/main/java/org/apache/druid/segment/metadata/SqlIndexingStateStorage.java index 15daa666181a..6319a9014c16 100644 --- a/server/src/main/java/org/apache/druid/segment/metadata/SqlCompactionStateStorage.java +++ b/server/src/main/java/org/apache/druid/segment/metadata/SqlIndexingStateStorage.java @@ -43,28 +43,28 @@ import java.util.List; /** - * Database-backed implementation of {@link CompactionStateStorage}. + * Database-backed implementation of {@link IndexingStateStorage}. *

    - * 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. + * Manages the persistence and retrieval of {@link CompactionState} (AKA IndexinState) objects in the metadata storage. + * Indexing states are uniquely identified by their fingerprints, which are SHA-256 hashes of their content. *

    *

    - * This implementation is designed to be called from a single thread (CompactionJobQueue) and relies on + * This implementation is designed to be called from a single thread and relies on * database constraints and the retry mechanism to handle any conflicts. Operations are idempotent - concurrent * upserts for the same fingerprint will either succeed or fail with a constraint violation that is safely ignored. *

    */ @LazySingleton -public class SqlCompactionStateStorage implements CompactionStateStorage +public class SqlIndexingStateStorage implements IndexingStateStorage { - private static final EmittingLogger log = new EmittingLogger(SqlCompactionStateStorage.class); + private static final EmittingLogger log = new EmittingLogger(SqlIndexingStateStorage.class); private final MetadataStorageTablesConfig dbTables; private final ObjectMapper jsonMapper; private final SQLMetadataConnector connector; @Inject - public SqlCompactionStateStorage( + public SqlIndexingStateStorage( @Nonnull MetadataStorageTablesConfig dbTables, @Nonnull ObjectMapper jsonMapper, @Nonnull SQLMetadataConnector connector @@ -76,7 +76,7 @@ public SqlCompactionStateStorage( } @Override - public void upsertCompactionState( + public void upsertIndexingState( @NotEmpty final String dataSource, @NotEmpty final String fingerprint, @Nonnull final CompactionState compactionState, @@ -113,7 +113,7 @@ public void upsertCompactionState( case EXISTS_AND_USED: // Fingerprint exists and is already marked as used - no operation needed log.debug( - "Compaction state for fingerprint[%s] in dataSource[%s] already exists and is marked as used. Skipping update.", + "Indexing state for fingerprint[%s] in dataSource[%s] already exists and is marked as used. Skipping update.", fingerprint, dataSource ); @@ -122,7 +122,7 @@ public void upsertCompactionState( case EXISTS_AND_UNUSED: // Fingerprint exists but is marked as unused - update the used flag log.info( - "Found existing compaction state in DB for fingerprint[%s] in dataSource[%s]. Marking as used.", + "Found existing indexing state in DB for fingerprint[%s] in dataSource[%s]. Marking as used.", fingerprint, dataSource ); @@ -137,12 +137,12 @@ public void upsertCompactionState( .bind("fingerprint", fingerprint) .execute(); - log.info("Updated existing compaction state for datasource[%s].", dataSource); + log.info("Updated existing indexing state for datasource[%s].", dataSource); break; case DOES_NOT_EXIST: // Fingerprint doesn't exist - insert new state - log.info("Inserting new compaction state for fingerprint[%s] in dataSource[%s].", fingerprint, dataSource); + log.info("Inserting new indexing state for fingerprint[%s] in dataSource[%s].", fingerprint, dataSource); String insertSql = StringUtils.format( "INSERT INTO %s (created_date, dataSource, fingerprint, payload, used, pending, used_status_last_updated) " @@ -162,7 +162,7 @@ public void upsertCompactionState( .execute(); log.info( - "Published compaction state for fingerprint[%s] to DB for datasource[%s].", + "Published indexing state for fingerprint[%s] to DB for datasource[%s].", fingerprint, dataSource ); @@ -170,7 +170,7 @@ public void upsertCompactionState( catch (JsonProcessingException e) { throw InternalServerError.exception( e, - "Failed to serialize compaction state for fingerprint[%s]", + "Failed to serialize indexing state for fingerprint[%s]", fingerprint ); } @@ -198,7 +198,7 @@ public void upsertCompactionState( } @Override - public int markUnreferencedCompactionStatesAsUnused() + public int markUnreferencedIndexingStatesAsUnused() { return connector.retryWithHandle( handle -> @@ -215,7 +215,7 @@ public int markUnreferencedCompactionStatesAsUnused() } @Override - public List findReferencedCompactionStateMarkedAsUnused() + public List findReferencedIndexingStateMarkedAsUnused() { return connector.retryWithHandle( handle -> @@ -231,7 +231,7 @@ public List findReferencedCompactionStateMarkedAsUnused() } @Override - public int markCompactionStatesAsUsed(List stateFingerprints) + public int markIndexingStatesAsUsed(List stateFingerprints) { if (stateFingerprints.isEmpty()) { return 0; @@ -256,7 +256,7 @@ public int markCompactionStatesAsUsed(List stateFingerprints) } @Override - public int markCompactionStatesAsActive(String stateFingerprint) + public int markIndexingStatesAsActive(String stateFingerprint) { return connector.retryWithHandle( handle -> handle.createStatement( @@ -270,7 +270,7 @@ public int markCompactionStatesAsActive(String stateFingerprint) } @Override - public int deleteUnusedCompactionStatesOlderThan(long timestamp) + public int deleteUnusedIndexingStatesOlderThan(long timestamp) { return connector.retryWithHandle( handle -> handle.createStatement( @@ -283,7 +283,7 @@ public int deleteUnusedCompactionStatesOlderThan(long timestamp) } @Override - public int deletePendingCompactionStatesOlderThan(long timestamp) + public int deletePendingIndexingStatesOlderThan(long timestamp) { return connector.retryWithHandle( handle -> handle.createStatement( @@ -295,9 +295,15 @@ public int deletePendingCompactionStatesOlderThan(long timestamp) .execute()); } + /** + * Checks if the indexing state for the given fingerprint is pending. + *

    + * Useful for testing purposes to verify the pending status of an indexing state. + *

    + */ @Nullable @VisibleForTesting - public Boolean isCompactionStatePending(final String fingerprint) + public Boolean isIndexingStatePending(final String fingerprint) { return connector.retryWithHandle( handle -> { diff --git a/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorMarkUsedTest.java b/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorMarkUsedTest.java index ae65f26f5378..dadad5e4f1ba 100644 --- a/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorMarkUsedTest.java +++ b/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorMarkUsedTest.java @@ -30,7 +30,7 @@ import org.apache.druid.metadata.segment.cache.NoopSegmentMetadataCache; import org.apache.druid.segment.TestDataSource; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; -import org.apache.druid.segment.metadata.HeapMemoryCompactionStateStorage; +import org.apache.druid.segment.metadata.HeapMemoryIndexingStateStorage; import org.apache.druid.server.coordinator.CreateDataSegments; import org.apache.druid.server.coordinator.simulate.TestDruidLeaderSelector; import org.apache.druid.server.metrics.NoopServiceEmitter; @@ -97,7 +97,7 @@ public int getMaxRetries() derbyConnector, null, CentralizedDatasourceSchemaConfig.create(), - new HeapMemoryCompactionStateStorage() + new HeapMemoryIndexingStateStorage() ); derbyConnector.createSegmentTable(); diff --git a/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorReadOnlyTest.java b/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorReadOnlyTest.java index 1617d6134109..d68da314ad16 100644 --- a/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorReadOnlyTest.java +++ b/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorReadOnlyTest.java @@ -36,7 +36,7 @@ import org.apache.druid.segment.TestDataSource; import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; -import org.apache.druid.segment.metadata.HeapMemoryCompactionStateStorage; +import org.apache.druid.segment.metadata.HeapMemoryIndexingStateStorage; import org.apache.druid.segment.metadata.NoopIndexingStateCache; import org.apache.druid.segment.metadata.NoopSegmentSchemaCache; import org.apache.druid.server.coordinator.simulate.BlockingExecutorService; @@ -182,7 +182,7 @@ private IndexerSQLMetadataStorageCoordinator createStorageCoordinator( derbyConnector, null, CentralizedDatasourceSchemaConfig.enabled(false), - new HeapMemoryCompactionStateStorage() + new HeapMemoryIndexingStateStorage() ); } diff --git a/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java b/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java index e7e8d69ea259..40b96eaa429b 100644 --- a/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java +++ b/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java @@ -48,12 +48,12 @@ import org.apache.druid.segment.TestDataSource; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; import org.apache.druid.segment.metadata.FingerprintGenerator; -import org.apache.druid.segment.metadata.HeapMemoryCompactionStateStorage; +import org.apache.druid.segment.metadata.HeapMemoryIndexingStateStorage; import org.apache.druid.segment.metadata.NoopIndexingStateCache; import org.apache.druid.segment.metadata.NoopSegmentSchemaCache; import org.apache.druid.segment.metadata.SegmentSchemaManager; import org.apache.druid.segment.metadata.SegmentSchemaTestUtils; -import org.apache.druid.segment.metadata.SqlCompactionStateStorage; +import org.apache.druid.segment.metadata.SqlIndexingStateStorage; import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; import org.apache.druid.server.coordinator.CreateDataSegments; import org.apache.druid.server.coordinator.simulate.BlockingExecutorService; @@ -118,7 +118,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest extends IndexerSqlMetadata private StubServiceEmitter emitter; private SqlSegmentMetadataTransactionFactory transactionFactory; private BlockingExecutorService cachePollExecutor; - private SqlCompactionStateStorage compactionStateStorage; + private SqlIndexingStateStorage compactionStateStorage; private final SegmentMetadataCache.UsageMode cacheMode; @@ -155,7 +155,7 @@ public void setUp() fingerprintGenerator = new FingerprintGenerator(mapper); segmentSchemaManager = new SegmentSchemaManager(derbyConnectorRule.metadataTablesConfigSupplier().get(), mapper, derbyConnector); segmentSchemaTestUtils = new SegmentSchemaTestUtils(derbyConnectorRule, derbyConnector, mapper); - compactionStateStorage = new SqlCompactionStateStorage( + compactionStateStorage = new SqlIndexingStateStorage( derbyConnectorRule.metadataTablesConfigSupplier().get(), mapper, derbyConnector @@ -810,7 +810,7 @@ public void testTransactionalAnnounceRetryAndSuccess() throws IOException derbyConnector, segmentSchemaManager, CentralizedDatasourceSchemaConfig.create(), - new HeapMemoryCompactionStateStorage() + new HeapMemoryIndexingStateStorage() ) { @Override @@ -970,7 +970,7 @@ public void test_commitSegmentsAndMetadata_isAtomic() derbyConnector, segmentSchemaManager, CentralizedDatasourceSchemaConfig.create(), - new HeapMemoryCompactionStateStorage() + new HeapMemoryIndexingStateStorage() ) { @Override @@ -4337,8 +4337,8 @@ public void testCommitSegmentsAndMetadata_marksPendingCompactionStateAsActive() { String fingerprint = "vanillaFingerprint"; CompactionState state = createTestCompactionState(); - compactionStateStorage.upsertCompactionState(TestDataSource.WIKI, fingerprint, state, DateTimes.nowUtc()); - Assert.assertEquals(Boolean.TRUE, compactionStateStorage.isCompactionStatePending(fingerprint)); + compactionStateStorage.upsertIndexingState(TestDataSource.WIKI, fingerprint, state, DateTimes.nowUtc()); + Assert.assertEquals(Boolean.TRUE, compactionStateStorage.isIndexingStatePending(fingerprint)); final DataSegment segment = CreateDataSegments.ofDatasource(TestDataSource.WIKI) .startingAt("2023-01-01") @@ -4354,7 +4354,7 @@ public void testCommitSegmentsAndMetadata_marksPendingCompactionStateAsActive() null ); - Assert.assertEquals(Boolean.FALSE, compactionStateStorage.isCompactionStatePending(fingerprint)); + Assert.assertEquals(Boolean.FALSE, compactionStateStorage.isIndexingStatePending(fingerprint)); } @Test @@ -4362,8 +4362,8 @@ public void testCommitReplaceSegments_marksPendingCompactionStateAsActive() { String fingerprint = "replaceFingerprint"; CompactionState state = createTestCompactionState(); - compactionStateStorage.upsertCompactionState(TestDataSource.WIKI, fingerprint, state, DateTimes.nowUtc()); - Assert.assertEquals(Boolean.TRUE, compactionStateStorage.isCompactionStatePending(fingerprint)); + compactionStateStorage.upsertIndexingState(TestDataSource.WIKI, fingerprint, state, DateTimes.nowUtc()); + Assert.assertEquals(Boolean.TRUE, compactionStateStorage.isIndexingStatePending(fingerprint)); final DataSegment segment = CreateDataSegments.ofDatasource(TestDataSource.WIKI) .startingAt("2023-01-01") @@ -4384,7 +4384,7 @@ public void testCommitReplaceSegments_marksPendingCompactionStateAsActive() null ); - Assert.assertEquals(Boolean.FALSE, compactionStateStorage.isCompactionStatePending(fingerprint)); + Assert.assertEquals(Boolean.FALSE, compactionStateStorage.isIndexingStatePending(fingerprint)); } @Test @@ -4392,8 +4392,8 @@ public void testCCommitAppendSegments_marksPendingCompactionStateAsActive() { String fingerprint = "appendFingerprint"; CompactionState state = createTestCompactionState(); - compactionStateStorage.upsertCompactionState(TestDataSource.WIKI, fingerprint, state, DateTimes.nowUtc()); - Assert.assertEquals(Boolean.TRUE, compactionStateStorage.isCompactionStatePending(fingerprint)); + compactionStateStorage.upsertIndexingState(TestDataSource.WIKI, fingerprint, state, DateTimes.nowUtc()); + Assert.assertEquals(Boolean.TRUE, compactionStateStorage.isIndexingStatePending(fingerprint)); final DataSegment segment = CreateDataSegments.ofDatasource(TestDataSource.WIKI) .startingAt("2023-01-01") @@ -4410,7 +4410,7 @@ public void testCCommitAppendSegments_marksPendingCompactionStateAsActive() null ); - Assert.assertEquals(Boolean.FALSE, compactionStateStorage.isCompactionStatePending(fingerprint)); + Assert.assertEquals(Boolean.FALSE, compactionStateStorage.isIndexingStatePending(fingerprint)); } private CompactionState createTestCompactionState() diff --git a/server/src/test/java/org/apache/druid/metadata/IndexerSqlMetadataStorageCoordinatorSchemaPersistenceTest.java b/server/src/test/java/org/apache/druid/metadata/IndexerSqlMetadataStorageCoordinatorSchemaPersistenceTest.java index d649c1b8cd1b..ba64c709fb18 100644 --- a/server/src/test/java/org/apache/druid/metadata/IndexerSqlMetadataStorageCoordinatorSchemaPersistenceTest.java +++ b/server/src/test/java/org/apache/druid/metadata/IndexerSqlMetadataStorageCoordinatorSchemaPersistenceTest.java @@ -39,7 +39,7 @@ import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; import org.apache.druid.segment.metadata.FingerprintGenerator; -import org.apache.druid.segment.metadata.HeapMemoryCompactionStateStorage; +import org.apache.druid.segment.metadata.HeapMemoryIndexingStateStorage; import org.apache.druid.segment.metadata.SegmentSchemaManager; import org.apache.druid.segment.metadata.SegmentSchemaTestUtils; import org.apache.druid.server.coordinator.simulate.TestDruidLeaderSelector; @@ -110,7 +110,7 @@ public void setUp() derbyConnector, segmentSchemaManager, centralizedDatasourceSchemaConfig, - new HeapMemoryCompactionStateStorage() + new HeapMemoryIndexingStateStorage() ) { @Override diff --git a/server/src/test/java/org/apache/druid/metadata/SqlSegmentsMetadataQueryTest.java b/server/src/test/java/org/apache/druid/metadata/SqlSegmentsMetadataQueryTest.java index bbcbc6273b22..c8747fadffe3 100644 --- a/server/src/test/java/org/apache/druid/metadata/SqlSegmentsMetadataQueryTest.java +++ b/server/src/test/java/org/apache/druid/metadata/SqlSegmentsMetadataQueryTest.java @@ -32,7 +32,7 @@ import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.TestDataSource; import org.apache.druid.segment.TestHelper; -import org.apache.druid.segment.metadata.SqlCompactionStateStorage; +import org.apache.druid.segment.metadata.SqlIndexingStateStorage; import org.apache.druid.server.coordinator.CreateDataSegments; import org.apache.druid.timeline.CompactionState; import org.apache.druid.timeline.DataSegment; @@ -667,7 +667,7 @@ private void insertCompactionStates(Map compactionState { ObjectMapper mapper = TestHelper.JSON_MAPPER; MetadataStorageTablesConfig tablesConfig = derbyConnectorRule.metadataTablesConfigSupplier().get(); - SqlCompactionStateStorage manager = new SqlCompactionStateStorage( + SqlIndexingStateStorage manager = new SqlIndexingStateStorage( tablesConfig, mapper, derbyConnectorRule.getConnector() @@ -675,7 +675,7 @@ private void insertCompactionStates(Map compactionState derbyConnectorRule.getConnector().retryWithHandle(handle -> { for (Map.Entry entry : compactionStates.entrySet()) { - manager.upsertCompactionState(TestDataSource.WIKI, entry.getKey(), entry.getValue(), DateTimes.nowUtc()); + manager.upsertIndexingState(TestDataSource.WIKI, entry.getKey(), entry.getValue(), DateTimes.nowUtc()); } return null; }); diff --git a/server/src/test/java/org/apache/druid/segment/metadata/HeapMemoryCompactionStateStorage.java b/server/src/test/java/org/apache/druid/segment/metadata/HeapMemoryIndexingStateStorage.java similarity index 69% rename from server/src/test/java/org/apache/druid/segment/metadata/HeapMemoryCompactionStateStorage.java rename to server/src/test/java/org/apache/druid/segment/metadata/HeapMemoryIndexingStateStorage.java index 9cd891927527..48ee3ffad69a 100644 --- a/server/src/test/java/org/apache/druid/segment/metadata/HeapMemoryCompactionStateStorage.java +++ b/server/src/test/java/org/apache/druid/segment/metadata/HeapMemoryIndexingStateStorage.java @@ -28,27 +28,27 @@ import java.util.concurrent.ConcurrentMap; /** - * In-memory implementation of {@link CompactionStateStorage} that stores - * compaction state fingerprints in heap memory without requiring a database. + * In-memory implementation of {@link IndexingStateStorage} that stores + * indexing state fingerprints and states in heap memory without requiring a database. *

    * Useful for simulations and unit tests where database persistence is not needed. * Database-specific operations (cleanup, unused marking) are no-ops in this implementation. */ -public class HeapMemoryCompactionStateStorage implements CompactionStateStorage +public class HeapMemoryIndexingStateStorage implements IndexingStateStorage { private final ConcurrentMap fingerprintToStateMap; /** - * Creates an in-memory compaction state manager with a default deterministic mapper. + * Creates an in-memory indexing state manager with a default deterministic mapper. * This is a convenience constructor for tests and simulations. */ - public HeapMemoryCompactionStateStorage() + public HeapMemoryIndexingStateStorage() { this.fingerprintToStateMap = new ConcurrentHashMap<>(); } @Override - public void upsertCompactionState( + public void upsertIndexingState( final String dataSource, final String fingerprint, final CompactionState compactionState, @@ -60,43 +60,43 @@ public void upsertCompactionState( } @Override - public int markUnreferencedCompactionStatesAsUnused() + public int markUnreferencedIndexingStatesAsUnused() { return 0; } @Override - public List findReferencedCompactionStateMarkedAsUnused() + public List findReferencedIndexingStateMarkedAsUnused() { return List.of(); } @Override - public int markCompactionStatesAsUsed(List stateFingerprints) + public int markIndexingStatesAsUsed(List stateFingerprints) { return 0; } @Override - public int markCompactionStatesAsActive(String stateFingerprint) + public int markIndexingStatesAsActive(String stateFingerprint) { return 0; } @Override - public int deletePendingCompactionStatesOlderThan(long timestamp) + public int deletePendingIndexingStatesOlderThan(long timestamp) { return 0; } @Override - public int deleteUnusedCompactionStatesOlderThan(long timestamp) + public int deleteUnusedIndexingStatesOlderThan(long timestamp) { return 0; } /** - * Gets all stored compaction states. For test verification only. + * Gets all stored indexing states. For test verification only. */ public Map getAllStoredStates() { @@ -104,7 +104,7 @@ public Map getAllStoredStates() } /** - * Clears all stored compaction states. Useful for test cleanup or resetting + * Clears all stored indexing states. Useful for test cleanup or resetting * state between test runs. */ public void clear() @@ -113,7 +113,7 @@ public void clear() } /** - * Returns the number of stored compaction state fingerprints. + * Returns the number of stored indexing state fingerprints. */ public int size() { diff --git a/server/src/test/java/org/apache/druid/segment/metadata/SqlCompactionStateStorageTest.java b/server/src/test/java/org/apache/druid/segment/metadata/SqlIndexingStateStorageTest.java similarity index 88% rename from server/src/test/java/org/apache/druid/segment/metadata/SqlCompactionStateStorageTest.java rename to server/src/test/java/org/apache/druid/segment/metadata/SqlIndexingStateStorageTest.java index 09b66b12923f..f987b2ed5e71 100644 --- a/server/src/test/java/org/apache/druid/segment/metadata/SqlCompactionStateStorageTest.java +++ b/server/src/test/java/org/apache/druid/segment/metadata/SqlIndexingStateStorageTest.java @@ -48,7 +48,7 @@ import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; -public class SqlCompactionStateStorageTest +public class SqlIndexingStateStorageTest { @RegisterExtension public static final TestDerbyConnector.DerbyConnectorRule5 DERBY_CONNECTOR_RULE = @@ -59,7 +59,7 @@ public class SqlCompactionStateStorageTest private static TestDerbyConnector derbyConnector; private static MetadataStorageTablesConfig tablesConfig; - private SqlCompactionStateStorage manager; + private SqlIndexingStateStorage manager; private static DefaultCompactionFingerprintMapper fingerprintMapper; @@ -85,16 +85,16 @@ public void setUp() return null; }); - manager = new SqlCompactionStateStorage(tablesConfig, jsonMapper, derbyConnector); + manager = new SqlIndexingStateStorage(tablesConfig, jsonMapper, derbyConnector); } @Test - public void test_upsertCompactionState_successfullyInsertsIntoDatabase() + public void test_upsertIndexingState_successfullyInsertsIntoDatabase() { CompactionState state1 = createTestCompactionState(); String fingerprint = "fingerprint_abc123"; - manager.upsertCompactionState( + manager.upsertIndexingState( "testDatasource", fingerprint, state1, @@ -114,39 +114,39 @@ public void test_upsertCompactionState_successfullyInsertsIntoDatabase() } @Test - public void test_upsertCompactionState_andThen_markUnreferencedCompactionStateAsUnused_andThen_markCompactionStatesAsUsed() + public void test_upsertCompactionState_andThen_markUnreferencedCompactionStateAsUnused_andThen_markIndexingStatesAsUsed() { CompactionState state1 = createTestCompactionState(); String fingerprint = "fingerprint_abc123"; - manager.upsertCompactionState( + manager.upsertIndexingState( "testDatasource", fingerprint, state1, DateTimes.nowUtc() ); - manager.markCompactionStatesAsActive(fingerprint); + manager.markIndexingStatesAsActive(fingerprint); - assertEquals(1, manager.markUnreferencedCompactionStatesAsUnused()); - assertEquals(1, manager.markCompactionStatesAsUsed(List.of(fingerprint))); + assertEquals(1, manager.markUnreferencedIndexingStatesAsUnused()); + assertEquals(1, manager.markIndexingStatesAsUsed(List.of(fingerprint))); } @Test - public void test_findReferencedCompactionStateMarkedAsUnused() + public void test_findReferencedIndexingStateMarkedAsUnused() { CompactionState state1 = createTestCompactionState(); String fingerprint = "fingerprint_abc123"; - manager.upsertCompactionState( + manager.upsertIndexingState( "testDatasource", fingerprint, state1, DateTimes.nowUtc() ); - manager.markCompactionStatesAsActive(fingerprint); + manager.markIndexingStatesAsActive(fingerprint); - manager.markUnreferencedCompactionStatesAsUnused(); - assertEquals(0, manager.findReferencedCompactionStateMarkedAsUnused().size()); + manager.markUnreferencedIndexingStatesAsUnused(); + assertEquals(0, manager.findReferencedIndexingStateMarkedAsUnused().size()); derbyConnector.retryWithHandle(handle -> { handle.createStatement( @@ -171,7 +171,7 @@ public void test_findReferencedCompactionStateMarkedAsUnused() return null; }); - List referenced = manager.findReferencedCompactionStateMarkedAsUnused(); + List referenced = manager.findReferencedIndexingStateMarkedAsUnused(); assertEquals(1, referenced.size()); assertEquals(fingerprint, referenced.get(0)); } @@ -227,7 +227,7 @@ public void test_deleteCompactionStatesOlderThan_deletesOnlyOldUnusedStates() }); // Delete states older than 30 days - int deleted = manager.deleteUnusedCompactionStatesOlderThan(cutoffTime.getMillis()); + int deleted = manager.deleteUnusedIndexingStatesOlderThan(cutoffTime.getMillis()); assertEquals(1, deleted); // Verify only 1 state remains in the table @@ -240,11 +240,11 @@ public void test_deleteCompactionStatesOlderThan_deletesOnlyOldUnusedStates() } @Test - public void test_upsertCompactionState_withNullState_throwsException() + public void test_upsertIndexingState_withNullState_throwsException() { Exception exception = assertThrows( Exception.class, - () -> manager.upsertCompactionState("ds", "somePrint", null, DateTimes.nowUtc()) + () -> manager.upsertIndexingState("ds", "somePrint", null, DateTimes.nowUtc()) ); assertTrue( @@ -254,12 +254,12 @@ public void test_upsertCompactionState_withNullState_throwsException() } @Test - public void test_upsertCompactionState_withEmptyFingerprint_throwsException() + public void test_upsertIndexingState_withEmptyFingerprint_throwsException() { // The exception ends up wrapped in a sql exception doe to the retryWithHandle so we will just check the message Exception exception = assertThrows( Exception.class, - () -> manager.upsertCompactionState("ds", "", createBasicCompactionState(), DateTimes.nowUtc()) + () -> manager.upsertIndexingState("ds", "", createBasicCompactionState(), DateTimes.nowUtc()) ); assertTrue( @@ -269,13 +269,13 @@ public void test_upsertCompactionState_withEmptyFingerprint_throwsException() } @Test - public void test_upsertCompactionState_verifyExistingFingerprintMarkedUsed() + public void test_upsertIndexingState_verifyExistingFingerprintMarkedUsed() { String fingerprint = "existing_fingerprint"; CompactionState state = createTestCompactionState(); // Persist initially - manager.upsertCompactionState("ds1", fingerprint, state, DateTimes.nowUtc()); + manager.upsertIndexingState("ds1", fingerprint, state, DateTimes.nowUtc()); // Verify it's marked as used Boolean usedBefore = derbyConnector.retryWithHandle(handle -> @@ -297,7 +297,7 @@ public void test_upsertCompactionState_verifyExistingFingerprintMarkedUsed() ); // Persist again with the same fingerprint (should UPDATE, not INSERT) - manager.upsertCompactionState("ds1", fingerprint, state, DateTimes.nowUtc()); + manager.upsertIndexingState("ds1", fingerprint, state, DateTimes.nowUtc()); // Verify it's marked as used again Boolean usedAfter = derbyConnector.retryWithHandle(handle -> @@ -320,14 +320,14 @@ public void test_upsertCompactionState_verifyExistingFingerprintMarkedUsed() } @Test - public void test_upsertCompactionState_whenAlreadyUsed_skipsUpdate() + public void test_upsertIndexingState_whenAlreadyUsed_skipsUpdate() { String fingerprint = "already_used_fingerprint"; CompactionState state = createTestCompactionState(); DateTime initialTime = DateTimes.of("2024-01-01T00:00:00.000Z"); // Insert fingerprint as used initially - manager.upsertCompactionState("ds1", fingerprint, state, initialTime); + manager.upsertIndexingState("ds1", fingerprint, state, initialTime); // Verify it's marked as used with the initial timestamp DateTime usedStatusBeforeUpdate = derbyConnector.retryWithHandle(handle -> @@ -343,7 +343,7 @@ public void test_upsertCompactionState_whenAlreadyUsed_skipsUpdate() // Call upsert again with a different timestamp // Since the fingerprint is already used, this should skip the UPDATE DateTime laterTime = DateTimes.of("2024-01-02T00:00:00.000Z"); - manager.upsertCompactionState("ds1", fingerprint, state, laterTime); + manager.upsertIndexingState("ds1", fingerprint, state, laterTime); // Verify the used_status_last_updated timestamp DID NOT change DateTime usedStatusAfterUpdate = derbyConnector.retryWithHandle(handle -> @@ -373,16 +373,16 @@ public void test_upsertCompactionState_whenAlreadyUsed_skipsUpdate() @Test public void test_markCompactionStateAsUsed_withEmptyList_returnsZero() { - assertEquals(0, manager.markCompactionStatesAsUsed(List.of())); + assertEquals(0, manager.markIndexingStatesAsUsed(List.of())); } @Test - public void test_markCompactionStatesAsActive_marksPendingStateAsActive() + public void test_markIndexingStatesAsActive_marksPendingStateAsActive() { String fingerprint = "pending_fingerprint"; CompactionState state = createTestCompactionState(); - manager.upsertCompactionState("ds1", fingerprint, state, DateTimes.nowUtc()); + manager.upsertIndexingState("ds1", fingerprint, state, DateTimes.nowUtc()); Boolean pendingBefore = derbyConnector.retryWithHandle(handle -> handle.createQuery("SELECT pending FROM " + tablesConfig.getIndexingStatesTable() + " WHERE fingerprint = :fp") @@ -392,7 +392,7 @@ public void test_markCompactionStatesAsActive_marksPendingStateAsActive() ); assertTrue(pendingBefore); - int rowsUpdated = manager.markCompactionStatesAsActive(fingerprint); + int rowsUpdated = manager.markIndexingStatesAsActive(fingerprint); assertEquals(1, rowsUpdated); Boolean pendingAfter = derbyConnector.retryWithHandle(handle -> @@ -405,17 +405,17 @@ public void test_markCompactionStatesAsActive_marksPendingStateAsActive() } @Test - public void test_markCompactionStatesAsActive_idempotent_returnsZeroWhenAlreadyActive() + public void test_markIndexingStatesAsActive_idempotent_returnsZeroWhenAlreadyActive() { String fingerprint = "already_active_fingerprint"; CompactionState state = createTestCompactionState(); - manager.upsertCompactionState("ds1", fingerprint, state, DateTimes.nowUtc()); + manager.upsertIndexingState("ds1", fingerprint, state, DateTimes.nowUtc()); - int firstUpdate = manager.markCompactionStatesAsActive(fingerprint); + int firstUpdate = manager.markIndexingStatesAsActive(fingerprint); assertEquals(1, firstUpdate); - int secondUpdate = manager.markCompactionStatesAsActive(fingerprint); + int secondUpdate = manager.markIndexingStatesAsActive(fingerprint); assertEquals(0, secondUpdate); Boolean pending = derbyConnector.retryWithHandle(handle -> @@ -428,9 +428,9 @@ public void test_markCompactionStatesAsActive_idempotent_returnsZeroWhenAlreadyA } @Test - public void test_markCompactionStatesAsActive_nonExistentFingerprint_returnsZero() + public void test_markIndexingStatesAsActive_nonExistentFingerprint_returnsZero() { - int rowsUpdated = manager.markCompactionStatesAsActive("does_not_exist"); + int rowsUpdated = manager.markIndexingStatesAsActive("does_not_exist"); assertEquals(0, rowsUpdated); } 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 aad3c9d15571..2dd9a30bb488 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 @@ -42,7 +42,7 @@ import org.apache.druid.segment.metadata.CompactionFingerprintMapper; import org.apache.druid.segment.metadata.CompactionTestUtils; import org.apache.druid.segment.metadata.DefaultCompactionFingerprintMapper; -import org.apache.druid.segment.metadata.HeapMemoryCompactionStateStorage; +import org.apache.druid.segment.metadata.HeapMemoryIndexingStateStorage; import org.apache.druid.segment.metadata.IndexingStateCache; import org.apache.druid.segment.nested.NestedCommonFormatColumnFormatSpec; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; @@ -71,14 +71,14 @@ public class CompactionStatusTest .size(100_000_000L) .build(); - private HeapMemoryCompactionStateStorage compactionStateStorage; + private HeapMemoryIndexingStateStorage compactionStateStorage; private IndexingStateCache indexingStateCache; private CompactionFingerprintMapper fingerprintMapper; @Before public void setUp() { - compactionStateStorage = new HeapMemoryCompactionStateStorage(); + compactionStateStorage = new HeapMemoryIndexingStateStorage(); indexingStateCache = new IndexingStateCache(); fingerprintMapper = new DefaultCompactionFingerprintMapper( indexingStateCache, @@ -603,7 +603,7 @@ public void test_evaluate_needsCompactionWhenAllSegmentsHaveUnexpectedCompaction CompactionState expectedState = compactionConfig.toCompactionState(); - compactionStateStorage.upsertCompactionState(TestDataSource.WIKI, "wrongFingerprint", wrongState, DateTimes.nowUtc()); + compactionStateStorage.upsertIndexingState(TestDataSource.WIKI, "wrongFingerprint", wrongState, DateTimes.nowUtc()); syncCacheFromManager(); verifyEvaluationNeedsCompactionBecauseWithCustomSegments( @@ -638,8 +638,8 @@ public void test_evaluate_needsCompactionWhenSomeSegmentsHaveUnexpectedCompactio DataSegment.builder(WIKI_SEGMENT_2).compactionStateFingerprint("wrongFingerprint").build() ); - compactionStateStorage.upsertCompactionState(TestDataSource.WIKI, expectedFingerprint, expectedState, DateTimes.nowUtc()); - compactionStateStorage.upsertCompactionState(TestDataSource.WIKI, "wrongFingerprint", wrongState, DateTimes.nowUtc()); + compactionStateStorage.upsertIndexingState(TestDataSource.WIKI, expectedFingerprint, expectedState, DateTimes.nowUtc()); + compactionStateStorage.upsertIndexingState(TestDataSource.WIKI, "wrongFingerprint", wrongState, DateTimes.nowUtc()); syncCacheFromManager(); verifyEvaluationNeedsCompactionBecauseWithCustomSegments( @@ -662,7 +662,7 @@ public void test_evaluate_noCompacationIfUnexpectedFingerprintHasExpectedCompact .build(); CompactionState expectedState = compactionConfig.toCompactionState(); - compactionStateStorage.upsertCompactionState(TestDataSource.WIKI, "wrongFingerprint", expectedState, DateTimes.nowUtc()); + compactionStateStorage.upsertIndexingState(TestDataSource.WIKI, "wrongFingerprint", expectedState, DateTimes.nowUtc()); syncCacheFromManager(); final CompactionStatus status = CompactionStatus.compute( @@ -730,7 +730,7 @@ public void test_evaluate_needsCompactionWhenNonFingerprintedSegmentsFailChecksO CompactionState expectedState = compactionConfig.toCompactionState(); String expectedFingerprint = fingerprintMapper.generateFingerprint(TestDataSource.WIKI, expectedState); - compactionStateStorage.upsertCompactionState(TestDataSource.WIKI, expectedFingerprint, expectedState, DateTimes.nowUtc()); + compactionStateStorage.upsertIndexingState(TestDataSource.WIKI, expectedFingerprint, expectedState, DateTimes.nowUtc()); syncCacheFromManager(); List segments = List.of( diff --git a/server/src/test/java/org/apache/druid/server/coordinator/duty/KillUnusedSegmentsTest.java b/server/src/test/java/org/apache/druid/server/coordinator/duty/KillUnusedSegmentsTest.java index 64587fe8af3b..37501cb849cc 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/duty/KillUnusedSegmentsTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/duty/KillUnusedSegmentsTest.java @@ -44,7 +44,7 @@ import org.apache.druid.rpc.indexing.NoopOverlordClient; import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; -import org.apache.druid.segment.metadata.HeapMemoryCompactionStateStorage; +import org.apache.druid.segment.metadata.HeapMemoryIndexingStateStorage; import org.apache.druid.server.coordinator.CoordinatorDynamicConfig; import org.apache.druid.server.coordinator.DruidCoordinatorRuntimeParams; import org.apache.druid.server.coordinator.config.KillUnusedSegmentsConfig; @@ -119,7 +119,7 @@ public void setup() connector, null, CentralizedDatasourceSchemaConfig.create(), - new HeapMemoryCompactionStateStorage() + new HeapMemoryIndexingStateStorage() ); this.config = derbyConnectorRule.metadataTablesConfigSupplier().get(); 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 09c266222663..94c7d7e9ddaa 100644 --- a/services/src/main/java/org/apache/druid/guice/MetadataManagerModule.java +++ b/services/src/main/java/org/apache/druid/guice/MetadataManagerModule.java @@ -41,11 +41,11 @@ 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.CompactionStateStorage; +import org.apache.druid.segment.metadata.IndexingStateStorage; import org.apache.druid.segment.metadata.IndexingStateCache; import org.apache.druid.segment.metadata.NoopSegmentSchemaCache; import org.apache.druid.segment.metadata.SegmentSchemaCache; -import org.apache.druid.segment.metadata.SqlCompactionStateStorage; +import org.apache.druid.segment.metadata.SqlIndexingStateStorage; import org.apache.druid.server.coordinator.CoordinatorConfigManager; import org.apache.druid.server.coordinator.MetadataManager; @@ -64,7 +64,7 @@ *

  • {@link SegmentMetadataCache}
  • *
  • {@link IndexingStateCache} - Overlord only
  • *
  • {@link SegmentSchemaCache} - Coordinator only
  • - *
  • {@link SqlCompactionStateStorage}
  • + *
  • {@link SqlIndexingStateStorage}
  • * */ public class MetadataManagerModule implements Module @@ -106,8 +106,8 @@ public void configure(Binder binder) binder.bind(SegmentMetadataCache.class) .to(HeapMemorySegmentMetadataCache.class) .in(LazySingleton.class); - binder.bind(CompactionStateStorage.class) - .to(SqlCompactionStateStorage.class) + binder.bind(IndexingStateStorage.class) + .to(SqlIndexingStateStorage.class) .in(ManageLifecycle.class); // Coordinator-only dependencies From acde92bd75576ef60fbf4a260b011cbda764c717 Mon Sep 17 00:00:00 2001 From: capistrant Date: Wed, 14 Jan 2026 13:02:03 -0600 Subject: [PATCH 65/72] Refactor compactionStateFingerprint out of the code in favor of indexingStateFingerprint --- .../compact/CompactionSupervisorTest.java | 6 +- .../common/task/AbstractBatchIndexTask.java | 10 +-- .../druid/indexing/common/task/IndexTask.java | 6 +- .../druid/indexing/common/task/Tasks.java | 6 +- .../parallel/ParallelIndexSupervisorTask.java | 6 +- .../CompactionConfigBasedJobTemplate.java | 6 +- .../druid/indexing/compact/CompactionJob.java | 16 ++-- .../indexing/compact/CompactionJobQueue.java | 8 +- .../compact/OverlordCompactionScheduler.java | 2 +- .../apache/druid/msq/exec/ControllerImpl.java | 6 +- .../input/table/DataSegmentWithLocation.java | 6 +- .../apache/druid/timeline/DataSegment.java | 40 +++++----- .../druid/timeline/DataSegmentTest.java | 34 ++++---- .../timeline/SegmentStatusInClusterTest.java | 4 +- .../IndexerSQLMetadataStorageCoordinator.java | 8 +- .../metadata/SqlSegmentsMetadataQuery.java | 48 +++++------ .../SqlSegmentMetadataTransaction.java | 2 +- .../cache/HeapMemorySegmentMetadataCache.java | 18 ++--- ...teRecord.java => IndexingStateRecord.java} | 6 +- .../server/compaction/CompactionStatus.java | 4 +- .../coordination/LoadableDataSegment.java | 4 +- .../coordinator/duty/CompactSegments.java | 8 +- .../druid/server/http/DataSegmentPlus.java | 16 ++-- ...exerSQLMetadataStorageCoordinatorTest.java | 6 +- .../metadata/SQLMetadataConnectorTest.java | 2 +- .../SqlSegmentsMetadataQueryTest.java | 80 +++++++++---------- .../HeapMemoryDatasourceSegmentCacheTest.java | 2 +- .../metadata/SqlIndexingStateStorageTest.java | 10 +-- .../compaction/CompactionStatusTest.java | 30 +++---- .../coordinator/CreateDataSegments.java | 14 ++-- .../server/http/DataSegmentPlusTest.java | 10 +-- .../druid/guice/MetadataManagerModule.java | 2 +- 32 files changed, 213 insertions(+), 213 deletions(-) rename server/src/main/java/org/apache/druid/metadata/segment/cache/{CompactionStateRecord.java => IndexingStateRecord.java} (87%) 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 8e82cdb824cb..8669440ec12f 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 @@ -280,8 +280,8 @@ private void verifySegmentsHaveNullLastCompactionStateAndNonNullFingerprint() "Segment " + segment.getId() + " should have null lastCompactionState" ); Assertions.assertNotNull( - segment.getCompactionStateFingerprint(), - "Segment " + segment.getId() + " should have non-null compactionStateFingerprint" + segment.getIndexingStateFingerprint(), + "Segment " + segment.getId() + " should have non-null indexingStateFingerprint" ); }); } @@ -305,7 +305,7 @@ private void verifyCompactedSegmentsHaveFingerprints(DataSourceCompactionConfig .forEach(segment -> { Assertions.assertEquals( expectedFingerprint, - segment.getCompactionStateFingerprint(), + segment.getIndexingStateFingerprint(), "Segment " + segment.getId() + " fingerprint should match expected fingerprint" ); }); 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 aa3f580a01c3..702f67b3bd12 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 @@ -642,17 +642,17 @@ public static boolean isGuaranteedRollup( } /** - * Returns a function that adds the given compaction state fingerprint to all segments. + * Returns a function that adds the given indexing state fingerprint to all segments. * If the fingerprint is null, returns an identity function that leaves segments unchanged. */ - public static Function, Set> addCompactionStateFingerprintToSegments( - String compactionStateFingerprint + public static Function, Set> addIndexingStateFingerprintToSegments( + String indexingStateFingerprint ) { - if (compactionStateFingerprint != null) { + if (indexingStateFingerprint != null) { return segments -> segments.stream() .map( - segment -> segment.withCompactionStateFingerprint(compactionStateFingerprint) + segment -> segment.withIndexingStateFingerprint(indexingStateFingerprint) ) .collect(Collectors.toSet()); } else { 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 abffa9c7860d..637a3f28c431 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 @@ -903,8 +903,8 @@ private TaskStatus generateAndPublishSegments( Tasks.DEFAULT_STORE_COMPACTION_STATE ); - final String compactionStateFingerprint = getContextValue( - Tasks.COMPACTION_STATE_FINGERPRINT_KEY, + final String indexingStateFingerprint = getContextValue( + Tasks.INDEXING_STATE_FINGERPRINT_KEY, null ); @@ -914,7 +914,7 @@ private TaskStatus generateAndPublishSegments( toolbox, ingestionSchema ).andThen( - addCompactionStateFingerprintToSegments(compactionStateFingerprint) + addIndexingStateFingerprintToSegments(indexingStateFingerprint) ); 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 b90e4bfd2f91..7f64a6cbad0d 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 @@ -70,11 +70,11 @@ public class Tasks } /** - * Context k:v pair that holds the fingerprint of the compaction state to be stored with the segment + * Context k:v pair that holds the fingerprint of the indexing state to be stored with the segment */ - public static final String COMPACTION_STATE_FINGERPRINT_KEY = "compactionStateFingerprint"; + public static final String INDEXING_STATE_FINGERPRINT_KEY = "indexingStateFingerprint"; static { - Verify.verify(COMPACTION_STATE_FINGERPRINT_KEY.equals(CompactSegments.COMPACTION_STATE_FINGERPRINT_KEY)); + Verify.verify(INDEXING_STATE_FINGERPRINT_KEY.equals(CompactSegments.INDEXING_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 38438d21b50b..02f11e8f5e20 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,8 +1162,8 @@ private void publishSegments( Tasks.STORE_COMPACTION_STATE_KEY, Tasks.DEFAULT_STORE_COMPACTION_STATE ); - final String compactionStateFingerprint = getContextValue( - Tasks.COMPACTION_STATE_FINGERPRINT_KEY, + final String indexingStateFingerprint = getContextValue( + Tasks.INDEXING_STATE_FINGERPRINT_KEY, null ); @@ -1173,7 +1173,7 @@ private void publishSegments( toolbox, ingestionSchema ).andThen( - addCompactionStateFingerprintToSegments(compactionStateFingerprint) + addIndexingStateFingerprintToSegments(indexingStateFingerprint) ); Set tombStones = Collections.emptySet(); 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 b41e13d43f2f..24b6e1f6af40 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 @@ -73,7 +73,7 @@ public List createCompactionJobs( CompactionState compactionState = config.toCompactionState(); - String compactionStateFingerprint = params.getFingerprintMapper().generateFingerprint( + String indexingStateFingerprint = params.getFingerprintMapper().generateFingerprint( config.getDataSource(), compactionState ); @@ -86,7 +86,7 @@ public List createCompactionJobs( candidate, config, params.getClusterCompactionConfig().getEngine(), - compactionStateFingerprint, + indexingStateFingerprint, params.getClusterCompactionConfig().isStoreCompactionStatePerSegment() ); jobs.add( @@ -94,7 +94,7 @@ public List createCompactionJobs( taskPayload, candidate, CompactionSlotManager.computeSlotsRequiredForTask(taskPayload), - compactionStateFingerprint, + indexingStateFingerprint, compactionState ) ); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionJob.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionJob.java index 93fba80a0e4d..51a5823357a1 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionJob.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionJob.java @@ -32,21 +32,21 @@ public class CompactionJob extends BatchIndexingJob { private final CompactionCandidate candidate; private final int maxRequiredTaskSlots; - private final String targetCompactionStateFingerprint; + private final String targetIndexingStateFingerprint; private final CompactionState targetCompactionState; public CompactionJob( ClientCompactionTaskQuery task, CompactionCandidate candidate, int maxRequiredTaskSlots, - String targetCompactionStateFingerprint, + String targetIndexingStateFingerprint, CompactionState targetCompactionState ) { super(task, null); this.candidate = candidate; this.maxRequiredTaskSlots = maxRequiredTaskSlots; - this.targetCompactionStateFingerprint = targetCompactionStateFingerprint; + this.targetIndexingStateFingerprint = targetIndexingStateFingerprint; this.targetCompactionState = targetCompactionState; } @@ -54,14 +54,14 @@ public CompactionJob( ClientSqlQuery msqQuery, CompactionCandidate candidate, int maxRequiredTaskSlots, - String targetCompactionStateFingerprint, + String targetIndexingStateFingerprint, CompactionState targetCompactionState ) { super(null, msqQuery); this.candidate = candidate; this.maxRequiredTaskSlots = maxRequiredTaskSlots; - this.targetCompactionStateFingerprint = targetCompactionStateFingerprint; + this.targetIndexingStateFingerprint = targetIndexingStateFingerprint; this.targetCompactionState = targetCompactionState; } @@ -80,9 +80,9 @@ public int getMaxRequiredTaskSlots() return maxRequiredTaskSlots; } - public String getTargetCompactionStateFingerprint() + public String getTargetIndexingStateFingerprint() { - return targetCompactionStateFingerprint; + return targetIndexingStateFingerprint; } public CompactionState getTargetCompactionState() @@ -97,7 +97,7 @@ public String toString() super.toString() + ", candidate=" + candidate + ", maxRequiredTaskSlots=" + maxRequiredTaskSlots + - ", targetCompactionStateFingerprint='" + targetCompactionStateFingerprint + '\'' + + ", targetIndexingStateFingerprint='" + targetIndexingStateFingerprint + '\'' + ", targetCompactionState=" + targetCompactionState + '}'; } 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 f4ef9a61a741..461eb077ee15 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,9 +36,9 @@ 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.IndexingStateStorage; import org.apache.druid.segment.metadata.DefaultCompactionFingerprintMapper; import org.apache.druid.segment.metadata.IndexingStateCache; +import org.apache.druid.segment.metadata.IndexingStateStorage; import org.apache.druid.server.compaction.CompactionCandidate; import org.apache.druid.server.compaction.CompactionCandidateSearchPolicy; import org.apache.druid.server.compaction.CompactionSlotManager; @@ -368,14 +368,14 @@ private String startTaskIfReady(CompactionJob job) */ private void persistPendingIndexingState(CompactionJob job) { - if (job.getTargetCompactionState() != null && job.getTargetCompactionStateFingerprint() != null) { + if (job.getTargetCompactionState() != null && job.getTargetIndexingStateFingerprint() != null) { indexingStateStorage.upsertIndexingState( job.getDataSource(), - job.getTargetCompactionStateFingerprint(), + job.getTargetIndexingStateFingerprint(), job.getTargetCompactionState(), DateTimes.nowUtc() ); - indexingStateCache.addIndexingState(job.getTargetCompactionStateFingerprint(), job.getTargetCompactionState()); + indexingStateCache.addIndexingState(job.getTargetIndexingStateFingerprint(), job.getTargetCompactionState()); } } 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 e2906a7fabef..cdc38fa16815 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 @@ -46,8 +46,8 @@ 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.IndexingStateStorage; import org.apache.druid.segment.metadata.IndexingStateCache; +import org.apache.druid.segment.metadata.IndexingStateStorage; import org.apache.druid.server.compaction.CompactionRunSimulator; import org.apache.druid.server.compaction.CompactionSimulateResult; import org.apache.druid.server.compaction.CompactionStatusTracker; 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 be72dc588bdc..51851cadd0b7 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 @@ -1700,7 +1700,7 @@ private void handleQueryResults( Tasks.DEFAULT_STORE_COMPACTION_STATE ); - final String compactionStateFingerprint = querySpec.getContext().getString(Tasks.COMPACTION_STATE_FINGERPRINT_KEY); + final String indexingStateFingerprint = querySpec.getContext().getString(Tasks.INDEXING_STATE_FINGERPRINT_KEY); if (storeCompactionState) { DataSourceMSQDestination destination = (DataSourceMSQDestination) querySpec.getDestination(); @@ -1727,9 +1727,9 @@ private void handleQueryResults( ); } } - if (compactionStateFingerprint != null) { + if (indexingStateFingerprint != null) { compactionStateAnnotateFunction = compactionStateAnnotateFunction.andThen( - AbstractBatchIndexTask.addCompactionStateFingerprintToSegments(compactionStateFingerprint)); + AbstractBatchIndexTask.addIndexingStateFingerprintToSegments(indexingStateFingerprint)); } log.info("Query [%s] publishing %d segments.", queryDef.getQueryId(), segments.size()); publishAllSegments(segments, compactionStateAnnotateFunction); 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 1714fe072992..2f97ed0b8b30 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,7 +61,7 @@ private DataSegmentWithLocation( @JsonProperty("binaryVersion") Integer binaryVersion, @JsonProperty("size") long size, @JsonProperty("servers") Set servers, - @JsonProperty("compactionStateFingerprint") String compactionStateFingerprint, + @JsonProperty("indexingStateFingerprint") String indexingStateFingerprint, @JacksonInject PruneSpecsHolder pruneSpecsHolder ) { @@ -77,7 +77,7 @@ private DataSegmentWithLocation( lastCompactionState, binaryVersion, size, - compactionStateFingerprint, + indexingStateFingerprint, pruneSpecsHolder ); this.servers = Preconditions.checkNotNull(servers, "servers"); @@ -100,7 +100,7 @@ public DataSegmentWithLocation( null, dataSegment.getBinaryVersion(), dataSegment.getSize(), - dataSegment.getCompactionStateFingerprint(), + dataSegment.getIndexingStateFingerprint(), PruneSpecsHolder.DEFAULT ); this.servers = servers; 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 45e6f0bc9a43..cfd34b1c2aae 100644 --- a/processing/src/main/java/org/apache/druid/timeline/DataSegment.java +++ b/processing/src/main/java/org/apache/druid/timeline/DataSegment.java @@ -117,12 +117,12 @@ public static class PruneSpecsHolder /** * SHA-256 fingerprint representation of the CompactionState. *

    - * A null fingerprint indicates that this segment either has not been compacted, or was compacted before compaction - * fingerprinting existed. In the latter case, the segment would have a non-null {@link #lastCompactionState}. + * A null fingerprint indicates that this segment either has not been compacted, or was compacted before indexing + * state fingerprinting existed. In the latter case, the segment would have a non-null {@link #lastCompactionState}. *

    */ @Nullable - private final String compactionStateFingerprint; + private final String indexingStateFingerprint; /** * @deprecated use {@link #builder(SegmentId)} or {@link #builder(DataSegment)} instead. @@ -208,7 +208,7 @@ private DataSegment( @JsonProperty("lastCompactionState") @Nullable CompactionState lastCompactionState, @JsonProperty("binaryVersion") Integer binaryVersion, @JsonProperty("size") long size, - @JsonProperty("compactionStateFingerprint") @Nullable String compactionStateFingerprint, + @JsonProperty("indexingStateFingerprint") @Nullable String indexingStateFingerprint, @JacksonInject PruneSpecsHolder pruneSpecsHolder ) { @@ -224,7 +224,7 @@ private DataSegment( lastCompactionState, binaryVersion, size, - compactionStateFingerprint, + indexingStateFingerprint, pruneSpecsHolder ); } @@ -241,7 +241,7 @@ public DataSegment( @Nullable CompactionState lastCompactionState, Integer binaryVersion, long size, - String compactionStateFingerprint, + String indexingStateFingerprint, PruneSpecsHolder pruneSpecsHolder ) { @@ -260,9 +260,9 @@ public DataSegment( this.binaryVersion = binaryVersion; Preconditions.checkArgument(size >= 0); this.size = size; - this.compactionStateFingerprint = compactionStateFingerprint == null ? - null : - STRING_INTERNER.intern(compactionStateFingerprint); + this.indexingStateFingerprint = indexingStateFingerprint == null ? + null : + STRING_INTERNER.intern(indexingStateFingerprint); } /** @@ -367,9 +367,9 @@ public boolean isTombstone() @Nullable @JsonProperty @JsonInclude(JsonInclude.Include.NON_NULL) - public String getCompactionStateFingerprint() + public String getIndexingStateFingerprint() { - return compactionStateFingerprint; + return indexingStateFingerprint; } @Override @@ -466,9 +466,9 @@ public DataSegment withLastCompactionState(CompactionState compactionState) return builder(this).lastCompactionState(compactionState).build(); } - public DataSegment withCompactionStateFingerprint(String compactionStateFingerprint) + public DataSegment withIndexingStateFingerprint(String indexingStateFingerprint) { - return builder(this).compactionStateFingerprint(compactionStateFingerprint).build(); + return builder(this).indexingStateFingerprint(indexingStateFingerprint).build(); } @Override @@ -505,7 +505,7 @@ public String toString() ", shardSpec=" + shardSpec + ", lastCompactionState=" + lastCompactionState + ", size=" + size + - ", compactionStateFingerprint=" + compactionStateFingerprint + + ", indexingStateFingerprint=" + indexingStateFingerprint + '}'; } @@ -579,7 +579,7 @@ public static class Builder private CompactionState lastCompactionState; private Integer binaryVersion; private long size; - private String compactionStateFingerprint; + private String indexingStateFingerprint; /** * @deprecated use {@link #Builder(SegmentId)} or {@link #Builder(DataSegment)} instead. @@ -605,7 +605,7 @@ private Builder(SegmentId segmentId) this.binaryVersion = 0; this.size = 0; this.lastCompactionState = null; - this.compactionStateFingerprint = null; + this.indexingStateFingerprint = null; } private Builder(DataSegment segment) @@ -621,7 +621,7 @@ private Builder(DataSegment segment) this.lastCompactionState = segment.getLastCompactionState(); this.binaryVersion = segment.getBinaryVersion(); this.size = segment.getSize(); - this.compactionStateFingerprint = segment.getCompactionStateFingerprint(); + this.indexingStateFingerprint = segment.getIndexingStateFingerprint(); } public Builder dataSource(String dataSource) @@ -690,9 +690,9 @@ public Builder size(long size) return this; } - public Builder compactionStateFingerprint(String compactionStateFingerprint) + public Builder indexingStateFingerprint(String indexingStateFingerprint) { - this.compactionStateFingerprint = compactionStateFingerprint; + this.indexingStateFingerprint = indexingStateFingerprint; return this; } @@ -716,7 +716,7 @@ public DataSegment build() lastCompactionState, binaryVersion, size, - compactionStateFingerprint, + indexingStateFingerprint, PruneSpecsHolder.DEFAULT ); } 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 f4ced3dfd38f..6807e6f334f2 100644 --- a/processing/src/test/java/org/apache/druid/timeline/DataSegmentTest.java +++ b/processing/src/test/java/org/apache/druid/timeline/DataSegmentTest.java @@ -514,7 +514,7 @@ public void testTombstoneType() } @Test - public void testSerializationWithCompactionStateFingerprint() throws Exception + public void testSerializationWithIndexingStateFingerprint() throws Exception { final Interval interval = Intervals.of("2011-10-01/2011-10-02"); final ImmutableMap loadSpec = ImmutableMap.of("something", "or_other"); @@ -525,7 +525,7 @@ public void testSerializationWithCompactionStateFingerprint() throws Exception .loadSpec(loadSpec) .dimensions(Arrays.asList("dim1", "dim2")) .metrics(Arrays.asList("met1", "met2")) - .compactionStateFingerprint(fingerprint) + .indexingStateFingerprint(fingerprint) .binaryVersion(TEST_VERSION) .size(1) .build(); @@ -535,16 +535,16 @@ public void testSerializationWithCompactionStateFingerprint() throws Exception MAPPER.writeValueAsString(segment), JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT ); - Assert.assertEquals(fingerprint, objectMap.get("compactionStateFingerprint")); + Assert.assertEquals(fingerprint, objectMap.get("indexingStateFingerprint")); // Verify deserialization preserves fingerprint DataSegment deserializedSegment = MAPPER.readValue(MAPPER.writeValueAsString(segment), DataSegment.class); - Assert.assertEquals(fingerprint, deserializedSegment.getCompactionStateFingerprint()); + Assert.assertEquals(fingerprint, deserializedSegment.getIndexingStateFingerprint()); Assert.assertEquals(segment.hashCode(), deserializedSegment.hashCode()); } @Test - public void testSerializationWithNullCompactionStateFingerprint() throws Exception + public void testSerializationWithNullIndexingStateFingerprint() throws Exception { final Interval interval = Intervals.of("2011-10-01/2011-10-02"); final ImmutableMap loadSpec = ImmutableMap.of("something", "or_other"); @@ -554,7 +554,7 @@ public void testSerializationWithNullCompactionStateFingerprint() throws Excepti .loadSpec(loadSpec) .dimensions(Arrays.asList("dim1", "dim2")) .metrics(Arrays.asList("met1", "met2")) - .compactionStateFingerprint(null) + .indexingStateFingerprint(null) .binaryVersion(TEST_VERSION) .size(1) .build(); @@ -564,19 +564,19 @@ public void testSerializationWithNullCompactionStateFingerprint() throws Excepti MAPPER.writeValueAsString(segment), JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT ); - Assert.assertFalse("compactionStateFingerprint should not be in JSON when null", - objectMap.containsKey("compactionStateFingerprint")); + Assert.assertFalse("indexingStateFingerprint should not be in JSON when null", + objectMap.containsKey("indexingStateFingerprint")); // Verify deserialization handles missing fingerprint DataSegment deserializedSegment = MAPPER.readValue(MAPPER.writeValueAsString(segment), DataSegment.class); - Assert.assertNull(deserializedSegment.getCompactionStateFingerprint()); + Assert.assertNull(deserializedSegment.getIndexingStateFingerprint()); Assert.assertEquals(segment.hashCode(), deserializedSegment.hashCode()); } @Test - public void testDeserializationBackwardCompatibility_missingCompactionStateFingerprint() throws Exception + public void testDeserializationBackwardCompatibility_missingIndexingStateFingerprint() throws Exception { - // Simulate JSON from old Druid version without compactionStateFingerprint field + // Simulate JSON from old Druid version without indexingStateFingerprint field String jsonWithoutFingerprint = "{" + "\"dataSource\": \"something\"," + "\"interval\": \"2011-10-01T00:00:00.000Z/2011-10-02T00:00:00.000Z\"," @@ -590,14 +590,14 @@ public void testDeserializationBackwardCompatibility_missingCompactionStateFinge + "}"; DataSegment deserializedSegment = MAPPER.readValue(jsonWithoutFingerprint, DataSegment.class); - Assert.assertNull("compactionStateFingerprint should be null for backward compatibility", - deserializedSegment.getCompactionStateFingerprint()); + Assert.assertNull("indexingStateFingerprint should be null for backward compatibility", + deserializedSegment.getIndexingStateFingerprint()); Assert.assertEquals("something", deserializedSegment.getDataSource()); Assert.assertEquals(Intervals.of("2011-10-01/2011-10-02"), deserializedSegment.getInterval()); } @Test - public void testWithCompactionStateFingerprint() + public void testWithIndexingStateFingerprint() { final String fingerprint = "test_fingerprint_12345"; final Interval interval = Intervals.of("2012-01-01/2012-01-02"); @@ -607,14 +607,14 @@ public void testWithCompactionStateFingerprint() final DataSegment segment1 = DataSegment.builder(segmentId) .size(0) - .compactionStateFingerprint(fingerprint) + .indexingStateFingerprint(fingerprint) .build(); final DataSegment segment2 = DataSegment.builder(segmentId) .size(0) .build(); - DataSegment withFingerprint = segment2.withCompactionStateFingerprint(fingerprint); - Assert.assertEquals(fingerprint, withFingerprint.getCompactionStateFingerprint()); + DataSegment withFingerprint = segment2.withIndexingStateFingerprint(fingerprint); + Assert.assertEquals(fingerprint, withFingerprint.getIndexingStateFingerprint()); Assert.assertEquals(segment1, withFingerprint); } 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 50e1c1781410..0413d9216165 100644 --- a/processing/src/test/java/org/apache/druid/timeline/SegmentStatusInClusterTest.java +++ b/processing/src/test/java/org/apache/druid/timeline/SegmentStatusInClusterTest.java @@ -163,7 +163,7 @@ public TestSegment( @JsonProperty("size") long size, @JsonProperty("overshadowed") boolean overshadowed, @JsonProperty("replicationFactor") Integer replicationFactor, - @JsonProperty("compactionStateFingerprint") String compactionStateFingerprint + @JsonProperty("indexingStateFingerprint") String indexingStateFingerprint ) { super( @@ -178,7 +178,7 @@ public TestSegment( lastCompactionState, binaryVersion, size, - compactionStateFingerprint, + indexingStateFingerprint, PruneSpecsHolder.DEFAULT ); this.overshadowed = overshadowed; 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 508c59aa439b..ef06a3d0a8ce 100644 --- a/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java +++ b/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java @@ -1833,7 +1833,7 @@ protected Set insertSegments( segmentMetadata == null ? null : segmentMetadata.getSchemaFingerprint(), segmentMetadata == null ? null : segmentMetadata.getNumRows(), null, - segment.getCompactionStateFingerprint() + segment.getIndexingStateFingerprint() ); }).collect(Collectors.toSet()); @@ -1956,7 +1956,7 @@ private Set createNewIdsOfAppendSegmentsAfterReplace( oldSegmentMetadata.getSchemaFingerprint(), oldSegmentMetadata.getNumRows(), upgradedFromSegmentId, - oldSegmentMetadata.getCompactionStateFingerprint() + oldSegmentMetadata.getIndexingStateFingerprint() ) ); } @@ -2049,7 +2049,7 @@ private Set insertSegments( segmentMetadata == null ? null : segmentMetadata.getSchemaFingerprint(), segmentMetadata == null ? null : segmentMetadata.getNumRows(), upgradedFromSegmentIdMap.get(segment.getId().toString()), - segment.getCompactionStateFingerprint() + segment.getIndexingStateFingerprint() ); }).collect(Collectors.toSet()); @@ -2733,7 +2733,7 @@ private void markCompactionFingerprintsAsActive(Set segments) // Collect unique non-null compaction state fingerprints final Set fingerprints = segments.stream() - .map(DataSegment::getCompactionStateFingerprint) + .map(DataSegment::getIndexingStateFingerprint) .filter(fp -> fp != null && !fp.isEmpty()) .collect(Collectors.toSet()); 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 ed3159c7fbbd..bf4e20c86a63 100644 --- a/server/src/main/java/org/apache/druid/metadata/SqlSegmentsMetadataQuery.java +++ b/server/src/main/java/org/apache/druid/metadata/SqlSegmentsMetadataQuery.java @@ -40,7 +40,7 @@ import org.apache.druid.java.util.common.jackson.JacksonUtils; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.java.util.common.parsers.CloseableIterator; -import org.apache.druid.metadata.segment.cache.CompactionStateRecord; +import org.apache.druid.metadata.segment.cache.IndexingStateRecord; import org.apache.druid.metadata.segment.cache.SegmentSchemaRecord; import org.apache.druid.segment.SchemaPayload; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; @@ -1706,12 +1706,12 @@ private SegmentSchemaRecord mapToSchemaRecord(ResultSet resultSet) } /** - * Retrieves all unique compaction state fingerprints currently marked as used. + * Retrieves all unique indexing state fingerprints currently marked as used. * This is used for delta syncs to determine which fingerprints are still active. * - * @return Set of compaction state fingerprints + * @return Set of indexing state fingerprints */ - public Set retrieveAllUsedCompactionStateFingerprints() + public Set retrieveAllUsedIndexingStateFingerprints() { final String sql = StringUtils.format( "SELECT fingerprint FROM %s WHERE used = true", @@ -1727,28 +1727,28 @@ public Set retrieveAllUsedCompactionStateFingerprints() } /** - * Retrieves all compaction states marked as used (full sync). + * Retrieves all indexing states marked as used (full sync). * - * @return List of CompactionStateRecord objects + * @return List of IndexingStateRecord objects */ - public List retrieveAllUsedCompactionStates() + public List retrieveAllUsedIndexingStates() { final String sql = StringUtils.format( "SELECT fingerprint, payload FROM %s WHERE used = true", dbTables.getIndexingStatesTable() ); - return retrieveValidCompactionStateRecordsWithQuery(handle.createQuery(sql)); + return retrieveValidIndexingStateRecordsWithQuery(handle.createQuery(sql)); } /** - * Retrieves compaction states for specific fingerprints (delta sync). - * Used to fetch only newly added compaction states. + * Retrieves indexing states for specific fingerprints (delta sync). + * Used to fetch only newly added indexing states. * * @param fingerprints Set of fingerprints to retrieve - * @return List of CompactionStateRecord objects + * @return List of IndexingStateRecord objects */ - public List retrieveCompactionStatesForFingerprints( + public List retrieveIndexingStatesForFingerprints( Set fingerprints ) { @@ -1757,10 +1757,10 @@ public List retrieveCompactionStatesForFingerprints( MAX_INTERVALS_PER_BATCH ); - final List records = new ArrayList<>(); + final List records = new ArrayList<>(); for (List fingerprintBatch : fingerprintBatches) { records.addAll( - retrieveBatchOfCompactionStates(fingerprintBatch) + retrieveBatchOfIndexingStates(fingerprintBatch) ); } @@ -1768,9 +1768,9 @@ public List retrieveCompactionStatesForFingerprints( } /** - * Retrieves a batch of compaction state records for the given fingerprints. + * Retrieves a batch of indexing state records for the given fingerprints. */ - private List retrieveBatchOfCompactionStates(List fingerprints) + private List retrieveBatchOfIndexingStates(List fingerprints) { final String sql = StringUtils.format( "SELECT fingerprint, payload FROM %s" @@ -1783,19 +1783,19 @@ private List retrieveBatchOfCompactionStates(List final Query> query = handle.createQuery(sql); bindColumnValuesToQueryWithInCondition("fingerprint", fingerprints, query); - return retrieveValidCompactionStateRecordsWithQuery(query); + return retrieveValidIndexingStateRecordsWithQuery(query); } /** - * Executes the given query and maps results to valid CompactionStateRecord objects. + * Executes the given query and maps results to valid IndexingStateRecord objects. * Records that fail to parse are filtered out. */ - private List retrieveValidCompactionStateRecordsWithQuery( + private List retrieveValidIndexingStateRecordsWithQuery( Query> query ) { return query.setFetchSize(connector.getStreamingFetchSize()) - .map((index, r, ctx) -> mapToCompactionStateRecord(r)) + .map((index, r, ctx) -> mapToIndexingStateRecord(r)) .list() .stream() .filter(Objects::nonNull) @@ -1803,22 +1803,22 @@ private List retrieveValidCompactionStateRecordsWithQuery } /** - * Tries to parse the fields of the result set into a {@link CompactionStateRecord}. + * Tries to parse the fields of the result set into a {@link IndexingStateRecord}. * * @return null if an error occurred while parsing the result */ - private CompactionStateRecord mapToCompactionStateRecord(ResultSet resultSet) + private IndexingStateRecord mapToIndexingStateRecord(ResultSet resultSet) { String fingerprint = null; try { fingerprint = resultSet.getString("fingerprint"); - return new CompactionStateRecord( + return new IndexingStateRecord( fingerprint, jsonMapper.readValue(resultSet.getBytes("payload"), CompactionState.class) ); } catch (Throwable t) { - log.error(t, "Could not read compaction state with fingerprint[%s]", fingerprint); + log.error(t, "Could not read indexing state with fingerprint[%s]", fingerprint); return 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 bb06c0a0bf12..4e144433260f 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 @@ -533,7 +533,7 @@ private int insertSegmentsInBatches( .bind("payload", getJsonBytes(segment)) .bind("used_status_last_updated", toNonNullString(segmentPlus.getUsedStatusLastUpdatedDate())) .bind("upgraded_from_segment_id", segmentPlus.getUpgradedFromSegmentId()) - .bind("indexing_state_fingerprint", segmentPlus.getCompactionStateFingerprint()); + .bind("indexing_state_fingerprint", segmentPlus.getIndexingStateFingerprint()); 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 7f3440d20be4..653223a5e6de 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 @@ -1154,14 +1154,14 @@ private void retrieveAndResetUsedCompactionStates() */ private Map buildFingerprintToStateMapForFullSync() { - final List records = query( - SqlSegmentsMetadataQuery::retrieveAllUsedCompactionStates + final List records = query( + SqlSegmentsMetadataQuery::retrieveAllUsedIndexingStates ); return records.stream().collect( Collectors.toMap( - CompactionStateRecord::getFingerprint, - CompactionStateRecord::getState + IndexingStateRecord::getFingerprint, + IndexingStateRecord::getState ) ); } @@ -1183,7 +1183,7 @@ private Map buildFingerprintToStateMapForDeltaSync() ); final Set cachedFingerprints = Set.copyOf(fingerprintToStateMap.keySet()); final Set persistedFingerprints = query( - SqlSegmentsMetadataQuery::retrieveAllUsedCompactionStateFingerprints + SqlSegmentsMetadataQuery::retrieveAllUsedIndexingStateFingerprints ); // Remove entry for compaction states that have been deleted from the metadata store @@ -1193,14 +1193,14 @@ private Map buildFingerprintToStateMapForDeltaSync() // Retrieve and add entry for compaction states that have been added to the metadata store final Set addedFingerprints = Sets.difference(persistedFingerprints, cachedFingerprints); - final List addedCompactionStateRecords = query( - sql -> sql.retrieveCompactionStatesForFingerprints(addedFingerprints) + final List addedIndexingStateRecords = query( + sql -> sql.retrieveIndexingStatesForFingerprints(addedFingerprints) ); - addedCompactionStateRecords.forEach( + addedIndexingStateRecords.forEach( record -> fingerprintToStateMap.put(record.getFingerprint(), record.getState()) ); - emitMetric(Metric.ADDED_COMPACTION_STATES, addedCompactionStateRecords.size()); + emitMetric(Metric.ADDED_COMPACTION_STATES, addedIndexingStateRecords.size()); return fingerprintToStateMap; } diff --git a/server/src/main/java/org/apache/druid/metadata/segment/cache/CompactionStateRecord.java b/server/src/main/java/org/apache/druid/metadata/segment/cache/IndexingStateRecord.java similarity index 87% rename from server/src/main/java/org/apache/druid/metadata/segment/cache/CompactionStateRecord.java rename to server/src/main/java/org/apache/druid/metadata/segment/cache/IndexingStateRecord.java index d6bbd268c2d2..b04484d87d58 100644 --- a/server/src/main/java/org/apache/druid/metadata/segment/cache/CompactionStateRecord.java +++ b/server/src/main/java/org/apache/druid/metadata/segment/cache/IndexingStateRecord.java @@ -22,14 +22,14 @@ import org.apache.druid.timeline.CompactionState; /** - * Represents a single record in the druid_compactionStates table. + * Represents a single record in the druid_indexingStates table. */ -public class CompactionStateRecord +public class IndexingStateRecord { private final String fingerprint; private final CompactionState state; - public CompactionStateRecord(String fingerprint, CompactionState state) + public IndexingStateRecord(String fingerprint, CompactionState state) { this.fingerprint = fingerprint; this.state = state; 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 cdff901aa192..e9caf8b48564 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 @@ -441,7 +441,7 @@ private CompactionStatus allFingerprintedCandidatesHaveExpectedFingerprint() { Map> mismatchedFingerprintToSegmentMap = new HashMap<>(); for (DataSegment segment : fingerprintedSegments) { - String fingerprint = segment.getCompactionStateFingerprint(); + String fingerprint = segment.getIndexingStateFingerprint(); if (fingerprint == null) { // Should not happen since we are iterating over fingerprintedSegments } else if (fingerprint.equals(targetFingerprint)) { @@ -506,7 +506,7 @@ private CompactionStatus allFingerprintedCandidatesHaveExpectedFingerprint() private CompactionStatus segmentsHaveBeenCompactedAtLeastOnce() { for (DataSegment segment : candidateSegments.getSegments()) { - final String fingerprint = segment.getCompactionStateFingerprint(); + final String fingerprint = segment.getIndexingStateFingerprint(); final CompactionState segmentState = segment.getLastCompactionState(); if (fingerprint != null) { fingerprintedSegments.add(segment); 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 72281b00e98b..e28d06e5e330 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 @@ -54,7 +54,7 @@ private LoadableDataSegment( @JsonProperty("lastCompactionState") @Nullable CompactionState lastCompactionState, @JsonProperty("binaryVersion") Integer binaryVersion, @JsonProperty("size") long size, - @JsonProperty("compactionStateFingerprint") String compactionStateFingerprint + @JsonProperty("indexingStateFingerprint") String indexingStateFingerprint ) { super( @@ -69,7 +69,7 @@ private LoadableDataSegment( lastCompactionState, binaryVersion, size, - compactionStateFingerprint, + indexingStateFingerprint, PruneSpecsHolder.DEFAULT ); } 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 3e4de17a51c3..060c16edfadb 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 @@ -78,9 +78,9 @@ public class CompactSegments implements CoordinatorCustomDuty public static final String STORE_COMPACTION_STATE_KEY = "storeCompactionState"; /** - * Must be the same as org.apache.druid.indexing.common.task.Tasks.COMPACTION_STATE_FINGERPRINT_KEY + * Must be the same as org.apache.druid.indexing.common.task.Tasks.INDEXING_STATE_FINGERPRINT_KEY */ - public static final String COMPACTION_STATE_FINGERPRINT_KEY = "compactionStateFingerprint"; + public static final String INDEXING_STATE_FINGERPRINT_KEY = "indexingStateFingerprint"; private static final String COMPACTION_REASON_KEY = "compactionReason"; @@ -293,7 +293,7 @@ public static ClientCompactionTaskQuery createCompactionTask( CompactionCandidate candidate, DataSourceCompactionConfig config, CompactionEngine defaultEngine, - String compactionStateFingerprint, + String indexingStateFingerprint, boolean storeCompactionStatePerSegment ) { @@ -373,7 +373,7 @@ public static ClientCompactionTaskQuery createCompactionTask( } autoCompactionContext.put(STORE_COMPACTION_STATE_KEY, storeCompactionStatePerSegment); - autoCompactionContext.put(COMPACTION_STATE_FINGERPRINT_KEY, compactionStateFingerprint); + autoCompactionContext.put(INDEXING_STATE_FINGERPRINT_KEY, indexingStateFingerprint); return compactSegments( candidate, 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 4f760b33e31c..e06241f6aae7 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 @@ -59,7 +59,7 @@ public class DataSegmentPlus private final String upgradedFromSegmentId; @Nullable - private final String compactionStateFingerprint; + private final String indexingStateFingerprint; @JsonCreator public DataSegmentPlus( @@ -70,7 +70,7 @@ public DataSegmentPlus( @JsonProperty("schemaFingerprint") @Nullable final String schemaFingerprint, @JsonProperty("numRows") @Nullable final Long numRows, @JsonProperty("upgradedFromSegmentId") @Nullable final String upgradedFromSegmentId, - @JsonProperty("compactionStateFingerprint") @Nullable String compactionStateFingerprint + @JsonProperty("indexingStateFingerprint") @Nullable String indexingStateFingerprint ) { this.dataSegment = dataSegment; @@ -80,7 +80,7 @@ public DataSegmentPlus( this.schemaFingerprint = schemaFingerprint; this.numRows = numRows; this.upgradedFromSegmentId = upgradedFromSegmentId; - this.compactionStateFingerprint = compactionStateFingerprint; + this.indexingStateFingerprint = indexingStateFingerprint; } @Nullable @@ -133,9 +133,9 @@ public String getUpgradedFromSegmentId() @Nullable @JsonProperty - public String getCompactionStateFingerprint() + public String getIndexingStateFingerprint() { - return compactionStateFingerprint; + return indexingStateFingerprint; } @Override @@ -155,7 +155,7 @@ public boolean equals(Object o) && Objects.equals(schemaFingerprint, that.getSchemaFingerprint()) && Objects.equals(numRows, that.getNumRows()) && Objects.equals(upgradedFromSegmentId, that.getUpgradedFromSegmentId()) - && Objects.equals(compactionStateFingerprint, that.getCompactionStateFingerprint()); + && Objects.equals(indexingStateFingerprint, that.getIndexingStateFingerprint()); } @Override @@ -169,7 +169,7 @@ public int hashCode() schemaFingerprint, numRows, upgradedFromSegmentId, - compactionStateFingerprint + indexingStateFingerprint ); } @@ -184,7 +184,7 @@ public String toString() ", schemaFingerprint=" + getSchemaFingerprint() + ", numRows=" + getNumRows() + ", upgradedFromSegmentId=" + getUpgradedFromSegmentId() + - ", compactionStateFingerprint=" + getCompactionStateFingerprint() + + ", indexingStateFingerprint=" + getIndexingStateFingerprint() + '}'; } } diff --git a/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java b/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java index 40b96eaa429b..d7d9ab6e2ab1 100644 --- a/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java +++ b/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java @@ -4342,7 +4342,7 @@ public void testCommitSegmentsAndMetadata_marksPendingCompactionStateAsActive() final DataSegment segment = CreateDataSegments.ofDatasource(TestDataSource.WIKI) .startingAt("2023-01-01") - .withCompactionStateFingerprint(fingerprint) + .withIndexingStateFingerprint(fingerprint) .eachOfSizeInMb(500) .get(0); @@ -4367,7 +4367,7 @@ public void testCommitReplaceSegments_marksPendingCompactionStateAsActive() final DataSegment segment = CreateDataSegments.ofDatasource(TestDataSource.WIKI) .startingAt("2023-01-01") - .withCompactionStateFingerprint(fingerprint) + .withIndexingStateFingerprint(fingerprint) .eachOfSizeInMb(500) .get(0); @@ -4397,7 +4397,7 @@ public void testCCommitAppendSegments_marksPendingCompactionStateAsActive() final DataSegment segment = CreateDataSegments.ofDatasource(TestDataSource.WIKI) .startingAt("2023-01-01") - .withCompactionStateFingerprint(fingerprint) + .withIndexingStateFingerprint(fingerprint) .eachOfSizeInMb(500) .get(0); 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 12b267838ef2..504d26549ad6 100644 --- a/server/src/test/java/org/apache/druid/metadata/SQLMetadataConnectorTest.java +++ b/server/src/test/java/org/apache/druid/metadata/SQLMetadataConnectorTest.java @@ -194,7 +194,7 @@ public void testAlterSegmentTableAddLastUsed() * in the segments table. */ @Test - public void testAlterSegmentTableAddCompactionStateFingerprint() + public void testAlterSegmentTableAddIndexingStateFingerprint() { connector.createSegmentTable(); derbyConnectorRule.segments().update("ALTER TABLE %1$s DROP COLUMN INDEXING_STATE_FINGERPRINT"); diff --git a/server/src/test/java/org/apache/druid/metadata/SqlSegmentsMetadataQueryTest.java b/server/src/test/java/org/apache/druid/metadata/SqlSegmentsMetadataQueryTest.java index c8747fadffe3..f01d3e002f54 100644 --- a/server/src/test/java/org/apache/druid/metadata/SqlSegmentsMetadataQueryTest.java +++ b/server/src/test/java/org/apache/druid/metadata/SqlSegmentsMetadataQueryTest.java @@ -27,7 +27,7 @@ import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.parsers.CloseableIterator; -import org.apache.druid.metadata.segment.cache.CompactionStateRecord; +import org.apache.druid.metadata.segment.cache.IndexingStateRecord; import org.apache.druid.metadata.storage.derby.DerbyConnector; import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.TestDataSource; @@ -378,17 +378,17 @@ private static Set getIds(Set segments) // ==================== Compaction State Tests ==================== @Test - public void test_retrieveAllUsedCompactionStateFingerprints_emptyDatabase() + public void test_retrieveAllUsedIndexingStateFingerprints_emptyDatabase() { derbyConnectorRule.getConnector().createIndexingStatesTable(); - Set fingerprints = read(SqlSegmentsMetadataQuery::retrieveAllUsedCompactionStateFingerprints); + Set fingerprints = read(SqlSegmentsMetadataQuery::retrieveAllUsedIndexingStateFingerprints); Assert.assertTrue("Should return empty set when no segments have compaction states", fingerprints.isEmpty()); } @Test - public void test_retrieveAllUsedCompactionStateFingerprints() + public void test_retrieveAllUsedIndexingStateFingerprints() { derbyConnectorRule.getConnector().createIndexingStatesTable(); @@ -405,13 +405,13 @@ public void test_retrieveAllUsedCompactionStateFingerprints() insertSegmentWithCompactionState("seg3", "fp1", true); // Duplicate fingerprint insertSegmentWithCompactionState("seg4", "fp3", false); // Unused segment - Set fingerprints = read(SqlSegmentsMetadataQuery::retrieveAllUsedCompactionStateFingerprints); + Set fingerprints = read(SqlSegmentsMetadataQuery::retrieveAllUsedIndexingStateFingerprints); Assert.assertEquals("Should return all fingerprints in the cache", Set.of("fp1", "fp2", "fp3"), fingerprints); } @Test - public void test_retrieveAllUsedCompactionStateFingerprints_ignoresNullFingerprints() + public void test_retrieveAllUsedIndexingStateFingerprints_ignoresNullFingerprints() { derbyConnectorRule.getConnector().createIndexingStatesTable(); @@ -422,23 +422,23 @@ public void test_retrieveAllUsedCompactionStateFingerprints_ignoresNullFingerpri insertSegmentWithCompactionState("seg1", "fp1", true); insertSegmentWithCompactionState("seg2", null, true); // No compaction state - Set fingerprints = read(SqlSegmentsMetadataQuery::retrieveAllUsedCompactionStateFingerprints); + Set fingerprints = read(SqlSegmentsMetadataQuery::retrieveAllUsedIndexingStateFingerprints); Assert.assertEquals("Should ignore segments without compaction states", Set.of("fp1"), fingerprints); } @Test - public void test_retrieveAllUsedCompactionStates_emptyDatabase() + public void test_retrieveAllUsedIndexingStates_emptyDatabase() { derbyConnectorRule.getConnector().createIndexingStatesTable(); - List records = read(SqlSegmentsMetadataQuery::retrieveAllUsedCompactionStates); + List records = read(SqlSegmentsMetadataQuery::retrieveAllUsedIndexingStates); Assert.assertTrue("Should return empty list when no compaction states exist", records.isEmpty()); } @Test - public void test_retrieveAllUsedCompactionStates_fullSync() + public void test_retrieveAllUsedIndexingStates_fullSync() { derbyConnectorRule.getConnector().createIndexingStatesTable(); @@ -465,20 +465,20 @@ public void test_retrieveAllUsedCompactionStates_fullSync() insertSegmentWithCompactionState("seg1", "fp1", true); insertSegmentWithCompactionState("seg2", "fp2", true); - List records = read(SqlSegmentsMetadataQuery::retrieveAllUsedCompactionStates); + List records = read(SqlSegmentsMetadataQuery::retrieveAllUsedIndexingStates); Assert.assertEquals("Should return all compaction states", 3, records.size()); Set retrievedFingerprints = records.stream() - .map(CompactionStateRecord::getFingerprint) + .map(IndexingStateRecord::getFingerprint) .collect(Collectors.toSet()); Assert.assertEquals("Should contain all fps", Set.of("fp1", "fp2", "fp3"), retrievedFingerprints); // Verify payloads Map retrievedStates = records.stream() .collect(Collectors.toMap( - CompactionStateRecord::getFingerprint, - CompactionStateRecord::getState + IndexingStateRecord::getFingerprint, + IndexingStateRecord::getState )); Assert.assertEquals("fp1 state should match", state1, retrievedStates.get("fp1")); Assert.assertEquals("fp2 state should match", state2, retrievedStates.get("fp2")); @@ -498,13 +498,13 @@ public void test_retrieveAllUsedCompactionStates_onlyFromUsedSegments() insertSegmentWithCompactionState("seg1", "fp1", true); // Used insertSegmentWithCompactionState("seg2", "fp2", false); // Unused - List records = read(SqlSegmentsMetadataQuery::retrieveAllUsedCompactionStates); + List records = read(SqlSegmentsMetadataQuery::retrieveAllUsedIndexingStates); Assert.assertEquals("Should only return all compaction states", 2, records.size()); } @Test - public void test_retrieveAllUsedCompactionStates_ignoresUnusedCompactionStates() + public void test_retrieveAllUsedCompactionStates_ignoresUnusedIndexingStates() { derbyConnectorRule.getConnector().createIndexingStatesTable(); @@ -517,25 +517,25 @@ public void test_retrieveAllUsedCompactionStates_ignoresUnusedCompactionStates() // Mark compaction state as unused markCompactionStateAsUnused("fp1"); - List records = read(SqlSegmentsMetadataQuery::retrieveAllUsedCompactionStates); + List records = read(SqlSegmentsMetadataQuery::retrieveAllUsedIndexingStates); Assert.assertTrue("Should not return unused compaction states", records.isEmpty()); } @Test - public void test_retrieveCompactionStatesForFingerprints_emptyInput() + public void test_retrieveIndexingStatesForFingerprints_emptyInput() { derbyConnectorRule.getConnector().createIndexingStatesTable(); - List records = read( - sql -> sql.retrieveCompactionStatesForFingerprints(Set.of()) + List records = read( + sql -> sql.retrieveIndexingStatesForFingerprints(Set.of()) ); Assert.assertTrue("Should return empty list for empty input", records.isEmpty()); } @Test - public void test_retrieveCompactionStatesForFingerprints_deltaSync() + public void test_retrieveIndexingStatesForFingerprints_deltaSync() { derbyConnectorRule.getConnector().createIndexingStatesTable(); @@ -547,20 +547,20 @@ public void test_retrieveCompactionStatesForFingerprints_deltaSync() insertCompactionStates(compactionStates); // Request specific fingerprints (delta sync scenario) - List records = read( - sql -> sql.retrieveCompactionStatesForFingerprints(Set.of("fp1", "fp3")) + List records = read( + sql -> sql.retrieveIndexingStatesForFingerprints(Set.of("fp1", "fp3")) ); Assert.assertEquals("Should return requested fingerprints", 2, records.size()); Set retrievedFingerprints = records.stream() - .map(CompactionStateRecord::getFingerprint) + .map(IndexingStateRecord::getFingerprint) .collect(Collectors.toSet()); Assert.assertEquals("Should contain only requested fingerprints", Set.of("fp1", "fp3"), retrievedFingerprints); } @Test - public void test_retrieveCompactionStatesForFingerprints_largeBatch() + public void test_retrieveIndexingStatesForFingerprints_largeBatch() { derbyConnectorRule.getConnector().createIndexingStatesTable(); @@ -575,20 +575,20 @@ public void test_retrieveCompactionStatesForFingerprints_largeBatch() insertCompactionStates(compactionStates); // Request all fingerprints - List records = read( - sql -> sql.retrieveCompactionStatesForFingerprints(expectedFingerprints) + List records = read( + sql -> sql.retrieveIndexingStatesForFingerprints(expectedFingerprints) ); Assert.assertEquals("Should return all fingerprints across multiple batches", 150, records.size()); Set retrievedFingerprints = records.stream() - .map(CompactionStateRecord::getFingerprint) + .map(IndexingStateRecord::getFingerprint) .collect(Collectors.toSet()); Assert.assertEquals("Should contain all requested fingerprints", expectedFingerprints, retrievedFingerprints); } @Test - public void test_retrieveCompactionStatesForFingerprints_nonexistentFingerprints() + public void test_retrieveIndexingStatesForFingerprints_nonexistentFingerprints() { derbyConnectorRule.getConnector().createIndexingStatesTable(); @@ -597,15 +597,15 @@ public void test_retrieveCompactionStatesForFingerprints_nonexistentFingerprints insertCompactionStates(compactionStates); // Request fingerprints that don't exist - List records = read( - sql -> sql.retrieveCompactionStatesForFingerprints(Set.of("fp999", "fp888")) + List records = read( + sql -> sql.retrieveIndexingStatesForFingerprints(Set.of("fp999", "fp888")) ); Assert.assertTrue("Should return empty list when fingerprints don't exist", records.isEmpty()); } @Test - public void test_retrieveCompactionStatesForFingerprints_mixedExistingAndNonexistent() + public void test_retrieveIndexingStatesForFingerprints_mixedExistingAndNonexistent() { derbyConnectorRule.getConnector().createIndexingStatesTable(); @@ -615,20 +615,20 @@ public void test_retrieveCompactionStatesForFingerprints_mixedExistingAndNonexis insertCompactionStates(compactionStates); // Mix existing and non-existing fingerprints - List records = read( - sql -> sql.retrieveCompactionStatesForFingerprints(Set.of("fp1", "fp999", "fp2", "fp888")) + List records = read( + sql -> sql.retrieveIndexingStatesForFingerprints(Set.of("fp1", "fp999", "fp2", "fp888")) ); Assert.assertEquals("Should return only existing fingerprints", 2, records.size()); Set retrievedFingerprints = records.stream() - .map(CompactionStateRecord::getFingerprint) + .map(IndexingStateRecord::getFingerprint) .collect(Collectors.toSet()); Assert.assertEquals("Should contain only existing fingerprints", Set.of("fp1", "fp2"), retrievedFingerprints); } @Test - public void test_retrieveCompactionStatesForFingerprints_onlyReturnsUsedStates() + public void test_retrieveIndexingStatesForFingerprints_onlyReturnsUsedStates() { derbyConnectorRule.getConnector().createIndexingStatesTable(); @@ -640,8 +640,8 @@ public void test_retrieveCompactionStatesForFingerprints_onlyReturnsUsedStates() // Mark fp2 as unused markCompactionStateAsUnused("fp2"); - List records = read( - sql -> sql.retrieveCompactionStatesForFingerprints(Set.of("fp1", "fp2")) + List records = read( + sql -> sql.retrieveIndexingStatesForFingerprints(Set.of("fp1", "fp2")) ); Assert.assertEquals("Should only return used compaction states", 1, records.size()); @@ -683,7 +683,7 @@ private void insertCompactionStates(Map compactionState private void insertSegmentWithCompactionState( String segmentId, - String compactionStateFingerprint, + String indexingStateFingerprint, boolean used ) { @@ -708,7 +708,7 @@ private void insertSegmentWithCompactionState( .bind("used", used) .bind("payload", TestHelper.JSON_MAPPER.writeValueAsBytes(WIKI_SEGMENTS_2X5D.get(0))) .bind("used_status_last_updated", DateTimes.nowUtc().toString()) - .bind("indexing_state_fingerprint", compactionStateFingerprint) + .bind("indexing_state_fingerprint", indexingStateFingerprint) .execute(); return null; }); 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 b375b78df82a..ae3deda0eadd 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 @@ -781,7 +781,7 @@ private static DataSegmentPlus updateSegment(DataSegmentPlus segment, DateTime n segment.getSchemaFingerprint(), segment.getNumRows(), segment.getUpgradedFromSegmentId(), - segment.getCompactionStateFingerprint() + segment.getIndexingStateFingerprint() ); } diff --git a/server/src/test/java/org/apache/druid/segment/metadata/SqlIndexingStateStorageTest.java b/server/src/test/java/org/apache/druid/segment/metadata/SqlIndexingStateStorageTest.java index f987b2ed5e71..c5237add8d54 100644 --- a/server/src/test/java/org/apache/druid/segment/metadata/SqlIndexingStateStorageTest.java +++ b/server/src/test/java/org/apache/druid/segment/metadata/SqlIndexingStateStorageTest.java @@ -437,7 +437,7 @@ public void test_markIndexingStatesAsActive_nonExistentFingerprint_returnsZero() // ===== Fingerprint Generation Tests ===== @Test - public void test_generateCompactionStateFingerprint_deterministicFingerprinting() + public void test_generateIndexingStateFingerprint_deterministicFingerprinting() { CompactionState compactionState1 = createBasicCompactionState(); CompactionState compactionState2 = createBasicCompactionState(); @@ -453,7 +453,7 @@ public void test_generateCompactionStateFingerprint_deterministicFingerprinting( } @Test - public void test_generateCompactionStateFingerprint_differentDatasourcesWithSameState_differentFingerprints() + public void test_generateIndexingStateFingerprint_differentDatasourcesWithSameState_differentFingerprints() { CompactionState compactionState = createBasicCompactionState(); @@ -468,7 +468,7 @@ public void test_generateCompactionStateFingerprint_differentDatasourcesWithSame } @Test - public void test_generateCompactionStateFingerprint_metricsListOrderDifferenceResultsInNewFingerprint() + public void test_generateIndexingStateFingerprint_metricsListOrderDifferenceResultsInNewFingerprint() { List metrics1 = Arrays.asList( new CountAggregatorFactory("count"), @@ -511,7 +511,7 @@ public void test_generateCompactionStateFingerprint_metricsListOrderDifferenceRe } @Test - public void test_generateCompactionStateFingerprint_dimensionsListOrderDifferenceResultsInNewFingerprint() + public void test_generateIndexingStateFingerprint_dimensionsListOrderDifferenceResultsInNewFingerprint() { DimensionsSpec dimensions1 = new DimensionsSpec( DimensionsSpec.getDefaultSchemas(ImmutableList.of("dim1", "dim2", "dim3")) @@ -552,7 +552,7 @@ public void test_generateCompactionStateFingerprint_dimensionsListOrderDifferenc } @Test - public void testGenerateCompactionStateFingerprint_differentPartitionsSpec() + public void testGenerateIndexingStateFingerprint_differentPartitionsSpec() { CompactionState state1 = new CompactionState( new DynamicPartitionsSpec(5000000, null), 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 2dd9a30bb488..fa4156ae1f19 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 @@ -581,11 +581,11 @@ public void testStatusWhenAutoSchemaMismatch() } @Test - public void test_evaluate_needsCompactionWhenAllSegmentsHaveUnexpectedCompactionStateFingerprint() + public void test_evaluate_needsCompactionWhenAllSegmentsHaveUnexpectedIndexingStateFingerprint() { List segments = List.of( - DataSegment.builder(WIKI_SEGMENT).compactionStateFingerprint("wrongFingerprint").build(), - DataSegment.builder(WIKI_SEGMENT_2).compactionStateFingerprint("wrongFingerprint").build() + DataSegment.builder(WIKI_SEGMENT).indexingStateFingerprint("wrongFingerprint").build(), + DataSegment.builder(WIKI_SEGMENT_2).indexingStateFingerprint("wrongFingerprint").build() ); final DataSourceCompactionConfig oldCompactionConfig = InlineSchemaDataSourceCompactionConfig @@ -614,7 +614,7 @@ public void test_evaluate_needsCompactionWhenAllSegmentsHaveUnexpectedCompaction } @Test - public void test_evaluate_needsCompactionWhenSomeSegmentsHaveUnexpectedCompactionStateFingerprint() + public void test_evaluate_needsCompactionWhenSomeSegmentsHaveUnexpectedIndexingStateFingerprint() { final DataSourceCompactionConfig oldCompactionConfig = InlineSchemaDataSourceCompactionConfig .builder() @@ -634,8 +634,8 @@ public void test_evaluate_needsCompactionWhenSomeSegmentsHaveUnexpectedCompactio String expectedFingerprint = fingerprintMapper.generateFingerprint(TestDataSource.WIKI, expectedState); List segments = List.of( - DataSegment.builder(WIKI_SEGMENT).compactionStateFingerprint(expectedFingerprint).build(), - DataSegment.builder(WIKI_SEGMENT_2).compactionStateFingerprint("wrongFingerprint").build() + DataSegment.builder(WIKI_SEGMENT).indexingStateFingerprint(expectedFingerprint).build(), + DataSegment.builder(WIKI_SEGMENT_2).indexingStateFingerprint("wrongFingerprint").build() ); compactionStateStorage.upsertIndexingState(TestDataSource.WIKI, expectedFingerprint, expectedState, DateTimes.nowUtc()); @@ -653,7 +653,7 @@ public void test_evaluate_needsCompactionWhenSomeSegmentsHaveUnexpectedCompactio public void test_evaluate_noCompacationIfUnexpectedFingerprintHasExpectedCompactionState() { List segments = List.of( - DataSegment.builder(WIKI_SEGMENT).compactionStateFingerprint("wrongFingerprint").build() + DataSegment.builder(WIKI_SEGMENT).indexingStateFingerprint("wrongFingerprint").build() ); final DataSourceCompactionConfig compactionConfig = InlineSchemaDataSourceCompactionConfig .builder() @@ -677,7 +677,7 @@ public void test_evaluate_noCompacationIfUnexpectedFingerprintHasExpectedCompact public void test_evaluate_needsCompactionWhenUnexpectedFingerprintAndNoFingerprintInMetadataStore() { List segments = List.of( - DataSegment.builder(WIKI_SEGMENT).compactionStateFingerprint("wrongFingerprint").build() + DataSegment.builder(WIKI_SEGMENT).indexingStateFingerprint("wrongFingerprint").build() ); final DataSourceCompactionConfig compactionConfig = InlineSchemaDataSourceCompactionConfig .builder() @@ -693,7 +693,7 @@ public void test_evaluate_needsCompactionWhenUnexpectedFingerprintAndNoFingerpri } @Test - public void test_evaluate_noCompactionWhenAllSegmentsHaveExpectedCompactionStateFingerprint() + public void test_evaluate_noCompactionWhenAllSegmentsHaveExpectedIndexingStateFingerprint() { final DataSourceCompactionConfig compactionConfig = InlineSchemaDataSourceCompactionConfig .builder() @@ -706,8 +706,8 @@ public void test_evaluate_noCompactionWhenAllSegmentsHaveExpectedCompactionState String expectedFingerprint = fingerprintMapper.generateFingerprint(TestDataSource.WIKI, expectedState); List segments = List.of( - DataSegment.builder(WIKI_SEGMENT).compactionStateFingerprint(expectedFingerprint).build(), - DataSegment.builder(WIKI_SEGMENT_2).compactionStateFingerprint(expectedFingerprint).build() + DataSegment.builder(WIKI_SEGMENT).indexingStateFingerprint(expectedFingerprint).build(), + DataSegment.builder(WIKI_SEGMENT_2).indexingStateFingerprint(expectedFingerprint).build() ); final CompactionStatus status = CompactionStatus.compute( @@ -734,8 +734,8 @@ public void test_evaluate_needsCompactionWhenNonFingerprintedSegmentsFailChecksO syncCacheFromManager(); List segments = List.of( - DataSegment.builder(WIKI_SEGMENT).compactionStateFingerprint(expectedFingerprint).build(), - DataSegment.builder(WIKI_SEGMENT_2).compactionStateFingerprint(null).lastCompactionState(createCompactionStateWithGranularity(Granularities.HOUR)).build() + DataSegment.builder(WIKI_SEGMENT).indexingStateFingerprint(expectedFingerprint).build(), + DataSegment.builder(WIKI_SEGMENT_2).indexingStateFingerprint(null).lastCompactionState(createCompactionStateWithGranularity(Granularities.HOUR)).build() ); @@ -760,8 +760,8 @@ public void test_evaluate_noCompactionWhenNonFingerprintedSegmentsPassChecksOnLa String expectedFingerprint = fingerprintMapper.generateFingerprint(TestDataSource.WIKI, expectedState); List segments = List.of( - DataSegment.builder(WIKI_SEGMENT).compactionStateFingerprint(expectedFingerprint).build(), - DataSegment.builder(WIKI_SEGMENT_2).compactionStateFingerprint(null).lastCompactionState(createCompactionStateWithGranularity(Granularities.DAY)).build() + DataSegment.builder(WIKI_SEGMENT).indexingStateFingerprint(expectedFingerprint).build(), + DataSegment.builder(WIKI_SEGMENT_2).indexingStateFingerprint(null).lastCompactionState(createCompactionStateWithGranularity(Granularities.DAY)).build() ); final CompactionStatus status = CompactionStatus.compute( 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 c14bd4540823..881e5b52f6ad 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,7 +63,7 @@ public class CreateDataSegments private String upgradedFromSegmentId; private String schemaFingerprint; private Long numRows; - private String compactionStateFingerprint; + private String indexingStateFingerprint; public static CreateDataSegments ofDatasource(String datasource) { @@ -128,9 +128,9 @@ public CreateDataSegments withSchemaFingerprint(String schemaFingerprint) return this; } - public CreateDataSegments withCompactionStateFingerprint(String compactionStateFingerprint) + public CreateDataSegments withIndexingStateFingerprint(String indexingStateFingerprint) { - this.compactionStateFingerprint = compactionStateFingerprint; + this.indexingStateFingerprint = indexingStateFingerprint; return this; } @@ -195,7 +195,7 @@ public List eachOfSize(long sizeInBytes) ++uniqueIdInInterval, compactionState, sizeInBytes, - compactionStateFingerprint + indexingStateFingerprint ) ); } @@ -215,7 +215,7 @@ private DataSegmentPlus plus(DataSegment segment) schemaFingerprint, numRows, upgradedFromSegmentId, - compactionStateFingerprint + indexingStateFingerprint ); } @@ -235,7 +235,7 @@ private NumberedDataSegment( int uniqueId, CompactionState compactionState, long size, - String compactionStateFingerprint + String indexingStateFingerprint ) { super( @@ -250,7 +250,7 @@ private NumberedDataSegment( compactionState, IndexIO.CURRENT_VERSION_ID, size, - compactionStateFingerprint, + indexingStateFingerprint, PruneSpecsHolder.DEFAULT ); this.uniqueId = uniqueId; 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 8e3ce33b71a0..901697c4a6b1 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,7 +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 indexingStateFingerprint = "abc123"; String createdDateStr = "2024-01-20T00:00:00.701Z"; String usedStatusLastUpdatedDateStr = "2024-01-20T01:00:00.701Z"; DateTime createdDate = DateTimes.of(createdDateStr); @@ -103,7 +103,7 @@ public void testSerde() throws JsonProcessingException ), TEST_VERSION, 1, - compactionStateFingerprint, + indexingStateFingerprint, DataSegment.PruneSpecsHolder.DEFAULT ), createdDate, @@ -112,7 +112,7 @@ public void testSerde() throws JsonProcessingException null, null, null, - compactionStateFingerprint + indexingStateFingerprint ); final Map objectMap = MAPPER.readValue( @@ -138,7 +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")); + Assert.assertEquals("abc123", segmentObjectMap.get("indexingStateFingerprint")); // verify extra metadata Assert.assertEquals(createdDateStr, objectMap.get("createdDate")); @@ -157,7 +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()); + Assert.assertEquals(segmentPlus.getDataSegment().getIndexingStateFingerprint(), deserializedSegmentPlus.getDataSegment().getIndexingStateFingerprint()); // verify extra metadata Assert.assertEquals(segmentPlus.getCreatedDate(), deserializedSegmentPlus.getCreatedDate()); 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 94c7d7e9ddaa..46fc9c835ac6 100644 --- a/services/src/main/java/org/apache/druid/guice/MetadataManagerModule.java +++ b/services/src/main/java/org/apache/druid/guice/MetadataManagerModule.java @@ -41,8 +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.IndexingStateStorage; import org.apache.druid.segment.metadata.IndexingStateCache; +import org.apache.druid.segment.metadata.IndexingStateStorage; import org.apache.druid.segment.metadata.NoopSegmentSchemaCache; import org.apache.druid.segment.metadata.SegmentSchemaCache; import org.apache.druid.segment.metadata.SqlIndexingStateStorage; From 5c2050cabaebcbfb43b993e231270c9cd6a964c9 Mon Sep 17 00:00:00 2001 From: capistrant Date: Wed, 14 Jan 2026 13:24:53 -0600 Subject: [PATCH 66/72] Refactor FingerprintMapper name to remove compaction for indexing state --- .../coordinator/NewestSegmentFirstPolicyBenchmark.java | 4 ++-- .../embedded/compact/CompactionSupervisorTest.java | 6 +++--- .../druid/indexing/compact/CompactionJobParams.java | 10 +++++----- .../druid/indexing/compact/CompactionJobQueue.java | 4 ++-- ...java => DefaultIndexingStateFingerprintMapper.java} | 6 +++--- ...Mapper.java => IndexingStateFingerprintMapper.java} | 4 ++-- .../druid/server/compaction/CompactionStatus.java | 8 ++++---- .../DataSourceCompactibleSegmentIterator.java | 8 ++++---- .../PriorityBasedCompactionSegmentIterator.java | 6 +++--- .../segment/metadata/SqlIndexingStateStorageTest.java | 4 ++-- .../druid/server/compaction/CompactionStatusTest.java | 8 ++++---- .../compaction/NewestSegmentFirstPolicyTest.java | 6 +++--- 12 files changed, 37 insertions(+), 37 deletions(-) rename server/src/main/java/org/apache/druid/segment/metadata/{DefaultCompactionFingerprintMapper.java => DefaultIndexingStateFingerprintMapper.java} (92%) rename server/src/main/java/org/apache/druid/segment/metadata/{CompactionFingerprintMapper.java => IndexingStateFingerprintMapper.java} (92%) 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 485d0f2f9979..58f5c6f2eb3b 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 @@ -23,7 +23,7 @@ import org.apache.druid.client.DataSourcesSnapshot; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.segment.metadata.CompactionTestUtils; -import org.apache.druid.segment.metadata.DefaultCompactionFingerprintMapper; +import org.apache.druid.segment.metadata.DefaultIndexingStateFingerprintMapper; import org.apache.druid.segment.metadata.NoopIndexingStateCache; import org.apache.druid.server.compaction.CompactionCandidateSearchPolicy; import org.apache.druid.server.compaction.CompactionSegmentIterator; @@ -139,7 +139,7 @@ public void measureNewestSegmentFirstPolicy(Blackhole blackhole) compactionConfigs, dataSources, Collections.emptyMap(), - new DefaultCompactionFingerprintMapper(new NoopIndexingStateCache(), CompactionTestUtils.createDeterministicMapper()) + new DefaultIndexingStateFingerprintMapper(new NoopIndexingStateCache(), CompactionTestUtils.createDeterministicMapper()) ); for (int i = 0; i < numCompactionTaskSlots && iterator.hasNext(); i++) { blackhole.consume(iterator.next()); diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionSupervisorTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionSupervisorTest.java index 8669440ec12f..7f29f97804dc 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 @@ -31,10 +31,10 @@ import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.query.DruidMetrics; import org.apache.druid.rpc.UpdateResponse; -import org.apache.druid.segment.metadata.CompactionFingerprintMapper; import org.apache.druid.segment.metadata.CompactionTestUtils; -import org.apache.druid.segment.metadata.DefaultCompactionFingerprintMapper; +import org.apache.druid.segment.metadata.DefaultIndexingStateFingerprintMapper; import org.apache.druid.segment.metadata.IndexingStateCache; +import org.apache.druid.segment.metadata.IndexingStateFingerprintMapper; import org.apache.druid.server.coordinator.ClusterCompactionConfig; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; import org.apache.druid.server.coordinator.InlineSchemaDataSourceCompactionConfig; @@ -289,7 +289,7 @@ private void verifySegmentsHaveNullLastCompactionStateAndNonNullFingerprint() private void verifyCompactedSegmentsHaveFingerprints(DataSourceCompactionConfig compactionConfig) { IndexingStateCache cache = overlord.bindings().getInstance(IndexingStateCache.class); - CompactionFingerprintMapper fingerprintMapper = new DefaultCompactionFingerprintMapper( + IndexingStateFingerprintMapper fingerprintMapper = new DefaultIndexingStateFingerprintMapper( cache, CompactionTestUtils.createDeterministicMapper() ); 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 d0e8219e742c..36cd075922d5 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionJobParams.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionJobParams.java @@ -19,7 +19,7 @@ package org.apache.druid.indexing.compact; -import org.apache.druid.segment.metadata.CompactionFingerprintMapper; +import org.apache.druid.segment.metadata.IndexingStateFingerprintMapper; import org.apache.druid.server.compaction.CompactionSnapshotBuilder; import org.apache.druid.server.coordinator.ClusterCompactionConfig; import org.apache.druid.timeline.SegmentTimeline; @@ -34,21 +34,21 @@ public class CompactionJobParams private final TimelineProvider timelineProvider; private final ClusterCompactionConfig clusterCompactionConfig; private final CompactionSnapshotBuilder snapshotBuilder; - private final CompactionFingerprintMapper fingerprintMapper; + private final IndexingStateFingerprintMapper fingerprintMapper; public CompactionJobParams( DateTime scheduleStartTime, ClusterCompactionConfig clusterCompactionConfig, TimelineProvider timelineProvider, CompactionSnapshotBuilder snapshotBuilder, - CompactionFingerprintMapper compactionFingerprintMapper + IndexingStateFingerprintMapper indexingStateFingerprintMapper ) { this.scheduleStartTime = scheduleStartTime; this.clusterCompactionConfig = clusterCompactionConfig; this.timelineProvider = timelineProvider; this.snapshotBuilder = snapshotBuilder; - this.fingerprintMapper = compactionFingerprintMapper; + this.fingerprintMapper = indexingStateFingerprintMapper; } /** @@ -92,7 +92,7 @@ public CompactionSnapshotBuilder getSnapshotBuilder() return snapshotBuilder; } - public CompactionFingerprintMapper getFingerprintMapper() + public IndexingStateFingerprintMapper getFingerprintMapper() { return fingerprintMapper; } 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 461eb077ee15..bdce53c31fcd 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,7 +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.DefaultCompactionFingerprintMapper; +import org.apache.druid.segment.metadata.DefaultIndexingStateFingerprintMapper; import org.apache.druid.segment.metadata.IndexingStateCache; import org.apache.druid.segment.metadata.IndexingStateStorage; import org.apache.druid.server.compaction.CompactionCandidate; @@ -130,7 +130,7 @@ public CompactionJobQueue( clusterCompactionConfig, dataSourcesSnapshot.getUsedSegmentsTimelinesPerDataSource()::get, snapshotBuilder, - new DefaultCompactionFingerprintMapper(indexingStateCache, deterministicCompactionStateMapper) + new DefaultIndexingStateFingerprintMapper(indexingStateCache, deterministicCompactionStateMapper) ); this.indexingStateStorage = indexingStateStorage; diff --git a/server/src/main/java/org/apache/druid/segment/metadata/DefaultCompactionFingerprintMapper.java b/server/src/main/java/org/apache/druid/segment/metadata/DefaultIndexingStateFingerprintMapper.java similarity index 92% rename from server/src/main/java/org/apache/druid/segment/metadata/DefaultCompactionFingerprintMapper.java rename to server/src/main/java/org/apache/druid/segment/metadata/DefaultIndexingStateFingerprintMapper.java index b6e79d1d98de..c9b56966b961 100644 --- a/server/src/main/java/org/apache/druid/segment/metadata/DefaultCompactionFingerprintMapper.java +++ b/server/src/main/java/org/apache/druid/segment/metadata/DefaultIndexingStateFingerprintMapper.java @@ -31,16 +31,16 @@ import java.util.Optional; /** - * Default implementation of {@link CompactionFingerprintMapper} that delegates to + * Default implementation of {@link IndexingStateFingerprintMapper} that delegates to * {@link IndexingStateStorage} for fingerprint generation and {@link IndexingStateCache} * for state lookups. */ -public class DefaultCompactionFingerprintMapper implements CompactionFingerprintMapper +public class DefaultIndexingStateFingerprintMapper implements IndexingStateFingerprintMapper { private final IndexingStateCache indexingStateCache; private final ObjectMapper deterministicMapper; - public DefaultCompactionFingerprintMapper( + public DefaultIndexingStateFingerprintMapper( IndexingStateCache indexingStateCache, @Deterministic ObjectMapper deterministicMapper ) diff --git a/server/src/main/java/org/apache/druid/segment/metadata/CompactionFingerprintMapper.java b/server/src/main/java/org/apache/druid/segment/metadata/IndexingStateFingerprintMapper.java similarity index 92% rename from server/src/main/java/org/apache/druid/segment/metadata/CompactionFingerprintMapper.java rename to server/src/main/java/org/apache/druid/segment/metadata/IndexingStateFingerprintMapper.java index 17f36d930998..a39442fc6949 100644 --- a/server/src/main/java/org/apache/druid/segment/metadata/CompactionFingerprintMapper.java +++ b/server/src/main/java/org/apache/druid/segment/metadata/IndexingStateFingerprintMapper.java @@ -24,12 +24,12 @@ import java.util.Optional; /** - * Provides operations for mapping between compaction state fingerprints and their corresponding states. + * Provides operations for mapping between indexing state fingerprints and their corresponding states. *

    * This interface abstracts the fingerprint generation and lookup operations, simplifying * dependencies and improving testability for classes that need both operations. */ -public interface CompactionFingerprintMapper +public interface IndexingStateFingerprintMapper { /** * Generates a deterministic fingerprint for the given compaction state and datasource. diff --git a/server/src/main/java/org/apache/druid/server/compaction/CompactionStatus.java b/server/src/main/java/org/apache/druid/server/compaction/CompactionStatus.java index e9caf8b48564..562a1cc99612 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 @@ -33,7 +33,7 @@ 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.CompactionFingerprintMapper; +import org.apache.druid.segment.metadata.IndexingStateFingerprintMapper; import org.apache.druid.segment.transform.CompactionTransformSpec; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; import org.apache.druid.server.coordinator.UserCompactionTaskGranularityConfig; @@ -260,7 +260,7 @@ public static CompactionStatus running(String message) static CompactionStatus compute( CompactionCandidate candidateSegments, DataSourceCompactionConfig config, - @Nullable CompactionFingerprintMapper fingerprintMapper + @Nullable IndexingStateFingerprintMapper fingerprintMapper ) { final CompactionState expectedState = config.toCompactionState(); @@ -358,13 +358,13 @@ private static class Evaluator @Nullable private final String targetFingerprint; - private final CompactionFingerprintMapper fingerprintMapper; + private final IndexingStateFingerprintMapper fingerprintMapper; private Evaluator( CompactionCandidate candidateSegments, DataSourceCompactionConfig compactionConfig, @Nullable String targetFingerprint, - @Nullable CompactionFingerprintMapper fingerprintMapper + @Nullable IndexingStateFingerprintMapper fingerprintMapper ) { this.candidateSegments = candidateSegments; diff --git a/server/src/main/java/org/apache/druid/server/compaction/DataSourceCompactibleSegmentIterator.java b/server/src/main/java/org/apache/druid/server/compaction/DataSourceCompactibleSegmentIterator.java index 1453463d0206..1994e87a6388 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,7 +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.CompactionFingerprintMapper; +import org.apache.druid.segment.metadata.IndexingStateFingerprintMapper; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.Partitions; @@ -69,7 +69,7 @@ public class DataSourceCompactibleSegmentIterator implements CompactionSegmentIt private final String dataSource; private final DataSourceCompactionConfig config; - private final CompactionFingerprintMapper fingerprintMapper; + private final IndexingStateFingerprintMapper fingerprintMapper; private final List compactedSegments = new ArrayList<>(); private final List skippedSegments = new ArrayList<>(); @@ -87,13 +87,13 @@ public DataSourceCompactibleSegmentIterator( SegmentTimeline timeline, List skipIntervals, CompactionCandidateSearchPolicy searchPolicy, - CompactionFingerprintMapper compactionFingerprintMapper + IndexingStateFingerprintMapper indexingStateFingerprintMapper ) { this.config = config; this.dataSource = config.getDataSource(); this.queue = new PriorityQueue<>(searchPolicy::compareCandidates); - this.fingerprintMapper = compactionFingerprintMapper; + this.fingerprintMapper = indexingStateFingerprintMapper; populateQueue(timeline, skipIntervals); } 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 4b0410975095..43028fa37ff4 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,7 +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.CompactionFingerprintMapper; +import org.apache.druid.segment.metadata.IndexingStateFingerprintMapper; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; import org.apache.druid.timeline.SegmentTimeline; import org.joda.time.Interval; @@ -50,7 +50,7 @@ public PriorityBasedCompactionSegmentIterator( Map compactionConfigs, Map datasourceToTimeline, Map> skipIntervals, - CompactionFingerprintMapper compactionFingerprintMapper + IndexingStateFingerprintMapper indexingStateFingerprintMapper ) { this.queue = new PriorityQueue<>(searchPolicy::compareCandidates); @@ -72,7 +72,7 @@ public PriorityBasedCompactionSegmentIterator( timeline, skipIntervals.getOrDefault(datasource, Collections.emptyList()), searchPolicy, - compactionFingerprintMapper + indexingStateFingerprintMapper ) ); addNextItemForDatasourceToQueue(datasource); diff --git a/server/src/test/java/org/apache/druid/segment/metadata/SqlIndexingStateStorageTest.java b/server/src/test/java/org/apache/druid/segment/metadata/SqlIndexingStateStorageTest.java index c5237add8d54..f5db846a1297 100644 --- a/server/src/test/java/org/apache/druid/segment/metadata/SqlIndexingStateStorageTest.java +++ b/server/src/test/java/org/apache/druid/segment/metadata/SqlIndexingStateStorageTest.java @@ -61,7 +61,7 @@ public class SqlIndexingStateStorageTest private static MetadataStorageTablesConfig tablesConfig; private SqlIndexingStateStorage manager; - private static DefaultCompactionFingerprintMapper fingerprintMapper; + private static DefaultIndexingStateFingerprintMapper fingerprintMapper; @BeforeAll public static void setUpClass() @@ -70,7 +70,7 @@ public static void setUpClass() tablesConfig = DERBY_CONNECTOR_RULE.metadataTablesConfigSupplier().get(); derbyConnector.createIndexingStatesTable(); derbyConnector.createSegmentTable(); - fingerprintMapper = new DefaultCompactionFingerprintMapper( + fingerprintMapper = new DefaultIndexingStateFingerprintMapper( new NoopIndexingStateCache(), DETERMINISTIC_MAPPER ); 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 fa4156ae1f19..b43507abd06e 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 @@ -39,11 +39,11 @@ 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.CompactionFingerprintMapper; import org.apache.druid.segment.metadata.CompactionTestUtils; -import org.apache.druid.segment.metadata.DefaultCompactionFingerprintMapper; +import org.apache.druid.segment.metadata.DefaultIndexingStateFingerprintMapper; import org.apache.druid.segment.metadata.HeapMemoryIndexingStateStorage; import org.apache.druid.segment.metadata.IndexingStateCache; +import org.apache.druid.segment.metadata.IndexingStateFingerprintMapper; import org.apache.druid.segment.nested.NestedCommonFormatColumnFormatSpec; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; import org.apache.druid.server.coordinator.InlineSchemaDataSourceCompactionConfig; @@ -73,14 +73,14 @@ public class CompactionStatusTest private HeapMemoryIndexingStateStorage compactionStateStorage; private IndexingStateCache indexingStateCache; - private CompactionFingerprintMapper fingerprintMapper; + private IndexingStateFingerprintMapper fingerprintMapper; @Before public void setUp() { compactionStateStorage = new HeapMemoryIndexingStateStorage(); indexingStateCache = new IndexingStateCache(); - fingerprintMapper = new DefaultCompactionFingerprintMapper( + fingerprintMapper = new DefaultIndexingStateFingerprintMapper( indexingStateCache, CompactionTestUtils.createDeterministicMapper() ); 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 45da1b14a3ae..966e33bfe84e 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 @@ -46,9 +46,9 @@ 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.CompactionFingerprintMapper; import org.apache.druid.segment.metadata.CompactionTestUtils; -import org.apache.druid.segment.metadata.DefaultCompactionFingerprintMapper; +import org.apache.druid.segment.metadata.DefaultIndexingStateFingerprintMapper; +import org.apache.druid.segment.metadata.IndexingStateFingerprintMapper; import org.apache.druid.segment.metadata.NoopIndexingStateCache; import org.apache.druid.segment.transform.CompactionTransformSpec; import org.apache.druid.server.coordinator.CreateDataSegments; @@ -84,7 +84,7 @@ public class NewestSegmentFirstPolicyTest private static final int DEFAULT_NUM_SEGMENTS_PER_SHARD = 4; private final ObjectMapper mapper = CompactionTestUtils.createDeterministicMapper(); private final NewestSegmentFirstPolicy policy = new NewestSegmentFirstPolicy(null); - private final CompactionFingerprintMapper fingerprintMapper = new DefaultCompactionFingerprintMapper(new NoopIndexingStateCache(), mapper); + private final IndexingStateFingerprintMapper fingerprintMapper = new DefaultIndexingStateFingerprintMapper(new NoopIndexingStateCache(), mapper); @Test public void testLargeOffsetAndSmallSegmentInterval() From ff2949f18406d2305639990a4d919c5f451c6618 Mon Sep 17 00:00:00 2001 From: capistrant Date: Wed, 14 Jan 2026 16:18:36 -0600 Subject: [PATCH 67/72] refactorings after self review --- docs/data-management/automatic-compaction.md | 4 +- .../druid/indexing/compact/CompactionJob.java | 16 +- .../indexing/compact/CompactionJobQueue.java | 6 +- .../duty/KillUnreferencedIndexingState.java | 11 ++ .../OverlordCompactionSchedulerTest.java | 3 +- .../apache/druid/timeline/DataSegment.java | 2 +- .../IndexerSQLMetadataStorageCoordinator.java | 25 ++- .../cache/HeapMemorySegmentMetadataCache.java | 54 +++---- .../druid/metadata/segment/cache/Metric.java | 12 +- ...DefaultIndexingStateFingerprintMapper.java | 16 +- .../segment/metadata/IndexingStateCache.java | 2 +- .../IndexingStateFingerprintMapper.java | 11 +- .../metadata/IndexingStateStorage.java | 10 +- .../metadata/SqlIndexingStateStorage.java | 10 +- .../server/compaction/CompactionStatus.java | 11 +- .../druid/server/coordinator/stats/Stats.java | 2 - ...exerSQLMetadataStorageCoordinatorTest.java | 38 ++--- .../SqlSegmentsMetadataQueryTest.java | 147 +++++++++--------- .../HeapMemoryIndexingStateStorage.java | 4 +- .../metadata/SqlIndexingStateStorageTest.java | 48 +++--- .../compaction/CompactionStatusTest.java | 20 +-- 21 files changed, 223 insertions(+), 229 deletions(-) diff --git a/docs/data-management/automatic-compaction.md b/docs/data-management/automatic-compaction.md index af0e5beb8117..f82c66d70fea 100644 --- a/docs/data-management/automatic-compaction.md +++ b/docs/data-management/automatic-compaction.md @@ -246,8 +246,8 @@ You can run automatic compaction using compaction supervisors on the Overlord ra To use compaction supervisors, the following configuration requirements must be met: -* You mut be using incremental segment metadata caching: - * `druid.manager.segments.useIncrementalCache` set to `always` or `ifSynced` in your Overlord runtime properties. +* You must be using incremental segment metadata caching: + * `druid.manager.segments.useIncrementalCache` set to `always` or `ifSynced` in your Overlord and Coordinator runtime properties. * See [Segment metadata caching](../configuration/index.md#metadata-retrieval) for full configuration documentation. * update the [compaction dynamic config](../api-reference/automatic-compaction-api.md#update-cluster-level-compaction-config) and set: diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionJob.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionJob.java index 51a5823357a1..7611485bd6d8 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionJob.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionJob.java @@ -33,21 +33,21 @@ public class CompactionJob extends BatchIndexingJob private final CompactionCandidate candidate; private final int maxRequiredTaskSlots; private final String targetIndexingStateFingerprint; - private final CompactionState targetCompactionState; + private final CompactionState targetIndexingState; public CompactionJob( ClientCompactionTaskQuery task, CompactionCandidate candidate, int maxRequiredTaskSlots, String targetIndexingStateFingerprint, - CompactionState targetCompactionState + CompactionState targetIndexingState ) { super(task, null); this.candidate = candidate; this.maxRequiredTaskSlots = maxRequiredTaskSlots; this.targetIndexingStateFingerprint = targetIndexingStateFingerprint; - this.targetCompactionState = targetCompactionState; + this.targetIndexingState = targetIndexingState; } public CompactionJob( @@ -55,14 +55,14 @@ public CompactionJob( CompactionCandidate candidate, int maxRequiredTaskSlots, String targetIndexingStateFingerprint, - CompactionState targetCompactionState + CompactionState targetIndexingState ) { super(null, msqQuery); this.candidate = candidate; this.maxRequiredTaskSlots = maxRequiredTaskSlots; this.targetIndexingStateFingerprint = targetIndexingStateFingerprint; - this.targetCompactionState = targetCompactionState; + this.targetIndexingState = targetIndexingState; } public String getDataSource() @@ -85,9 +85,9 @@ public String getTargetIndexingStateFingerprint() return targetIndexingStateFingerprint; } - public CompactionState getTargetCompactionState() + public CompactionState getTargetIndexingState() { - return targetCompactionState; + return targetIndexingState; } @Override @@ -98,7 +98,7 @@ public String toString() ", candidate=" + candidate + ", maxRequiredTaskSlots=" + maxRequiredTaskSlots + ", targetIndexingStateFingerprint='" + targetIndexingStateFingerprint + '\'' + - ", targetCompactionState=" + targetCompactionState + + ", targetIndexingState=" + targetIndexingState + '}'; } } 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 bdce53c31fcd..72ba2d6bf845 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 @@ -368,14 +368,14 @@ private String startTaskIfReady(CompactionJob job) */ private void persistPendingIndexingState(CompactionJob job) { - if (job.getTargetCompactionState() != null && job.getTargetIndexingStateFingerprint() != null) { + if (job.getTargetIndexingState() != null && job.getTargetIndexingStateFingerprint() != null) { indexingStateStorage.upsertIndexingState( job.getDataSource(), job.getTargetIndexingStateFingerprint(), - job.getTargetCompactionState(), + job.getTargetIndexingState(), DateTimes.nowUtc() ); - indexingStateCache.addIndexingState(job.getTargetIndexingStateFingerprint(), job.getTargetCompactionState()); + indexingStateCache.addIndexingState(job.getTargetIndexingStateFingerprint(), job.getTargetIndexingState()); } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/duty/KillUnreferencedIndexingState.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/duty/KillUnreferencedIndexingState.java index b1afb6df2911..13708d5565f0 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/duty/KillUnreferencedIndexingState.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/duty/KillUnreferencedIndexingState.java @@ -27,6 +27,17 @@ import javax.inject.Inject; import java.util.List; +/** + * Duty that cleans up unreferenced indexing states from the indexing state storage. + *

    + * The cleanup process involves: + *

      + *
    1. Marking unreferenced indexing states as unused.
    2. + *
    3. Repairing any unused states that are still referenced by segments.
    4. + *
    5. Deleting unused indexing states older than the configured retention duration.
    6. + *
    7. Deleting any pending indexing states that are older than the configured retention duration.
    8. + *
    + */ public class KillUnreferencedIndexingState extends OverlordMetadataCleanupDuty { private static final Logger log = new Logger(KillUnreferencedIndexingState.class); 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 4bb0a2df2a4f..3c85beb5bcd4 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.CompactionTestUtils; import org.apache.druid.segment.metadata.HeapMemoryIndexingStateStorage; import org.apache.druid.segment.metadata.IndexingStateCache; import org.apache.druid.server.compaction.CompactionSimulateResult; @@ -236,7 +237,7 @@ private void initScheduler() OBJECT_MAPPER, new HeapMemoryIndexingStateStorage(), new IndexingStateCache(), - OBJECT_MAPPER // TODO fix + CompactionTestUtils.createDeterministicMapper() ); } 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 cfd34b1c2aae..a392fc19f277 100644 --- a/processing/src/main/java/org/apache/druid/timeline/DataSegment.java +++ b/processing/src/main/java/org/apache/druid/timeline/DataSegment.java @@ -358,7 +358,7 @@ public boolean isTombstone() } /** - * Get the compaction state fingerprint associated with this segment. + * Get the inexing state fingerprint associated with this segment. *

    * A null fingerprint indicates that this segment either has not been compacted, or was compacted before compaction * fingerprinting existed. In the latter case, the segment would have a non-null {@link #lastCompactionState}. 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 ef06a3d0a8ce..973b921c2ff4 100644 --- a/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java +++ b/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java @@ -469,7 +469,7 @@ public SegmentPublishResult commitSegmentsAndMetadata( // Mark compaction state fingerprints as active after successful publish if (result.isSuccess()) { - markCompactionFingerprintsAsActive(result.getSegments()); + markIndexingStateFingerprintsAsActive(result.getSegments()); } return result; @@ -534,7 +534,7 @@ public SegmentPublishResult commitReplaceSegments( // Mark compaction state fingerprints as active after successful publish if (result.isSuccess()) { - markCompactionFingerprintsAsActive(result.getSegments()); + markIndexingStateFingerprintsAsActive(result.getSegments()); } return result; @@ -1275,7 +1275,7 @@ private SegmentPublishResult commitAppendSegmentsAndMetadataInTransaction( // Mark compaction state fingerprints as active after successful publish if (result.isSuccess()) { - markCompactionFingerprintsAsActive(result.getSegments()); + markIndexingStateFingerprintsAsActive(result.getSegments()); } return result; @@ -2714,24 +2714,21 @@ public Map> retrieveUpgradedToSegmentIds( } /** - * Marks compaction state fingerprints as active (non-pending) for successfully published segments. + * Marks indexing state fingerprints as active (non-pending) for successfully published segments. *

    - * Extracts unique compaction state fingerprints from the given segments and marks them as active - * in the compaction state storage. This is called after successful segment publishing to indicate - * that the compaction state is no longer pending and can be retained with the regular grace period. - *

    - * Most calls result in 0-row updates since fingerprints are stable and the first task already - * marked them as active. + * Extracts unique indexing state fingerprints from the given segments and marks them as active + * in the inexing state storage. This is called after successful segment publishing to indicate + * that the indexing state is no longer pending and can be retained with the regular grace period. * * @param segments The segments that were successfully published */ - private void markCompactionFingerprintsAsActive(Set segments) + private void markIndexingStateFingerprintsAsActive(Set segments) { if (segments == null || segments.isEmpty()) { return; } - // Collect unique non-null compaction state fingerprints + // Collect unique non-null indexing state fingerprints final Set fingerprints = segments.stream() .map(DataSegment::getIndexingStateFingerprint) .filter(fp -> fp != null && !fp.isEmpty()) @@ -2742,13 +2739,13 @@ private void markCompactionFingerprintsAsActive(Set segments) try { int rowsUpdated = indexingStateStorage.markIndexingStatesAsActive(fingerprint); if (rowsUpdated > 0) { - log.info("Marked compaction state fingerprint[%s] as active (non-pending).", fingerprint); + log.info("Marked indexing state fingerprint[%s] as active.", fingerprint); } } catch (Exception e) { // Log but don't fail the overall operation - the fingerprint will stay pending // and be cleaned up by the pending grace period - log.warn(e, "Failed to mark compaction state fingerprint[%s] as active. Will retry on next publish.", fingerprint); + log.warn(e, "Failed to mark indexing state fingerprint[%s] as active. Future segments publishes may remediate", fingerprint); } } } 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 653223a5e6de..0b85243c8f25 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 @@ -139,7 +139,7 @@ private enum CacheState private final boolean useSchemaCache; private final SegmentSchemaCache segmentSchemaCache; - private final boolean useCompactionStateCache; + private final boolean useIndexingStateCache; private final IndexingStateCache indexingStateCache; private final ListeningScheduledExecutorService pollExecutor; @@ -185,7 +185,7 @@ public HeapMemorySegmentMetadataCache( this.tablesConfig = tablesConfig.get(); this.useSchemaCache = segmentSchemaCache.isEnabled(); this.segmentSchemaCache = segmentSchemaCache; - this.useCompactionStateCache = indexingStateCache.isEnabled(); + this.useIndexingStateCache = indexingStateCache.isEnabled(); this.indexingStateCache = indexingStateCache; this.connector = connector; this.pollExecutor = isEnabled() @@ -240,7 +240,7 @@ public void stop() datasourceToSegmentCache.forEach((datasource, cache) -> cache.stop()); datasourceToSegmentCache.clear(); datasourcesSnapshot.set(null); - if (useCompactionStateCache) { + if (useIndexingStateCache) { indexingStateCache.clear(); } syncFinishTime.set(null); @@ -587,8 +587,8 @@ private long syncWithMetadataStore() retrieveAndResetUsedSegmentSchemas(datasourceToSummary); } - if (useCompactionStateCache) { - retrieveAndResetUsedCompactionStates(); + if (useIndexingStateCache) { + retrieveAndResetUsedIndexingStates(); } markCacheSynced(syncStartTime); @@ -1123,36 +1123,36 @@ private void emitMetric(String datasource, String metric, long value) } /** - * Retrieves required used compaction states from the metadata store and resets + * Retrieves required used indexing states from the metadata store and resets * them in the {@link IndexingStateCache}. If this is the first sync, all used - * compaction states are retrieved from the metadata store. If this is a delta sync, - * first only the fingerprints of all used compaction states are retrieved. Payloads are + * indexing states are retrieved from the metadata store. If this is a delta sync, + * first only the fingerprints of all used indexing states are retrieved. Payloads are * then fetched for only the fingerprints which are not present in the cache. */ - private void retrieveAndResetUsedCompactionStates() + private void retrieveAndResetUsedIndexingStates() { - final Stopwatch compactionStateSyncDuration = Stopwatch.createStarted(); + final Stopwatch indexingStateSyncDuration = Stopwatch.createStarted(); - // Reset the IndexingStateCache with latest compaction states + // Reset the IndexingStateCache with latest indexing states final Map fingerprintToStateMap; if (syncFinishTime.get() == null) { - fingerprintToStateMap = buildFingerprintToStateMapForFullSync(); + fingerprintToStateMap = buildIndexingStateFingerprintToStateMapForFullSync(); } else { - fingerprintToStateMap = buildFingerprintToStateMapForDeltaSync(); + fingerprintToStateMap = buildIndexingStateFingerprintToStateMapForDeltaSync(); } indexingStateCache.resetIndexingStatesForPublishedSegments(fingerprintToStateMap); // Emit metrics for the current contents of the cache indexingStateCache.getAndResetStats().forEach(this::emitMetric); - emitMetric(Metric.RETRIEVE_COMPACTION_STATES_DURATION_MILLIS, compactionStateSyncDuration.millisElapsed()); + emitMetric(Metric.RETRIEVE_INDEXING_STATES_DURATION_MILLIS, indexingStateSyncDuration.millisElapsed()); } /** - * Retrieves all used compaction states from the metadata store and builds a - * fresh map from compaction state fingerprint to state. + * Retrieves all used indexing states from the metadata store and builds a + * fresh map from indexing state fingerprint to state. */ - private Map buildFingerprintToStateMapForFullSync() + private Map buildIndexingStateFingerprintToStateMapForFullSync() { final List records = query( SqlSegmentsMetadataQuery::retrieveAllUsedIndexingStates @@ -1167,15 +1167,15 @@ private Map buildFingerprintToStateMapForFullSync() } /** - * Retrieves compaction states from the metadata store if they are not present + * Retrieves indexing states from the metadata store if they are not present * in the cache or have been recently updated in the metadata store. These - * compaction states along with those already present in the cache are used to - * build a complete updated map from compaction state fingerprint to state. + * indexing states along with those already present in the cache are used to + * build a complete updated map from indexing state fingerprint to state. * - * @return Complete updated map from compaction state fingerprint to state for all - * used compaction states currently persisted in the metadata store. + * @return Complete updated map from indexing state fingerprint to state for all + * used indexing states currently persisted in the metadata store. */ - private Map buildFingerprintToStateMapForDeltaSync() + private Map buildIndexingStateFingerprintToStateMapForDeltaSync() { // Identify fingerprints in the cache and in the metadata store final Map fingerprintToStateMap = new HashMap<>( @@ -1186,12 +1186,12 @@ private Map buildFingerprintToStateMapForDeltaSync() SqlSegmentsMetadataQuery::retrieveAllUsedIndexingStateFingerprints ); - // Remove entry for compaction states that have been deleted from the metadata store + // Remove entry for indexing states that have been deleted from the metadata store final Set deletedFingerprints = Sets.difference(cachedFingerprints, persistedFingerprints); deletedFingerprints.forEach(fingerprintToStateMap::remove); - emitMetric(Metric.DELETED_COMPACTION_STATES, deletedFingerprints.size()); + emitMetric(Metric.DELETED_INDEXING_STATES, deletedFingerprints.size()); - // Retrieve and add entry for compaction states that have been added to the metadata store + // Retrieve and add entry for indexing states that have been added to the metadata store final Set addedFingerprints = Sets.difference(persistedFingerprints, cachedFingerprints); final List addedIndexingStateRecords = query( sql -> sql.retrieveIndexingStatesForFingerprints(addedFingerprints) @@ -1200,7 +1200,7 @@ private Map buildFingerprintToStateMapForDeltaSync() addedIndexingStateRecords.forEach( record -> fingerprintToStateMap.put(record.getFingerprint(), record.getState()) ); - emitMetric(Metric.ADDED_COMPACTION_STATES, addedIndexingStateRecords.size()); + emitMetric(Metric.ADDED_INDEXING_STATES, addedIndexingStateRecords.size()); return fingerprintToStateMap; } diff --git a/server/src/main/java/org/apache/druid/metadata/segment/cache/Metric.java b/server/src/main/java/org/apache/druid/metadata/segment/cache/Metric.java index 625b2f43a242..7ea59b9bf695 100644 --- a/server/src/main/java/org/apache/druid/metadata/segment/cache/Metric.java +++ b/server/src/main/java/org/apache/druid/metadata/segment/cache/Metric.java @@ -111,9 +111,9 @@ private Metric() public static final String RETRIEVE_SEGMENT_SCHEMAS_DURATION_MILLIS = METRIC_NAME_PREFIX + "fetchSchemas/time"; /** - * Time taken in milliseconds to fetch all compaction states from the metadata store. + * Time taken in milliseconds to fetch all indexing states from the metadata store. */ - public static final String RETRIEVE_COMPACTION_STATES_DURATION_MILLIS = METRIC_NAME_PREFIX + "fetchCompactionStates/time"; + public static final String RETRIEVE_INDEXING_STATES_DURATION_MILLIS = METRIC_NAME_PREFIX + "fetchIndexingStates/time"; /** * Time taken to update the datasource snapshot in the cache. @@ -164,14 +164,14 @@ private Metric() public static final String SKIPPED_SEGMENT_SCHEMAS = METRIC_NAME_PREFIX + "schema/skipped"; /** - * Number of compaction states added to the cache in the latest sync. + * Number of indexing states added to the cache in the latest sync. */ - public static final String ADDED_COMPACTION_STATES = METRIC_NAME_PREFIX + "compactionState/added"; + public static final String ADDED_INDEXING_STATES = METRIC_NAME_PREFIX + "indexingState/added"; /** - * Number of compaction states deleted from the cache in the latest sync. + * Number of indexing states deleted from the cache in the latest sync. */ - public static final String DELETED_COMPACTION_STATES = METRIC_NAME_PREFIX + "compactionState/deleted"; + public static final String DELETED_INDEXING_STATES = METRIC_NAME_PREFIX + "indexingState/deleted"; /** * Number of unparseable pending segment records skipped while refreshing the cache. diff --git a/server/src/main/java/org/apache/druid/segment/metadata/DefaultIndexingStateFingerprintMapper.java b/server/src/main/java/org/apache/druid/segment/metadata/DefaultIndexingStateFingerprintMapper.java index c9b56966b961..e5825543f122 100644 --- a/server/src/main/java/org/apache/druid/segment/metadata/DefaultIndexingStateFingerprintMapper.java +++ b/server/src/main/java/org/apache/druid/segment/metadata/DefaultIndexingStateFingerprintMapper.java @@ -49,19 +49,9 @@ public DefaultIndexingStateFingerprintMapper( this.deterministicMapper = deterministicMapper; } - /** - * Generates a deterministic fingerprint for the given compaction state and datasource. - *

    - * The fingerprint is a SHA-256 hash of the datasource name and serialized compaction state that is globally unique in - * the segment space. - * - * @param compactionState The compaction configuration to fingerprint - * @param dataSource The datasource name - * @return A hex-encoded SHA-256 fingerprint string - */ @SuppressWarnings("UnstableApiUsage") @Override - public String generateFingerprint(String dataSource, CompactionState compactionState) + public String generateFingerprint(String dataSource, CompactionState indexingState) { final Hasher hasher = Hashing.sha256().newHasher(); @@ -69,10 +59,10 @@ public String generateFingerprint(String dataSource, CompactionState compactionS hasher.putByte((byte) 0xff); try { - hasher.putBytes(deterministicMapper.writeValueAsBytes(compactionState)); + hasher.putBytes(deterministicMapper.writeValueAsBytes(indexingState)); } catch (JsonProcessingException e) { - throw new RuntimeException("Failed to serialize CompactionState for fingerprinting", e); + throw new RuntimeException("Failed to serialize CompactionState object for fingerprinting", e); } hasher.putByte((byte) 0xff); diff --git a/server/src/main/java/org/apache/druid/segment/metadata/IndexingStateCache.java b/server/src/main/java/org/apache/druid/segment/metadata/IndexingStateCache.java index be98766322fc..900276e91fa8 100644 --- a/server/src/main/java/org/apache/druid/segment/metadata/IndexingStateCache.java +++ b/server/src/main/java/org/apache/druid/segment/metadata/IndexingStateCache.java @@ -102,7 +102,7 @@ public Optional getIndexingStateByFingerprint(String fingerprin /** * Adds or updates a single indexing state in the cache. *

    - * This is called when a new compaction state is persisted to the database via upsertIndexingState + * This is called when a new indexing state is persisted to the database via upsertIndexingState * to ensure the cache is immediately consistent without waiting for the next sync. *

    * This method checks if the state is already cached before performing the atomic update. diff --git a/server/src/main/java/org/apache/druid/segment/metadata/IndexingStateFingerprintMapper.java b/server/src/main/java/org/apache/druid/segment/metadata/IndexingStateFingerprintMapper.java index a39442fc6949..dc8760010b25 100644 --- a/server/src/main/java/org/apache/druid/segment/metadata/IndexingStateFingerprintMapper.java +++ b/server/src/main/java/org/apache/druid/segment/metadata/IndexingStateFingerprintMapper.java @@ -32,13 +32,16 @@ public interface IndexingStateFingerprintMapper { /** - * Generates a deterministic fingerprint for the given compaction state and datasource. + * Generates a deterministic fingerprint for the given indexing state and datasource. + *

    + * The fingerprint is a SHA-256 hash of the datasource name and serialized indexing state that is globally unique in + * the segment space. * - * @param dataSource The datasource name - * @param compactionState The compaction configuration to fingerprint + * @param dataSource The datasource name + * @param indexingState The compaction configuration to fingerprint * @return A hex-encoded SHA-256 fingerprint string */ - String generateFingerprint(String dataSource, CompactionState compactionState); + String generateFingerprint(String dataSource, CompactionState indexingState); /** * Retrieves a compaction state by its fingerprint. diff --git a/server/src/main/java/org/apache/druid/segment/metadata/IndexingStateStorage.java b/server/src/main/java/org/apache/druid/segment/metadata/IndexingStateStorage.java index 529ee5b37d04..b45d738d52e6 100644 --- a/server/src/main/java/org/apache/druid/segment/metadata/IndexingStateStorage.java +++ b/server/src/main/java/org/apache/druid/segment/metadata/IndexingStateStorage.java @@ -38,16 +38,16 @@ public interface IndexingStateStorage * If a fingerprint already exists, update to reflect proper used state and timestamp. * If a fingerprint doesn't exist, inserts a new row with the full state payload. * - * @param dataSource The datasource name - * @param fingerprint The fingerprint of the compaction state - * @param compactionState The compaction state to upsert - * @param updateTime The timestamp for this update + * @param dataSource The datasource name + * @param fingerprint The fingerprint of the indexing state + * @param indexingState The indexing state to upsert + * @param updateTime The timestamp for this update */ void upsertIndexingState( String dataSource, String fingerprint, - CompactionState compactionState, + CompactionState indexingState, DateTime updateTime ); diff --git a/server/src/main/java/org/apache/druid/segment/metadata/SqlIndexingStateStorage.java b/server/src/main/java/org/apache/druid/segment/metadata/SqlIndexingStateStorage.java index 6319a9014c16..93f61afacf28 100644 --- a/server/src/main/java/org/apache/druid/segment/metadata/SqlIndexingStateStorage.java +++ b/server/src/main/java/org/apache/druid/segment/metadata/SqlIndexingStateStorage.java @@ -45,7 +45,7 @@ /** * Database-backed implementation of {@link IndexingStateStorage}. *

    - * Manages the persistence and retrieval of {@link CompactionState} (AKA IndexinState) objects in the metadata storage. + * Manages the persistence and retrieval of {@link CompactionState} (AKA IndexingState) objects in the metadata storage. * Indexing states are uniquely identified by their fingerprints, which are SHA-256 hashes of their content. *

    *

    @@ -79,7 +79,7 @@ public SqlIndexingStateStorage( public void upsertIndexingState( @NotEmpty final String dataSource, @NotEmpty final String fingerprint, - @Nonnull final CompactionState compactionState, + @Nonnull final CompactionState indexingState, @Nonnull final DateTime updateTime ) { @@ -91,8 +91,8 @@ public void upsertIndexingState( if (fingerprint == null || fingerprint.isEmpty()) { errors.append("fingerprint cannot be empty; "); } - if (compactionState == null) { - errors.append("compactionState cannot be null; "); + if (indexingState == null) { + errors.append("indexingState cannot be null; "); } if (updateTime == null) { errors.append("updateTime cannot be null; "); @@ -155,7 +155,7 @@ public void upsertIndexingState( .bind("created_date", now) .bind("dataSource", dataSource) .bind("fingerprint", fingerprint) - .bind("payload", jsonMapper.writeValueAsBytes(compactionState)) + .bind("payload", jsonMapper.writeValueAsBytes(indexingState)) .bind("used", true) .bind("pending", true) .bind("used_status_last_updated", now) 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 562a1cc99612..837004670631 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 @@ -67,7 +67,7 @@ public enum State } /** - * List of checks performed to determine if compaction is already complete based on compaction state fingerprints. + * List of checks performed to determine if compaction is already complete based on indexing state fingerprints. */ private static final List> FINGERPRINT_CHECKS = List.of( Evaluator::allFingerprintedCandidatesHaveExpectedFingerprint @@ -434,7 +434,7 @@ private CompactionStatus evaluate() *

    * 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. + * {@link #unknownStateToSegments} where their indexing states will be analyzed. *

    */ private CompactionStatus allFingerprintedCandidatesHaveExpectedFingerprint() @@ -475,7 +475,7 @@ private CompactionStatus allFingerprintedCandidatesHaveExpectedFingerprint() String fingerprint = e.getKey(); CompactionState stateToValidate = fingerprintMapper.getStateForFingerprint(fingerprint).orElse(null); if (stateToValidate == null) { - log.warn("No compaction state found for fingerprint[%s]", fingerprint); + log.warn("No indexing state found for fingerprint[%s]", fingerprint); fingerprintedSegmentWithoutCachedStateFound = true; uncompactedSegments.addAll(e.getValue()); } else { @@ -493,7 +493,7 @@ private CompactionStatus allFingerprintedCandidatesHaveExpectedFingerprint() } if (fingerprintedSegmentWithoutCachedStateFound) { - return CompactionStatus.pending("One or more fingerprinted segments do not have a cached compaction state"); + return CompactionStatus.pending("One or more fingerprinted segments do not have a cached indexing state"); } else { return COMPLETE; } @@ -513,7 +513,7 @@ private CompactionStatus segmentsHaveBeenCompactedAtLeastOnce() } else if (segmentState == null) { uncompactedSegments.add(segment); } else { - unknownStateToSegments.computeIfAbsent(segmentState, k -> new ArrayList<>()).add(segment); + unknownStateToSegments.computeIfAbsent(segmentState, s -> new ArrayList<>()).add(segment); } } @@ -803,5 +803,4 @@ private static CompactionStatistics createStats(List segments) return CompactionStatistics.create(totalBytes, segments.size(), segmentIntervals.size()); } } - } 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 d6f379a2838a..3851b38fc91d 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/stats/Stats.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/stats/Stats.java @@ -170,8 +170,6 @@ 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/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java b/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java index d7d9ab6e2ab1..0beecb318956 100644 --- a/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java +++ b/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java @@ -118,7 +118,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest extends IndexerSqlMetadata private StubServiceEmitter emitter; private SqlSegmentMetadataTransactionFactory transactionFactory; private BlockingExecutorService cachePollExecutor; - private SqlIndexingStateStorage compactionStateStorage; + private SqlIndexingStateStorage indexingStateStorage; private final SegmentMetadataCache.UsageMode cacheMode; @@ -155,7 +155,7 @@ public void setUp() fingerprintGenerator = new FingerprintGenerator(mapper); segmentSchemaManager = new SegmentSchemaManager(derbyConnectorRule.metadataTablesConfigSupplier().get(), mapper, derbyConnector); segmentSchemaTestUtils = new SegmentSchemaTestUtils(derbyConnectorRule, derbyConnector, mapper); - compactionStateStorage = new SqlIndexingStateStorage( + indexingStateStorage = new SqlIndexingStateStorage( derbyConnectorRule.metadataTablesConfigSupplier().get(), mapper, derbyConnector @@ -213,7 +213,7 @@ public int getMaxRetries() derbyConnector, segmentSchemaManager, CentralizedDatasourceSchemaConfig.create(), - compactionStateStorage + indexingStateStorage ) { @Override @@ -4333,12 +4333,12 @@ public void testWriteOperation_alwaysUsesCache_inModeIfSynced() } @Test - public void testCommitSegmentsAndMetadata_marksPendingCompactionStateAsActive() + public void testCommitSegmentsAndMetadata_marksPendingIndexingStateAsActive() { String fingerprint = "vanillaFingerprint"; - CompactionState state = createTestCompactionState(); - compactionStateStorage.upsertIndexingState(TestDataSource.WIKI, fingerprint, state, DateTimes.nowUtc()); - Assert.assertEquals(Boolean.TRUE, compactionStateStorage.isIndexingStatePending(fingerprint)); + CompactionState state = createTestIndexingState(); + indexingStateStorage.upsertIndexingState(TestDataSource.WIKI, fingerprint, state, DateTimes.nowUtc()); + Assert.assertEquals(Boolean.TRUE, indexingStateStorage.isIndexingStatePending(fingerprint)); final DataSegment segment = CreateDataSegments.ofDatasource(TestDataSource.WIKI) .startingAt("2023-01-01") @@ -4354,16 +4354,16 @@ public void testCommitSegmentsAndMetadata_marksPendingCompactionStateAsActive() null ); - Assert.assertEquals(Boolean.FALSE, compactionStateStorage.isIndexingStatePending(fingerprint)); + Assert.assertEquals(Boolean.FALSE, indexingStateStorage.isIndexingStatePending(fingerprint)); } @Test - public void testCommitReplaceSegments_marksPendingCompactionStateAsActive() + public void testCommitReplaceSegments_marksPendingIndexingStateAsActive() { String fingerprint = "replaceFingerprint"; - CompactionState state = createTestCompactionState(); - compactionStateStorage.upsertIndexingState(TestDataSource.WIKI, fingerprint, state, DateTimes.nowUtc()); - Assert.assertEquals(Boolean.TRUE, compactionStateStorage.isIndexingStatePending(fingerprint)); + CompactionState state = createTestIndexingState(); + indexingStateStorage.upsertIndexingState(TestDataSource.WIKI, fingerprint, state, DateTimes.nowUtc()); + Assert.assertEquals(Boolean.TRUE, indexingStateStorage.isIndexingStatePending(fingerprint)); final DataSegment segment = CreateDataSegments.ofDatasource(TestDataSource.WIKI) .startingAt("2023-01-01") @@ -4384,16 +4384,16 @@ public void testCommitReplaceSegments_marksPendingCompactionStateAsActive() null ); - Assert.assertEquals(Boolean.FALSE, compactionStateStorage.isIndexingStatePending(fingerprint)); + Assert.assertEquals(Boolean.FALSE, indexingStateStorage.isIndexingStatePending(fingerprint)); } @Test - public void testCCommitAppendSegments_marksPendingCompactionStateAsActive() + public void testCommitAppendSegments_marksPendingIndexingStateAsActive() { String fingerprint = "appendFingerprint"; - CompactionState state = createTestCompactionState(); - compactionStateStorage.upsertIndexingState(TestDataSource.WIKI, fingerprint, state, DateTimes.nowUtc()); - Assert.assertEquals(Boolean.TRUE, compactionStateStorage.isIndexingStatePending(fingerprint)); + CompactionState state = createTestIndexingState(); + indexingStateStorage.upsertIndexingState(TestDataSource.WIKI, fingerprint, state, DateTimes.nowUtc()); + Assert.assertEquals(Boolean.TRUE, indexingStateStorage.isIndexingStatePending(fingerprint)); final DataSegment segment = CreateDataSegments.ofDatasource(TestDataSource.WIKI) .startingAt("2023-01-01") @@ -4410,10 +4410,10 @@ public void testCCommitAppendSegments_marksPendingCompactionStateAsActive() null ); - Assert.assertEquals(Boolean.FALSE, compactionStateStorage.isIndexingStatePending(fingerprint)); + Assert.assertEquals(Boolean.FALSE, indexingStateStorage.isIndexingStatePending(fingerprint)); } - private CompactionState createTestCompactionState() + private CompactionState createTestIndexingState() { return new CompactionState( new DynamicPartitionsSpec(100, null), diff --git a/server/src/test/java/org/apache/druid/metadata/SqlSegmentsMetadataQueryTest.java b/server/src/test/java/org/apache/druid/metadata/SqlSegmentsMetadataQueryTest.java index f01d3e002f54..8a8fed1c87c3 100644 --- a/server/src/test/java/org/apache/druid/metadata/SqlSegmentsMetadataQueryTest.java +++ b/server/src/test/java/org/apache/druid/metadata/SqlSegmentsMetadataQueryTest.java @@ -375,7 +375,7 @@ private static Set getIds(Set segments) return segments.stream().map(DataSegment::getId).collect(Collectors.toSet()); } - // ==================== Compaction State Tests ==================== + // ==================== Indexing State Tests ==================== @Test public void test_retrieveAllUsedIndexingStateFingerprints_emptyDatabase() @@ -384,7 +384,7 @@ public void test_retrieveAllUsedIndexingStateFingerprints_emptyDatabase() Set fingerprints = read(SqlSegmentsMetadataQuery::retrieveAllUsedIndexingStateFingerprints); - Assert.assertTrue("Should return empty set when no segments have compaction states", fingerprints.isEmpty()); + Assert.assertTrue("Should return empty set when no segments have indexing states", fingerprints.isEmpty()); } @Test @@ -392,18 +392,16 @@ public void test_retrieveAllUsedIndexingStateFingerprints() { derbyConnectorRule.getConnector().createIndexingStatesTable(); - // Insert compaction states - Map compactionStates = new HashMap<>(); - compactionStates.put("fp1", createTestCompactionState()); - compactionStates.put("fp2", createTestCompactionState()); - compactionStates.put("fp3", createTestCompactionState()); - insertCompactionStates(compactionStates); + Map indexingStates = new HashMap<>(); + indexingStates.put("fp1", createTestIndexingState()); + indexingStates.put("fp2", createTestIndexingState()); + indexingStates.put("fp3", createTestIndexingState()); + insertIndexingStates(indexingStates); - // Insert segments referencing compaction states - insertSegmentWithCompactionState("seg1", "fp1", true); - insertSegmentWithCompactionState("seg2", "fp2", true); - insertSegmentWithCompactionState("seg3", "fp1", true); // Duplicate fingerprint - insertSegmentWithCompactionState("seg4", "fp3", false); // Unused segment + insertSegmentWithIndexingState("seg1", "fp1", true); + insertSegmentWithIndexingState("seg2", "fp2", true); + insertSegmentWithIndexingState("seg3", "fp1", true); // Duplicate fingerprint + insertSegmentWithIndexingState("seg4", "fp3", false); // Unused segment Set fingerprints = read(SqlSegmentsMetadataQuery::retrieveAllUsedIndexingStateFingerprints); @@ -415,16 +413,16 @@ public void test_retrieveAllUsedIndexingStateFingerprints_ignoresNullFingerprint { derbyConnectorRule.getConnector().createIndexingStatesTable(); - Map compactionStates = new HashMap<>(); - compactionStates.put("fp1", createTestCompactionState()); - insertCompactionStates(compactionStates); + Map indexingStates = new HashMap<>(); + indexingStates.put("fp1", createTestIndexingState()); + insertIndexingStates(indexingStates); - insertSegmentWithCompactionState("seg1", "fp1", true); - insertSegmentWithCompactionState("seg2", null, true); // No compaction state + insertSegmentWithIndexingState("seg1", "fp1", true); + insertSegmentWithIndexingState("seg2", null, true); // No indexing state Set fingerprints = read(SqlSegmentsMetadataQuery::retrieveAllUsedIndexingStateFingerprints); - Assert.assertEquals("Should ignore segments without compaction states", Set.of("fp1"), fingerprints); + Assert.assertEquals("Should ignore segments without indexing states", Set.of("fp1"), fingerprints); } @Test @@ -434,7 +432,7 @@ public void test_retrieveAllUsedIndexingStates_emptyDatabase() List records = read(SqlSegmentsMetadataQuery::retrieveAllUsedIndexingStates); - Assert.assertTrue("Should return empty list when no compaction states exist", records.isEmpty()); + Assert.assertTrue("Should return empty list when no indexing states exist", records.isEmpty()); } @Test @@ -442,8 +440,7 @@ public void test_retrieveAllUsedIndexingStates_fullSync() { derbyConnectorRule.getConnector().createIndexingStatesTable(); - // Create distinct compaction states - CompactionState state1 = createTestCompactionState(); + CompactionState state1 = createTestIndexingState(); CompactionState state2 = new CompactionState( new DynamicPartitionsSpec(200, null), DimensionsSpec.EMPTY, @@ -453,21 +450,21 @@ public void test_retrieveAllUsedIndexingStates_fullSync() null, null ); - CompactionState state3 = createTestCompactionState(); + CompactionState state3 = createTestIndexingState(); - Map compactionStates = new HashMap<>(); - compactionStates.put("fp1", state1); - compactionStates.put("fp2", state2); - compactionStates.put("fp3", state3); // Unreferenced state - insertCompactionStates(compactionStates); + Map indexingStates = new HashMap<>(); + indexingStates.put("fp1", state1); + indexingStates.put("fp2", state2); + indexingStates.put("fp3", state3); // Unreferenced state + insertIndexingStates(indexingStates); // Only reference fp1 and fp2 - insertSegmentWithCompactionState("seg1", "fp1", true); - insertSegmentWithCompactionState("seg2", "fp2", true); + insertSegmentWithIndexingState("seg1", "fp1", true); + insertSegmentWithIndexingState("seg2", "fp2", true); List records = read(SqlSegmentsMetadataQuery::retrieveAllUsedIndexingStates); - Assert.assertEquals("Should return all compaction states", 3, records.size()); + Assert.assertEquals("Should return all indexing states", 3, records.size()); Set retrievedFingerprints = records.stream() .map(IndexingStateRecord::getFingerprint) @@ -486,40 +483,39 @@ public void test_retrieveAllUsedIndexingStates_fullSync() } @Test - public void test_retrieveAllUsedCompactionStates_onlyFromUsedSegments() + public void test_retrieveAllUsedIndexingStates_onlyFromUsedSegments() { derbyConnectorRule.getConnector().createIndexingStatesTable(); - Map compactionStates = new HashMap<>(); - compactionStates.put("fp1", createTestCompactionState()); - compactionStates.put("fp2", createTestCompactionState()); - insertCompactionStates(compactionStates); + Map indexingStates = new HashMap<>(); + indexingStates.put("fp1", createTestIndexingState()); + indexingStates.put("fp2", createTestIndexingState()); + insertIndexingStates(indexingStates); - insertSegmentWithCompactionState("seg1", "fp1", true); // Used - insertSegmentWithCompactionState("seg2", "fp2", false); // Unused + insertSegmentWithIndexingState("seg1", "fp1", true); // Used + insertSegmentWithIndexingState("seg2", "fp2", false); // Unused List records = read(SqlSegmentsMetadataQuery::retrieveAllUsedIndexingStates); - Assert.assertEquals("Should only return all compaction states", 2, records.size()); + Assert.assertEquals("Should only return all indexing states", 2, records.size()); } @Test - public void test_retrieveAllUsedCompactionStates_ignoresUnusedIndexingStates() + public void test_retrieveAllUsedIndexingStates_ignoresUnusedIndexingStates() { derbyConnectorRule.getConnector().createIndexingStatesTable(); - Map compactionStates = new HashMap<>(); - compactionStates.put("fp1", createTestCompactionState()); - insertCompactionStates(compactionStates); + Map indexingStates = new HashMap<>(); + indexingStates.put("fp1", createTestIndexingState()); + insertIndexingStates(indexingStates); - insertSegmentWithCompactionState("seg1", "fp1", true); + insertSegmentWithIndexingState("seg1", "fp1", true); - // Mark compaction state as unused - markCompactionStateAsUnused("fp1"); + markIndexingStateAsUnused("fp1"); List records = read(SqlSegmentsMetadataQuery::retrieveAllUsedIndexingStates); - Assert.assertTrue("Should not return unused compaction states", records.isEmpty()); + Assert.assertTrue("Should not return unused indexing states", records.isEmpty()); } @Test @@ -539,12 +535,11 @@ public void test_retrieveIndexingStatesForFingerprints_deltaSync() { derbyConnectorRule.getConnector().createIndexingStatesTable(); - // Insert multiple compaction states - Map compactionStates = new HashMap<>(); - compactionStates.put("fp1", createTestCompactionState()); - compactionStates.put("fp2", createTestCompactionState()); - compactionStates.put("fp3", createTestCompactionState()); - insertCompactionStates(compactionStates); + Map indexingStates = new HashMap<>(); + indexingStates.put("fp1", createTestIndexingState()); + indexingStates.put("fp2", createTestIndexingState()); + indexingStates.put("fp3", createTestIndexingState()); + insertIndexingStates(indexingStates); // Request specific fingerprints (delta sync scenario) List records = read( @@ -564,15 +559,15 @@ public void test_retrieveIndexingStatesForFingerprints_largeBatch() { derbyConnectorRule.getConnector().createIndexingStatesTable(); - // Insert 150 compaction states (exceeds batching threshold of 100) - Map compactionStates = new HashMap<>(); + // Insert 150 indexing states (exceeds batching threshold of 100) + Map indexingStates = new HashMap<>(); Set expectedFingerprints = new HashSet<>(); for (int i = 0; i < 150; i++) { String fingerprint = "fp" + i; - compactionStates.put(fingerprint, createTestCompactionState()); + indexingStates.put(fingerprint, createTestIndexingState()); expectedFingerprints.add(fingerprint); } - insertCompactionStates(compactionStates); + insertIndexingStates(indexingStates); // Request all fingerprints List records = read( @@ -592,9 +587,9 @@ public void test_retrieveIndexingStatesForFingerprints_nonexistentFingerprints() { derbyConnectorRule.getConnector().createIndexingStatesTable(); - Map compactionStates = new HashMap<>(); - compactionStates.put("fp1", createTestCompactionState()); - insertCompactionStates(compactionStates); + Map indexingStates = new HashMap<>(); + indexingStates.put("fp1", createTestIndexingState()); + insertIndexingStates(indexingStates); // Request fingerprints that don't exist List records = read( @@ -609,10 +604,10 @@ public void test_retrieveIndexingStatesForFingerprints_mixedExistingAndNonexiste { derbyConnectorRule.getConnector().createIndexingStatesTable(); - Map compactionStates = new HashMap<>(); - compactionStates.put("fp1", createTestCompactionState()); - compactionStates.put("fp2", createTestCompactionState()); - insertCompactionStates(compactionStates); + Map indexingStates = new HashMap<>(); + indexingStates.put("fp1", createTestIndexingState()); + indexingStates.put("fp2", createTestIndexingState()); + insertIndexingStates(indexingStates); // Mix existing and non-existing fingerprints List records = read( @@ -632,25 +627,25 @@ public void test_retrieveIndexingStatesForFingerprints_onlyReturnsUsedStates() { derbyConnectorRule.getConnector().createIndexingStatesTable(); - Map compactionStates = new HashMap<>(); - compactionStates.put("fp1", createTestCompactionState()); - compactionStates.put("fp2", createTestCompactionState()); - insertCompactionStates(compactionStates); + Map indexingStates = new HashMap<>(); + indexingStates.put("fp1", createTestIndexingState()); + indexingStates.put("fp2", createTestIndexingState()); + insertIndexingStates(indexingStates); // Mark fp2 as unused - markCompactionStateAsUnused("fp2"); + markIndexingStateAsUnused("fp2"); List records = read( sql -> sql.retrieveIndexingStatesForFingerprints(Set.of("fp1", "fp2")) ); - Assert.assertEquals("Should only return used compaction states", 1, records.size()); + Assert.assertEquals("Should only return used indexing states", 1, records.size()); Assert.assertEquals("Should return fp1", "fp1", records.get(0).getFingerprint()); } - // ==================== Helper Methods for Compaction State Tests ==================== + // ==================== Helper Methods for Indexing State Tests ==================== - private CompactionState createTestCompactionState() + private CompactionState createTestIndexingState() { return new CompactionState( new DynamicPartitionsSpec(100, null), @@ -663,7 +658,7 @@ private CompactionState createTestCompactionState() ); } - private void insertCompactionStates(Map compactionStates) + private void insertIndexingStates(Map indexingStates) { ObjectMapper mapper = TestHelper.JSON_MAPPER; MetadataStorageTablesConfig tablesConfig = derbyConnectorRule.metadataTablesConfigSupplier().get(); @@ -674,14 +669,14 @@ private void insertCompactionStates(Map compactionState ); derbyConnectorRule.getConnector().retryWithHandle(handle -> { - for (Map.Entry entry : compactionStates.entrySet()) { + for (Map.Entry entry : indexingStates.entrySet()) { manager.upsertIndexingState(TestDataSource.WIKI, entry.getKey(), entry.getValue(), DateTimes.nowUtc()); } return null; }); } - private void insertSegmentWithCompactionState( + private void insertSegmentWithIndexingState( String segmentId, String indexingStateFingerprint, boolean used @@ -714,7 +709,7 @@ private void insertSegmentWithCompactionState( }); } - private void markCompactionStateAsUnused(String fingerprint) + private void markIndexingStateAsUnused(String fingerprint) { MetadataStorageTablesConfig tablesConfig = derbyConnectorRule.metadataTablesConfigSupplier().get(); DerbyConnector connector = derbyConnectorRule.getConnector(); diff --git a/server/src/test/java/org/apache/druid/segment/metadata/HeapMemoryIndexingStateStorage.java b/server/src/test/java/org/apache/druid/segment/metadata/HeapMemoryIndexingStateStorage.java index 48ee3ffad69a..866739f9da45 100644 --- a/server/src/test/java/org/apache/druid/segment/metadata/HeapMemoryIndexingStateStorage.java +++ b/server/src/test/java/org/apache/druid/segment/metadata/HeapMemoryIndexingStateStorage.java @@ -51,12 +51,12 @@ public HeapMemoryIndexingStateStorage() public void upsertIndexingState( final String dataSource, final String fingerprint, - final CompactionState compactionState, + final CompactionState indexingState, final DateTime updateTime ) { // Store in memory for lookup during simulations/tests - this.fingerprintToStateMap.put(fingerprint, compactionState); + this.fingerprintToStateMap.put(fingerprint, indexingState); } @Override diff --git a/server/src/test/java/org/apache/druid/segment/metadata/SqlIndexingStateStorageTest.java b/server/src/test/java/org/apache/druid/segment/metadata/SqlIndexingStateStorageTest.java index f5db846a1297..7b3c529e4c9d 100644 --- a/server/src/test/java/org/apache/druid/segment/metadata/SqlIndexingStateStorageTest.java +++ b/server/src/test/java/org/apache/druid/segment/metadata/SqlIndexingStateStorageTest.java @@ -91,7 +91,7 @@ public void setUp() @Test public void test_upsertIndexingState_successfullyInsertsIntoDatabase() { - CompactionState state1 = createTestCompactionState(); + CompactionState state1 = createTestIndexingState(); String fingerprint = "fingerprint_abc123"; manager.upsertIndexingState( @@ -114,9 +114,9 @@ public void test_upsertIndexingState_successfullyInsertsIntoDatabase() } @Test - public void test_upsertCompactionState_andThen_markUnreferencedCompactionStateAsUnused_andThen_markIndexingStatesAsUsed() + public void test_upsertIndexingState_andThen_markUnreferencedIndexingStateAsUnused_andThen_markIndexingStatesAsUsed() { - CompactionState state1 = createTestCompactionState(); + CompactionState state1 = createTestIndexingState(); String fingerprint = "fingerprint_abc123"; manager.upsertIndexingState( @@ -134,7 +134,7 @@ public void test_upsertCompactionState_andThen_markUnreferencedCompactionStateAs @Test public void test_findReferencedIndexingStateMarkedAsUnused() { - CompactionState state1 = createTestCompactionState(); + CompactionState state1 = createTestIndexingState(); String fingerprint = "fingerprint_abc123"; manager.upsertIndexingState( @@ -177,7 +177,7 @@ public void test_findReferencedIndexingStateMarkedAsUnused() } @Test - public void test_deleteCompactionStatesOlderThan_deletesOnlyOldUnusedStates() + public void test_deleteIndexingStatesOlderThan_deletesOnlyOldUnusedStates() { DateTime now = DateTimes.nowUtc(); DateTime oldTime = now.minusDays(60); @@ -187,8 +187,8 @@ public void test_deleteCompactionStatesOlderThan_deletesOnlyOldUnusedStates() String oldFingerprint = "old_fp_should_delete"; String recentFingerprint = "recent_fp_should_keep"; - CompactionState oldState = createTestCompactionState(); - CompactionState recentState = createTestCompactionState(); + CompactionState oldState = createTestIndexingState(); + CompactionState recentState = createTestIndexingState(); // Insert old unused state (60 days old) derbyConnector.retryWithHandle(handle -> { @@ -248,8 +248,8 @@ public void test_upsertIndexingState_withNullState_throwsException() ); assertTrue( - exception.getMessage().contains("compactionState cannot be null"), - "Exception message should contain 'compactionState cannot be null'" + exception.getMessage().contains("indexingState cannot be null"), + "Exception message should contain 'indexingState cannot be null'" ); } @@ -259,7 +259,7 @@ public void test_upsertIndexingState_withEmptyFingerprint_throwsException() // The exception ends up wrapped in a sql exception doe to the retryWithHandle so we will just check the message Exception exception = assertThrows( Exception.class, - () -> manager.upsertIndexingState("ds", "", createBasicCompactionState(), DateTimes.nowUtc()) + () -> manager.upsertIndexingState("ds", "", createBasicIndexingState(), DateTimes.nowUtc()) ); assertTrue( @@ -272,7 +272,7 @@ public void test_upsertIndexingState_withEmptyFingerprint_throwsException() public void test_upsertIndexingState_verifyExistingFingerprintMarkedUsed() { String fingerprint = "existing_fingerprint"; - CompactionState state = createTestCompactionState(); + CompactionState state = createTestIndexingState(); // Persist initially manager.upsertIndexingState("ds1", fingerprint, state, DateTimes.nowUtc()); @@ -323,7 +323,7 @@ public void test_upsertIndexingState_verifyExistingFingerprintMarkedUsed() public void test_upsertIndexingState_whenAlreadyUsed_skipsUpdate() { String fingerprint = "already_used_fingerprint"; - CompactionState state = createTestCompactionState(); + CompactionState state = createTestIndexingState(); DateTime initialTime = DateTimes.of("2024-01-01T00:00:00.000Z"); // Insert fingerprint as used initially @@ -371,7 +371,7 @@ public void test_upsertIndexingState_whenAlreadyUsed_skipsUpdate() } @Test - public void test_markCompactionStateAsUsed_withEmptyList_returnsZero() + public void test_markIndexingStateAsUsed_withEmptyList_returnsZero() { assertEquals(0, manager.markIndexingStatesAsUsed(List.of())); } @@ -380,7 +380,7 @@ public void test_markCompactionStateAsUsed_withEmptyList_returnsZero() public void test_markIndexingStatesAsActive_marksPendingStateAsActive() { String fingerprint = "pending_fingerprint"; - CompactionState state = createTestCompactionState(); + CompactionState state = createTestIndexingState(); manager.upsertIndexingState("ds1", fingerprint, state, DateTimes.nowUtc()); @@ -408,7 +408,7 @@ public void test_markIndexingStatesAsActive_marksPendingStateAsActive() public void test_markIndexingStatesAsActive_idempotent_returnsZeroWhenAlreadyActive() { String fingerprint = "already_active_fingerprint"; - CompactionState state = createTestCompactionState(); + CompactionState state = createTestIndexingState(); manager.upsertIndexingState("ds1", fingerprint, state, DateTimes.nowUtc()); @@ -439,11 +439,11 @@ public void test_markIndexingStatesAsActive_nonExistentFingerprint_returnsZero() @Test public void test_generateIndexingStateFingerprint_deterministicFingerprinting() { - CompactionState compactionState1 = createBasicCompactionState(); - CompactionState compactionState2 = createBasicCompactionState(); + CompactionState indexingState1 = createBasicIndexingState(); + CompactionState indexingState2 = createBasicIndexingState(); - String fingerprint1 = fingerprintMapper.generateFingerprint("test-ds", compactionState1); - String fingerprint2 = fingerprintMapper.generateFingerprint("test-ds", compactionState2); + String fingerprint1 = fingerprintMapper.generateFingerprint("test-ds", indexingState1); + String fingerprint2 = fingerprintMapper.generateFingerprint("test-ds", indexingState2); assertEquals( fingerprint1, @@ -455,10 +455,10 @@ public void test_generateIndexingStateFingerprint_deterministicFingerprinting() @Test public void test_generateIndexingStateFingerprint_differentDatasourcesWithSameState_differentFingerprints() { - CompactionState compactionState = createBasicCompactionState(); + CompactionState indexingState = createBasicIndexingState(); - String fingerprint1 = fingerprintMapper.generateFingerprint("ds1", compactionState); - String fingerprint2 = fingerprintMapper.generateFingerprint("ds2", compactionState); + String fingerprint1 = fingerprintMapper.generateFingerprint("ds1", indexingState); + String fingerprint2 = fingerprintMapper.generateFingerprint("ds2", indexingState); assertNotEquals( fingerprint1, @@ -584,7 +584,7 @@ public void testGenerateIndexingStateFingerprint_differentPartitionsSpec() ); } - private CompactionState createBasicCompactionState() + private CompactionState createBasicIndexingState() { return new CompactionState( new DynamicPartitionsSpec(5000000, null), @@ -597,7 +597,7 @@ private CompactionState createBasicCompactionState() ); } - private CompactionState createTestCompactionState() + private CompactionState createTestIndexingState() { return new CompactionState( new DynamicPartitionsSpec(100, null), 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 b43507abd06e..33c360b02549 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 @@ -71,14 +71,14 @@ public class CompactionStatusTest .size(100_000_000L) .build(); - private HeapMemoryIndexingStateStorage compactionStateStorage; + private HeapMemoryIndexingStateStorage indexingStateStorage; private IndexingStateCache indexingStateCache; private IndexingStateFingerprintMapper fingerprintMapper; @Before public void setUp() { - compactionStateStorage = new HeapMemoryIndexingStateStorage(); + indexingStateStorage = new HeapMemoryIndexingStateStorage(); indexingStateCache = new IndexingStateCache(); fingerprintMapper = new DefaultIndexingStateFingerprintMapper( indexingStateCache, @@ -91,7 +91,7 @@ public void setUp() */ private void syncCacheFromManager() { - indexingStateCache.resetIndexingStatesForPublishedSegments(compactionStateStorage.getAllStoredStates()); + indexingStateCache.resetIndexingStatesForPublishedSegments(indexingStateStorage.getAllStoredStates()); } @Test @@ -603,7 +603,7 @@ public void test_evaluate_needsCompactionWhenAllSegmentsHaveUnexpectedIndexingSt CompactionState expectedState = compactionConfig.toCompactionState(); - compactionStateStorage.upsertIndexingState(TestDataSource.WIKI, "wrongFingerprint", wrongState, DateTimes.nowUtc()); + indexingStateStorage.upsertIndexingState(TestDataSource.WIKI, "wrongFingerprint", wrongState, DateTimes.nowUtc()); syncCacheFromManager(); verifyEvaluationNeedsCompactionBecauseWithCustomSegments( @@ -638,8 +638,8 @@ public void test_evaluate_needsCompactionWhenSomeSegmentsHaveUnexpectedIndexingS DataSegment.builder(WIKI_SEGMENT_2).indexingStateFingerprint("wrongFingerprint").build() ); - compactionStateStorage.upsertIndexingState(TestDataSource.WIKI, expectedFingerprint, expectedState, DateTimes.nowUtc()); - compactionStateStorage.upsertIndexingState(TestDataSource.WIKI, "wrongFingerprint", wrongState, DateTimes.nowUtc()); + indexingStateStorage.upsertIndexingState(TestDataSource.WIKI, expectedFingerprint, expectedState, DateTimes.nowUtc()); + indexingStateStorage.upsertIndexingState(TestDataSource.WIKI, "wrongFingerprint", wrongState, DateTimes.nowUtc()); syncCacheFromManager(); verifyEvaluationNeedsCompactionBecauseWithCustomSegments( @@ -650,7 +650,7 @@ public void test_evaluate_needsCompactionWhenSomeSegmentsHaveUnexpectedIndexingS } @Test - public void test_evaluate_noCompacationIfUnexpectedFingerprintHasExpectedCompactionState() + public void test_evaluate_noCompacationIfUnexpectedFingerprintHasExpectedIndexingState() { List segments = List.of( DataSegment.builder(WIKI_SEGMENT).indexingStateFingerprint("wrongFingerprint").build() @@ -662,7 +662,7 @@ public void test_evaluate_noCompacationIfUnexpectedFingerprintHasExpectedCompact .build(); CompactionState expectedState = compactionConfig.toCompactionState(); - compactionStateStorage.upsertIndexingState(TestDataSource.WIKI, "wrongFingerprint", expectedState, DateTimes.nowUtc()); + indexingStateStorage.upsertIndexingState(TestDataSource.WIKI, "wrongFingerprint", expectedState, DateTimes.nowUtc()); syncCacheFromManager(); final CompactionStatus status = CompactionStatus.compute( @@ -688,7 +688,7 @@ public void test_evaluate_needsCompactionWhenUnexpectedFingerprintAndNoFingerpri verifyEvaluationNeedsCompactionBecauseWithCustomSegments( CompactionCandidate.from(segments, null), compactionConfig, - "One or more fingerprinted segments do not have a cached compaction state" + "One or more fingerprinted segments do not have a cached indexing state" ); } @@ -730,7 +730,7 @@ public void test_evaluate_needsCompactionWhenNonFingerprintedSegmentsFailChecksO CompactionState expectedState = compactionConfig.toCompactionState(); String expectedFingerprint = fingerprintMapper.generateFingerprint(TestDataSource.WIKI, expectedState); - compactionStateStorage.upsertIndexingState(TestDataSource.WIKI, expectedFingerprint, expectedState, DateTimes.nowUtc()); + indexingStateStorage.upsertIndexingState(TestDataSource.WIKI, expectedFingerprint, expectedState, DateTimes.nowUtc()); syncCacheFromManager(); List segments = List.of( From c034d05d1934c1aa4dc0cda531e9d4571fada414 Mon Sep 17 00:00:00 2001 From: capistrant Date: Wed, 14 Jan 2026 17:08:37 -0600 Subject: [PATCH 68/72] fixup a few things post merge with master --- .../org/apache/druid/guice/DruidSecondaryModule.java | 9 ++++++--- .../apache/druid/client/CachingClusteredClientTest.java | 1 + .../apache/druid/server/http/DataSegmentPlusTest.java | 2 +- 3 files changed, 8 insertions(+), 4 deletions(-) diff --git a/processing/src/main/java/org/apache/druid/guice/DruidSecondaryModule.java b/processing/src/main/java/org/apache/druid/guice/DruidSecondaryModule.java index da6f7cc000a5..f012cdc93db2 100644 --- a/processing/src/main/java/org/apache/druid/guice/DruidSecondaryModule.java +++ b/processing/src/main/java/org/apache/druid/guice/DruidSecondaryModule.java @@ -31,10 +31,10 @@ import com.google.inject.Key; import com.google.inject.Module; import com.google.inject.Provides; -import org.apache.druid.guice.annotations.Deterministic; import com.google.inject.TypeLiteral; import com.google.inject.multibindings.MapBinder; import org.apache.druid.frame.wire.FrameWireTransferable; +import org.apache.druid.guice.annotations.Deterministic; import org.apache.druid.guice.annotations.Json; import org.apache.druid.guice.annotations.JsonNonNull; import org.apache.druid.guice.annotations.Smile; @@ -171,12 +171,15 @@ public WireTransferableContext getWireTransferableContext( @Provides @LazySingleton @Deterministic - public ObjectMapper getSortedMapper(Injector injector) + public ObjectMapper getSortedMapper( + Injector injector, + Map wtDeserializers + ) { final ObjectMapper sortedMapper = new DefaultObjectMapper(); sortedMapper.configure(SerializationFeature.ORDER_MAP_ENTRIES_BY_KEYS, true); sortedMapper.configure(MapperFeature.SORT_PROPERTIES_ALPHABETICALLY, true); - setupJackson(injector, sortedMapper); + setupJackson(injector, sortedMapper, wtDeserializers, isUseLegacyFrameSerialization()); return sortedMapper; } diff --git a/server/src/test/java/org/apache/druid/client/CachingClusteredClientTest.java b/server/src/test/java/org/apache/druid/client/CachingClusteredClientTest.java index 9aa508cbba4f..b7264204bfe5 100644 --- a/server/src/test/java/org/apache/druid/client/CachingClusteredClientTest.java +++ b/server/src/test/java/org/apache/druid/client/CachingClusteredClientTest.java @@ -2766,6 +2766,7 @@ private MyDataSegment() -1, 0, 0, + null, PruneSpecsHolder.DEFAULT ); } 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 84e10a213c63..c86a64bb077b 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 @@ -126,7 +126,7 @@ public void testSerde() throws JsonProcessingException ); // verify dataSegment - Assert.assertEquals(13, segmentObjectMap.size()); + Assert.assertEquals(14, segmentObjectMap.size()); Assert.assertEquals("something", segmentObjectMap.get("dataSource")); Assert.assertEquals(interval.toString(), segmentObjectMap.get("interval")); Assert.assertEquals("1", segmentObjectMap.get("version")); From 0cdbb3740ab33d65e029c457ab6ca8e1b972ffd7 Mon Sep 17 00:00:00 2001 From: capistrant Date: Fri, 16 Jan 2026 09:28:46 -0600 Subject: [PATCH 69/72] Cleanup and refactor after code review round Batch marking of indexing states as active to avoid chained updates where only one is needed Build segments table missing columns error column by column refactor how we are configuring and executing the ol metadata cleanup duties. fix missed naming refactor Improve readability of upsertIndexingState Fixup SqlIndexingStateStorage constructor drop default impl of isUniqueConstraintViolation Refactor how the deterministic mapper is handled for reindexing --- .../NewestSegmentFirstPolicyBenchmark.java | 4 +- .../compact/CompactionSupervisorTest.java | 4 +- .../indexing/compact/CompactionJobQueue.java | 5 +- .../compact/OverlordCompactionScheduler.java | 9 +- ...g.java => IndexingStateCleanupConfig.java} | 52 ++---- .../overlord/config/OverlordKillConfigs.java | 8 +- .../duty/KillUnreferencedIndexingState.java | 62 +++++-- .../duty/OverlordMetadataCleanupDuty.java | 47 ++--- .../OverlordCompactionSchedulerTest.java | 4 +- .../KillUnreferencedIndexingStateTest.java | 36 ++-- .../druid/guice/DruidSecondaryModule.java | 20 --- .../guice/annotations/Deterministic.java | 38 ---- .../IndexerSQLMetadataStorageCoordinator.java | 28 ++- .../druid/metadata/SQLMetadataConnector.java | 30 ++-- ...DefaultIndexingStateFingerprintMapper.java | 18 +- .../metadata/IndexingStateStorage.java | 6 +- .../metadata/SqlIndexingStateStorage.java | 163 +++++++++++------- .../metadata/SQLMetadataConnectorTest.java | 6 + .../segment/metadata/CompactionTestUtils.java | 51 ------ .../HeapMemoryIndexingStateStorage.java | 2 +- .../metadata/SqlIndexingStateStorageTest.java | 19 +- .../compaction/CompactionStatusTest.java | 4 +- .../NewestSegmentFirstPolicyTest.java | 4 +- .../org/apache/druid/cli/CliOverlord.java | 4 +- 24 files changed, 303 insertions(+), 321 deletions(-) rename indexing-service/src/main/java/org/apache/druid/indexing/overlord/config/{OverlordMetadataCleanupConfig.java => IndexingStateCleanupConfig.java} (58%) delete mode 100644 processing/src/main/java/org/apache/druid/guice/annotations/Deterministic.java delete mode 100644 server/src/test/java/org/apache/druid/segment/metadata/CompactionTestUtils.java diff --git a/benchmarks/src/test/java/org/apache/druid/server/coordinator/NewestSegmentFirstPolicyBenchmark.java b/benchmarks/src/test/java/org/apache/druid/server/coordinator/NewestSegmentFirstPolicyBenchmark.java index 58f5c6f2eb3b..091cfcc8b9e9 100644 --- a/benchmarks/src/test/java/org/apache/druid/server/coordinator/NewestSegmentFirstPolicyBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/server/coordinator/NewestSegmentFirstPolicyBenchmark.java @@ -21,8 +21,8 @@ import com.google.common.collect.ImmutableList; import org.apache.druid.client.DataSourcesSnapshot; +import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.DateTimes; -import org.apache.druid.segment.metadata.CompactionTestUtils; import org.apache.druid.segment.metadata.DefaultIndexingStateFingerprintMapper; import org.apache.druid.segment.metadata.NoopIndexingStateCache; import org.apache.druid.server.compaction.CompactionCandidateSearchPolicy; @@ -139,7 +139,7 @@ public void measureNewestSegmentFirstPolicy(Blackhole blackhole) compactionConfigs, dataSources, Collections.emptyMap(), - new DefaultIndexingStateFingerprintMapper(new NoopIndexingStateCache(), CompactionTestUtils.createDeterministicMapper()) + new DefaultIndexingStateFingerprintMapper(new NoopIndexingStateCache(), new DefaultObjectMapper()) ); for (int i = 0; i < numCompactionTaskSlots && iterator.hasNext(); i++) { blackhole.consume(iterator.next()); diff --git a/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionSupervisorTest.java b/embedded-tests/src/test/java/org/apache/druid/testing/embedded/compact/CompactionSupervisorTest.java index 7f29f97804dc..5355f1601ae4 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 @@ -27,11 +27,11 @@ import org.apache.druid.indexing.common.task.IndexTask; import org.apache.druid.indexing.compact.CompactionSupervisorSpec; import org.apache.druid.indexing.overlord.Segments; +import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.query.DruidMetrics; import org.apache.druid.rpc.UpdateResponse; -import org.apache.druid.segment.metadata.CompactionTestUtils; import org.apache.druid.segment.metadata.DefaultIndexingStateFingerprintMapper; import org.apache.druid.segment.metadata.IndexingStateCache; import org.apache.druid.segment.metadata.IndexingStateFingerprintMapper; @@ -291,7 +291,7 @@ private void verifyCompactedSegmentsHaveFingerprints(DataSourceCompactionConfig IndexingStateCache cache = overlord.bindings().getInstance(IndexingStateCache.class); IndexingStateFingerprintMapper fingerprintMapper = new DefaultIndexingStateFingerprintMapper( cache, - CompactionTestUtils.createDeterministicMapper() + new DefaultObjectMapper() ); String expectedFingerprint = fingerprintMapper.generateFingerprint( dataSource, diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionJobQueue.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionJobQueue.java index 72ba2d6bf845..9446ac664f29 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 @@ -112,8 +112,7 @@ public CompactionJobQueue( BrokerClient brokerClient, ObjectMapper objectMapper, IndexingStateStorage indexingStateStorage, - IndexingStateCache indexingStateCache, - ObjectMapper deterministicCompactionStateMapper + IndexingStateCache indexingStateCache ) { this.runStats = new CoordinatorRunStats(); @@ -130,7 +129,7 @@ public CompactionJobQueue( clusterCompactionConfig, dataSourcesSnapshot.getUsedSegmentsTimelinesPerDataSource()::get, snapshotBuilder, - new DefaultIndexingStateFingerprintMapper(indexingStateCache, deterministicCompactionStateMapper) + new DefaultIndexingStateFingerprintMapper(indexingStateCache, objectMapper) ); this.indexingStateStorage = indexingStateStorage; 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 cdc38fa16815..11709e616c71 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/compact/OverlordCompactionScheduler.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/OverlordCompactionScheduler.java @@ -27,7 +27,6 @@ import org.apache.druid.client.broker.BrokerClient; import org.apache.druid.client.indexing.ClientCompactionRunnerInfo; import org.apache.druid.error.DruidException; -import org.apache.druid.guice.annotations.Deterministic; import org.apache.druid.indexer.TaskLocation; import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexing.common.actions.TaskActionClientFactory; @@ -145,7 +144,6 @@ public class OverlordCompactionScheduler implements CompactionScheduler private final IndexingStateStorage indexingStateStorage; private final IndexingStateCache indexingStateCache; - private final ObjectMapper deterministicMapper; @Inject public OverlordCompactionScheduler( @@ -164,8 +162,7 @@ public OverlordCompactionScheduler( ServiceEmitter emitter, ObjectMapper objectMapper, IndexingStateStorage indexingStateStorage, - IndexingStateCache indexingStateCache, - @Deterministic ObjectMapper deterministicMapper + IndexingStateCache indexingStateCache ) { final long segmentPollPeriodMillis = @@ -193,7 +190,6 @@ public OverlordCompactionScheduler( this.druidInputSourceFactory = druidInputSourceFactory; this.indexingStateStorage = indexingStateStorage; this.indexingStateCache = indexingStateCache; - this.deterministicMapper = deterministicMapper; this.taskRunnerListener = new TaskRunnerListener() { @Override @@ -393,8 +389,7 @@ private synchronized void resetCompactionJobQueue() brokerClient, objectMapper, indexingStateStorage, - indexingStateCache, - deterministicMapper + indexingStateCache ); latestJobQueue.set(queue); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/config/OverlordMetadataCleanupConfig.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/config/IndexingStateCleanupConfig.java similarity index 58% rename from indexing-service/src/main/java/org/apache/druid/indexing/overlord/config/OverlordMetadataCleanupConfig.java rename to indexing-service/src/main/java/org/apache/druid/indexing/overlord/config/IndexingStateCleanupConfig.java index 4142bdc5cd01..fad8ea101f49 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/config/OverlordMetadataCleanupConfig.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/config/IndexingStateCleanupConfig.java @@ -22,60 +22,40 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import org.apache.druid.common.config.Configs; +import org.apache.druid.server.coordinator.config.MetadataCleanupConfig; import org.joda.time.Duration; import java.util.Objects; -public class OverlordMetadataCleanupConfig +/** + * Configuration for cleaning up indexing state metadata. + *

    + * Extends {@link MetadataCleanupConfig} to add support for pending state retention. + */ +public class IndexingStateCleanupConfig extends MetadataCleanupConfig { - public static final OverlordMetadataCleanupConfig DEFAULT = new OverlordMetadataCleanupConfig(null, null, null, null); - - @JsonProperty("on") - private final boolean cleanupEnabled; - - @JsonProperty("period") - private final Duration cleanupPeriod; - - @JsonProperty("durationToRetain") - private final Duration durationToRetain; + public static final IndexingStateCleanupConfig DEFAULT = new IndexingStateCleanupConfig(null, null, null, null); @JsonProperty("pendingDurationToRetain") private final Duration pendingDurationToRetain; @JsonCreator - public OverlordMetadataCleanupConfig( + public IndexingStateCleanupConfig( @JsonProperty("on") Boolean cleanupEnabled, @JsonProperty("period") Duration cleanupPeriod, @JsonProperty("durationToRetain") Duration durationToRetain, @JsonProperty("pendingDurationToRetain") Duration pendingDurationToRetain ) { - this.cleanupEnabled = Configs.valueOrDefault(cleanupEnabled, true); - this.cleanupPeriod = Configs.valueOrDefault(cleanupPeriod, Duration.standardDays(1)); - this.durationToRetain = Configs.valueOrDefault(durationToRetain, Duration.standardDays(7)); + super(cleanupEnabled, cleanupPeriod, durationToRetain); this.pendingDurationToRetain = Configs.valueOrDefault(pendingDurationToRetain, Duration.standardDays(7)); } - public Duration getCleanupPeriod() - { - return cleanupPeriod; - } - - public Duration getDurationToRetain() - { - return durationToRetain; - } - public Duration getPendingDurationToRetain() { return pendingDurationToRetain; } - public boolean isCleanupEnabled() - { - return cleanupEnabled; - } - @Override public boolean equals(Object o) { @@ -85,16 +65,16 @@ public boolean equals(Object o) if (o == null || getClass() != o.getClass()) { return false; } - OverlordMetadataCleanupConfig that = (OverlordMetadataCleanupConfig) o; - return cleanupEnabled == that.cleanupEnabled - && Objects.equals(cleanupPeriod, that.cleanupPeriod) - && Objects.equals(durationToRetain, that.durationToRetain) - && Objects.equals(pendingDurationToRetain, that.pendingDurationToRetain); + if (!super.equals(o)) { + return false; + } + IndexingStateCleanupConfig that = (IndexingStateCleanupConfig) o; + return Objects.equals(pendingDurationToRetain, that.pendingDurationToRetain); } @Override public int hashCode() { - return Objects.hash(cleanupEnabled, cleanupPeriod, durationToRetain, pendingDurationToRetain); + return Objects.hash(super.hashCode(), pendingDurationToRetain); } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/config/OverlordKillConfigs.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/config/OverlordKillConfigs.java index 76e30ebdcca6..3e77a8f1eae7 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/config/OverlordKillConfigs.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/config/OverlordKillConfigs.java @@ -27,16 +27,16 @@ public class OverlordKillConfigs public static OverlordKillConfigs DEFAULT = new OverlordKillConfigs(null); @JsonProperty("indexingStates") - private final OverlordMetadataCleanupConfig indexingStates; + private final IndexingStateCleanupConfig indexingStates; public OverlordKillConfigs( - @JsonProperty("indexingStates") OverlordMetadataCleanupConfig indexingStates + @JsonProperty("indexingStates") IndexingStateCleanupConfig indexingStates ) { - this.indexingStates = Configs.valueOrDefault(indexingStates, OverlordMetadataCleanupConfig.DEFAULT); + this.indexingStates = Configs.valueOrDefault(indexingStates, IndexingStateCleanupConfig.DEFAULT); } - public OverlordMetadataCleanupConfig indexingStates() + public IndexingStateCleanupConfig indexingStates() { return indexingStates; } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/duty/KillUnreferencedIndexingState.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/duty/KillUnreferencedIndexingState.java index 13708d5565f0..a00688b6ea22 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/duty/KillUnreferencedIndexingState.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/duty/KillUnreferencedIndexingState.java @@ -19,7 +19,7 @@ package org.apache.druid.indexing.overlord.duty; -import org.apache.druid.indexing.overlord.config.OverlordMetadataCleanupConfig; +import org.apache.druid.indexing.overlord.config.IndexingStateCleanupConfig; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.segment.metadata.IndexingStateStorage; import org.joda.time.DateTime; @@ -42,38 +42,76 @@ public class KillUnreferencedIndexingState extends OverlordMetadataCleanupDuty { private static final Logger log = new Logger(KillUnreferencedIndexingState.class); private final IndexingStateStorage indexingStateStorage; + private final IndexingStateCleanupConfig config; @Inject public KillUnreferencedIndexingState( - OverlordMetadataCleanupConfig config, + IndexingStateCleanupConfig config, IndexingStateStorage indexingStateStorage ) { super("indexingStates", config); + this.config = config; this.indexingStateStorage = indexingStateStorage; } @Override - protected int cleanupEntriesCreatedBeforeDurationToRetain(DateTime minCreatedTime) + public void run() + { + if (!config.isCleanupEnabled()) { + return; + } + + final DateTime now = getCurrentTime(); + + if (getLastCleanupTime().plus(config.getCleanupPeriod()).isBefore(now)) { + try { + // Pending cleanup (specific to indexing states) + DateTime pendingMinCreatedTime = now.minus(config.getPendingDurationToRetain()); + int deletedPendingEntries = indexingStateStorage.deletePendingIndexingStatesOlderThan( + pendingMinCreatedTime.getMillis() + ); + if (deletedPendingEntries > 0) { + log.info( + "Removed [%,d] pending [%s] created before [%s].", + deletedPendingEntries, + getEntryType(), + pendingMinCreatedTime + ); + } + } + catch (Exception e) { + log.error(e, "Failed to perform pending cleanup of [%s]", getEntryType()); + } + + // Delegate to parent for the non-specialized cleanup + super.run(); + } + } + + /** + * Cleans up unreferenced indexing states created before the specified time. + *

    + * Before deletion, it executes the following steps to ensure data integrity: + *

      + *
    1. Marks unreferenced indexing states as unused.
    2. + *
    3. Finds any unused indexing states that are still referenced by used segments and marks them as used to avoid unwanted deletion.
    4. + *
    + * @param minCreatedTime the minimum creation time for indexing states to be considered for deletion + * @return the number of indexing states deleted + */ + @Override + protected int cleanupEntriesCreatedBefore(DateTime minCreatedTime) { - // 1: Mark unreferenced states as unused int unused = indexingStateStorage.markUnreferencedIndexingStatesAsUnused(); log.info("Marked [%s] unreferenced indexing states as unused.", unused); - // 2: Repair - find unused states still referenced by segments List stateFingerprints = indexingStateStorage.findReferencedIndexingStateMarkedAsUnused(); if (!stateFingerprints.isEmpty()) { int numUpdated = indexingStateStorage.markIndexingStatesAsUsed(stateFingerprints); log.info("Marked [%s] unused indexing states referenced by used segments as used.", numUpdated); } - // 3: Delete unused states older than threshold return indexingStateStorage.deleteUnusedIndexingStatesOlderThan(minCreatedTime.getMillis()); } - - @Override - protected int cleanupEntriesCreatedBeforePendingDurationToRetain(DateTime minCreatedTime) - { - return indexingStateStorage.deletePendingIndexingStatesOlderThan(minCreatedTime.getMillis()); - } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/duty/OverlordMetadataCleanupDuty.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/duty/OverlordMetadataCleanupDuty.java index 57da62e7de4b..34d0fe55f1ab 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/duty/OverlordMetadataCleanupDuty.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/duty/OverlordMetadataCleanupDuty.java @@ -19,9 +19,10 @@ package org.apache.druid.indexing.overlord.duty; -import org.apache.druid.indexing.overlord.config.OverlordMetadataCleanupConfig; +import com.google.common.annotations.VisibleForTesting; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.server.coordinator.config.MetadataCleanupConfig; import org.joda.time.DateTime; /** @@ -29,7 +30,7 @@ *

    * In every invocation of {@link #run}, the duty checks if the {@code cleanupPeriod} * has elapsed since the {@link #lastCleanupTime}. If it has, then the method - * {@link #cleanupEntriesCreatedBeforeDurationToRetain(DateTime)} is invoked. Otherwise, the duty + * {@link #cleanupEntriesCreatedBefore(DateTime)} is invoked. Otherwise, the duty * completes immediately without making any changes. */ public abstract class OverlordMetadataCleanupDuty implements OverlordDuty @@ -37,11 +38,11 @@ public abstract class OverlordMetadataCleanupDuty implements OverlordDuty private static final Logger log = new Logger(OverlordMetadataCleanupDuty.class); private final String entryType; - private final OverlordMetadataCleanupConfig cleanupConfig; + private final MetadataCleanupConfig cleanupConfig; private DateTime lastCleanupTime = DateTimes.utc(0); - protected OverlordMetadataCleanupDuty(String entryType, OverlordMetadataCleanupConfig cleanupConfig) + protected OverlordMetadataCleanupDuty(String entryType, MetadataCleanupConfig cleanupConfig) { this.entryType = entryType; this.cleanupConfig = cleanupConfig; @@ -63,21 +64,15 @@ public void run() final DateTime now = getCurrentTime(); - // Perform cleanup only if cleanup period has elapsed if (lastCleanupTime.plus(cleanupConfig.getCleanupPeriod()).isBefore(now)) { - lastCleanupTime = now; + setLastCleanupTime(now); try { DateTime minCreatedTime = now.minus(cleanupConfig.getDurationToRetain()); - int deletedEntries = cleanupEntriesCreatedBeforeDurationToRetain(minCreatedTime); + int deletedEntries = cleanupEntriesCreatedBefore(minCreatedTime); if (deletedEntries > 0) { log.info("Removed [%,d] [%s] created before [%s].", deletedEntries, entryType, minCreatedTime); } - DateTime pendingMinCreatedTime = now.minus(cleanupConfig.getPendingDurationToRetain()); - int deletedPendingEntries = cleanupEntriesCreatedBeforePendingDurationToRetain(pendingMinCreatedTime); - if (deletedPendingEntries > 0) { - log.info("Removed [%,d] pending entries [%s] created before [%s].", deletedPendingEntries, entryType, pendingMinCreatedTime); - } } catch (Exception e) { log.error(e, "Failed to perform cleanup of [%s]", entryType); @@ -85,6 +80,21 @@ public void run() } } + protected String getEntryType() + { + return entryType; + } + + protected DateTime getLastCleanupTime() + { + return lastCleanupTime; + } + + protected void setLastCleanupTime(DateTime time) + { + lastCleanupTime = time; + } + @Override public boolean isEnabled() { @@ -102,23 +112,20 @@ public DutySchedule getSchedule() } /** - * Cleans up metadata entries created before the {@code minCreatedTime} calculated with {@link OverlordMetadataCleanupConfig#durationToRetain}. + * Cleans up metadata entries created before the {@code minCreatedTime}. *

    * This method is not invoked if the {@code cleanupPeriod} has not elapsed since the {@link #lastCleanupTime}. * * @return Number of deleted metadata entries */ - protected abstract int cleanupEntriesCreatedBeforeDurationToRetain(DateTime minCreatedTime); + protected abstract int cleanupEntriesCreatedBefore(DateTime minCreatedTime); /** - * Cleans up pending metadata entries created before the {@code minCreatedTime} calculated with {@link OverlordMetadataCleanupConfig#pendingDurationToRetain}. + * Returns the current time. *

    - * This method is not invoked if the {@code cleanupPeriod} has not elapsed since the {@link #lastCleanupTime}. - * - * @return Number of deleted pending metadata entries + * Exists so testing can spoof the current time to validate behavior Duty behavior. */ - protected abstract int cleanupEntriesCreatedBeforePendingDurationToRetain(DateTime minCreatedTime); - + @VisibleForTesting protected DateTime getCurrentTime() { return DateTimes.nowUtc(); 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 3c85beb5bcd4..4c98b52f48c1 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,7 +64,6 @@ 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.CompactionTestUtils; import org.apache.druid.segment.metadata.HeapMemoryIndexingStateStorage; import org.apache.druid.segment.metadata.IndexingStateCache; import org.apache.druid.server.compaction.CompactionSimulateResult; @@ -236,8 +235,7 @@ private void initScheduler() serviceEmitter, OBJECT_MAPPER, new HeapMemoryIndexingStateStorage(), - new IndexingStateCache(), - CompactionTestUtils.createDeterministicMapper() + new IndexingStateCache() ); } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/duty/KillUnreferencedIndexingStateTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/duty/KillUnreferencedIndexingStateTest.java index 59ab2077d9c5..cbb77f624fae 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/duty/KillUnreferencedIndexingStateTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/duty/KillUnreferencedIndexingStateTest.java @@ -21,7 +21,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; -import org.apache.druid.indexing.overlord.config.OverlordMetadataCleanupConfig; +import org.apache.druid.indexing.overlord.config.IndexingStateCleanupConfig; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.metadata.MetadataStorageTablesConfig; @@ -71,10 +71,13 @@ public void test_killUnreferencedCompactionState_validateLifecycleOfActiveCompac List dateTimes = new ArrayList<>(); DateTime now = DateTimes.nowUtc(); dateTimes.add(now); + dateTimes.add(now); + dateTimes.add(now.plusMinutes(61)); dateTimes.add(now.plusMinutes(61)); dateTimes.add(now.plusMinutes(6 * 60 + 1)); + dateTimes.add(now.plusMinutes(6 * 60 + 1)); - OverlordMetadataCleanupConfig cleanupConfig = new OverlordMetadataCleanupConfig( + IndexingStateCleanupConfig cleanupConfig = new IndexingStateCleanupConfig( true, Period.parse("PT1H").toStandardDuration(), Period.parse("PT6H").toStandardDuration(), // Unused and over 6 hours old should be deleted @@ -88,7 +91,7 @@ public void test_killUnreferencedCompactionState_validateLifecycleOfActiveCompac CompactionState state = createTestCompactionState(); compactionStateStorage.upsertIndexingState("test-ds", fingerprint, state, DateTimes.nowUtc()); - compactionStateStorage.markIndexingStatesAsActive(fingerprint); + compactionStateStorage.markIndexingStatesAsActive(List.of(fingerprint)); Assert.assertEquals(Boolean.TRUE, getCompactionStateUsedStatus(fingerprint)); @@ -111,9 +114,11 @@ public void test_killUnreferencedCompactionState_validateRepair() List dateTimes = new ArrayList<>(); DateTime now = DateTimes.nowUtc(); dateTimes.add(now); + dateTimes.add(now); + dateTimes.add(now.plusMinutes(61)); dateTimes.add(now.plusMinutes(61)); - OverlordMetadataCleanupConfig cleanupConfig = new OverlordMetadataCleanupConfig( + IndexingStateCleanupConfig cleanupConfig = new IndexingStateCleanupConfig( true, Period.parse("PT1H").toStandardDuration(), Period.parse("PT6H").toStandardDuration(), @@ -128,7 +133,7 @@ public void test_killUnreferencedCompactionState_validateRepair() CompactionState state = createTestCompactionState(); compactionStateStorage.upsertIndexingState("test-ds", fingerprint, state, DateTimes.nowUtc()); - compactionStateStorage.markIndexingStatesAsActive(fingerprint); + compactionStateStorage.markIndexingStatesAsActive(List.of(fingerprint)); Assert.assertEquals(Boolean.TRUE, getCompactionStateUsedStatus(fingerprint)); duty.run(); @@ -166,7 +171,7 @@ public void test_killUnreferencedCompactionState_validateRepair() @Test public void test_killUnreferencedCompactionState_disabled() { - OverlordMetadataCleanupConfig cleanupConfig = new OverlordMetadataCleanupConfig( + IndexingStateCleanupConfig cleanupConfig = new IndexingStateCleanupConfig( false, // cleanup disabled Period.parse("PT1H").toStandardDuration(), Period.parse("PT6H").toStandardDuration(), @@ -179,7 +184,7 @@ public void test_killUnreferencedCompactionState_disabled() // Insert compaction state String fingerprint = "disabled_fingerprint"; compactionStateStorage.upsertIndexingState("test-ds", fingerprint, createTestCompactionState(), DateTimes.nowUtc()); - compactionStateStorage.markIndexingStatesAsActive(fingerprint); + compactionStateStorage.markIndexingStatesAsActive(List.of(fingerprint)); // Run duty - should do nothing duty.run(); @@ -194,9 +199,11 @@ public void test_killUnreferencedCompactionState_validateLifecycleOfPendingCompa List dateTimes = new ArrayList<>(); DateTime now = DateTimes.nowUtc(); dateTimes.add(now.plusDays(8)); + dateTimes.add(now.plusDays(8)); + dateTimes.add(now.plusDays(15)); dateTimes.add(now.plusDays(15)); - OverlordMetadataCleanupConfig cleanupConfig = new OverlordMetadataCleanupConfig( + IndexingStateCleanupConfig cleanupConfig = new IndexingStateCleanupConfig( true, Period.parse("PT1H").toStandardDuration(), Period.parse("P7D").toStandardDuration(), @@ -228,9 +235,11 @@ public void test_killUnreferencedCompactionState_validateMixedPendingAndActiveCo List dateTimes = new ArrayList<>(); DateTime now = DateTimes.nowUtc(); dateTimes.add(now.plusDays(8)); + dateTimes.add(now.plusDays(8)); + dateTimes.add(now.plusDays(31)); dateTimes.add(now.plusDays(31)); - OverlordMetadataCleanupConfig cleanupConfig = new OverlordMetadataCleanupConfig( + IndexingStateCleanupConfig cleanupConfig = new IndexingStateCleanupConfig( true, Period.parse("PT1H").toStandardDuration(), Period.parse("P7D").toStandardDuration(), @@ -246,7 +255,7 @@ public void test_killUnreferencedCompactionState_validateMixedPendingAndActiveCo compactionStateStorage.upsertIndexingState("test-ds", pendingFingerprint, state, DateTimes.nowUtc()); compactionStateStorage.upsertIndexingState("test-ds", nonPendingFingerprint, state, DateTimes.nowUtc()); - compactionStateStorage.markIndexingStatesAsActive(nonPendingFingerprint); + compactionStateStorage.markIndexingStatesAsActive(List.of(nonPendingFingerprint)); Assert.assertEquals(Boolean.TRUE, compactionStateStorage.isIndexingStatePending(pendingFingerprint)); Assert.assertNotNull(getCompactionStateUsedStatus(nonPendingFingerprint)); @@ -266,8 +275,9 @@ public void test_killUnreferencedCompactionState_pendingStateMarkedActiveNotDele List dateTimes = new ArrayList<>(); DateTime now = DateTimes.nowUtc(); dateTimes.add(now.plusDays(31)); // The state would be removed if it was still pending + dateTimes.add(now.plusDays(31)); // The state would be removed if it was still pending - OverlordMetadataCleanupConfig cleanupConfig = new OverlordMetadataCleanupConfig( + IndexingStateCleanupConfig cleanupConfig = new IndexingStateCleanupConfig( true, Period.parse("PT1H").toStandardDuration(), Period.parse("P7D").toStandardDuration(), @@ -307,7 +317,7 @@ public void test_killUnreferencedCompactionState_pendingStateMarkedActiveNotDele return null; }); - compactionStateStorage.markIndexingStatesAsActive(fingerprint); + compactionStateStorage.markIndexingStatesAsActive(List.of(fingerprint)); Assert.assertNotEquals(Boolean.TRUE, compactionStateStorage.isIndexingStatePending(fingerprint)); duty.run(); @@ -340,7 +350,7 @@ private static class TestKillUnreferencedIndexingState extends KillUnreferencedI private int index = -1; public TestKillUnreferencedIndexingState( - OverlordMetadataCleanupConfig config, + IndexingStateCleanupConfig config, IndexingStateStorage indexingStateStorage, List dateTimes ) diff --git a/processing/src/main/java/org/apache/druid/guice/DruidSecondaryModule.java b/processing/src/main/java/org/apache/druid/guice/DruidSecondaryModule.java index f012cdc93db2..3c8cc228d984 100644 --- a/processing/src/main/java/org/apache/druid/guice/DruidSecondaryModule.java +++ b/processing/src/main/java/org/apache/druid/guice/DruidSecondaryModule.java @@ -20,9 +20,7 @@ package org.apache.druid.guice; import com.fasterxml.jackson.databind.AnnotationIntrospector; -import com.fasterxml.jackson.databind.MapperFeature; import com.fasterxml.jackson.databind.ObjectMapper; -import com.fasterxml.jackson.databind.SerializationFeature; import com.fasterxml.jackson.databind.introspect.AnnotationIntrospectorPair; import com.fasterxml.jackson.databind.module.SimpleModule; import com.google.inject.Binder; @@ -34,11 +32,9 @@ import com.google.inject.TypeLiteral; import com.google.inject.multibindings.MapBinder; import org.apache.druid.frame.wire.FrameWireTransferable; -import org.apache.druid.guice.annotations.Deterministic; import org.apache.druid.guice.annotations.Json; import org.apache.druid.guice.annotations.JsonNonNull; import org.apache.druid.guice.annotations.Smile; -import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.query.rowsandcols.RowsAndColumns; import org.apache.druid.query.rowsandcols.concrete.FrameRowsAndColumns; @@ -168,22 +164,6 @@ public WireTransferableContext getWireTransferableContext( return new WireTransferableContext(smileMapper, concreteDeserializer, isUseLegacyFrameSerialization()); } - @Provides - @LazySingleton - @Deterministic - public ObjectMapper getSortedMapper( - Injector injector, - Map wtDeserializers - ) - { - final ObjectMapper sortedMapper = new DefaultObjectMapper(); - sortedMapper.configure(SerializationFeature.ORDER_MAP_ENTRIES_BY_KEYS, true); - sortedMapper.configure(MapperFeature.SORT_PROPERTIES_ALPHABETICALLY, true); - setupJackson(injector, sortedMapper, wtDeserializers, isUseLegacyFrameSerialization()); - return sortedMapper; - } - - public static void setupJackson( final Injector injector, final ObjectMapper mapper, diff --git a/processing/src/main/java/org/apache/druid/guice/annotations/Deterministic.java b/processing/src/main/java/org/apache/druid/guice/annotations/Deterministic.java deleted file mode 100644 index 12206e263e92..000000000000 --- a/processing/src/main/java/org/apache/druid/guice/annotations/Deterministic.java +++ /dev/null @@ -1,38 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.druid.guice.annotations; - -import com.google.inject.BindingAnnotation; - -import java.lang.annotation.ElementType; -import java.lang.annotation.Retention; -import java.lang.annotation.RetentionPolicy; -import java.lang.annotation.Target; - -/** - * ObjectMapper with deterministic serialization for fingerprinting. - * Serializes with sorted map keys and alphabetically ordered properties. - */ -@Target({ElementType.FIELD, ElementType.PARAMETER, ElementType.METHOD}) -@Retention(RetentionPolicy.RUNTIME) -@BindingAnnotation -public @interface Deterministic -{ -} 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 973b921c2ff4..63c30de68865 100644 --- a/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java +++ b/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java @@ -2729,25 +2729,23 @@ private void markIndexingStateFingerprintsAsActive(Set segments) } // Collect unique non-null indexing state fingerprints - final Set fingerprints = segments.stream() + final List fingerprints = segments.stream() .map(DataSegment::getIndexingStateFingerprint) .filter(fp -> fp != null && !fp.isEmpty()) - .collect(Collectors.toSet()); - - // Mark each fingerprint as active - for (String fingerprint : fingerprints) { - try { - int rowsUpdated = indexingStateStorage.markIndexingStatesAsActive(fingerprint); - if (rowsUpdated > 0) { - log.info("Marked indexing state fingerprint[%s] as active.", fingerprint); - } - } - catch (Exception e) { - // Log but don't fail the overall operation - the fingerprint will stay pending - // and be cleaned up by the pending grace period - log.warn(e, "Failed to mark indexing state fingerprint[%s] as active. Future segments publishes may remediate", fingerprint); + .distinct() + .collect(Collectors.toList()); + + try { + int rowsUpdated = indexingStateStorage.markIndexingStatesAsActive(fingerprints); + if (rowsUpdated > 0) { + log.info("Marked indexing states active for the following fingerprints: %s", fingerprints); } } + catch (Exception e) { + // Log but don't fail the overall operation - the fingerprint will stay pending + // and be cleaned up by the pending grace period + log.warn(e, "Failed to mark indexing states for the following fingerprints as active (Future segments publishes may remediate): %s", fingerprints); + } } /** 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 d1321f5c9779..74829cddbfbf 100644 --- a/server/src/main/java/org/apache/druid/metadata/SQLMetadataConnector.java +++ b/server/src/main/java/org/apache/druid/metadata/SQLMetadataConnector.java @@ -232,13 +232,9 @@ protected boolean isRootCausePacketTooBigException(Throwable t) /** * Checks if the root cause of the given exception is a unique constraint violation. * - * @return false by default. Specific implementations should override this method - * to correctly classify their unique constraint violation exceptions. + * @return true if t is a unique constraint violation, false otherwise */ - public boolean isUniqueConstraintViolation(Throwable t) - { - return false; - } + public abstract boolean isUniqueConstraintViolation(Throwable t); /** * Creates the given table and indexes if the table doesn't already exist. @@ -1300,15 +1296,27 @@ private void validateSegmentsTable() (tableHasColumn(segmentsTables, "schema_fingerprint") && tableHasColumn(segmentsTables, "num_rows")); - if (tableHasColumn(segmentsTables, "used_status_last_updated") && schemaPersistenceRequirementMet && tableHasColumn(segmentsTables, "indexing_state_fingerprint")) { - // do nothing - } else { + StringBuilder missingColumns = new StringBuilder(); + if (!tableHasColumn(segmentsTables, "used_status_last_updated")) { + missingColumns.append("used_status_last_updated, "); + } + if (!schemaPersistenceRequirementMet) { + missingColumns.append("schema_fingerprint, num_rows, "); + } + if (!tableHasColumn(segmentsTables, "indexing_state_fingerprint")) { + missingColumns.append("indexing_state_fingerprint, "); + } + + if (missingColumns.length() > 0) { 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, indexing_state_fingerprint] does not exist in table." + + " Reason: The following columns do not exist in the table: [%s]" + " See https://druid.apache.org/docs/latest/operations/upgrade-prep.html for more info on remediation.", - tablesConfigSupplier.get().getSegmentsTable() + tablesConfigSupplier.get().getSegmentsTable(), + missingColumns.substring(0, missingColumns.length() - 2) ); + } else { + // do nothing } } diff --git a/server/src/main/java/org/apache/druid/segment/metadata/DefaultIndexingStateFingerprintMapper.java b/server/src/main/java/org/apache/druid/segment/metadata/DefaultIndexingStateFingerprintMapper.java index e5825543f122..8b81b0e5cade 100644 --- a/server/src/main/java/org/apache/druid/segment/metadata/DefaultIndexingStateFingerprintMapper.java +++ b/server/src/main/java/org/apache/druid/segment/metadata/DefaultIndexingStateFingerprintMapper.java @@ -20,11 +20,12 @@ package org.apache.druid.segment.metadata; 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.guice.annotations.Deterministic; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.timeline.CompactionState; @@ -42,11 +43,11 @@ public class DefaultIndexingStateFingerprintMapper implements IndexingStateFinge public DefaultIndexingStateFingerprintMapper( IndexingStateCache indexingStateCache, - @Deterministic ObjectMapper deterministicMapper + ObjectMapper jsonMapper ) { this.indexingStateCache = indexingStateCache; - this.deterministicMapper = deterministicMapper; + this.deterministicMapper = createDeterministicMapper(jsonMapper); } @SuppressWarnings("UnstableApiUsage") @@ -74,4 +75,15 @@ public Optional getStateForFingerprint(String fingerprint) { return indexingStateCache.getIndexingStateByFingerprint(fingerprint); } + + /** + * Decorate the provided {@link ObjectMapper} to ensure deterministic serialization of IndexingState objects. + */ + private static ObjectMapper createDeterministicMapper(ObjectMapper baseMapper) + { + ObjectMapper sortedMapper = baseMapper.copy(); + sortedMapper.configure(SerializationFeature.ORDER_MAP_ENTRIES_BY_KEYS, true); + sortedMapper.configure(MapperFeature.SORT_PROPERTIES_ALPHABETICALLY, true); + return sortedMapper; + } } diff --git a/server/src/main/java/org/apache/druid/segment/metadata/IndexingStateStorage.java b/server/src/main/java/org/apache/druid/segment/metadata/IndexingStateStorage.java index b45d738d52e6..926553563858 100644 --- a/server/src/main/java/org/apache/druid/segment/metadata/IndexingStateStorage.java +++ b/server/src/main/java/org/apache/druid/segment/metadata/IndexingStateStorage.java @@ -80,12 +80,12 @@ void upsertIndexingState( int markIndexingStatesAsUsed(List stateFingerprints); /** - * Marks indexing states as active for a given fingerprint. + * Marks indexing states as active * - * @param stateFingerprint The fingerprint to mark as active + * @param stateFingerprints List of fingerprints to mark as active * @return Number of rows updated, or 0 if not applicable */ - int markIndexingStatesAsActive(String stateFingerprint); + int markIndexingStatesAsActive(List stateFingerprints); /** * Deletes pending indexing states older than the given timestamp. diff --git a/server/src/main/java/org/apache/druid/segment/metadata/SqlIndexingStateStorage.java b/server/src/main/java/org/apache/druid/segment/metadata/SqlIndexingStateStorage.java index 93f61afacf28..4232d6b67455 100644 --- a/server/src/main/java/org/apache/druid/segment/metadata/SqlIndexingStateStorage.java +++ b/server/src/main/java/org/apache/druid/segment/metadata/SqlIndexingStateStorage.java @@ -26,6 +26,7 @@ import org.apache.druid.error.DruidException; import org.apache.druid.error.InternalServerError; import org.apache.druid.guice.LazySingleton; +import org.apache.druid.guice.annotations.Json; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.emitter.EmittingLogger; @@ -40,6 +41,7 @@ import javax.annotation.Nonnull; import javax.annotation.Nullable; import javax.validation.constraints.NotEmpty; +import java.util.ArrayList; import java.util.List; /** @@ -65,9 +67,9 @@ public class SqlIndexingStateStorage implements IndexingStateStorage @Inject public SqlIndexingStateStorage( - @Nonnull MetadataStorageTablesConfig dbTables, - @Nonnull ObjectMapper jsonMapper, - @Nonnull SQLMetadataConnector connector + MetadataStorageTablesConfig dbTables, + @Json ObjectMapper jsonMapper, + SQLMetadataConnector connector ) { this.dbTables = dbTables; @@ -120,60 +122,11 @@ public void upsertIndexingState( break; case EXISTS_AND_UNUSED: - // Fingerprint exists but is marked as unused - update the used flag - log.info( - "Found existing indexing state in DB for fingerprint[%s] in dataSource[%s]. Marking as used.", - fingerprint, - dataSource - ); - String updateSql = StringUtils.format( - "UPDATE %s SET used = :used, used_status_last_updated = :used_status_last_updated " - + "WHERE fingerprint = :fingerprint", - dbTables.getIndexingStatesTable() - ); - handle.createStatement(updateSql) - .bind("used", true) - .bind("used_status_last_updated", now) - .bind("fingerprint", fingerprint) - .execute(); - - log.info("Updated existing indexing state for datasource[%s].", dataSource); + updateExistingUnusedState(handle, fingerprint, dataSource, now); break; case DOES_NOT_EXIST: - // Fingerprint doesn't exist - insert new state - log.info("Inserting new indexing state for fingerprint[%s] in dataSource[%s].", fingerprint, dataSource); - - String insertSql = StringUtils.format( - "INSERT INTO %s (created_date, dataSource, fingerprint, payload, used, pending, used_status_last_updated) " - + "VALUES (:created_date, :dataSource, :fingerprint, :payload, :used, :pending, :used_status_last_updated)", - dbTables.getIndexingStatesTable() - ); - - try { - handle.createStatement(insertSql) - .bind("created_date", now) - .bind("dataSource", dataSource) - .bind("fingerprint", fingerprint) - .bind("payload", jsonMapper.writeValueAsBytes(indexingState)) - .bind("used", true) - .bind("pending", true) - .bind("used_status_last_updated", now) - .execute(); - - log.info( - "Published indexing state for fingerprint[%s] to DB for datasource[%s].", - fingerprint, - dataSource - ); - } - catch (JsonProcessingException e) { - throw InternalServerError.exception( - e, - "Failed to serialize indexing state for fingerprint[%s]", - fingerprint - ); - } + insertNewState(handle, fingerprint, dataSource, indexingState, now); break; default: @@ -256,16 +209,27 @@ public int markIndexingStatesAsUsed(List stateFingerprints) } @Override - public int markIndexingStatesAsActive(String stateFingerprint) + public int markIndexingStatesAsActive(List stateFingerprints) { + if (stateFingerprints.isEmpty()) { + return 0; + } + return connector.retryWithHandle( - handle -> handle.createStatement( - StringUtils.format( - "UPDATE %s SET pending = false WHERE fingerprint = :fingerprint AND pending = true", - dbTables.getIndexingStatesTable() - )) - .bind("fingerprint", stateFingerprint) - .execute() + handle -> { + Update statement = handle.createStatement( + StringUtils.format( + "UPDATE %s SET pending = false" + + " WHERE pending = true AND fingerprint IN (%s)", + dbTables.getIndexingStatesTable(), + buildParameterizedInClause("fp", stateFingerprints.size()) + ) + ); + + bindValuesToInClause(new ArrayList<>(stateFingerprints), "fp", statement); + + return statement.execute(); + } ); } @@ -335,6 +299,81 @@ private enum FingerprintState EXISTS_AND_UNUSED } + /** + * Updates an existing unused indexing state to mark it as used. + */ + private void updateExistingUnusedState( + Handle handle, + String fingerprint, + String dataSource, + String updateTime + ) + { + log.info( + "Found existing indexing state in DB for fingerprint[%s] in dataSource[%s]. Marking as used.", + fingerprint, + dataSource + ); + + String updateSql = StringUtils.format( + "UPDATE %s SET used = :used, used_status_last_updated = :used_status_last_updated " + + "WHERE fingerprint = :fingerprint", + dbTables.getIndexingStatesTable() + ); + handle.createStatement(updateSql) + .bind("used", true) + .bind("used_status_last_updated", updateTime) + .bind("fingerprint", fingerprint) + .execute(); + + log.info("Updated existing indexing state for datasource[%s].", dataSource); + } + + /** + * Inserts a new indexing state into the database. + */ + private void insertNewState( + Handle handle, + String fingerprint, + String dataSource, + CompactionState indexingState, + String updateTime + ) + { + log.info("Inserting new indexing state for fingerprint[%s] in dataSource[%s].", fingerprint, dataSource); + + String insertSql = StringUtils.format( + "INSERT INTO %s (created_date, dataSource, fingerprint, payload, used, pending, used_status_last_updated) " + + "VALUES (:created_date, :dataSource, :fingerprint, :payload, :used, :pending, :used_status_last_updated)", + dbTables.getIndexingStatesTable() + ); + + try { + handle.createStatement(insertSql) + .bind("created_date", updateTime) + .bind("dataSource", dataSource) + .bind("fingerprint", fingerprint) + .bind("payload", jsonMapper.writeValueAsBytes(indexingState)) + .bind("used", true) + .bind("pending", true) + .bind("used_status_last_updated", updateTime) + .execute(); + + log.info( + "Published indexing state for fingerprint[%s] to DB for datasource[%s].", + fingerprint, + dataSource + ); + } + catch (JsonProcessingException e) { + throw InternalServerError.exception( + e, + "Failed to serialize indexing state for fingerprint[%s]", + fingerprint + ); + } + } + /** * Checks the state of a fingerprint in the metadata DB. * 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 504d26549ad6..3fef28a963a6 100644 --- a/server/src/test/java/org/apache/druid/metadata/SQLMetadataConnectorTest.java +++ b/server/src/test/java/org/apache/druid/metadata/SQLMetadataConnectorTest.java @@ -492,6 +492,12 @@ public String limitClause(int limit) return ""; } + @Override + public boolean isUniqueConstraintViolation(Throwable t) + { + return false; + } + @Override public String getQuoteString() { diff --git a/server/src/test/java/org/apache/druid/segment/metadata/CompactionTestUtils.java b/server/src/test/java/org/apache/druid/segment/metadata/CompactionTestUtils.java deleted file mode 100644 index c20a9015c05a..000000000000 --- a/server/src/test/java/org/apache/druid/segment/metadata/CompactionTestUtils.java +++ /dev/null @@ -1,51 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.druid.segment.metadata; - -import com.fasterxml.jackson.databind.MapperFeature; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.fasterxml.jackson.databind.SerializationFeature; -import org.apache.druid.jackson.DefaultObjectMapper; - -/** - * Test utilities for compaction-related tests. - */ -public class CompactionTestUtils -{ - /** - * Creates a deterministic ObjectMapper for fingerprinting tests. - * This mapper is configured to serialize with sorted map keys and alphabetically ordered properties, - * ensuring consistent fingerprints across test runs. - * - * @return A deterministic ObjectMapper instance - */ - public static ObjectMapper createDeterministicMapper() - { - ObjectMapper mapper = new DefaultObjectMapper(); - mapper.configure(SerializationFeature.ORDER_MAP_ENTRIES_BY_KEYS, true); - mapper.configure(MapperFeature.SORT_PROPERTIES_ALPHABETICALLY, true); - return mapper; - } - - private CompactionTestUtils() - { - // Utility class - } -} diff --git a/server/src/test/java/org/apache/druid/segment/metadata/HeapMemoryIndexingStateStorage.java b/server/src/test/java/org/apache/druid/segment/metadata/HeapMemoryIndexingStateStorage.java index 866739f9da45..e22c5430e52a 100644 --- a/server/src/test/java/org/apache/druid/segment/metadata/HeapMemoryIndexingStateStorage.java +++ b/server/src/test/java/org/apache/druid/segment/metadata/HeapMemoryIndexingStateStorage.java @@ -78,7 +78,7 @@ public int markIndexingStatesAsUsed(List stateFingerprints) } @Override - public int markIndexingStatesAsActive(String stateFingerprint) + public int markIndexingStatesAsActive(List stateFingerprints) { return 0; } diff --git a/server/src/test/java/org/apache/druid/segment/metadata/SqlIndexingStateStorageTest.java b/server/src/test/java/org/apache/druid/segment/metadata/SqlIndexingStateStorageTest.java index 7b3c529e4c9d..fca63f86131e 100644 --- a/server/src/test/java/org/apache/druid/segment/metadata/SqlIndexingStateStorageTest.java +++ b/server/src/test/java/org/apache/druid/segment/metadata/SqlIndexingStateStorageTest.java @@ -53,7 +53,6 @@ public class SqlIndexingStateStorageTest @RegisterExtension public static final TestDerbyConnector.DerbyConnectorRule5 DERBY_CONNECTOR_RULE = new TestDerbyConnector.DerbyConnectorRule5(); - private static final ObjectMapper DETERMINISTIC_MAPPER = CompactionTestUtils.createDeterministicMapper(); private final ObjectMapper jsonMapper = new DefaultObjectMapper(); @@ -72,7 +71,7 @@ public static void setUpClass() derbyConnector.createSegmentTable(); fingerprintMapper = new DefaultIndexingStateFingerprintMapper( new NoopIndexingStateCache(), - DETERMINISTIC_MAPPER + new DefaultObjectMapper() ); } @@ -125,7 +124,7 @@ public void test_upsertIndexingState_andThen_markUnreferencedIndexingStateAsUnus state1, DateTimes.nowUtc() ); - manager.markIndexingStatesAsActive(fingerprint); + manager.markIndexingStatesAsActive(List.of(fingerprint)); assertEquals(1, manager.markUnreferencedIndexingStatesAsUnused()); assertEquals(1, manager.markIndexingStatesAsUsed(List.of(fingerprint))); @@ -143,7 +142,7 @@ public void test_findReferencedIndexingStateMarkedAsUnused() state1, DateTimes.nowUtc() ); - manager.markIndexingStatesAsActive(fingerprint); + manager.markIndexingStatesAsActive(List.of(fingerprint)); manager.markUnreferencedIndexingStatesAsUnused(); assertEquals(0, manager.findReferencedIndexingStateMarkedAsUnused().size()); @@ -380,9 +379,11 @@ public void test_markIndexingStateAsUsed_withEmptyList_returnsZero() public void test_markIndexingStatesAsActive_marksPendingStateAsActive() { String fingerprint = "pending_fingerprint"; + String fingerprint2 = "other_pending_fingerprint"; CompactionState state = createTestIndexingState(); manager.upsertIndexingState("ds1", fingerprint, state, DateTimes.nowUtc()); + manager.upsertIndexingState("ds1", fingerprint2, state, DateTimes.nowUtc()); Boolean pendingBefore = derbyConnector.retryWithHandle(handle -> handle.createQuery("SELECT pending FROM " + tablesConfig.getIndexingStatesTable() + " WHERE fingerprint = :fp") @@ -392,8 +393,8 @@ public void test_markIndexingStatesAsActive_marksPendingStateAsActive() ); assertTrue(pendingBefore); - int rowsUpdated = manager.markIndexingStatesAsActive(fingerprint); - assertEquals(1, rowsUpdated); + int rowsUpdated = manager.markIndexingStatesAsActive(List.of(fingerprint, fingerprint2)); + assertEquals(2, rowsUpdated); Boolean pendingAfter = derbyConnector.retryWithHandle(handle -> handle.createQuery("SELECT pending FROM " + tablesConfig.getIndexingStatesTable() + " WHERE fingerprint = :fp") @@ -412,10 +413,10 @@ public void test_markIndexingStatesAsActive_idempotent_returnsZeroWhenAlreadyAct manager.upsertIndexingState("ds1", fingerprint, state, DateTimes.nowUtc()); - int firstUpdate = manager.markIndexingStatesAsActive(fingerprint); + int firstUpdate = manager.markIndexingStatesAsActive(List.of(fingerprint)); assertEquals(1, firstUpdate); - int secondUpdate = manager.markIndexingStatesAsActive(fingerprint); + int secondUpdate = manager.markIndexingStatesAsActive(List.of(fingerprint)); assertEquals(0, secondUpdate); Boolean pending = derbyConnector.retryWithHandle(handle -> @@ -430,7 +431,7 @@ public void test_markIndexingStatesAsActive_idempotent_returnsZeroWhenAlreadyAct @Test public void test_markIndexingStatesAsActive_nonExistentFingerprint_returnsZero() { - int rowsUpdated = manager.markIndexingStatesAsActive("does_not_exist"); + int rowsUpdated = manager.markIndexingStatesAsActive(List.of("does_not_exist")); assertEquals(0, rowsUpdated); } 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 c342e39ac106..eda549b3cd47 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 @@ -30,6 +30,7 @@ import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; import org.apache.druid.indexer.partitions.HashedPartitionsSpec; import org.apache.druid.indexer.partitions.PartitionsSpec; +import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.granularity.Granularities; @@ -39,7 +40,6 @@ 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.CompactionTestUtils; import org.apache.druid.segment.metadata.DefaultIndexingStateFingerprintMapper; import org.apache.druid.segment.metadata.HeapMemoryIndexingStateStorage; import org.apache.druid.segment.metadata.IndexingStateCache; @@ -82,7 +82,7 @@ public void setUp() indexingStateCache = new IndexingStateCache(); fingerprintMapper = new DefaultIndexingStateFingerprintMapper( indexingStateCache, - CompactionTestUtils.createDeterministicMapper() + new DefaultObjectMapper() ); } 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 966e33bfe84e..4e86abcd2468 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 @@ -30,6 +30,7 @@ import org.apache.druid.indexer.granularity.UniformGranularitySpec; import org.apache.druid.indexer.partitions.DimensionRangePartitionsSpec; import org.apache.druid.indexer.partitions.PartitionsSpec; +import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.granularity.Granularities; @@ -46,7 +47,6 @@ 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.CompactionTestUtils; import org.apache.druid.segment.metadata.DefaultIndexingStateFingerprintMapper; import org.apache.druid.segment.metadata.IndexingStateFingerprintMapper; import org.apache.druid.segment.metadata.NoopIndexingStateCache; @@ -82,7 +82,7 @@ public class NewestSegmentFirstPolicyTest { private static final int DEFAULT_NUM_SEGMENTS_PER_SHARD = 4; - private final ObjectMapper mapper = CompactionTestUtils.createDeterministicMapper(); + private final ObjectMapper mapper = new DefaultObjectMapper(); private final NewestSegmentFirstPolicy policy = new NewestSegmentFirstPolicy(null); private final IndexingStateFingerprintMapper fingerprintMapper = new DefaultIndexingStateFingerprintMapper(new NoopIndexingStateCache(), mapper); diff --git a/services/src/main/java/org/apache/druid/cli/CliOverlord.java b/services/src/main/java/org/apache/druid/cli/CliOverlord.java index 7f3e0b5907d0..6518766817ba 100644 --- a/services/src/main/java/org/apache/druid/cli/CliOverlord.java +++ b/services/src/main/java/org/apache/druid/cli/CliOverlord.java @@ -90,8 +90,8 @@ import org.apache.druid.indexing.overlord.autoscaling.SimpleWorkerProvisioningConfig; import org.apache.druid.indexing.overlord.autoscaling.SimpleWorkerProvisioningStrategy; import org.apache.druid.indexing.overlord.config.DefaultTaskConfig; +import org.apache.druid.indexing.overlord.config.IndexingStateCleanupConfig; import org.apache.druid.indexing.overlord.config.OverlordKillConfigs; -import org.apache.druid.indexing.overlord.config.OverlordMetadataCleanupConfig; import org.apache.druid.indexing.overlord.config.TaskLockConfig; import org.apache.druid.indexing.overlord.config.TaskQueueConfig; import org.apache.druid.indexing.overlord.duty.KillUnreferencedIndexingState; @@ -414,7 +414,7 @@ public TaskStorageDirTracker getTaskStorageDirTracker(WorkerConfig workerConfig, @Provides @LazySingleton - public OverlordMetadataCleanupConfig provideCompactionStateCleanupConfig(OverlordKillConfigs killConfigs) + public IndexingStateCleanupConfig provideIndexingStateCleanupConfig(OverlordKillConfigs killConfigs) { return killConfigs.indexingStates(); } From 7768a8a4b9e1f0bc2e69dc2d08afd02fff7c9bf1 Mon Sep 17 00:00:00 2001 From: capistrant Date: Fri, 16 Jan 2026 15:26:33 -0600 Subject: [PATCH 70/72] cleanup --- .../druid/metadata/IndexerSQLMetadataStorageCoordinator.java | 2 +- .../apache/druid/server/compaction/CompactionStatusTest.java | 2 -- 2 files changed, 1 insertion(+), 3 deletions(-) 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 63c30de68865..4a72e01e7c46 100644 --- a/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java +++ b/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java @@ -2732,7 +2732,7 @@ private void markIndexingStateFingerprintsAsActive(Set segments) final List fingerprints = segments.stream() .map(DataSegment::getIndexingStateFingerprint) .filter(fp -> fp != null && !fp.isEmpty()) - .distinct() + .distinct() .collect(Collectors.toList()); try { 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 eda549b3cd47..2b274e805d4c 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 @@ -603,8 +603,6 @@ public void test_evaluate_needsCompactionWhenAllSegmentsHaveUnexpectedIndexingSt .withGranularitySpec(new UserCompactionTaskGranularityConfig(Granularities.DAY, null, null)) .build(); - CompactionState expectedState = compactionConfig.toCompactionState(); - indexingStateStorage.upsertIndexingState(TestDataSource.WIKI, "wrongFingerprint", wrongState, DateTimes.nowUtc()); syncCacheFromManager(); From ec668a8119c8e0b8d6130ac140c2797b2484cace Mon Sep 17 00:00:00 2001 From: capistrant Date: Tue, 20 Jan 2026 16:10:34 -0600 Subject: [PATCH 71/72] use effective state for dimspec and indexspec for reindexing fingerprinting --- .../data/input/impl/DimensionSchema.java | 2 +- .../DataSourceCompactionConfig.java | 29 +++++++++++-------- 2 files changed, 18 insertions(+), 13 deletions(-) diff --git a/processing/src/main/java/org/apache/druid/data/input/impl/DimensionSchema.java b/processing/src/main/java/org/apache/druid/data/input/impl/DimensionSchema.java index fc8e9e654bc3..b8179e463c18 100644 --- a/processing/src/main/java/org/apache/druid/data/input/impl/DimensionSchema.java +++ b/processing/src/main/java/org/apache/druid/data/input/impl/DimensionSchema.java @@ -200,7 +200,7 @@ public DimensionHandler getDimensionHandler() /** * Computes the 'effective' {@link DimensionSchema}, allowing columns which provide mechanisms for customizing storage - * format to fill in values from the segment level {@link IndexSpec} defaults. This is useful for comparising the + * format to fill in values from the segment level {@link IndexSpec} defaults. This is useful for comparing the * operator explicitly defined schema with the 'effective' schema that was written to the segments for things like * comparing compaction state. */ diff --git a/server/src/main/java/org/apache/druid/server/coordinator/DataSourceCompactionConfig.java b/server/src/main/java/org/apache/druid/server/coordinator/DataSourceCompactionConfig.java index 727637e7bee4..39a24d3eb5e8 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/DataSourceCompactionConfig.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/DataSourceCompactionConfig.java @@ -40,6 +40,7 @@ import java.util.Arrays; import java.util.List; import java.util.Map; +import java.util.stream.Collectors; @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type", defaultImpl = InlineSchemaDataSourceCompactionConfig.class) @JsonSubTypes(value = { @@ -101,34 +102,39 @@ public interface DataSourceCompactionConfig /** * Converts this compaction config to a {@link CompactionState}. + *

    + * For IndexSpec and DimensionsSpec, we convert to their effective specs so that the fingerprint and associated state + * reflect the actual layout of the segments after compaction (with all missing defaults not included in the compaction + * config filled in). This is consistent with how {@link org.apache.druid.timeline.DataSegment#lastCompactionState } + * has been computed historically. */ default CompactionState toCompactionState() { ClientCompactionTaskQueryTuningConfig tuningConfig = ClientCompactionTaskQueryTuningConfig.from(this); - // 1. PartitionsSpec - reuse existing method PartitionsSpec partitionsSpec = CompactionStatus.findPartitionsSpecFromConfig(tuningConfig); - // 2. DimensionsSpec + IndexSpec indexSpec = tuningConfig.getIndexSpec() == null + ? IndexSpec.getDefault().getEffectiveSpec() + : tuningConfig.getIndexSpec().getEffectiveSpec(); + DimensionsSpec dimensionsSpec = null; if (getDimensionsSpec() != null && getDimensionsSpec().getDimensions() != null) { - dimensionsSpec = new DimensionsSpec(getDimensionsSpec().getDimensions()); + dimensionsSpec = DimensionsSpec.builder() + .setDimensions( + getDimensionsSpec().getDimensions() + .stream() + .map(dim -> dim.getEffectiveSchema(indexSpec)) + .collect(Collectors.toList()) + ).build(); } - // 3. Metrics List metricsSpec = getMetricsSpec() == null ? null : Arrays.asList(getMetricsSpec()); - // 4. Transform CompactionTransformSpec transformSpec = getTransformSpec(); - // 5. IndexSpec - IndexSpec indexSpec = tuningConfig.getIndexSpec() == null - ? IndexSpec.getDefault() - : tuningConfig.getIndexSpec(); - - // 6. GranularitySpec GranularitySpec granularitySpec = null; if (getGranularitySpec() != null) { UserCompactionTaskGranularityConfig userGranularityConfig = getGranularitySpec(); @@ -140,7 +146,6 @@ default CompactionState toCompactionState() ); } - // 7. Projections List projections = getProjections(); return new CompactionState( From e58266788c7158fdd12bdf21012eaa111ded4820 Mon Sep 17 00:00:00 2001 From: capistrant Date: Tue, 20 Jan 2026 19:03:36 -0600 Subject: [PATCH 72/72] Only call into running checks if there are unknown states to check --- .../server/compaction/CompactionStatus.java | 34 ++++++++++--------- 1 file changed, 18 insertions(+), 16 deletions(-) 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 837004670631..99e1eef21465 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 @@ -400,23 +400,25 @@ private CompactionStatus evaluate() } } - reasonsForCompaction.addAll( - CHECKS.stream() - .map(f -> f.apply(this)) - .filter(status -> !status.isComplete()) - .map(CompactionStatus::getReason) - .collect(Collectors.toList()) - ); + if (!unknownStateToSegments.isEmpty()) { + // Run CHECKS against any states with uknown compaction status + reasonsForCompaction.addAll( + CHECKS.stream() + .map(f -> f.apply(this)) + .filter(status -> !status.isComplete()) + .map(CompactionStatus::getReason) + .collect(Collectors.toList()) + ); - // Consider segments which have passed all checks to be compacted - // Includes segments with correct fingerprints and segments that passed all state checks - this.compactedSegments.addAll( - unknownStateToSegments - .values() - .stream() - .flatMap(List::stream) - .collect(Collectors.toList()) - ); + // Any segments left in unknownStateToSegments passed all checks and are considered compacted + this.compactedSegments.addAll( + unknownStateToSegments + .values() + .stream() + .flatMap(List::stream) + .collect(Collectors.toList()) + ); + } if (reasonsForCompaction.isEmpty()) { return COMPLETE;