diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionConfigBasedJobTemplate.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionConfigBasedJobTemplate.java
index 25643231b2c8..7bbf89854419 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionConfigBasedJobTemplate.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionConfigBasedJobTemplate.java
@@ -25,7 +25,7 @@
import org.apache.druid.indexing.input.DruidInputSource;
import org.apache.druid.java.util.common.Intervals;
import org.apache.druid.java.util.common.granularity.Granularity;
-import org.apache.druid.server.compaction.CompactionCandidate;
+import org.apache.druid.server.compaction.CompactionCandidateAndStatus;
import org.apache.druid.server.compaction.CompactionSlotManager;
import org.apache.druid.server.compaction.DataSourceCompactibleSegmentIterator;
import org.apache.druid.server.compaction.NewestSegmentFirstPolicy;
@@ -90,7 +90,7 @@ public List createCompactionJobs(
// Create a job for each CompactionCandidate
while (segmentIterator.hasNext()) {
- final CompactionCandidate candidate = segmentIterator.next();
+ final CompactionCandidateAndStatus candidate = segmentIterator.next();
// Allow template-specific customization of the config per candidate
DataSourceCompactionConfig finalConfig = configOptimizer.optimizeConfig(config, candidate, params);
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionJob.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionJob.java
index 7611485bd6d8..021e0de70c18 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionJob.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionJob.java
@@ -22,7 +22,7 @@
import org.apache.druid.client.indexing.ClientCompactionTaskQuery;
import org.apache.druid.indexing.template.BatchIndexingJob;
import org.apache.druid.query.http.ClientSqlQuery;
-import org.apache.druid.server.compaction.CompactionCandidate;
+import org.apache.druid.server.compaction.CompactionCandidateAndStatus;
import org.apache.druid.timeline.CompactionState;
/**
@@ -30,14 +30,14 @@
*/
public class CompactionJob extends BatchIndexingJob
{
- private final CompactionCandidate candidate;
+ private final CompactionCandidateAndStatus candidate;
private final int maxRequiredTaskSlots;
private final String targetIndexingStateFingerprint;
private final CompactionState targetIndexingState;
public CompactionJob(
ClientCompactionTaskQuery task,
- CompactionCandidate candidate,
+ CompactionCandidateAndStatus candidate,
int maxRequiredTaskSlots,
String targetIndexingStateFingerprint,
CompactionState targetIndexingState
@@ -52,7 +52,7 @@ public CompactionJob(
public CompactionJob(
ClientSqlQuery msqQuery,
- CompactionCandidate candidate,
+ CompactionCandidateAndStatus candidate,
int maxRequiredTaskSlots,
String targetIndexingStateFingerprint,
CompactionState targetIndexingState
@@ -70,7 +70,7 @@ public String getDataSource()
return candidate.getDataSource();
}
- public CompactionCandidate getCandidate()
+ public CompactionCandidateAndStatus getCandidate()
{
return candidate;
}
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionJobQueue.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionJobQueue.java
index 97cd15808f9f..66b6cc2faa9c 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionJobQueue.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionJobQueue.java
@@ -25,6 +25,7 @@
import org.apache.druid.client.indexing.ClientCompactionTaskQuery;
import org.apache.druid.client.indexing.ClientTaskQuery;
import org.apache.druid.common.guava.FutureUtils;
+import org.apache.druid.error.DruidException;
import org.apache.druid.indexer.TaskState;
import org.apache.druid.indexer.TaskStatus;
import org.apache.druid.indexing.common.actions.TaskActionClientFactory;
@@ -39,11 +40,10 @@
import org.apache.druid.segment.metadata.DefaultIndexingStateFingerprintMapper;
import org.apache.druid.segment.metadata.IndexingStateCache;
import org.apache.druid.segment.metadata.IndexingStateStorage;
-import org.apache.druid.server.compaction.CompactionCandidate;
+import org.apache.druid.server.compaction.CompactionCandidateAndStatus;
import org.apache.druid.server.compaction.CompactionCandidateSearchPolicy;
import org.apache.druid.server.compaction.CompactionSlotManager;
import org.apache.druid.server.compaction.CompactionSnapshotBuilder;
-import org.apache.druid.server.compaction.CompactionStatus;
import org.apache.druid.server.compaction.CompactionStatusTracker;
import org.apache.druid.server.coordinator.AutoCompactionSnapshot;
import org.apache.druid.server.coordinator.ClusterCompactionConfig;
@@ -161,7 +161,7 @@ public void createAndEnqueueJobs(
if (supervisor.shouldCreateJobs() && !activeSupervisors.contains(supervisorId)) {
// Queue fresh jobs
final List jobs = supervisor.createJobs(source, jobParams);
- jobs.forEach(job -> snapshotBuilder.addToPending(job.getCandidate()));
+ jobs.forEach(job -> snapshotBuilder.addToPending(job.getCandidate().getCandidate()));
queue.addAll(jobs);
activeSupervisors.add(supervisorId);
@@ -242,7 +242,7 @@ public void onTaskFinished(String taskId, TaskStatus taskStatus)
// Add this job back to the queue
queue.add(job);
} else {
- snapshotBuilder.moveFromPendingToCompleted(job.getCandidate());
+ snapshotBuilder.moveFromPendingToCompleted(job.getCandidate().getCandidate());
}
}
@@ -273,27 +273,32 @@ private boolean startJobIfPendingAndReady(
)
{
// Check if the job is a valid compaction job
- final CompactionCandidate candidate = job.getCandidate();
+ final CompactionCandidateAndStatus candidate = job.getCandidate();
final CompactionConfigValidationResult validationResult = validateCompactionJob(job);
if (!validationResult.isValid()) {
log.error("Skipping invalid compaction job[%s] due to reason[%s].", job, validationResult.getReason());
- snapshotBuilder.moveFromPendingToSkipped(candidate);
+ snapshotBuilder.moveFromPendingToSkipped(candidate.getCandidate());
return false;
}
- // Check if the job is already running, completed or skipped
- final CompactionStatus compactionStatus = statusTracker.computeCompactionStatus(job.getCandidate(), policy);
- switch (compactionStatus.getState()) {
- case RUNNING:
- return false;
- case COMPLETE:
- snapshotBuilder.moveFromPendingToCompleted(candidate);
- return false;
- case SKIPPED:
- snapshotBuilder.moveFromPendingToSkipped(candidate);
- return false;
- default:
- break;
+ CompactionCandidateSearchPolicy.Eligibility eligibility = policy.checkEligibilityForCompaction(candidate);
+ if (!eligibility.isEligible()) {
+ statusTracker.onSkippedCandidate(candidate, eligibility.getReason());
+ return false;
+ }
+ // Check if the job is already running or succeeded
+ final TaskState candidateState = statusTracker.computeCompactionTaskState(job.getCandidate().getCandidate());
+ if (candidateState != null) {
+ switch (candidateState) {
+ case RUNNING:
+ return false;
+ case SUCCESS:
+ snapshotBuilder.moveFromPendingToCompleted(candidate.getCandidate());
+ return false;
+ case FAILED:
+ default:
+ throw DruidException.defensive("unexpected compaction candidate state[%s]", candidateState);
+ }
}
// Check if enough compaction task slots are available
@@ -307,7 +312,7 @@ private boolean startJobIfPendingAndReady(
final String taskId = startTaskIfReady(job);
if (taskId == null) {
// Mark the job as skipped for now as the intervals might be locked by other tasks
- snapshotBuilder.moveFromPendingToSkipped(candidate);
+ snapshotBuilder.moveFromPendingToSkipped(candidate.getCandidate());
return false;
} else {
statusTracker.onTaskSubmitted(taskId, job.getCandidate());
@@ -341,7 +346,7 @@ private String startTaskIfReady(CompactionJob job)
log.debug(
"Checking readiness of task[%s] with interval[%s]",
- task.getId(), job.getCandidate().getCompactionInterval()
+ task.getId(), job.getCandidate().getCandidate().getCompactionInterval()
);
try {
taskLockbox.add(task);
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/OverlordCompactionScheduler.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/OverlordCompactionScheduler.java
index 11709e616c71..0f524c4e7613 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/compact/OverlordCompactionScheduler.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/OverlordCompactionScheduler.java
@@ -499,7 +499,7 @@ public CompactionSimulateResult simulateRunWithConfigUpdate(ClusterCompactionCon
updateRequest.getEngine()
);
} else {
- return new CompactionSimulateResult(Collections.emptyMap());
+ return new CompactionSimulateResult(Collections.emptyMap(), null, null);
}
}
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/ReindexingConfigOptimizer.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/ReindexingConfigOptimizer.java
index d091d5433f22..f6d165a9a56a 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/compact/ReindexingConfigOptimizer.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/ReindexingConfigOptimizer.java
@@ -20,6 +20,7 @@
package org.apache.druid.indexing.compact;
import org.apache.druid.server.compaction.CompactionCandidate;
+import org.apache.druid.server.compaction.CompactionCandidateAndStatus;
import org.apache.druid.server.coordinator.DataSourceCompactionConfig;
/**
@@ -43,7 +44,7 @@ public interface ReindexingConfigOptimizer
*/
DataSourceCompactionConfig optimizeConfig(
DataSourceCompactionConfig config,
- CompactionCandidate candidate,
+ CompactionCandidateAndStatus candidate,
CompactionJobParams params
);
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/ReindexingDeletionRuleOptimizer.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/ReindexingDeletionRuleOptimizer.java
index 9b4bbb597977..ce8141cfcd45 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/compact/ReindexingDeletionRuleOptimizer.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/ReindexingDeletionRuleOptimizer.java
@@ -28,6 +28,7 @@
import org.apache.druid.segment.metadata.IndexingStateFingerprintMapper;
import org.apache.druid.segment.transform.CompactionTransformSpec;
import org.apache.druid.server.compaction.CompactionCandidate;
+import org.apache.druid.server.compaction.CompactionCandidateAndStatus;
import org.apache.druid.server.compaction.CompactionStatus;
import org.apache.druid.server.compaction.ReindexingDeletionRule;
import org.apache.druid.server.coordinator.DataSourceCompactionConfig;
@@ -59,7 +60,7 @@ public class ReindexingDeletionRuleOptimizer implements ReindexingConfigOptimize
@Override
public DataSourceCompactionConfig optimizeConfig(
DataSourceCompactionConfig config,
- CompactionCandidate candidate,
+ CompactionCandidateAndStatus candidate,
CompactionJobParams params
)
{
@@ -96,13 +97,13 @@ public DataSourceCompactionConfig optimizeConfig(
*
*
* @param candidateSegments the {@link CompactionCandidate}
- * @param expectedFilter the expected filter (as a NotDimFilter wrapping an OrDimFilter)
+ * @param expectedFilter the expected filter (as a NotDimFilter wrapping an OrDimFilter)
* @param fingerprintMapper the fingerprint mapper to retrieve applied rules from segment fingerprints
* @return the set of unapplied deletion rules wrapped in a NotDimFilter, or null if all rules have been applied
*/
@Nullable
private NotDimFilter computeRequiredSetOfFilterRulesForCandidate(
- CompactionCandidate candidateSegments,
+ CompactionCandidateAndStatus candidateSegments,
NotDimFilter expectedFilter,
IndexingStateFingerprintMapper fingerprintMapper
)
@@ -114,7 +115,7 @@ private NotDimFilter computeRequiredSetOfFilterRulesForCandidate(
expectedFilters = ((OrDimFilter) expectedFilter.getField()).getFields();
}
- Set uniqueFingerprints = candidateSegments.getSegments().stream()
+ Set uniqueFingerprints = candidateSegments.getCandidate().getSegments().stream()
.map(DataSegment::getIndexingStateFingerprint)
.filter(Objects::nonNull)
.collect(Collectors.toSet());
@@ -229,15 +230,15 @@ private static Set extractAppliedFilters(CompactionState state)
* Returns true only if the candidate has been compacted before and has a NotDimFilter.
*/
private boolean shouldOptimizeFilterRules(
- CompactionCandidate candidate,
+ CompactionCandidateAndStatus candidate,
DataSourceCompactionConfig config
)
{
- if (candidate.getCurrentStatus() == null) {
+ if (CompactionStatus.State.NOT_ELIGIBLE.equals(candidate.getStatus().getState())) {
return false;
}
-
- if (candidate.getCurrentStatus().getReason().equals(CompactionStatus.NEVER_COMPACTED_REASON)) {
+ if (CompactionStatus.State.ELIGIBLE.equals(candidate.getStatus().getState())
+ && CompactionStatus.NEVER_COMPACTED_REASON.equals(candidate.getStatus().getReason())) {
return false;
}
diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/compact/OverlordCompactionSchedulerTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/compact/OverlordCompactionSchedulerTest.java
index 4c98b52f48c1..dadda3241df9 100644
--- a/indexing-service/src/test/java/org/apache/druid/indexing/compact/OverlordCompactionSchedulerTest.java
+++ b/indexing-service/src/test/java/org/apache/druid/indexing/compact/OverlordCompactionSchedulerTest.java
@@ -68,7 +68,6 @@
import org.apache.druid.segment.metadata.IndexingStateCache;
import org.apache.druid.server.compaction.CompactionSimulateResult;
import org.apache.druid.server.compaction.CompactionStatistics;
-import org.apache.druid.server.compaction.CompactionStatus;
import org.apache.druid.server.compaction.CompactionStatusTracker;
import org.apache.druid.server.compaction.Table;
import org.apache.druid.server.coordinator.AutoCompactionSnapshot;
@@ -450,8 +449,8 @@ public void test_simulateRunWithConfigUpdate()
final CompactionSimulateResult simulateResult = scheduler.simulateRunWithConfigUpdate(
new ClusterCompactionConfig(null, null, null, null, null, null)
);
- Assert.assertEquals(1, simulateResult.getCompactionStates().size());
- final Table pendingCompactionTable = simulateResult.getCompactionStates().get(CompactionStatus.State.PENDING);
+ Assert.assertEquals(0, simulateResult.getCompactionStates().size());
+ final Table pendingCompactionTable = simulateResult.getQueuedIntervals();
Assert.assertEquals(
Arrays.asList("dataSource", "interval", "numSegments", "bytes", "maxTaskSlots", "reasonToCompact"),
pendingCompactionTable.getColumnNames()
diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/compact/ReindexingDeletionRuleOptimizerTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/compact/ReindexingDeletionRuleOptimizerTest.java
index 47b69031218a..4e3aaf52f7dd 100644
--- a/indexing-service/src/test/java/org/apache/druid/indexing/compact/ReindexingDeletionRuleOptimizerTest.java
+++ b/indexing-service/src/test/java/org/apache/druid/indexing/compact/ReindexingDeletionRuleOptimizerTest.java
@@ -39,6 +39,7 @@
import org.apache.druid.segment.transform.CompactionTransformSpec;
import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
import org.apache.druid.server.compaction.CompactionCandidate;
+import org.apache.druid.server.compaction.CompactionCandidateAndStatus;
import org.apache.druid.server.compaction.CompactionStatus;
import org.apache.druid.server.coordinator.DataSourceCompactionConfig;
import org.apache.druid.server.coordinator.InlineSchemaDataSourceCompactionConfig;
@@ -89,8 +90,8 @@ public void testOptimize_SingleFilter_NotOrFilter_NoFingerprints_ReturnsUnchange
DimFilter filterA = new SelectorDimFilter("country", "US", null);
NotDimFilter expectedFilter = new NotDimFilter(filterA);
- CompactionCandidate candidate = createCandidateWithFingerprints("fp1");
InlineSchemaDataSourceCompactionConfig config = createConfigWithFilter(expectedFilter, null);
+ CompactionCandidateAndStatus candidate = createCandidateWithFingerprints(config, "fp1");
CompactionJobParams params = createParams();
DataSourceCompactionConfig result = optimizer.optimizeConfig(config, candidate, params);
@@ -108,8 +109,8 @@ public void testOptimize_SingleFilter_AlreadyApplied_RemovesTransformSpec()
syncCacheFromManager();
NotDimFilter expectedFilter = new NotDimFilter(filterB);
- CompactionCandidate candidate = createCandidateWithFingerprints("fp1");
InlineSchemaDataSourceCompactionConfig config = createConfigWithFilter(expectedFilter, null);
+ CompactionCandidateAndStatus candidate = createCandidateWithFingerprints(config, "fp1");
CompactionJobParams params = createParams();
DataSourceCompactionConfig result = optimizer.optimizeConfig(config, candidate, params);
@@ -130,8 +131,8 @@ public void testOptimize_AllFiltersAlreadyApplied_RemovesTransformSpec()
syncCacheFromManager();
NotDimFilter expectedFilter = new NotDimFilter(new OrDimFilter(Arrays.asList(filterA, filterB, filterC)));
- CompactionCandidate candidate = createCandidateWithFingerprints("fp1");
InlineSchemaDataSourceCompactionConfig config = createConfigWithFilter(expectedFilter, null);
+ CompactionCandidateAndStatus candidate = createCandidateWithFingerprints(config, "fp1");
CompactionJobParams params = createParams();
DataSourceCompactionConfig result = optimizer.optimizeConfig(config, candidate, params);
@@ -152,8 +153,8 @@ public void testOptimize_NoFiltersApplied_ReturnsAllExpected()
syncCacheFromManager();
NotDimFilter expectedFilter = new NotDimFilter(new OrDimFilter(Arrays.asList(filterA, filterB, filterC)));
- CompactionCandidate candidate = createCandidateWithFingerprints("fp1");
InlineSchemaDataSourceCompactionConfig config = createConfigWithFilter(expectedFilter, null);
+ CompactionCandidateAndStatus candidate = createCandidateWithFingerprints(config, "fp1");
CompactionJobParams params = createParams();
DataSourceCompactionConfig result = optimizer.optimizeConfig(config, candidate, params);
@@ -180,8 +181,8 @@ public void testOptimize_PartiallyApplied_ReturnsDelta()
NotDimFilter expectedFilter = new NotDimFilter(
new OrDimFilter(Arrays.asList(filterA, filterB, filterC, filterD))
);
- CompactionCandidate candidate = createCandidateWithFingerprints("fp1");
InlineSchemaDataSourceCompactionConfig config = createConfigWithFilter(expectedFilter, null);
+ CompactionCandidateAndStatus candidate = createCandidateWithFingerprints(config, "fp1");
CompactionJobParams params = createParams();
DataSourceCompactionConfig result = optimizer.optimizeConfig(config, candidate, params);
@@ -213,8 +214,8 @@ public void testOptimize_MultipleFingerprints_UnionOfMissing()
NotDimFilter expectedFilter = new NotDimFilter(
new OrDimFilter(Arrays.asList(filterA, filterB, filterC, filterD))
);
- CompactionCandidate candidate = createCandidateWithFingerprints("fp1", "fp2");
InlineSchemaDataSourceCompactionConfig config = createConfigWithFilter(expectedFilter, null);
+ CompactionCandidateAndStatus candidate = createCandidateWithFingerprints(config, "fp1", "fp2");
CompactionJobParams params = createParams();
DataSourceCompactionConfig result = optimizer.optimizeConfig(config, candidate, params);
@@ -246,8 +247,8 @@ public void testOptimize_MultipleFingerprints_NoDuplicates()
NotDimFilter expectedFilter = new NotDimFilter(
new OrDimFilter(Arrays.asList(filterA, filterB, filterC))
);
- CompactionCandidate candidate = createCandidateWithFingerprints("fp1", "fp2");
InlineSchemaDataSourceCompactionConfig config = createConfigWithFilter(expectedFilter, null);
+ CompactionCandidateAndStatus candidate = createCandidateWithFingerprints(config, "fp1", "fp2");
CompactionJobParams params = createParams();
DataSourceCompactionConfig result = optimizer.optimizeConfig(config, candidate, params);
@@ -272,8 +273,8 @@ public void testOptimize_MissingCompactionState_ReturnsAllFilters()
NotDimFilter expectedFilter = new NotDimFilter(
new OrDimFilter(Arrays.asList(filterA, filterB, filterC))
);
- CompactionCandidate candidate = createCandidateWithFingerprints("fp1");
InlineSchemaDataSourceCompactionConfig config = createConfigWithFilter(expectedFilter, null);
+ CompactionCandidateAndStatus candidate = createCandidateWithFingerprints(config, "fp1");
CompactionJobParams params = createParams();
DataSourceCompactionConfig result = optimizer.optimizeConfig(config, candidate, params);
@@ -296,8 +297,8 @@ public void testOptimize_TransformSpecWithSingleFilter()
NotDimFilter expectedFilter = new NotDimFilter(
new OrDimFilter(Arrays.asList(filterA, filterB, filterC))
);
- CompactionCandidate candidate = createCandidateWithFingerprints("fp1");
InlineSchemaDataSourceCompactionConfig config = createConfigWithFilter(expectedFilter, null);
+ CompactionCandidateAndStatus candidate = createCandidateWithFingerprints(config, "fp1");
CompactionJobParams params = createParams();
DataSourceCompactionConfig result = optimizer.optimizeConfig(config, candidate, params);
@@ -315,13 +316,11 @@ public void testOptimize_SegmentsWithNoFingerprints()
DimFilter filterA = new SelectorDimFilter("country", "US", null);
DimFilter filterB = new SelectorDimFilter("country", "UK", null);
DimFilter filterC = new SelectorDimFilter("country", "FR", null);
-
- CompactionCandidate candidate = createCandidateWithNullFingerprints(3);
-
NotDimFilter expectedFilter = new NotDimFilter(
new OrDimFilter(Arrays.asList(filterA, filterB, filterC))
);
InlineSchemaDataSourceCompactionConfig config = createConfigWithFilter(expectedFilter, null);
+ CompactionCandidateAndStatus candidate = createCandidateWithNullFingerprints(config, 3);
CompactionJobParams params = createParams();
DataSourceCompactionConfig result = optimizer.optimizeConfig(config, candidate, params);
@@ -331,8 +330,6 @@ public void testOptimize_SegmentsWithNoFingerprints()
}
-
-
// Helper methods
private InlineSchemaDataSourceCompactionConfig createConfigWithFilter(
@@ -345,9 +342,9 @@ private InlineSchemaDataSourceCompactionConfig createConfigWithFilter(
: new CompactionTransformSpec(filter, virtualColumns);
return InlineSchemaDataSourceCompactionConfig.builder()
- .forDataSource(TestDataSource.WIKI)
- .withTransformSpec(transformSpec)
- .build();
+ .forDataSource(TestDataSource.WIKI)
+ .withTransformSpec(transformSpec)
+ .build();
}
private CompactionJobParams createParams()
@@ -358,23 +355,27 @@ private CompactionJobParams createParams()
return mockParams;
}
- private CompactionCandidate createCandidateWithFingerprints(String... fingerprints)
+ private CompactionCandidateAndStatus createCandidateWithFingerprints(DataSourceCompactionConfig config, String... fingerprints)
{
- List segments = Arrays.stream(fingerprints)
- .map(fp -> DataSegment.builder(WIKI_SEGMENT).indexingStateFingerprint(fp).build())
- .collect(Collectors.toList());
- return CompactionCandidate.from(segments, null)
- .withCurrentStatus(CompactionStatus.pending("segments need compaction"));
+ List segments =
+ Arrays.stream(fingerprints)
+ .map(fp -> DataSegment.builder(WIKI_SEGMENT).indexingStateFingerprint(fp).build())
+ .collect(Collectors.toList());
+ CompactionCandidate proposed = CompactionCandidate.from(segments, null);
+ return new CompactionCandidateAndStatus(proposed, CompactionStatus.compute(proposed, config, fingerprintMapper));
}
- private CompactionCandidate createCandidateWithNullFingerprints(int count)
+ private CompactionCandidateAndStatus createCandidateWithNullFingerprints(DataSourceCompactionConfig config, int count)
{
List segments = new ArrayList<>();
for (int i = 0; i < count; i++) {
- segments.add(DataSegment.builder(WIKI_SEGMENT).indexingStateFingerprint(null).build());
+ segments.add(DataSegment.builder(WIKI_SEGMENT)
+ .indexingStateFingerprint(null)
+ .lastCompactionState(CompactionState.builder().indexSpec(IndexSpec.getDefault()).build())
+ .build());
}
- return CompactionCandidate.from(segments, null)
- .withCurrentStatus(CompactionStatus.pending("segments need compaction"));
+ CompactionCandidate proposed = CompactionCandidate.from(segments, null);
+ return new CompactionCandidateAndStatus(proposed, CompactionStatus.compute(proposed, config, fingerprintMapper));
}
private CompactionState createStateWithFilters(DimFilter... filters)
@@ -445,8 +446,8 @@ public void testOptimize_FilterVirtualColumns_SomeColumnsReferenced()
NotDimFilter notFilter = new NotDimFilter(filter);
// Candidate has no filters applied, so all filters remain
- CompactionCandidate candidate = createCandidateWithNullFingerprints(1);
InlineSchemaDataSourceCompactionConfig config = createConfigWithFilter(notFilter, virtualColumns);
+ CompactionCandidateAndStatus candidate = createCandidateWithNullFingerprints(config, 1);
CompactionJobParams params = createParams();
DataSourceCompactionConfig result = optimizer.optimizeConfig(config, candidate, params);
@@ -482,8 +483,8 @@ public void testOptimize_FilterVirtualColumns_NoColumnsReferenced()
NotDimFilter notFilter = new NotDimFilter(filter);
// Candidate has no filters applied
- CompactionCandidate candidate = createCandidateWithNullFingerprints(1);
InlineSchemaDataSourceCompactionConfig config = createConfigWithFilter(notFilter, virtualColumns);
+ CompactionCandidateAndStatus candidate = createCandidateWithNullFingerprints(config, 1);
CompactionJobParams params = createParams();
DataSourceCompactionConfig result = optimizer.optimizeConfig(config, candidate, params);
@@ -500,13 +501,13 @@ public void testOptimize_CandidateNeverCompacted_NoOptimization()
// Candidate with NEVER_COMPACTED status
List segments = new ArrayList<>();
segments.add(DataSegment.builder(WIKI_SEGMENT).indexingStateFingerprint(null).build());
- CompactionCandidate candidate = CompactionCandidate.from(segments, null)
- .withCurrentStatus(CompactionStatus.pending(CompactionStatus.NEVER_COMPACTED_REASON));
InlineSchemaDataSourceCompactionConfig config = createConfigWithFilter(expectedFilter, null);
+ CompactionCandidate candidate = CompactionCandidate.from(segments, null);
+ CompactionCandidateAndStatus candidateAndStatus = new CompactionCandidateAndStatus(candidate, CompactionStatus.compute(candidate, config, fingerprintMapper));
CompactionJobParams params = createParams();
- DataSourceCompactionConfig result = optimizer.optimizeConfig(config, candidate, params);
+ DataSourceCompactionConfig result = optimizer.optimizeConfig(config, candidateAndStatus, params);
// Should return config unchanged since candidate was never compacted
Assertions.assertSame(config, result);
@@ -517,10 +518,10 @@ public void testOptimize_NoTransformSpec_NoOptimization()
{
// Config without transform spec
InlineSchemaDataSourceCompactionConfig config = InlineSchemaDataSourceCompactionConfig.builder()
- .forDataSource(TestDataSource.WIKI)
- .build();
+ .forDataSource(TestDataSource.WIKI)
+ .build();
- CompactionCandidate candidate = createCandidateWithFingerprints("fp1");
+ CompactionCandidateAndStatus candidate = createCandidateWithFingerprints(config, "fp1");
CompactionJobParams params = createParams();
DataSourceCompactionConfig result = optimizer.optimizeConfig(config, candidate, params);
diff --git a/processing/src/main/java/org/apache/druid/timeline/CompactionState.java b/processing/src/main/java/org/apache/druid/timeline/CompactionState.java
index f5d5ecf73622..8a70ccd70075 100644
--- a/processing/src/main/java/org/apache/druid/timeline/CompactionState.java
+++ b/processing/src/main/java/org/apache/druid/timeline/CompactionState.java
@@ -213,6 +213,11 @@ public Builder toBuilder()
return new Builder(this);
}
+ public static Builder builder()
+ {
+ return new Builder();
+ }
+
public static class Builder
{
private PartitionsSpec partitionsSpec;
diff --git a/server/src/main/java/org/apache/druid/server/compaction/BaseCandidateSearchPolicy.java b/server/src/main/java/org/apache/druid/server/compaction/BaseCandidateSearchPolicy.java
index 2a9107132623..b48af469465e 100644
--- a/server/src/main/java/org/apache/druid/server/compaction/BaseCandidateSearchPolicy.java
+++ b/server/src/main/java/org/apache/druid/server/compaction/BaseCandidateSearchPolicy.java
@@ -36,7 +36,7 @@
public abstract class BaseCandidateSearchPolicy implements CompactionCandidateSearchPolicy
{
private final String priorityDatasource;
- private final Comparator comparator;
+ private final Comparator comparator;
protected BaseCandidateSearchPolicy(@Nullable String priorityDatasource)
{
@@ -45,7 +45,7 @@ protected BaseCandidateSearchPolicy(@Nullable String priorityDatasource)
this.comparator = getSegmentComparator();
} else {
this.comparator = Comparators.alwaysFirst(priorityDatasource)
- .onResultOf(CompactionCandidate::getDataSource)
+ .onResultOf(CompactionCandidateAndStatus::getDataSource)
.thenComparing(getSegmentComparator());
}
}
@@ -62,16 +62,13 @@ public final String getPriorityDatasource()
}
@Override
- public final int compareCandidates(CompactionCandidate o1, CompactionCandidate o2)
+ public final int compareCandidates(CompactionCandidateAndStatus o1, CompactionCandidateAndStatus o2)
{
return comparator.compare(o1, o2);
}
@Override
- public Eligibility checkEligibilityForCompaction(
- CompactionCandidate candidate,
- CompactionTaskStatus latestTaskStatus
- )
+ public Eligibility checkEligibilityForCompaction(CompactionCandidateAndStatus candidate)
{
return Eligibility.OK;
}
@@ -80,7 +77,7 @@ public Eligibility checkEligibilityForCompaction(
* Compares between two compaction candidates. Used to determine the
* order in which segments and intervals should be picked for compaction.
*/
- protected abstract Comparator getSegmentComparator();
+ protected abstract Comparator getSegmentComparator();
@Override
public boolean equals(Object o)
diff --git a/server/src/main/java/org/apache/druid/server/compaction/CompactionCandidate.java b/server/src/main/java/org/apache/druid/server/compaction/CompactionCandidate.java
index af8b32ebe6db..b82126573b8c 100644
--- a/server/src/main/java/org/apache/druid/server/compaction/CompactionCandidate.java
+++ b/server/src/main/java/org/apache/druid/server/compaction/CompactionCandidate.java
@@ -44,8 +44,6 @@ public class CompactionCandidate
private final long totalBytes;
private final int numIntervals;
- private final CompactionStatus currentStatus;
-
public static CompactionCandidate from(
List segments,
@Nullable Granularity targetSegmentGranularity
@@ -67,8 +65,7 @@ public static CompactionCandidate from(
segments,
umbrellaInterval,
compactionInterval,
- segmentIntervals.size(),
- null
+ segmentIntervals.size()
);
}
@@ -76,8 +73,7 @@ private CompactionCandidate(
List segments,
Interval umbrellaInterval,
Interval compactionInterval,
- int numDistinctSegmentIntervals,
- @Nullable CompactionStatus currentStatus
+ int numDistinctSegmentIntervals
)
{
this.segments = segments;
@@ -88,7 +84,6 @@ private CompactionCandidate(
this.numIntervals = numDistinctSegmentIntervals;
this.dataSource = segments.get(0).getDataSource();
- this.currentStatus = currentStatus;
}
/**
@@ -137,37 +132,6 @@ public CompactionStatistics getStats()
return CompactionStatistics.create(totalBytes, numSegments(), numIntervals);
}
- @Nullable
- public CompactionStatistics getCompactedStats()
- {
- return (currentStatus == null || currentStatus.getCompactedStats() == null)
- ? null : currentStatus.getCompactedStats();
- }
-
- @Nullable
- public CompactionStatistics getUncompactedStats()
- {
- return (currentStatus == null || currentStatus.getUncompactedStats() == null)
- ? null : currentStatus.getUncompactedStats();
- }
-
- /**
- * Current compaction status of the time chunk corresponding to this candidate.
- */
- @Nullable
- public CompactionStatus getCurrentStatus()
- {
- return currentStatus;
- }
-
- /**
- * Creates a copy of this CompactionCandidate object with the given status.
- */
- public CompactionCandidate withCurrentStatus(CompactionStatus status)
- {
- return new CompactionCandidate(segments, umbrellaInterval, compactionInterval, numIntervals, status);
- }
-
@Override
public String toString()
{
@@ -175,7 +139,6 @@ public String toString()
"datasource=" + dataSource +
", segments=" + SegmentUtils.commaSeparatedIdentifiers(segments) +
", totalSize=" + totalBytes +
- ", currentStatus=" + currentStatus +
'}';
}
}
diff --git a/server/src/main/java/org/apache/druid/server/compaction/CompactionCandidateAndStatus.java b/server/src/main/java/org/apache/druid/server/compaction/CompactionCandidateAndStatus.java
new file mode 100644
index 000000000000..b3c83dacb1c1
--- /dev/null
+++ b/server/src/main/java/org/apache/druid/server/compaction/CompactionCandidateAndStatus.java
@@ -0,0 +1,84 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.server.compaction;
+
+import java.util.Objects;
+
+/**
+ * Wrapper class that combines a compaction candidate with its evaluated status.
+ */
+public class CompactionCandidateAndStatus
+{
+ private final CompactionCandidate candidate;
+ private final CompactionStatus status;
+
+ public CompactionCandidateAndStatus(
+ CompactionCandidate candidate,
+ CompactionStatus status
+ )
+ {
+ this.candidate = candidate;
+ this.status = status;
+ }
+
+ public CompactionCandidate getCandidate()
+ {
+ return candidate;
+ }
+
+ public CompactionStatus getStatus()
+ {
+ return status;
+ }
+
+ public String getDataSource()
+ {
+ return candidate.getDataSource();
+ }
+
+ @Override
+ public boolean equals(Object object)
+ {
+ if (this == object) {
+ return true;
+ }
+ if (object == null || getClass() != object.getClass()) {
+ return false;
+ }
+ CompactionCandidateAndStatus that = (CompactionCandidateAndStatus) object;
+ return Objects.equals(candidate, that.candidate)
+ && Objects.equals(status, that.status);
+ }
+
+ @Override
+ public int hashCode()
+ {
+ return Objects.hash(candidate, status);
+ }
+
+ @Override
+ public String toString()
+ {
+ return "CompactionCandidateAndStatus{"
+ + "candidate=" + candidate
+ + ", status=" + status
+ + '}';
+ }
+}
diff --git a/server/src/main/java/org/apache/druid/server/compaction/CompactionCandidateSearchPolicy.java b/server/src/main/java/org/apache/druid/server/compaction/CompactionCandidateSearchPolicy.java
index eeb5d612b0e4..5749419ddd26 100644
--- a/server/src/main/java/org/apache/druid/server/compaction/CompactionCandidateSearchPolicy.java
+++ b/server/src/main/java/org/apache/druid/server/compaction/CompactionCandidateSearchPolicy.java
@@ -46,19 +46,16 @@ public interface CompactionCandidateSearchPolicy
* positive value if {@code candidateB} should be picked first or zero if the
* order does not matter.
*/
- int compareCandidates(CompactionCandidate candidateA, CompactionCandidate candidateB);
+ int compareCandidates(CompactionCandidateAndStatus candidateA, CompactionCandidateAndStatus candidateB);
/**
- * Checks if the given {@link CompactionCandidate} is eligible for compaction
+ * Checks if the given {@link CompactionCandidateAndStatus} is eligible for compaction
* in the current iteration. A policy may implement this method to skip
* compacting intervals or segments that do not fulfil some required criteria.
*
* @return {@link Eligibility#OK} only if eligible.
*/
- Eligibility checkEligibilityForCompaction(
- CompactionCandidate candidate,
- CompactionTaskStatus latestTaskStatus
- );
+ Eligibility checkEligibilityForCompaction(CompactionCandidateAndStatus candidateAndStatus);
/**
* Describes the eligibility of an interval for compaction.
diff --git a/server/src/main/java/org/apache/druid/server/compaction/CompactionRunSimulator.java b/server/src/main/java/org/apache/druid/server/compaction/CompactionRunSimulator.java
index 586f16c3b23f..dd57c8e7b3fd 100644
--- a/server/src/main/java/org/apache/druid/server/compaction/CompactionRunSimulator.java
+++ b/server/src/main/java/org/apache/druid/server/compaction/CompactionRunSimulator.java
@@ -25,7 +25,10 @@
import org.apache.druid.client.indexing.ClientCompactionTaskQueryTuningConfig;
import org.apache.druid.client.indexing.IndexingTotalWorkerCapacityInfo;
import org.apache.druid.client.indexing.TaskPayloadResponse;
+import org.apache.druid.common.guava.GuavaUtils;
+import org.apache.druid.error.DruidException;
import org.apache.druid.indexer.CompactionEngine;
+import org.apache.druid.indexer.TaskState;
import org.apache.druid.indexer.TaskStatus;
import org.apache.druid.indexer.TaskStatusPlus;
import org.apache.druid.java.util.common.parsers.CloseableIterator;
@@ -92,47 +95,56 @@ public CompactionSimulateResult simulateRunWithConfig(
// account for the active tasks
final CompactionStatusTracker simulationStatusTracker = new CompactionStatusTracker()
{
+
@Override
- public CompactionStatus computeCompactionStatus(
- CompactionCandidate candidate,
- CompactionCandidateSearchPolicy searchPolicy
+ public void onSkippedCandidate(
+ CompactionCandidateAndStatus candidateSegments,
+ @Nullable String policyNote
)
{
- return statusTracker.computeCompactionStatus(candidate, searchPolicy);
+ skippedIntervals.addRow(createRow(
+ candidateSegments.getCandidate(),
+ null,
+ GuavaUtils.firstNonNull(policyNote, candidateSegments.getStatus().getReason())
+ ));
}
@Override
- public void onCompactionStatusComputed(
- CompactionCandidate candidateSegments,
+ public void onCompactionTaskStateComputed(
+ CompactionCandidateAndStatus candidateSegments,
+ TaskState taskState,
DataSourceCompactionConfig config
)
{
- final CompactionStatus status = candidateSegments.getCurrentStatus();
- if (status == null) {
- // do nothing
- } else if (status.getState() == CompactionStatus.State.COMPLETE) {
- compactedIntervals.addRow(
- createRow(candidateSegments, null, null)
- );
- } else if (status.getState() == CompactionStatus.State.RUNNING) {
- runningIntervals.addRow(
- createRow(candidateSegments, ClientCompactionTaskQueryTuningConfig.from(config), status.getReason())
- );
- } else if (status.getState() == CompactionStatus.State.SKIPPED) {
- skippedIntervals.addRow(
- createRow(candidateSegments, null, status.getReason())
- );
+ if (taskState == null) {
+ return;
+ }
+ switch (taskState) {
+ case SUCCESS:
+ compactedIntervals.addRow(createRow(candidateSegments.getCandidate(), null, null));
+ break;
+ case RUNNING:
+ runningIntervals.addRow(createRow(
+ candidateSegments.getCandidate(),
+ ClientCompactionTaskQueryTuningConfig.from(config),
+ candidateSegments.getStatus().getReason()
+ ));
+ break;
+ case FAILED:
+ default:
+ throw DruidException.defensive("unknown compaction task state[%s]", taskState);
}
}
@Override
- public void onTaskSubmitted(String taskId, CompactionCandidate candidateSegments)
+ public void onTaskSubmitted(String taskId, CompactionCandidateAndStatus candidateSegments)
{
// Add a row for each task in order of submission
- final CompactionStatus status = candidateSegments.getCurrentStatus();
- queuedIntervals.addRow(
- createRow(candidateSegments, null, status == null ? "" : status.getReason())
- );
+ queuedIntervals.addRow(createRow(
+ candidateSegments.getCandidate(),
+ null,
+ candidateSegments.getStatus().getReason()
+ ));
}
};
@@ -155,21 +167,19 @@ public void onTaskSubmitted(String taskId, CompactionCandidate candidateSegments
stats
);
- final Map compactionStates = new HashMap<>();
+ final Map compactionStates = new HashMap<>();
if (!compactedIntervals.isEmpty()) {
- compactionStates.put(CompactionStatus.State.COMPLETE, compactedIntervals);
+ compactionStates.put(TaskState.SUCCESS, compactedIntervals);
}
if (!runningIntervals.isEmpty()) {
- compactionStates.put(CompactionStatus.State.RUNNING, runningIntervals);
- }
- if (!queuedIntervals.isEmpty()) {
- compactionStates.put(CompactionStatus.State.PENDING, queuedIntervals);
- }
- if (!skippedIntervals.isEmpty()) {
- compactionStates.put(CompactionStatus.State.SKIPPED, skippedIntervals);
+ compactionStates.put(TaskState.RUNNING, runningIntervals);
}
- return new CompactionSimulateResult(compactionStates);
+ return new CompactionSimulateResult(
+ compactionStates,
+ skippedIntervals,
+ queuedIntervals.isEmpty() ? null : queuedIntervals
+ );
}
private Object[] createRow(
diff --git a/server/src/main/java/org/apache/druid/server/compaction/CompactionSegmentIterator.java b/server/src/main/java/org/apache/druid/server/compaction/CompactionSegmentIterator.java
index 295aa2881adf..ba35c88b1cc4 100644
--- a/server/src/main/java/org/apache/druid/server/compaction/CompactionSegmentIterator.java
+++ b/server/src/main/java/org/apache/druid/server/compaction/CompactionSegmentIterator.java
@@ -25,7 +25,7 @@
/**
* Iterator over compactible segments.
*/
-public interface CompactionSegmentIterator extends Iterator
+public interface CompactionSegmentIterator extends Iterator
{
/**
* List of candidate segments that are already compacted and do not need to be
@@ -38,6 +38,6 @@ public interface CompactionSegmentIterator extends Iterator
* cannot be compacted due to some reason. None of these segments are returned
* by {@link #next()}.
*/
- List getSkippedSegments();
+ List getSkippedSegments();
}
diff --git a/server/src/main/java/org/apache/druid/server/compaction/CompactionSimulateResult.java b/server/src/main/java/org/apache/druid/server/compaction/CompactionSimulateResult.java
index 7a48ccf0e5ba..54f65ef0285e 100644
--- a/server/src/main/java/org/apache/druid/server/compaction/CompactionSimulateResult.java
+++ b/server/src/main/java/org/apache/druid/server/compaction/CompactionSimulateResult.java
@@ -21,24 +21,43 @@
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
+import org.apache.druid.indexer.TaskState;
import java.util.Map;
public class CompactionSimulateResult
{
- private final Map compactionStates;
+ private final Map compactionStates;
+ private final Table skippedIntervals;
+ private final Table queuedIntervals;
@JsonCreator
public CompactionSimulateResult(
- @JsonProperty("compactionStates") Map compactionStates
+ @JsonProperty("compactionStates") Map compactionStates,
+ @JsonProperty("skippedIntervals") Table skippedIntervals,
+ @JsonProperty("queuedIntervals") Table queuedIntervals
)
{
this.compactionStates = compactionStates;
+ this.skippedIntervals = skippedIntervals;
+ this.queuedIntervals = queuedIntervals;
}
@JsonProperty
- public Map getCompactionStates()
+ public Map getCompactionStates()
{
return compactionStates;
}
+
+ @JsonProperty
+ public Table getSkippedIntervals()
+ {
+ return skippedIntervals;
+ }
+
+ @JsonProperty
+ public Table getQueuedIntervals()
+ {
+ return queuedIntervals;
+ }
}
diff --git a/server/src/main/java/org/apache/druid/server/compaction/CompactionStatus.java b/server/src/main/java/org/apache/druid/server/compaction/CompactionStatus.java
index 10076a719241..297143fa9b6a 100644
--- a/server/src/main/java/org/apache/druid/server/compaction/CompactionStatus.java
+++ b/server/src/main/java/org/apache/druid/server/compaction/CompactionStatus.java
@@ -19,10 +19,13 @@
package org.apache.druid.server.compaction;
+import com.google.common.base.Strings;
import org.apache.commons.lang3.ArrayUtils;
import org.apache.druid.client.indexing.ClientCompactionTaskQueryTuningConfig;
import org.apache.druid.common.config.Configs;
import org.apache.druid.data.input.impl.DimensionSchema;
+import org.apache.druid.error.DruidException;
+import org.apache.druid.error.InvalidInput;
import org.apache.druid.indexer.partitions.DimensionRangePartitionsSpec;
import org.apache.druid.indexer.partitions.DynamicPartitionsSpec;
import org.apache.druid.indexer.partitions.HashedPartitionsSpec;
@@ -48,29 +51,28 @@
import java.util.HashMap;
import java.util.List;
import java.util.Map;
+import java.util.Objects;
import java.util.Set;
import java.util.function.Function;
import java.util.stream.Collectors;
/**
- * Represents the status of compaction for a given {@link CompactionCandidate}.
+ * Describes the eligibility of an interval for compaction.
*/
public class CompactionStatus
{
- private static final Logger log = new Logger(CompactionStatus.class);
-
- private static final CompactionStatus COMPLETE = new CompactionStatus(State.COMPLETE, null, null, null);
+ public static final CompactionStatus COMPLETE = new CompactionStatus(State.COMPLETE, "", null, null);
public static final String NEVER_COMPACTED_REASON = "not compacted yet";
public enum State
{
- COMPLETE, PENDING, RUNNING, SKIPPED
+ COMPLETE, ELIGIBLE, NOT_ELIGIBLE
}
/**
* List of checks performed to determine if compaction is already complete based on indexing state fingerprints.
*/
- private static final List> FINGERPRINT_CHECKS = List.of(
+ static final List> FINGERPRINT_CHECKS = List.of(
Evaluator::allFingerprintedCandidatesHaveExpectedFingerprint
);
@@ -79,7 +81,7 @@ public enum State
*
* The order of the checks must be honored while evaluating them.
*/
- private static final List> CHECKS = Arrays.asList(
+ static final List> CHECKS = Arrays.asList(
Evaluator::partitionsSpecIsUpToDate,
Evaluator::indexSpecIsUpToDate,
Evaluator::segmentGranularityIsUpToDate,
@@ -91,32 +93,56 @@ public enum State
Evaluator::projectionsAreUpToDate
);
+ public static CompactionStatus notEligible(String messageFormat, Object... args)
+ {
+ return new CompactionStatus(State.NOT_ELIGIBLE, StringUtils.format(messageFormat, args), null, null);
+ }
+
private final State state;
private final String reason;
- private final CompactionStatistics compactedStats;
- private final CompactionStatistics uncompactedStats;
+
+ @Nullable
+ private final CompactionStatistics compacted;
+ @Nullable
+ private final CompactionStatistics uncompacted;
private CompactionStatus(
State state,
String reason,
- CompactionStatistics compactedStats,
- CompactionStatistics uncompactedStats
+ @Nullable CompactionStatistics compacted,
+ @Nullable CompactionStatistics uncompacted
)
{
this.state = state;
this.reason = reason;
- this.compactedStats = compactedStats;
- this.uncompactedStats = uncompactedStats;
+ switch (state) {
+ case COMPLETE:
+ break;
+ case NOT_ELIGIBLE:
+ InvalidInput.conditionalException(
+ !Strings.isNullOrEmpty(reason),
+ "must provide a reason why compaction not eligible"
+ );
+ break;
+ case ELIGIBLE:
+ InvalidInput.conditionalException(compacted != null, "must provide compacted stats for compaction");
+ InvalidInput.conditionalException(uncompacted != null, "must provide uncompacted stats for compaction");
+ break;
+ default:
+ throw DruidException.defensive("unexpected compaction status state[%s]", state);
+ }
+ this.compacted = compacted;
+ this.uncompacted = uncompacted;
}
- public boolean isComplete()
+ static CompactionStatusBuilder builder(State state, String reason)
{
- return state == State.COMPLETE;
+ return new CompactionStatusBuilder(state, reason);
}
- public boolean isSkipped()
+ public State getState()
{
- return state == State.SKIPPED;
+ return state;
}
public String getReason()
@@ -124,58 +150,81 @@ public String getReason()
return reason;
}
- public State getState()
+ @Nullable
+ public CompactionStatistics getUncompactedStats()
{
- return state;
+ return uncompacted;
}
+ @Nullable
public CompactionStatistics getCompactedStats()
{
- return compactedStats;
+ return compacted;
}
- public CompactionStatistics getUncompactedStats()
+ /**
+ * Evaluates a compaction candidate to determine its eligibility and compaction status.
+ *
+ * This method performs a two-stage evaluation:
+ *
+ *
First, uses {@link Evaluator} to check if the candidate needs compaction
+ * based on the compaction config (e.g., checking segment granularity, partitions spec, etc.)
+ *
Then, applies the search policy to determine if this candidate should be compacted in the
+ * current run (e.g., checking minimum segment count, bytes, or other policy criteria)
+ *
+ *
+ * @param candidate the compaction candidate to evaluate
+ * @param config the compaction configuration for the datasource
+ * @param fingerprintMapper mapper for indexing state fingerprints
+ * @return {@link CompactionStatus}
+ */
+ public static CompactionStatus compute(
+ CompactionCandidate candidate,
+ DataSourceCompactionConfig config,
+ IndexingStateFingerprintMapper fingerprintMapper
+ )
{
- return uncompactedStats;
+ return new Evaluator(candidate, config, fingerprintMapper).evaluate();
}
@Override
- public String toString()
+ public boolean equals(Object object)
{
- return "CompactionStatus{" +
- "state=" + state +
- ", reason=" + reason +
- ", compactedStats=" + compactedStats +
- ", uncompactedStats=" + uncompactedStats +
- '}';
+ if (this == object) {
+ return true;
+ }
+ if (object == null || getClass() != object.getClass()) {
+ return false;
+ }
+ CompactionStatus that = (CompactionStatus) object;
+ return state == that.state
+ && Objects.equals(reason, that.reason)
+ && Objects.equals(compacted, that.compacted)
+ && Objects.equals(uncompacted, that.uncompacted);
}
- public static CompactionStatus pending(String reasonFormat, Object... args)
+ @Override
+ public int hashCode()
{
- return new CompactionStatus(State.PENDING, StringUtils.format(reasonFormat, args), null, null);
+ return Objects.hash(state, reason, compacted, uncompacted);
}
- public static CompactionStatus pending(
- CompactionStatistics compactedStats,
- CompactionStatistics uncompactedStats,
- String reasonFormat,
- Object... args
- )
+ @Override
+ public String toString()
{
- return new CompactionStatus(
- State.PENDING,
- StringUtils.format(reasonFormat, args),
- compactedStats,
- uncompactedStats
- );
+ return "CompactionStatus{"
+ + "state=" + state
+ + ", reason='" + reason + '\''
+ + ", compacted=" + compacted
+ + ", uncompacted=" + uncompacted
+ + '}';
}
/**
- * Computes compaction status for the given field. The status is assumed to be
- * COMPLETE (i.e. no further compaction is required) if the configured value
- * of the field is null or equal to the current value.
+ * Returns a 'mismatch' reason to be eligible for compaction if config doesn't match, NULL if config matches.
*/
- private static CompactionStatus completeIfNullOrEqual(
+ @Nullable
+ private static Evaluator.CheckResult getConfigMismatchReason(
String field,
T configured,
T current,
@@ -183,25 +232,25 @@ private static CompactionStatus completeIfNullOrEqual(
)
{
if (configured == null || configured.equals(current)) {
- return COMPLETE;
+ return Evaluator.CheckResult.PASS;
} else {
return configChanged(field, configured, current, stringFunction);
}
}
- private static CompactionStatus configChanged(
+ private static Evaluator.CheckResult configChanged(
String field,
T target,
T current,
Function stringFunction
)
{
- return CompactionStatus.pending(
+ return Evaluator.CheckResult.fail(StringUtils.format(
"'%s' mismatch: required[%s], current[%s]",
field,
target == null ? null : stringFunction.apply(target),
current == null ? null : stringFunction.apply(current)
- );
+ ));
}
private static String asString(Granularity granularity)
@@ -242,41 +291,6 @@ private static String asString(PartitionsSpec partitionsSpec)
}
}
- public static CompactionStatus skipped(String reasonFormat, Object... args)
- {
- return new CompactionStatus(State.SKIPPED, StringUtils.format(reasonFormat, args), null, null);
- }
-
- public static CompactionStatus running(String message)
- {
- return new CompactionStatus(State.RUNNING, message, null, null);
- }
-
- /**
- * Determines the CompactionStatus of the given candidate segments by evaluating
- * the {@link #CHECKS} one by one. If any check returns an incomplete status,
- * further checks are still performed to determine the number of uncompacted
- * segments but only the first incomplete status is returned.
- */
- static CompactionStatus compute(
- CompactionCandidate candidateSegments,
- DataSourceCompactionConfig config,
- @Nullable IndexingStateFingerprintMapper fingerprintMapper
- )
- {
- final CompactionState expectedState = config.toCompactionState();
- String expectedFingerprint;
- if (fingerprintMapper == null) {
- expectedFingerprint = null;
- } else {
- expectedFingerprint = fingerprintMapper.generateFingerprint(
- config.getDataSource(),
- expectedState
- );
- }
- return new Evaluator(candidateSegments, config, expectedFingerprint, fingerprintMapper).evaluate();
- }
-
@Nullable
public static PartitionsSpec findPartitionsSpecFromConfig(ClientCompactionTaskQueryTuningConfig tuningConfig)
{
@@ -347,8 +361,10 @@ static DimensionRangePartitionsSpec getEffectiveRangePartitionsSpec(DimensionRan
*/
private static class Evaluator
{
+ private static final Logger log = new Logger(Evaluator.class);
+
private final DataSourceCompactionConfig compactionConfig;
- private final CompactionCandidate candidateSegments;
+ private final CompactionCandidate candidate;
private final ClientCompactionTaskQueryTuningConfig tuningConfig;
private final UserCompactionTaskGranularityConfig configuredGranularitySpec;
@@ -358,47 +374,63 @@ private static class Evaluator
private final Map> unknownStateToSegments = new HashMap<>();
@Nullable
- private final String targetFingerprint;
private final IndexingStateFingerprintMapper fingerprintMapper;
+ @Nullable
+ private final String targetFingerprint;
private Evaluator(
- CompactionCandidate candidateSegments,
+ CompactionCandidate candidate,
DataSourceCompactionConfig compactionConfig,
- @Nullable String targetFingerprint,
@Nullable IndexingStateFingerprintMapper fingerprintMapper
)
{
- this.candidateSegments = candidateSegments;
+ this.candidate = candidate;
this.compactionConfig = compactionConfig;
this.tuningConfig = ClientCompactionTaskQueryTuningConfig.from(compactionConfig);
this.configuredGranularitySpec = compactionConfig.getGranularitySpec();
- this.targetFingerprint = targetFingerprint;
this.fingerprintMapper = fingerprintMapper;
+ if (fingerprintMapper == null) {
+ targetFingerprint = null;
+ } else {
+ targetFingerprint = fingerprintMapper.generateFingerprint(
+ compactionConfig.getDataSource(),
+ compactionConfig.toCompactionState()
+ );
+ }
}
+ /**
+ * Evaluates the compaction status of candidate segments through a multi-step process:
+ *
+ *
Validates input bytes are within limits
+ *
Categorizes segments by compaction state (fingerprinted, uncompacted, or unknown)
+ *
Performs fingerprint-based validation if available (fast path)
+ *
Runs detailed checks against unknown states via {@link CompactionStatus#CHECKS}
+ *
+ *
+ * @return Pair of eligibility status and compaction status with reason for first failed check
+ */
private CompactionStatus evaluate()
{
- final CompactionStatus inputBytesCheck = inputBytesAreWithinLimit();
- if (inputBytesCheck.isSkipped()) {
- return inputBytesCheck;
+ final CheckResult inputBytesCheck = inputBytesAreWithinLimit();
+ if (!CheckResult.PASS.equals(inputBytesCheck)) {
+ return CompactionStatus.notEligible(inputBytesCheck.reason);
}
- List reasonsForCompaction = new ArrayList<>();
- CompactionStatus compactedOnceCheck = segmentsHaveBeenCompactedAtLeastOnce();
- if (!compactedOnceCheck.isComplete()) {
- reasonsForCompaction.add(compactedOnceCheck.getReason());
+ List reasonsForCompaction = new ArrayList<>();
+ CheckResult compactedOnceCheck = segmentsHaveBeenCompactedAtLeastOnce();
+ if (!CheckResult.PASS.equals(compactedOnceCheck)) {
+ reasonsForCompaction.add(compactedOnceCheck);
}
if (fingerprintMapper != null && targetFingerprint != null) {
// First try fingerprint-based evaluation (fast path)
- CompactionStatus fingerprintStatus = FINGERPRINT_CHECKS.stream()
- .map(f -> f.apply(this))
- .filter(status -> !status.isComplete())
- .findFirst().orElse(COMPLETE);
+ FINGERPRINT_CHECKS.stream()
+ .map(f -> f.apply(this))
+ .filter(r -> !CheckResult.PASS.equals(r))
+ .findFirst()
+ .ifPresent(reasonsForCompaction::add);
- if (!fingerprintStatus.isComplete()) {
- reasonsForCompaction.add(fingerprintStatus.getReason());
- }
}
if (!unknownStateToSegments.isEmpty()) {
@@ -406,29 +438,26 @@ private CompactionStatus evaluate()
reasonsForCompaction.addAll(
CHECKS.stream()
.map(f -> f.apply(this))
- .filter(status -> !status.isComplete())
- .map(CompactionStatus::getReason)
- .collect(Collectors.toList())
+ .filter(r -> !CheckResult.PASS.equals(r))
+ .toList()
);
// Any segments left in unknownStateToSegments passed all checks and are considered compacted
- this.compactedSegments.addAll(
+ compactedSegments.addAll(
unknownStateToSegments
.values()
.stream()
.flatMap(List::stream)
- .collect(Collectors.toList())
+ .toList()
);
}
if (reasonsForCompaction.isEmpty()) {
- return COMPLETE;
+ return CompactionStatus.COMPLETE;
} else {
- return CompactionStatus.pending(
- createStats(this.compactedSegments),
- createStats(uncompactedSegments),
- reasonsForCompaction.get(0)
- );
+ return builder(State.ELIGIBLE, reasonsForCompaction.get(0).reason).compacted(createStats(compactedSegments))
+ .uncompacted(createStats(uncompactedSegments))
+ .build();
}
}
@@ -440,7 +469,7 @@ private CompactionStatus evaluate()
* {@link #unknownStateToSegments} where their indexing states will be analyzed.
*
*/
- private CompactionStatus allFingerprintedCandidatesHaveExpectedFingerprint()
+ private CheckResult allFingerprintedCandidatesHaveExpectedFingerprint()
{
Map> mismatchedFingerprintToSegmentMap = new HashMap<>();
for (DataSegment segment : fingerprintedSegments) {
@@ -458,18 +487,18 @@ private CompactionStatus allFingerprintedCandidatesHaveExpectedFingerprint()
if (mismatchedFingerprintToSegmentMap.isEmpty()) {
// All fingerprinted segments have the expected fingerprint - compaction is complete
- return COMPLETE;
+ return CheckResult.PASS;
}
if (fingerprintMapper == null) {
// Cannot evaluate further without a fingerprint mapper
uncompactedSegments.addAll(
mismatchedFingerprintToSegmentMap.values()
- .stream()
- .flatMap(List::stream)
- .collect(Collectors.toList())
+ .stream()
+ .flatMap(List::stream)
+ .collect(Collectors.toList())
);
- return CompactionStatus.pending("Segments have a mismatched fingerprint and no fingerprint mapper is available");
+ return CheckResult.fail("Segments have a mismatched fingerprint and no fingerprint mapper is available");
}
boolean fingerprintedSegmentWithoutCachedStateFound = false;
@@ -491,14 +520,15 @@ private CompactionStatus allFingerprintedCandidatesHaveExpectedFingerprint()
}
segments.addAll(e.getValue());
return segments;
- });
+ }
+ );
}
}
if (fingerprintedSegmentWithoutCachedStateFound) {
- return CompactionStatus.pending("One or more fingerprinted segments do not have a cached indexing state");
+ return CheckResult.fail("One or more fingerprinted segments do not have a cached indexing state");
} else {
- return COMPLETE;
+ return CheckResult.PASS;
}
}
@@ -506,9 +536,9 @@ private CompactionStatus allFingerprintedCandidatesHaveExpectedFingerprint()
* Checks if all the segments have been compacted at least once and groups them into uncompacted, fingerprinted, or
* non-fingerprinted.
*/
- private CompactionStatus segmentsHaveBeenCompactedAtLeastOnce()
+ private CheckResult segmentsHaveBeenCompactedAtLeastOnce()
{
- for (DataSegment segment : candidateSegments.getSegments()) {
+ for (DataSegment segment : candidate.getSegments()) {
final String fingerprint = segment.getIndexingStateFingerprint();
final CompactionState segmentState = segment.getLastCompactionState();
if (fingerprint != null) {
@@ -521,58 +551,58 @@ private CompactionStatus segmentsHaveBeenCompactedAtLeastOnce()
}
if (uncompactedSegments.isEmpty()) {
- return COMPLETE;
+ return CheckResult.PASS;
} else {
- return CompactionStatus.pending(NEVER_COMPACTED_REASON);
+ return CheckResult.fail(NEVER_COMPACTED_REASON);
}
}
- private CompactionStatus partitionsSpecIsUpToDate()
+ private CheckResult partitionsSpecIsUpToDate()
{
return evaluateForAllCompactionStates(this::partitionsSpecIsUpToDate);
}
- private CompactionStatus indexSpecIsUpToDate()
+ private CheckResult indexSpecIsUpToDate()
{
return evaluateForAllCompactionStates(this::indexSpecIsUpToDate);
}
- private CompactionStatus projectionsAreUpToDate()
+ private CheckResult projectionsAreUpToDate()
{
return evaluateForAllCompactionStates(this::projectionsAreUpToDate);
}
- private CompactionStatus segmentGranularityIsUpToDate()
+ private CheckResult segmentGranularityIsUpToDate()
{
return evaluateForAllCompactionStates(this::segmentGranularityIsUpToDate);
}
- private CompactionStatus rollupIsUpToDate()
+ private CheckResult rollupIsUpToDate()
{
return evaluateForAllCompactionStates(this::rollupIsUpToDate);
}
- private CompactionStatus queryGranularityIsUpToDate()
+ private CheckResult queryGranularityIsUpToDate()
{
return evaluateForAllCompactionStates(this::queryGranularityIsUpToDate);
}
- private CompactionStatus dimensionsSpecIsUpToDate()
+ private CheckResult dimensionsSpecIsUpToDate()
{
return evaluateForAllCompactionStates(this::dimensionsSpecIsUpToDate);
}
- private CompactionStatus metricsSpecIsUpToDate()
+ private CheckResult metricsSpecIsUpToDate()
{
return evaluateForAllCompactionStates(this::metricsSpecIsUpToDate);
}
- private CompactionStatus transformSpecIsUpToDate()
+ private CheckResult transformSpecIsUpToDate()
{
return evaluateForAllCompactionStates(this::transformSpecIsUpToDate);
}
- private CompactionStatus partitionsSpecIsUpToDate(CompactionState lastCompactionState)
+ private CheckResult partitionsSpecIsUpToDate(CompactionState lastCompactionState)
{
PartitionsSpec existingPartionsSpec = lastCompactionState.getPartitionsSpec();
if (existingPartionsSpec instanceof DimensionRangePartitionsSpec) {
@@ -580,9 +610,10 @@ private CompactionStatus partitionsSpecIsUpToDate(CompactionState lastCompaction
} else if (existingPartionsSpec instanceof DynamicPartitionsSpec) {
existingPartionsSpec = new DynamicPartitionsSpec(
existingPartionsSpec.getMaxRowsPerSegment(),
- ((DynamicPartitionsSpec) existingPartionsSpec).getMaxTotalRowsOr(Long.MAX_VALUE));
+ ((DynamicPartitionsSpec) existingPartionsSpec).getMaxTotalRowsOr(Long.MAX_VALUE)
+ );
}
- return CompactionStatus.completeIfNullOrEqual(
+ return getConfigMismatchReason(
"partitionsSpec",
findPartitionsSpecFromConfig(tuningConfig),
existingPartionsSpec,
@@ -590,9 +621,9 @@ private CompactionStatus partitionsSpecIsUpToDate(CompactionState lastCompaction
);
}
- private CompactionStatus indexSpecIsUpToDate(CompactionState lastCompactionState)
+ private CheckResult indexSpecIsUpToDate(CompactionState lastCompactionState)
{
- return CompactionStatus.completeIfNullOrEqual(
+ return getConfigMismatchReason(
"indexSpec",
Configs.valueOrDefault(tuningConfig.getIndexSpec(), IndexSpec.getDefault()).getEffectiveSpec(),
lastCompactionState.getIndexSpec().getEffectiveSpec(),
@@ -600,9 +631,9 @@ private CompactionStatus indexSpecIsUpToDate(CompactionState lastCompactionState
);
}
- private CompactionStatus projectionsAreUpToDate(CompactionState lastCompactionState)
+ private CheckResult projectionsAreUpToDate(CompactionState lastCompactionState)
{
- return CompactionStatus.completeIfNullOrEqual(
+ return getConfigMismatchReason(
"projections",
compactionConfig.getProjections(),
lastCompactionState.getProjections(),
@@ -610,24 +641,23 @@ private CompactionStatus projectionsAreUpToDate(CompactionState lastCompactionSt
);
}
- private CompactionStatus inputBytesAreWithinLimit()
+ private CheckResult inputBytesAreWithinLimit()
{
final long inputSegmentSize = compactionConfig.getInputSegmentSizeBytes();
- if (candidateSegments.getTotalBytes() > inputSegmentSize) {
- return CompactionStatus.skipped(
+ if (candidate.getTotalBytes() > inputSegmentSize) {
+ return CheckResult.fail(StringUtils.format(
"'inputSegmentSize' exceeded: Total segment size[%d] is larger than allowed inputSegmentSize[%d]",
- candidateSegments.getTotalBytes(), inputSegmentSize
- );
- } else {
- return COMPLETE;
+ candidate.getTotalBytes(), inputSegmentSize
+ ));
}
+ return CheckResult.PASS;
}
- private CompactionStatus segmentGranularityIsUpToDate(CompactionState lastCompactionState)
+ private CheckResult segmentGranularityIsUpToDate(CompactionState lastCompactionState)
{
if (configuredGranularitySpec == null
|| configuredGranularitySpec.getSegmentGranularity() == null) {
- return COMPLETE;
+ return CheckResult.PASS;
}
final Granularity configuredSegmentGranularity = configuredGranularitySpec.getSegmentGranularity();
@@ -636,7 +666,7 @@ private CompactionStatus segmentGranularityIsUpToDate(CompactionState lastCompac
= existingGranularitySpec == null ? null : existingGranularitySpec.getSegmentGranularity();
if (configuredSegmentGranularity.equals(existingSegmentGranularity)) {
- return COMPLETE;
+ return CheckResult.PASS;
} else if (existingSegmentGranularity == null) {
// Candidate segments were compacted without segment granularity specified
// Check if the segments already have the desired segment granularity
@@ -645,13 +675,13 @@ private CompactionStatus segmentGranularityIsUpToDate(CompactionState lastCompac
segment -> !configuredSegmentGranularity.isAligned(segment.getInterval())
);
if (needsCompaction) {
- return CompactionStatus.pending(
+ return CheckResult.fail(StringUtils.format(
"segmentGranularity: segments do not align with target[%s]",
- asString(configuredSegmentGranularity)
- );
+ CompactionStatus.asString(configuredSegmentGranularity)
+ ));
}
} else {
- return CompactionStatus.configChanged(
+ return configChanged(
"segmentGranularity",
configuredSegmentGranularity,
existingSegmentGranularity,
@@ -659,17 +689,17 @@ private CompactionStatus segmentGranularityIsUpToDate(CompactionState lastCompac
);
}
- return COMPLETE;
+ return CheckResult.PASS;
}
- private CompactionStatus rollupIsUpToDate(CompactionState lastCompactionState)
+ private CheckResult rollupIsUpToDate(CompactionState lastCompactionState)
{
if (configuredGranularitySpec == null) {
- return COMPLETE;
+ return CheckResult.PASS;
} else {
final UserCompactionTaskGranularityConfig existingGranularitySpec
= getGranularitySpec(lastCompactionState);
- return CompactionStatus.completeIfNullOrEqual(
+ return getConfigMismatchReason(
"rollup",
configuredGranularitySpec.isRollup(),
existingGranularitySpec == null ? null : existingGranularitySpec.isRollup(),
@@ -678,14 +708,14 @@ private CompactionStatus rollupIsUpToDate(CompactionState lastCompactionState)
}
}
- private CompactionStatus queryGranularityIsUpToDate(CompactionState lastCompactionState)
+ private CheckResult queryGranularityIsUpToDate(CompactionState lastCompactionState)
{
if (configuredGranularitySpec == null) {
- return COMPLETE;
+ return CheckResult.PASS;
} else {
final UserCompactionTaskGranularityConfig existingGranularitySpec
= getGranularitySpec(lastCompactionState);
- return CompactionStatus.completeIfNullOrEqual(
+ return getConfigMismatchReason(
"queryGranularity",
configuredGranularitySpec.getQueryGranularity(),
existingGranularitySpec == null ? null : existingGranularitySpec.getQueryGranularity(),
@@ -699,10 +729,10 @@ private CompactionStatus queryGranularityIsUpToDate(CompactionState lastCompacti
* which can create a mismatch between expected and actual order of dimensions. Partition dimensions are separately
* covered in {@link Evaluator#partitionsSpecIsUpToDate()} check.
*/
- private CompactionStatus dimensionsSpecIsUpToDate(CompactionState lastCompactionState)
+ private CheckResult dimensionsSpecIsUpToDate(CompactionState lastCompactionState)
{
if (compactionConfig.getDimensionsSpec() == null) {
- return COMPLETE;
+ return CheckResult.PASS;
} else {
List existingDimensions = getNonPartitioningDimensions(
lastCompactionState.getDimensionsSpec() == null
@@ -718,7 +748,7 @@ private CompactionStatus dimensionsSpecIsUpToDate(CompactionState lastCompaction
? IndexSpec.getDefault()
: compactionConfig.getTuningConfig().getIndexSpec()
);
- return CompactionStatus.completeIfNullOrEqual(
+ return getConfigMismatchReason(
"dimensionsSpec",
configuredDimensions,
existingDimensions,
@@ -727,11 +757,11 @@ private CompactionStatus dimensionsSpecIsUpToDate(CompactionState lastCompaction
}
}
- private CompactionStatus metricsSpecIsUpToDate(CompactionState lastCompactionState)
+ private CheckResult metricsSpecIsUpToDate(CompactionState lastCompactionState)
{
final AggregatorFactory[] configuredMetricsSpec = compactionConfig.getMetricsSpec();
if (ArrayUtils.isEmpty(configuredMetricsSpec)) {
- return COMPLETE;
+ return CheckResult.PASS;
}
final List metricSpecList = lastCompactionState.getMetricsSpec();
@@ -740,30 +770,30 @@ private CompactionStatus metricsSpecIsUpToDate(CompactionState lastCompactionSta
? null : metricSpecList.toArray(new AggregatorFactory[0]);
if (existingMetricsSpec == null || !Arrays.deepEquals(configuredMetricsSpec, existingMetricsSpec)) {
- return CompactionStatus.configChanged(
+ return configChanged(
"metricsSpec",
configuredMetricsSpec,
existingMetricsSpec,
Arrays::toString
);
} else {
- return COMPLETE;
+ return CheckResult.PASS;
}
}
- private CompactionStatus transformSpecIsUpToDate(CompactionState lastCompactionState)
+ private CheckResult transformSpecIsUpToDate(CompactionState lastCompactionState)
{
final CompactionTransformSpec configuredSpec = compactionConfig.getTransformSpec();
if (configuredSpec == null
|| (configuredSpec.getFilter() == null && configuredSpec.getVirtualColumns().isEmpty())) {
- return COMPLETE;
+ return CheckResult.PASS;
}
final CompactionTransformSpec existingSpec = Configs.valueOrDefault(
lastCompactionState.getTransformSpec(),
new CompactionTransformSpec(null, null)
);
- return CompactionStatus.completeIfNullOrEqual(
+ return getConfigMismatchReason(
"transformSpec",
configuredSpec,
existingSpec,
@@ -778,22 +808,60 @@ private CompactionStatus transformSpecIsUpToDate(CompactionState lastCompactionS
*
* @return The first status which is not COMPLETE.
*/
- private CompactionStatus evaluateForAllCompactionStates(
- Function check
- )
+ private CheckResult evaluateForAllCompactionStates(Function check)
{
- CompactionStatus firstIncompleteStatus = null;
+ CheckResult firstIncomplete = CheckResult.PASS;
for (CompactionState state : List.copyOf(unknownStateToSegments.keySet())) {
- final CompactionStatus status = check.apply(state);
- if (!status.isComplete()) {
+ final CheckResult eligibleReason = check.apply(state);
+ if (!CheckResult.PASS.equals(eligibleReason)) {
uncompactedSegments.addAll(unknownStateToSegments.remove(state));
- if (firstIncompleteStatus == null) {
- firstIncompleteStatus = status;
+ if (CheckResult.PASS.equals(firstIncomplete)) {
+ firstIncomplete = eligibleReason;
}
}
}
- return firstIncompleteStatus == null ? COMPLETE : firstIncompleteStatus;
+ return firstIncomplete;
+ }
+
+ static class CheckResult
+ {
+ static final CheckResult PASS = new CheckResult(null);
+
+ @Nullable
+ String reason;
+
+ private CheckResult(@Nullable String reason)
+ {
+ this.reason = reason;
+ }
+
+ static CheckResult fail(String reason)
+ {
+ if (Strings.isNullOrEmpty(reason)) {
+ throw DruidException.defensive("expecting a non-null & non-empty check fail reason");
+ }
+ return new CheckResult(reason);
+ }
+
+ @Override
+ public boolean equals(Object o)
+ {
+ if (this == o) {
+ return true;
+ }
+ if (o == null || getClass() != o.getClass()) {
+ return false;
+ }
+ CheckResult that = (CheckResult) o;
+ return Objects.equals(reason, that.reason);
+ }
+
+ @Override
+ public int hashCode()
+ {
+ return Objects.hash(reason);
+ }
}
private static UserCompactionTaskGranularityConfig getGranularitySpec(
@@ -811,4 +879,35 @@ private static CompactionStatistics createStats(List segments)
return CompactionStatistics.create(totalBytes, segments.size(), segmentIntervals.size());
}
}
+
+ static class CompactionStatusBuilder
+ {
+ private State state;
+ private CompactionStatistics compacted;
+ private CompactionStatistics uncompacted;
+ private String reason;
+
+ CompactionStatusBuilder(State state, String reason)
+ {
+ this.state = state;
+ this.reason = reason;
+ }
+
+ CompactionStatusBuilder compacted(CompactionStatistics compacted)
+ {
+ this.compacted = compacted;
+ return this;
+ }
+
+ CompactionStatusBuilder uncompacted(CompactionStatistics uncompacted)
+ {
+ this.uncompacted = uncompacted;
+ return this;
+ }
+
+ CompactionStatus build()
+ {
+ return new CompactionStatus(state, reason, compacted, uncompacted);
+ }
+ }
}
diff --git a/server/src/main/java/org/apache/druid/server/compaction/CompactionStatusTracker.java b/server/src/main/java/org/apache/druid/server/compaction/CompactionStatusTracker.java
index e98e3575f47b..5afa93b6de7e 100644
--- a/server/src/main/java/org/apache/druid/server/compaction/CompactionStatusTracker.java
+++ b/server/src/main/java/org/apache/druid/server/compaction/CompactionStatusTracker.java
@@ -86,15 +86,13 @@ public Set getSubmittedTaskIds()
* duty and will be removed in the future.
*/
@Deprecated
- public CompactionStatus computeCompactionStatus(
- CompactionCandidate candidate,
- CompactionCandidateSearchPolicy searchPolicy
- )
+ @Nullable
+ public TaskState computeCompactionTaskState(CompactionCandidate candidate)
{
// Skip intervals that already have a running task
final CompactionTaskStatus lastTaskStatus = getLatestTaskStatus(candidate);
if (lastTaskStatus != null && lastTaskStatus.getState() == TaskState.RUNNING) {
- return CompactionStatus.running("Task for interval is already running");
+ return TaskState.RUNNING;
}
// Skip intervals that have been recently compacted if segment timeline is not updated yet
@@ -102,27 +100,28 @@ public CompactionStatus computeCompactionStatus(
if (lastTaskStatus != null
&& lastTaskStatus.getState() == TaskState.SUCCESS
&& snapshotTime != null && snapshotTime.isBefore(lastTaskStatus.getUpdatedTime())) {
- return CompactionStatus.skipped(
- "Segment timeline not updated since last compaction task succeeded"
- );
+ return TaskState.SUCCESS;
}
- // Skip intervals that have been filtered out by the policy
- final CompactionCandidateSearchPolicy.Eligibility eligibility
- = searchPolicy.checkEligibilityForCompaction(candidate, lastTaskStatus);
- if (eligibility.isEligible()) {
- return CompactionStatus.pending("Not compacted yet");
- } else {
- return CompactionStatus.skipped("Rejected by search policy: %s", eligibility.getReason());
- }
+ // if task has never been run or has failed recently, run the compaction candidate.
+ return null;
}
/**
* Tracks the latest compaction status of the given compaction candidates.
* Used only by the {@link CompactionRunSimulator}.
*/
- public void onCompactionStatusComputed(
- CompactionCandidate candidateSegments,
+ public void onSkippedCandidate(
+ CompactionCandidateAndStatus candidateSegments,
+ @Nullable String policyNote
+ )
+ {
+ // Nothing to do, used by simulator
+ }
+
+ public void onCompactionTaskStateComputed(
+ CompactionCandidateAndStatus candidateSegments,
+ TaskState taskState,
DataSourceCompactionConfig config
)
{
@@ -155,12 +154,12 @@ public void resetActiveDatasources(Set compactionEnabledDatasources)
public void onTaskSubmitted(
String taskId,
- CompactionCandidate candidateSegments
+ CompactionCandidateAndStatus candidateSegments
)
{
- submittedTaskIdToSegments.put(taskId, candidateSegments);
- getOrComputeDatasourceStatus(candidateSegments.getDataSource())
- .handleSubmittedTask(candidateSegments);
+ submittedTaskIdToSegments.put(taskId, candidateSegments.getCandidate());
+ getOrComputeDatasourceStatus(candidateSegments.getCandidate().getDataSource())
+ .handleSubmittedTask(candidateSegments.getCandidate());
}
public void onTaskFinished(String taskId, TaskStatus taskStatus)
diff --git a/server/src/main/java/org/apache/druid/server/compaction/DataSourceCompactibleSegmentIterator.java b/server/src/main/java/org/apache/druid/server/compaction/DataSourceCompactibleSegmentIterator.java
index ed80ec967b6f..d49a3116dff5 100644
--- a/server/src/main/java/org/apache/druid/server/compaction/DataSourceCompactibleSegmentIterator.java
+++ b/server/src/main/java/org/apache/druid/server/compaction/DataSourceCompactibleSegmentIterator.java
@@ -24,9 +24,11 @@
import com.google.common.base.Preconditions;
import com.google.common.collect.Iterators;
import com.google.common.collect.Lists;
+import org.apache.druid.error.DruidException;
import org.apache.druid.java.util.common.DateTimes;
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.guava.Comparators;
import org.apache.druid.java.util.common.logger.Logger;
@@ -72,7 +74,7 @@ public class DataSourceCompactibleSegmentIterator implements CompactionSegmentIt
private final IndexingStateFingerprintMapper fingerprintMapper;
private final List compactedSegments = new ArrayList<>();
- private final List skippedSegments = new ArrayList<>();
+ private final List skippedSegments = new ArrayList<>();
// This is needed for datasource that has segmentGranularity configured
// If configured segmentGranularity in config is finer than current segmentGranularity, the same set of segments
@@ -80,7 +82,8 @@ public class DataSourceCompactibleSegmentIterator implements CompactionSegmentIt
// run of the compaction job and skip any interval that was already previously compacted.
private final Set queuedIntervals = new HashSet<>();
- private final PriorityQueue queue;
+ private final CompactionCandidateSearchPolicy searchPolicy;
+ private final PriorityQueue queue;
public DataSourceCompactibleSegmentIterator(
DataSourceCompactionConfig config,
@@ -92,6 +95,7 @@ public DataSourceCompactibleSegmentIterator(
{
this.config = config;
this.dataSource = config.getDataSource();
+ this.searchPolicy = searchPolicy;
this.queue = new PriorityQueue<>(searchPolicy::compareCandidates);
this.fingerprintMapper = indexingStateFingerprintMapper;
@@ -121,10 +125,12 @@ private void populateQueue(SegmentTimeline timeline, List skipInterval
if (!partialEternitySegments.isEmpty()) {
// Do not use the target segment granularity in the CompactionCandidate
// as Granularities.getIterable() will cause OOM due to the above issue
- CompactionCandidate candidatesWithStatus = CompactionCandidate
- .from(partialEternitySegments, null)
- .withCurrentStatus(CompactionStatus.skipped("Segments have partial-eternity intervals"));
- skippedSegments.add(candidatesWithStatus);
+ skippedSegments.add(
+ new CompactionCandidateAndStatus(
+ CompactionCandidate.from(partialEternitySegments, null),
+ CompactionStatus.notEligible("partial eternal")
+ )
+ );
return;
}
@@ -201,7 +207,7 @@ public List getCompactedSegments()
}
@Override
- public List getSkippedSegments()
+ public List getSkippedSegments()
{
return skippedSegments;
}
@@ -213,7 +219,7 @@ public boolean hasNext()
}
@Override
- public CompactionCandidate next()
+ public CompactionCandidateAndStatus next()
{
if (hasNext()) {
return queue.poll();
@@ -329,17 +335,30 @@ private void findAndEnqueueSegmentsToCompact(CompactibleSegmentIterator compacti
continue;
}
- final CompactionCandidate candidates = CompactionCandidate.from(segments, config.getSegmentGranularity());
- final CompactionStatus compactionStatus = CompactionStatus.compute(candidates, config, fingerprintMapper);
- final CompactionCandidate candidatesWithStatus = candidates.withCurrentStatus(compactionStatus);
-
- if (compactionStatus.isComplete()) {
- compactedSegments.add(candidatesWithStatus);
- } else if (compactionStatus.isSkipped()) {
- skippedSegments.add(candidatesWithStatus);
- } else if (!queuedIntervals.contains(candidates.getUmbrellaInterval())) {
- queue.add(candidatesWithStatus);
- queuedIntervals.add(candidates.getUmbrellaInterval());
+ final CompactionCandidate candidate = CompactionCandidate.from(segments, config.getSegmentGranularity());
+ final CompactionCandidateAndStatus candidateAndStatus = new CompactionCandidateAndStatus(
+ candidate,
+ CompactionStatus.compute(
+ candidate,
+ config,
+ fingerprintMapper
+ )
+ );
+ switch (candidateAndStatus.getStatus().getState()) {
+ case COMPLETE:
+ compactedSegments.add(candidate);
+ break;
+ case NOT_ELIGIBLE:
+ skippedSegments.add(candidateAndStatus);
+ break;
+ case ELIGIBLE:
+ if (!queuedIntervals.contains(candidate.getUmbrellaInterval())) {
+ queue.add(candidateAndStatus);
+ queuedIntervals.add(candidate.getUmbrellaInterval());
+ }
+ break;
+ default:
+ throw DruidException.defensive("unknown compaction state[%s]", candidateAndStatus.getStatus().getState());
}
}
}
@@ -374,15 +393,14 @@ private List findInitialSearchInterval(
if (!CollectionUtils.isNullOrEmpty(segments)) {
final CompactionCandidate candidates = CompactionCandidate.from(segments, config.getSegmentGranularity());
- final CompactionStatus reason;
+ final String skipReason;
if (candidates.getCompactionInterval().overlaps(latestSkipInterval)) {
- reason = CompactionStatus.skipped("skip offset from latest[%s]", skipOffset);
+ skipReason = StringUtils.format("skip offset from latest[%s]", skipOffset);
} else {
- reason = CompactionStatus.skipped("interval locked by another task");
+ skipReason = "interval locked by another task";
}
- final CompactionCandidate candidatesWithStatus = candidates.withCurrentStatus(reason);
- skippedSegments.add(candidatesWithStatus);
+ skippedSegments.add(new CompactionCandidateAndStatus(candidates, CompactionStatus.notEligible(skipReason)));
}
}
@@ -436,7 +454,8 @@ static Interval computeLatestSkipInterval(
if (configuredSegmentGranularity == null) {
return new Interval(skipOffsetFromLatest, latestDataTimestamp);
} else {
- DateTime skipFromLastest = new DateTime(latestDataTimestamp, latestDataTimestamp.getZone()).minus(skipOffsetFromLatest);
+ DateTime skipFromLastest =
+ new DateTime(latestDataTimestamp, latestDataTimestamp.getZone()).minus(skipOffsetFromLatest);
DateTime skipOffsetBucketToSegmentGranularity = configuredSegmentGranularity.bucketStart(skipFromLastest);
return new Interval(skipOffsetBucketToSegmentGranularity, latestDataTimestamp);
}
diff --git a/server/src/main/java/org/apache/druid/server/compaction/FixedIntervalOrderPolicy.java b/server/src/main/java/org/apache/druid/server/compaction/FixedIntervalOrderPolicy.java
index 24a2f001afe3..2831389e3f67 100644
--- a/server/src/main/java/org/apache/druid/server/compaction/FixedIntervalOrderPolicy.java
+++ b/server/src/main/java/org/apache/druid/server/compaction/FixedIntervalOrderPolicy.java
@@ -50,20 +50,17 @@ public List getEligibleCandidates()
}
@Override
- public int compareCandidates(CompactionCandidate candidateA, CompactionCandidate candidateB)
+ public int compareCandidates(CompactionCandidateAndStatus candidateA, CompactionCandidateAndStatus candidateB)
{
- return findIndex(candidateA) - findIndex(candidateB);
+ return findIndex(candidateA.getCandidate()) - findIndex(candidateB.getCandidate());
}
@Override
- public Eligibility checkEligibilityForCompaction(
- CompactionCandidate candidate,
- CompactionTaskStatus latestTaskStatus
- )
+ public Eligibility checkEligibilityForCompaction(CompactionCandidateAndStatus candidateAndStatus)
{
- return findIndex(candidate) < Integer.MAX_VALUE
- ? Eligibility.OK
- : Eligibility.fail("Datasource/Interval is not in the list of 'eligibleCandidates'");
+ return findIndex(candidateAndStatus.getCandidate()) < Integer.MAX_VALUE
+ ? Eligibility.OK
+ : Eligibility.fail("Datasource/Interval is not in the list of 'eligibleCandidates'");
}
private int findIndex(CompactionCandidate candidate)
diff --git a/server/src/main/java/org/apache/druid/server/compaction/MostFragmentedIntervalFirstPolicy.java b/server/src/main/java/org/apache/druid/server/compaction/MostFragmentedIntervalFirstPolicy.java
index 345988ee7fc2..a9e20a913899 100644
--- a/server/src/main/java/org/apache/druid/server/compaction/MostFragmentedIntervalFirstPolicy.java
+++ b/server/src/main/java/org/apache/druid/server/compaction/MostFragmentedIntervalFirstPolicy.java
@@ -107,9 +107,9 @@ public HumanReadableBytes getMaxAverageUncompactedBytesPerSegment()
}
@Override
- protected Comparator getSegmentComparator()
+ protected Comparator getSegmentComparator()
{
- return this::compare;
+ return Comparator.comparing(o -> Objects.requireNonNull(o.getStatus()), this::compare);
}
@Override
@@ -141,15 +141,16 @@ public int hashCode()
@Override
public String toString()
{
- return "MostFragmentedIntervalFirstPolicy{" +
- "minUncompactedCount=" + minUncompactedCount +
- ", minUncompactedBytes=" + minUncompactedBytes +
- ", maxAverageUncompactedBytesPerSegment=" + maxAverageUncompactedBytesPerSegment +
- ", priorityDataSource='" + getPriorityDatasource() + '\'' +
- '}';
+ return
+ "MostFragmentedIntervalFirstPolicy{" +
+ "minUncompactedCount=" + minUncompactedCount +
+ ", minUncompactedBytes=" + minUncompactedBytes +
+ ", maxAverageUncompactedBytesPerSegment=" + maxAverageUncompactedBytesPerSegment +
+ ", priorityDataSource='" + getPriorityDatasource() + '\'' +
+ '}';
}
- private int compare(CompactionCandidate candidateA, CompactionCandidate candidateB)
+ private int compare(CompactionStatus candidateA, CompactionStatus candidateB)
{
final double fragmentationDiff
= computeFragmentationIndex(candidateB) - computeFragmentationIndex(candidateA);
@@ -157,25 +158,23 @@ private int compare(CompactionCandidate candidateA, CompactionCandidate candidat
}
@Override
- public Eligibility checkEligibilityForCompaction(
- CompactionCandidate candidate,
- CompactionTaskStatus latestTaskStatus
- )
+ public Eligibility checkEligibilityForCompaction(CompactionCandidateAndStatus candidate)
{
- final CompactionStatistics uncompacted = candidate.getUncompactedStats();
- if (uncompacted == null) {
- return Eligibility.OK;
- } else if (uncompacted.getNumSegments() < 1) {
+ final CompactionStatistics uncompacted = Objects.requireNonNull(candidate.getStatus().getUncompactedStats());
+
+ if (uncompacted.getNumSegments() < 1) {
return Eligibility.fail("No uncompacted segments in interval");
} else if (uncompacted.getNumSegments() < minUncompactedCount) {
return Eligibility.fail(
"Uncompacted segments[%,d] in interval must be at least [%,d]",
- uncompacted.getNumSegments(), minUncompactedCount
+ uncompacted.getNumSegments(),
+ minUncompactedCount
);
} else if (uncompacted.getTotalBytes() < minUncompactedBytes.getBytes()) {
return Eligibility.fail(
"Uncompacted bytes[%,d] in interval must be at least [%,d]",
- uncompacted.getTotalBytes(), minUncompactedBytes.getBytes()
+ uncompacted.getTotalBytes(),
+ minUncompactedBytes.getBytes()
);
}
@@ -183,11 +182,11 @@ public Eligibility checkEligibilityForCompaction(
if (avgSegmentSize > maxAverageUncompactedBytesPerSegment.getBytes()) {
return Eligibility.fail(
"Average size[%,d] of uncompacted segments in interval must be at most [%,d]",
- avgSegmentSize, maxAverageUncompactedBytesPerSegment.getBytes()
+ avgSegmentSize,
+ maxAverageUncompactedBytesPerSegment.getBytes()
);
- } else {
- return Eligibility.OK;
}
+ return Eligibility.OK;
}
/**
@@ -197,9 +196,9 @@ public Eligibility checkEligibilityForCompaction(
* A higher fragmentation index causes the candidate to be higher in priority
* for compaction.
*/
- private double computeFragmentationIndex(CompactionCandidate candidate)
+ private double computeFragmentationIndex(CompactionStatus eligibility)
{
- final CompactionStatistics uncompacted = candidate.getUncompactedStats();
+ final CompactionStatistics uncompacted = eligibility.getUncompactedStats();
if (uncompacted == null || uncompacted.getNumSegments() < 1 || uncompacted.getTotalBytes() < 1) {
return 0;
}
diff --git a/server/src/main/java/org/apache/druid/server/compaction/NewestSegmentFirstPolicy.java b/server/src/main/java/org/apache/druid/server/compaction/NewestSegmentFirstPolicy.java
index 7c440900ba74..073308be40b0 100644
--- a/server/src/main/java/org/apache/druid/server/compaction/NewestSegmentFirstPolicy.java
+++ b/server/src/main/java/org/apache/druid/server/compaction/NewestSegmentFirstPolicy.java
@@ -41,9 +41,10 @@ public NewestSegmentFirstPolicy(
}
@Override
- protected Comparator getSegmentComparator()
+ protected Comparator getSegmentComparator()
{
- return (o1, o2) -> Comparators.intervalsByStartThenEnd()
- .compare(o2.getUmbrellaInterval(), o1.getUmbrellaInterval());
+ return (o1, o2) ->
+ Comparators.intervalsByStartThenEnd()
+ .compare(o2.getCandidate().getUmbrellaInterval(), o1.getCandidate().getUmbrellaInterval());
}
}
diff --git a/server/src/main/java/org/apache/druid/server/compaction/PriorityBasedCompactionSegmentIterator.java b/server/src/main/java/org/apache/druid/server/compaction/PriorityBasedCompactionSegmentIterator.java
index 43028fa37ff4..9a0f98e74bdf 100644
--- a/server/src/main/java/org/apache/druid/server/compaction/PriorityBasedCompactionSegmentIterator.java
+++ b/server/src/main/java/org/apache/druid/server/compaction/PriorityBasedCompactionSegmentIterator.java
@@ -42,7 +42,7 @@ public class PriorityBasedCompactionSegmentIterator implements CompactionSegment
{
private static final Logger log = new Logger(PriorityBasedCompactionSegmentIterator.class);
- private final PriorityQueue queue;
+ private final PriorityQueue queue;
private final Map datasourceIterators;
public PriorityBasedCompactionSegmentIterator(
@@ -88,7 +88,7 @@ public List getCompactedSegments()
}
@Override
- public List getSkippedSegments()
+ public List getSkippedSegments()
{
return datasourceIterators.values().stream().flatMap(
iterator -> iterator.getSkippedSegments().stream()
@@ -102,18 +102,18 @@ public boolean hasNext()
}
@Override
- public CompactionCandidate next()
+ public CompactionCandidateAndStatus next()
{
if (!hasNext()) {
throw new NoSuchElementException();
}
- final CompactionCandidate entry = queue.poll();
+ final CompactionCandidateAndStatus entry = queue.poll();
if (entry == null) {
throw new NoSuchElementException();
}
- addNextItemForDatasourceToQueue(entry.getDataSource());
+ addNextItemForDatasourceToQueue(entry.getCandidate().getDataSource());
return entry;
}
@@ -121,7 +121,7 @@ private void addNextItemForDatasourceToQueue(String dataSourceName)
{
final DataSourceCompactibleSegmentIterator iterator = datasourceIterators.get(dataSourceName);
if (iterator.hasNext()) {
- final CompactionCandidate compactionCandidate = iterator.next();
+ final CompactionCandidateAndStatus compactionCandidate = iterator.next();
if (compactionCandidate != null) {
queue.add(compactionCandidate);
}
diff --git a/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java b/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java
index c376aac34406..e10e8d1a456c 100644
--- a/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java
+++ b/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java
@@ -34,7 +34,9 @@
import org.apache.druid.common.guava.FutureUtils;
import org.apache.druid.common.utils.IdUtils;
import org.apache.druid.data.input.impl.AggregateProjectionSpec;
+import org.apache.druid.error.DruidException;
import org.apache.druid.indexer.CompactionEngine;
+import org.apache.druid.indexer.TaskState;
import org.apache.druid.java.util.common.granularity.Granularity;
import org.apache.druid.java.util.common.granularity.GranularityType;
import org.apache.druid.java.util.common.logger.Logger;
@@ -42,11 +44,11 @@
import org.apache.druid.rpc.indexing.OverlordClient;
import org.apache.druid.segment.transform.CompactionTransformSpec;
import org.apache.druid.server.compaction.CompactionCandidate;
+import org.apache.druid.server.compaction.CompactionCandidateAndStatus;
import org.apache.druid.server.compaction.CompactionCandidateSearchPolicy;
import org.apache.druid.server.compaction.CompactionSegmentIterator;
import org.apache.druid.server.compaction.CompactionSlotManager;
import org.apache.druid.server.compaction.CompactionSnapshotBuilder;
-import org.apache.druid.server.compaction.CompactionStatus;
import org.apache.druid.server.compaction.CompactionStatusTracker;
import org.apache.druid.server.compaction.PriorityBasedCompactionSegmentIterator;
import org.apache.druid.server.coordinator.AutoCompactionSnapshot;
@@ -240,28 +242,40 @@ private int submitCompactionTasks(
int totalTaskSlotsAssigned = 0;
while (iterator.hasNext() && totalTaskSlotsAssigned < slotManager.getNumAvailableTaskSlots()) {
- final CompactionCandidate entry = iterator.next();
- final String dataSourceName = entry.getDataSource();
+ final CompactionCandidateAndStatus candidateAndStatus = iterator.next();
+ final CompactionCandidate candidate = candidateAndStatus.getCandidate();
+ final String dataSourceName = candidate.getDataSource();
final DataSourceCompactionConfig config = compactionConfigs.get(dataSourceName);
- final CompactionStatus compactionStatus =
- statusTracker.computeCompactionStatus(entry, policy);
- final CompactionCandidate candidatesWithStatus = entry.withCurrentStatus(compactionStatus);
- statusTracker.onCompactionStatusComputed(candidatesWithStatus, config);
-
- if (compactionStatus.isComplete()) {
- snapshotBuilder.addToComplete(candidatesWithStatus);
- continue;
- } else if (compactionStatus.isSkipped()) {
- snapshotBuilder.addToSkipped(candidatesWithStatus);
- continue;
- } else {
- // As these segments will be compacted, we will aggregate the statistic to the Compacted statistics
- snapshotBuilder.addToComplete(entry);
+ final TaskState compactionTaskState = statusTracker.computeCompactionTaskState(candidate);
+ statusTracker.onCompactionTaskStateComputed(candidateAndStatus, compactionTaskState, config);
+
+ switch (candidateAndStatus.getStatus().getState()) {
+ case COMPLETE:
+ snapshotBuilder.addToComplete(candidate);
+ continue;
+ case NOT_ELIGIBLE:
+ snapshotBuilder.addToSkipped(candidate);
+ continue;
+ case ELIGIBLE:
+ CompactionCandidateSearchPolicy.Eligibility eligibility = policy.checkEligibilityForCompaction(candidateAndStatus);
+ if (!eligibility.isEligible()) {
+ statusTracker.onSkippedCandidate(candidateAndStatus, eligibility.getReason());
+ snapshotBuilder.addToSkipped(candidate);
+ continue;
+ }
+ // As these segments will be compacted, we will aggregate the statistic to the Compacted statistics
+ snapshotBuilder.addToComplete(candidate);
+ break;
+ default:
+ throw DruidException.defensive(
+ "unexpected compaction candidate state[%s]",
+ candidateAndStatus.getStatus().getState()
+ );
}
final ClientCompactionTaskQuery taskPayload = createCompactionTask(
- entry,
+ candidateAndStatus,
config,
defaultEngine,
null,
@@ -270,13 +284,13 @@ private int submitCompactionTasks(
final String taskId = taskPayload.getId();
FutureUtils.getUnchecked(overlordClient.runTask(taskId, taskPayload), true);
- statusTracker.onTaskSubmitted(taskId, entry);
+ statusTracker.onTaskSubmitted(taskId, candidateAndStatus);
LOG.debug(
"Submitted a compaction task[%s] for [%d] segments in datasource[%s], umbrella interval[%s].",
- taskId, entry.numSegments(), dataSourceName, entry.getUmbrellaInterval()
+ taskId, candidate.numSegments(), dataSourceName, candidate.getUmbrellaInterval()
);
- LOG.debugSegments(entry.getSegments(), "Compacting segments");
+ LOG.debugSegments(candidate.getSegments(), "Compacting segments");
numSubmittedTasks++;
totalTaskSlotsAssigned += slotManager.computeSlotsRequiredForTask(taskPayload, config);
}
@@ -290,14 +304,14 @@ private int submitCompactionTasks(
* {@link OverlordClient} to start a compaction task.
*/
public static ClientCompactionTaskQuery createCompactionTask(
- CompactionCandidate candidate,
+ CompactionCandidateAndStatus candidate,
DataSourceCompactionConfig config,
CompactionEngine defaultEngine,
String indexingStateFingerprint,
boolean storeCompactionStatePerSegment
)
{
- final List segmentsToCompact = candidate.getSegments();
+ final List segmentsToCompact = candidate.getCandidate().getSegments();
// Create granularitySpec to send to compaction task
Granularity segmentGranularityToUse = null;
@@ -371,15 +385,12 @@ public static ClientCompactionTaskQuery createCompactionTask(
}
final Map autoCompactionContext = newAutoCompactionContext(config.getTaskContext());
- if (candidate.getCurrentStatus() != null) {
- autoCompactionContext.put(COMPACTION_REASON_KEY, candidate.getCurrentStatus().getReason());
- }
-
+ autoCompactionContext.put(COMPACTION_REASON_KEY, candidate.getStatus().getReason());
autoCompactionContext.put(STORE_COMPACTION_STATE_KEY, storeCompactionStatePerSegment);
autoCompactionContext.put(INDEXING_STATE_FINGERPRINT_KEY, indexingStateFingerprint);
return compactSegments(
- candidate,
+ candidate.getCandidate(),
config.getTaskPriority(),
ClientCompactionTaskQueryTuningConfig.from(
config.getTuningConfig(),
@@ -414,12 +425,12 @@ private void updateCompactionSnapshotStats(
{
// Mark all the segments remaining in the iterator as "awaiting compaction"
while (iterator.hasNext()) {
- snapshotBuilder.addToPending(iterator.next());
+ snapshotBuilder.addToPending(iterator.next().getCandidate());
}
iterator.getCompactedSegments().forEach(snapshotBuilder::addToComplete);
iterator.getSkippedSegments().forEach(entry -> {
- statusTracker.onCompactionStatusComputed(entry, datasourceToConfig.get(entry.getDataSource()));
- snapshotBuilder.addToSkipped(entry);
+ statusTracker.onSkippedCandidate(entry, null);
+ snapshotBuilder.addToSkipped(entry.getCandidate());
});
// Atomic update of autoCompactionSnapshotPerDataSource with the latest from this coordinator run
@@ -463,14 +474,13 @@ private static ClientCompactionTaskQuery compactSegments(
context.put("priority", compactionTaskPriority);
final String taskId = IdUtils.newTaskId(TASK_ID_PREFIX, ClientCompactionTaskQuery.TYPE, dataSource, null);
+ final ClientCompactionIntervalSpec clientCompactionIntervalSpec =
+ new ClientCompactionIntervalSpec(entry.getCompactionInterval(), null);
return new ClientCompactionTaskQuery(
taskId,
dataSource,
- new ClientCompactionIOConfig(
- new ClientCompactionIntervalSpec(entry.getCompactionInterval(), null),
- dropExisting
- ),
+ new ClientCompactionIOConfig(clientCompactionIntervalSpec, dropExisting),
tuningConfig,
granularitySpec,
dimensionsSpec,
diff --git a/server/src/test/java/org/apache/druid/client/indexing/ClientCompactionIntervalSpecTest.java b/server/src/test/java/org/apache/druid/client/indexing/ClientCompactionIntervalSpecTest.java
index 46ecc64d72d1..4f83e5d7e715 100644
--- a/server/src/test/java/org/apache/druid/client/indexing/ClientCompactionIntervalSpecTest.java
+++ b/server/src/test/java/org/apache/druid/client/indexing/ClientCompactionIntervalSpecTest.java
@@ -19,7 +19,6 @@
package org.apache.druid.client.indexing;
-import com.google.common.collect.ImmutableList;
import org.apache.druid.java.util.common.DateTimes;
import org.apache.druid.java.util.common.Intervals;
import org.apache.druid.java.util.common.granularity.Granularities;
@@ -32,6 +31,7 @@
import java.util.ArrayList;
import java.util.HashMap;
+import java.util.List;
public class ClientCompactionIntervalSpecTest
{
@@ -73,7 +73,7 @@ public class ClientCompactionIntervalSpecTest
public void testFromSegmentWithNoSegmentGranularity()
{
// The umbrella interval of segments is 2015-02-12/2015-04-14
- CompactionCandidate actual = CompactionCandidate.from(ImmutableList.of(dataSegment1, dataSegment2, dataSegment3), null);
+ CompactionCandidate actual = CompactionCandidate.from(List.of(dataSegment1, dataSegment2, dataSegment3), null);
Assert.assertEquals(Intervals.of("2015-02-12/2015-04-14"), actual.getCompactionInterval());
}
@@ -81,7 +81,7 @@ public void testFromSegmentWithNoSegmentGranularity()
public void testFromSegmentWitSegmentGranularitySameAsSegment()
{
// The umbrella interval of segments is 2015-04-11/2015-04-12
- CompactionCandidate actual = CompactionCandidate.from(ImmutableList.of(dataSegment1), Granularities.DAY);
+ CompactionCandidate actual = CompactionCandidate.from(List.of(dataSegment1), Granularities.DAY);
Assert.assertEquals(Intervals.of("2015-04-11/2015-04-12"), actual.getCompactionInterval());
}
@@ -89,7 +89,10 @@ public void testFromSegmentWitSegmentGranularitySameAsSegment()
public void testFromSegmentWithCoarserSegmentGranularity()
{
// The umbrella interval of segments is 2015-02-12/2015-04-14
- CompactionCandidate actual = CompactionCandidate.from(ImmutableList.of(dataSegment1, dataSegment2, dataSegment3), Granularities.YEAR);
+ CompactionCandidate actual = CompactionCandidate.from(
+ List.of(dataSegment1, dataSegment2, dataSegment3),
+ Granularities.YEAR
+ );
// The compaction interval should be expanded to start of the year and end of the year to cover the segmentGranularity
Assert.assertEquals(Intervals.of("2015-01-01/2016-01-01"), actual.getCompactionInterval());
}
@@ -98,7 +101,10 @@ public void testFromSegmentWithCoarserSegmentGranularity()
public void testFromSegmentWithFinerSegmentGranularityAndUmbrellaIntervalAlign()
{
// The umbrella interval of segments is 2015-02-12/2015-04-14
- CompactionCandidate actual = CompactionCandidate.from(ImmutableList.of(dataSegment1, dataSegment2, dataSegment3), Granularities.DAY);
+ CompactionCandidate actual = CompactionCandidate.from(
+ List.of(dataSegment1, dataSegment2, dataSegment3),
+ Granularities.DAY
+ );
// The segmentGranularity of DAY align with the umbrella interval (umbrella interval can be evenly divide into the segmentGranularity)
Assert.assertEquals(Intervals.of("2015-02-12/2015-04-14"), actual.getCompactionInterval());
}
@@ -107,7 +113,10 @@ public void testFromSegmentWithFinerSegmentGranularityAndUmbrellaIntervalAlign()
public void testFromSegmentWithFinerSegmentGranularityAndUmbrellaIntervalNotAlign()
{
// The umbrella interval of segments is 2015-02-12/2015-04-14
- CompactionCandidate actual = CompactionCandidate.from(ImmutableList.of(dataSegment1, dataSegment2, dataSegment3), Granularities.WEEK);
+ CompactionCandidate actual = CompactionCandidate.from(
+ List.of(dataSegment1, dataSegment2, dataSegment3),
+ Granularities.WEEK
+ );
// The segmentGranularity of WEEK does not align with the umbrella interval (umbrella interval cannot be evenly divide into the segmentGranularity)
// Hence the compaction interval is modified to aling with the segmentGranularity
Assert.assertEquals(Intervals.of("2015-02-09/2015-04-20"), actual.getCompactionInterval());
diff --git a/server/src/test/java/org/apache/druid/server/compaction/CompactionCandidateTest.java b/server/src/test/java/org/apache/druid/server/compaction/CompactionCandidateTest.java
new file mode 100644
index 000000000000..2c8f2c0ee8c2
--- /dev/null
+++ b/server/src/test/java/org/apache/druid/server/compaction/CompactionCandidateTest.java
@@ -0,0 +1,72 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.server.compaction;
+
+import org.apache.druid.error.DruidException;
+import org.apache.druid.java.util.common.granularity.Granularities;
+import org.apache.druid.server.coordinator.CreateDataSegments;
+import org.apache.druid.timeline.DataSegment;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.util.Collections;
+import java.util.List;
+
+public class CompactionCandidateTest
+{
+ private static final String DATASOURCE = "test_datasource";
+
+ @Test
+ public void testFrom()
+ {
+ List segments = createTestSegments(3);
+
+ CompactionCandidate candidate = CompactionCandidate.from(segments, null);
+
+ Assert.assertEquals(segments, candidate.getSegments());
+ Assert.assertEquals(DATASOURCE, candidate.getDataSource());
+ Assert.assertEquals(3, candidate.numSegments());
+ Assert.assertNotNull(candidate.getUmbrellaInterval());
+ Assert.assertNotNull(candidate.getCompactionInterval());
+ Assert.assertNotNull(candidate.getStats());
+ }
+
+ @Test
+ public void testThrowsOnNullOrEmptySegments()
+ {
+ Assert.assertThrows(
+ DruidException.class,
+ () -> CompactionCandidate.from(null, null)
+ );
+
+ Assert.assertThrows(
+ DruidException.class,
+ () -> CompactionCandidate.from(Collections.emptyList(), null)
+ );
+ }
+
+ private static List createTestSegments(int count)
+ {
+ return CreateDataSegments.ofDatasource(DATASOURCE)
+ .forIntervals(count, Granularities.DAY)
+ .startingAt("2024-01-01")
+ .eachOfSizeInMb(100);
+ }
+}
diff --git a/server/src/test/java/org/apache/druid/server/compaction/CompactionRunSimulatorTest.java b/server/src/test/java/org/apache/druid/server/compaction/CompactionRunSimulatorTest.java
index 56ec8525bb83..f0e9acc47744 100644
--- a/server/src/test/java/org/apache/druid/server/compaction/CompactionRunSimulatorTest.java
+++ b/server/src/test/java/org/apache/druid/server/compaction/CompactionRunSimulatorTest.java
@@ -23,6 +23,7 @@
import com.google.common.util.concurrent.ListenableFuture;
import org.apache.druid.client.indexing.TaskPayloadResponse;
import org.apache.druid.indexer.CompactionEngine;
+import org.apache.druid.indexer.TaskState;
import org.apache.druid.indexer.TaskStatus;
import org.apache.druid.indexer.TaskStatusPlus;
import org.apache.druid.java.util.common.CloseableIterators;
@@ -80,13 +81,13 @@ public void testSimulateClusterCompactionConfigUpdate()
Assert.assertNotNull(simulateResult);
- final Map compactionStates = simulateResult.getCompactionStates();
+ final Map compactionStates = simulateResult.getCompactionStates();
Assert.assertNotNull(compactionStates);
- Assert.assertNull(compactionStates.get(CompactionStatus.State.COMPLETE));
- Assert.assertNull(compactionStates.get(CompactionStatus.State.RUNNING));
+ Assert.assertNull(compactionStates.get(TaskState.SUCCESS));
+ Assert.assertNull(compactionStates.get(TaskState.RUNNING));
- final Table queuedTable = compactionStates.get(CompactionStatus.State.PENDING);
+ final Table queuedTable = simulateResult.getQueuedIntervals();
Assert.assertEquals(
Arrays.asList("dataSource", "interval", "numSegments", "bytes", "maxTaskSlots", "reasonToCompact"),
queuedTable.getColumnNames()
@@ -106,7 +107,7 @@ public void testSimulateClusterCompactionConfigUpdate()
queuedTable.getRows()
);
- final Table skippedTable = compactionStates.get(CompactionStatus.State.SKIPPED);
+ final Table skippedTable = simulateResult.getSkippedIntervals();
Assert.assertEquals(
Arrays.asList("dataSource", "interval", "numSegments", "bytes", "reasonToSkip"),
skippedTable.getColumnNames()
@@ -153,13 +154,13 @@ public void testSimulate_withFixedIntervalOrderPolicy()
Assert.assertNotNull(simulateResult);
- final Map compactionStates = simulateResult.getCompactionStates();
+ final Map compactionStates = simulateResult.getCompactionStates();
Assert.assertNotNull(compactionStates);
- Assert.assertNull(compactionStates.get(CompactionStatus.State.COMPLETE));
- Assert.assertNull(compactionStates.get(CompactionStatus.State.RUNNING));
+ Assert.assertNull(compactionStates.get(TaskState.SUCCESS));
+ Assert.assertNull(compactionStates.get(TaskState.RUNNING));
- final Table pendingTable = compactionStates.get(CompactionStatus.State.PENDING);
+ final Table pendingTable = simulateResult.getQueuedIntervals();
Assert.assertEquals(
List.of("dataSource", "interval", "numSegments", "bytes", "maxTaskSlots", "reasonToCompact"),
pendingTable.getColumnNames()
@@ -172,13 +173,12 @@ public void testSimulate_withFixedIntervalOrderPolicy()
pendingTable.getRows()
);
- final Table skippedTable = compactionStates.get(CompactionStatus.State.SKIPPED);
+ final Table skippedTable = simulateResult.getSkippedIntervals();
Assert.assertEquals(
List.of("dataSource", "interval", "numSegments", "bytes", "reasonToSkip"),
skippedTable.getColumnNames()
);
- final String rejectedMessage
- = "Rejected by search policy: Datasource/Interval is not in the list of 'eligibleCandidates'";
+ final String rejectedMessage = "Datasource/Interval is not in the list of 'eligibleCandidates'";
Assert.assertEquals(
List.of(
List.of("wiki", Intervals.of("2013-01-02/P1D"), 10, 1_000_000_000L, 1, rejectedMessage),
diff --git a/server/src/test/java/org/apache/druid/server/compaction/CompactionStatusBuilderTest.java b/server/src/test/java/org/apache/druid/server/compaction/CompactionStatusBuilderTest.java
new file mode 100644
index 000000000000..d0c381b02706
--- /dev/null
+++ b/server/src/test/java/org/apache/druid/server/compaction/CompactionStatusBuilderTest.java
@@ -0,0 +1,165 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.server.compaction;
+
+import org.apache.druid.error.DruidException;
+import org.apache.druid.java.util.common.granularity.Granularities;
+import org.apache.druid.server.coordinator.CreateDataSegments;
+import org.apache.druid.timeline.DataSegment;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.util.Collections;
+import java.util.List;
+
+public class CompactionStatusBuilderTest
+{
+ private static final String DATASOURCE = "test_datasource";
+
+ @Test
+ public void testNotEligible()
+ {
+ CompactionStatus eligibility = CompactionStatus.notEligible("test reason: %s", "failure");
+
+ Assert.assertEquals(CompactionStatus.State.NOT_ELIGIBLE, eligibility.getState());
+ Assert.assertEquals("test reason: failure", eligibility.getReason());
+ Assert.assertNull(eligibility.getCompactedStats());
+ Assert.assertNull(eligibility.getUncompactedStats());
+ }
+
+ @Test
+ public void testBuilderWithCompactionStats()
+ {
+ CompactionStatistics compactedStats = CompactionStatistics.create(1000, 5, 2);
+ CompactionStatistics uncompactedStats = CompactionStatistics.create(500, 3, 1);
+
+ CompactionStatus eligibility =
+ CompactionStatus.builder(CompactionStatus.State.ELIGIBLE, "needs full compaction")
+ .compacted(compactedStats)
+ .uncompacted(uncompactedStats)
+ .build();
+
+ Assert.assertEquals(CompactionStatus.State.ELIGIBLE, eligibility.getState());
+ Assert.assertEquals("needs full compaction", eligibility.getReason());
+ Assert.assertEquals(compactedStats, eligibility.getCompactedStats());
+ Assert.assertEquals(uncompactedStats, eligibility.getUncompactedStats());
+ }
+
+ @Test
+ public void testEqualsAndHashCode()
+ {
+ // Test with simple eligibility objects (same state and reason)
+ CompactionStatus simple1 = CompactionStatus.notEligible("reason");
+ CompactionStatus simple2 = CompactionStatus.notEligible("reason");
+ Assert.assertEquals(simple1, simple2);
+ Assert.assertEquals(simple1.hashCode(), simple2.hashCode());
+
+ // Test with different reasons
+ CompactionStatus differentReason = CompactionStatus.notEligible("different");
+ Assert.assertNotEquals(simple1, differentReason);
+
+ // Test with different states
+ CompactionStatus differentState = CompactionStatus.COMPLETE;
+ Assert.assertNotEquals(simple1, differentState);
+
+ // Test with full compaction eligibility (with stats)
+ CompactionStatistics stats1 = CompactionStatistics.create(1000, 5, 2);
+ CompactionStatistics stats2 = CompactionStatistics.create(500, 3, 1);
+
+ CompactionStatus withStats1 =
+ CompactionStatus.builder(CompactionStatus.State.ELIGIBLE, "reason")
+ .compacted(stats1)
+ .uncompacted(stats2)
+ .build();
+
+ CompactionStatus withStats2 =
+ CompactionStatus.builder(CompactionStatus.State.ELIGIBLE, "reason")
+ .compacted(stats1)
+ .uncompacted(stats2)
+ .build();
+
+ // Same values - should be equal
+ Assert.assertEquals(withStats1, withStats2);
+ Assert.assertEquals(withStats1.hashCode(), withStats2.hashCode());
+
+ // Test with different compacted stats
+ CompactionStatistics differentStats = CompactionStatistics.create(2000, 10, 5);
+ CompactionStatus differentCompactedStats =
+ CompactionStatus.builder(CompactionStatus.State.ELIGIBLE, "reason")
+ .compacted(differentStats)
+ .uncompacted(stats2)
+ .build();
+ Assert.assertNotEquals(withStats1, differentCompactedStats);
+
+ // Test with different uncompacted stats
+ CompactionStatus differentUncompactedStats =
+ CompactionStatus.builder(CompactionStatus.State.ELIGIBLE, "reason")
+ .compacted(stats1)
+ .uncompacted(differentStats)
+ .build();
+ Assert.assertNotEquals(withStats1, differentUncompactedStats);
+ }
+
+ @Test
+ public void testBuilderRequiresReasonForNotEligible()
+ {
+ Assert.assertThrows(
+ DruidException.class,
+ () -> CompactionStatus.builder(CompactionStatus.State.NOT_ELIGIBLE, null).build()
+ );
+ }
+
+ @Test
+ public void testBuilderRequiresStatsForFullCompaction()
+ {
+ // Should throw when neither stat is provided
+ Assert.assertThrows(
+ DruidException.class,
+ () -> CompactionStatus.builder(CompactionStatus.State.ELIGIBLE, "reason").build()
+ );
+
+ // Should throw when only compacted stat is provided
+ Assert.assertThrows(
+ DruidException.class,
+ () -> CompactionStatus.builder(CompactionStatus.State.ELIGIBLE, "reason")
+ .compacted(CompactionStatistics.create(1000, 5, 2))
+ .build()
+ );
+
+ // Should succeed when both stats are provided
+ CompactionStatus status = CompactionStatus.builder(CompactionStatus.State.ELIGIBLE, "reason")
+ .compacted(CompactionStatistics.create(1000, 5, 2))
+ .uncompacted(CompactionStatistics.create(500, 3, 1))
+ .build();
+ Assert.assertNotNull(status);
+ }
+
+ private static List createTestSegments(int count)
+ {
+ if (count == 0) {
+ return Collections.emptyList();
+ }
+
+ return CreateDataSegments.ofDatasource(DATASOURCE)
+ .forIntervals(count, Granularities.DAY)
+ .startingAt("2024-01-01")
+ .eachOfSizeInMb(100);
+ }
+}
diff --git a/server/src/test/java/org/apache/druid/server/compaction/CompactionStatusTest.java b/server/src/test/java/org/apache/druid/server/compaction/CompactionStatusTest.java
index 370dcaf444d5..6341ba3ace73 100644
--- a/server/src/test/java/org/apache/druid/server/compaction/CompactionStatusTest.java
+++ b/server/src/test/java/org/apache/druid/server/compaction/CompactionStatusTest.java
@@ -105,9 +105,7 @@ public void testFindPartitionsSpecWhenGivenIsNull()
{
final ClientCompactionTaskQueryTuningConfig tuningConfig
= ClientCompactionTaskQueryTuningConfig.from(null);
- Assert.assertNull(
- CompactionStatus.findPartitionsSpecFromConfig(tuningConfig)
- );
+ Assert.assertNull(CompactionStatus.findPartitionsSpecFromConfig(tuningConfig));
}
@Test
@@ -179,9 +177,7 @@ public void testFindPartitionsSpecFromConfigWithDeprecatedMaxRowsPerSegmentAndMa
.build();
Assert.assertEquals(
new DynamicPartitionsSpec(100, 1000L),
- CompactionStatus.findPartitionsSpecFromConfig(
- ClientCompactionTaskQueryTuningConfig.from(config)
- )
+ CompactionStatus.findPartitionsSpecFromConfig(ClientCompactionTaskQueryTuningConfig.from(config))
);
}
@@ -227,7 +223,7 @@ public void testFindPartitionsSpecWhenGivenIsRangeWithTargetRows()
@Test
public void testStatusWhenLastCompactionStateIsNull()
{
- verifyCompactionStatusIsPendingBecause(
+ verifyCompactionIsEligibleBecause(
null,
InlineSchemaDataSourceCompactionConfig.builder().forDataSource(TestDataSource.WIKI).build(),
"not compacted yet"
@@ -238,7 +234,7 @@ public void testStatusWhenLastCompactionStateIsNull()
public void testStatusWhenLastCompactionStateIsEmpty()
{
final PartitionsSpec requiredPartitionsSpec = new DynamicPartitionsSpec(5_000_000, null);
- verifyCompactionStatusIsPendingBecause(
+ verifyCompactionIsEligibleBecause(
new CompactionState(null, null, null, null, null, null, null),
InlineSchemaDataSourceCompactionConfig
.builder()
@@ -263,7 +259,7 @@ public void testStatusOnPartitionsSpecMismatch()
.forDataSource(TestDataSource.WIKI)
.build();
- verifyCompactionStatusIsPendingBecause(
+ verifyCompactionIsEligibleBecause(
lastCompactionState,
compactionConfig,
"'partitionsSpec' mismatch: required['dynamic' with 5,000,000 rows],"
@@ -293,7 +289,7 @@ public void testStatusOnIndexSpecMismatch()
.withTuningConfig(createTuningConfig(currentPartitionsSpec, null))
.build();
- verifyCompactionStatusIsPendingBecause(
+ verifyCompactionIsEligibleBecause(
lastCompactionState,
compactionConfig,
"'indexSpec' mismatch: "
@@ -335,7 +331,7 @@ public void testStatusOnSegmentGranularityMismatch()
.withGranularitySpec(new UserCompactionTaskGranularityConfig(Granularities.DAY, null, null))
.build();
- verifyCompactionStatusIsPendingBecause(
+ verifyCompactionIsEligibleBecause(
lastCompactionState,
compactionConfig,
"'segmentGranularity' mismatch: required[DAY], current[HOUR]"
@@ -372,7 +368,7 @@ public void testStatusWhenLastCompactionStateSameAsRequired()
compactionConfig,
fingerprintMapper
);
- Assert.assertTrue(status.isComplete());
+ Assert.assertEquals(CompactionStatus.State.COMPLETE, status.getState());
}
@Test
@@ -422,7 +418,7 @@ public void testStatusWhenProjectionsMatch()
compactionConfig,
fingerprintMapper
);
- Assert.assertTrue(status.isComplete());
+ Assert.assertEquals(CompactionStatus.COMPLETE, status);
}
@Test
@@ -477,7 +473,8 @@ public void testStatusWhenProjectionsMismatch()
compactionConfig,
fingerprintMapper
);
- Assert.assertFalse(status.isComplete());
+ Assert.assertEquals(CompactionStatus.State.ELIGIBLE, status.getState());
+ Assert.assertTrue(status.getReason().contains("'projections' mismatch"));
}
@Test
@@ -509,7 +506,7 @@ public void testStatusWhenTransformSpecVirtualColumnsMatch()
CompactionStatus status = CompactionStatus.compute(
CompactionCandidate.from(List.of(segment), null), compactionConfig, fingerprintMapper
);
- Assert.assertTrue(status.isComplete());
+ Assert.assertEquals(CompactionStatus.State.COMPLETE, status.getState());
}
@Test
@@ -542,7 +539,7 @@ public void testStatusWhenTransformSpecVirtualColumnsMismatch()
CompactionStatus status = CompactionStatus.compute(
CompactionCandidate.from(List.of(segment), null), compactionConfig, fingerprintMapper
);
- Assert.assertFalse(status.isComplete());
+ Assert.assertEquals(CompactionStatus.State.ELIGIBLE, status.getState());
Assert.assertTrue(status.getReason().startsWith("'transformSpec' mismatch"));
}
@@ -577,7 +574,7 @@ public void test_evaluate_needsCompactionWhenMismatchedFingerprintStateHasDiffer
CompactionStatus status = CompactionStatus.compute(
CompactionCandidate.from(segments, null), newConfig, fingerprintMapper
);
- Assert.assertFalse(status.isComplete());
+ Assert.assertEquals(CompactionStatus.State.ELIGIBLE, status.getState());
Assert.assertTrue(status.getReason().startsWith("'transformSpec' mismatch"));
}
@@ -632,7 +629,7 @@ public void testStatusWhenAutoSchemaMatch()
compactionConfig,
fingerprintMapper
);
- Assert.assertTrue(status.isComplete());
+ Assert.assertEquals(CompactionStatus.COMPLETE, status);
}
@Test
@@ -686,7 +683,8 @@ public void testStatusWhenAutoSchemaMismatch()
compactionConfig,
fingerprintMapper
);
- Assert.assertFalse(status.isComplete());
+ Assert.assertEquals(CompactionStatus.State.ELIGIBLE, status.getState());
+ Assert.assertTrue(status.getReason().contains("'dimensionsSpec' mismatch"));
}
@Test
@@ -777,7 +775,7 @@ public void test_evaluate_noCompacationIfUnexpectedFingerprintHasExpectedIndexin
compactionConfig,
fingerprintMapper
);
- Assert.assertTrue(status.isComplete());
+ Assert.assertEquals(CompactionStatus.COMPLETE, status);
}
@Test
@@ -822,7 +820,7 @@ public void test_evaluate_noCompactionWhenAllSegmentsHaveExpectedIndexingStateFi
compactionConfig,
fingerprintMapper
);
- Assert.assertTrue(status.isComplete());
+ Assert.assertEquals(CompactionStatus.COMPLETE, status);
}
@Test
@@ -876,7 +874,7 @@ public void test_evaluate_noCompactionWhenNonFingerprintedSegmentsPassChecksOnLa
compactionConfig,
fingerprintMapper
);
- Assert.assertTrue(status.isComplete());
+ Assert.assertEquals(CompactionStatus.COMPLETE, status);
}
// ============================
@@ -907,8 +905,7 @@ public void test_evaluate_isSkippedWhenInputBytesExceedLimit()
fingerprintMapper
);
- Assert.assertFalse(status.isComplete());
- Assert.assertTrue(status.isSkipped());
+ Assert.assertEquals(CompactionStatus.State.NOT_ELIGIBLE, status.getState());
Assert.assertTrue(status.getReason().contains("'inputSegmentSize' exceeded"));
Assert.assertTrue(status.getReason().contains("200000000"));
Assert.assertTrue(status.getReason().contains("150000000"));
@@ -924,33 +921,26 @@ private void verifyEvaluationNeedsCompactionBecauseWithCustomSegments(
String expectedReason
)
{
- final CompactionStatus status = CompactionStatus.compute(
- candidate,
- compactionConfig,
- fingerprintMapper
- );
+ final CompactionStatus status = CompactionStatus.compute(candidate, compactionConfig, fingerprintMapper);
- Assert.assertFalse(status.isComplete());
+ Assert.assertEquals(CompactionStatus.State.ELIGIBLE, status.getState());
Assert.assertEquals(expectedReason, status.getReason());
}
- private void verifyCompactionStatusIsPendingBecause(
+ private void verifyCompactionIsEligibleBecause(
CompactionState lastCompactionState,
DataSourceCompactionConfig compactionConfig,
String expectedReason
)
{
- final DataSegment segment
- = DataSegment.builder(WIKI_SEGMENT)
- .lastCompactionState(lastCompactionState)
- .build();
+ final DataSegment segment = DataSegment.builder(WIKI_SEGMENT).lastCompactionState(lastCompactionState).build();
final CompactionStatus status = CompactionStatus.compute(
CompactionCandidate.from(List.of(segment), null),
compactionConfig,
fingerprintMapper
);
- Assert.assertFalse(status.isComplete());
+ Assert.assertEquals(CompactionStatus.State.ELIGIBLE, status.getState());
Assert.assertEquals(expectedReason, status.getReason());
}
diff --git a/server/src/test/java/org/apache/druid/server/compaction/CompactionStatusTrackerTest.java b/server/src/test/java/org/apache/druid/server/compaction/CompactionStatusTrackerTest.java
index 1314a1a0bc79..d9e8c4045b61 100644
--- a/server/src/test/java/org/apache/druid/server/compaction/CompactionStatusTrackerTest.java
+++ b/server/src/test/java/org/apache/druid/server/compaction/CompactionStatusTrackerTest.java
@@ -22,6 +22,7 @@
import org.apache.druid.indexer.TaskState;
import org.apache.druid.indexer.TaskStatus;
import org.apache.druid.java.util.common.DateTimes;
+import org.apache.druid.java.util.common.granularity.Granularity;
import org.apache.druid.segment.TestDataSource;
import org.apache.druid.server.coordinator.CreateDataSegments;
import org.apache.druid.timeline.DataSegment;
@@ -29,6 +30,7 @@
import org.junit.Before;
import org.junit.Test;
+import javax.annotation.Nullable;
import java.util.List;
public class CompactionStatusTrackerTest
@@ -47,35 +49,32 @@ public void setup()
@Test
public void testGetLatestTaskStatusForSubmittedTask()
{
- final CompactionCandidate candidateSegments
- = CompactionCandidate.from(List.of(WIKI_SEGMENT), null);
+ final CompactionCandidateAndStatus candidateSegments = createCandidate(List.of(WIKI_SEGMENT), null);
statusTracker.onTaskSubmitted("task1", candidateSegments);
- CompactionTaskStatus status = statusTracker.getLatestTaskStatus(candidateSegments);
+ CompactionTaskStatus status = statusTracker.getLatestTaskStatus(candidateSegments.getCandidate());
Assert.assertEquals(TaskState.RUNNING, status.getState());
}
@Test
public void testGetLatestTaskStatusForSuccessfulTask()
{
- final CompactionCandidate candidateSegments
- = CompactionCandidate.from(List.of(WIKI_SEGMENT), null);
+ final CompactionCandidateAndStatus candidateSegments = createCandidate(List.of(WIKI_SEGMENT), null);
statusTracker.onTaskSubmitted("task1", candidateSegments);
statusTracker.onTaskFinished("task1", TaskStatus.success("task1"));
- CompactionTaskStatus status = statusTracker.getLatestTaskStatus(candidateSegments);
+ CompactionTaskStatus status = statusTracker.getLatestTaskStatus(candidateSegments.getCandidate());
Assert.assertEquals(TaskState.SUCCESS, status.getState());
}
@Test
public void testGetLatestTaskStatusForFailedTask()
{
- final CompactionCandidate candidateSegments
- = CompactionCandidate.from(List.of(WIKI_SEGMENT), null);
+ final CompactionCandidateAndStatus candidateSegments = createCandidate(List.of(WIKI_SEGMENT), null);
statusTracker.onTaskSubmitted("task1", candidateSegments);
statusTracker.onTaskFinished("task1", TaskStatus.failure("task1", "some failure"));
- CompactionTaskStatus status = statusTracker.getLatestTaskStatus(candidateSegments);
+ CompactionTaskStatus status = statusTracker.getLatestTaskStatus(candidateSegments.getCandidate());
Assert.assertEquals(TaskState.FAILED, status.getState());
Assert.assertEquals(1, status.getNumConsecutiveFailures());
}
@@ -83,52 +82,60 @@ public void testGetLatestTaskStatusForFailedTask()
@Test
public void testGetLatestTaskStatusForRepeatedlyFailingTask()
{
- final CompactionCandidate candidateSegments
- = CompactionCandidate.from(List.of(WIKI_SEGMENT), null);
+ final CompactionCandidateAndStatus candidateSegments = createCandidate(List.of(WIKI_SEGMENT), null);
statusTracker.onTaskSubmitted("task1", candidateSegments);
statusTracker.onTaskFinished("task1", TaskStatus.failure("task1", "some failure"));
statusTracker.onTaskSubmitted("task2", candidateSegments);
- CompactionTaskStatus status = statusTracker.getLatestTaskStatus(candidateSegments);
+ CompactionTaskStatus status = statusTracker.getLatestTaskStatus(candidateSegments.getCandidate());
Assert.assertEquals(TaskState.RUNNING, status.getState());
Assert.assertEquals(1, status.getNumConsecutiveFailures());
statusTracker.onTaskFinished("task2", TaskStatus.failure("task2", "second failure"));
- status = statusTracker.getLatestTaskStatus(candidateSegments);
+ status = statusTracker.getLatestTaskStatus(candidateSegments.getCandidate());
Assert.assertEquals(TaskState.FAILED, status.getState());
Assert.assertEquals(2, status.getNumConsecutiveFailures());
}
@Test
- public void testComputeCompactionStatusForSuccessfulTask()
+ public void testComputeCompactionTaskStateForSuccessfulTask()
{
final NewestSegmentFirstPolicy policy = new NewestSegmentFirstPolicy(null);
- final CompactionCandidate candidateSegments
- = CompactionCandidate.from(List.of(WIKI_SEGMENT), null);
+ final CompactionCandidateAndStatus candidateSegments = createCandidate(List.of(WIKI_SEGMENT), null);
// Verify that interval is originally eligible for compaction
- CompactionStatus status
- = statusTracker.computeCompactionStatus(candidateSegments, policy);
- Assert.assertEquals(CompactionStatus.State.PENDING, status.getState());
- Assert.assertEquals("Not compacted yet", status.getReason());
+ TaskState status = statusTracker.computeCompactionTaskState(candidateSegments.getCandidate());
+ Assert.assertNull(status);
// Verify that interval is skipped for compaction after task has finished
statusTracker.onSegmentTimelineUpdated(DateTimes.nowUtc().minusMinutes(1));
statusTracker.onTaskSubmitted("task1", candidateSegments);
statusTracker.onTaskFinished("task1", TaskStatus.success("task1"));
- status = statusTracker.computeCompactionStatus(candidateSegments, policy);
- Assert.assertEquals(CompactionStatus.State.SKIPPED, status.getState());
- Assert.assertEquals(
- "Segment timeline not updated since last compaction task succeeded",
- status.getReason()
- );
+ status = statusTracker.computeCompactionTaskState(candidateSegments.getCandidate());
+ Assert.assertEquals(TaskState.SUCCESS, status);
// Verify that interval becomes eligible again after timeline has been updated
statusTracker.onSegmentTimelineUpdated(DateTimes.nowUtc());
- status = statusTracker.computeCompactionStatus(candidateSegments, policy);
- Assert.assertEquals(CompactionStatus.State.PENDING, status.getState());
+ status = statusTracker.computeCompactionTaskState(candidateSegments.getCandidate());
+ Assert.assertNull(status);
+ }
+
+ private static CompactionCandidateAndStatus createCandidate(
+ List segments,
+ @Nullable Granularity targetSegmentGranularity
+ )
+ {
+ CompactionCandidate candidate = CompactionCandidate.from(
+ segments,
+ targetSegmentGranularity
+ );
+ CompactionStatus status = CompactionStatus.builder(CompactionStatus.State.ELIGIBLE, "approve without check")
+ .compacted(CompactionStatistics.create(1, 1, 1))
+ .uncompacted(CompactionStatistics.create(1, 1, 1))
+ .build();
+ return new CompactionCandidateAndStatus(candidate, status);
}
}
diff --git a/server/src/test/java/org/apache/druid/server/compaction/MostFragmentedIntervalFirstPolicyTest.java b/server/src/test/java/org/apache/druid/server/compaction/MostFragmentedIntervalFirstPolicyTest.java
index 1b93bfa03a55..562886a69b04 100644
--- a/server/src/test/java/org/apache/druid/server/compaction/MostFragmentedIntervalFirstPolicyTest.java
+++ b/server/src/test/java/org/apache/druid/server/compaction/MostFragmentedIntervalFirstPolicyTest.java
@@ -22,8 +22,8 @@
import nl.jqno.equalsverifier.EqualsVerifier;
import org.apache.druid.jackson.DefaultObjectMapper;
import org.apache.druid.java.util.common.HumanReadableBytes;
-import org.apache.druid.segment.TestDataSource;
-import org.apache.druid.server.coordinator.CreateDataSegments;
+import org.apache.druid.java.util.common.Intervals;
+import org.apache.druid.segment.TestSegmentUtils;
import org.apache.druid.timeline.DataSegment;
import org.junit.Test;
import org.junit.jupiter.api.Assertions;
@@ -34,12 +34,18 @@
public class MostFragmentedIntervalFirstPolicyTest
{
private static final DataSegment SEGMENT =
- CreateDataSegments.ofDatasource(TestDataSource.WIKI).eachOfSizeInMb(100).get(0);
+ TestSegmentUtils.makeSegment("foo", "1", Intervals.ETERNITY);
+ private static final DataSegment SEGMENT2 =
+ TestSegmentUtils.makeSegment("foo", "2", Intervals.ETERNITY);
+ private static final CompactionCandidate PROPOSED_COMPACTION = CompactionCandidate.from(List.of(SEGMENT, SEGMENT2), null);
+
+ private static final CompactionStatistics DUMMY_COMPACTION_STATS = CompactionStatistics.create(1L, 1L, 1L);
@Test
public void test_thresholdValues_ofDefaultPolicy()
{
- final MostFragmentedIntervalFirstPolicy policy = new MostFragmentedIntervalFirstPolicy(null, null, null, null);
+ final MostFragmentedIntervalFirstPolicy policy =
+ new MostFragmentedIntervalFirstPolicy(null, null, null, null);
Assertions.assertEquals(100, policy.getMinUncompactedCount());
Assertions.assertEquals(new HumanReadableBytes("10MiB"), policy.getMinUncompactedBytes());
Assertions.assertEquals(new HumanReadableBytes("2GiB"), policy.getMaxAverageUncompactedBytesPerSegment());
@@ -47,7 +53,7 @@ public void test_thresholdValues_ofDefaultPolicy()
}
@Test
- public void test_checkEligibilityForCompaction_fails_ifUncompactedCountLessThanCutoff()
+ public void test_createCandidate_fails_ifUncompactedCountLessThanCutoff()
{
final int minUncompactedCount = 10_000;
final MostFragmentedIntervalFirstPolicy policy = new MostFragmentedIntervalFirstPolicy(
@@ -57,20 +63,27 @@ public void test_checkEligibilityForCompaction_fails_ifUncompactedCountLessThanC
null
);
+ final CompactionStatus status1 =
+ eligibilityBuilder().compacted(DUMMY_COMPACTION_STATS).uncompacted(createStats(1, 100L)).build();
+ final CompactionCandidateSearchPolicy.Eligibility eligibility1 =
+ policy.checkEligibilityForCompaction(new CompactionCandidateAndStatus(PROPOSED_COMPACTION, status1));
Assertions.assertEquals(
- CompactionCandidateSearchPolicy.Eligibility.fail(
- "Uncompacted segments[1] in interval must be at least [10,000]"
- ),
- policy.checkEligibilityForCompaction(createCandidate(1, 100L), null)
- );
- Assertions.assertEquals(
- CompactionCandidateSearchPolicy.Eligibility.OK,
- policy.checkEligibilityForCompaction(createCandidate(10_001, 100L), null)
+ "Uncompacted segments[1] in interval must be at least [10,000]",
+ eligibility1.getReason()
);
+ Assertions.assertFalse(eligibility1.isEligible());
+ Assertions.assertNull(eligibility1.getMode());
+
+ final CompactionStatus status2 =
+ eligibilityBuilder().compacted(DUMMY_COMPACTION_STATS).uncompacted(createStats(10_001, 100L)).build();
+ final CompactionCandidateSearchPolicy.Eligibility eligibility2 =
+ policy.checkEligibilityForCompaction(new CompactionCandidateAndStatus(PROPOSED_COMPACTION, status2));
+ Assertions.assertTrue(eligibility2.isEligible());
+ Assertions.assertEquals(CompactionMode.FULL_COMPACTION, eligibility2.getMode());
}
@Test
- public void test_checkEligibilityForCompaction_fails_ifUncompactedBytesLessThanCutoff()
+ public void test_createCandidate_fails_ifUncompactedBytesLessThanCutoff()
{
final HumanReadableBytes minUncompactedBytes = HumanReadableBytes.valueOf(10_000);
final MostFragmentedIntervalFirstPolicy policy = new MostFragmentedIntervalFirstPolicy(
@@ -80,20 +93,20 @@ public void test_checkEligibilityForCompaction_fails_ifUncompactedBytesLessThanC
null
);
- Assertions.assertEquals(
- CompactionCandidateSearchPolicy.Eligibility.fail(
- "Uncompacted bytes[100] in interval must be at least [10,000]"
- ),
- policy.checkEligibilityForCompaction(createCandidate(1, 100L), null)
- );
- Assertions.assertEquals(
- CompactionCandidateSearchPolicy.Eligibility.OK,
- policy.checkEligibilityForCompaction(createCandidate(100, 10_000L), null)
- );
+ final CompactionStatus status1 =
+ eligibilityBuilder().compacted(DUMMY_COMPACTION_STATS).uncompacted(createStats(1, 100L)).build();
+ final CompactionCandidateSearchPolicy.Eligibility eligibility1 = policy.checkEligibilityForCompaction(new CompactionCandidateAndStatus(PROPOSED_COMPACTION, status1));
+ Assertions.assertEquals("Uncompacted bytes[100] in interval must be at least [10,000]", eligibility1.getReason());
+ Assertions.assertFalse(eligibility1.isEligible());
+
+ final CompactionStatus status2 =
+ eligibilityBuilder().compacted(DUMMY_COMPACTION_STATS).uncompacted(createStats(100, 10_000L)).build();
+ final CompactionCandidateSearchPolicy.Eligibility eligibility2 = policy.checkEligibilityForCompaction(new CompactionCandidateAndStatus(PROPOSED_COMPACTION, status2));
+ Assertions.assertTrue(eligibility2.isEligible());
}
@Test
- public void test_checkEligibilityForCompaction_fails_ifAvgSegmentSizeGreaterThanCutoff()
+ public void test_createCandidate_fails_ifAvgSegmentSizeGreaterThanCutoff()
{
final HumanReadableBytes maxAvgSegmentSize = HumanReadableBytes.valueOf(100);
final MostFragmentedIntervalFirstPolicy policy = new MostFragmentedIntervalFirstPolicy(
@@ -103,16 +116,18 @@ public void test_checkEligibilityForCompaction_fails_ifAvgSegmentSizeGreaterThan
null
);
+ final CompactionStatus status1 =
+ eligibilityBuilder().compacted(DUMMY_COMPACTION_STATS).uncompacted(createStats(1, 10_000L)).build();
+ final CompactionCandidateSearchPolicy.Eligibility eligibility1 = policy.checkEligibilityForCompaction(new CompactionCandidateAndStatus(PROPOSED_COMPACTION, status1));
Assertions.assertEquals(
- CompactionCandidateSearchPolicy.Eligibility.fail(
- "Average size[10,000] of uncompacted segments in interval must be at most [100]"
- ),
- policy.checkEligibilityForCompaction(createCandidate(1, 10_000L), null)
- );
- Assertions.assertEquals(
- CompactionCandidateSearchPolicy.Eligibility.OK,
- policy.checkEligibilityForCompaction(createCandidate(1, 100L), null)
+ "Average size[10,000] of uncompacted segments in interval must be at most [100]",
+ eligibility1.getReason()
);
+ Assertions.assertFalse(eligibility1.isEligible());
+ final CompactionStatus status2 =
+ eligibilityBuilder().compacted(DUMMY_COMPACTION_STATS).uncompacted(createStats(1, 100L)).build();
+ final CompactionCandidateSearchPolicy.Eligibility eligibility2 = policy.checkEligibilityForCompaction(new CompactionCandidateAndStatus(PROPOSED_COMPACTION, status2));
+ Assertions.assertTrue(eligibility2.isEligible());
}
@Test
@@ -125,12 +140,13 @@ public void test_policy_favorsIntervalWithMoreUncompactedSegments_ifTotalBytesIs
null
);
- final CompactionCandidate candidateA = createCandidate(1, 1000L);
- final CompactionCandidate candidateB = createCandidate(2, 500L);
-
- verifyCandidateIsEligible(candidateA, policy);
- verifyCandidateIsEligible(candidateB, policy);
+ final CompactionStatus status1 =
+ eligibilityBuilder().compacted(DUMMY_COMPACTION_STATS).uncompacted(createStats(1, 1_000L)).build();
+ final CompactionStatus status2 =
+ eligibilityBuilder().compacted(DUMMY_COMPACTION_STATS).uncompacted(createStats(2, 500L)).build();
+ final CompactionCandidateAndStatus candidateA = new CompactionCandidateAndStatus(PROPOSED_COMPACTION, status1);
+ final CompactionCandidateAndStatus candidateB = new CompactionCandidateAndStatus(PROPOSED_COMPACTION, status2);
Assertions.assertTrue(policy.compareCandidates(candidateA, candidateB) > 0);
Assertions.assertTrue(policy.compareCandidates(candidateB, candidateA) < 0);
}
@@ -145,12 +161,13 @@ public void test_policy_favorsIntervalWithMoreUncompactedSegments_ifAverageSizeI
null
);
- final CompactionCandidate candidateA = createCandidate(1, 1000L);
- final CompactionCandidate candidateB = createCandidate(2, 1000L);
-
- verifyCandidateIsEligible(candidateA, policy);
- verifyCandidateIsEligible(candidateB, policy);
+ final CompactionStatus status1 =
+ eligibilityBuilder().compacted(DUMMY_COMPACTION_STATS).uncompacted(createStats(1, 1000L)).build();
+ final CompactionStatus status2 =
+ eligibilityBuilder().compacted(DUMMY_COMPACTION_STATS).uncompacted(createStats(2, 1000L)).build();
+ final CompactionCandidateAndStatus candidateA = new CompactionCandidateAndStatus(PROPOSED_COMPACTION, status1);
+ final CompactionCandidateAndStatus candidateB = new CompactionCandidateAndStatus(PROPOSED_COMPACTION, status2);
Assertions.assertTrue(policy.compareCandidates(candidateA, candidateB) > 0);
Assertions.assertTrue(policy.compareCandidates(candidateB, candidateA) < 0);
}
@@ -165,12 +182,13 @@ public void test_policy_favorsIntervalWithSmallerSegments_ifCountIsEqual()
null
);
- final CompactionCandidate candidateA = createCandidate(10, 500L);
- final CompactionCandidate candidateB = createCandidate(10, 1000L);
-
- verifyCandidateIsEligible(candidateA, policy);
- verifyCandidateIsEligible(candidateB, policy);
+ final CompactionStatus status1 =
+ eligibilityBuilder().compacted(DUMMY_COMPACTION_STATS).uncompacted(createStats(10, 500L)).build();
+ final CompactionStatus status2 =
+ eligibilityBuilder().compacted(DUMMY_COMPACTION_STATS).uncompacted(createStats(10, 1000L)).build();
+ final CompactionCandidateAndStatus candidateA = new CompactionCandidateAndStatus(PROPOSED_COMPACTION, status1);
+ final CompactionCandidateAndStatus candidateB = new CompactionCandidateAndStatus(PROPOSED_COMPACTION, status2);
Assertions.assertTrue(policy.compareCandidates(candidateA, candidateB) < 0);
Assertions.assertTrue(policy.compareCandidates(candidateB, candidateA) > 0);
}
@@ -185,12 +203,13 @@ public void test_compareCandidates_returnsZeroIfSegmentCountAndAvgSizeScaleEquiv
null
);
- final CompactionCandidate candidateA = createCandidate(100, 25);
- final CompactionCandidate candidateB = createCandidate(400, 100);
-
- verifyCandidateIsEligible(candidateA, policy);
- verifyCandidateIsEligible(candidateB, policy);
+ final CompactionStatus status1 =
+ eligibilityBuilder().compacted(DUMMY_COMPACTION_STATS).uncompacted(createStats(100, 25)).build();
+ final CompactionStatus status2 =
+ eligibilityBuilder().compacted(DUMMY_COMPACTION_STATS).uncompacted(createStats(400, 100)).build();
+ final CompactionCandidateAndStatus candidateA = new CompactionCandidateAndStatus(PROPOSED_COMPACTION, status1);
+ final CompactionCandidateAndStatus candidateB = new CompactionCandidateAndStatus(PROPOSED_COMPACTION, status2);
Assertions.assertEquals(0, policy.compareCandidates(candidateA, candidateB));
Assertions.assertEquals(0, policy.compareCandidates(candidateB, candidateA));
}
@@ -222,30 +241,21 @@ public void test_serde_allFieldsSet() throws IOException
@Test
public void test_serde_noFieldsSet() throws IOException
{
- final MostFragmentedIntervalFirstPolicy policy = new MostFragmentedIntervalFirstPolicy(null, null, null, null);
+ final MostFragmentedIntervalFirstPolicy policy =
+ new MostFragmentedIntervalFirstPolicy(null, null, null, null);
final DefaultObjectMapper mapper = new DefaultObjectMapper();
final CompactionCandidateSearchPolicy policy2 =
mapper.readValue(mapper.writeValueAsString(policy), CompactionCandidateSearchPolicy.class);
Assertions.assertEquals(policy, policy2);
}
- private CompactionCandidate createCandidate(int numSegments, long avgSizeBytes)
+ private CompactionStatistics createStats(int numSegments, long avgSizeBytes)
{
- final CompactionStatistics dummyCompactedStats = CompactionStatistics.create(1L, 1L, 1L);
- final CompactionStatistics uncompactedStats = CompactionStatistics.create(
- avgSizeBytes * numSegments,
- numSegments,
- 1L
- );
- return CompactionCandidate.from(List.of(SEGMENT), null)
- .withCurrentStatus(CompactionStatus.pending(dummyCompactedStats, uncompactedStats, ""));
+ return CompactionStatistics.create(avgSizeBytes * numSegments, numSegments, 1L);
}
- private void verifyCandidateIsEligible(CompactionCandidate candidate, MostFragmentedIntervalFirstPolicy policy)
+ private static CompactionStatus.CompactionStatusBuilder eligibilityBuilder()
{
- Assertions.assertEquals(
- CompactionCandidateSearchPolicy.Eligibility.OK,
- policy.checkEligibilityForCompaction(candidate, null)
- );
+ return CompactionStatus.builder(CompactionStatus.State.ELIGIBLE, "approve");
}
}
diff --git a/server/src/test/java/org/apache/druid/server/compaction/NewestSegmentFirstPolicyTest.java b/server/src/test/java/org/apache/druid/server/compaction/NewestSegmentFirstPolicyTest.java
index ee2a442d2921..1fcd889c7798 100644
--- a/server/src/test/java/org/apache/druid/server/compaction/NewestSegmentFirstPolicyTest.java
+++ b/server/src/test/java/org/apache/druid/server/compaction/NewestSegmentFirstPolicyTest.java
@@ -84,7 +84,10 @@ public class NewestSegmentFirstPolicyTest
private static final int DEFAULT_NUM_SEGMENTS_PER_SHARD = 4;
private final ObjectMapper mapper = new DefaultObjectMapper();
private final NewestSegmentFirstPolicy policy = new NewestSegmentFirstPolicy(null);
- private final IndexingStateFingerprintMapper fingerprintMapper = new DefaultIndexingStateFingerprintMapper(new NoopIndexingStateCache(), mapper);
+ private final IndexingStateFingerprintMapper fingerprintMapper = new DefaultIndexingStateFingerprintMapper(
+ new NoopIndexingStateCache(),
+ mapper
+ );
@Test
public void testLargeOffsetAndSmallSegmentInterval()
@@ -200,7 +203,7 @@ public void testHugeShard()
Interval lastInterval = null;
while (iterator.hasNext()) {
- final List segments = iterator.next().getSegments();
+ final List segments = iterator.next().getCandidate().getSegments();
lastInterval = segments.get(0).getInterval();
Interval prevInterval = null;
@@ -240,7 +243,7 @@ public void testManySegmentsPerShard()
Interval lastInterval = null;
while (iterator.hasNext()) {
- final List segments = iterator.next().getSegments();
+ final List segments = iterator.next().getCandidate().getSegments();
lastInterval = segments.get(0).getInterval();
Interval prevInterval = null;
@@ -335,6 +338,7 @@ public void testClearSegmentsToCompactWhenSkippingSegments()
expectedSegmentsToCompact2.sort(Comparator.naturalOrder());
Set> observedSegments = Streams.sequentialStreamFrom(iterator)
+ .map(CompactionCandidateAndStatus::getCandidate)
.map(CompactionCandidate::getSegments)
.collect(Collectors.toSet());
Assert.assertEquals(
@@ -406,7 +410,7 @@ public void testIfSegmentsSkipOffsetWithConfiguredSegmentGranularityEqual()
Assert.assertTrue(iterator.hasNext());
Set observedSegmentsToCompact = Streams.sequentialStreamFrom(iterator)
- .flatMap(s -> s.getSegments().stream())
+ .flatMap(s -> s.getCandidate().getSegments().stream())
.collect(Collectors.toSet());
Assert.assertEquals(
ImmutableSet.copyOf(expectedSegmentsToCompact),
@@ -444,7 +448,7 @@ public void testIfSegmentsSkipOffsetWithConfiguredSegmentGranularityLarger()
);
Assert.assertTrue(iterator.hasNext());
- List actual = iterator.next().getSegments();
+ List actual = iterator.next().getCandidate().getSegments();
Assert.assertEquals(expectedSegmentsToCompact.size(), actual.size());
Assert.assertEquals(ImmutableSet.copyOf(expectedSegmentsToCompact), ImmutableSet.copyOf(actual));
Assert.assertFalse(iterator.hasNext());
@@ -480,7 +484,7 @@ public void testIfSegmentsSkipOffsetWithConfiguredSegmentGranularitySmaller()
Assert.assertTrue(iterator.hasNext());
Set observedSegmentsToCompact = Streams.sequentialStreamFrom(iterator)
- .flatMap(s -> s.getSegments().stream())
+ .flatMap(s -> s.getCandidate().getSegments().stream())
.collect(Collectors.toSet());
Assert.assertEquals(
ImmutableSet.copyOf(expectedSegmentsToCompact),
@@ -610,7 +614,7 @@ public void testIteratorReturnsSegmentsInConfiguredSegmentGranularity()
);
Assert.assertEquals(
ImmutableSet.copyOf(expectedSegmentsToCompact),
- ImmutableSet.copyOf(iterator.next().getSegments())
+ ImmutableSet.copyOf(iterator.next().getCandidate().getSegments())
);
// Month of Nov
Assert.assertTrue(iterator.hasNext());
@@ -622,7 +626,7 @@ public void testIteratorReturnsSegmentsInConfiguredSegmentGranularity()
);
Assert.assertEquals(
ImmutableSet.copyOf(expectedSegmentsToCompact),
- ImmutableSet.copyOf(iterator.next().getSegments())
+ ImmutableSet.copyOf(iterator.next().getCandidate().getSegments())
);
// Month of Oct
Assert.assertTrue(iterator.hasNext());
@@ -634,7 +638,7 @@ public void testIteratorReturnsSegmentsInConfiguredSegmentGranularity()
);
Assert.assertEquals(
ImmutableSet.copyOf(expectedSegmentsToCompact),
- ImmutableSet.copyOf(iterator.next().getSegments())
+ ImmutableSet.copyOf(iterator.next().getCandidate().getSegments())
);
// No more
Assert.assertFalse(iterator.hasNext());
@@ -661,7 +665,7 @@ public void testIteratorReturnsSegmentsInMultipleIntervalIfConfiguredSegmentGran
timeline.findNonOvershadowedObjectsInInterval(Intervals.of("2020-01-28/2020-02-15"), Partitions.ONLY_COMPLETE)
);
Assert.assertTrue(iterator.hasNext());
- List actual = iterator.next().getSegments();
+ List actual = iterator.next().getCandidate().getSegments();
Assert.assertEquals(expectedSegmentsToCompact.size(), actual.size());
Assert.assertEquals(ImmutableSet.copyOf(expectedSegmentsToCompact), ImmutableSet.copyOf(actual));
// Month of Jan
@@ -669,7 +673,7 @@ public void testIteratorReturnsSegmentsInMultipleIntervalIfConfiguredSegmentGran
timeline.findNonOvershadowedObjectsInInterval(Intervals.of("2020-01-01/2020-02-03"), Partitions.ONLY_COMPLETE)
);
Assert.assertTrue(iterator.hasNext());
- actual = iterator.next().getSegments();
+ actual = iterator.next().getCandidate().getSegments();
Assert.assertEquals(expectedSegmentsToCompact.size(), actual.size());
Assert.assertEquals(ImmutableSet.copyOf(expectedSegmentsToCompact), ImmutableSet.copyOf(actual));
// No more
@@ -697,7 +701,7 @@ public void testIteratorDoesNotReturnCompactedInterval()
Assert.assertTrue(iterator.hasNext());
Assert.assertEquals(
ImmutableSet.copyOf(expectedSegmentsToCompact),
- ImmutableSet.copyOf(iterator.next().getSegments())
+ ImmutableSet.copyOf(iterator.next().getCandidate().getSegments())
);
// Iterator should return only once since all the "minute" interval of the iterator contains the same interval
Assert.assertFalse(iterator.hasNext());
@@ -728,7 +732,7 @@ public void testIteratorReturnsAllMixedVersionSegmentsInConfiguredSegmentGranula
);
Assert.assertEquals(
ImmutableSet.copyOf(expectedSegmentsToCompact),
- ImmutableSet.copyOf(iterator.next().getSegments())
+ ImmutableSet.copyOf(iterator.next().getCandidate().getSegments())
);
// No more
Assert.assertFalse(iterator.hasNext());
@@ -841,7 +845,7 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentSeg
);
Assert.assertEquals(
ImmutableSet.copyOf(expectedSegmentsToCompact),
- ImmutableSet.copyOf(iterator.next().getSegments())
+ ImmutableSet.copyOf(iterator.next().getCandidate().getSegments())
);
// No more
Assert.assertFalse(iterator.hasNext());
@@ -890,7 +894,7 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentSeg
);
Assert.assertEquals(
ImmutableSet.copyOf(expectedSegmentsToCompact),
- ImmutableSet.copyOf(iterator.next().getSegments())
+ ImmutableSet.copyOf(iterator.next().getCandidate().getSegments())
);
// No more
Assert.assertFalse(iterator.hasNext());
@@ -940,7 +944,7 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentTim
);
Assert.assertEquals(
ImmutableSet.copyOf(expectedSegmentsToCompact),
- ImmutableSet.copyOf(iterator.next().getSegments())
+ ImmutableSet.copyOf(iterator.next().getCandidate().getSegments())
);
// No more
Assert.assertFalse(iterator.hasNext());
@@ -989,7 +993,7 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentOri
);
Assert.assertEquals(
ImmutableSet.copyOf(expectedSegmentsToCompact),
- ImmutableSet.copyOf(iterator.next().getSegments())
+ ImmutableSet.copyOf(iterator.next().getCandidate().getSegments())
);
// No more
Assert.assertFalse(iterator.hasNext());
@@ -1067,7 +1071,7 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentRol
);
Assert.assertEquals(
ImmutableSet.copyOf(expectedSegmentsToCompact),
- ImmutableSet.copyOf(iterator.next().getSegments())
+ ImmutableSet.copyOf(iterator.next().getCandidate().getSegments())
);
Assert.assertTrue(iterator.hasNext());
expectedSegmentsToCompact = new ArrayList<>(
@@ -1078,7 +1082,7 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentRol
);
Assert.assertEquals(
ImmutableSet.copyOf(expectedSegmentsToCompact),
- ImmutableSet.copyOf(iterator.next().getSegments())
+ ImmutableSet.copyOf(iterator.next().getCandidate().getSegments())
);
// No more
Assert.assertFalse(iterator.hasNext());
@@ -1162,7 +1166,7 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentQue
);
Assert.assertEquals(
ImmutableSet.copyOf(expectedSegmentsToCompact),
- ImmutableSet.copyOf(iterator.next().getSegments())
+ ImmutableSet.copyOf(iterator.next().getCandidate().getSegments())
);
Assert.assertTrue(iterator.hasNext());
expectedSegmentsToCompact = new ArrayList<>(
@@ -1173,7 +1177,7 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentQue
);
Assert.assertEquals(
ImmutableSet.copyOf(expectedSegmentsToCompact),
- ImmutableSet.copyOf(iterator.next().getSegments())
+ ImmutableSet.copyOf(iterator.next().getCandidate().getSegments())
);
// No more
Assert.assertFalse(iterator.hasNext());
@@ -1260,7 +1264,7 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentDim
);
Assert.assertEquals(
ImmutableSet.copyOf(expectedSegmentsToCompact),
- ImmutableSet.copyOf(iterator.next().getSegments())
+ ImmutableSet.copyOf(iterator.next().getCandidate().getSegments())
);
Assert.assertTrue(iterator.hasNext());
expectedSegmentsToCompact = new ArrayList<>(
@@ -1271,7 +1275,7 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentDim
);
Assert.assertEquals(
ImmutableSet.copyOf(expectedSegmentsToCompact),
- ImmutableSet.copyOf(iterator.next().getSegments())
+ ImmutableSet.copyOf(iterator.next().getCandidate().getSegments())
);
Assert.assertTrue(iterator.hasNext());
expectedSegmentsToCompact = new ArrayList<>(
@@ -1282,7 +1286,7 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentDim
);
Assert.assertEquals(
ImmutableSet.copyOf(expectedSegmentsToCompact),
- ImmutableSet.copyOf(iterator.next().getSegments())
+ ImmutableSet.copyOf(iterator.next().getCandidate().getSegments())
);
// No more
Assert.assertFalse(iterator.hasNext());
@@ -1402,7 +1406,7 @@ public void testIteratorDoesNotReturnsSegmentsWhenPartitionDimensionsPrefixed()
);
Assert.assertEquals(
ImmutableSet.copyOf(expectedSegmentsToCompact),
- ImmutableSet.copyOf(iterator.next().getSegments())
+ ImmutableSet.copyOf(iterator.next().getCandidate().getSegments())
);
// No more
Assert.assertFalse(iterator.hasNext());
@@ -1488,7 +1492,7 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentFil
);
Assert.assertEquals(
ImmutableSet.copyOf(expectedSegmentsToCompact),
- ImmutableSet.copyOf(iterator.next().getSegments())
+ ImmutableSet.copyOf(iterator.next().getCandidate().getSegments())
);
Assert.assertTrue(iterator.hasNext());
expectedSegmentsToCompact = new ArrayList<>(
@@ -1499,7 +1503,7 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentFil
);
Assert.assertEquals(
ImmutableSet.copyOf(expectedSegmentsToCompact),
- ImmutableSet.copyOf(iterator.next().getSegments())
+ ImmutableSet.copyOf(iterator.next().getCandidate().getSegments())
);
Assert.assertTrue(iterator.hasNext());
expectedSegmentsToCompact = new ArrayList<>(
@@ -1510,7 +1514,7 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentFil
);
Assert.assertEquals(
ImmutableSet.copyOf(expectedSegmentsToCompact),
- ImmutableSet.copyOf(iterator.next().getSegments())
+ ImmutableSet.copyOf(iterator.next().getCandidate().getSegments())
);
// No more
Assert.assertFalse(iterator.hasNext());
@@ -1613,7 +1617,7 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentMet
);
Assert.assertEquals(
ImmutableSet.copyOf(expectedSegmentsToCompact),
- ImmutableSet.copyOf(iterator.next().getSegments())
+ ImmutableSet.copyOf(iterator.next().getCandidate().getSegments())
);
Assert.assertTrue(iterator.hasNext());
expectedSegmentsToCompact = new ArrayList<>(
@@ -1624,7 +1628,7 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentMet
);
Assert.assertEquals(
ImmutableSet.copyOf(expectedSegmentsToCompact),
- ImmutableSet.copyOf(iterator.next().getSegments())
+ ImmutableSet.copyOf(iterator.next().getCandidate().getSegments())
);
Assert.assertTrue(iterator.hasNext());
expectedSegmentsToCompact = new ArrayList<>(
@@ -1635,7 +1639,7 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentMet
);
Assert.assertEquals(
ImmutableSet.copyOf(expectedSegmentsToCompact),
- ImmutableSet.copyOf(iterator.next().getSegments())
+ ImmutableSet.copyOf(iterator.next().getCandidate().getSegments())
);
// No more
Assert.assertFalse(iterator.hasNext());
@@ -1677,7 +1681,7 @@ public void testIteratorReturnsSegmentsSmallerSegmentGranularityCoveringMultiple
);
Assert.assertEquals(
ImmutableSet.copyOf(expectedSegmentsToCompact),
- ImmutableSet.copyOf(iterator.next().getSegments())
+ ImmutableSet.copyOf(iterator.next().getCandidate().getSegments())
);
// No more
Assert.assertFalse(iterator.hasNext());
@@ -1725,7 +1729,7 @@ public void testIteratorReturnsSegmentsAsCompactionStateChangedWithCompactedStat
);
Assert.assertEquals(
ImmutableSet.copyOf(expectedSegmentsToCompact),
- ImmutableSet.copyOf(iterator.next().getSegments())
+ ImmutableSet.copyOf(iterator.next().getCandidate().getSegments())
);
// No more
Assert.assertFalse(iterator.hasNext());
@@ -1975,7 +1979,7 @@ public void testSkipCompactionForIntervalsContainingSingleTombstone()
// Return all segments in 2023/2024 since at least one of them has data despite there being a tombstone.
Assert.assertEquals(
ImmutableList.of(tombstone2023, dataSegment2023),
- iterator.next().getSegments()
+ iterator.next().getCandidate().getSegments()
);
final DataSegment tombstone2025Jan = new DataSegment(
@@ -2027,7 +2031,7 @@ public void testSkipCompactionForIntervalsContainingSingleTombstone()
// Does not skip the tombstones in 2025 since there are multiple of them which could potentially be condensed to one
Assert.assertEquals(
ImmutableList.of(tombstone2025Jan, tombstone2025Feb, tombstone2025Mar),
- iterator.next().getSegments()
+ iterator.next().getCandidate().getSegments()
);
}
@@ -2065,12 +2069,12 @@ TestDataSource.KOALA, configBuilder().forDataSource(TestDataSource.KOALA).build(
// Verify that the segments of WIKI are preferred even though they are older
Assert.assertTrue(iterator.hasNext());
- CompactionCandidate next = iterator.next();
+ CompactionCandidate next = iterator.next().getCandidate();
Assert.assertEquals(TestDataSource.WIKI, next.getDataSource());
Assert.assertEquals(Intervals.of("2012-01-01/P1D"), next.getUmbrellaInterval());
Assert.assertTrue(iterator.hasNext());
- next = iterator.next();
+ next = iterator.next().getCandidate();
Assert.assertEquals(TestDataSource.KOALA, next.getDataSource());
Assert.assertEquals(Intervals.of("2013-01-01/P1D"), next.getUmbrellaInterval());
}
@@ -2096,7 +2100,7 @@ private static void assertCompactSegmentIntervals(
{
Interval expectedSegmentIntervalStart = to;
while (iterator.hasNext()) {
- final List segments = iterator.next().getSegments();
+ final List segments = iterator.next().getCandidate().getSegments();
final Interval firstInterval = segments.get(0).getInterval();
Assert.assertTrue(
diff --git a/server/src/test/java/org/apache/druid/server/coordinator/duty/CompactSegmentsTest.java b/server/src/test/java/org/apache/druid/server/coordinator/duty/CompactSegmentsTest.java
index 4ea98e43ad57..d528498da9ff 100644
--- a/server/src/test/java/org/apache/druid/server/coordinator/duty/CompactSegmentsTest.java
+++ b/server/src/test/java/org/apache/druid/server/coordinator/duty/CompactSegmentsTest.java
@@ -101,6 +101,7 @@
import org.apache.druid.server.coordinator.stats.Stats;
import org.apache.druid.timeline.CompactionState;
import org.apache.druid.timeline.DataSegment;
+import org.apache.druid.timeline.SegmentId;
import org.apache.druid.timeline.SegmentTimeline;
import org.apache.druid.timeline.TimelineObjectHolder;
import org.apache.druid.timeline.partition.HashBasedNumberedShardSpec;
@@ -217,7 +218,10 @@ public void setup()
final String dataSource = DATA_SOURCE_PREFIX + i;
for (int j : new int[]{0, 1, 2, 3, 7, 8}) {
for (int k = 0; k < PARTITION_PER_TIME_INTERVAL; k++) {
- List segmentForDatasource = datasourceToSegments.computeIfAbsent(dataSource, key -> new ArrayList<>());
+ List segmentForDatasource = datasourceToSegments.computeIfAbsent(
+ dataSource,
+ key -> new ArrayList<>()
+ );
DataSegment dataSegment = createSegment(dataSource, j, true, k);
allSegments.add(dataSegment);
segmentForDatasource.add(dataSegment);
@@ -250,17 +254,10 @@ private DataSegment createSegment(String dataSource, int startDay, boolean befor
startDay + 2
)
);
- return new DataSegment(
- dataSource,
- interval,
- "version",
- null,
- Collections.emptyList(),
- Collections.emptyList(),
- shardSpec,
- 0,
- 10L
- );
+ return DataSegment.builder(SegmentId.of(dataSource, interval, "version", partition))
+ .shardSpec(shardSpec)
+ .size(10L)
+ .build();
}
@Test
@@ -865,8 +862,12 @@ public void testCompactWithGranularitySpec()
.withTuningConfig(getTuningConfig(3))
.withEngine(engine)
.withGranularitySpec(
- new UserCompactionTaskGranularityConfig(Granularities.YEAR, null, null)
- )
+ new UserCompactionTaskGranularityConfig(
+ Granularities.YEAR,
+ null,
+ null
+ )
+ )
.build()
);
doCompactSegments(compactSegments, compactionConfigs);
@@ -901,10 +902,10 @@ public void testCompactWithDimensionSpec()
.withSkipOffsetFromLatest(new Period("PT0H")) // smaller than segment interval
.withTuningConfig(getTuningConfig(3))
.withDimensionsSpec(
- new UserCompactionTaskDimensionsConfig(
- DimensionsSpec.getDefaultSchemas(ImmutableList.of("bar", "foo"))
- )
- )
+ new UserCompactionTaskDimensionsConfig(
+ DimensionsSpec.getDefaultSchemas(ImmutableList.of("bar", "foo"))
+ )
+ )
.withEngine(engine)
.build()
);
@@ -968,10 +969,10 @@ public void testCompactWithProjections()
.withSkipOffsetFromLatest(new Period("PT0H")) // smaller than segment interval
.withTuningConfig(getTuningConfig(3))
.withDimensionsSpec(
- new UserCompactionTaskDimensionsConfig(
- DimensionsSpec.getDefaultSchemas(ImmutableList.of("bar", "foo"))
- )
- )
+ new UserCompactionTaskDimensionsConfig(
+ DimensionsSpec.getDefaultSchemas(ImmutableList.of("bar", "foo"))
+ )
+ )
.withProjections(projections)
.withEngine(engine)
.build()
@@ -1056,8 +1057,12 @@ public void testCompactWithRollupInGranularitySpec()
.withSkipOffsetFromLatest(new Period("PT0H")) // smaller than segment interval
.withTuningConfig(getTuningConfig(3))
.withGranularitySpec(
- new UserCompactionTaskGranularityConfig(Granularities.YEAR, null, true)
- )
+ new UserCompactionTaskGranularityConfig(
+ Granularities.YEAR,
+ null,
+ true
+ )
+ )
.withEngine(engine)
.build()
);
@@ -1146,8 +1151,12 @@ public void testCompactWithGranularitySpecConflictWithActiveCompactionTask()
.withSkipOffsetFromLatest(new Period("PT0H")) // smaller than segment interval
.withTuningConfig(getTuningConfig(3))
.withGranularitySpec(
- new UserCompactionTaskGranularityConfig(Granularities.YEAR, null, null)
- )
+ new UserCompactionTaskGranularityConfig(
+ Granularities.YEAR,
+ null,
+ null
+ )
+ )
.withEngine(engine)
.build()
);
@@ -1288,11 +1297,11 @@ public void testCompactWithTransformSpec()
.withSkipOffsetFromLatest(new Period("PT0H")) // smaller than segment interval
.withTuningConfig(getTuningConfig(3))
.withTransformSpec(
- new CompactionTransformSpec(
- new SelectorDimFilter("dim1", "foo", null),
- null
- )
- )
+ new CompactionTransformSpec(
+ new SelectorDimFilter("dim1", "foo", null),
+ null
+ )
+ )
.withEngine(engine)
.build()
);
@@ -1329,7 +1338,7 @@ public void testCompactWithoutCustomSpecs()
@Test
public void testCompactWithMetricsSpec()
{
- AggregatorFactory[] aggregatorFactories = new AggregatorFactory[] {new CountAggregatorFactory("cnt")};
+ AggregatorFactory[] aggregatorFactories = new AggregatorFactory[]{new CountAggregatorFactory("cnt")};
final OverlordClient mockClient = Mockito.mock(OverlordClient.class);
final ArgumentCaptor