From ef27a95d0ac7e973e08a92549cf84771bec988fe Mon Sep 17 00:00:00 2001 From: GWphua Date: Thu, 12 Feb 2026 17:09:50 +0800 Subject: [PATCH 01/45] Test Driven Dev CompactionTaskTest TaskLockHelperTest --- .../common/task/CompactionTaskTest.java | 86 +++++++ .../common/task/TaskLockHelperTest.java | 214 ++++++++++++++++++ 2 files changed, 300 insertions(+) create mode 100644 indexing-service/src/test/java/org/apache/druid/indexing/common/task/TaskLockHelperTest.java diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java index acb0dba5027b..4b634b5dbcdd 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java @@ -2025,6 +2025,92 @@ public void drop(DataSegment segment) .build(); } + /** + * Scenario: + * - Multiple segments exist in the same interval with non-consecutive partition numbers (0, 2, 4, 6, 7, 8, 10, 12) + * - We want to compact only segments 6,7,8 (which ARE consecutive) + * + * This test demonstrates: + * 1. findSegmentsToLock() correctly filters to only the specified segments + * 2. The filtered segments (6, 7, 8) are consecutive and would pass validation + * 3. If ALL segments were validated, it would fail because of gaps (e.g., between 0 and 2) + */ + @Test + public void testSpecificSegmentsSpecFiltersSegmentsInFindSegmentsToLock() throws Exception + { + final Interval testInterval = Intervals.of("2024-11-18T00:00:00.000Z/2024-11-25T00:00:00.000Z"); + final String version = "2024-11-17T23:49:06.823Z"; + + // Create segments with non-consecutive partition numbers in the same interval + // This simulates the real-world scenario where segments 0, 2, 4, 6, 7, 8, 10, 12 exist + final List allSegmentsInInterval = new ArrayList<>(); + + // Add segments with gaps: 0, 2, 4, 6, 7, 8, 10, 12 + allSegmentsInInterval.add(createSegmentWithPartition(testInterval, version, 0)); + allSegmentsInInterval.add(createSegmentWithPartition(testInterval, version, 2)); + allSegmentsInInterval.add(createSegmentWithPartition(testInterval, version, 4)); + + final DataSegment segment6 = createSegmentWithPartition(testInterval, version, 6); + final DataSegment segment7 = createSegmentWithPartition(testInterval, version, 7); + final DataSegment segment8 = createSegmentWithPartition(testInterval, version, 8); + allSegmentsInInterval.add(segment6); + allSegmentsInInterval.add(segment7); + allSegmentsInInterval.add(segment8); + + allSegmentsInInterval.add(createSegmentWithPartition(testInterval, version, 10)); + allSegmentsInInterval.add(createSegmentWithPartition(testInterval, version, 12)); + + // Verify that if we validate ALL segments, it would fail due to non-consecutive rootPartitionId ranges + Assert.assertThrows( + ISE.class, + () -> TaskLockHelper.verifyRootPartitionIsAdjacentAndAtomicUpdateGroupIsFull(allSegmentsInInterval) + ); + + // Use ArrayList instead of ImmutableList because SpecificSegmentsSpec constructor sorts the list in-place + final SpecificSegmentsSpec specificSegmentsSpec = new SpecificSegmentsSpec( + new ArrayList<>( + ImmutableList.of( + segment6.getId().toString(), + segment7.getId().toString(), + segment8.getId().toString() + ) + ) + ); + + // Create CompactionTask with SpecificSegmentsSpec using Builder + final CompactionTask compactionTask = new Builder(DATA_SOURCE, segmentCacheManagerFactory) + .inputSpec(specificSegmentsSpec) + .build(); + + // Create TaskActionClient that returns ALL segments in the interval + // This simulates what RetrieveUsedSegmentsAction would return + final TestTaskActionClient taskActionClient = new TestTaskActionClient(allSegmentsInInterval); + + // Verify that findSegmentsToLock() returns only the 3 specific segments, not all segments + final List segmentsToLock = compactionTask.findSegmentsToLock( + taskActionClient, + ImmutableList.of(testInterval) + ); + Assert.assertEquals(3, segmentsToLock.size()); + + Assert.assertTrue(segmentsToLock.stream().anyMatch(s -> s.getShardSpec().getPartitionNum() == 6)); + Assert.assertTrue(segmentsToLock.stream().anyMatch(s -> s.getShardSpec().getPartitionNum() == 7)); + Assert.assertTrue(segmentsToLock.stream().anyMatch(s -> s.getShardSpec().getPartitionNum() == 8)); + + TaskLockHelper.verifyRootPartitionIsAdjacentAndAtomicUpdateGroupIsFull(segmentsToLock); + } + + private DataSegment createSegmentWithPartition(Interval interval, String version, int partitionNum) + { + return DataSegment.builder() + .dataSource(DATA_SOURCE) + .interval(interval) + .version(version) + .shardSpec(new NumberedShardSpec(partitionNum, 0)) + .size(100) + .build(); + } + private static class TestTaskActionClient implements TaskActionClient { private final List segments; diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TaskLockHelperTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TaskLockHelperTest.java new file mode 100644 index 000000000000..cc53a2f7b2f6 --- /dev/null +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TaskLockHelperTest.java @@ -0,0 +1,214 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.common.task; + +import com.google.common.collect.ImmutableList; +import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.partition.NumberedOverwriteShardSpec; +import org.apache.druid.timeline.partition.PartitionIds; +import org.joda.time.Interval; +import org.junit.Test; + +import java.util.Collections; +import java.util.List; + +public class TaskLockHelperTest +{ + private static final String DATA_SOURCE = "test_datasource"; + private static final Interval TEST_INTERVAL = Intervals.of("2017-01-01/2017-01-02"); + private static final String TEST_VERSION = DateTimes.nowUtc().toString(); + + @Test(expected = ISE.class) + public void testVerifyNonConsecutiveSegmentsInInputFails() + { + // Test that non-consecutive segments within the input list fail. + // Compacting segments {0, 1, 3} should fail because root partition 2 is missing. + final List segments = ImmutableList.of( + createSegment(0, 0, 1, (short) 1, (short) 1), // rootPartitionRange [0, 1) + createSegment(1, 1, 2, (short) 1, (short) 1), // rootPartitionRange [1, 2) + createSegment(3, 3, 4, (short) 1, (short) 1) // rootPartitionRange [3, 4) + ); + + TaskLockHelper.verifyRootPartitionIsAdjacentAndAtomicUpdateGroupIsFull(segments); + } + + @Test + public void testVerifyConsecutiveSegmentsSucceedEvenIfOtherSegmentsMissing() + { + final List segments = ImmutableList.of( + createSegment(3, 3, 4, (short) 1, (short) 1), // rootPartitionRange [3, 4) + createSegment(4, 4, 5, (short) 1, (short) 1), // rootPartitionRange [4, 5) + createSegment(5, 5, 6, (short) 1, (short) 1) // rootPartitionRange [5, 6) + ); + + TaskLockHelper.verifyRootPartitionIsAdjacentAndAtomicUpdateGroupIsFull(segments); + } + + @Test + public void testVerifyConsecutiveSegmentsStillWorks() + { + final List segments = ImmutableList.of( + createSegment(0, 0, 1, (short) 1, (short) 1), + createSegment(1, 1, 2, (short) 1, (short) 1), + createSegment(2, 2, 3, (short) 1, (short) 1) + ); + + TaskLockHelper.verifyRootPartitionIsAdjacentAndAtomicUpdateGroupIsFull(segments); + } + + @Test(expected = ISE.class) + public void testVerifyLargeGapSegmentsFails() + { + final List segments = ImmutableList.of( + createSegment(0, 0, 1, (short) 1, (short) 1), + createSegment(1, 1, 2, (short) 1, (short) 1), + createSegment(10, 10, 11, (short) 1, (short) 1) + ); + + TaskLockHelper.verifyRootPartitionIsAdjacentAndAtomicUpdateGroupIsFull(segments); + } + + @Test + public void testVerifyAtomicUpdateGroupValidationStillWorks() + { + final List segments = ImmutableList.of( + createSegment(0, 0, 1, (short) 1, (short) 2), + createSegment(1, 0, 1, (short) 1, (short) 2) + ); + + TaskLockHelper.verifyRootPartitionIsAdjacentAndAtomicUpdateGroupIsFull(segments); + } + + @Test(expected = ISE.class) + public void testVerifyAtomicUpdateGroupIncompleteFails() + { + final List segments = ImmutableList.of( + createSegment(0, 0, 1, (short) 1, (short) 3), + createSegment(1, 0, 1, (short) 1, (short) 3) + ); + + // Should throw ISE because atomicUpdateGroupSize is 3 but we only have 2 segments + TaskLockHelper.verifyRootPartitionIsAdjacentAndAtomicUpdateGroupIsFull(segments); + } + + @Test(expected = ISE.class) + public void testVerifyDifferentMinorVersionsFail() + { + // Test that segments with same root partition range but different minor versions fail + final List segments = ImmutableList.of( + createSegment(0, 0, 1, (short) 1, (short) 2), + createSegment(1, 0, 1, (short) 2, (short) 2) // Different minor version + ); + + TaskLockHelper.verifyRootPartitionIsAdjacentAndAtomicUpdateGroupIsFull(segments); + } + + @Test(expected = ISE.class) + public void testVerifyDifferentAtomicUpdateGroupSizesFail() + { + // Test that segments with same root partition range but different atomicUpdateGroupSize fail + final List segments = ImmutableList.of( + createSegment(0, 0, 1, (short) 1, (short) 2), + createSegment(1, 0, 1, (short) 1, (short) 3) + ); + + TaskLockHelper.verifyRootPartitionIsAdjacentAndAtomicUpdateGroupIsFull(segments); + } + + @Test + public void testVerifyEmptySegmentsList() + { + final List segments = Collections.emptyList(); + TaskLockHelper.verifyRootPartitionIsAdjacentAndAtomicUpdateGroupIsFull(segments); + } + + @Test + public void testVerifySingleSegment() + { + final List segments = ImmutableList.of( + createSegment(0, 0, 1, (short) 1, (short) 1) + ); + TaskLockHelper.verifyRootPartitionIsAdjacentAndAtomicUpdateGroupIsFull(segments); + } + + @Test(expected = IllegalArgumentException.class) + public void testVerifyDifferentIntervalsFail() + { + final List segments = ImmutableList.of( + DataSegment.builder() + .dataSource(DATA_SOURCE) + .interval(Intervals.of("2017-01-01/2017-01-02")) + .version(TEST_VERSION) + .shardSpec(new NumberedOverwriteShardSpec( + PartitionIds.NON_ROOT_GEN_START_PARTITION_ID, + 0, + 1, + (short) 1, + (short) 1 + )) + .size(0) + .build(), + DataSegment.builder() + .dataSource(DATA_SOURCE) + .interval(Intervals.of("2017-01-02/2017-01-03")) // Different interval + .version(TEST_VERSION) + .shardSpec(new NumberedOverwriteShardSpec( + PartitionIds.NON_ROOT_GEN_START_PARTITION_ID + 1, + 1, + 2, + (short) 1, + (short) 1 + )) + .size(0) + .build() + ); + + TaskLockHelper.verifyRootPartitionIsAdjacentAndAtomicUpdateGroupIsFull(segments); + } + + /** + * Helper method to create a test segment with NumberedOverwriteShardSpec + */ + private DataSegment createSegment( + int partitionId, + int startRootPartitionId, + int endRootPartitionId, + short minorVersion, + short atomicUpdateGroupSize + ) + { + return DataSegment.builder() + .dataSource(DATA_SOURCE) + .interval(TEST_INTERVAL) + .version(TEST_VERSION) + .shardSpec(new NumberedOverwriteShardSpec( + PartitionIds.NON_ROOT_GEN_START_PARTITION_ID + partitionId, + startRootPartitionId, + endRootPartitionId, + minorVersion, + atomicUpdateGroupSize + )) + .size(0) + .build(); + } +} From 2f4cd1d009b45d2b0b4f297d05f5c6718483c117 Mon Sep 17 00:00:00 2001 From: GWphua Date: Thu, 12 Feb 2026 17:25:19 +0800 Subject: [PATCH 02/45] Minor Compaction Impl --- .../indexing/common/task/CompactionTask.java | 79 +++++++++++++++++-- .../druid/indexing/common/task/IndexTask.java | 19 ++++- .../common/task/NativeCompactionRunner.java | 70 ++++++++++++++-- .../parallel/ParallelIndexSupervisorTask.java | 19 ++++- .../batch/parallel/SinglePhaseSubTask.java | 19 ++++- .../msq/indexing/MSQCompactionRunner.java | 9 +++ 6 files changed, 197 insertions(+), 18 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java index 65bdee9a04a9..7473dd04746d 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java @@ -150,6 +150,12 @@ public class CompactionTask extends AbstractBatchIndexTask implements PendingSeg */ public static final String CTX_KEY_APPENDERATOR_TRACKING_TASK_ID = "appenderatorTrackingTaskId"; + /** + * Context key to pass specific segment IDs to sub-tasks when compacting a subset of segments. + * This ensures that sub-tasks lock only the specified segments, not all segments in the interval. + */ + public static final String CTX_KEY_SPECIFIC_SEGMENTS_TO_COMPACT = "specificSegmentsToCompact"; + private final CompactionIOConfig ioConfig; @Nullable private final DimensionsSpec dimensionsSpec; @@ -216,6 +222,15 @@ public CompactionTask( //noinspection ConstantConditions this.ioConfig = new CompactionIOConfig(SpecificSegmentsSpec.fromSegments(segments), false, null); } + // Validate that dropExisting is not used with SpecificSegmentsSpec. + // dropExisting replaces ALL segments in the interval, which contradicts minor compaction intent. + if (this.ioConfig.getInputSpec() instanceof SpecificSegmentsSpec && this.ioConfig.isDropExisting()) { + throw new IAE( + "Cannot use dropExisting with SpecificSegmentsSpec. " + + "Minor compaction (compacting specific segments) is incompatible with replacing all segments in the interval." + ); + } + this.dimensionsSpec = dimensionsSpec == null ? dimensions : dimensionsSpec; this.transformSpec = transformSpec; this.metricsSpec = metricsSpec; @@ -454,9 +469,22 @@ public boolean requireLockExistingSegments() public List findSegmentsToLock(TaskActionClient taskActionClient, List intervals) throws IOException { - return ImmutableList.copyOf( + List allSegmentsInInterval = ImmutableList.copyOf( taskActionClient.submit(new RetrieveUsedSegmentsAction(getDataSource(), intervals)) ); + + // When using SpecificSegmentsSpec, only return the specific segments requested. + // This ensures that validation only checks the segments being compacted, not all segments in the interval. + if (ioConfig.getInputSpec() instanceof SpecificSegmentsSpec) { + SpecificSegmentsSpec spec = (SpecificSegmentsSpec) ioConfig.getInputSpec(); + Set specificSegmentIds = new HashSet<>(spec.getSegments()); + return allSegmentsInInterval + .stream() + .filter(segment -> specificSegmentIds.contains(segment.getId().toString())) + .collect(Collectors.toList()); + } + + return allSegmentsInInterval; } @Override @@ -661,9 +689,16 @@ private static Iterable retrieveRelevantTimelineHolders( final List usedSegments = segmentProvider.findSegments(toolbox.getTaskActionClient()); segmentProvider.checkSegments(lockGranularityInUse, usedSegments); - final List> timelineSegments = SegmentTimeline - .forSegments(usedSegments) - .lookup(segmentProvider.interval); + + SegmentTimeline segmentTimeline = SegmentTimeline.forSegments(usedSegments); + final List> timelineSegments; + if (segmentProvider.inputSpec instanceof SpecificSegmentsSpec) { + // When compacting a subset of segments, the timeline may have incomplete partitions. + // Use lookupWithIncompletePartitions to include them. + timelineSegments = segmentTimeline.lookupWithIncompletePartitions(segmentProvider.interval); + } else { + timelineSegments = segmentTimeline.lookup(segmentProvider.interval); + } return VersionedIntervalTimeline.getAllObjects(timelineSegments); } @@ -1221,29 +1256,59 @@ static class SegmentProvider private final String dataSource; private final CompactionInputSpec inputSpec; private final Interval interval; + private List allSegmentsInInterval; SegmentProvider(String dataSource, CompactionInputSpec inputSpec) { this.dataSource = Preconditions.checkNotNull(dataSource); this.inputSpec = inputSpec; this.interval = inputSpec.findInterval(dataSource); + this.allSegmentsInInterval = new ArrayList<>(); } - List findSegments(TaskActionClient actionClient) throws IOException + List findAndCacheAllSegments(TaskActionClient actionClient) throws IOException { - return new ArrayList<>( + this.allSegmentsInInterval = new ArrayList<>( actionClient.submit( new RetrieveUsedSegmentsAction(dataSource, ImmutableList.of(interval)) ) ); + return this.allSegmentsInInterval; } + List findSegments(TaskActionClient actionClient) throws IOException + { + List allSegmentsInInterval = findAndCacheAllSegments(actionClient); + + if (inputSpec instanceof SpecificSegmentsSpec) { + SpecificSegmentsSpec spec = (SpecificSegmentsSpec) inputSpec; + Set specificSegmentIds = new HashSet<>(spec.getSegments()); + return allSegmentsInInterval + .stream() + .filter(segment -> specificSegmentIds.contains(segment.getId().toString())) + .collect(Collectors.toList()); + } + + return allSegmentsInInterval; + } + + /** + * The compaction task needs to check that in the case TIME_CHUNK lock granularity is used, all segments in the + * interval are to be checked. Since this is always called after {@code findSegments}, which caches all segments + * in the datasource to {@code allSegmentsInInterval}, we can safely use {@code allSegmentsInInterval} should + * we realize that our lock granularity is of type TIME_CHUNK. + */ void checkSegments(LockGranularity lockGranularityInUse, List latestSegments) { if (latestSegments.isEmpty()) { throw new ISE("No segments found for compaction. Please check that datasource name and interval are correct."); } - if (!inputSpec.validateSegments(lockGranularityInUse, latestSegments)) { + + final List segmentsToBeChecked = lockGranularityInUse == LockGranularity.TIME_CHUNK + ? allSegmentsInInterval + : latestSegments; + + if (!inputSpec.validateSegments(lockGranularityInUse, segmentsToBeChecked)) { throw new ISE( "Specified segments in the spec are different from the current used segments. " + "Possibly new segments would have been added or some segments have been unpublished." 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 637a3f28c431..3210151a45ef 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 @@ -117,6 +117,7 @@ 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.Objects; @@ -268,11 +269,27 @@ public boolean requireLockExistingSegments() public List findSegmentsToLock(TaskActionClient taskActionClient, List intervals) throws IOException { - return findInputSegments( + List allSegments = findInputSegments( getDataSource(), taskActionClient, intervals ); + + // Check if specific segments were passed via context (from CompactionTask) + @SuppressWarnings("unchecked") + List specificSegmentIds = (List) getContext().get( + CompactionTask.CTX_KEY_SPECIFIC_SEGMENTS_TO_COMPACT + ); + + if (specificSegmentIds != null && !specificSegmentIds.isEmpty()) { + // Filter to only the specified segments + Set segmentIdSet = new HashSet<>(specificSegmentIds); + return allSegments.stream() + .filter(segment -> segmentIdSet.contains(segment.getId().toString())) + .collect(Collectors.toList()); + } + + return allSegments; } @Override diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/NativeCompactionRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/NativeCompactionRunner.java index 89eb331df49f..89787a88e1a6 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/NativeCompactionRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/NativeCompactionRunner.java @@ -35,6 +35,7 @@ import org.apache.druid.indexing.common.task.batch.parallel.ParallelIndexIngestionSpec; import org.apache.druid.indexing.common.task.batch.parallel.ParallelIndexSupervisorTask; import org.apache.druid.indexing.input.DruidInputSource; +import org.apache.druid.indexing.input.WindowedSegmentId; import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.StringUtils; @@ -43,10 +44,13 @@ import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.server.coordinator.CompactionConfigValidationResult; import org.apache.druid.server.coordinator.duty.CompactSegments; +import org.apache.druid.timeline.SegmentId; import org.apache.druid.utils.CollectionUtils; import org.joda.time.Interval; import javax.annotation.Nullable; +import java.util.ArrayList; +import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -125,6 +129,34 @@ static List createIngestionSpecs( ).collect(Collectors.toList()); } + /** + * When using {@link SpecificSegmentsSpec}, resolves specific segment IDs that belong to the given interval + * and returns them as {@link WindowedSegmentId} objects. Returns null for interval-based compaction. + */ + @Nullable + private static List resolveSegmentIdsForInterval( + CompactionInputSpec inputSpec, + String dataSource, + Interval interval + ) + { + if (!(inputSpec instanceof SpecificSegmentsSpec)) { + return null; + } + SpecificSegmentsSpec spec = (SpecificSegmentsSpec) inputSpec; + List segmentIds = new ArrayList<>(); + for (String segmentIdStr : spec.getSegments()) { + SegmentId segmentId = SegmentId.tryParse(dataSource, segmentIdStr); + if (segmentId != null && interval.contains(segmentId.getInterval())) { + segmentIds.add(new WindowedSegmentId( + segmentIdStr, + Collections.singletonList(segmentId.getInterval()) + )); + } + } + return segmentIds.isEmpty() ? null : segmentIds; + } + private String createIndexTaskSpecId(String taskId, int i) { return StringUtils.format("%s_%d", taskId, i); @@ -140,18 +172,33 @@ private static ParallelIndexIOConfig createIoConfig( CompactionIOConfig compactionIOConfig ) { - if (!compactionIOConfig.isAllowNonAlignedInterval()) { - // Validate interval alignment. + // Resolve specific segment IDs for minor compaction if using SpecificSegmentsSpec + final List segmentIds = resolveSegmentIdsForInterval( + compactionIOConfig.getInputSpec(), + dataSchema.getDataSource(), + interval + ); + + final Interval inputInterval; + if (segmentIds != null && !segmentIds.isEmpty()) { + // When compacting specific segments, use segment IDs instead of interval + inputInterval = null; + } else { + inputInterval = interval; + } + + if (inputInterval != null && !compactionIOConfig.isAllowNonAlignedInterval()) { + // Validate interval alignment only when using interval-based input (not segment-ID mode). final Granularity segmentGranularity = dataSchema.getGranularitySpec().getSegmentGranularity(); final Interval widenedInterval = Intervals.utc( - segmentGranularity.bucketStart(interval.getStart()).getMillis(), - segmentGranularity.bucketEnd(interval.getEnd().minus(1)).getMillis() + segmentGranularity.bucketStart(inputInterval.getStart()).getMillis(), + segmentGranularity.bucketEnd(inputInterval.getEnd().minus(1)).getMillis() ); - if (!interval.equals(widenedInterval)) { + if (!inputInterval.equals(widenedInterval)) { throw new IAE( "Interval[%s] to compact is not aligned with segmentGranularity[%s]", - interval, + inputInterval, segmentGranularity ); } @@ -160,8 +207,8 @@ private static ParallelIndexIOConfig createIoConfig( return new ParallelIndexIOConfig( new DruidInputSource( dataSchema.getDataSource(), - interval, - null, + inputInterval, + segmentIds, null, null, null, @@ -304,6 +351,13 @@ Map createContextForSubtask(CompactionTask compactionTask) newContext.putIfAbsent(CompactSegments.STORE_COMPACTION_STATE_KEY, STORE_COMPACTION_STATE); // Set the priority of the compaction task. newContext.put(Tasks.PRIORITY_KEY, compactionTask.getPriority()); + + // Pass specific segment IDs to sub-tasks when using SpecificSegmentsSpec. + // This ensures sub-tasks lock only the specified segments, not all segments in the interval. + if (compactionTask.getIoConfig().getInputSpec() instanceof SpecificSegmentsSpec) { + SpecificSegmentsSpec specificSpec = (SpecificSegmentsSpec) compactionTask.getIoConfig().getInputSpec(); + newContext.put(CompactionTask.CTX_KEY_SPECIFIC_SEGMENTS_TO_COMPACT, specificSpec.getSegments()); + } return newContext; } 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 c4f37192ee11..740fc07ccd6e 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 @@ -48,6 +48,7 @@ import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.common.actions.TaskActionClient; import org.apache.druid.indexing.common.task.AbstractBatchIndexTask; +import org.apache.druid.indexing.common.task.CompactionTask; import org.apache.druid.indexing.common.task.CurrentSubTaskHolder; import org.apache.druid.indexing.common.task.IndexTask; import org.apache.druid.indexing.common.task.IndexTask.IndexIngestionSpec; @@ -456,11 +457,27 @@ public boolean isReady(TaskActionClient taskActionClient) throws Exception public List findSegmentsToLock(TaskActionClient taskActionClient, List intervals) throws IOException { - return findInputSegments( + List allSegments = findInputSegments( getDataSource(), taskActionClient, intervals ); + + // Check if specific segments were passed via context (from CompactionTask) + @SuppressWarnings("unchecked") + List specificSegmentIds = (List) getContext().get( + CompactionTask.CTX_KEY_SPECIFIC_SEGMENTS_TO_COMPACT + ); + + if (specificSegmentIds != null && !specificSegmentIds.isEmpty()) { + // Filter to only the specified segments + Set segmentIdSet = new HashSet<>(specificSegmentIds); + return allSegments.stream() + .filter(segment -> segmentIdSet.contains(segment.getId().toString())) + .collect(Collectors.toList()); + } + + return allSegments; } @Override diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseSubTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseSubTask.java index f706aafe39dd..ec47cd30fbc5 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseSubTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseSubTask.java @@ -41,6 +41,7 @@ import org.apache.druid.indexing.common.task.AbstractBatchIndexTask; import org.apache.druid.indexing.common.task.AbstractTask; import org.apache.druid.indexing.common.task.BatchAppenderators; +import org.apache.druid.indexing.common.task.CompactionTask; import org.apache.druid.indexing.common.task.IndexTask; import org.apache.druid.indexing.common.task.SegmentAllocatorForBatch; import org.apache.druid.indexing.common.task.SegmentAllocators; @@ -307,11 +308,27 @@ public boolean requireLockExistingSegments() public List findSegmentsToLock(TaskActionClient taskActionClient, List intervals) throws IOException { - return findInputSegments( + List allSegments = findInputSegments( getDataSource(), taskActionClient, intervals ); + + // Check if specific segments were passed via context (from CompactionTask) + @SuppressWarnings("unchecked") + List specificSegmentIds = (List) getContext().get( + CompactionTask.CTX_KEY_SPECIFIC_SEGMENTS_TO_COMPACT + ); + + if (specificSegmentIds != null && !specificSegmentIds.isEmpty()) { + // Filter to only the specified segments + Set segmentIdSet = new HashSet<>(specificSegmentIds); + return allSegments.stream() + .filter(segment -> segmentIdSet.contains(segment.getId().toString())) + .collect(Collectors.toList()); + } + + return allSegments; } @Override diff --git a/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java b/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java index 2310ff84072d..7cee8209c756 100644 --- a/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java +++ b/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java @@ -38,6 +38,7 @@ import org.apache.druid.indexing.common.task.CompactionRunner; import org.apache.druid.indexing.common.task.CompactionTask; import org.apache.druid.indexing.common.task.CurrentSubTaskHolder; +import org.apache.druid.indexing.common.task.SpecificSegmentsSpec; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.AllGranularity; import org.apache.druid.java.util.common.granularity.Granularities; @@ -155,6 +156,14 @@ public CompactionConfigValidationResult validateCompactionTask( Map intervalToDataSchemaMap ) { + // MSQ compaction uses REPLACE mode which locks the entire interval and replaces all segments. + // Minor compaction (SpecificSegmentsSpec) is fundamentally incompatible with this behavior. + if (compactionTask.getIoConfig().getInputSpec() instanceof SpecificSegmentsSpec) { + return CompactionConfigValidationResult.failure( + "MSQ: Minor compaction with SpecificSegmentsSpec is not supported. " + + "Use the native compaction engine for compacting specific segments." + ); + } if (intervalToDataSchemaMap.size() > 1) { // We are currently not able to handle multiple intervals in the map for multiple reasons, one of them being that // the subsequent worker ids clash -- since they are derived from MSQControllerTask ID which in turn is equal to From 558e0493300de4d30ada5c63897f208614eda9d5 Mon Sep 17 00:00:00 2001 From: GWphua Date: Thu, 12 Feb 2026 17:33:25 +0800 Subject: [PATCH 03/45] Deprecated fixes --- .../common/task/CompactionTaskTest.java | 6 ++---- .../common/task/TaskLockHelperTest.java | 18 ++++++------------ 2 files changed, 8 insertions(+), 16 deletions(-) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java index 4b634b5dbcdd..c6d458b4baa5 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java @@ -145,6 +145,7 @@ import org.apache.druid.server.security.AuthorizerMapper; import org.apache.druid.server.security.ResourceAction; import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.SegmentId; import org.apache.druid.timeline.partition.NumberedShardSpec; import org.hamcrest.CoreMatchers; import org.joda.time.Interval; @@ -2102,10 +2103,7 @@ public void testSpecificSegmentsSpecFiltersSegmentsInFindSegmentsToLock() throws private DataSegment createSegmentWithPartition(Interval interval, String version, int partitionNum) { - return DataSegment.builder() - .dataSource(DATA_SOURCE) - .interval(interval) - .version(version) + return DataSegment.builder(SegmentId.of(DATA_SOURCE, interval, version, partitionNum)) .shardSpec(new NumberedShardSpec(partitionNum, 0)) .size(100) .build(); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TaskLockHelperTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TaskLockHelperTest.java index cc53a2f7b2f6..3007a5079ef7 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TaskLockHelperTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TaskLockHelperTest.java @@ -24,6 +24,7 @@ import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.SegmentId; import org.apache.druid.timeline.partition.NumberedOverwriteShardSpec; import org.apache.druid.timeline.partition.PartitionIds; import org.joda.time.Interval; @@ -154,11 +155,10 @@ public void testVerifySingleSegment() @Test(expected = IllegalArgumentException.class) public void testVerifyDifferentIntervalsFail() { + final Interval interval1 = Intervals.of("2017-01-01/2017-01-02"); + final Interval interval2 = Intervals.of("2017-01-02/2017-01-03"); final List segments = ImmutableList.of( - DataSegment.builder() - .dataSource(DATA_SOURCE) - .interval(Intervals.of("2017-01-01/2017-01-02")) - .version(TEST_VERSION) + DataSegment.builder(SegmentId.of(DATA_SOURCE, interval1, TEST_VERSION, 0)) .shardSpec(new NumberedOverwriteShardSpec( PartitionIds.NON_ROOT_GEN_START_PARTITION_ID, 0, @@ -168,10 +168,7 @@ public void testVerifyDifferentIntervalsFail() )) .size(0) .build(), - DataSegment.builder() - .dataSource(DATA_SOURCE) - .interval(Intervals.of("2017-01-02/2017-01-03")) // Different interval - .version(TEST_VERSION) + DataSegment.builder(SegmentId.of(DATA_SOURCE, interval2, TEST_VERSION, 0)) .shardSpec(new NumberedOverwriteShardSpec( PartitionIds.NON_ROOT_GEN_START_PARTITION_ID + 1, 1, @@ -197,10 +194,7 @@ private DataSegment createSegment( short atomicUpdateGroupSize ) { - return DataSegment.builder() - .dataSource(DATA_SOURCE) - .interval(TEST_INTERVAL) - .version(TEST_VERSION) + return DataSegment.builder(SegmentId.of(DATA_SOURCE, TEST_INTERVAL, TEST_VERSION, partitionId)) .shardSpec(new NumberedOverwriteShardSpec( PartitionIds.NON_ROOT_GEN_START_PARTITION_ID + partitionId, startRootPartitionId, From 8718cce32685fb8e1c3a24b5262f9f0608aad3b4 Mon Sep 17 00:00:00 2001 From: GWphua Date: Tue, 17 Mar 2026 11:43:01 +0800 Subject: [PATCH 04/45] Check minor compaction --- docs/data-management/manual-compaction.md | 10 +- .../indexing/common/task/CompactionTask.java | 29 +-- .../common/task/SpecificSegmentsSpec.java | 3 +- .../common/task/CompactionTaskTest.java | 173 ++++++++++++++---- .../common/task/SpecificSegmentsSpecTest.java | 43 +++++ .../batch/parallel/PartialCompactionTest.java | 43 +++++ .../msq/indexing/MSQCompactionRunner.java | 2 +- .../msq/indexing/MSQCompactionRunnerTest.java | 35 ++++ 8 files changed, 280 insertions(+), 58 deletions(-) diff --git a/docs/data-management/manual-compaction.md b/docs/data-management/manual-compaction.md index e6e34dba82ef..446c42004d5c 100644 --- a/docs/data-management/manual-compaction.md +++ b/docs/data-management/manual-compaction.md @@ -57,7 +57,7 @@ You can perform manual compaction where you submit a one-time compaction task fo |`segmentGranularity`|Deprecated. Use `granularitySpec`.|No| |`tuningConfig`|[Tuning configuration](../ingestion/native-batch.md#tuningconfig) for parallel indexing. `awaitSegmentAvailabilityTimeoutMillis` value is not supported for compaction tasks. Leave this parameter at the default value, 0.|No| |`granularitySpec`|When set, the compaction task uses the specified `granularitySpec` rather than generating one from existing segments. See [Compaction `granularitySpec`](#compaction-granularity-spec) for details.|No| -|`context`|[Task context](../ingestion/tasks.md#context-parameters)|No| +|`context`|[Task context](../ingestion/tasks.md#context-parameters). For minor compaction (segments `inputSpec`), set `useConcurrentLocks: true`.|No| :::info Note: Use `granularitySpec` over `segmentGranularity` and only set one of these values. If you specify different values for these in the same compaction spec, the task fails. @@ -130,13 +130,19 @@ The compaction task has two kinds of `inputSpec`: |`type`|Task type. Set the value to `interval`.|Yes| |`interval`|Interval to compact.|Yes| -### Segments `inputSpec` +### Segments `inputSpec` (native minor compaction) |Field|Description|Required| |-----|-----------|--------| |`type`|Task type. Set the value to `segments`.|Yes| |`segments`|A list of segment IDs.|Yes| +When using the segments `inputSpec`, the task compacts only the specified segments. Segments in the same interval that are not in the spec are upgraded in place rather than compacted. This allows compacting a subset of segments while preserving others. + +- Set `useConcurrentLocks: true` in the task context. Minor compaction uses REPLACE (TIME_CHUNK) locks over the entire interval. +- `dropExisting: true` is allowed with segments `inputSpec`; the task replaces only the compacted segments. +- MSQ compaction does not support segments `inputSpec`. Use `MinorCompactionInputSpec` (type: uncompacted) for MSQ minor compaction. + ## Compaction dimensions spec |Field|Description|Required| diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java index be91213331e8..0fe1a723456a 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java @@ -229,14 +229,6 @@ public CompactionTask( //noinspection ConstantConditions this.ioConfig = new CompactionIOConfig(SpecificSegmentsSpec.fromSegments(segments), false, null); } - // Validate that dropExisting is not used with SpecificSegmentsSpec. - // dropExisting replaces ALL segments in the interval, which contradicts minor compaction intent. - if (this.ioConfig.getInputSpec() instanceof SpecificSegmentsSpec && this.ioConfig.isDropExisting()) { - throw new IAE( - "Cannot use dropExisting with SpecificSegmentsSpec. " - + "Minor compaction (compacting specific segments) is incompatible with replacing all segments in the interval." - ); - } this.dimensionsSpec = dimensionsSpec == null ? dimensions : dimensionsSpec; this.transformSpec = transformSpec; @@ -480,17 +472,6 @@ public List findSegmentsToLock(TaskActionClient taskActionClient, L taskActionClient.submit(new RetrieveUsedSegmentsAction(getDataSource(), intervals)) ); - // When using SpecificSegmentsSpec, only return the specific segments requested. - // This ensures that validation only checks the segments being compacted, not all segments in the interval. - if (ioConfig.getInputSpec() instanceof SpecificSegmentsSpec) { - SpecificSegmentsSpec spec = (SpecificSegmentsSpec) ioConfig.getInputSpec(); - Set specificSegmentIds = new HashSet<>(spec.getSegments()); - return allSegmentsInInterval - .stream() - .filter(segment -> specificSegmentIds.contains(segment.getId().toString())) - .collect(Collectors.toList()); - } - return allSegmentsInInterval; } @@ -1338,6 +1319,16 @@ static class SegmentProvider if (inputSpec instanceof MinorCompactionInputSpec) { minorCompaction = true; uncompactedSegments = Set.copyOf(((MinorCompactionInputSpec) inputSpec).getUncompactedSegments()); + } else if (inputSpec instanceof SpecificSegmentsSpec) { + minorCompaction = true; + uncompactedSegments = ((SpecificSegmentsSpec) inputSpec).getSegments() + .stream() + .map(segmentIdString -> { + SegmentId segmentId = SegmentId.tryParse(dataSource, segmentIdString); + return segmentId != null ? segmentId.toDescriptor() : null; + }) + .filter(Objects::nonNull) + .collect(Collectors.toSet()); } else { minorCompaction = false; uncompactedSegments = null; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/SpecificSegmentsSpec.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/SpecificSegmentsSpec.java index 7b26c22a4a54..c21e650ad784 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/SpecificSegmentsSpec.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/SpecificSegmentsSpec.java @@ -82,7 +82,8 @@ public boolean validateSegments(LockGranularity lockGranularityInUse, List allSegmentsInInterval = new ArrayList<>(); - - // Add segments with gaps: 0, 2, 4, 6, 7, 8, 10, 12 allSegmentsInInterval.add(createSegmentWithPartition(testInterval, version, 0)); allSegmentsInInterval.add(createSegmentWithPartition(testInterval, version, 2)); allSegmentsInInterval.add(createSegmentWithPartition(testInterval, version, 4)); - final DataSegment segment6 = createSegmentWithPartition(testInterval, version, 6); final DataSegment segment7 = createSegmentWithPartition(testInterval, version, 7); final DataSegment segment8 = createSegmentWithPartition(testInterval, version, 8); allSegmentsInInterval.add(segment6); allSegmentsInInterval.add(segment7); allSegmentsInInterval.add(segment8); - allSegmentsInInterval.add(createSegmentWithPartition(testInterval, version, 10)); allSegmentsInInterval.add(createSegmentWithPartition(testInterval, version, 12)); - // Verify that if we validate ALL segments, it would fail due to non-consecutive rootPartitionId ranges - Assert.assertThrows( - ISE.class, - () -> TaskLockHelper.verifyRootPartitionIsAdjacentAndAtomicUpdateGroupIsFull(allSegmentsInInterval) - ); - - // Use ArrayList instead of ImmutableList because SpecificSegmentsSpec constructor sorts the list in-place final SpecificSegmentsSpec specificSegmentsSpec = new SpecificSegmentsSpec( - new ArrayList<>( - ImmutableList.of( - segment6.getId().toString(), - segment7.getId().toString(), - segment8.getId().toString() - ) - ) + new ArrayList<>(ImmutableList.of( + segment6.getId().toString(), + segment7.getId().toString(), + segment8.getId().toString() + )) ); - // Create CompactionTask with SpecificSegmentsSpec using Builder final CompactionTask compactionTask = new Builder(DATA_SOURCE, segmentCacheManagerFactory) .inputSpec(specificSegmentsSpec) + .context(ImmutableMap.of(Tasks.USE_CONCURRENT_LOCKS, true)) .build(); - // Create TaskActionClient that returns ALL segments in the interval - // This simulates what RetrieveUsedSegmentsAction would return final TestTaskActionClient taskActionClient = new TestTaskActionClient(allSegmentsInInterval); - // Verify that findSegmentsToLock() returns only the 3 specific segments, not all segments + // Verify findSegmentsToLock() returns ALL segments in interval (no filtering) final List segmentsToLock = compactionTask.findSegmentsToLock( taskActionClient, ImmutableList.of(testInterval) ); - Assert.assertEquals(3, segmentsToLock.size()); + Assert.assertEquals(8, segmentsToLock.size()); + } + + /** + * When SpecificSegmentsSpec + useConcurrentLocks: true, compaction uses REPLACE mode which forces TIME_CHUNK lock. + * DEPRECATE_WHEN_SEGMENT_LOCK_REMOVED + */ + @Test + public void testSpecificSegmentsSpecUsesTimeChunkLockWithConcurrentLocks() throws Exception + { + final Interval testInterval = Intervals.of("2024-11-18T00:00:00.000Z/2024-11-25T00:00:00.000Z"); + final List segments = ImmutableList.of( + createSegmentWithPartition(testInterval, "v1", 0), + createSegmentWithPartition(testInterval, "v1", 1) + ); + final SpecificSegmentsSpec spec = SpecificSegmentsSpec.fromSegments(segments); - Assert.assertTrue(segmentsToLock.stream().anyMatch(s -> s.getShardSpec().getPartitionNum() == 6)); - Assert.assertTrue(segmentsToLock.stream().anyMatch(s -> s.getShardSpec().getPartitionNum() == 7)); - Assert.assertTrue(segmentsToLock.stream().anyMatch(s -> s.getShardSpec().getPartitionNum() == 8)); + final CompactionTask task = new Builder(DATA_SOURCE, segmentCacheManagerFactory) + .inputSpec(spec) + .context(ImmutableMap.of(Tasks.USE_CONCURRENT_LOCKS, true)) + .build(); - TaskLockHelper.verifyRootPartitionIsAdjacentAndAtomicUpdateGroupIsFull(segmentsToLock); + taskActionTestKit.getTaskLockbox().add(task); + final TaskActionClient taskActionClient = new LocalTaskActionClient( + task, + taskActionTestKit.getTaskActionToolbox() + ); + // Use a client that returns segments for RetrieveUsedSegmentsAction - wrap to inject segments + final TaskActionClient segmentAwareClient = new TaskActionClient() + { + @Override + public RetType submit(TaskAction action) throws java.io.IOException + { + if (action instanceof RetrieveUsedSegmentsAction) { + return (RetType) segments; + } + return taskActionClient.submit(action); + } + }; + task.determineLockGranularityAndTryLock(segmentAwareClient, ImmutableList.of(testInterval)); + + Assert.assertEquals(LockGranularity.TIME_CHUNK, task.getTaskLockHelper().getLockGranularityToUse()); + } + + /** + * SegmentProvider.checkSegments() with TIME_CHUNK + SpecificSegmentsSpec must allow subset: + * specified segments must exist; non-specified segments are upgraded, not validated as missing. + * DEPRECATE_WHEN_SEGMENT_LOCK_REMOVED + */ + @Test + public void testSegmentProviderCheckSegmentsAllowsSubsetForTimeChunk() throws Exception + { + final Interval testInterval = Intervals.of("2024-11-18T00:00:00.000Z/2024-11-25T00:00:00.000Z"); + final List allSegments = ImmutableList.of( + createSegmentWithPartition(testInterval, "v1", 0), + createSegmentWithPartition(testInterval, "v1", 1), + createSegmentWithPartition(testInterval, "v1", 2) + ); + final SpecificSegmentsSpec spec = new SpecificSegmentsSpec( + new ArrayList<>(ImmutableList.of(allSegments.get(0).getId().toString(), allSegments.get(1).getId().toString())) + ); + + final SegmentProvider provider = new SegmentProvider(DATA_SOURCE, spec); + final TestTaskActionClient client = new TestTaskActionClient(allSegments); + provider.findSegments(client); + + // Should not throw: specified segments (0,1) exist; segment 2 is not in spec but is in interval (will be upgraded) + provider.checkSegments(LockGranularity.TIME_CHUNK, allSegments); + } + + /** + * NativeCompactionRunner.createIoConfig passes segment IDs to DruidInputSource when using SpecificSegmentsSpec. + */ + @Test + public void testDruidInputSourceReceivesSegmentIdsForSpecificSegmentsSpec() + { + final Interval interval = Intervals.of("2024-01-01/2024-01-02"); + final List segments = ImmutableList.of( + createSegmentWithPartition(interval, "v1", 0), + createSegmentWithPartition(interval, "v1", 1) + ); + final SpecificSegmentsSpec spec = SpecificSegmentsSpec.fromSegments(segments); + + final DataSchema dataSchema = DataSchema.builder() + .withDataSource(DATA_SOURCE) + .withTimestamp(new TimestampSpec(TIMESTAMP_COLUMN, null, null)) + .withDimensions( + new DimensionsSpec( + ImmutableList.of( + new StringDimensionSchema("dim1"), + new StringDimensionSchema("dim2") + ) + ) + ) + .withGranularity( + new UniformGranularitySpec( + Granularities.DAY, + Granularities.HOUR, + false, + ImmutableList.of(interval) + ) + ) + .build(); + + final List ingestionSpecs = NativeCompactionRunner.createIngestionSpecs( + ImmutableMap.of( + new org.apache.druid.query.spec.MultipleIntervalSegmentSpec(ImmutableList.of(interval)), + dataSchema + ), + toolbox, + new CompactionIOConfig(spec, false, null), + new PartitionConfigurationManager(null), + COORDINATOR_CLIENT, + segmentCacheManagerFactory + ); + + Assert.assertEquals(1, ingestionSpecs.size()); + final InputSource inputSource = ingestionSpecs.get(0).getIOConfig().getInputSource(); + Assert.assertTrue(inputSource instanceof DruidInputSource); + final DruidInputSource druidInputSource = (DruidInputSource) inputSource; + Assert.assertNotNull(druidInputSource.getSegmentIds()); + Assert.assertEquals(2, druidInputSource.getSegmentIds().size()); } private DataSegment createSegmentWithPartition(Interval interval, String version, int partitionNum) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/SpecificSegmentsSpecTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/SpecificSegmentsSpecTest.java index 4fb1558f0ef7..5f98888e5c81 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/SpecificSegmentsSpecTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/SpecificSegmentsSpecTest.java @@ -19,12 +19,15 @@ package org.apache.druid.indexing.common.task; +import org.apache.druid.indexing.common.LockGranularity; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.partition.NumberedShardSpec; import org.joda.time.Interval; import org.junit.Assert; import org.junit.Test; +import com.google.common.collect.ImmutableList; import java.util.Collections; import java.util.List; import java.util.concurrent.ThreadLocalRandom; @@ -33,6 +36,31 @@ public class SpecificSegmentsSpecTest { + /** + * validateSegments() with TIME_CHUNK must allow subset: specified segments exist in thoseSegments; + * thoseSegments may contain additional segments (non-specified will be upgraded). + * DEPRECATE_WHEN_SEGMENT_LOCK_REMOVED + */ + @Test + public void testValidateSegmentsTimeChunkAllowsSubset() + { + final Interval interval = Intervals.of("2019-01-01/2019-01-02"); + final List allSegments = IntStream.range(0, 4) + .mapToObj(i -> newSegmentWithPartition(interval, i)) + .collect(Collectors.toList()); + // Spec has only first 2 segments (subset) + final SpecificSegmentsSpec spec = new SpecificSegmentsSpec( + ImmutableList.of( + allSegments.get(0).getId().toString(), + allSegments.get(1).getId().toString() + ).stream().collect(Collectors.toList()) + ); + // thoseSegments = all 4 segments in interval. Subset check: thoseSegments.containsAll(spec.segments) + Assert.assertTrue( + spec.validateSegments(LockGranularity.TIME_CHUNK, allSegments) + ); + } + @Test public void createTest() { @@ -63,4 +91,19 @@ private static DataSegment newSegment(Interval interval) 10 ); } + + private static DataSegment newSegmentWithPartition(Interval interval, int partitionNum) + { + return new DataSegment( + "datasource", + interval, + "version", + null, + null, + null, + new NumberedShardSpec(partitionNum, 4), + 9, + 10 + ); + } } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialCompactionTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialCompactionTest.java index 9ef316e8aaa2..459f53b2b188 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialCompactionTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialCompactionTest.java @@ -32,6 +32,7 @@ import org.apache.druid.indexing.common.task.CompactionTask; import org.apache.druid.indexing.common.task.CompactionTask.Builder; import org.apache.druid.indexing.common.task.SpecificSegmentsSpec; +import org.apache.druid.indexing.common.task.Tasks; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.segment.DataSegmentsWithSchemas; @@ -50,6 +51,7 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Comparator; +import com.google.common.collect.ImmutableMap; import java.util.List; import java.util.Map; @@ -213,6 +215,47 @@ public void testPartialCompactRangeAndDynamicPartitionedSegments() } } + /** + * End-to-end minor compaction: compact a subset of segments with useConcurrentLocks (TIME_CHUNK). + * Non-compacted segments in the interval are upgraded via MarkSegmentToUpgradeAction. + * DEPRECATE_WHEN_SEGMENT_LOCK_REMOVED + */ + @Test + public void testMinorCompactionUpgradesNonCompactedSegments() + { + DataSegmentsWithSchemas dataSegmentsWithSchemas = + runTestTask( + new HashedPartitionsSpec(null, 4, null), + TaskState.SUCCESS, + false + ); + verifySchema(dataSegmentsWithSchemas); + final Map> hashPartitionedSegments = + SegmentUtils.groupSegmentsByInterval(dataSegmentsWithSchemas.getSegments()); + + hashPartitionedSegments.values().forEach( + segmentsInInterval -> segmentsInInterval.sort( + Comparator.comparing(segment -> segment.getShardSpec().getPartitionNum()) + ) + ); + // Pick a subset to compact (e.g. first 2 of each interval) + final List segmentsToCompact = new ArrayList<>(); + for (List segmentsInInterval : hashPartitionedSegments.values()) { + segmentsToCompact.addAll(segmentsInInterval.subList(0, Math.min(2, segmentsInInterval.size()))); + } + final CompactionTask compactionTask = newCompactionTaskBuilder() + .inputSpec(SpecificSegmentsSpec.fromSegments(segmentsToCompact)) + .tuningConfig(newTuningConfig(new DynamicPartitionsSpec(20, null), 2, false)) + .context(ImmutableMap.of(Tasks.USE_CONCURRENT_LOCKS, true)) + .build(); + dataSegmentsWithSchemas = runTask(compactionTask, TaskState.SUCCESS); + verifySchema(dataSegmentsWithSchemas); + // After minor compaction: compacted subset produces new segments; non-compacted segments were upgraded + final Map> resultSegments = + SegmentUtils.groupSegmentsByInterval(dataSegmentsWithSchemas.getSegments()); + Assert.assertFalse("Compaction should produce segments", resultSegments.isEmpty()); + } + private DataSegmentsWithSchemas runTestTask( PartitionsSpec partitionsSpec, TaskState expectedTaskState, diff --git a/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java b/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java index 1d2ac5ff67ad..215c71b69fae 100644 --- a/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java +++ b/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java @@ -163,7 +163,7 @@ public CompactionConfigValidationResult validateCompactionTask( if (compactionTask.getIoConfig().getInputSpec() instanceof SpecificSegmentsSpec) { return CompactionConfigValidationResult.failure( "MSQ: Minor compaction with SpecificSegmentsSpec is not supported. " - + "Use the native compaction engine for compacting specific segments." + + "Use MinorCompactionInputSpec (type: uncompacted) for MSQ minor compaction." ); } if (intervalToDataSchemaMap.size() > 1) { diff --git a/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQCompactionRunnerTest.java b/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQCompactionRunnerTest.java index fa124f44da35..c4c619b784ec 100644 --- a/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQCompactionRunnerTest.java +++ b/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQCompactionRunnerTest.java @@ -43,6 +43,7 @@ import org.apache.druid.indexer.partitions.PartitionsSpec; import org.apache.druid.indexing.common.task.CompactionIntervalSpec; import org.apache.druid.indexing.common.task.CompactionTask; +import org.apache.druid.indexing.common.task.SpecificSegmentsSpec; import org.apache.druid.indexing.common.task.TuningConfigBuilder; import org.apache.druid.initialization.CoreInjectorBuilder; import org.apache.druid.jackson.DefaultObjectMapper; @@ -71,6 +72,8 @@ import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.NestedDataColumnSchema; import org.apache.druid.segment.column.ColumnHolder; +import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.partition.NumberedShardSpec; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.data.CompressionFactory; import org.apache.druid.segment.data.CompressionStrategy; @@ -188,6 +191,38 @@ public void testMultipleDisjointCompactionIntervalsAreInvalid() ); } + @Test + public void testRejectsSpecificSegmentsSpec() + { + final List segments = ImmutableList.of( + new DataSegment( + DATA_SOURCE, + COMPACTION_INTERVAL, + "v1", + null, + null, + null, + new NumberedShardSpec(0, 1), + 0, + 10 + ) + ); + final CompactionTask compactionTask = new CompactionTask.Builder(DATA_SOURCE, null) + .inputSpec(SpecificSegmentsSpec.fromSegments(segments)) + .compactionRunner(MSQ_COMPACTION_RUNNER) + .build(); + CompactionConfigValidationResult validationResult = MSQ_COMPACTION_RUNNER.validateCompactionTask( + compactionTask, + INTERVAL_DATASCHEMAS + ); + Assert.assertFalse(validationResult.isValid()); + Assert.assertTrue( + "Error message should mention MinorCompactionInputSpec", + validationResult.getReason().contains("MinorCompactionInputSpec") + || validationResult.getReason().contains("SpecificSegmentsSpec") + ); + } + @Test public void testHashedPartitionsSpecIsInvalid() { From 5d6518e8fbcc97b4a40a4559d287ee7688f37e6d Mon Sep 17 00:00:00 2001 From: GWphua Date: Tue, 17 Mar 2026 11:58:02 +0800 Subject: [PATCH 05/45] Minor compaction impl --- .../druid/indexing/common/task/SpecificSegmentsSpec.java | 8 ++------ 1 file changed, 2 insertions(+), 6 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/SpecificSegmentsSpec.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/SpecificSegmentsSpec.java index c21e650ad784..2bbd3e6b4564 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/SpecificSegmentsSpec.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/SpecificSegmentsSpec.java @@ -81,12 +81,8 @@ public boolean validateSegments(LockGranularity lockGranularityInUse, List segment.getId().toString()) .sorted() .collect(Collectors.toList()); - if (lockGranularityInUse == LockGranularity.TIME_CHUNK) { - // Subset check: specified segments must exist in thoseSegments; extra segments are upgraded - return thoseSegments.containsAll(segments); - } else { - return thoseSegments.containsAll(segments); - } + + return thoseSegments.containsAll(segments); } @Override From d8fa8d68a6117da6793ddee49009a6847f238c7a Mon Sep 17 00:00:00 2001 From: GWphua Date: Tue, 17 Mar 2026 12:00:55 +0800 Subject: [PATCH 06/45] Documentations --- docs/data-management/manual-compaction.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/data-management/manual-compaction.md b/docs/data-management/manual-compaction.md index 446c42004d5c..b2239d323473 100644 --- a/docs/data-management/manual-compaction.md +++ b/docs/data-management/manual-compaction.md @@ -57,7 +57,7 @@ You can perform manual compaction where you submit a one-time compaction task fo |`segmentGranularity`|Deprecated. Use `granularitySpec`.|No| |`tuningConfig`|[Tuning configuration](../ingestion/native-batch.md#tuningconfig) for parallel indexing. `awaitSegmentAvailabilityTimeoutMillis` value is not supported for compaction tasks. Leave this parameter at the default value, 0.|No| |`granularitySpec`|When set, the compaction task uses the specified `granularitySpec` rather than generating one from existing segments. See [Compaction `granularitySpec`](#compaction-granularity-spec) for details.|No| -|`context`|[Task context](../ingestion/tasks.md#context-parameters). For minor compaction (segments `inputSpec`), set `useConcurrentLocks: true`.|No| +|`context`|[Task context](../ingestion/tasks.md#context-parameters)|No| :::info Note: Use `granularitySpec` over `segmentGranularity` and only set one of these values. If you specify different values for these in the same compaction spec, the task fails. @@ -123,14 +123,14 @@ The compaction `ioConfig` requires specifying `inputSpec` as follows: The compaction task has two kinds of `inputSpec`: -### Interval `inputSpec` +### Interval `inputSpec` (Major Compaction) |Field|Description|Required| |-----|-----------|--------| |`type`|Task type. Set the value to `interval`.|Yes| |`interval`|Interval to compact.|Yes| -### Segments `inputSpec` (native minor compaction) +### Segments `inputSpec` (Native Minor Compaction) |Field|Description|Required| |-----|-----------|--------| From fe7661bd8d65db39307dcb3947f2db9243d74336 Mon Sep 17 00:00:00 2001 From: GWphua Date: Tue, 17 Mar 2026 14:48:11 +0800 Subject: [PATCH 07/45] Tidy up segmentProvider --- .../indexing/common/task/CompactionTask.java | 70 +++++-------------- .../common/task/SpecificSegmentsSpec.java | 22 ++++++ .../common/task/CompactionTaskTest.java | 24 ++++--- 3 files changed, 55 insertions(+), 61 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java index 0fe1a723456a..f3d376255fe2 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java @@ -229,7 +229,6 @@ public CompactionTask( //noinspection ConstantConditions this.ioConfig = new CompactionIOConfig(SpecificSegmentsSpec.fromSegments(segments), false, null); } - this.dimensionsSpec = dimensionsSpec == null ? dimensions : dimensionsSpec; this.transformSpec = transformSpec; this.metricsSpec = metricsSpec; @@ -468,11 +467,9 @@ public boolean requireLockExistingSegments() public List findSegmentsToLock(TaskActionClient taskActionClient, List intervals) throws IOException { - List allSegmentsInInterval = ImmutableList.copyOf( + return ImmutableList.copyOf( taskActionClient.submit(new RetrieveUsedSegmentsAction(getDataSource(), intervals)) ); - - return allSegmentsInInterval; } @Override @@ -743,16 +740,9 @@ private static Iterable retrieveRelevantTimelineHolders( { final List usedSegments = segmentProvider.findSegments(toolbox.getTaskActionClient()); segmentProvider.checkSegments(lockGranularityInUse, usedSegments); - - SegmentTimeline segmentTimeline = SegmentTimeline.forSegments(usedSegments); - final List> timelineSegments; - if (segmentProvider.inputSpec instanceof SpecificSegmentsSpec) { - // When compacting a subset of segments, the timeline may have incomplete partitions. - // Use lookupWithIncompletePartitions to include them. - timelineSegments = segmentTimeline.lookupWithIncompletePartitions(segmentProvider.interval); - } else { - timelineSegments = segmentTimeline.lookup(segmentProvider.interval); - } + final List> timelineSegments = SegmentTimeline + .forSegments(usedSegments) + .lookup(segmentProvider.interval); return VersionedIntervalTimeline.getAllObjects(timelineSegments); } @@ -1300,13 +1290,17 @@ static Granularity compareWithCurrent(Granularity queryGranularity, Granularity } } + /** + * Provides segment discovery and validation for compaction. + * For minor compaction (SpecificSegmentsSpec, MinorCompactionInputSpec), finds all segments + * in the interval and partitions them into compact vs upgrade via {@link #shouldUpgradeSegment}. + */ @VisibleForTesting static class SegmentProvider { private final String dataSource; private final CompactionInputSpec inputSpec; private final Interval interval; - private List allSegmentsInInterval; private final boolean minorCompaction; private final Set uncompactedSegments; @@ -1321,14 +1315,7 @@ static class SegmentProvider uncompactedSegments = Set.copyOf(((MinorCompactionInputSpec) inputSpec).getUncompactedSegments()); } else if (inputSpec instanceof SpecificSegmentsSpec) { minorCompaction = true; - uncompactedSegments = ((SpecificSegmentsSpec) inputSpec).getSegments() - .stream() - .map(segmentIdString -> { - SegmentId segmentId = SegmentId.tryParse(dataSource, segmentIdString); - return segmentId != null ? segmentId.toDescriptor() : null; - }) - .filter(Objects::nonNull) - .collect(Collectors.toSet()); + uncompactedSegments = Set.copyOf(((SpecificSegmentsSpec) inputSpec).getSegmentDescriptors(dataSource)); } else { minorCompaction = false; uncompactedSegments = null; @@ -1344,37 +1331,22 @@ private boolean shouldUpgradeSegment(DataSegment s) } } - List findAndCacheAllSegments(TaskActionClient actionClient) throws IOException + /** + * Returns all segments in the interval. For minor compaction (SpecificSegmentsSpec, MinorCompactionInputSpec), + * partitioning into compact vs upgrade is done via {@link #shouldUpgradeSegment}. + */ + List findSegments(TaskActionClient actionClient) throws IOException { - this.allSegmentsInInterval = new ArrayList<>( + return new ArrayList<>( actionClient.submit( new RetrieveUsedSegmentsAction(dataSource, ImmutableList.of(interval)) ) ); - return this.allSegmentsInInterval; - } - - List findSegments(TaskActionClient actionClient) throws IOException - { - List allSegmentsInInterval = findAndCacheAllSegments(actionClient); - - if (inputSpec instanceof SpecificSegmentsSpec) { - SpecificSegmentsSpec spec = (SpecificSegmentsSpec) inputSpec; - Set specificSegmentIds = new HashSet<>(spec.getSegments()); - return allSegmentsInInterval - .stream() - .filter(segment -> specificSegmentIds.contains(segment.getId().toString())) - .collect(Collectors.toList()); - } - - return allSegmentsInInterval; } /** - * The compaction task needs to check that in the case TIME_CHUNK lock granularity is used, all segments in the - * interval are to be checked. Since this is always called after {@code findSegments}, which caches all segments - * in the datasource to {@code allSegmentsInInterval}, we can safely use {@code allSegmentsInInterval} should - * we realize that our lock granularity is of type TIME_CHUNK. + * Validates that the specified segments exist in the latest segments. For TIME_CHUNK and minor compaction, + * the caller passes the full interval set from {@code findSegments}. */ void checkSegments(LockGranularity lockGranularityInUse, List latestSegments) { @@ -1382,11 +1354,7 @@ void checkSegments(LockGranularity lockGranularityInUse, List lates throw new ISE("No segments found for compaction. Please check that datasource name and interval are correct."); } - final List segmentsToBeChecked = lockGranularityInUse == LockGranularity.TIME_CHUNK - ? allSegmentsInInterval - : latestSegments; - - if (!inputSpec.validateSegments(lockGranularityInUse, segmentsToBeChecked)) { + if (!inputSpec.validateSegments(lockGranularityInUse, latestSegments)) { throw new ISE( "Specified segments in the spec are different from the current used segments. " + "Possibly new segments would have been added or some segments have been unpublished." diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/SpecificSegmentsSpec.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/SpecificSegmentsSpec.java index 2bbd3e6b4564..ed0dd4a2fff1 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/SpecificSegmentsSpec.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/SpecificSegmentsSpec.java @@ -24,15 +24,23 @@ import com.google.common.base.Preconditions; import org.apache.druid.indexing.common.LockGranularity; import org.apache.druid.java.util.common.JodaUtils; +import org.apache.druid.query.SegmentDescriptor; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.SegmentId; import org.joda.time.Interval; +import java.util.ArrayList; import java.util.Collections; import java.util.List; import java.util.Objects; +import java.util.Set; import java.util.stream.Collectors; +/** + * Compaction input spec that targets specific segments by ID. + * Used for native engine-based minor compaction: the timeline is built from all segments in the interval, + * and segments not in this spec are upgraded (no physical re-compaction). + */ public class SpecificSegmentsSpec implements CompactionInputSpec { public static final String TYPE = "segments"; @@ -50,6 +58,7 @@ public static SpecificSegmentsSpec fromSegments(List segments) @JsonCreator public SpecificSegmentsSpec(@JsonProperty("segments") List segments) { + Preconditions.checkArgument(segments != null && !segments.isEmpty(), "Segments must not be null or empty"); this.segments = segments; // Sort segments to use in validateSegments. Collections.sort(this.segments); @@ -61,6 +70,19 @@ public List getSegments() return segments; } + /** + * Parses segment IDs to descriptors for minor compaction (compact vs upgrade partitioning). + * Invalid IDs are filtered out. + */ + List getSegmentDescriptors(String dataSource) + { + return segments.stream() + .map(id -> SegmentId.tryParse(dataSource, id)) + .filter(Objects::nonNull) + .map(SegmentId::toDescriptor) + .collect(Collectors.toList()); + } + @Override public Interval findInterval(String dataSource) { diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java index fd81d4467ea3..05217bece6fc 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java @@ -63,6 +63,7 @@ import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.common.TestUtils; import org.apache.druid.indexing.common.actions.LocalTaskActionClient; +import org.apache.druid.indexing.common.actions.MarkSegmentToUpgradeAction; import org.apache.druid.indexing.common.actions.RetrieveUsedSegmentsAction; import org.apache.druid.indexing.common.actions.TaskAction; import org.apache.druid.indexing.common.actions.TaskActionClient; @@ -1204,14 +1205,15 @@ public void testCreateIngestionSchemaWithDifferentSegmentSet() throws IOExceptio expectedException.expect(CoreMatchers.instanceOf(IllegalStateException.class)); expectedException.expectMessage(CoreMatchers.containsString("are different from the current used segments")); - final List segments = new ArrayList<>(SEGMENTS); - Collections.sort(segments); - // Remove one segment in the middle - segments.remove(segments.size() / 2); + // Spec includes a segment ID that does not exist in metadata - validation should fail + final List segmentIds = new ArrayList<>( + SEGMENTS.stream().map(s -> s.getId().toString()).collect(Collectors.toList()) + ); + segmentIds.add(DATA_SOURCE + "_2020-01-01T00:00:00.000Z_2020-02-01T00:00:00.000Z_x_0"); final Map inputSchemas = CompactionTask.createInputDataSchemas( toolbox, LockGranularity.TIME_CHUNK, - new SegmentProvider(DATA_SOURCE, SpecificSegmentsSpec.fromSegments(segments)), + new SegmentProvider(DATA_SOURCE, new SpecificSegmentsSpec(segmentIds)), null, null, null, @@ -1239,11 +1241,10 @@ public void testMissingMetadata() throws IOException final TestIndexIO indexIO = (TestIndexIO) toolbox.getIndexIO(); indexIO.removeMetadata(Iterables.getFirst(indexIO.getQueryableIndexMap().keySet(), null)); - final List segments = new ArrayList<>(SEGMENTS); final Map inputSchemas = CompactionTask.createInputDataSchemas( toolbox, LockGranularity.TIME_CHUNK, - new SegmentProvider(DATA_SOURCE, SpecificSegmentsSpec.fromSegments(segments)), + new SegmentProvider(DATA_SOURCE, new CompactionIntervalSpec(COMPACTION_INTERVAL, null)), null, null, null, @@ -2231,10 +2232,13 @@ private static class TestTaskActionClient implements TaskActionClient @Override public RetType submit(TaskAction taskAction) { - if (!(taskAction instanceof RetrieveUsedSegmentsAction)) { - throw new ISE("action[%s] is not supported", taskAction); + if (taskAction instanceof RetrieveUsedSegmentsAction) { + return (RetType) segments; + } + if (taskAction instanceof MarkSegmentToUpgradeAction) { + return (RetType) Integer.valueOf(0); } - return (RetType) segments; + throw new ISE("action[%s] is not supported", taskAction); } } From f58a242ec920d22356ad47bc51a65e1f8e976601 Mon Sep 17 00:00:00 2001 From: GWphua Date: Tue, 17 Mar 2026 15:14:54 +0800 Subject: [PATCH 08/45] We no longer need the context key --- .../indexing/common/task/CompactionTask.java | 6 -- .../druid/indexing/common/task/IndexTask.java | 19 +---- .../common/task/NativeCompactionRunner.java | 9 +-- .../common/task/SpecificSegmentsSpec.java | 2 - .../parallel/ParallelIndexSupervisorTask.java | 19 +---- .../batch/parallel/SinglePhaseSubTask.java | 19 +---- .../common/task/CompactionTaskTest.java | 79 +++++++++++++++++++ 7 files changed, 84 insertions(+), 69 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java index f3d376255fe2..a9de6b15c0a0 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java @@ -157,12 +157,6 @@ public class CompactionTask extends AbstractBatchIndexTask implements PendingSeg */ public static final String CTX_KEY_APPENDERATOR_TRACKING_TASK_ID = "appenderatorTrackingTaskId"; - /** - * Context key to pass specific segment IDs to sub-tasks when compacting a subset of segments. - * This ensures that sub-tasks lock only the specified segments, not all segments in the interval. - */ - public static final String CTX_KEY_SPECIFIC_SEGMENTS_TO_COMPACT = "specificSegmentsToCompact"; - private final CompactionIOConfig ioConfig; @Nullable private final DimensionsSpec dimensionsSpec; 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 3210151a45ef..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 @@ -117,7 +117,6 @@ 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.Objects; @@ -269,27 +268,11 @@ public boolean requireLockExistingSegments() public List findSegmentsToLock(TaskActionClient taskActionClient, List intervals) throws IOException { - List allSegments = findInputSegments( + return findInputSegments( getDataSource(), taskActionClient, intervals ); - - // Check if specific segments were passed via context (from CompactionTask) - @SuppressWarnings("unchecked") - List specificSegmentIds = (List) getContext().get( - CompactionTask.CTX_KEY_SPECIFIC_SEGMENTS_TO_COMPACT - ); - - if (specificSegmentIds != null && !specificSegmentIds.isEmpty()) { - // Filter to only the specified segments - Set segmentIdSet = new HashSet<>(specificSegmentIds); - return allSegments.stream() - .filter(segment -> segmentIdSet.contains(segment.getId().toString())) - .collect(Collectors.toList()); - } - - return allSegments; } @Override diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/NativeCompactionRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/NativeCompactionRunner.java index 96cad82d3934..1b22c7a0cb4c 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/NativeCompactionRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/NativeCompactionRunner.java @@ -365,13 +365,8 @@ Map createContextForSubtask(CompactionTask compactionTask) newContext.putIfAbsent(CompactSegments.STORE_COMPACTION_STATE_KEY, STORE_COMPACTION_STATE); // Set the priority of the compaction task. newContext.put(Tasks.PRIORITY_KEY, compactionTask.getPriority()); - - // Pass specific segment IDs to sub-tasks when using SpecificSegmentsSpec. - // This ensures sub-tasks lock only the specified segments, not all segments in the interval. - if (compactionTask.getIoConfig().getInputSpec() instanceof SpecificSegmentsSpec) { - SpecificSegmentsSpec specificSpec = (SpecificSegmentsSpec) compactionTask.getIoConfig().getInputSpec(); - newContext.put(CompactionTask.CTX_KEY_SPECIFIC_SEGMENTS_TO_COMPACT, specificSpec.getSegments()); - } + // Native compaction subtasks always use TIME_CHUNK (REPLACE mode). + newContext.put(Tasks.FORCE_TIME_CHUNK_LOCK_KEY, true); return newContext; } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/SpecificSegmentsSpec.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/SpecificSegmentsSpec.java index ed0dd4a2fff1..dcf22c67ee5e 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/SpecificSegmentsSpec.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/SpecificSegmentsSpec.java @@ -29,11 +29,9 @@ import org.apache.druid.timeline.SegmentId; import org.joda.time.Interval; -import java.util.ArrayList; import java.util.Collections; import java.util.List; import java.util.Objects; -import java.util.Set; import java.util.stream.Collectors; /** 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 740fc07ccd6e..c4f37192ee11 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 @@ -48,7 +48,6 @@ import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.common.actions.TaskActionClient; import org.apache.druid.indexing.common.task.AbstractBatchIndexTask; -import org.apache.druid.indexing.common.task.CompactionTask; import org.apache.druid.indexing.common.task.CurrentSubTaskHolder; import org.apache.druid.indexing.common.task.IndexTask; import org.apache.druid.indexing.common.task.IndexTask.IndexIngestionSpec; @@ -457,27 +456,11 @@ public boolean isReady(TaskActionClient taskActionClient) throws Exception public List findSegmentsToLock(TaskActionClient taskActionClient, List intervals) throws IOException { - List allSegments = findInputSegments( + return findInputSegments( getDataSource(), taskActionClient, intervals ); - - // Check if specific segments were passed via context (from CompactionTask) - @SuppressWarnings("unchecked") - List specificSegmentIds = (List) getContext().get( - CompactionTask.CTX_KEY_SPECIFIC_SEGMENTS_TO_COMPACT - ); - - if (specificSegmentIds != null && !specificSegmentIds.isEmpty()) { - // Filter to only the specified segments - Set segmentIdSet = new HashSet<>(specificSegmentIds); - return allSegments.stream() - .filter(segment -> segmentIdSet.contains(segment.getId().toString())) - .collect(Collectors.toList()); - } - - return allSegments; } @Override diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseSubTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseSubTask.java index ec47cd30fbc5..f706aafe39dd 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseSubTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseSubTask.java @@ -41,7 +41,6 @@ import org.apache.druid.indexing.common.task.AbstractBatchIndexTask; import org.apache.druid.indexing.common.task.AbstractTask; import org.apache.druid.indexing.common.task.BatchAppenderators; -import org.apache.druid.indexing.common.task.CompactionTask; import org.apache.druid.indexing.common.task.IndexTask; import org.apache.druid.indexing.common.task.SegmentAllocatorForBatch; import org.apache.druid.indexing.common.task.SegmentAllocators; @@ -308,27 +307,11 @@ public boolean requireLockExistingSegments() public List findSegmentsToLock(TaskActionClient taskActionClient, List intervals) throws IOException { - List allSegments = findInputSegments( + return findInputSegments( getDataSource(), taskActionClient, intervals ); - - // Check if specific segments were passed via context (from CompactionTask) - @SuppressWarnings("unchecked") - List specificSegmentIds = (List) getContext().get( - CompactionTask.CTX_KEY_SPECIFIC_SEGMENTS_TO_COMPACT - ); - - if (specificSegmentIds != null && !specificSegmentIds.isEmpty()) { - // Filter to only the specified segments - Set segmentIdSet = new HashSet<>(specificSegmentIds); - return allSegments.stream() - .filter(segment -> segmentIdSet.contains(segment.getId().toString())) - .collect(Collectors.toList()); - } - - return allSegments; } @Override diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java index 05217bece6fc..f6667e671f1a 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java @@ -77,6 +77,7 @@ import org.apache.druid.indexing.common.task.NativeCompactionRunner.PartitionConfigurationManager; import org.apache.druid.indexing.common.task.batch.parallel.ParallelIndexIOConfig; import org.apache.druid.indexing.common.task.batch.parallel.ParallelIndexIngestionSpec; +import org.apache.druid.indexing.common.task.batch.parallel.ParallelIndexSupervisorTask; import org.apache.druid.indexing.common.task.batch.parallel.ParallelIndexTuningConfig; import org.apache.druid.indexing.input.DruidInputSource; import org.apache.druid.jackson.DefaultObjectMapper; @@ -2131,6 +2132,84 @@ public RetType submit(TaskAction action) throws java.io.IOExc Assert.assertEquals(LockGranularity.TIME_CHUNK, task.getTaskLockHelper().getLockGranularityToUse()); } + /** + * Native compaction subtasks must use TIME_CHUNK lock (enforced via context from createContextForSubtask). + */ + @Test + public void testNativeCompactionSubtaskUsesTimeChunkLock() throws Exception + { + final Interval testInterval = Intervals.of("2024-11-18T00:00:00.000Z/2024-11-25T00:00:00.000Z"); + final List segments = ImmutableList.of( + createSegmentWithPartition(testInterval, "v1", 0), + createSegmentWithPartition(testInterval, "v1", 1) + ); + final SpecificSegmentsSpec spec = SpecificSegmentsSpec.fromSegments(segments); + + final CompactionTask compactionTask = new Builder(DATA_SOURCE, segmentCacheManagerFactory) + .inputSpec(spec) + .context(ImmutableMap.of(Tasks.USE_CONCURRENT_LOCKS, true)) + .build(); + + final NativeCompactionRunner runner = new NativeCompactionRunner(segmentCacheManagerFactory); + final Map subtaskContext = runner.createContextForSubtask(compactionTask); + + final DataSchema dataSchema = DataSchema.builder() + .withDataSource(DATA_SOURCE) + .withTimestamp(new TimestampSpec(TIMESTAMP_COLUMN, null, null)) + .withDimensions( + new DimensionsSpec(ImmutableList.of(new StringDimensionSchema("dim1"), new StringDimensionSchema("dim2"))) + ) + .withGranularity( + new UniformGranularitySpec(Granularities.DAY, Granularities.HOUR, false, ImmutableList.of(testInterval)) + ) + .build(); + + final List ingestionSpecs = NativeCompactionRunner.createIngestionSpecs( + ImmutableMap.of( + new org.apache.druid.query.spec.MultipleIntervalSegmentSpec(ImmutableList.of(testInterval)), + dataSchema + ), + toolbox, + new CompactionIOConfig(spec, false, null), + new PartitionConfigurationManager(null), + COORDINATOR_CLIENT, + segmentCacheManagerFactory + ); + + final ParallelIndexSupervisorTask subtask = new ParallelIndexSupervisorTask( + "test_subtask", + "test_group", + null, + ingestionSpecs.get(0), + "base_0", + subtaskContext, + true + ); + + taskActionTestKit.getTaskLockbox().add(subtask); + final TaskActionClient taskActionClient = new LocalTaskActionClient( + subtask, + taskActionTestKit.getTaskActionToolbox() + ); + final TaskActionClient segmentAwareClient = new TaskActionClient() + { + @Override + public RetType submit(TaskAction action) throws java.io.IOException + { + if (action instanceof RetrieveUsedSegmentsAction) { + return (RetType) segments; + } + return taskActionClient.submit(action); + } + }; + subtask.determineLockGranularityAndTryLock(segmentAwareClient, ImmutableList.of(testInterval)); + + Assert.assertEquals( + LockGranularity.TIME_CHUNK, + subtask.getTaskLockHelper().getLockGranularityToUse() + ); + } + /** * SegmentProvider.checkSegments() with TIME_CHUNK + SpecificSegmentsSpec must allow subset: * specified segments must exist; non-specified segments are upgraded, not validated as missing. From f7867dd5a6abb2832292307c40727016bcbcd7e8 Mon Sep 17 00:00:00 2001 From: GWphua Date: Tue, 17 Mar 2026 15:46:34 +0800 Subject: [PATCH 09/45] Bug fixes --- .../common/task/SpecificSegmentsSpec.java | 8 ++-- .../common/task/CompactionTaskTest.java | 40 ++++--------------- .../common/task/SpecificSegmentsSpecTest.java | 12 ++---- .../batch/parallel/PartialCompactionTest.java | 7 ++-- 4 files changed, 18 insertions(+), 49 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/SpecificSegmentsSpec.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/SpecificSegmentsSpec.java index dcf22c67ee5e..13097af3a702 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/SpecificSegmentsSpec.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/SpecificSegmentsSpec.java @@ -84,13 +84,13 @@ List getSegmentDescriptors(String dataSource) @Override public Interval findInterval(String dataSource) { - final List segmentIds = segments + final List intervals = segments .stream() .map(segment -> SegmentId.tryParse(dataSource, segment)) + .filter(Objects::nonNull) + .map(SegmentId::getInterval) .collect(Collectors.toList()); - return JodaUtils.umbrellaInterval( - segmentIds.stream().map(SegmentId::getInterval).collect(Collectors.toList()) - ); + return JodaUtils.umbrellaInterval(intervals); } @Override diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java index f6667e671f1a..2f32b104dd97 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java @@ -41,9 +41,9 @@ import org.apache.druid.client.coordinator.NoopCoordinatorClient; import org.apache.druid.client.indexing.ClientCompactionTaskGranularitySpec; import org.apache.druid.common.guava.SettableSupplier; +import org.apache.druid.data.input.InputSource; import org.apache.druid.data.input.impl.AggregateProjectionSpec; import org.apache.druid.data.input.impl.DimensionSchema; -import org.apache.druid.data.input.InputSource; import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.DoubleDimensionSchema; import org.apache.druid.data.input.impl.FloatDimensionSchema; @@ -72,7 +72,6 @@ import org.apache.druid.indexing.common.config.TaskConfigBuilder; import org.apache.druid.indexing.common.task.CompactionTask.Builder; import org.apache.druid.indexing.common.task.CompactionTask.SegmentProvider; -import org.apache.druid.indexing.common.task.Tasks; import org.apache.druid.indexing.common.task.IndexTask.IndexTuningConfig; import org.apache.druid.indexing.common.task.NativeCompactionRunner.PartitionConfigurationManager; import org.apache.druid.indexing.common.task.batch.parallel.ParallelIndexIOConfig; @@ -102,6 +101,7 @@ import org.apache.druid.query.aggregation.firstlast.last.DoubleLastAggregatorFactory; import org.apache.druid.query.expression.TestExprMacroTable; import org.apache.druid.query.filter.SelectorDimFilter; +import org.apache.druid.query.spec.MultipleIntervalSegmentSpec; import org.apache.druid.query.spec.QuerySegmentSpec; import org.apache.druid.segment.AutoTypeColumnSchema; import org.apache.druid.segment.IndexIO; @@ -1207,9 +1207,7 @@ public void testCreateIngestionSchemaWithDifferentSegmentSet() throws IOExceptio expectedException.expectMessage(CoreMatchers.containsString("are different from the current used segments")); // Spec includes a segment ID that does not exist in metadata - validation should fail - final List segmentIds = new ArrayList<>( - SEGMENTS.stream().map(s -> s.getId().toString()).collect(Collectors.toList()) - ); + final List segmentIds = SEGMENTS.stream().map(s -> s.getId().toString()).collect(Collectors.toList()); segmentIds.add(DATA_SOURCE + "_2020-01-01T00:00:00.000Z_2020-02-01T00:00:00.000Z_x_0"); final Map inputSchemas = CompactionTask.createInputDataSchemas( toolbox, @@ -2040,15 +2038,6 @@ public void drop(DataSegment segment) .build(); } - /** - * Scenario: - * - Multiple segments exist in the same interval with non-consecutive partition numbers (0, 2, 4, 6, 7, 8, 10, 12) - * - We want to compact only segments 6,7,8 (which ARE consecutive) - * - * Per minor compaction PRD: findSegmentsToLock() must return ALL segments in the interval (no filtering) - * so that we can lock the entire interval with TIME_CHUNK/REPLACE lock. Non-compacted segments are upgraded. - * DEPRECATE_WHEN_SEGMENT_LOCK_REMOVED - */ @Test public void testFindSegmentsToLockReturnsAllSegmentsForSpecificSegmentsSpec() throws Exception { @@ -2091,10 +2080,6 @@ public void testFindSegmentsToLockReturnsAllSegmentsForSpecificSegmentsSpec() th Assert.assertEquals(8, segmentsToLock.size()); } - /** - * When SpecificSegmentsSpec + useConcurrentLocks: true, compaction uses REPLACE mode which forces TIME_CHUNK lock. - * DEPRECATE_WHEN_SEGMENT_LOCK_REMOVED - */ @Test public void testSpecificSegmentsSpecUsesTimeChunkLockWithConcurrentLocks() throws Exception { @@ -2119,7 +2104,7 @@ public void testSpecificSegmentsSpecUsesTimeChunkLockWithConcurrentLocks() throw final TaskActionClient segmentAwareClient = new TaskActionClient() { @Override - public RetType submit(TaskAction action) throws java.io.IOException + public RetType submit(TaskAction action) throws IOException { if (action instanceof RetrieveUsedSegmentsAction) { return (RetType) segments; @@ -2132,9 +2117,6 @@ public RetType submit(TaskAction action) throws java.io.IOExc Assert.assertEquals(LockGranularity.TIME_CHUNK, task.getTaskLockHelper().getLockGranularityToUse()); } - /** - * Native compaction subtasks must use TIME_CHUNK lock (enforced via context from createContextForSubtask). - */ @Test public void testNativeCompactionSubtaskUsesTimeChunkLock() throws Exception { @@ -2166,7 +2148,7 @@ public void testNativeCompactionSubtaskUsesTimeChunkLock() throws Exception final List ingestionSpecs = NativeCompactionRunner.createIngestionSpecs( ImmutableMap.of( - new org.apache.druid.query.spec.MultipleIntervalSegmentSpec(ImmutableList.of(testInterval)), + new MultipleIntervalSegmentSpec(ImmutableList.of(testInterval)), dataSchema ), toolbox, @@ -2194,7 +2176,7 @@ public void testNativeCompactionSubtaskUsesTimeChunkLock() throws Exception final TaskActionClient segmentAwareClient = new TaskActionClient() { @Override - public RetType submit(TaskAction action) throws java.io.IOException + public RetType submit(TaskAction action) throws IOException { if (action instanceof RetrieveUsedSegmentsAction) { return (RetType) segments; @@ -2210,11 +2192,6 @@ public RetType submit(TaskAction action) throws java.io.IOExc ); } - /** - * SegmentProvider.checkSegments() with TIME_CHUNK + SpecificSegmentsSpec must allow subset: - * specified segments must exist; non-specified segments are upgraded, not validated as missing. - * DEPRECATE_WHEN_SEGMENT_LOCK_REMOVED - */ @Test public void testSegmentProviderCheckSegmentsAllowsSubsetForTimeChunk() throws Exception { @@ -2236,9 +2213,6 @@ public void testSegmentProviderCheckSegmentsAllowsSubsetForTimeChunk() throws Ex provider.checkSegments(LockGranularity.TIME_CHUNK, allSegments); } - /** - * NativeCompactionRunner.createIoConfig passes segment IDs to DruidInputSource when using SpecificSegmentsSpec. - */ @Test public void testDruidInputSourceReceivesSegmentIdsForSpecificSegmentsSpec() { @@ -2272,7 +2246,7 @@ public void testDruidInputSourceReceivesSegmentIdsForSpecificSegmentsSpec() final List ingestionSpecs = NativeCompactionRunner.createIngestionSpecs( ImmutableMap.of( - new org.apache.druid.query.spec.MultipleIntervalSegmentSpec(ImmutableList.of(interval)), + new MultipleIntervalSegmentSpec(ImmutableList.of(interval)), dataSchema ), toolbox, diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/SpecificSegmentsSpecTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/SpecificSegmentsSpecTest.java index 5f98888e5c81..48150d791ec2 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/SpecificSegmentsSpecTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/SpecificSegmentsSpecTest.java @@ -27,7 +27,7 @@ import org.junit.Assert; import org.junit.Test; -import com.google.common.collect.ImmutableList; +import java.util.ArrayList; import java.util.Collections; import java.util.List; import java.util.concurrent.ThreadLocalRandom; @@ -36,11 +36,6 @@ public class SpecificSegmentsSpecTest { - /** - * validateSegments() with TIME_CHUNK must allow subset: specified segments exist in thoseSegments; - * thoseSegments may contain additional segments (non-specified will be upgraded). - * DEPRECATE_WHEN_SEGMENT_LOCK_REMOVED - */ @Test public void testValidateSegmentsTimeChunkAllowsSubset() { @@ -50,12 +45,11 @@ public void testValidateSegmentsTimeChunkAllowsSubset() .collect(Collectors.toList()); // Spec has only first 2 segments (subset) final SpecificSegmentsSpec spec = new SpecificSegmentsSpec( - ImmutableList.of( + new ArrayList<>(List.of( allSegments.get(0).getId().toString(), allSegments.get(1).getId().toString() - ).stream().collect(Collectors.toList()) + )) ); - // thoseSegments = all 4 segments in interval. Subset check: thoseSegments.containsAll(spec.segments) Assert.assertTrue( spec.validateSegments(LockGranularity.TIME_CHUNK, allSegments) ); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialCompactionTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialCompactionTest.java index 459f53b2b188..616e3503fb78 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialCompactionTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialCompactionTest.java @@ -19,6 +19,7 @@ package org.apache.druid.indexing.common.task.batch.parallel; +import com.google.common.collect.ImmutableMap; import org.apache.druid.data.input.InputFormat; import org.apache.druid.data.input.impl.CsvInputFormat; import org.apache.druid.data.input.impl.DimensionsSpec; @@ -51,7 +52,6 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Comparator; -import com.google.common.collect.ImmutableMap; import java.util.List; import java.util.Map; @@ -75,7 +75,7 @@ public class PartialCompactionTest extends AbstractMultiPhaseParallelIndexingTes public PartialCompactionTest() { - super(LockGranularity.SEGMENT, true, DEFAULT_TRANSIENT_TASK_FAILURE_RATE, DEFAULT_TRANSIENT_API_FAILURE_RATE); + super(LockGranularity.TIME_CHUNK, true, DEFAULT_TRANSIENT_TASK_FAILURE_RATE, DEFAULT_TRANSIENT_API_FAILURE_RATE); } @Before @@ -141,6 +141,7 @@ public void testPartialCompactHashAndDynamicPartitionedSegments() final CompactionTask compactionTask = newCompactionTaskBuilder() .inputSpec(SpecificSegmentsSpec.fromSegments(segmentsToCompact)) .tuningConfig(newTuningConfig(new DynamicPartitionsSpec(20, null), 2, false)) + .context(ImmutableMap.of(Tasks.USE_CONCURRENT_LOCKS, true)) .build(); dataSegmentsWithSchemas = runTask(compactionTask, TaskState.SUCCESS); verifySchema(dataSegmentsWithSchemas); @@ -201,6 +202,7 @@ public void testPartialCompactRangeAndDynamicPartitionedSegments() final CompactionTask compactionTask = newCompactionTaskBuilder() .inputSpec(SpecificSegmentsSpec.fromSegments(segmentsToCompact)) .tuningConfig(newTuningConfig(new DynamicPartitionsSpec(20, null), 2, false)) + .context(ImmutableMap.of(Tasks.USE_CONCURRENT_LOCKS, true)) .build(); dataSegmentsWithSchemas = runTask(compactionTask, TaskState.SUCCESS); @@ -218,7 +220,6 @@ public void testPartialCompactRangeAndDynamicPartitionedSegments() /** * End-to-end minor compaction: compact a subset of segments with useConcurrentLocks (TIME_CHUNK). * Non-compacted segments in the interval are upgraded via MarkSegmentToUpgradeAction. - * DEPRECATE_WHEN_SEGMENT_LOCK_REMOVED */ @Test public void testMinorCompactionUpgradesNonCompactedSegments() From 3dc0abfcebfeb73157709111a663b8bd9a072aee Mon Sep 17 00:00:00 2001 From: GWphua Date: Tue, 17 Mar 2026 16:00:54 +0800 Subject: [PATCH 10/45] Trim up unnecessary code --- .../druid/indexing/common/task/CompactionTask.java | 11 +---------- .../indexing/common/task/SpecificSegmentsSpec.java | 7 ++++--- 2 files changed, 5 insertions(+), 13 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java index a9de6b15c0a0..6367f2257a2a 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java @@ -1287,7 +1287,7 @@ static Granularity compareWithCurrent(Granularity queryGranularity, Granularity /** * Provides segment discovery and validation for compaction. * For minor compaction (SpecificSegmentsSpec, MinorCompactionInputSpec), finds all segments - * in the interval and partitions them into compact vs upgrade via {@link #shouldUpgradeSegment}. + * in the interval and partitions them into 'compact and upgrade metadata' vs 'upgrade metadata only' via {@link #shouldUpgradeSegment}. */ @VisibleForTesting static class SegmentProvider @@ -1325,10 +1325,6 @@ private boolean shouldUpgradeSegment(DataSegment s) } } - /** - * Returns all segments in the interval. For minor compaction (SpecificSegmentsSpec, MinorCompactionInputSpec), - * partitioning into compact vs upgrade is done via {@link #shouldUpgradeSegment}. - */ List findSegments(TaskActionClient actionClient) throws IOException { return new ArrayList<>( @@ -1338,16 +1334,11 @@ List findSegments(TaskActionClient actionClient) throws IOException ); } - /** - * Validates that the specified segments exist in the latest segments. For TIME_CHUNK and minor compaction, - * the caller passes the full interval set from {@code findSegments}. - */ void checkSegments(LockGranularity lockGranularityInUse, List latestSegments) { if (latestSegments.isEmpty()) { throw new ISE("No segments found for compaction. Please check that datasource name and interval are correct."); } - if (!inputSpec.validateSegments(lockGranularityInUse, latestSegments)) { throw new ISE( "Specified segments in the spec are different from the current used segments. " diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/SpecificSegmentsSpec.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/SpecificSegmentsSpec.java index 13097af3a702..4497e11d6197 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/SpecificSegmentsSpec.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/SpecificSegmentsSpec.java @@ -35,9 +35,11 @@ import java.util.stream.Collectors; /** + * Used only for native engine-based minor compaction. + *

* Compaction input spec that targets specific segments by ID. - * Used for native engine-based minor compaction: the timeline is built from all segments in the interval, - * and segments not in this spec are upgraded (no physical re-compaction). + * The timeline is built from all segments in the interval. Segments in the interval, + * but not specified in the spec, are upgraded with no physical re-compaction. */ public class SpecificSegmentsSpec implements CompactionInputSpec { @@ -70,7 +72,6 @@ public List getSegments() /** * Parses segment IDs to descriptors for minor compaction (compact vs upgrade partitioning). - * Invalid IDs are filtered out. */ List getSegmentDescriptors(String dataSource) { From 7eb5ee2b8ab61fa2b3416507701a04b593685c0b Mon Sep 17 00:00:00 2001 From: GWphua Date: Tue, 17 Mar 2026 16:21:09 +0800 Subject: [PATCH 11/45] Docs and stylecheck --- docs/data-management/manual-compaction.md | 12 +++++++----- .../druid/msq/indexing/MSQCompactionRunnerTest.java | 2 +- 2 files changed, 8 insertions(+), 6 deletions(-) diff --git a/docs/data-management/manual-compaction.md b/docs/data-management/manual-compaction.md index b2239d323473..07a81de614c6 100644 --- a/docs/data-management/manual-compaction.md +++ b/docs/data-management/manual-compaction.md @@ -123,25 +123,27 @@ The compaction `ioConfig` requires specifying `inputSpec` as follows: The compaction task has two kinds of `inputSpec`: -### Interval `inputSpec` (Major Compaction) +### Interval `inputSpec` |Field|Description|Required| |-----|-----------|--------| -|`type`|Task type. Set the value to `interval`.|Yes| +|`type`|Task type. Set the value to `interval` to trigger native-engine major compaction.|Yes| |`interval`|Interval to compact.|Yes| -### Segments `inputSpec` (Native Minor Compaction) +### Segments `inputSpec` |Field|Description|Required| |-----|-----------|--------| -|`type`|Task type. Set the value to `segments`.|Yes| +|`type`|Task type. Set the value to `segments` to trigger native-engine minor compaction.|Yes| |`segments`|A list of segment IDs.|Yes| +Note: MSQ compaction does not support segments `inputSpec`. Use `MinorCompactionInputSpec` (type: uncompacted) for MSQ minor compaction. + When using the segments `inputSpec`, the task compacts only the specified segments. Segments in the same interval that are not in the spec are upgraded in place rather than compacted. This allows compacting a subset of segments while preserving others. +There are some requirements when triggering a native minor compaction: - Set `useConcurrentLocks: true` in the task context. Minor compaction uses REPLACE (TIME_CHUNK) locks over the entire interval. - `dropExisting: true` is allowed with segments `inputSpec`; the task replaces only the compacted segments. -- MSQ compaction does not support segments `inputSpec`. Use `MinorCompactionInputSpec` (type: uncompacted) for MSQ minor compaction. ## Compaction dimensions spec diff --git a/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQCompactionRunnerTest.java b/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQCompactionRunnerTest.java index c4c619b784ec..4d4b4b2c80d2 100644 --- a/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQCompactionRunnerTest.java +++ b/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQCompactionRunnerTest.java @@ -72,9 +72,9 @@ import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.NestedDataColumnSchema; import org.apache.druid.segment.column.ColumnHolder; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.partition.NumberedShardSpec; -import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.data.CompressionFactory; import org.apache.druid.segment.data.CompressionStrategy; import org.apache.druid.segment.data.RoaringBitmapSerdeFactory; From 071e407a5a7ad64376840e67c28683227adeb3a5 Mon Sep 17 00:00:00 2001 From: GWphua Date: Tue, 17 Mar 2026 16:57:04 +0800 Subject: [PATCH 12/45] Checkstyle --- .../apache/druid/msq/indexing/MSQCompactionRunnerTest.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQCompactionRunnerTest.java b/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQCompactionRunnerTest.java index 4d4b4b2c80d2..f77d9cce46ce 100644 --- a/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQCompactionRunnerTest.java +++ b/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQCompactionRunnerTest.java @@ -73,8 +73,6 @@ import org.apache.druid.segment.NestedDataColumnSchema; import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.column.ColumnType; -import org.apache.druid.timeline.DataSegment; -import org.apache.druid.timeline.partition.NumberedShardSpec; import org.apache.druid.segment.data.CompressionFactory; import org.apache.druid.segment.data.CompressionStrategy; import org.apache.druid.segment.data.RoaringBitmapSerdeFactory; @@ -85,6 +83,8 @@ import org.apache.druid.server.coordinator.CompactionConfigValidationResult; import org.apache.druid.server.initialization.AuthorizerMapperModule; import org.apache.druid.sql.calcite.parser.DruidSqlInsert; +import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.partition.NumberedShardSpec; import org.joda.time.Interval; import org.junit.Assert; import org.junit.Test; From 5f516f9992100e0a2ecaef481d9a746ad84480c1 Mon Sep 17 00:00:00 2001 From: GWphua Date: Tue, 17 Mar 2026 17:26:49 +0800 Subject: [PATCH 13/45] Documentations first --- docs/data-management/manual-compaction.md | 48 +++++++++++++++++++---- 1 file changed, 40 insertions(+), 8 deletions(-) diff --git a/docs/data-management/manual-compaction.md b/docs/data-management/manual-compaction.md index 07a81de614c6..4f3effb5f695 100644 --- a/docs/data-management/manual-compaction.md +++ b/docs/data-management/manual-compaction.md @@ -117,9 +117,9 @@ The compaction `ioConfig` requires specifying `inputSpec` as follows: |Field|Description|Default|Required| |-----|-----------|-------|--------| |`type`|Task type. Set the value to `compact`.|none|Yes| -|`inputSpec`|Specification of the target [interval](#interval-inputspec) or [segments](#segments-inputspec).|none|Yes| +|`inputSpec`|Specification of the target [interval](#interval-inputspec) or [uncompacted](#uncompacted-inputspec).|none|Yes| |`dropExisting`|If `true`, the task replaces all existing segments fully contained by either of the following:
- the `interval` in the `interval` type `inputSpec`.
- the umbrella interval of the `segments` in the `segment` type `inputSpec`.
If compaction fails, Druid does not change any of the existing segments.
**WARNING**: `dropExisting` in `ioConfig` is a beta feature. |false|No| -|`allowNonAlignedInterval`|If `true`, the task allows an explicit [`segmentGranularity`](#compaction-granularity-spec) that is not aligned with the provided [interval](#interval-inputspec) or [segments](#segments-inputspec). This parameter is only used if [`segmentGranularity`](#compaction-granularity-spec) is explicitly provided.

This parameter is provided for backwards compatibility. In most scenarios it should not be set, as it can lead to data being accidentally overshadowed. This parameter may be removed in a future release.|false|No| +|`allowNonAlignedInterval`|If `true`, the task allows an explicit [`segmentGranularity`](#compaction-granularity-spec) that is not aligned with the provided [interval](#interval-inputspec) or [uncompacted](#uncompacted-inputspec). This parameter is only used if [`segmentGranularity`](#compaction-granularity-spec) is explicitly provided.

This parameter is provided for backwards compatibility. In most scenarios it should not be set, as it can lead to data being accidentally overshadowed. This parameter may be removed in a future release.|false|No| The compaction task has two kinds of `inputSpec`: @@ -127,24 +127,56 @@ The compaction task has two kinds of `inputSpec`: |Field|Description|Required| |-----|-----------|--------| -|`type`|Task type. Set the value to `interval` to trigger native-engine major compaction.|Yes| +|`type`|Task type. Set the value to `interval` to trigger major compaction.|Yes| |`interval`|Interval to compact.|Yes| -### Segments `inputSpec` +### Uncompacted `inputSpec` |Field|Description|Required| |-----|-----------|--------| -|`type`|Task type. Set the value to `segments` to trigger native-engine minor compaction.|Yes| -|`segments`|A list of segment IDs.|Yes| +|`type`|Task type. Set the value to `uncompacted` to trigger native-engine minor compaction.|Yes| +|`interval`|Interval to compact.|Yes| +|`uncompactedSegments`|A list of segment descriptors.|Yes| + +The required segment descriptor fields can be retrieved from the "Segments" section in the web console. -Note: MSQ compaction does not support segments `inputSpec`. Use `MinorCompactionInputSpec` (type: uncompacted) for MSQ minor compaction. +|Field|Description|Required| +|-----|-----------|--------| +|`itvl`|Interval of segment to compact.|Yes| +|`ver`|Version of the segment.|Yes| +|`part`|Partition number of the segment.|Yes| -When using the segments `inputSpec`, the task compacts only the specified segments. Segments in the same interval that are not in the spec are upgraded in place rather than compacted. This allows compacting a subset of segments while preserving others. +#### Example uncompacted inputSpec + +```json +{ + "type": "uncompacted", + "interval": "2020-01-01T00:00:00.000Z/2020-01-01T01:00:00.000Z", + "uncompactedSegments": [ + { + "itvl": "2020-01-01T00:00:00.000Z/2020-01-01T01:00:00.000Z", + "ver": "2020-01-01T00:07:18.186Z", + "part": 0 + }, + { + "itvl": "2020-01-01T00:00:00.000Z/2020-01-01T01:00:00.000Z", + "ver": "2020-01-01T00:07:18.186Z", + "part": 1 + } + ] +} +``` + +When using the uncompacted `inputSpec`, the task compacts only the specified segments. Segments in the same interval that are not in the spec are upgraded in place rather than compacted. This allows compacting a subset of segments while preserving others. There are some requirements when triggering a native minor compaction: - Set `useConcurrentLocks: true` in the task context. Minor compaction uses REPLACE (TIME_CHUNK) locks over the entire interval. - `dropExisting: true` is allowed with segments `inputSpec`; the task replaces only the compacted segments. +### Segment `inputSpec` + +The segment `inputSpec` is deprecated, instructions for usage will no longer be documented. Please use the above 2 `inputSpec` instead. + ## Compaction dimensions spec |Field|Description|Required| From f8279ba16f7433479ce8ae6fe7ee9025b9f132c5 Mon Sep 17 00:00:00 2001 From: GWphua Date: Tue, 17 Mar 2026 17:53:38 +0800 Subject: [PATCH 14/45] Integrate minor compaction spec --- .../indexing/common/task/CompactionTask.java | 5 +-- .../common/task/NativeCompactionRunner.java | 29 +++++++++------ .../common/task/SpecificSegmentsSpec.java | 20 +--------- .../common/task/CompactionTaskTest.java | 37 +++++++++++-------- .../common/task/SpecificSegmentsSpecTest.java | 36 ------------------ .../batch/parallel/PartialCompactionTest.java | 18 +++++++-- 6 files changed, 56 insertions(+), 89 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java index 6367f2257a2a..96f10802e9a3 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java @@ -1286,7 +1286,7 @@ static Granularity compareWithCurrent(Granularity queryGranularity, Granularity /** * Provides segment discovery and validation for compaction. - * For minor compaction (SpecificSegmentsSpec, MinorCompactionInputSpec), finds all segments + * For minor compaction (MinorCompactionInputSpec), finds all segments * in the interval and partitions them into 'compact and upgrade metadata' vs 'upgrade metadata only' via {@link #shouldUpgradeSegment}. */ @VisibleForTesting @@ -1307,9 +1307,6 @@ static class SegmentProvider if (inputSpec instanceof MinorCompactionInputSpec) { minorCompaction = true; uncompactedSegments = Set.copyOf(((MinorCompactionInputSpec) inputSpec).getUncompactedSegments()); - } else if (inputSpec instanceof SpecificSegmentsSpec) { - minorCompaction = true; - uncompactedSegments = Set.copyOf(((SpecificSegmentsSpec) inputSpec).getSegmentDescriptors(dataSource)); } else { minorCompaction = false; uncompactedSegments = null; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/NativeCompactionRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/NativeCompactionRunner.java index 1b22c7a0cb4c..2ce41eb41a20 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/NativeCompactionRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/NativeCompactionRunner.java @@ -37,6 +37,7 @@ import org.apache.druid.indexing.input.DruidInputSource; import org.apache.druid.indexing.input.WindowedSegmentId; import org.apache.druid.java.util.common.IAE; +import org.apache.druid.query.SegmentDescriptor; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.JodaUtils; import org.apache.druid.java.util.common.StringUtils; @@ -103,9 +104,10 @@ public CompactionConfigValidationResult validateCompactionTask( "Virtual columns in filter rules are not supported by the Native compaction engine. Use MSQ compaction engine instead." ); } - if (compactionTask.getIoConfig().getInputSpec() instanceof MinorCompactionInputSpec) { + if (compactionTask.getIoConfig().getInputSpec() instanceof SpecificSegmentsSpec) { return CompactionConfigValidationResult.failure( - "Minor compaction is not supported by Native compaction engine. Use MSQ compaction engine instead."); + "SpecificSegmentsSpec is deprecated. Use MinorCompactionInputSpec (type: uncompacted) for minor compaction." + ); } return CompactionConfigValidationResult.success(); } @@ -144,8 +146,9 @@ static List createIngestionSpecs( } /** - * When using {@link SpecificSegmentsSpec}, resolves specific segment IDs that belong to the given interval - * and returns them as {@link WindowedSegmentId} objects. Returns null for interval-based compaction. + * When using {@link MinorCompactionInputSpec}, resolves uncompacted segment descriptors that belong + * to the given interval and returns them as {@link WindowedSegmentId} objects. + * Returns null for interval-based compaction. */ @Nullable private static List resolveSegmentIdsForInterval( @@ -154,17 +157,19 @@ private static List resolveSegmentIdsForInterval( Interval interval ) { - if (!(inputSpec instanceof SpecificSegmentsSpec)) { + if (!(inputSpec instanceof MinorCompactionInputSpec)) { return null; } - SpecificSegmentsSpec spec = (SpecificSegmentsSpec) inputSpec; - List segmentIds = new ArrayList<>(); - for (String segmentIdStr : spec.getSegments()) { - SegmentId segmentId = SegmentId.tryParse(dataSource, segmentIdStr); - if (segmentId != null && interval.contains(segmentId.getInterval())) { + final MinorCompactionInputSpec spec = (MinorCompactionInputSpec) inputSpec; + final List segmentIds = new ArrayList<>(); + for (SegmentDescriptor desc : spec.getUncompactedSegments()) { + if (interval.contains(desc.getInterval())) { + final String segmentIdStr = SegmentId.of( + dataSource, desc.getInterval(), desc.getVersion(), desc.getPartitionNumber() + ).toString(); segmentIds.add(new WindowedSegmentId( segmentIdStr, - Collections.singletonList(segmentId.getInterval()) + Collections.singletonList(desc.getInterval()) )); } } @@ -186,7 +191,7 @@ private static ParallelIndexIOConfig createIoConfig( CompactionIOConfig compactionIOConfig ) { - // Resolve specific segment IDs for minor compaction if using SpecificSegmentsSpec + // Resolve specific segment IDs for minor compaction if using MinorCompactionInputSpec final List segmentIds = resolveSegmentIdsForInterval( compactionIOConfig.getInputSpec(), dataSchema.getDataSource(), diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/SpecificSegmentsSpec.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/SpecificSegmentsSpec.java index 4497e11d6197..9fb70732559a 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/SpecificSegmentsSpec.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/SpecificSegmentsSpec.java @@ -24,7 +24,6 @@ import com.google.common.base.Preconditions; import org.apache.druid.indexing.common.LockGranularity; import org.apache.druid.java.util.common.JodaUtils; -import org.apache.druid.query.SegmentDescriptor; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.SegmentId; import org.joda.time.Interval; @@ -35,12 +34,9 @@ import java.util.stream.Collectors; /** - * Used only for native engine-based minor compaction. - *

- * Compaction input spec that targets specific segments by ID. - * The timeline is built from all segments in the interval. Segments in the interval, - * but not specified in the spec, are upgraded with no physical re-compaction. + * @deprecated Use {@link MinorCompactionInputSpec} for minor compaction in both native and MSQ engines. */ +@Deprecated public class SpecificSegmentsSpec implements CompactionInputSpec { public static final String TYPE = "segments"; @@ -70,18 +66,6 @@ public List getSegments() return segments; } - /** - * Parses segment IDs to descriptors for minor compaction (compact vs upgrade partitioning). - */ - List getSegmentDescriptors(String dataSource) - { - return segments.stream() - .map(id -> SegmentId.tryParse(dataSource, id)) - .filter(Objects::nonNull) - .map(SegmentId::toDescriptor) - .collect(Collectors.toList()); - } - @Override public Interval findInterval(String dataSource) { diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java index 2f32b104dd97..4c737ed075be 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java @@ -2039,7 +2039,7 @@ public void drop(DataSegment segment) } @Test - public void testFindSegmentsToLockReturnsAllSegmentsForSpecificSegmentsSpec() throws Exception + public void testFindSegmentsToLockReturnsAllSegmentsForMinorCompaction() throws Exception { final Interval testInterval = Intervals.of("2024-11-18T00:00:00.000Z/2024-11-25T00:00:00.000Z"); final String version = "2024-11-17T23:49:06.823Z"; @@ -2057,16 +2057,13 @@ public void testFindSegmentsToLockReturnsAllSegmentsForSpecificSegmentsSpec() th allSegmentsInInterval.add(createSegmentWithPartition(testInterval, version, 10)); allSegmentsInInterval.add(createSegmentWithPartition(testInterval, version, 12)); - final SpecificSegmentsSpec specificSegmentsSpec = new SpecificSegmentsSpec( - new ArrayList<>(ImmutableList.of( - segment6.getId().toString(), - segment7.getId().toString(), - segment8.getId().toString() - )) + final MinorCompactionInputSpec minorSpec = new MinorCompactionInputSpec( + testInterval, + ImmutableList.of(segment6.toDescriptor(), segment7.toDescriptor(), segment8.toDescriptor()) ); final CompactionTask compactionTask = new Builder(DATA_SOURCE, segmentCacheManagerFactory) - .inputSpec(specificSegmentsSpec) + .inputSpec(minorSpec) .context(ImmutableMap.of(Tasks.USE_CONCURRENT_LOCKS, true)) .build(); @@ -2081,14 +2078,17 @@ public void testFindSegmentsToLockReturnsAllSegmentsForSpecificSegmentsSpec() th } @Test - public void testSpecificSegmentsSpecUsesTimeChunkLockWithConcurrentLocks() throws Exception + public void testMinorCompactionUsesTimeChunkLockWithConcurrentLocks() throws Exception { final Interval testInterval = Intervals.of("2024-11-18T00:00:00.000Z/2024-11-25T00:00:00.000Z"); final List segments = ImmutableList.of( createSegmentWithPartition(testInterval, "v1", 0), createSegmentWithPartition(testInterval, "v1", 1) ); - final SpecificSegmentsSpec spec = SpecificSegmentsSpec.fromSegments(segments); + final MinorCompactionInputSpec spec = new MinorCompactionInputSpec( + testInterval, + segments.stream().map(DataSegment::toDescriptor).collect(Collectors.toList()) + ); final CompactionTask task = new Builder(DATA_SOURCE, segmentCacheManagerFactory) .inputSpec(spec) @@ -2125,7 +2125,10 @@ public void testNativeCompactionSubtaskUsesTimeChunkLock() throws Exception createSegmentWithPartition(testInterval, "v1", 0), createSegmentWithPartition(testInterval, "v1", 1) ); - final SpecificSegmentsSpec spec = SpecificSegmentsSpec.fromSegments(segments); + final MinorCompactionInputSpec spec = new MinorCompactionInputSpec( + testInterval, + segments.stream().map(DataSegment::toDescriptor).collect(Collectors.toList()) + ); final CompactionTask compactionTask = new Builder(DATA_SOURCE, segmentCacheManagerFactory) .inputSpec(spec) @@ -2201,8 +2204,9 @@ public void testSegmentProviderCheckSegmentsAllowsSubsetForTimeChunk() throws Ex createSegmentWithPartition(testInterval, "v1", 1), createSegmentWithPartition(testInterval, "v1", 2) ); - final SpecificSegmentsSpec spec = new SpecificSegmentsSpec( - new ArrayList<>(ImmutableList.of(allSegments.get(0).getId().toString(), allSegments.get(1).getId().toString())) + final MinorCompactionInputSpec spec = new MinorCompactionInputSpec( + testInterval, + ImmutableList.of(allSegments.get(0).toDescriptor(), allSegments.get(1).toDescriptor()) ); final SegmentProvider provider = new SegmentProvider(DATA_SOURCE, spec); @@ -2214,14 +2218,17 @@ public void testSegmentProviderCheckSegmentsAllowsSubsetForTimeChunk() throws Ex } @Test - public void testDruidInputSourceReceivesSegmentIdsForSpecificSegmentsSpec() + public void testDruidInputSourceReceivesSegmentIdsForMinorCompaction() { final Interval interval = Intervals.of("2024-01-01/2024-01-02"); final List segments = ImmutableList.of( createSegmentWithPartition(interval, "v1", 0), createSegmentWithPartition(interval, "v1", 1) ); - final SpecificSegmentsSpec spec = SpecificSegmentsSpec.fromSegments(segments); + final MinorCompactionInputSpec spec = new MinorCompactionInputSpec( + interval, + segments.stream().map(DataSegment::toDescriptor).collect(Collectors.toList()) + ); final DataSchema dataSchema = DataSchema.builder() .withDataSource(DATA_SOURCE) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/SpecificSegmentsSpecTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/SpecificSegmentsSpecTest.java index 48150d791ec2..b6c5802c159b 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/SpecificSegmentsSpecTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/SpecificSegmentsSpecTest.java @@ -19,15 +19,12 @@ package org.apache.druid.indexing.common.task; -import org.apache.druid.indexing.common.LockGranularity; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.timeline.DataSegment; -import org.apache.druid.timeline.partition.NumberedShardSpec; import org.joda.time.Interval; import org.junit.Assert; import org.junit.Test; -import java.util.ArrayList; import java.util.Collections; import java.util.List; import java.util.concurrent.ThreadLocalRandom; @@ -36,25 +33,6 @@ public class SpecificSegmentsSpecTest { - @Test - public void testValidateSegmentsTimeChunkAllowsSubset() - { - final Interval interval = Intervals.of("2019-01-01/2019-01-02"); - final List allSegments = IntStream.range(0, 4) - .mapToObj(i -> newSegmentWithPartition(interval, i)) - .collect(Collectors.toList()); - // Spec has only first 2 segments (subset) - final SpecificSegmentsSpec spec = new SpecificSegmentsSpec( - new ArrayList<>(List.of( - allSegments.get(0).getId().toString(), - allSegments.get(1).getId().toString() - )) - ); - Assert.assertTrue( - spec.validateSegments(LockGranularity.TIME_CHUNK, allSegments) - ); - } - @Test public void createTest() { @@ -86,18 +64,4 @@ private static DataSegment newSegment(Interval interval) ); } - private static DataSegment newSegmentWithPartition(Interval interval, int partitionNum) - { - return new DataSegment( - "datasource", - interval, - "version", - null, - null, - null, - new NumberedShardSpec(partitionNum, 4), - 9, - 10 - ); - } } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialCompactionTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialCompactionTest.java index 616e3503fb78..ccb13b2738f8 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialCompactionTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialCompactionTest.java @@ -32,7 +32,7 @@ import org.apache.druid.indexing.common.LockGranularity; import org.apache.druid.indexing.common.task.CompactionTask; import org.apache.druid.indexing.common.task.CompactionTask.Builder; -import org.apache.druid.indexing.common.task.SpecificSegmentsSpec; +import org.apache.druid.indexing.common.task.MinorCompactionInputSpec; import org.apache.druid.indexing.common.task.Tasks; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.StringUtils; @@ -54,6 +54,7 @@ import java.util.Comparator; import java.util.List; import java.util.Map; +import java.util.stream.Collectors; public class PartialCompactionTest extends AbstractMultiPhaseParallelIndexingTest { @@ -139,7 +140,10 @@ public void testPartialCompactHashAndDynamicPartitionedSegments() ); } final CompactionTask compactionTask = newCompactionTaskBuilder() - .inputSpec(SpecificSegmentsSpec.fromSegments(segmentsToCompact)) + .inputSpec(new MinorCompactionInputSpec( + INTERVAL_TO_INDEX, + segmentsToCompact.stream().map(DataSegment::toDescriptor).collect(Collectors.toList()) + )) .tuningConfig(newTuningConfig(new DynamicPartitionsSpec(20, null), 2, false)) .context(ImmutableMap.of(Tasks.USE_CONCURRENT_LOCKS, true)) .build(); @@ -200,7 +204,10 @@ public void testPartialCompactRangeAndDynamicPartitionedSegments() ); } final CompactionTask compactionTask = newCompactionTaskBuilder() - .inputSpec(SpecificSegmentsSpec.fromSegments(segmentsToCompact)) + .inputSpec(new MinorCompactionInputSpec( + INTERVAL_TO_INDEX, + segmentsToCompact.stream().map(DataSegment::toDescriptor).collect(Collectors.toList()) + )) .tuningConfig(newTuningConfig(new DynamicPartitionsSpec(20, null), 2, false)) .context(ImmutableMap.of(Tasks.USE_CONCURRENT_LOCKS, true)) .build(); @@ -245,7 +252,10 @@ public void testMinorCompactionUpgradesNonCompactedSegments() segmentsToCompact.addAll(segmentsInInterval.subList(0, Math.min(2, segmentsInInterval.size()))); } final CompactionTask compactionTask = newCompactionTaskBuilder() - .inputSpec(SpecificSegmentsSpec.fromSegments(segmentsToCompact)) + .inputSpec(new MinorCompactionInputSpec( + INTERVAL_TO_INDEX, + segmentsToCompact.stream().map(DataSegment::toDescriptor).collect(Collectors.toList()) + )) .tuningConfig(newTuningConfig(new DynamicPartitionsSpec(20, null), 2, false)) .context(ImmutableMap.of(Tasks.USE_CONCURRENT_LOCKS, true)) .build(); From 7e0262ae08d371ac1a6a7d859e3264df6aae18c1 Mon Sep 17 00:00:00 2001 From: GWphua Date: Tue, 17 Mar 2026 18:03:53 +0800 Subject: [PATCH 15/45] Checkstyle --- .../common/task/NativeCompactionRunner.java | 16 +++++---- .../msq/indexing/MSQCompactionRunner.java | 9 ----- .../msq/indexing/MSQCompactionRunnerTest.java | 35 ------------------- 3 files changed, 9 insertions(+), 51 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/NativeCompactionRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/NativeCompactionRunner.java index 2ce41eb41a20..b0b3a741a337 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/NativeCompactionRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/NativeCompactionRunner.java @@ -37,12 +37,12 @@ import org.apache.druid.indexing.input.DruidInputSource; import org.apache.druid.indexing.input.WindowedSegmentId; import org.apache.druid.java.util.common.IAE; -import org.apache.druid.query.SegmentDescriptor; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.JodaUtils; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.query.SegmentDescriptor; import org.apache.druid.query.spec.QuerySegmentSpec; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.server.coordinator.CompactionConfigValidationResult; @@ -160,15 +160,17 @@ private static List resolveSegmentIdsForInterval( if (!(inputSpec instanceof MinorCompactionInputSpec)) { return null; } - final MinorCompactionInputSpec spec = (MinorCompactionInputSpec) inputSpec; final List segmentIds = new ArrayList<>(); - for (SegmentDescriptor desc : spec.getUncompactedSegments()) { + for (SegmentDescriptor desc : ((MinorCompactionInputSpec) inputSpec).getUncompactedSegments()) { if (interval.contains(desc.getInterval())) { - final String segmentIdStr = SegmentId.of( - dataSource, desc.getInterval(), desc.getVersion(), desc.getPartitionNumber() - ).toString(); + final SegmentId segmentId = SegmentId.of( + dataSource, + desc.getInterval(), + desc.getVersion(), + desc.getPartitionNumber() + ); segmentIds.add(new WindowedSegmentId( - segmentIdStr, + segmentId.toString(), Collections.singletonList(desc.getInterval()) )); } diff --git a/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java b/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java index 215c71b69fae..07dec885a77f 100644 --- a/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java +++ b/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java @@ -38,7 +38,6 @@ import org.apache.druid.indexing.common.task.CompactionRunner; import org.apache.druid.indexing.common.task.CompactionTask; import org.apache.druid.indexing.common.task.CurrentSubTaskHolder; -import org.apache.druid.indexing.common.task.SpecificSegmentsSpec; import org.apache.druid.java.util.common.JodaUtils; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.AllGranularity; @@ -158,14 +157,6 @@ public CompactionConfigValidationResult validateCompactionTask( Map intervalToDataSchemaMap ) { - // MSQ compaction uses REPLACE mode which locks the entire interval and replaces all segments. - // Minor compaction (SpecificSegmentsSpec) is fundamentally incompatible with this behavior. - if (compactionTask.getIoConfig().getInputSpec() instanceof SpecificSegmentsSpec) { - return CompactionConfigValidationResult.failure( - "MSQ: Minor compaction with SpecificSegmentsSpec is not supported. " - + "Use MinorCompactionInputSpec (type: uncompacted) for MSQ minor compaction." - ); - } if (intervalToDataSchemaMap.size() > 1) { // We are currently not able to handle multiple intervals in the map for multiple reasons, one of them being that // the subsequent worker ids clash -- since they are derived from MSQControllerTask ID which in turn is equal to diff --git a/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQCompactionRunnerTest.java b/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQCompactionRunnerTest.java index f77d9cce46ce..fa124f44da35 100644 --- a/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQCompactionRunnerTest.java +++ b/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQCompactionRunnerTest.java @@ -43,7 +43,6 @@ import org.apache.druid.indexer.partitions.PartitionsSpec; import org.apache.druid.indexing.common.task.CompactionIntervalSpec; import org.apache.druid.indexing.common.task.CompactionTask; -import org.apache.druid.indexing.common.task.SpecificSegmentsSpec; import org.apache.druid.indexing.common.task.TuningConfigBuilder; import org.apache.druid.initialization.CoreInjectorBuilder; import org.apache.druid.jackson.DefaultObjectMapper; @@ -83,8 +82,6 @@ import org.apache.druid.server.coordinator.CompactionConfigValidationResult; import org.apache.druid.server.initialization.AuthorizerMapperModule; import org.apache.druid.sql.calcite.parser.DruidSqlInsert; -import org.apache.druid.timeline.DataSegment; -import org.apache.druid.timeline.partition.NumberedShardSpec; import org.joda.time.Interval; import org.junit.Assert; import org.junit.Test; @@ -191,38 +188,6 @@ public void testMultipleDisjointCompactionIntervalsAreInvalid() ); } - @Test - public void testRejectsSpecificSegmentsSpec() - { - final List segments = ImmutableList.of( - new DataSegment( - DATA_SOURCE, - COMPACTION_INTERVAL, - "v1", - null, - null, - null, - new NumberedShardSpec(0, 1), - 0, - 10 - ) - ); - final CompactionTask compactionTask = new CompactionTask.Builder(DATA_SOURCE, null) - .inputSpec(SpecificSegmentsSpec.fromSegments(segments)) - .compactionRunner(MSQ_COMPACTION_RUNNER) - .build(); - CompactionConfigValidationResult validationResult = MSQ_COMPACTION_RUNNER.validateCompactionTask( - compactionTask, - INTERVAL_DATASCHEMAS - ); - Assert.assertFalse(validationResult.isValid()); - Assert.assertTrue( - "Error message should mention MinorCompactionInputSpec", - validationResult.getReason().contains("MinorCompactionInputSpec") - || validationResult.getReason().contains("SpecificSegmentsSpec") - ); - } - @Test public void testHashedPartitionsSpecIsInvalid() { From 8c482b1dc5628bf8c5848a1b0391face294ced05 Mon Sep 17 00:00:00 2001 From: GWphua Date: Tue, 17 Mar 2026 18:11:51 +0800 Subject: [PATCH 16/45] Docs --- docs/data-management/manual-compaction.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/data-management/manual-compaction.md b/docs/data-management/manual-compaction.md index 4f3effb5f695..fe4b3b7052f8 100644 --- a/docs/data-management/manual-compaction.md +++ b/docs/data-management/manual-compaction.md @@ -134,7 +134,7 @@ The compaction task has two kinds of `inputSpec`: |Field|Description|Required| |-----|-----------|--------| -|`type`|Task type. Set the value to `uncompacted` to trigger native-engine minor compaction.|Yes| +|`type`|Task type. Set the value to `uncompacted` to trigger minor compaction.|Yes| |`interval`|Interval to compact.|Yes| |`uncompactedSegments`|A list of segment descriptors.|Yes| From 2cc684c5005558918b057a3caebc038d127d1a45 Mon Sep 17 00:00:00 2001 From: GWphua Date: Tue, 17 Mar 2026 18:12:04 +0800 Subject: [PATCH 17/45] Tidy up changes --- .../druid/indexing/common/task/NativeCompactionRunner.java | 5 ----- .../druid/indexing/common/task/SpecificSegmentsSpecTest.java | 1 - 2 files changed, 6 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/NativeCompactionRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/NativeCompactionRunner.java index b0b3a741a337..e96f8a4a040b 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/NativeCompactionRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/NativeCompactionRunner.java @@ -104,11 +104,6 @@ public CompactionConfigValidationResult validateCompactionTask( "Virtual columns in filter rules are not supported by the Native compaction engine. Use MSQ compaction engine instead." ); } - if (compactionTask.getIoConfig().getInputSpec() instanceof SpecificSegmentsSpec) { - return CompactionConfigValidationResult.failure( - "SpecificSegmentsSpec is deprecated. Use MinorCompactionInputSpec (type: uncompacted) for minor compaction." - ); - } return CompactionConfigValidationResult.success(); } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/SpecificSegmentsSpecTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/SpecificSegmentsSpecTest.java index b6c5802c159b..4fb1558f0ef7 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/SpecificSegmentsSpecTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/SpecificSegmentsSpecTest.java @@ -63,5 +63,4 @@ private static DataSegment newSegment(Interval interval) 10 ); } - } From 3a95032acf040190b25e382d3e1e887509292076 Mon Sep 17 00:00:00 2001 From: GWphua Date: Tue, 17 Mar 2026 18:16:31 +0800 Subject: [PATCH 18/45] Revert changes to SpecificSegmentsSpec --- .../common/task/SpecificSegmentsSpec.java | 18 ++++++++++-------- 1 file changed, 10 insertions(+), 8 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/SpecificSegmentsSpec.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/SpecificSegmentsSpec.java index 9fb70732559a..93f181d0dcaf 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/SpecificSegmentsSpec.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/SpecificSegmentsSpec.java @@ -34,7 +34,7 @@ import java.util.stream.Collectors; /** - * @deprecated Use {@link MinorCompactionInputSpec} for minor compaction in both native and MSQ engines. + * @deprecated Use {@link MinorCompactionInputSpec} for minor compaction instead. */ @Deprecated public class SpecificSegmentsSpec implements CompactionInputSpec @@ -54,7 +54,6 @@ public static SpecificSegmentsSpec fromSegments(List segments) @JsonCreator public SpecificSegmentsSpec(@JsonProperty("segments") List segments) { - Preconditions.checkArgument(segments != null && !segments.isEmpty(), "Segments must not be null or empty"); this.segments = segments; // Sort segments to use in validateSegments. Collections.sort(this.segments); @@ -69,13 +68,13 @@ public List getSegments() @Override public Interval findInterval(String dataSource) { - final List intervals = segments + final List segmentIds = segments .stream() .map(segment -> SegmentId.tryParse(dataSource, segment)) - .filter(Objects::nonNull) - .map(SegmentId::getInterval) .collect(Collectors.toList()); - return JodaUtils.umbrellaInterval(intervals); + return JodaUtils.umbrellaInterval( + segmentIds.stream().map(SegmentId::getInterval).collect(Collectors.toList()) + ); } @Override @@ -86,8 +85,11 @@ public boolean validateSegments(LockGranularity lockGranularityInUse, List segment.getId().toString()) .sorted() .collect(Collectors.toList()); - - return thoseSegments.containsAll(segments); + if (lockGranularityInUse == LockGranularity.TIME_CHUNK) { + return this.segments.equals(thoseSegments); + } else { + return thoseSegments.containsAll(segments); + } } @Override From 20e34a9af8722aae3fed702dd24106a6a27b27e9 Mon Sep 17 00:00:00 2001 From: GWphua Date: Tue, 17 Mar 2026 18:32:05 +0800 Subject: [PATCH 19/45] Spell check --- docs/data-management/manual-compaction.md | 2 +- website/.spelling | 1 + 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/docs/data-management/manual-compaction.md b/docs/data-management/manual-compaction.md index fe4b3b7052f8..de9fc439c8f3 100644 --- a/docs/data-management/manual-compaction.md +++ b/docs/data-management/manual-compaction.md @@ -170,7 +170,7 @@ The required segment descriptor fields can be retrieved from the "Segments" sect When using the uncompacted `inputSpec`, the task compacts only the specified segments. Segments in the same interval that are not in the spec are upgraded in place rather than compacted. This allows compacting a subset of segments while preserving others. There are some requirements when triggering a native minor compaction: -- Set `useConcurrentLocks: true` in the task context. Minor compaction uses REPLACE (TIME_CHUNK) locks over the entire interval. +- Set `useConcurrentLocks: true` in the task context. Minor compaction uses REPLACE locks over the entire interval. - `dropExisting: true` is allowed with segments `inputSpec`; the task replaces only the compacted segments. ### Segment `inputSpec` diff --git a/website/.spelling b/website/.spelling index f2b87496eb2b..2a0b7bf9bba7 100644 --- a/website/.spelling +++ b/website/.spelling @@ -599,6 +599,7 @@ unannouncements unary unassign uncomment +uncompacted underutilization unintuitive unioned From 2851351a24cd6710e9679de31925031863d5a4af Mon Sep 17 00:00:00 2001 From: GWphua Date: Wed, 18 Mar 2026 10:30:06 +0800 Subject: [PATCH 20/45] Force time chunk lock only for minor compaction --- .../druid/indexing/common/task/NativeCompactionRunner.java | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/NativeCompactionRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/NativeCompactionRunner.java index e96f8a4a040b..554bb1b7fa34 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/NativeCompactionRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/NativeCompactionRunner.java @@ -367,8 +367,10 @@ Map createContextForSubtask(CompactionTask compactionTask) newContext.putIfAbsent(CompactSegments.STORE_COMPACTION_STATE_KEY, STORE_COMPACTION_STATE); // Set the priority of the compaction task. newContext.put(Tasks.PRIORITY_KEY, compactionTask.getPriority()); - // Native compaction subtasks always use TIME_CHUNK (REPLACE mode). - newContext.put(Tasks.FORCE_TIME_CHUNK_LOCK_KEY, true); + // Native minor compaction uses REPLACE ingestion mode, which uses time chunk lock. + if (compactionTask.getIoConfig().getInputSpec() instanceof MinorCompactionInputSpec) { + newContext.put(Tasks.FORCE_TIME_CHUNK_LOCK_KEY, true); + } return newContext; } From 15f1d7b74803f550ddd1badb8014ff6de084cf64 Mon Sep 17 00:00:00 2001 From: GWphua Date: Wed, 18 Mar 2026 13:21:55 +0800 Subject: [PATCH 21/45] Docs --- docs/data-management/manual-compaction.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/data-management/manual-compaction.md b/docs/data-management/manual-compaction.md index de9fc439c8f3..3a68707ef327 100644 --- a/docs/data-management/manual-compaction.md +++ b/docs/data-management/manual-compaction.md @@ -169,7 +169,7 @@ The required segment descriptor fields can be retrieved from the "Segments" sect When using the uncompacted `inputSpec`, the task compacts only the specified segments. Segments in the same interval that are not in the spec are upgraded in place rather than compacted. This allows compacting a subset of segments while preserving others. -There are some requirements when triggering a native minor compaction: +There are some requirements when triggering a minor compaction: - Set `useConcurrentLocks: true` in the task context. Minor compaction uses REPLACE locks over the entire interval. - `dropExisting: true` is allowed with segments `inputSpec`; the task replaces only the compacted segments. From b7d68f5e70c885cd469b446a4ba22619f56b1282 Mon Sep 17 00:00:00 2001 From: GWphua Date: Wed, 18 Mar 2026 14:09:43 +0800 Subject: [PATCH 22/45] Minor compaction test --- .../druid/indexing/common/task/CompactionTaskTest.java | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java index 4c737ed075be..ac466e572637 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java @@ -1206,13 +1206,14 @@ public void testCreateIngestionSchemaWithDifferentSegmentSet() throws IOExceptio expectedException.expect(CoreMatchers.instanceOf(IllegalStateException.class)); expectedException.expectMessage(CoreMatchers.containsString("are different from the current used segments")); - // Spec includes a segment ID that does not exist in metadata - validation should fail - final List segmentIds = SEGMENTS.stream().map(s -> s.getId().toString()).collect(Collectors.toList()); - segmentIds.add(DATA_SOURCE + "_2020-01-01T00:00:00.000Z_2020-02-01T00:00:00.000Z_x_0"); + final List segments = new ArrayList<>(SEGMENTS); + Collections.sort(segments); + // Remove one segment in the middle + segments.remove(segments.size() / 2); final Map inputSchemas = CompactionTask.createInputDataSchemas( toolbox, LockGranularity.TIME_CHUNK, - new SegmentProvider(DATA_SOURCE, new SpecificSegmentsSpec(segmentIds)), + new SegmentProvider(DATA_SOURCE, SpecificSegmentsSpec.fromSegments(segments)), null, null, null, From 3b08dc96f5c1f70d43db27b4f6a212fd0fa05ecc Mon Sep 17 00:00:00 2001 From: GWphua Date: Thu, 19 Mar 2026 11:43:48 +0800 Subject: [PATCH 23/45] Native minor compaction add in default values --- .../indexing/common/task/NativeCompactionRunner.java | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/NativeCompactionRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/NativeCompactionRunner.java index 554bb1b7fa34..63895569267a 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/NativeCompactionRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/NativeCompactionRunner.java @@ -196,9 +196,12 @@ private static ParallelIndexIOConfig createIoConfig( ); final Interval inputInterval; + boolean isDropExisting = compactionIOConfig.isDropExisting(); if (segmentIds != null && !segmentIds.isEmpty()) { - // When compacting specific segments, use segment IDs instead of interval + // When doing minor compaction, use segment IDs instead of interval. inputInterval = null; + // Force dropExisting to true in minor compaction. + isDropExisting = true; } else { inputInterval = interval; } @@ -235,7 +238,7 @@ private static ParallelIndexIOConfig createIoConfig( ).withTaskToolbox(toolbox), null, false, - compactionIOConfig.isDropExisting() + isDropExisting ); } @@ -370,6 +373,7 @@ Map createContextForSubtask(CompactionTask compactionTask) // Native minor compaction uses REPLACE ingestion mode, which uses time chunk lock. if (compactionTask.getIoConfig().getInputSpec() instanceof MinorCompactionInputSpec) { newContext.put(Tasks.FORCE_TIME_CHUNK_LOCK_KEY, true); + newContext.put(Tasks.USE_CONCURRENT_LOCKS, true); } return newContext; } From ee97aea5288d6d37b62030db61a7dbff25d96a4a Mon Sep 17 00:00:00 2001 From: GWphua Date: Thu, 19 Mar 2026 14:55:16 +0800 Subject: [PATCH 24/45] Junit5 for TaskLockHelperTest --- .../common/task/TaskLockHelperTest.java | 48 ++++++++++++------- 1 file changed, 32 insertions(+), 16 deletions(-) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TaskLockHelperTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TaskLockHelperTest.java index 3007a5079ef7..3a47c5a0ab71 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TaskLockHelperTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TaskLockHelperTest.java @@ -28,7 +28,8 @@ import org.apache.druid.timeline.partition.NumberedOverwriteShardSpec; import org.apache.druid.timeline.partition.PartitionIds; import org.joda.time.Interval; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; import java.util.Collections; import java.util.List; @@ -39,7 +40,7 @@ public class TaskLockHelperTest private static final Interval TEST_INTERVAL = Intervals.of("2017-01-01/2017-01-02"); private static final String TEST_VERSION = DateTimes.nowUtc().toString(); - @Test(expected = ISE.class) + @Test public void testVerifyNonConsecutiveSegmentsInInputFails() { // Test that non-consecutive segments within the input list fail. @@ -50,7 +51,10 @@ public void testVerifyNonConsecutiveSegmentsInInputFails() createSegment(3, 3, 4, (short) 1, (short) 1) // rootPartitionRange [3, 4) ); - TaskLockHelper.verifyRootPartitionIsAdjacentAndAtomicUpdateGroupIsFull(segments); + Assertions.assertThrows( + ISE.class, + () -> TaskLockHelper.verifyRootPartitionIsAdjacentAndAtomicUpdateGroupIsFull(segments) + ); } @Test @@ -77,7 +81,7 @@ public void testVerifyConsecutiveSegmentsStillWorks() TaskLockHelper.verifyRootPartitionIsAdjacentAndAtomicUpdateGroupIsFull(segments); } - @Test(expected = ISE.class) + @Test public void testVerifyLargeGapSegmentsFails() { final List segments = ImmutableList.of( @@ -86,7 +90,10 @@ public void testVerifyLargeGapSegmentsFails() createSegment(10, 10, 11, (short) 1, (short) 1) ); - TaskLockHelper.verifyRootPartitionIsAdjacentAndAtomicUpdateGroupIsFull(segments); + Assertions.assertThrows( + ISE.class, + () -> TaskLockHelper.verifyRootPartitionIsAdjacentAndAtomicUpdateGroupIsFull(segments) + ); } @Test @@ -100,7 +107,7 @@ public void testVerifyAtomicUpdateGroupValidationStillWorks() TaskLockHelper.verifyRootPartitionIsAdjacentAndAtomicUpdateGroupIsFull(segments); } - @Test(expected = ISE.class) + @Test public void testVerifyAtomicUpdateGroupIncompleteFails() { final List segments = ImmutableList.of( @@ -109,10 +116,13 @@ public void testVerifyAtomicUpdateGroupIncompleteFails() ); // Should throw ISE because atomicUpdateGroupSize is 3 but we only have 2 segments - TaskLockHelper.verifyRootPartitionIsAdjacentAndAtomicUpdateGroupIsFull(segments); + Assertions.assertThrows( + ISE.class, + () -> TaskLockHelper.verifyRootPartitionIsAdjacentAndAtomicUpdateGroupIsFull(segments) + ); } - @Test(expected = ISE.class) + @Test public void testVerifyDifferentMinorVersionsFail() { // Test that segments with same root partition range but different minor versions fail @@ -120,11 +130,13 @@ public void testVerifyDifferentMinorVersionsFail() createSegment(0, 0, 1, (short) 1, (short) 2), createSegment(1, 0, 1, (short) 2, (short) 2) // Different minor version ); - - TaskLockHelper.verifyRootPartitionIsAdjacentAndAtomicUpdateGroupIsFull(segments); + Assertions.assertThrows( + ISE.class, + () -> TaskLockHelper.verifyRootPartitionIsAdjacentAndAtomicUpdateGroupIsFull(segments) + ); } - @Test(expected = ISE.class) + @Test public void testVerifyDifferentAtomicUpdateGroupSizesFail() { // Test that segments with same root partition range but different atomicUpdateGroupSize fail @@ -132,8 +144,10 @@ public void testVerifyDifferentAtomicUpdateGroupSizesFail() createSegment(0, 0, 1, (short) 1, (short) 2), createSegment(1, 0, 1, (short) 1, (short) 3) ); - - TaskLockHelper.verifyRootPartitionIsAdjacentAndAtomicUpdateGroupIsFull(segments); + Assertions.assertThrows( + ISE.class, + () -> TaskLockHelper.verifyRootPartitionIsAdjacentAndAtomicUpdateGroupIsFull(segments) + ); } @Test @@ -152,7 +166,7 @@ public void testVerifySingleSegment() TaskLockHelper.verifyRootPartitionIsAdjacentAndAtomicUpdateGroupIsFull(segments); } - @Test(expected = IllegalArgumentException.class) + @Test public void testVerifyDifferentIntervalsFail() { final Interval interval1 = Intervals.of("2017-01-01/2017-01-02"); @@ -179,8 +193,10 @@ public void testVerifyDifferentIntervalsFail() .size(0) .build() ); - - TaskLockHelper.verifyRootPartitionIsAdjacentAndAtomicUpdateGroupIsFull(segments); + Assertions.assertThrows( + IllegalArgumentException.class, + () -> TaskLockHelper.verifyRootPartitionIsAdjacentAndAtomicUpdateGroupIsFull(segments) + ); } /** From 5c65976b563fc2f4006331bd744846fd57a43414 Mon Sep 17 00:00:00 2001 From: GWphua Date: Thu, 19 Mar 2026 15:26:08 +0800 Subject: [PATCH 25/45] ImmutableXXX.of() -> XXX.of() methods in Test --- .../common/task/CompactionTaskTest.java | 89 +++++++++---------- 1 file changed, 41 insertions(+), 48 deletions(-) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java index ac466e572637..f4073c2d19ec 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java @@ -29,9 +29,6 @@ import com.fasterxml.jackson.databind.module.SimpleModule; import com.google.common.base.Preconditions; import com.google.common.base.Supplier; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; -import com.google.common.collect.ImmutableSet; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.google.common.collect.Maps; @@ -191,7 +188,7 @@ public class CompactionTaskTest private static final String TIMESTAMP_COLUMN = "timestamp"; private static final String MIXED_TYPE_COLUMN = "string_to_double"; private static final Interval COMPACTION_INTERVAL = Intervals.of("2017-01-01/2017-07-01"); - private static final List SEGMENT_INTERVALS = ImmutableList.of( + private static final List SEGMENT_INTERVALS = List.of( Intervals.of("2017-01-01/2017-02-01"), Intervals.of("2017-02-01/2017-03-01"), Intervals.of("2017-03-01/2017-04-01"), @@ -276,7 +273,7 @@ public static void setupClass() DATA_SOURCE, SEGMENT_INTERVALS.get(i), "version_" + i, - ImmutableMap.of(), + Map.of(), findDimensions(i, SEGMENT_INTERVALS.get(i)), AGGREGATORS.stream().map(AggregatorFactory::getName).collect(Collectors.toList()), new NumberedShardSpec(0, 1), @@ -304,7 +301,7 @@ private static ObjectMapper setupInjectablesInObjectMapper(ObjectMapper objectMa ); GuiceInjectableValues injectableValues = new GuiceInjectableValues( GuiceInjectors.makeStartupInjectorWithModules( - ImmutableList.of( + List.of( binder -> { binder.bind(AuthorizerMapper.class).toInstance(AuthTestUtils.TEST_AUTHORIZER_MAPPER); binder.bind(ChatHandlerProvider.class).toInstance(new NoopChatHandlerProvider()); @@ -528,7 +525,7 @@ public void testSerdeWithInterval() throws IOException new CompactionIntervalSpec(COMPACTION_INTERVAL, SegmentUtils.hashIds(SEGMENTS)) ) .tuningConfig(createTuningConfig()) - .context(ImmutableMap.of("testKey", "testContext")) + .context(Map.of("testKey", "testContext")) .build(); final byte[] bytes = OBJECT_MAPPER.writeValueAsBytes(task); @@ -546,7 +543,7 @@ public void testSerdeWithSegments() throws IOException final CompactionTask task = builder .segments(SEGMENTS) .tuningConfig(createTuningConfig()) - .context(ImmutableMap.of("testKey", "testContext")) + .context(Map.of("testKey", "testContext")) .build(); final byte[] bytes = OBJECT_MAPPER.writeValueAsBytes(task); @@ -566,7 +563,7 @@ public void testSerdeWithDimensions() throws IOException .segments(SEGMENTS) .dimensionsSpec( new DimensionsSpec( - ImmutableList.of( + List.of( new StringDimensionSchema("dim1"), new StringDimensionSchema("dim2"), new StringDimensionSchema("dim3") @@ -574,7 +571,7 @@ public void testSerdeWithDimensions() throws IOException ) ) .tuningConfig(createTuningConfig()) - .context(ImmutableMap.of("testKey", "testVal")) + .context(Map.of("testKey", "testVal")) .build(); final byte[] bytes = OBJECT_MAPPER.writeValueAsBytes(task); @@ -683,7 +680,7 @@ public void testInputSourceResources() new CompactionIntervalSpec(COMPACTION_INTERVAL, SegmentUtils.hashIds(SEGMENTS)) ) .tuningConfig(createTuningConfig()) - .context(ImmutableMap.of("testKey", "testContext")) + .context(Map.of("testKey", "testContext")) .build(); Assert.assertTrue(task.getInputSourceResources().isEmpty()); @@ -791,7 +788,7 @@ public void testSegmentProviderFindSegmentsWithEmptySegmentsThrowException() expectedException.expectMessage( "No segments found for compaction. Please check that datasource name and interval are correct." ); - provider.checkSegments(LockGranularity.TIME_CHUNK, ImmutableList.of()); + provider.checkSegments(LockGranularity.TIME_CHUNK, List.of()); } @Test @@ -1305,7 +1302,7 @@ public void testSegmentGranularityAndNullQueryGranularity() throws IOException COORDINATOR_CLIENT, segmentCacheManagerFactory ); - final List expectedDimensionsSpec = ImmutableList.of( + final List expectedDimensionsSpec = List.of( new DimensionsSpec(getDimensionSchema(new DoubleDimensionSchema("string_to_double"))) ); @@ -1400,7 +1397,7 @@ public void testQueryGranularityAndSegmentGranularityNonNull() throws IOExceptio ); - final List expectedDimensionsSpec = ImmutableList.of( + final List expectedDimensionsSpec = List.of( new DimensionsSpec(getDimensionSchema(new DoubleDimensionSchema("string_to_double"))) ); @@ -1633,7 +1630,7 @@ public void testMSQRangePartitionWithNoDimensionsSpecNeedsMVDInfo() new DimensionRangePartitionsSpec( 3, null, - ImmutableList.of( + List.of( "string_dim_1"), false )) @@ -1654,7 +1651,7 @@ public void testMSQRollupOnStringNeedsMVDInfo() builder.granularitySpec(new ClientCompactionTaskGranularitySpec(null, null, true)); DimensionSchema stringDim = new StringDimensionSchema("string_dim_1", null, null); - builder.dimensionsSpec(new DimensionsSpec(ImmutableList.of(stringDim))); + builder.dimensionsSpec(new DimensionsSpec(List.of(stringDim))); final CompactionTask compactionTask = builder.build(); // A string dimension with rollup=true should need MVD info Assert.assertTrue(compactionTask.identifyMultiValuedDimensions()); @@ -1677,12 +1674,12 @@ public void testMSQRangePartitionOnStringNeedsMVDInfo() new DimensionRangePartitionsSpec( 3, null, - ImmutableList.of( + List.of( stringDim.getName()), false )) .build()); - builder.dimensionsSpec(new DimensionsSpec(ImmutableList.of(stringDim))); + builder.dimensionsSpec(new DimensionsSpec(List.of(stringDim))); CompactionTask compactionTask = builder.build(); Assert.assertTrue(compactionTask.identifyMultiValuedDimensions()); } @@ -1709,7 +1706,7 @@ public void testMSQRangePartitionOnAutoStringDoesNotNeedMVDInfo() ) ) .build()); - builder.dimensionsSpec(new DimensionsSpec(ImmutableList.of(stringDim))); + builder.dimensionsSpec(new DimensionsSpec(List.of(stringDim))); CompactionTask compactionTask = builder.build(); Assert.assertFalse(compactionTask.identifyMultiValuedDimensions()); } @@ -1734,7 +1731,7 @@ public void testChooseFinestGranularityWithNulls() @Test public void testChooseFinestGranularityNone() { - List input = ImmutableList.of( + List input = List.of( Granularities.DAY, Granularities.SECOND, Granularities.MINUTE, @@ -1799,7 +1796,7 @@ private Granularity chooseFinestGranularityHelper(List granularitie private static List getExpectedDimensionsSpecForAutoGeneration() { - return ImmutableList.of( + return List.of( new DimensionsSpec(getDimensionSchema(new StringDimensionSchema("string_to_double", DimensionSchema.MultiValueHandling.ARRAY, null))), new DimensionsSpec(getDimensionSchema(new StringDimensionSchema("string_to_double", DimensionSchema.MultiValueHandling.ARRAY, null))), new DimensionsSpec(getDimensionSchema(new StringDimensionSchema("string_to_double", DimensionSchema.MultiValueHandling.ARRAY, null))), @@ -1962,7 +1959,7 @@ public ListenableFuture> fetchUsedSegments( List intervals ) { - return Futures.immediateFuture(ImmutableList.copyOf(segmentMap.keySet())); + return Futures.immediateFuture(List.copyOf(segmentMap.keySet())); } } @@ -2060,12 +2057,12 @@ public void testFindSegmentsToLockReturnsAllSegmentsForMinorCompaction() throws final MinorCompactionInputSpec minorSpec = new MinorCompactionInputSpec( testInterval, - ImmutableList.of(segment6.toDescriptor(), segment7.toDescriptor(), segment8.toDescriptor()) + List.of(segment6.toDescriptor(), segment7.toDescriptor(), segment8.toDescriptor()) ); final CompactionTask compactionTask = new Builder(DATA_SOURCE, segmentCacheManagerFactory) .inputSpec(minorSpec) - .context(ImmutableMap.of(Tasks.USE_CONCURRENT_LOCKS, true)) + .context(Map.of(Tasks.USE_CONCURRENT_LOCKS, true)) .build(); final TestTaskActionClient taskActionClient = new TestTaskActionClient(allSegmentsInInterval); @@ -2073,7 +2070,7 @@ public void testFindSegmentsToLockReturnsAllSegmentsForMinorCompaction() throws // Verify findSegmentsToLock() returns ALL segments in interval (no filtering) final List segmentsToLock = compactionTask.findSegmentsToLock( taskActionClient, - ImmutableList.of(testInterval) + List.of(testInterval) ); Assert.assertEquals(8, segmentsToLock.size()); } @@ -2082,7 +2079,7 @@ public void testFindSegmentsToLockReturnsAllSegmentsForMinorCompaction() throws public void testMinorCompactionUsesTimeChunkLockWithConcurrentLocks() throws Exception { final Interval testInterval = Intervals.of("2024-11-18T00:00:00.000Z/2024-11-25T00:00:00.000Z"); - final List segments = ImmutableList.of( + final List segments = List.of( createSegmentWithPartition(testInterval, "v1", 0), createSegmentWithPartition(testInterval, "v1", 1) ); @@ -2093,7 +2090,7 @@ public void testMinorCompactionUsesTimeChunkLockWithConcurrentLocks() throws Exc final CompactionTask task = new Builder(DATA_SOURCE, segmentCacheManagerFactory) .inputSpec(spec) - .context(ImmutableMap.of(Tasks.USE_CONCURRENT_LOCKS, true)) + .context(Map.of(Tasks.USE_CONCURRENT_LOCKS, true)) .build(); taskActionTestKit.getTaskLockbox().add(task); @@ -2108,13 +2105,15 @@ public void testMinorCompactionUsesTimeChunkLockWithConcurrentLocks() throws Exc public RetType submit(TaskAction action) throws IOException { if (action instanceof RetrieveUsedSegmentsAction) { - return (RetType) segments; + @SuppressWarnings("unchecked") + RetType retVal = (RetType) segments; + return retVal; } return taskActionClient.submit(action); } }; - task.determineLockGranularityAndTryLock(segmentAwareClient, ImmutableList.of(testInterval)); + task.determineLockGranularityAndTryLock(segmentAwareClient, List.of(testInterval)); Assert.assertEquals(LockGranularity.TIME_CHUNK, task.getTaskLockHelper().getLockGranularityToUse()); } @@ -2122,7 +2121,7 @@ public RetType submit(TaskAction action) throws IOException public void testNativeCompactionSubtaskUsesTimeChunkLock() throws Exception { final Interval testInterval = Intervals.of("2024-11-18T00:00:00.000Z/2024-11-25T00:00:00.000Z"); - final List segments = ImmutableList.of( + final List segments = List.of( createSegmentWithPartition(testInterval, "v1", 0), createSegmentWithPartition(testInterval, "v1", 1) ); @@ -2133,7 +2132,7 @@ public void testNativeCompactionSubtaskUsesTimeChunkLock() throws Exception final CompactionTask compactionTask = new Builder(DATA_SOURCE, segmentCacheManagerFactory) .inputSpec(spec) - .context(ImmutableMap.of(Tasks.USE_CONCURRENT_LOCKS, true)) + .context(Map.of(Tasks.USE_CONCURRENT_LOCKS, true)) .build(); final NativeCompactionRunner runner = new NativeCompactionRunner(segmentCacheManagerFactory); @@ -2143,18 +2142,15 @@ public void testNativeCompactionSubtaskUsesTimeChunkLock() throws Exception .withDataSource(DATA_SOURCE) .withTimestamp(new TimestampSpec(TIMESTAMP_COLUMN, null, null)) .withDimensions( - new DimensionsSpec(ImmutableList.of(new StringDimensionSchema("dim1"), new StringDimensionSchema("dim2"))) + new DimensionsSpec(List.of(new StringDimensionSchema("dim1"), new StringDimensionSchema("dim2"))) ) .withGranularity( - new UniformGranularitySpec(Granularities.DAY, Granularities.HOUR, false, ImmutableList.of(testInterval)) + new UniformGranularitySpec(Granularities.DAY, Granularities.HOUR, false, List.of(testInterval)) ) .build(); final List ingestionSpecs = NativeCompactionRunner.createIngestionSpecs( - ImmutableMap.of( - new MultipleIntervalSegmentSpec(ImmutableList.of(testInterval)), - dataSchema - ), + Map.of(new MultipleIntervalSegmentSpec(List.of(testInterval)), dataSchema), toolbox, new CompactionIOConfig(spec, false, null), new PartitionConfigurationManager(null), @@ -2188,7 +2184,7 @@ public RetType submit(TaskAction action) throws IOException return taskActionClient.submit(action); } }; - subtask.determineLockGranularityAndTryLock(segmentAwareClient, ImmutableList.of(testInterval)); + subtask.determineLockGranularityAndTryLock(segmentAwareClient, List.of(testInterval)); Assert.assertEquals( LockGranularity.TIME_CHUNK, @@ -2200,14 +2196,14 @@ public RetType submit(TaskAction action) throws IOException public void testSegmentProviderCheckSegmentsAllowsSubsetForTimeChunk() throws Exception { final Interval testInterval = Intervals.of("2024-11-18T00:00:00.000Z/2024-11-25T00:00:00.000Z"); - final List allSegments = ImmutableList.of( + final List allSegments = List.of( createSegmentWithPartition(testInterval, "v1", 0), createSegmentWithPartition(testInterval, "v1", 1), createSegmentWithPartition(testInterval, "v1", 2) ); final MinorCompactionInputSpec spec = new MinorCompactionInputSpec( testInterval, - ImmutableList.of(allSegments.get(0).toDescriptor(), allSegments.get(1).toDescriptor()) + List.of(allSegments.get(0).toDescriptor(), allSegments.get(1).toDescriptor()) ); final SegmentProvider provider = new SegmentProvider(DATA_SOURCE, spec); @@ -2222,7 +2218,7 @@ public void testSegmentProviderCheckSegmentsAllowsSubsetForTimeChunk() throws Ex public void testDruidInputSourceReceivesSegmentIdsForMinorCompaction() { final Interval interval = Intervals.of("2024-01-01/2024-01-02"); - final List segments = ImmutableList.of( + final List segments = List.of( createSegmentWithPartition(interval, "v1", 0), createSegmentWithPartition(interval, "v1", 1) ); @@ -2236,7 +2232,7 @@ public void testDruidInputSourceReceivesSegmentIdsForMinorCompaction() .withTimestamp(new TimestampSpec(TIMESTAMP_COLUMN, null, null)) .withDimensions( new DimensionsSpec( - ImmutableList.of( + List.of( new StringDimensionSchema("dim1"), new StringDimensionSchema("dim2") ) @@ -2247,16 +2243,13 @@ public void testDruidInputSourceReceivesSegmentIdsForMinorCompaction() Granularities.DAY, Granularities.HOUR, false, - ImmutableList.of(interval) + List.of(interval) ) ) .build(); final List ingestionSpecs = NativeCompactionRunner.createIngestionSpecs( - ImmutableMap.of( - new MultipleIntervalSegmentSpec(ImmutableList.of(interval)), - dataSchema - ), + Map.of(new MultipleIntervalSegmentSpec(List.of(interval)), dataSchema), toolbox, new CompactionIOConfig(spec, false, null), new PartitionConfigurationManager(null), @@ -2566,7 +2559,7 @@ public String getType() @Override public Set getInputSourceResources() { - return ImmutableSet.of(); + return Set.of(); } @JsonProperty From 82ca1f236076be5897b26fe55af9e222afb722e1 Mon Sep 17 00:00:00 2001 From: GWphua Date: Thu, 19 Mar 2026 15:59:50 +0800 Subject: [PATCH 26/45] Validate minor compaction task configs --- .../common/task/NativeCompactionRunner.java | 84 +++++++++++-------- 1 file changed, 48 insertions(+), 36 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/NativeCompactionRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/NativeCompactionRunner.java index 63895569267a..ff572baca8ae 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/NativeCompactionRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/NativeCompactionRunner.java @@ -42,6 +42,7 @@ import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.math.expr.Function; import org.apache.druid.query.SegmentDescriptor; import org.apache.druid.query.spec.QuerySegmentSpec; import org.apache.druid.segment.indexing.DataSchema; @@ -53,7 +54,6 @@ import javax.annotation.Nullable; import java.util.ArrayList; -import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -104,6 +104,21 @@ public CompactionConfigValidationResult validateCompactionTask( "Virtual columns in filter rules are not supported by the Native compaction engine. Use MSQ compaction engine instead." ); } + + if (compactionTask.getIoConfig().getInputSpec() instanceof MinorCompactionInputSpec) { + boolean usingConcurrentLocks = compactionTask.getContextValue(Tasks.USE_CONCURRENT_LOCKS); + if (!usingConcurrentLocks) { + return CompactionConfigValidationResult.failure( + "Task context[%s] must be true when using native minor compaction", Tasks.USE_CONCURRENT_LOCKS + ); + } + + if (!compactionTask.getIoConfig().isDropExisting()) { + return CompactionConfigValidationResult.failure( + "Task ioConfig[dropExisting] must be true when using native minor compaction" + ); + } + } return CompactionConfigValidationResult.success(); } @@ -140,39 +155,6 @@ static List createIngestionSpecs( ).collect(Collectors.toList()); } - /** - * When using {@link MinorCompactionInputSpec}, resolves uncompacted segment descriptors that belong - * to the given interval and returns them as {@link WindowedSegmentId} objects. - * Returns null for interval-based compaction. - */ - @Nullable - private static List resolveSegmentIdsForInterval( - CompactionInputSpec inputSpec, - String dataSource, - Interval interval - ) - { - if (!(inputSpec instanceof MinorCompactionInputSpec)) { - return null; - } - final List segmentIds = new ArrayList<>(); - for (SegmentDescriptor desc : ((MinorCompactionInputSpec) inputSpec).getUncompactedSegments()) { - if (interval.contains(desc.getInterval())) { - final SegmentId segmentId = SegmentId.of( - dataSource, - desc.getInterval(), - desc.getVersion(), - desc.getPartitionNumber() - ); - segmentIds.add(new WindowedSegmentId( - segmentId.toString(), - Collections.singletonList(desc.getInterval()) - )); - } - } - return segmentIds.isEmpty() ? null : segmentIds; - } - private String createIndexTaskSpecId(String taskId, int i) { return StringUtils.format("%s_%d", taskId, i); @@ -189,7 +171,7 @@ private static ParallelIndexIOConfig createIoConfig( ) { // Resolve specific segment IDs for minor compaction if using MinorCompactionInputSpec - final List segmentIds = resolveSegmentIdsForInterval( + final List segmentIds = resolveSegmentIdsForMinorCompaction( compactionIOConfig.getInputSpec(), dataSchema.getDataSource(), interval @@ -242,6 +224,37 @@ private static ParallelIndexIOConfig createIoConfig( ); } + /** + * When using {@link MinorCompactionInputSpec}, resolves uncompacted segment descriptors that belong + * to the given interval and returns them as {@link WindowedSegmentId} objects. + */ + private static List resolveSegmentIdsForMinorCompaction( + CompactionInputSpec inputSpec, + String dataSource, + Interval interval + ) + { + if (!(inputSpec instanceof MinorCompactionInputSpec)) { + return List.of(); + } + + final List segmentIds = new ArrayList<>(); + for (SegmentDescriptor desc : ((MinorCompactionInputSpec) inputSpec).getUncompactedSegments()) { + if (interval.contains(desc.getInterval())) { + final SegmentId segmentId = SegmentId.of( + dataSource, + desc.getInterval(), + desc.getVersion(), + desc.getPartitionNumber() + ); + segmentIds.add( + new WindowedSegmentId(segmentId.toString(), List.of(desc.getInterval())) + ); + } + } + return segmentIds.isEmpty() ? List.of() : segmentIds; + } + @Override public TaskStatus runCompactionTasks( CompactionTask compactionTask, @@ -373,7 +386,6 @@ Map createContextForSubtask(CompactionTask compactionTask) // Native minor compaction uses REPLACE ingestion mode, which uses time chunk lock. if (compactionTask.getIoConfig().getInputSpec() instanceof MinorCompactionInputSpec) { newContext.put(Tasks.FORCE_TIME_CHUNK_LOCK_KEY, true); - newContext.put(Tasks.USE_CONCURRENT_LOCKS, true); } return newContext; } From 80d4d77a76a40f620b4bf7e21e295f3c91b8bae7 Mon Sep 17 00:00:00 2001 From: GWphua Date: Thu, 19 Mar 2026 16:26:34 +0800 Subject: [PATCH 27/45] Compaction IO Config creation refactor --- .../common/task/NativeCompactionRunner.java | 88 +++++++++++++------ 1 file changed, 60 insertions(+), 28 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/NativeCompactionRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/NativeCompactionRunner.java index ff572baca8ae..657516f610a4 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/NativeCompactionRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/NativeCompactionRunner.java @@ -25,6 +25,7 @@ import com.fasterxml.jackson.core.JsonProcessingException; import com.google.common.annotations.VisibleForTesting; import org.apache.druid.client.coordinator.CoordinatorClient; +import org.apache.druid.error.DruidException; import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; import org.apache.druid.indexer.partitions.PartitionsSpec; @@ -42,7 +43,6 @@ import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.common.logger.Logger; -import org.apache.druid.math.expr.Function; import org.apache.druid.query.SegmentDescriptor; import org.apache.druid.query.spec.QuerySegmentSpec; import org.apache.druid.segment.indexing.DataSchema; @@ -170,26 +170,22 @@ private static ParallelIndexIOConfig createIoConfig( CompactionIOConfig compactionIOConfig ) { - // Resolve specific segment IDs for minor compaction if using MinorCompactionInputSpec - final List segmentIds = resolveSegmentIdsForMinorCompaction( - compactionIOConfig.getInputSpec(), - dataSchema.getDataSource(), - interval - ); - final Interval inputInterval; - boolean isDropExisting = compactionIOConfig.isDropExisting(); - if (segmentIds != null && !segmentIds.isEmpty()) { - // When doing minor compaction, use segment IDs instead of interval. - inputInterval = null; - // Force dropExisting to true in minor compaction. - isDropExisting = true; - } else { - inputInterval = interval; - } + return (compactionIOConfig.getInputSpec() instanceof MinorCompactionInputSpec) + ? createMinorCompactionIoConfig(toolbox, dataSchema, interval, coordinatorClient, segmentCacheManagerFactory, compactionIOConfig) + : createMajorCompactionIoConfig(toolbox, dataSchema, interval, coordinatorClient, segmentCacheManagerFactory, compactionIOConfig); + } - if (inputInterval != null && !compactionIOConfig.isAllowNonAlignedInterval()) { - // Validate interval alignment only when using interval-based input (not segment-ID mode). + private static ParallelIndexIOConfig createMajorCompactionIoConfig( + TaskToolbox toolbox, + DataSchema dataSchema, + Interval inputInterval, + CoordinatorClient coordinatorClient, + SegmentCacheManagerFactory segmentCacheManagerFactory, + CompactionIOConfig compactionIOConfig + ) + { + if (!compactionIOConfig.isAllowNonAlignedInterval()) { final Granularity segmentGranularity = dataSchema.getGranularitySpec().getSegmentGranularity(); final Interval widenedInterval = Intervals.utc( segmentGranularity.bucketStart(inputInterval.getStart()).getMillis(), @@ -209,6 +205,46 @@ private static ParallelIndexIOConfig createIoConfig( new DruidInputSource( dataSchema.getDataSource(), inputInterval, + null, + null, + null, + null, + toolbox.getIndexIO(), + coordinatorClient, + segmentCacheManagerFactory, + toolbox.getConfig() + ).withTaskToolbox(toolbox), + null, + false, + compactionIOConfig.isDropExisting() + ); + } + + private static ParallelIndexIOConfig createMinorCompactionIoConfig( + TaskToolbox toolbox, + DataSchema dataSchema, + Interval interval, + CoordinatorClient coordinatorClient, + SegmentCacheManagerFactory segmentCacheManagerFactory, + CompactionIOConfig compactionIOConfig + ) + { + final List segmentIds = resolveSegmentIdsForMinorCompaction( + (MinorCompactionInputSpec) compactionIOConfig.getInputSpec(), + dataSchema.getDataSource(), + interval + ); + + if (segmentIds.isEmpty()) { + throw DruidException.forPersona(DruidException.Persona.USER) + .ofCategory(DruidException.Category.INVALID_INPUT) + .build("Minor compaction spec submitted targets no segments."); + } + + return new ParallelIndexIOConfig( + new DruidInputSource( + dataSchema.getDataSource(), + null, segmentIds, null, null, @@ -220,26 +256,22 @@ private static ParallelIndexIOConfig createIoConfig( ).withTaskToolbox(toolbox), null, false, - isDropExisting + compactionIOConfig.isDropExisting() ); } /** - * When using {@link MinorCompactionInputSpec}, resolves uncompacted segment descriptors that belong + * When using {@link MinorCompactionInputSpec}, resolves segment descriptors to compact that belong * to the given interval and returns them as {@link WindowedSegmentId} objects. */ private static List resolveSegmentIdsForMinorCompaction( - CompactionInputSpec inputSpec, + MinorCompactionInputSpec inputSpec, String dataSource, Interval interval ) { - if (!(inputSpec instanceof MinorCompactionInputSpec)) { - return List.of(); - } - final List segmentIds = new ArrayList<>(); - for (SegmentDescriptor desc : ((MinorCompactionInputSpec) inputSpec).getUncompactedSegments()) { + for (SegmentDescriptor desc : inputSpec.getUncompactedSegments()) { if (interval.contains(desc.getInterval())) { final SegmentId segmentId = SegmentId.of( dataSource, @@ -252,7 +284,7 @@ private static List resolveSegmentIdsForMinorCompaction( ); } } - return segmentIds.isEmpty() ? List.of() : segmentIds; + return segmentIds; } @Override From 234f6194747c80efe007a37d1dc20d8fa6495649 Mon Sep 17 00:00:00 2001 From: GWphua Date: Thu, 19 Mar 2026 16:43:36 +0800 Subject: [PATCH 28/45] Validations --- .../druid/indexing/common/task/CompactionTask.java | 10 ++++++++++ .../indexing/common/task/NativeCompactionRunner.java | 6 ------ 2 files changed, 10 insertions(+), 6 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java index 96f10802e9a3..03682a4cbca5 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java @@ -223,6 +223,16 @@ public CompactionTask( //noinspection ConstantConditions this.ioConfig = new CompactionIOConfig(SpecificSegmentsSpec.fromSegments(segments), false, null); } + + assert(ioConfig != null); + if (ioConfig.getInputSpec() != null && ioConfig.getInputSpec() instanceof MinorCompactionInputSpec) { + if (computeCompactionIngestionMode(ioConfig) != IngestionMode.REPLACE) { + throw DruidException.forPersona(DruidException.Persona.USER) + .ofCategory(DruidException.Category.INVALID_INPUT) + .build("Minor compaction is only used with REPLACE ingestion mode. Please set ioconfig[isDropExisting] to true."); + } + } + this.dimensionsSpec = dimensionsSpec == null ? dimensions : dimensionsSpec; this.transformSpec = transformSpec; this.metricsSpec = metricsSpec; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/NativeCompactionRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/NativeCompactionRunner.java index 657516f610a4..2199e86216b7 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/NativeCompactionRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/NativeCompactionRunner.java @@ -112,12 +112,6 @@ public CompactionConfigValidationResult validateCompactionTask( "Task context[%s] must be true when using native minor compaction", Tasks.USE_CONCURRENT_LOCKS ); } - - if (!compactionTask.getIoConfig().isDropExisting()) { - return CompactionConfigValidationResult.failure( - "Task ioConfig[dropExisting] must be true when using native minor compaction" - ); - } } return CompactionConfigValidationResult.success(); } From fbba2f029c216079217d2152c92e64c9ce1706b5 Mon Sep 17 00:00:00 2001 From: GWphua Date: Thu, 19 Mar 2026 16:56:13 +0800 Subject: [PATCH 29/45] Test fixes --- .../apache/druid/indexing/common/task/CompactionTask.java | 3 +-- .../druid/indexing/common/task/CompactionTaskTest.java | 8 ++++---- 2 files changed, 5 insertions(+), 6 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java index 03682a4cbca5..a435185d5c40 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java @@ -224,8 +224,7 @@ public CompactionTask( this.ioConfig = new CompactionIOConfig(SpecificSegmentsSpec.fromSegments(segments), false, null); } - assert(ioConfig != null); - if (ioConfig.getInputSpec() != null && ioConfig.getInputSpec() instanceof MinorCompactionInputSpec) { + if (ioConfig != null && ioConfig.getInputSpec() != null && ioConfig.getInputSpec() instanceof MinorCompactionInputSpec) { if (computeCompactionIngestionMode(ioConfig) != IngestionMode.REPLACE) { throw DruidException.forPersona(DruidException.Persona.USER) .ofCategory(DruidException.Category.INVALID_INPUT) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java index f4073c2d19ec..1a151cefa059 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java @@ -2061,7 +2061,7 @@ public void testFindSegmentsToLockReturnsAllSegmentsForMinorCompaction() throws ); final CompactionTask compactionTask = new Builder(DATA_SOURCE, segmentCacheManagerFactory) - .inputSpec(minorSpec) + .inputSpec(minorSpec, true) .context(Map.of(Tasks.USE_CONCURRENT_LOCKS, true)) .build(); @@ -2089,7 +2089,7 @@ public void testMinorCompactionUsesTimeChunkLockWithConcurrentLocks() throws Exc ); final CompactionTask task = new Builder(DATA_SOURCE, segmentCacheManagerFactory) - .inputSpec(spec) + .inputSpec(spec, true) .context(Map.of(Tasks.USE_CONCURRENT_LOCKS, true)) .build(); @@ -2118,7 +2118,7 @@ public RetType submit(TaskAction action) throws IOException } @Test - public void testNativeCompactionSubtaskUsesTimeChunkLock() throws Exception + public void testNativeMinorCompactionSubtaskUsesTimeChunkLock() throws Exception { final Interval testInterval = Intervals.of("2024-11-18T00:00:00.000Z/2024-11-25T00:00:00.000Z"); final List segments = List.of( @@ -2131,7 +2131,7 @@ public void testNativeCompactionSubtaskUsesTimeChunkLock() throws Exception ); final CompactionTask compactionTask = new Builder(DATA_SOURCE, segmentCacheManagerFactory) - .inputSpec(spec) + .inputSpec(spec, true) .context(Map.of(Tasks.USE_CONCURRENT_LOCKS, true)) .build(); From eee95242f7419a98ee9f1dfe13ce5982856850e7 Mon Sep 17 00:00:00 2001 From: GWphua Date: Thu, 19 Mar 2026 16:57:38 +0800 Subject: [PATCH 30/45] Remove test asserting finding segments to lock --- .../common/task/CompactionTaskTest.java | 39 ------------------- 1 file changed, 39 deletions(-) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java index 1a151cefa059..0153e736ea63 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java @@ -2036,45 +2036,6 @@ public void drop(DataSegment segment) .build(); } - @Test - public void testFindSegmentsToLockReturnsAllSegmentsForMinorCompaction() throws Exception - { - final Interval testInterval = Intervals.of("2024-11-18T00:00:00.000Z/2024-11-25T00:00:00.000Z"); - final String version = "2024-11-17T23:49:06.823Z"; - - final List allSegmentsInInterval = new ArrayList<>(); - allSegmentsInInterval.add(createSegmentWithPartition(testInterval, version, 0)); - allSegmentsInInterval.add(createSegmentWithPartition(testInterval, version, 2)); - allSegmentsInInterval.add(createSegmentWithPartition(testInterval, version, 4)); - final DataSegment segment6 = createSegmentWithPartition(testInterval, version, 6); - final DataSegment segment7 = createSegmentWithPartition(testInterval, version, 7); - final DataSegment segment8 = createSegmentWithPartition(testInterval, version, 8); - allSegmentsInInterval.add(segment6); - allSegmentsInInterval.add(segment7); - allSegmentsInInterval.add(segment8); - allSegmentsInInterval.add(createSegmentWithPartition(testInterval, version, 10)); - allSegmentsInInterval.add(createSegmentWithPartition(testInterval, version, 12)); - - final MinorCompactionInputSpec minorSpec = new MinorCompactionInputSpec( - testInterval, - List.of(segment6.toDescriptor(), segment7.toDescriptor(), segment8.toDescriptor()) - ); - - final CompactionTask compactionTask = new Builder(DATA_SOURCE, segmentCacheManagerFactory) - .inputSpec(minorSpec, true) - .context(Map.of(Tasks.USE_CONCURRENT_LOCKS, true)) - .build(); - - final TestTaskActionClient taskActionClient = new TestTaskActionClient(allSegmentsInInterval); - - // Verify findSegmentsToLock() returns ALL segments in interval (no filtering) - final List segmentsToLock = compactionTask.findSegmentsToLock( - taskActionClient, - List.of(testInterval) - ); - Assert.assertEquals(8, segmentsToLock.size()); - } - @Test public void testMinorCompactionUsesTimeChunkLockWithConcurrentLocks() throws Exception { From 0f728ab0ccd08cc7cafea81ce6c91b843f53a73b Mon Sep 17 00:00:00 2001 From: GWphua Date: Thu, 19 Mar 2026 17:07:48 +0800 Subject: [PATCH 31/45] Validation tests --- .idea/inspectionProfiles/Druid.xml | 310 ++++++++++-------- .../common/task/CompactionTaskTest.java | 35 +- 2 files changed, 200 insertions(+), 145 deletions(-) diff --git a/.idea/inspectionProfiles/Druid.xml b/.idea/inspectionProfiles/Druid.xml index 0889de4cc67a..34156d4cb2f1 100644 --- a/.idea/inspectionProfiles/Druid.xml +++ b/.idea/inspectionProfiles/Druid.xml @@ -39,9 +39,13 @@