From d89802ab5b1755582ac6da6c5b3456ccbd08e779 Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Mon, 22 Jul 2024 10:28:29 +0530 Subject: [PATCH 01/26] Add Compaction Scheduler --- .../NewestSegmentFirstPolicyBenchmark.java | 10 +- .../indexing/compact/CompactionScheduler.java | 42 ++ .../compact/CompactionSchedulerImpl.java | 436 ++++++++++++++++++ .../indexing/compact/NoopOverlordClient.java | 135 ++++++ .../indexing/overlord/DruidOverlord.java | 4 + .../indexing/overlord/TaskQueryTool.java | 9 +- .../http/OverlordCompactionResource.java | 95 ++++ .../batch/parallel/PartialCompactionTest.java | 3 - .../indexing/overlord/http/OverlordTest.java | 2 + .../metadata/SegmentsMetadataManager.java | 4 - .../metadata/SqlSegmentsMetadataManager.java | 4 +- .../AsyncManagementForwardingServlet.java | 20 +- .../CompactionSchedulerConfig.java | 88 ++++ .../CoordinatorCompactionConfig.java | 42 +- .../server/coordinator/DruidCoordinator.java | 42 +- .../compact/BaseSegmentSearchPolicy.java | 89 ++++ .../CompactionSegmentSearchPolicy.java | 10 +- .../coordinator/compact/CompactionStatus.java | 37 +- .../compact/CompactionStatusTracker.java | 224 +++++++++ .../DataSourceCompactibleSegmentIterator.java | 27 +- .../compact/NewestSegmentFirstPolicy.java | 41 +- ...riorityBasedCompactionSegmentIterator.java | 27 +- .../compact/SmallestSegmentFirstPolicy.java | 48 ++ .../coordinator/duty/CompactSegments.java | 48 +- .../http/CompactionConfigUpdateRequest.java | 88 ++++ .../CoordinatorCompactionConfigsResource.java | 27 +- .../AsyncManagementForwardingServletTest.java | 51 ++ .../CoordinatorCompactionConfigTest.java | 40 ++ .../coordinator/DruidCoordinatorTest.java | 39 +- .../compact/NewestSegmentFirstPolicyTest.java | 139 ++++-- .../coordinator/duty/CompactSegmentsTest.java | 61 +-- .../CoordinatorSimulationBuilder.java | 11 +- .../simulate/TestSegmentsMetadataManager.java | 6 - .../org/apache/druid/cli/CliCoordinator.java | 9 +- .../org/apache/druid/cli/CliOverlord.java | 15 + .../java/org/apache/druid/cli/CliRouter.java | 2 + 36 files changed, 1748 insertions(+), 227 deletions(-) create mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionScheduler.java create mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionSchedulerImpl.java create mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/compact/NoopOverlordClient.java create mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordCompactionResource.java create mode 100644 server/src/main/java/org/apache/druid/server/coordinator/CompactionSchedulerConfig.java create mode 100644 server/src/main/java/org/apache/druid/server/coordinator/compact/BaseSegmentSearchPolicy.java create mode 100644 server/src/main/java/org/apache/druid/server/coordinator/compact/CompactionStatusTracker.java create mode 100644 server/src/main/java/org/apache/druid/server/coordinator/compact/SmallestSegmentFirstPolicy.java create mode 100644 server/src/main/java/org/apache/druid/server/http/CompactionConfigUpdateRequest.java create mode 100644 server/src/test/java/org/apache/druid/server/coordinator/CoordinatorCompactionConfigTest.java diff --git a/benchmarks/src/test/java/org/apache/druid/server/coordinator/NewestSegmentFirstPolicyBenchmark.java b/benchmarks/src/test/java/org/apache/druid/server/coordinator/NewestSegmentFirstPolicyBenchmark.java index 98c27c4b2b8c..84bf5442fb0e 100644 --- a/benchmarks/src/test/java/org/apache/druid/server/coordinator/NewestSegmentFirstPolicyBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/server/coordinator/NewestSegmentFirstPolicyBenchmark.java @@ -26,6 +26,7 @@ import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.server.coordinator.compact.CompactionSegmentIterator; import org.apache.druid.server.coordinator.compact.CompactionSegmentSearchPolicy; +import org.apache.druid.server.coordinator.compact.CompactionStatusTracker; import org.apache.druid.server.coordinator.compact.NewestSegmentFirstPolicy; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.SegmentTimeline; @@ -61,7 +62,7 @@ public class NewestSegmentFirstPolicyBenchmark { private static final String DATA_SOURCE_PREFIX = "dataSource_"; - private final CompactionSegmentSearchPolicy policy = new NewestSegmentFirstPolicy(new DefaultObjectMapper()); + private final CompactionSegmentSearchPolicy policy = new NewestSegmentFirstPolicy(null); @Param("100") private int numDataSources; @@ -141,7 +142,12 @@ public void setup() @Benchmark public void measureNewestSegmentFirstPolicy(Blackhole blackhole) { - final CompactionSegmentIterator iterator = policy.createIterator(compactionConfigs, dataSources, Collections.emptyMap()); + final CompactionSegmentIterator iterator = policy.createIterator( + compactionConfigs, + dataSources, + Collections.emptyMap(), + new CompactionStatusTracker(new DefaultObjectMapper()) + ); for (int i = 0; i < numCompactionTaskSlots && iterator.hasNext(); i++) { blackhole.consume(iterator.next()); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionScheduler.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionScheduler.java new file mode 100644 index 000000000000..85db08aa3b34 --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionScheduler.java @@ -0,0 +1,42 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.compact; + +import org.apache.druid.server.coordinator.AutoCompactionSnapshot; + +import java.util.Map; + +/** + * Compaction Scheduler that runs on the Overlord if + * {@code druid.compaction.scheduler.enabled=true}. + */ +public interface CompactionScheduler +{ + void becomeLeader(); + + void stopBeingLeader(); + + Map getAllCompactionSnapshots(); + + AutoCompactionSnapshot getCompactionSnapshot(String dataSource); + + Long getSegmentBytesYetToBeCompacted(String dataSource); + +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionSchedulerImpl.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionSchedulerImpl.java new file mode 100644 index 000000000000..02af6727f64a --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionSchedulerImpl.java @@ -0,0 +1,436 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.compact; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Optional; +import com.google.common.util.concurrent.Futures; +import com.google.common.util.concurrent.ListenableFuture; +import com.google.inject.Inject; +import org.apache.druid.client.DataSourcesSnapshot; +import org.apache.druid.client.indexing.ClientCompactionTaskQuery; +import org.apache.druid.client.indexing.IndexingTotalWorkerCapacityInfo; +import org.apache.druid.client.indexing.TaskPayloadResponse; +import org.apache.druid.common.config.JacksonConfigManager; +import org.apache.druid.error.DruidException; +import org.apache.druid.indexer.TaskLocation; +import org.apache.druid.indexer.TaskStatus; +import org.apache.druid.indexer.TaskStatusPlus; +import org.apache.druid.indexing.common.task.CompactionTask; +import org.apache.druid.indexing.overlord.TaskMaster; +import org.apache.druid.indexing.overlord.TaskQueryTool; +import org.apache.druid.indexing.overlord.TaskQueue; +import org.apache.druid.indexing.overlord.TaskRunner; +import org.apache.druid.indexing.overlord.TaskRunnerListener; +import org.apache.druid.indexing.overlord.http.TotalWorkerCapacityResponse; +import org.apache.druid.java.util.common.CloseableIterators; +import org.apache.druid.java.util.common.Stopwatch; +import org.apache.druid.java.util.common.concurrent.Execs; +import org.apache.druid.java.util.common.concurrent.ScheduledExecutorFactory; +import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.java.util.common.parsers.CloseableIterator; +import org.apache.druid.java.util.emitter.service.ServiceEmitter; +import org.apache.druid.java.util.emitter.service.ServiceMetricEvent; +import org.apache.druid.metadata.LockFilterPolicy; +import org.apache.druid.metadata.SegmentsMetadataManager; +import org.apache.druid.server.coordinator.AutoCompactionSnapshot; +import org.apache.druid.server.coordinator.CompactionSchedulerConfig; +import org.apache.druid.server.coordinator.CoordinatorCompactionConfig; +import org.apache.druid.server.coordinator.CoordinatorOverlordServiceConfig; +import org.apache.druid.server.coordinator.compact.CompactionStatusTracker; +import org.apache.druid.server.coordinator.duty.CompactSegments; +import org.apache.druid.server.coordinator.stats.CoordinatorRunStats; +import org.apache.druid.server.coordinator.stats.CoordinatorStat; +import org.apache.druid.server.coordinator.stats.Dimension; +import org.joda.time.Duration; +import org.joda.time.Interval; + +import javax.annotation.Nullable; +import java.io.IOException; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.function.Supplier; + +/** + * TODO: pending items + * - [x] make config static. + * - [?] bind scheduler only when enabled + * - [x] route compaction status API to overlord if scheduler is enabled + * - [x] skip run on coordinator if scheduler is enabled + * - [x] task state listener + * - [x] handle success and failure inside CompactionStatusTracker + * - [x] make policy serializable + * - [ ] handle priority datasource in policy + * - [ ] add another policy - newestSegmentFirst, smallestSegmentFirst, auto + * - [x] enable segments polling if overlord is standalone + * - [ ] test on cluster - standalone, coordinator-overlord + * - [ ] unit tests + * - [ ] integration tests + */ +public class CompactionSchedulerImpl implements CompactionScheduler +{ + private static final Logger log = new Logger(CompactionSchedulerImpl.class); + + private final TaskMaster taskMaster; + private final TaskQueryTool taskQueryTool; + private final JacksonConfigManager configManager; + private final SegmentsMetadataManager segmentManager; + private final ServiceEmitter emitter; + private final ObjectMapper objectMapper; + + /** + * Single-threaded executor to process the compaction queue. + */ + private final ScheduledExecutorService executor; + + private final TaskRunnerListener taskStateListener; + private final CompactionStatusTracker statusTracker; + private final AtomicBoolean isLeader = new AtomicBoolean(false); + private final CompactSegments duty; + + /** + * The scheduler should enable/disable polling of segments only if the Overlord + * is running in standalone mode, otherwise this is handled by the DruidCoordinator + * class itself. + */ + private final boolean shouldPollSegments; + + private final Stopwatch sinceStatsEmitted = Stopwatch.createStarted(); + private final CompactionSchedulerConfig schedulerConfig; + + @Inject + public CompactionSchedulerImpl( + TaskMaster taskMaster, + TaskQueryTool taskQueryTool, + CompactionStatusTracker statusTracker, + SegmentsMetadataManager segmentManager, + JacksonConfigManager configManager, + CompactionSchedulerConfig schedulerConfig, + CoordinatorOverlordServiceConfig coordinatorOverlordServiceConfig, + ScheduledExecutorFactory executorFactory, + ServiceEmitter emitter, + ObjectMapper objectMapper + ) + { + this.taskMaster = taskMaster; + this.taskQueryTool = taskQueryTool; + this.configManager = configManager; + this.segmentManager = segmentManager; + this.statusTracker = statusTracker; + this.objectMapper = objectMapper; + this.emitter = emitter; + this.schedulerConfig = schedulerConfig; + this.executor = executorFactory.create(1, "CompactionScheduler-%s"); + this.shouldPollSegments = segmentManager != null + && !coordinatorOverlordServiceConfig.isEnabled(); + this.duty = new CompactSegments(statusTracker, new LocalOverlordClient()); + this.taskStateListener = new TaskRunnerListener() + { + @Override + public String getListenerId() + { + return "CompactionScheduler"; + } + + @Override + public void locationChanged(String taskId, TaskLocation newLocation) + { + // Do nothing + } + + @Override + public void statusChanged(String taskId, TaskStatus status) + { + runOnExecutor(() -> statusTracker.onTaskFinished(taskId, status)); + } + }; + } + + @Override + public void becomeLeader() + { + if (isEnabled() && isLeader.compareAndSet(false, true)) { + log.info("Starting compaction scheduler as we are now the leader."); + runOnExecutor(() -> { + initState(); + checkSchedulingStatus(); + }); + } + } + + @Override + public void stopBeingLeader() + { + if (isEnabled() && isLeader.compareAndSet(true, false)) { + log.info("Stopping compaction scheduler as we are not the leader anymore."); + runOnExecutor(this::cleanupState); + } + } + + private void runOnExecutor(Runnable runnable) + { + executor.submit(() -> { + try { + runnable.run(); + } + catch (Throwable t) { + log.error(t, "Error while executing runnable"); + } + }); + } + + private synchronized void initState() + { + Optional taskRunner = taskMaster.getTaskRunner(); + if (taskRunner.isPresent()) { + taskRunner.get().registerListener(taskStateListener, executor); + } else { + log.warn("No TaskRunner. Unable to register callbacks."); + } + + if (shouldPollSegments) { + segmentManager.startPollingDatabasePeriodically(); + } + } + + private synchronized void cleanupState() + { + statusTracker.reset(); + + Optional taskRunner = taskMaster.getTaskRunner(); + if (taskRunner.isPresent()) { + taskRunner.get().unregisterListener(taskStateListener.getListenerId()); + } + + if (shouldPollSegments) { + segmentManager.stopPollingDatabasePeriodically(); + } + } + + private TaskQueue getValidTaskQueue() + { + Optional taskQueue = taskMaster.getTaskQueue(); + if (taskQueue.isPresent()) { + return taskQueue.get(); + } else { + throw DruidException.defensive("No TaskQueue. Cannot proceed."); + } + } + + public boolean isEnabled() + { + return schedulerConfig.isEnabled(); + } + + private synchronized void checkSchedulingStatus() + { + if (isLeader.get() && isEnabled()) { + try { + processCompactionQueue(getLatestConfig()); + } + catch (Exception e) { + log.error(e, "Error processing compaction queue. Continuing schedule."); + } + executor.schedule(this::checkSchedulingStatus, 5, TimeUnit.SECONDS); + } else { + cleanupState(); + } + } + + private synchronized void processCompactionQueue( + CoordinatorCompactionConfig currentConfig + ) + { + DataSourcesSnapshot dataSourcesSnapshot + = segmentManager.getSnapshotOfDataSourcesWithAllUsedSegments(); + final CoordinatorRunStats stats = new CoordinatorRunStats(); + + duty.run( + currentConfig, + dataSourcesSnapshot.getUsedSegmentsTimelinesPerDataSource(), + stats + ); + + // Emit stats only every 5 minutes + if (sinceStatsEmitted.hasElapsed(Duration.standardMinutes(5))) { + stats.forEachStat( + (stat, dimensions, value) -> { + if (stat.shouldEmit()) { + emitStat(stat, dimensions.getValues(), value); + } + } + ); + sinceStatsEmitted.restart(); + } + } + + private void emitStat(CoordinatorStat stat, Map dimensionValues, long value) + { + ServiceMetricEvent.Builder eventBuilder = new ServiceMetricEvent.Builder(); + dimensionValues.forEach( + (dim, dimValue) -> eventBuilder.setDimension(dim.reportedName(), dimValue) + ); + emitter.emit(eventBuilder.setMetric(stat.getMetricName(), value)); + } + + private CoordinatorCompactionConfig getLatestConfig() + { + return configManager.watch( + CoordinatorCompactionConfig.CONFIG_KEY, + CoordinatorCompactionConfig.class, + CoordinatorCompactionConfig.empty() + ).get(); + } + + @Override + public AutoCompactionSnapshot getCompactionSnapshot(String dataSource) + { + return duty.getAutoCompactionSnapshot(dataSource); + } + + @Override + public Long getSegmentBytesYetToBeCompacted(String dataSource) + { + return duty.getTotalSizeOfSegmentsAwaitingCompaction(dataSource); + } + + @Override + public Map getAllCompactionSnapshots() + { + return duty.getAutoCompactionSnapshot(); + } + + /** + * Dummy Overlord client used by the {@link #duty} to fetch task related info. + * This client simply redirects all queries to the {@link TaskQueryTool}. + */ + private class LocalOverlordClient extends NoopOverlordClient + { + @Override + public ListenableFuture runTask(String taskId, Object clientTaskQuery) + { + return futureOf(() -> { + getValidTaskQueue().add( + convertTask(clientTaskQuery, ClientCompactionTaskQuery.class, CompactionTask.class) + ); + return null; + }); + } + + @Override + public ListenableFuture cancelTask(String taskId) + { + return futureOf(() -> { + getValidTaskQueue().shutdown(taskId, "Shutdown by Compaction Scheduler"); + return null; + }); + } + + @Override + public ListenableFuture taskPayload(String taskId) + { + ClientCompactionTaskQuery taskPayload = taskQueryTool.getTask(taskId).transform( + task -> convertTask(task, CompactionTask.class, ClientCompactionTaskQuery.class) + ).orNull(); + return futureOf( + () -> new TaskPayloadResponse(taskId, taskPayload) + ); + } + + @Override + public ListenableFuture> taskStatuses( + @Nullable String state, + @Nullable String dataSource, + @Nullable Integer maxCompletedTasks + ) + { + final ListenableFuture> tasksFuture + = futureOf(taskQueryTool::getAllActiveTasks); + return Futures.transform( + tasksFuture, + taskList -> CloseableIterators.withEmptyBaggage(taskList.iterator()), + Execs.directExecutor() + ); + } + + @Override + public ListenableFuture>> findLockedIntervals(List lockFilterPolicies) + { + return futureOf(() -> taskQueryTool.getLockedIntervals(lockFilterPolicies)); + } + + @Override + public ListenableFuture getTotalWorkerCapacity() + { + return futureOf(() -> dutyCompatible(taskQueryTool.getTotalWorkerCapacity())); + } + + private ListenableFuture futureOf(Supplier supplier) + { + try { + return Futures.immediateFuture(supplier.get()); + } + catch (Exception e) { + return Futures.immediateFailedFuture(e); + } + } + + private IndexingTotalWorkerCapacityInfo dutyCompatible(TotalWorkerCapacityResponse capacity) + { + if (capacity == null) { + return null; + } else { + return new IndexingTotalWorkerCapacityInfo( + capacity.getCurrentClusterCapacity(), + capacity.getMaximumCapacityWithAutoScale() + ); + } + } + + private V convertTask(Object taskPayload, Class inputType, Class outputType) + { + if (taskPayload == null) { + return null; + } else if (inputType.isAssignableFrom(taskPayload.getClass())) { + throw DruidException.defensive( + "Unknown type[%s] for compaction task. Expected type[%s].", + taskPayload.getClass().getSimpleName(), inputType.getSimpleName() + ); + } + + try { + return objectMapper.readValue( + objectMapper.writeValueAsBytes(taskPayload), + outputType + ); + } + catch (IOException e) { + log.warn(e, "Could not convert task[%s] to client compatible object", taskPayload); + throw DruidException.defensive( + "Could not convert task[%s] to compatible object.", + taskPayload + ); + } + } + } +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/NoopOverlordClient.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/NoopOverlordClient.java new file mode 100644 index 000000000000..38343568c105 --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/NoopOverlordClient.java @@ -0,0 +1,135 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.compact; + +import com.google.common.util.concurrent.ListenableFuture; +import org.apache.druid.client.indexing.IndexingTotalWorkerCapacityInfo; +import org.apache.druid.client.indexing.IndexingWorkerInfo; +import org.apache.druid.client.indexing.TaskPayloadResponse; +import org.apache.druid.client.indexing.TaskStatusResponse; +import org.apache.druid.indexer.TaskStatus; +import org.apache.druid.indexer.TaskStatusPlus; +import org.apache.druid.indexer.report.TaskReport; +import org.apache.druid.indexing.overlord.supervisor.SupervisorStatus; +import org.apache.druid.java.util.common.parsers.CloseableIterator; +import org.apache.druid.metadata.LockFilterPolicy; +import org.apache.druid.rpc.ServiceRetryPolicy; +import org.apache.druid.rpc.indexing.OverlordClient; +import org.joda.time.Interval; + +import javax.annotation.Nullable; +import java.net.URI; +import java.util.List; +import java.util.Map; +import java.util.Set; + +public class NoopOverlordClient implements OverlordClient +{ + @Override + public ListenableFuture findCurrentLeader() + { + throw new UnsupportedOperationException(); + } + + @Override + public ListenableFuture runTask(String taskId, Object taskObject) + { + throw new UnsupportedOperationException(); + } + + @Override + public ListenableFuture cancelTask(String taskId) + { + throw new UnsupportedOperationException(); + } + + @Override + public ListenableFuture> taskStatuses( + @Nullable String state, + @Nullable String dataSource, + @Nullable Integer maxCompletedTasks + ) + { + throw new UnsupportedOperationException(); + } + + @Override + public ListenableFuture> taskStatuses(Set taskIds) + { + throw new UnsupportedOperationException(); + } + + @Override + public ListenableFuture taskStatus(String taskId) + { + throw new UnsupportedOperationException(); + } + + @Override + public ListenableFuture taskPayload(String taskId) + { + throw new UnsupportedOperationException(); + } + + @Override + public ListenableFuture taskReportAsMap(String taskId) + { + throw new UnsupportedOperationException(); + } + + @Override + public ListenableFuture> supervisorStatuses() + { + throw new UnsupportedOperationException(); + } + + @Override + public ListenableFuture>> findLockedIntervals( + List lockFilterPolicies + ) + { + throw new UnsupportedOperationException(); + } + + @Override + public ListenableFuture killPendingSegments(String dataSource, Interval interval) + { + throw new UnsupportedOperationException(); + } + + @Override + public ListenableFuture> getWorkers() + { + throw new UnsupportedOperationException(); + } + + @Override + public ListenableFuture getTotalWorkerCapacity() + { + throw new UnsupportedOperationException(); + } + + @Override + public OverlordClient withRetryPolicy(ServiceRetryPolicy retryPolicy) + { + // Ignore retryPolicy for the test client. + return this; + } +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/DruidOverlord.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/DruidOverlord.java index f99189d35894..90ac36a96791 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/DruidOverlord.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/DruidOverlord.java @@ -29,6 +29,7 @@ import org.apache.druid.indexing.common.actions.SegmentAllocationQueue; import org.apache.druid.indexing.common.actions.TaskActionClientFactory; import org.apache.druid.indexing.common.task.TaskContextEnricher; +import org.apache.druid.indexing.compact.CompactionScheduler; import org.apache.druid.indexing.overlord.config.DefaultTaskConfig; import org.apache.druid.indexing.overlord.config.TaskLockConfig; import org.apache.druid.indexing.overlord.config.TaskQueueConfig; @@ -87,6 +88,7 @@ public DruidOverlord( final OverlordDutyExecutor overlordDutyExecutor, @IndexingService final DruidLeaderSelector overlordLeaderSelector, final SegmentAllocationQueue segmentAllocationQueue, + final CompactionScheduler compactionScheduler, final ObjectMapper mapper, final TaskContextEnricher taskContextEnricher ) @@ -140,6 +142,7 @@ public void start() { segmentAllocationQueue.becomeLeader(); taskMaster.becomeLeader(taskRunner, taskQueue); + compactionScheduler.becomeLeader(); // Announce the node only after all the services have been initialized initialized = true; @@ -150,6 +153,7 @@ public void start() public void stop() { serviceAnnouncer.unannounce(node); + compactionScheduler.stopBeingLeader(); taskMaster.stopBeingLeader(); segmentAllocationQueue.stopBeingLeader(); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskQueryTool.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskQueryTool.java index f5351d7c6e51..c092b62190e4 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskQueryTool.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskQueryTool.java @@ -150,6 +150,11 @@ public TaskInfo getTaskInfo(String taskId) return storage.getTaskInfo(taskId); } + public List getAllActiveTasks() + { + return getTaskStatusPlusList(TaskStateLookup.ALL, null, null, 0, null); + } + public List getTaskStatusPlusList( TaskStateLookup state, @Nullable String dataSource, @@ -178,7 +183,7 @@ public List getTaskStatusPlusList( // This way, we can use the snapshot from taskStorage as the source of truth for the set of tasks to process // and use the snapshot from taskRunner as a reference for potential task state updates happened // after the first snapshotting. - Stream taskStatusPlusStream = getTaskStatusPlusList( + Stream taskStatusPlusStream = getTaskStatusPlusStream( state, dataSource, createdTimeDuration, @@ -244,7 +249,7 @@ public List getTaskStatusPlusList( return taskStatuses; } - private Stream getTaskStatusPlusList( + private Stream getTaskStatusPlusStream( TaskStateLookup state, @Nullable String dataSource, Duration createdTimeDuration, diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordCompactionResource.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordCompactionResource.java new file mode 100644 index 000000000000..6676a9199d69 --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordCompactionResource.java @@ -0,0 +1,95 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.overlord.http; + +import com.google.inject.Inject; +import com.sun.jersey.spi.container.ResourceFilters; +import org.apache.druid.indexing.compact.CompactionScheduler; +import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.server.coordinator.AutoCompactionSnapshot; +import org.apache.druid.server.http.security.StateResourceFilter; + +import javax.ws.rs.GET; +import javax.ws.rs.Path; +import javax.ws.rs.Produces; +import javax.ws.rs.QueryParam; +import javax.ws.rs.core.MediaType; +import javax.ws.rs.core.Response; +import java.util.Collection; +import java.util.Collections; + +@Path("/druid/indexer/v1/compaction") +public class OverlordCompactionResource +{ + private static final Logger log = new Logger(OverlordCompactionResource.class); + + private final CompactionScheduler scheduler; + + @Inject + public OverlordCompactionResource( + CompactionScheduler scheduler + ) + { + this.scheduler = scheduler; + log.info("Creating the new overlord compaction resource."); + } + + @GET + @Path("/progress") + @Produces(MediaType.APPLICATION_JSON) + @ResourceFilters(StateResourceFilter.class) + public Response getCompactionProgress( + @QueryParam("dataSource") String dataSource + ) + { + final Long notCompactedSegmentSizeBytes = scheduler.getSegmentBytesYetToBeCompacted(dataSource); + if (notCompactedSegmentSizeBytes == null) { + return Response.status(Response.Status.NOT_FOUND) + .entity(Collections.singletonMap("error", "Unknown DataSource")) + .build(); + } else { + return Response.ok(Collections.singletonMap("remainingSegmentSize", notCompactedSegmentSizeBytes)) + .build(); + } + } + + @GET + @Path("/status") + @Produces(MediaType.APPLICATION_JSON) + @ResourceFilters(StateResourceFilter.class) + public Response getCompactionSnapshotForDataSource( + @QueryParam("dataSource") String dataSource + ) + { + final Collection snapshots; + if (dataSource == null || dataSource.isEmpty()) { + snapshots = scheduler.getAllCompactionSnapshots().values(); + } else { + AutoCompactionSnapshot autoCompactionSnapshot = scheduler.getCompactionSnapshot(dataSource); + if (autoCompactionSnapshot == null) { + return Response.status(Response.Status.NOT_FOUND) + .entity(Collections.singletonMap("error", "Unknown DataSource")) + .build(); + } + snapshots = Collections.singleton(autoCompactionSnapshot); + } + return Response.ok(Collections.singletonMap("latestStatus", snapshots)).build(); + } +} diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialCompactionTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialCompactionTest.java index 15201e884d3e..9962dfcd821c 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialCompactionTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialCompactionTest.java @@ -29,8 +29,6 @@ import org.apache.druid.indexer.partitions.PartitionsSpec; import org.apache.druid.indexer.partitions.SingleDimensionPartitionsSpec; import org.apache.druid.indexing.common.LockGranularity; -import org.apache.druid.indexing.common.RetryPolicyConfig; -import org.apache.druid.indexing.common.RetryPolicyFactory; import org.apache.druid.indexing.common.task.CompactionTask; import org.apache.druid.indexing.common.task.CompactionTask.Builder; import org.apache.druid.indexing.common.task.SpecificSegmentsSpec; @@ -69,7 +67,6 @@ public class PartialCompactionTest extends AbstractMultiPhaseParallelIndexingTes 0 ); private static final Interval INTERVAL_TO_INDEX = Intervals.of("2017-12/P1M"); - private static final RetryPolicyFactory RETRY_POLICY_FACTORY = new RetryPolicyFactory(new RetryPolicyConfig()); private File inputDir; diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/http/OverlordTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/http/OverlordTest.java index d1c2167b9290..f1a8b65a509f 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/http/OverlordTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/http/OverlordTest.java @@ -47,6 +47,7 @@ import org.apache.druid.indexing.common.task.NoopTask; import org.apache.druid.indexing.common.task.NoopTaskContextEnricher; import org.apache.druid.indexing.common.task.Task; +import org.apache.druid.indexing.compact.CompactionScheduler; import org.apache.druid.indexing.overlord.DruidOverlord; import org.apache.druid.indexing.overlord.HeapMemoryTaskStorage; import org.apache.druid.indexing.overlord.IndexerMetadataStorageAdapter; @@ -257,6 +258,7 @@ public MockTaskRunner get() EasyMock.createNiceMock(OverlordDutyExecutor.class), new TestDruidLeaderSelector(), EasyMock.createNiceMock(SegmentAllocationQueue.class), + EasyMock.createNiceMock(CompactionScheduler.class), new DefaultObjectMapper(), new NoopTaskContextEnricher() ); diff --git a/server/src/main/java/org/apache/druid/metadata/SegmentsMetadataManager.java b/server/src/main/java/org/apache/druid/metadata/SegmentsMetadataManager.java index b0aaa54d5bac..35b0d423f271 100644 --- a/server/src/main/java/org/apache/druid/metadata/SegmentsMetadataManager.java +++ b/server/src/main/java/org/apache/druid/metadata/SegmentsMetadataManager.java @@ -19,7 +19,6 @@ package org.apache.druid.metadata; -import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Optional; import org.apache.druid.client.DataSourcesSnapshot; import org.apache.druid.client.ImmutableDruidDataSource; @@ -207,9 +206,6 @@ List getUnusedSegmentIntervals( DateTime maxUsedStatusLastUpdatedTime ); - @VisibleForTesting - void poll(); - /** * Populates used_status_last_updated column in the segments table iteratively until there are no segments with a NULL * value for that column. diff --git a/server/src/main/java/org/apache/druid/metadata/SqlSegmentsMetadataManager.java b/server/src/main/java/org/apache/druid/metadata/SqlSegmentsMetadataManager.java index 2c81603e529c..6deed9c3f2b9 100644 --- a/server/src/main/java/org/apache/druid/metadata/SqlSegmentsMetadataManager.java +++ b/server/src/main/java/org/apache/druid/metadata/SqlSegmentsMetadataManager.java @@ -1008,8 +1008,8 @@ public Set retrieveAllDataSourceNames() ); } - @Override - public void poll() + @VisibleForTesting + void poll() { // See the comment to the pollLock field, explaining this synchronized block synchronized (pollLock) { diff --git a/server/src/main/java/org/apache/druid/server/AsyncManagementForwardingServlet.java b/server/src/main/java/org/apache/druid/server/AsyncManagementForwardingServlet.java index 4c8db118d37b..a85268f52f23 100644 --- a/server/src/main/java/org/apache/druid/server/AsyncManagementForwardingServlet.java +++ b/server/src/main/java/org/apache/druid/server/AsyncManagementForwardingServlet.java @@ -30,6 +30,7 @@ import org.apache.druid.guice.annotations.Json; import org.apache.druid.guice.http.DruidHttpClientConfig; import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.server.coordinator.CompactionSchedulerConfig; import org.apache.druid.server.initialization.jetty.StandardResponseHeaderFilterHolder; import org.apache.druid.server.security.AuthConfig; import org.apache.druid.server.security.AuthorizationUtils; @@ -66,6 +67,12 @@ public class AsyncManagementForwardingServlet extends AsyncProxyServlet private static final String ARBITRARY_COORDINATOR_BASE_PATH = "/proxy/coordinator"; private static final String ARBITRARY_OVERLORD_BASE_PATH = "/proxy/overlord"; + // If Compaction Scheduler is enabled, compaction status APIs must be forwarded to the Overlord + // Client Request: https://{ROUTER_HOST}:9088/druid/coordinator/v1/compaction/status + // Proxy Request: https://{OVERLORD_HOST}:8281/druid/indexer/v1/compaction/status + private static final String COMPACTION_COORDINATOR_PATH = "/druid/coordinator/v1/compaction"; + private static final String COMPACTION_OVERLORD_PATH = "/druid/indexer/v1/compaction"; + // This path is used to check if the managment proxy is enabled, it simply returns {"enabled":true} private static final String ENABLED_PATH = "/proxy/enabled"; @@ -75,6 +82,7 @@ public class AsyncManagementForwardingServlet extends AsyncProxyServlet private final DruidLeaderSelector coordLeaderSelector; private final DruidLeaderSelector overlordLeaderSelector; private final AuthorizerMapper authorizerMapper; + private final CompactionSchedulerConfig compactionSchedulerConfig; @Inject public AsyncManagementForwardingServlet( @@ -83,6 +91,7 @@ public AsyncManagementForwardingServlet( @Global DruidHttpClientConfig httpClientConfig, @Coordinator DruidLeaderSelector coordLeaderSelector, @IndexingService DruidLeaderSelector overlordLeaderSelector, + CompactionSchedulerConfig compactionSchedulerConfig, AuthorizerMapper authorizerMapper ) { @@ -91,6 +100,7 @@ public AsyncManagementForwardingServlet( this.httpClientConfig = httpClientConfig; this.coordLeaderSelector = coordLeaderSelector; this.overlordLeaderSelector = overlordLeaderSelector; + this.compactionSchedulerConfig = compactionSchedulerConfig; this.authorizerMapper = authorizerMapper; } @@ -99,7 +109,15 @@ protected void service(HttpServletRequest request, HttpServletResponse response) { String currentLeader; String requestURI = StringUtils.toLowerCase(request.getRequestURI()); - if (requestURI.startsWith(STANDARD_COORDINATOR_BASE_PATH)) { + if (compactionSchedulerConfig.isEnabled() + && requestURI.startsWith(COMPACTION_COORDINATOR_PATH)) { + // If Compaction Scheduler is enabled, compaction APIs must be forwarded to the Overlord + currentLeader = overlordLeaderSelector.getCurrentLeader(); + request.setAttribute( + MODIFIED_PATH_ATTRIBUTE, + request.getRequestURI().replace(COMPACTION_COORDINATOR_PATH, COMPACTION_OVERLORD_PATH) + ); + } else if (requestURI.startsWith(STANDARD_COORDINATOR_BASE_PATH)) { currentLeader = coordLeaderSelector.getCurrentLeader(); } else if (requestURI.startsWith(STANDARD_OVERLORD_BASE_PATH)) { currentLeader = overlordLeaderSelector.getCurrentLeader(); diff --git a/server/src/main/java/org/apache/druid/server/coordinator/CompactionSchedulerConfig.java b/server/src/main/java/org/apache/druid/server/coordinator/CompactionSchedulerConfig.java new file mode 100644 index 000000000000..345e0af6230b --- /dev/null +++ b/server/src/main/java/org/apache/druid/server/coordinator/CompactionSchedulerConfig.java @@ -0,0 +1,88 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.server.coordinator; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.druid.common.config.Configs; + +import javax.annotation.Nullable; +import java.util.Objects; + +/** + * This config must be bound on the following services: + *
    + *
  • CliOverlord - to run the compaction scheduler on the Overlord
  • + *
  • CliOverlord - to prevent the Coordinator from running auto-compaction duty
  • + *
  • CliRouter - to allow the Router to forward compaction stats requests to the Overlord
  • + *
+ */ +public class CompactionSchedulerConfig +{ + private static final CompactionSchedulerConfig DEFAULT = new CompactionSchedulerConfig(null); + + @JsonProperty + private final boolean enabled; + + public static CompactionSchedulerConfig defaultConfig() + { + return DEFAULT; + } + + @JsonCreator + public CompactionSchedulerConfig( + @JsonProperty("enabled") @Nullable Boolean enabled + ) + { + this.enabled = Configs.valueOrDefault(enabled, false); + } + + public boolean isEnabled() + { + return enabled; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + CompactionSchedulerConfig that = (CompactionSchedulerConfig) o; + return enabled == that.enabled; + } + + @Override + public int hashCode() + { + return Objects.hashCode(enabled); + } + + @Override + public String toString() + { + return "CompactionSchedulerConfig{" + + "enabled=" + enabled + + '}'; + } +} diff --git a/server/src/main/java/org/apache/druid/server/coordinator/CoordinatorCompactionConfig.java b/server/src/main/java/org/apache/druid/server/coordinator/CoordinatorCompactionConfig.java index 036c53121e91..47efe462bc1b 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/CoordinatorCompactionConfig.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/CoordinatorCompactionConfig.java @@ -24,6 +24,9 @@ import com.google.common.collect.ImmutableList; import org.apache.druid.common.config.Configs; import org.apache.druid.indexer.CompactionEngine; +import org.apache.druid.server.coordinator.compact.CompactionSegmentSearchPolicy; +import org.apache.druid.server.coordinator.compact.NewestSegmentFirstPolicy; +import org.apache.druid.server.http.CompactionConfigUpdateRequest; import javax.annotation.Nullable; import java.util.List; @@ -37,12 +40,14 @@ public class CoordinatorCompactionConfig private static final int DEFAULT_MAX_COMPACTION_TASK_SLOTS = Integer.MAX_VALUE; private static final boolean DEFAULT_USE_AUTO_SCALE_SLOTS = false; private static final CompactionEngine DEFAULT_COMPACTION_ENGINE = CompactionEngine.NATIVE; + private static final CompactionSegmentSearchPolicy DEFAULT_COMPACTION_POLICY = new NewestSegmentFirstPolicy(null); private final List compactionConfigs; private final double compactionTaskSlotRatio; private final int maxCompactionTaskSlots; private final boolean useAutoScaleSlots; private final CompactionEngine compactionEngine; + private final CompactionSegmentSearchPolicy compactionPolicy; public static CoordinatorCompactionConfig from( CoordinatorCompactionConfig baseConfig, @@ -54,34 +59,34 @@ public static CoordinatorCompactionConfig from( baseConfig.compactionTaskSlotRatio, baseConfig.maxCompactionTaskSlots, baseConfig.useAutoScaleSlots, - null + baseConfig.compactionEngine, + baseConfig.compactionPolicy ); } public static CoordinatorCompactionConfig from( CoordinatorCompactionConfig baseConfig, - @Nullable Double compactionTaskSlotRatio, - @Nullable Integer maxCompactionTaskSlots, - @Nullable Boolean useAutoScaleSlots + CompactionConfigUpdateRequest update ) { return new CoordinatorCompactionConfig( baseConfig.compactionConfigs, - compactionTaskSlotRatio == null ? baseConfig.compactionTaskSlotRatio : compactionTaskSlotRatio, - maxCompactionTaskSlots == null ? baseConfig.maxCompactionTaskSlots : maxCompactionTaskSlots, - useAutoScaleSlots == null ? baseConfig.useAutoScaleSlots : useAutoScaleSlots, - null + Configs.valueOrDefault(update.getCompactionTaskSlotRatio(), baseConfig.compactionTaskSlotRatio), + Configs.valueOrDefault(update.getMaxCompactionTaskSlots(), baseConfig.maxCompactionTaskSlots), + Configs.valueOrDefault(update.getUseAutoScaleSlots(), baseConfig.useAutoScaleSlots), + Configs.valueOrDefault(update.getCompactionEngine(), baseConfig.compactionEngine), + Configs.valueOrDefault(update.getCompactionPolicy(), baseConfig.compactionPolicy) ); } public static CoordinatorCompactionConfig from(List compactionConfigs) { - return new CoordinatorCompactionConfig(compactionConfigs, null, null, null, null); + return new CoordinatorCompactionConfig(compactionConfigs, null, null, null, null, null); } public static CoordinatorCompactionConfig empty() { - return new CoordinatorCompactionConfig(ImmutableList.of(), null, null, null, null); + return new CoordinatorCompactionConfig(ImmutableList.of(), null, null, null, null, null); } @JsonCreator @@ -90,7 +95,8 @@ public CoordinatorCompactionConfig( @JsonProperty("compactionTaskSlotRatio") @Nullable Double compactionTaskSlotRatio, @JsonProperty("maxCompactionTaskSlots") @Nullable Integer maxCompactionTaskSlots, @JsonProperty("useAutoScaleSlots") @Nullable Boolean useAutoScaleSlots, - @JsonProperty("compactionEngine") @Nullable CompactionEngine compactionEngine + @JsonProperty("compactionEngine") @Nullable CompactionEngine compactionEngine, + @JsonProperty("compactionPolicy") @Nullable CompactionSegmentSearchPolicy compactionPolicy ) { this.compactionConfigs = compactionConfigs; @@ -98,6 +104,7 @@ public CoordinatorCompactionConfig( this.maxCompactionTaskSlots = Configs.valueOrDefault(maxCompactionTaskSlots, DEFAULT_MAX_COMPACTION_TASK_SLOTS); this.useAutoScaleSlots = Configs.valueOrDefault(useAutoScaleSlots, DEFAULT_USE_AUTO_SCALE_SLOTS); this.compactionEngine = Configs.valueOrDefault(compactionEngine, DEFAULT_COMPACTION_ENGINE); + this.compactionPolicy = Configs.valueOrDefault(compactionPolicy, DEFAULT_COMPACTION_POLICY); } @JsonProperty @@ -130,6 +137,12 @@ public CompactionEngine getEngine() return compactionEngine; } + @JsonProperty + public CompactionSegmentSearchPolicy getCompactionPolicy() + { + return compactionPolicy; + } + @Override public boolean equals(Object o) { @@ -144,7 +157,8 @@ public boolean equals(Object o) maxCompactionTaskSlots == that.maxCompactionTaskSlots && useAutoScaleSlots == that.useAutoScaleSlots && compactionEngine == that.compactionEngine && - Objects.equals(compactionConfigs, that.compactionConfigs); + Objects.equals(compactionConfigs, that.compactionConfigs) && + Objects.equals(compactionPolicy, that.compactionPolicy); } @Override @@ -155,7 +169,8 @@ public int hashCode() compactionTaskSlotRatio, maxCompactionTaskSlots, useAutoScaleSlots, - compactionEngine + compactionEngine, + compactionPolicy ); } @@ -168,6 +183,7 @@ public String toString() ", maxCompactionTaskSlots=" + maxCompactionTaskSlots + ", useAutoScaleSlots=" + useAutoScaleSlots + ", compactionEngine=" + compactionEngine + + ", compactionPolicy=" + compactionPolicy + '}'; } } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java b/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java index 9710bda79b44..4264fd9bf2ab 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java @@ -55,7 +55,7 @@ import org.apache.druid.segment.metadata.CoordinatorSegmentMetadataCache; import org.apache.druid.server.DruidNode; import org.apache.druid.server.coordinator.balancer.BalancerStrategyFactory; -import org.apache.druid.server.coordinator.compact.CompactionSegmentSearchPolicy; +import org.apache.druid.server.coordinator.compact.CompactionStatusTracker; import org.apache.druid.server.coordinator.config.CoordinatorKillConfigs; import org.apache.druid.server.coordinator.config.DruidCoordinatorConfig; import org.apache.druid.server.coordinator.config.KillUnusedSegmentsConfig; @@ -157,6 +157,7 @@ public class DruidCoordinator @Nullable private final CoordinatorSegmentMetadataCache coordinatorSegmentMetadataCache; private final CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig; + private final CompactionSchedulerConfig compactionSchedulerConfig; private volatile boolean started = false; @@ -200,9 +201,10 @@ public DruidCoordinator( CoordinatorCustomDutyGroups customDutyGroups, LookupCoordinatorManager lookupCoordinatorManager, @Coordinator DruidLeaderSelector coordLeaderSelector, - CompactionSegmentSearchPolicy compactionSegmentSearchPolicy, @Nullable CoordinatorSegmentMetadataCache coordinatorSegmentMetadataCache, - CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig + CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig, + CompactionSchedulerConfig compactionSchedulerConfig, + CompactionStatusTracker compactionStatusTracker ) { this.config = config; @@ -220,10 +222,11 @@ public DruidCoordinator( this.balancerStrategyFactory = config.getBalancerStrategyFactory(); this.lookupCoordinatorManager = lookupCoordinatorManager; this.coordLeaderSelector = coordLeaderSelector; - this.compactSegments = initializeCompactSegmentsDuty(compactionSegmentSearchPolicy); + this.compactSegments = initializeCompactSegmentsDuty(compactionStatusTracker); this.loadQueueManager = loadQueueManager; this.coordinatorSegmentMetadataCache = coordinatorSegmentMetadataCache; this.centralizedDatasourceSchemaConfig = centralizedDatasourceSchemaConfig; + this.compactionSchedulerConfig = compactionSchedulerConfig; } public boolean isLeader() @@ -590,8 +593,7 @@ List makeIndexingServiceDuties() duties.add(new KillStalePendingSegments(overlordClient)); } - // CompactSegmentsDuty should be the last duty as it can take a long time to complete - // We do not have to add compactSegments if it is already enabled in the custom duty group + // Do not add compactSegments if it is already included in the custom duty groups if (getCompactSegmentsDutyFromCustomGroups().isEmpty()) { duties.add(compactSegments); } @@ -625,11 +627,11 @@ private List makeMetadataStoreManagementDuties() } @VisibleForTesting - CompactSegments initializeCompactSegmentsDuty(CompactionSegmentSearchPolicy compactionSegmentSearchPolicy) + CompactSegments initializeCompactSegmentsDuty(CompactionStatusTracker statusTracker) { List compactSegmentsDutyFromCustomGroups = getCompactSegmentsDutyFromCustomGroups(); if (compactSegmentsDutyFromCustomGroups.isEmpty()) { - return new CompactSegments(compactionSegmentSearchPolicy, overlordClient); + return new CompactSegments(statusTracker, overlordClient); } else { if (compactSegmentsDutyFromCustomGroups.size() > 1) { log.warn( @@ -735,6 +737,10 @@ public void run() && coordLeaderSelector.isLeader() && startingLeaderCounter == coordLeaderSelector.localTerm()) { + if (shouldSkipAutoCompactDuty(duty)) { + continue; + } + dutyRunTime.restart(); params = duty.run(params); dutyRunTime.stop(); @@ -781,6 +787,26 @@ public void run() } } + /** + * @return true if this is an auto-compact CompactSegments duty and should + * not be run in case Compaction Scheduler is already running on Overlord. + * Manually triggered compaction should always be run. + */ + private boolean shouldSkipAutoCompactDuty(CoordinatorDuty duty) + { + final boolean shouldSkipDuty = compactionSchedulerConfig.isEnabled() + && duty instanceof CompactSegments + && !COMPACT_SEGMENTS_DUTIES_DUTY_GROUP.equals(dutyGroupName); + if (shouldSkipDuty) { + log.warn( + "Skipping Compact Segments duty in group[%s] since compaction" + + " scheduler is already running on Overlord.", + dutyGroupName + ); + } + return shouldSkipDuty; + } + private void emitStat(CoordinatorStat stat, Map dimensionValues, long value) { ServiceMetricEvent.Builder eventBuilder = new ServiceMetricEvent.Builder() diff --git a/server/src/main/java/org/apache/druid/server/coordinator/compact/BaseSegmentSearchPolicy.java b/server/src/main/java/org/apache/druid/server/coordinator/compact/BaseSegmentSearchPolicy.java new file mode 100644 index 000000000000..0e74f4299c7e --- /dev/null +++ b/server/src/main/java/org/apache/druid/server/coordinator/compact/BaseSegmentSearchPolicy.java @@ -0,0 +1,89 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.server.coordinator.compact; + +import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.druid.server.coordinator.DataSourceCompactionConfig; +import org.apache.druid.timeline.SegmentTimeline; +import org.joda.time.Interval; + +import javax.annotation.Nullable; +import java.util.Comparator; +import java.util.List; +import java.util.Map; +import java.util.Objects; + +public abstract class BaseSegmentSearchPolicy implements CompactionSegmentSearchPolicy +{ + private final String priorityDatasource; + + protected BaseSegmentSearchPolicy( + @Nullable String priorityDatasource + ) + { + this.priorityDatasource = priorityDatasource; + } + + @Nullable + @JsonProperty + public final String getPriorityDatasource() + { + return priorityDatasource; + } + + @Override + public CompactionSegmentIterator createIterator( + Map compactionConfigs, + Map dataSources, + Map> skipIntervals, + CompactionStatusTracker statusTracker + ) + { + return new PriorityBasedCompactionSegmentIterator( + compactionConfigs, + dataSources, + skipIntervals, + getPriorityDatasource(), + getSegmentComparator(), + statusTracker + ); + } + + protected abstract Comparator getSegmentComparator(); + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + BaseSegmentSearchPolicy that = (BaseSegmentSearchPolicy) o; + return Objects.equals(priorityDatasource, that.priorityDatasource); + } + + @Override + public int hashCode() + { + return Objects.hashCode(priorityDatasource); + } +} diff --git a/server/src/main/java/org/apache/druid/server/coordinator/compact/CompactionSegmentSearchPolicy.java b/server/src/main/java/org/apache/druid/server/coordinator/compact/CompactionSegmentSearchPolicy.java index cc5f4f59d856..5d71f8f6edd1 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/compact/CompactionSegmentSearchPolicy.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/compact/CompactionSegmentSearchPolicy.java @@ -19,6 +19,8 @@ package org.apache.druid.server.coordinator.compact; +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; import org.apache.druid.server.coordinator.duty.CompactSegments; import org.apache.druid.timeline.SegmentTimeline; @@ -30,6 +32,11 @@ /** * Segment searching policy used by {@link CompactSegments}. */ +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type") +@JsonSubTypes(value = { + @JsonSubTypes.Type(name = "newestSegmentFirst", value = NewestSegmentFirstPolicy.class), + @JsonSubTypes.Type(name = "smallestSegmentFirst", value = SmallestSegmentFirstPolicy.class) +}) public interface CompactionSegmentSearchPolicy { /** @@ -38,6 +45,7 @@ public interface CompactionSegmentSearchPolicy CompactionSegmentIterator createIterator( Map compactionConfigs, Map dataSources, - Map> skipIntervals + Map> skipIntervals, + CompactionStatusTracker statusTracker ); } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/compact/CompactionStatus.java b/server/src/main/java/org/apache/druid/server/coordinator/compact/CompactionStatus.java index fa053fb8d6ad..a63aef39315d 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/compact/CompactionStatus.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/compact/CompactionStatus.java @@ -48,7 +48,12 @@ */ public class CompactionStatus { - private static final CompactionStatus COMPLETE = new CompactionStatus(true, null); + private static final CompactionStatus COMPLETE = new CompactionStatus(State.COMPLETE, null); + + private enum State + { + COMPLETE, PENDING, SKIPPED + } /** * List of checks performed to determine if compaction is already complete. @@ -68,28 +73,33 @@ public class CompactionStatus Evaluator::transformSpecFilterIsUpToDate ); - private final boolean complete; - private final String reasonToCompact; + private final State state; + private final String reason; - private CompactionStatus(boolean complete, String reason) + private CompactionStatus(State state, String reason) { - this.complete = complete; - this.reasonToCompact = reason; + this.state = state; + this.reason = reason; } public boolean isComplete() { - return complete; + return state == State.COMPLETE; } - public String getReasonToCompact() + public boolean isSkipped() { - return reasonToCompact; + return state == State.SKIPPED; + } + + public String getReason() + { + return reason; } private static CompactionStatus incomplete(String reasonFormat, Object... args) { - return new CompactionStatus(false, StringUtils.format(reasonFormat, args)); + return new CompactionStatus(State.PENDING, StringUtils.format(reasonFormat, args)); } private static CompactionStatus completeIfEqual(String field, Object configured, Object current) @@ -109,12 +119,17 @@ private static CompactionStatus configChanged(String field, Object configured, O ); } + static CompactionStatus skipped(String reasonFormat, Object... args) + { + return new CompactionStatus(State.SKIPPED, StringUtils.format(reasonFormat, args)); + } + /** * 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 not performed and the incomplete status is returned. */ - static CompactionStatus of( + static CompactionStatus compute( SegmentsToCompact candidateSegments, DataSourceCompactionConfig config, ObjectMapper objectMapper diff --git a/server/src/main/java/org/apache/druid/server/coordinator/compact/CompactionStatusTracker.java b/server/src/main/java/org/apache/druid/server/coordinator/compact/CompactionStatusTracker.java new file mode 100644 index 000000000000..fbe74ec6142d --- /dev/null +++ b/server/src/main/java/org/apache/druid/server/coordinator/compact/CompactionStatusTracker.java @@ -0,0 +1,224 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.server.coordinator.compact; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.inject.Inject; +import org.apache.druid.client.indexing.ClientCompactionTaskQuery; +import org.apache.druid.indexer.TaskStatus; +import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.server.coordinator.CoordinatorCompactionConfig; +import org.apache.druid.server.coordinator.DataSourceCompactionConfig; +import org.joda.time.Interval; + +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; + +/** + * Tracks status of both recently submitted compaction tasks and the compaction + * state of segments. Can be used to check if a set of segments is currently + * eligible for compaction. + */ +public class CompactionStatusTracker +{ + private static final Logger log = new Logger(CompactionStatusTracker.class); + + private final ObjectMapper objectMapper; + private final Map datasourceStatuses = new HashMap<>(); + private final Map submittedTaskIdToPayload = new HashMap<>(); + + @Inject + public CompactionStatusTracker( + ObjectMapper objectMapper + ) + { + this.objectMapper = objectMapper; + } + + public CompactionStatus computeCompactionStatus( + SegmentsToCompact candidate, + DataSourceCompactionConfig config + ) + { + final CompactionStatus compactionStatus = CompactionStatus.compute(candidate, config, objectMapper); + if (compactionStatus.isComplete()) { + return compactionStatus; + } + + final long inputSegmentSize = config.getInputSegmentSizeBytes(); + if (candidate.getTotalBytes() > inputSegmentSize) { + return CompactionStatus.skipped( + "Total segment size[%d] is larger than allowed inputSegmentSize[%d]", + candidate.getTotalBytes(), inputSegmentSize + ); + } + + final Interval compactionInterval = candidate.getUmbrellaInterval(); + + final IntervalStatus intervalStatus + = datasourceStatuses.getOrDefault(config.getDataSource(), DatasourceStatus.EMPTY) + .getIntervalStatuses() + .get(compactionInterval); + + if (intervalStatus == null) { + return compactionStatus; + } + + switch (intervalStatus.state) { + case TASK_SUBMITTED: + case COMPACTED: + case FAILED_ALL_RETRIES: + return CompactionStatus.skipped( + "Interval[%s] was recently submitted for compaction and has state[%s].", + compactionInterval, intervalStatus.state + ); + default: + break; + } + + return compactionStatus; + } + + public void onCompactionConfigUpdated(CoordinatorCompactionConfig compactionConfig) + { + final Set compactionEnabledDatasources = new HashSet<>(); + if (compactionConfig.getCompactionConfigs() != null) { + compactionConfig.getCompactionConfigs().forEach( + config -> compactionEnabledDatasources.add(config.getDataSource()) + ); + } + + // Clean up state for datasources where compaction has been freshly disabled + final Set allDatasources = new HashSet<>(datasourceStatuses.keySet()); + allDatasources.forEach(datasource -> { + if (!compactionEnabledDatasources.contains(datasource)) { + datasourceStatuses.remove(datasource); + } + }); + } + + public void onTaskSubmitted( + ClientCompactionTaskQuery taskPayload, + SegmentsToCompact candidateSegments + ) + { + submittedTaskIdToPayload.put(taskPayload.getId(), taskPayload); + getOrComputeDatasourceStatus(taskPayload.getDataSource()) + .handleSubmittedTask(candidateSegments); + } + + public void onTaskFinished(String taskId, TaskStatus taskStatus) + { + if (!taskStatus.isComplete()) { + return; + } + + final ClientCompactionTaskQuery taskPayload = submittedTaskIdToPayload.remove(taskId); + if (taskPayload == null) { + // Nothing to do since we don't know the corresponding datasource or interval + return; + } + + final Interval compactionInterval = taskPayload.getIoConfig().getInputSpec().getInterval(); + getOrComputeDatasourceStatus(taskPayload.getDataSource()) + .handleTaskStatus(compactionInterval, taskStatus); + } + + public void reset() + { + datasourceStatuses.clear(); + } + + private DatasourceStatus getOrComputeDatasourceStatus(String datasource) + { + return datasourceStatuses.computeIfAbsent(datasource, ds -> new DatasourceStatus()); + } + + private static class DatasourceStatus + { + static final DatasourceStatus EMPTY = new DatasourceStatus(); + + final Map intervalStatus = new HashMap<>(); + + void handleTaskStatus(Interval compactionInterval, TaskStatus taskStatus) + { + final IntervalStatus lastKnownStatus = intervalStatus.get(compactionInterval); + + if (taskStatus.isSuccess()) { + intervalStatus.put(compactionInterval, new IntervalStatus(IntervalState.COMPACTED, 10)); + } else if (lastKnownStatus == null) { + // This is the first failure + intervalStatus.put(compactionInterval, new IntervalStatus(IntervalState.FAILED, 0)); + } else if (lastKnownStatus.state == IntervalState.FAILED && ++lastKnownStatus.retryCount > 10) { + // Failure retries have been exhausted + intervalStatus.put(compactionInterval, new IntervalStatus(IntervalState.FAILED_ALL_RETRIES, 10)); + } + } + + void handleSubmittedTask(SegmentsToCompact candidateSegments) + { + getIntervalStatuses().computeIfAbsent( + candidateSegments.getUmbrellaInterval(), + i -> new IntervalStatus(IntervalState.TASK_SUBMITTED, 0) + ); + + final Set readyIntervals = new HashSet<>(); + intervalStatus.forEach((interval, status) -> { + status.turnsToSkip--; + if (status.isReady()) { + readyIntervals.add(interval); + } + }); + + readyIntervals.forEach(intervalStatus::remove); + } + + Map getIntervalStatuses() + { + return intervalStatus; + } + } + + private static class IntervalStatus + { + final IntervalState state; + int turnsToSkip; + int retryCount; + + IntervalStatus(IntervalState state, int turnsToSkip) + { + this.state = state; + this.turnsToSkip = turnsToSkip; + } + + boolean isReady() + { + return turnsToSkip <= 0 + && (state == IntervalState.COMPACTED || state == IntervalState.FAILED_ALL_RETRIES); + } + } + + private enum IntervalState + { + TASK_SUBMITTED, COMPACTED, FAILED, FAILED_ALL_RETRIES + } +} diff --git a/server/src/main/java/org/apache/druid/server/coordinator/compact/DataSourceCompactibleSegmentIterator.java b/server/src/main/java/org/apache/druid/server/coordinator/compact/DataSourceCompactibleSegmentIterator.java index c086be3112bd..e0e00a25a624 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/compact/DataSourceCompactibleSegmentIterator.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/compact/DataSourceCompactibleSegmentIterator.java @@ -19,7 +19,6 @@ package org.apache.druid.server.coordinator.compact; -import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; import com.google.common.collect.Lists; @@ -64,8 +63,8 @@ public class DataSourceCompactibleSegmentIterator implements Iterator compactedIntervals = new HashSet<>(); + private final Set queuedIntervals = new HashSet<>(); private final PriorityQueue queue; @@ -82,10 +81,10 @@ public DataSourceCompactibleSegmentIterator( SegmentTimeline timeline, List skipIntervals, Comparator segmentPriority, - ObjectMapper objectMapper + CompactionStatusTracker statusTracker ) { - this.objectMapper = objectMapper; + this.statusTracker = statusTracker; this.config = config; this.dataSource = config.getDataSource(); this.queue = new PriorityQueue<>(segmentPriority); @@ -296,7 +295,6 @@ public List next() */ private void findAndEnqueueSegmentsToCompact(CompactibleSegmentIterator compactibleSegmentIterator) { - final long inputSegmentSize = config.getInputSegmentSizeBytes(); while (compactibleSegmentIterator.hasNext()) { List segments = compactibleSegmentIterator.next(); @@ -309,29 +307,28 @@ private void findAndEnqueueSegmentsToCompact(CompactibleSegmentIterator compacti final SegmentsToCompact candidates = SegmentsToCompact.from(segments); final Interval interval = candidates.getUmbrellaInterval(); - final CompactionStatus compactionStatus = CompactionStatus.of(candidates, config, objectMapper); + final CompactionStatus compactionStatus = statusTracker.computeCompactionStatus(candidates, config); if (!compactionStatus.isComplete()) { log.debug( "Datasource[%s], interval[%s] has [%d] segments that need to be compacted because [%s].", - dataSource, interval, candidates.size(), compactionStatus.getReasonToCompact() + dataSource, interval, candidates.size(), compactionStatus.getReason() ); } if (compactionStatus.isComplete()) { compactedSegmentStats.increment(candidates.getStats()); - } else if (candidates.getTotalBytes() > inputSegmentSize) { + } else if (compactionStatus.isSkipped()) { skippedSegmentStats.increment(candidates.getStats()); log.warn( - "Skipping compaction for datasource[%s], interval[%s] as total segment size[%d]" - + " is larger than allowed inputSegmentSize[%d].", - dataSource, interval, candidates.getTotalBytes(), inputSegmentSize + "Skipping compaction for datasource[%s], interval[%s] due to reason[%s].", + dataSource, interval, compactionStatus.getReason() ); } else if (config.getGranularitySpec() != null && config.getGranularitySpec().getSegmentGranularity() != null) { - if (compactedIntervals.contains(interval)) { - // Skip these candidate segments as we have already compacted this interval + if (queuedIntervals.contains(interval)) { + // Skip these candidate segments as we have already queued this interval } else { - compactedIntervals.add(interval); + queuedIntervals.add(interval); queue.add(candidates); } } else { diff --git a/server/src/main/java/org/apache/druid/server/coordinator/compact/NewestSegmentFirstPolicy.java b/server/src/main/java/org/apache/druid/server/coordinator/compact/NewestSegmentFirstPolicy.java index bc923da4f805..55ca55eafc42 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/compact/NewestSegmentFirstPolicy.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/compact/NewestSegmentFirstPolicy.java @@ -19,44 +19,31 @@ package org.apache.druid.server.coordinator.compact; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.inject.Inject; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.curator.shaded.com.google.common.collect.Ordering; import org.apache.druid.java.util.common.guava.Comparators; -import org.apache.druid.server.coordinator.DataSourceCompactionConfig; -import org.apache.druid.timeline.SegmentTimeline; -import org.joda.time.Interval; -import java.util.List; -import java.util.Map; +import javax.annotation.Nullable; +import java.util.Comparator; /** * This policy searches segments for compaction from newest to oldest. */ -public class NewestSegmentFirstPolicy implements CompactionSegmentSearchPolicy +public class NewestSegmentFirstPolicy extends BaseSegmentSearchPolicy { - private final ObjectMapper objectMapper; - - @Inject - public NewestSegmentFirstPolicy(ObjectMapper objectMapper) + @JsonCreator + public NewestSegmentFirstPolicy( + @JsonProperty("priorityDatasource") @Nullable String priorityDatasource + ) { - this.objectMapper = objectMapper; + super(priorityDatasource); } @Override - public CompactionSegmentIterator createIterator( - Map compactionConfigs, - Map dataSources, - Map> skipIntervals - ) + protected Comparator getSegmentComparator() { - return new PriorityBasedCompactionSegmentIterator( - compactionConfigs, - dataSources, - skipIntervals, - (o1, o2) -> Comparators.intervalsByStartThenEnd() - .compare(o2.getUmbrellaInterval(), o1.getUmbrellaInterval()), - objectMapper - ); + return (o1, o2) -> Comparators.intervalsByStartThenEnd() + .compare(o2.getUmbrellaInterval(), o1.getUmbrellaInterval()); } - } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/compact/PriorityBasedCompactionSegmentIterator.java b/server/src/main/java/org/apache/druid/server/coordinator/compact/PriorityBasedCompactionSegmentIterator.java index 33aea2a0451c..bb0006aef2d0 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/compact/PriorityBasedCompactionSegmentIterator.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/compact/PriorityBasedCompactionSegmentIterator.java @@ -19,9 +19,8 @@ package org.apache.druid.server.coordinator.compact; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.base.Preconditions; import com.google.common.collect.Maps; +import com.google.common.collect.Ordering; import org.apache.druid.error.DruidException; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; @@ -29,6 +28,8 @@ import org.apache.druid.utils.CollectionUtils; import org.joda.time.Interval; +import javax.annotation.Nullable; +import java.util.Arrays; import java.util.Collections; import java.util.Comparator; import java.util.List; @@ -51,11 +52,24 @@ public PriorityBasedCompactionSegmentIterator( Map compactionConfigs, Map datasourceToTimeline, Map> skipIntervals, + @Nullable String priorityDatasource, Comparator segmentPriority, - ObjectMapper objectMapper + CompactionStatusTracker statusTracker ) { - this.queue = new PriorityQueue<>(segmentPriority); + final Comparator comparator; + if (priorityDatasource == null) { + comparator = segmentPriority; + } else { + comparator = Ordering.compound( + Arrays.asList( + Ordering.explicit(priorityDatasource).onResultOf(entry -> entry.getFirst().getDataSource()), + segmentPriority + ) + ); + } + this.queue = new PriorityQueue<>(comparator); + this.datasourceIterators = Maps.newHashMapWithExpectedSize(datasourceToTimeline.size()); compactionConfigs.forEach((datasource, config) -> { if (config == null) { @@ -74,7 +88,7 @@ public PriorityBasedCompactionSegmentIterator( timeline, skipIntervals.getOrDefault(datasource, Collections.emptyList()), segmentPriority, - objectMapper + statusTracker ) ); addNextItemForDatasourceToQueue(datasource); @@ -113,10 +127,9 @@ public SegmentsToCompact next() } final SegmentsToCompact entry = queue.poll(); - if (entry == null) { + if (entry == null || entry.isEmpty()) { throw new NoSuchElementException(); } - Preconditions.checkState(!entry.isEmpty(), "Queue entry must not be empty"); addNextItemForDatasourceToQueue(entry.getFirst().getDataSource()); return entry; diff --git a/server/src/main/java/org/apache/druid/server/coordinator/compact/SmallestSegmentFirstPolicy.java b/server/src/main/java/org/apache/druid/server/coordinator/compact/SmallestSegmentFirstPolicy.java new file mode 100644 index 000000000000..1ab0f7d8e698 --- /dev/null +++ b/server/src/main/java/org/apache/druid/server/coordinator/compact/SmallestSegmentFirstPolicy.java @@ -0,0 +1,48 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.server.coordinator.compact; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.collect.Ordering; + +import javax.annotation.Nullable; +import java.util.Comparator; + +/** + * This policy searches segments for compaction from smallest to largest. + */ +public class SmallestSegmentFirstPolicy extends BaseSegmentSearchPolicy +{ + @JsonCreator + public SmallestSegmentFirstPolicy( + @JsonProperty("priorityDatasource") @Nullable String priorityDatasource + ) + { + super(priorityDatasource); + } + + @Override + protected Comparator getSegmentComparator() + { + return Ordering.natural() + .onResultOf(entry -> entry.getTotalBytes() / entry.size()); + } +} 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 01f3bc77e9ee..cecc30672154 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 @@ -24,7 +24,6 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; import com.google.common.base.Predicate; -import com.google.inject.Inject; import org.apache.druid.client.indexing.ClientCompactionIOConfig; import org.apache.druid.client.indexing.ClientCompactionIntervalSpec; import org.apache.druid.client.indexing.ClientCompactionRunnerInfo; @@ -34,7 +33,6 @@ import org.apache.druid.client.indexing.ClientCompactionTaskQueryTuningConfig; import org.apache.druid.client.indexing.ClientCompactionTaskTransformSpec; import org.apache.druid.client.indexing.ClientMSQContext; -import org.apache.druid.client.indexing.ClientTaskQuery; import org.apache.druid.client.indexing.TaskPayloadResponse; import org.apache.druid.common.guava.FutureUtils; import org.apache.druid.common.utils.IdUtils; @@ -54,6 +52,7 @@ import org.apache.druid.server.coordinator.DruidCoordinatorRuntimeParams; import org.apache.druid.server.coordinator.compact.CompactionSegmentIterator; import org.apache.druid.server.coordinator.compact.CompactionSegmentSearchPolicy; +import org.apache.druid.server.coordinator.compact.CompactionStatusTracker; import org.apache.druid.server.coordinator.compact.SegmentsToCompact; import org.apache.druid.server.coordinator.stats.CoordinatorRunStats; import org.apache.druid.server.coordinator.stats.Dimension; @@ -90,22 +89,21 @@ public class CompactSegments implements CoordinatorCustomDuty private static final Predicate IS_COMPACTION_TASK = status -> null != status && COMPACTION_TASK_TYPE.equals(status.getType()); - private final CompactionSegmentSearchPolicy policy; + private final CompactionStatusTracker statusTracker; private final OverlordClient overlordClient; // This variable is updated by the Coordinator thread executing duties and // read by HTTP threads processing Coordinator API calls. private final AtomicReference> autoCompactionSnapshotPerDataSource = new AtomicReference<>(); - @Inject @JsonCreator public CompactSegments( - @JacksonInject CompactionSegmentSearchPolicy policy, + @JacksonInject CompactionStatusTracker statusTracker, @JacksonInject OverlordClient overlordClient ) { - this.policy = policy; this.overlordClient = overlordClient; + this.statusTracker = statusTracker; resetCompactionSnapshot(); } @@ -117,22 +115,37 @@ public OverlordClient getOverlordClient() @Override public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params) + { + statusTracker.reset(); + run( + params.getCoordinatorCompactionConfig(), + params.getUsedSegmentsTimelinesPerDataSource(), + params.getCoordinatorStats() + ); + return params; + } + + public void run( + CoordinatorCompactionConfig dynamicConfig, + Map dataSources, + CoordinatorRunStats stats + ) { LOG.info("Running CompactSegments duty"); - final CoordinatorCompactionConfig dynamicConfig = params.getCoordinatorCompactionConfig(); final int maxCompactionTaskSlots = dynamicConfig.getMaxCompactionTaskSlots(); if (maxCompactionTaskSlots <= 0) { LOG.info("Skipping compaction as maxCompactionTaskSlots is [%d].", maxCompactionTaskSlots); resetCompactionSnapshot(); - return params; + return; } + statusTracker.onCompactionConfigUpdated(dynamicConfig); List compactionConfigList = dynamicConfig.getCompactionConfigs(); if (compactionConfigList == null || compactionConfigList.isEmpty()) { LOG.info("Skipping compaction as compaction config list is empty."); resetCompactionSnapshot(); - return params; + return; } Map compactionConfigs = compactionConfigList @@ -194,9 +207,9 @@ public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params) ); // Get iterator over segments to compact and submit compaction tasks - Map dataSources = params.getUsedSegmentsTimelinesPerDataSource(); + final CompactionSegmentSearchPolicy policy = dynamicConfig.getCompactionPolicy(); final CompactionSegmentIterator iterator = - policy.createIterator(compactionConfigs, dataSources, intervalsToSkipCompaction); + policy.createIterator(compactionConfigs, dataSources, intervalsToSkipCompaction, statusTracker); final int compactionTaskCapacity = getCompactionTaskCapacity(dynamicConfig); final int availableCompactionTaskSlots @@ -211,13 +224,10 @@ public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params) dynamicConfig.getEngine() ); - final CoordinatorRunStats stats = params.getCoordinatorStats(); stats.add(Stats.Compaction.MAX_SLOTS, compactionTaskCapacity); stats.add(Stats.Compaction.AVAILABLE_SLOTS, availableCompactionTaskSlots); stats.add(Stats.Compaction.SUBMITTED_TASKS, numSubmittedCompactionTasks); updateCompactionSnapshotStats(currentRunAutoCompactionSnapshotBuilders, iterator, stats); - - return params; } private void resetCompactionSnapshot() @@ -397,6 +407,7 @@ private int submitCompactionTasks( } final String dataSourceName = entry.getFirst().getDataSource(); + LOG.info("Compacting version[%s].", entry.getFirst().getVersion()); // As these segments will be compacted, we will aggregate the statistic to the Compacted statistics currentRunAutoCompactionSnapshotBuilders @@ -505,7 +516,7 @@ private int submitCompactionTasks( } final String taskId = compactSegments( - segmentsToCompact, + entry, config.getTaskPriority(), ClientCompactionTaskQueryTuningConfig.from( config.getTuningConfig(), @@ -625,7 +636,7 @@ public Map getAutoCompactionSnapshot() } private String compactSegments( - List segments, + SegmentsToCompact entry, int compactionTaskPriority, @Nullable ClientCompactionTaskQueryTuningConfig tuningConfig, @Nullable ClientCompactionTaskGranularitySpec granularitySpec, @@ -637,6 +648,7 @@ private String compactSegments( ClientCompactionRunnerInfo compactionRunner ) { + final List segments = entry.getSegments(); Preconditions.checkArgument(!segments.isEmpty(), "Expect non-empty segments to compact"); final String dataSource = segments.get(0).getDataSource(); @@ -650,7 +662,7 @@ private String compactSegments( final String taskId = IdUtils.newTaskId(TASK_ID_PREFIX, ClientCompactionTaskQuery.TYPE, dataSource, null); final Granularity segmentGranularity = granularitySpec == null ? null : granularitySpec.getSegmentGranularity(); - final ClientTaskQuery taskPayload = new ClientCompactionTaskQuery( + final ClientCompactionTaskQuery taskPayload = new ClientCompactionTaskQuery( taskId, dataSource, new ClientCompactionIOConfig( @@ -666,6 +678,8 @@ private String compactSegments( compactionRunner ); FutureUtils.getUnchecked(overlordClient.runTask(taskId, taskPayload), true); + statusTracker.onTaskSubmitted(taskPayload, entry); + return taskId; } } diff --git a/server/src/main/java/org/apache/druid/server/http/CompactionConfigUpdateRequest.java b/server/src/main/java/org/apache/druid/server/http/CompactionConfigUpdateRequest.java new file mode 100644 index 000000000000..dc9d62a55f27 --- /dev/null +++ b/server/src/main/java/org/apache/druid/server/http/CompactionConfigUpdateRequest.java @@ -0,0 +1,88 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.server.http; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.druid.indexer.CompactionEngine; +import org.apache.druid.server.coordinator.compact.CompactionSegmentSearchPolicy; + +import javax.annotation.Nullable; + +public class CompactionConfigUpdateRequest +{ + private final Double compactionTaskSlotRatio; + private final Integer maxCompactionTaskSlots; + private final Boolean useAutoScaleSlots; + private final CompactionEngine compactionEngine; + private final CompactionSegmentSearchPolicy compactionPolicy; + + @JsonCreator + public CompactionConfigUpdateRequest( + @JsonProperty("compactionTaskSlotRatio") @Nullable Double compactionTaskSlotRatio, + @JsonProperty("maxCompactionTaskSlots") @Nullable Integer maxCompactionTaskSlots, + @JsonProperty("useAutoScaleSlots") @Nullable Boolean useAutoScaleSlots, + @JsonProperty("compactionEngine") @Nullable CompactionEngine compactionEngine, + @JsonProperty("compactionPolicy") @Nullable CompactionSegmentSearchPolicy compactionPolicy + ) + { + this.compactionTaskSlotRatio = compactionTaskSlotRatio; + this.maxCompactionTaskSlots = maxCompactionTaskSlots; + this.useAutoScaleSlots = useAutoScaleSlots; + this.compactionEngine = compactionEngine; + this.compactionPolicy = compactionPolicy; + } + + @Nullable + @JsonProperty + public Double getCompactionTaskSlotRatio() + { + return compactionTaskSlotRatio; + } + + @Nullable + @JsonProperty + public Integer getMaxCompactionTaskSlots() + { + return maxCompactionTaskSlots; + } + + @Nullable + @JsonProperty + public Boolean getUseAutoScaleSlots() + { + return useAutoScaleSlots; + } + + @Nullable + @JsonProperty + public CompactionEngine getCompactionEngine() + { + return compactionEngine; + } + + @Nullable + @JsonProperty + public CompactionSegmentSearchPolicy getCompactionPolicy() + { + return compactionPolicy; + } + +} diff --git a/server/src/main/java/org/apache/druid/server/http/CoordinatorCompactionConfigsResource.java b/server/src/main/java/org/apache/druid/server/http/CoordinatorCompactionConfigsResource.java index 0bba5cf63fa1..9549c1d51d66 100644 --- a/server/src/main/java/org/apache/druid/server/http/CoordinatorCompactionConfigsResource.java +++ b/server/src/main/java/org/apache/druid/server/http/CoordinatorCompactionConfigsResource.java @@ -91,6 +91,19 @@ public Response getCompactionConfig() return Response.ok(configManager.getCurrentCompactionConfig()).build(); } + @POST + @Path("/dynamic") + @Consumes(MediaType.APPLICATION_JSON) + public Response updateCompactionDynamicConfig( + CompactionConfigUpdateRequest updatePayload, + @Context HttpServletRequest req + ) + { + UnaryOperator operator = + current -> CoordinatorCompactionConfig.from(current, updatePayload); + return updateConfigHelper(operator, AuthorizationUtils.buildAuditInfo(req)); + } + @POST @Path("/taskslots") @Consumes(MediaType.APPLICATION_JSON) @@ -101,14 +114,16 @@ public Response setCompactionTaskLimit( @Context HttpServletRequest req ) { - UnaryOperator operator = - current -> CoordinatorCompactionConfig.from( - current, + return updateCompactionDynamicConfig( + new CompactionConfigUpdateRequest( compactionTaskSlotRatio, maxCompactionTaskSlots, - useAutoScaleSlots - ); - return updateConfigHelper(operator, AuthorizationUtils.buildAuditInfo(req)); + useAutoScaleSlots, + null, + null + ), + req + ); } @POST diff --git a/server/src/test/java/org/apache/druid/server/AsyncManagementForwardingServletTest.java b/server/src/test/java/org/apache/druid/server/AsyncManagementForwardingServletTest.java index ba7c78b99b0a..5ca6fcc9dccb 100644 --- a/server/src/test/java/org/apache/druid/server/AsyncManagementForwardingServletTest.java +++ b/server/src/test/java/org/apache/druid/server/AsyncManagementForwardingServletTest.java @@ -35,6 +35,7 @@ import org.apache.druid.guice.http.DruidHttpClientConfig; import org.apache.druid.initialization.Initialization; import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.server.coordinator.CompactionSchedulerConfig; import org.apache.druid.server.initialization.BaseJettyTest; import org.apache.druid.server.initialization.ServerConfig; import org.apache.druid.server.initialization.jetty.JettyServerInitUtils; @@ -76,6 +77,7 @@ public class AsyncManagementForwardingServletTest extends BaseJettyTest private static int coordinatorPort; private static int overlordPort; private static boolean isValidLeader; + private static boolean isCompactionSchedulerEnabled; private Server coordinator; private Server overlord; @@ -115,6 +117,7 @@ public void setup() throws Exception coordinator.start(); overlord.start(); isValidLeader = true; + isCompactionSchedulerEnabled = false; } @After @@ -258,6 +261,47 @@ public void testCoordinatorProxySegments() throws Exception Assert.assertFalse("overlord called", OVERLORD_EXPECTED_REQUEST.called); } + @Test + public void testCoordinatorCompactionStatus() throws Exception + { + isCompactionSchedulerEnabled = false; + + COORDINATOR_EXPECTED_REQUEST.path = "/druid/coordinator/v1/compaction/status"; + COORDINATOR_EXPECTED_REQUEST.method = "GET"; + COORDINATOR_EXPECTED_REQUEST.headers = ImmutableMap.of("Authorization", "Basic bXl1c2VyOm15cGFzc3dvcmQ="); + + HttpURLConnection connection = ((HttpURLConnection) + new URL(StringUtils.format("http://localhost:%d/druid/coordinator/v1/compaction/status", port)) + .openConnection()); + connection.setRequestMethod(COORDINATOR_EXPECTED_REQUEST.method); + + COORDINATOR_EXPECTED_REQUEST.headers.forEach(connection::setRequestProperty); + + Assert.assertEquals(200, connection.getResponseCode()); + Assert.assertTrue("coordinator called", COORDINATOR_EXPECTED_REQUEST.called); + Assert.assertFalse("overlord called", OVERLORD_EXPECTED_REQUEST.called); + } + + @Test + public void testOverlordCompactionStatus() throws Exception + { + isCompactionSchedulerEnabled = true; + + OVERLORD_EXPECTED_REQUEST.path = "/druid/indexer/v1/compaction/status"; + OVERLORD_EXPECTED_REQUEST.method = "GET"; + OVERLORD_EXPECTED_REQUEST.headers = ImmutableMap.of("Authorization", "Basic bXl1c2VyOm15cGFzc3dvcmQ="); + + HttpURLConnection connection = ((HttpURLConnection) + new URL(StringUtils.format("http://localhost:%d/druid/coordinator/v1/compaction/status", port)) + .openConnection()); + connection.setRequestMethod("GET"); + OVERLORD_EXPECTED_REQUEST.headers.forEach(connection::setRequestProperty); + + Assert.assertEquals(200, connection.getResponseCode()); + Assert.assertFalse("coordinator called", COORDINATOR_EXPECTED_REQUEST.called); + Assert.assertTrue("overlord called", OVERLORD_EXPECTED_REQUEST.called); + } + @Test public void testOverlordPostTask() throws Exception { @@ -496,6 +540,13 @@ public String getCurrentLeader() injector.getInstance(DruidHttpClientConfig.class), coordinatorLeaderSelector, overlordLeaderSelector, + new CompactionSchedulerConfig(false) { + @Override + public boolean isEnabled() + { + return isCompactionSchedulerEnabled; + } + }, new AuthorizerMapper(ImmutableMap.of("allowAll", new AllowAllAuthorizer())) ) ); diff --git a/server/src/test/java/org/apache/druid/server/coordinator/CoordinatorCompactionConfigTest.java b/server/src/test/java/org/apache/druid/server/coordinator/CoordinatorCompactionConfigTest.java new file mode 100644 index 000000000000..6f39f5527916 --- /dev/null +++ b/server/src/test/java/org/apache/druid/server/coordinator/CoordinatorCompactionConfigTest.java @@ -0,0 +1,40 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.server.coordinator; + +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.jackson.DefaultObjectMapper; +import org.junit.Assert; +import org.junit.Test; + +public class CoordinatorCompactionConfigTest +{ + private static final ObjectMapper MAPPER = new DefaultObjectMapper(); + + @Test + public void testSerdeDefaultConfig() throws Exception + { + final CoordinatorCompactionConfig defaultConfig = CoordinatorCompactionConfig.empty(); + final String json = MAPPER.writeValueAsString(defaultConfig); + + CoordinatorCompactionConfig deserialized = MAPPER.readValue(json, CoordinatorCompactionConfig.class); + Assert.assertEquals(defaultConfig, deserialized); + } +} diff --git a/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorTest.java b/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorTest.java index 8c7e0ae14e5f..6111ba2f9d72 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorTest.java @@ -52,7 +52,7 @@ import org.apache.druid.server.DruidNode; import org.apache.druid.server.coordination.ServerType; import org.apache.druid.server.coordinator.balancer.CostBalancerStrategyFactory; -import org.apache.druid.server.coordinator.compact.NewestSegmentFirstPolicy; +import org.apache.druid.server.coordinator.compact.CompactionStatusTracker; import org.apache.druid.server.coordinator.config.CoordinatorKillConfigs; import org.apache.druid.server.coordinator.config.CoordinatorPeriodConfig; import org.apache.druid.server.coordinator.config.CoordinatorRunConfig; @@ -113,7 +113,7 @@ public class DruidCoordinatorTest extends CuratorTestBase private DruidCoordinatorConfig druidCoordinatorConfig; private ObjectMapper objectMapper; private DruidNode druidNode; - private NewestSegmentFirstPolicy newestSegmentFirstPolicy; + private CompactionStatusTracker statusTracker; private final LatchableServiceEmitter serviceEmitter = new LatchableServiceEmitter(); @Before @@ -147,7 +147,7 @@ public void setUp() throws Exception curator.blockUntilConnected(); curator.create().creatingParentsIfNeeded().forPath(LOADPATH); objectMapper = new DefaultObjectMapper(); - newestSegmentFirstPolicy = new NewestSegmentFirstPolicy(objectMapper); + statusTracker = new CompactionStatusTracker(objectMapper); druidCoordinatorConfig = new DruidCoordinatorConfig( new CoordinatorRunConfig(new Duration(COORDINATOR_START_DELAY), new Duration(COORDINATOR_PERIOD)), new CoordinatorPeriodConfig(null, null), @@ -190,8 +190,9 @@ public void setUp() throws Exception EasyMock.createNiceMock(LookupCoordinatorManager.class), new TestDruidLeaderSelector(), null, - null, - CentralizedDatasourceSchemaConfig.create() + CentralizedDatasourceSchemaConfig.create(), + CompactionSchedulerConfig.defaultConfig(), + statusTracker ); } @@ -621,8 +622,9 @@ public void testCompactSegmentsDutyWhenCustomDutyGroupEmpty() EasyMock.createNiceMock(LookupCoordinatorManager.class), new TestDruidLeaderSelector(), null, - null, - CentralizedDatasourceSchemaConfig.create() + CentralizedDatasourceSchemaConfig.create(), + CompactionSchedulerConfig.defaultConfig(), + statusTracker ); // Since CompactSegments is not enabled in Custom Duty Group, then CompactSegments must be created in IndexingServiceDuties List indexingDuties = coordinator.makeIndexingServiceDuties(); @@ -633,7 +635,7 @@ public void testCompactSegmentsDutyWhenCustomDutyGroupEmpty() Assert.assertTrue(compactSegmentsDutyFromCustomGroups.isEmpty()); // CompactSegments returned by this method should be created using the DruidCoordinatorConfig in the DruidCoordinator - CompactSegments duty = coordinator.initializeCompactSegmentsDuty(newestSegmentFirstPolicy); + CompactSegments duty = coordinator.initializeCompactSegmentsDuty(statusTracker); Assert.assertNotNull(duty); } @@ -661,8 +663,9 @@ public void testInitializeCompactSegmentsDutyWhenCustomDutyGroupDoesNotContainsC EasyMock.createNiceMock(LookupCoordinatorManager.class), new TestDruidLeaderSelector(), null, - null, - CentralizedDatasourceSchemaConfig.create() + CentralizedDatasourceSchemaConfig.create(), + CompactionSchedulerConfig.defaultConfig(), + statusTracker ); // Since CompactSegments is not enabled in Custom Duty Group, then CompactSegments must be created in IndexingServiceDuties List indexingDuties = coordinator.makeIndexingServiceDuties(); @@ -673,7 +676,7 @@ public void testInitializeCompactSegmentsDutyWhenCustomDutyGroupDoesNotContainsC Assert.assertTrue(compactSegmentsDutyFromCustomGroups.isEmpty()); // CompactSegments returned by this method should be created using the DruidCoordinatorConfig in the DruidCoordinator - CompactSegments duty = coordinator.initializeCompactSegmentsDuty(newestSegmentFirstPolicy); + CompactSegments duty = coordinator.initializeCompactSegmentsDuty(statusTracker); Assert.assertNotNull(duty); } @@ -683,7 +686,7 @@ public void testInitializeCompactSegmentsDutyWhenCustomDutyGroupContainsCompactS CoordinatorCustomDutyGroup compactSegmentCustomGroup = new CoordinatorCustomDutyGroup( "group1", Duration.standardSeconds(1), - ImmutableList.of(new CompactSegments(null, null)) + ImmutableList.of(new CompactSegments(statusTracker, null)) ); CoordinatorCustomDutyGroups customDutyGroups = new CoordinatorCustomDutyGroups(ImmutableSet.of(compactSegmentCustomGroup)); coordinator = new DruidCoordinator( @@ -701,8 +704,9 @@ public void testInitializeCompactSegmentsDutyWhenCustomDutyGroupContainsCompactS EasyMock.createNiceMock(LookupCoordinatorManager.class), new TestDruidLeaderSelector(), null, - null, - CentralizedDatasourceSchemaConfig.create() + CentralizedDatasourceSchemaConfig.create(), + CompactionSchedulerConfig.defaultConfig(), + statusTracker ); // Since CompactSegments is enabled in Custom Duty Group, then CompactSegments must not be created in IndexingServiceDuties List indexingDuties = coordinator.makeIndexingServiceDuties(); @@ -715,7 +719,7 @@ public void testInitializeCompactSegmentsDutyWhenCustomDutyGroupContainsCompactS Assert.assertNotNull(compactSegmentsDutyFromCustomGroups.get(0)); // CompactSegments returned by this method should be from the Custom Duty Group - CompactSegments duty = coordinator.initializeCompactSegmentsDuty(newestSegmentFirstPolicy); + CompactSegments duty = coordinator.initializeCompactSegmentsDuty(statusTracker); Assert.assertNotNull(duty); } @@ -806,8 +810,9 @@ public void testCoordinatorCustomDutyGroupsRunAsExpected() throws Exception EasyMock.createNiceMock(LookupCoordinatorManager.class), new TestDruidLeaderSelector(), null, - null, - CentralizedDatasourceSchemaConfig.create() + CentralizedDatasourceSchemaConfig.create(), + CompactionSchedulerConfig.defaultConfig(), + statusTracker ); coordinator.start(); diff --git a/server/src/test/java/org/apache/druid/server/coordinator/compact/NewestSegmentFirstPolicyTest.java b/server/src/test/java/org/apache/druid/server/coordinator/compact/NewestSegmentFirstPolicyTest.java index 8f24a4ebb7e4..5d5d10cec67d 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/compact/NewestSegmentFirstPolicyTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/compact/NewestSegmentFirstPolicyTest.java @@ -63,6 +63,7 @@ import org.joda.time.Interval; import org.joda.time.Period; import org.junit.Assert; +import org.junit.Before; import org.junit.Test; import java.util.ArrayList; @@ -82,7 +83,14 @@ public class NewestSegmentFirstPolicyTest private static final long DEFAULT_SEGMENT_SIZE = 1000; private static final int DEFAULT_NUM_SEGMENTS_PER_SHARD = 4; private final ObjectMapper mapper = new DefaultObjectMapper(); - private final NewestSegmentFirstPolicy policy = new NewestSegmentFirstPolicy(mapper); + private final NewestSegmentFirstPolicy policy = new NewestSegmentFirstPolicy(null); + private CompactionStatusTracker statusTracker; + + @Before + public void setup() + { + statusTracker = new CompactionStatusTracker(mapper); + } @Test public void testLargeOffsetAndSmallSegmentInterval() @@ -97,7 +105,8 @@ public void testLargeOffsetAndSmallSegmentInterval() new SegmentGenerateSpec(Intervals.of("2017-11-14T00:00:00/2017-11-16T07:00:00"), segmentPeriod) ) ), - Collections.emptyMap() + Collections.emptyMap(), + statusTracker ); assertCompactSegmentIntervals( @@ -122,7 +131,8 @@ public void testSmallOffsetAndLargeSegmentInterval() new SegmentGenerateSpec(Intervals.of("2017-11-14T00:00:00/2017-11-16T07:00:00"), segmentPeriod) ) ), - Collections.emptyMap() + Collections.emptyMap(), + statusTracker ); assertCompactSegmentIntervals( @@ -156,7 +166,8 @@ public void testLargeGapInData() new SegmentGenerateSpec(Intervals.of("2017-11-14T00:00:00/2017-11-15T07:00:00"), segmentPeriod) ) ), - Collections.emptyMap() + Collections.emptyMap(), + statusTracker ); assertCompactSegmentIntervals( @@ -204,7 +215,8 @@ public void testHugeShard() ) ) ), - Collections.emptyMap() + Collections.emptyMap(), + statusTracker ); Interval lastInterval = null; @@ -260,7 +272,8 @@ public void testManySegmentsPerShard() ) ) ), - Collections.emptyMap() + Collections.emptyMap(), + statusTracker ); Interval lastInterval = null; @@ -301,7 +314,8 @@ public void testSkipUnknownDataSource() new SegmentGenerateSpec(Intervals.of("2017-11-14T00:00:00/2017-11-16T07:00:00"), segmentPeriod) ) ), - Collections.emptyMap() + Collections.emptyMap(), + statusTracker ); assertCompactSegmentIntervals( @@ -340,7 +354,8 @@ public void testClearSegmentsToCompactWhenSkippingSegments() final CompactionSegmentIterator iterator = policy.createIterator( ImmutableMap.of(DATA_SOURCE, createCompactionConfig(inputSegmentSizeBytes, new Period("P0D"), null)), ImmutableMap.of(DATA_SOURCE, timeline), - Collections.emptyMap() + Collections.emptyMap(), + statusTracker ); final List expectedSegmentsToCompact = new ArrayList<>( @@ -377,7 +392,8 @@ public void testIfFirstSegmentIsInSkipOffset() final CompactionSegmentIterator iterator = policy.createIterator( ImmutableMap.of(DATA_SOURCE, createCompactionConfig(40000, new Period("P1D"), null)), ImmutableMap.of(DATA_SOURCE, timeline), - Collections.emptyMap() + Collections.emptyMap(), + statusTracker ); Assert.assertFalse(iterator.hasNext()); @@ -398,7 +414,8 @@ public void testIfFirstSegmentOverlapsSkipOffset() final CompactionSegmentIterator iterator = policy.createIterator( ImmutableMap.of(DATA_SOURCE, createCompactionConfig(40000, new Period("P1D"), null)), ImmutableMap.of(DATA_SOURCE, timeline), - Collections.emptyMap() + Collections.emptyMap(), + statusTracker ); Assert.assertFalse(iterator.hasNext()); @@ -415,7 +432,8 @@ public void testIfSegmentsSkipOffsetWithConfiguredSegmentGranularityEqual() final CompactionSegmentIterator iterator = policy.createIterator( ImmutableMap.of(DATA_SOURCE, createCompactionConfig(40000, new Period("P1D"), new UserCompactionTaskGranularityConfig(Granularities.DAY, null, null))), ImmutableMap.of(DATA_SOURCE, timeline), - Collections.emptyMap() + Collections.emptyMap(), + statusTracker ); // We should only get segments in Oct @@ -448,7 +466,8 @@ public void testIfSegmentsSkipOffsetWithConfiguredSegmentGranularityLarger() final CompactionSegmentIterator iterator = policy.createIterator( ImmutableMap.of(DATA_SOURCE, createCompactionConfig(40000, new Period("P1D"), new UserCompactionTaskGranularityConfig(Granularities.MONTH, null, null))), ImmutableMap.of(DATA_SOURCE, timeline), - Collections.emptyMap() + Collections.emptyMap(), + statusTracker ); // We should only get segments in Oct @@ -474,7 +493,8 @@ public void testIfSegmentsSkipOffsetWithConfiguredSegmentGranularitySmaller() final CompactionSegmentIterator iterator = policy.createIterator( ImmutableMap.of(DATA_SOURCE, createCompactionConfig(40000, new Period("P1D"), new UserCompactionTaskGranularityConfig(Granularities.MINUTE, null, null))), ImmutableMap.of(DATA_SOURCE, timeline), - Collections.emptyMap() + Collections.emptyMap(), + statusTracker ); // We should only get segments in Oct @@ -512,7 +532,8 @@ public void testWithSkipIntervals() Intervals.of("2017-11-15T00:00:00/2017-11-15T20:00:00"), Intervals.of("2017-11-13T00:00:00/2017-11-14T01:00:00") ) - ) + ), + statusTracker ); assertCompactSegmentIntervals( @@ -550,7 +571,8 @@ public void testHoleInSearchInterval() Intervals.of("2017-11-16T04:00:00/2017-11-16T10:00:00"), Intervals.of("2017-11-16T14:00:00/2017-11-16T20:00:00") ) - ) + ), + statusTracker ); assertCompactSegmentIntervals( @@ -589,7 +611,8 @@ public void testIteratorReturnsSegmentsInConfiguredSegmentGranularity() final CompactionSegmentIterator iterator = policy.createIterator( ImmutableMap.of(DATA_SOURCE, createCompactionConfig(130000, new Period("P0D"), new UserCompactionTaskGranularityConfig(Granularities.MONTH, null, null))), ImmutableMap.of(DATA_SOURCE, timeline), - Collections.emptyMap() + Collections.emptyMap(), + statusTracker ); // We should get all segments in timeline back since skip offset is P0D. @@ -638,7 +661,8 @@ public void testIteratorReturnsSegmentsInMultipleIntervalIfConfiguredSegmentGran final CompactionSegmentIterator iterator = policy.createIterator( ImmutableMap.of(DATA_SOURCE, createCompactionConfig(130000, new Period("P0D"), new UserCompactionTaskGranularityConfig(Granularities.MONTH, null, null))), ImmutableMap.of(DATA_SOURCE, timeline), - Collections.emptyMap() + Collections.emptyMap(), + statusTracker ); // We should get the segment of "2020-01-28/2020-02-03" back twice when the iterator returns for Jan and when the // iterator returns for Feb. @@ -673,7 +697,8 @@ public void testIteratorDoesNotReturnCompactedInterval() final CompactionSegmentIterator iterator = policy.createIterator( ImmutableMap.of(DATA_SOURCE, createCompactionConfig(40000, new Period("P0D"), new UserCompactionTaskGranularityConfig(Granularities.MINUTE, null, null))), ImmutableMap.of(DATA_SOURCE, timeline), - Collections.emptyMap() + Collections.emptyMap(), + statusTracker ); final List expectedSegmentsToCompact = new ArrayList<>( @@ -699,7 +724,8 @@ public void testIteratorReturnsAllMixedVersionSegmentsInConfiguredSegmentGranula final CompactionSegmentIterator iterator = policy.createIterator( ImmutableMap.of(DATA_SOURCE, createCompactionConfig(130000, new Period("P0D"), new UserCompactionTaskGranularityConfig(Granularities.MONTH, null, null))), ImmutableMap.of(DATA_SOURCE, timeline), - Collections.emptyMap() + Collections.emptyMap(), + statusTracker ); // We should get all segments in timeline back since skip offset is P0D. @@ -743,7 +769,8 @@ public void testIteratorReturnsNothingAsSegmentsWasCompactedAndHaveSameSegmentGr final CompactionSegmentIterator iterator = policy.createIterator( ImmutableMap.of(DATA_SOURCE, createCompactionConfig(130000, new Period("P0D"), new UserCompactionTaskGranularityConfig(Granularities.DAY, null, null))), ImmutableMap.of(DATA_SOURCE, timeline), - Collections.emptyMap() + Collections.emptyMap(), + statusTracker ); Assert.assertFalse(iterator.hasNext()); } @@ -776,7 +803,8 @@ public void testIteratorReturnsNothingAsSegmentsWasCompactedAndHaveSameSegmentGr final CompactionSegmentIterator iterator = policy.createIterator( ImmutableMap.of(DATA_SOURCE, createCompactionConfig(130000, new Period("P0D"), new UserCompactionTaskGranularityConfig(Granularities.DAY, null, null))), ImmutableMap.of(DATA_SOURCE, timeline), - Collections.emptyMap() + Collections.emptyMap(), + statusTracker ); Assert.assertFalse(iterator.hasNext()); } @@ -809,7 +837,8 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentSeg final CompactionSegmentIterator iterator = policy.createIterator( ImmutableMap.of(DATA_SOURCE, createCompactionConfig(130000, new Period("P0D"), new UserCompactionTaskGranularityConfig(Granularities.YEAR, null, null))), ImmutableMap.of(DATA_SOURCE, timeline), - Collections.emptyMap() + Collections.emptyMap(), + statusTracker ); // We should get all segments in timeline back since skip offset is P0D. Assert.assertTrue(iterator.hasNext()); @@ -852,7 +881,8 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentSeg final CompactionSegmentIterator iterator = policy.createIterator( ImmutableMap.of(DATA_SOURCE, createCompactionConfig(130000, new Period("P0D"), new UserCompactionTaskGranularityConfig(Granularities.YEAR, null, null))), ImmutableMap.of(DATA_SOURCE, timeline), - Collections.emptyMap() + Collections.emptyMap(), + statusTracker ); // We should get all segments in timeline back since skip offset is P0D. Assert.assertTrue(iterator.hasNext()); @@ -904,7 +934,8 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentTim ) ), ImmutableMap.of(DATA_SOURCE, timeline), - Collections.emptyMap() + Collections.emptyMap(), + statusTracker ); // We should get all segments in timeline back since skip offset is P0D. Assert.assertTrue(iterator.hasNext()); @@ -955,7 +986,8 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentOri ) ), ImmutableMap.of(DATA_SOURCE, timeline), - Collections.emptyMap() + Collections.emptyMap(), + statusTracker ); // We should get all segments in timeline back since skip offset is P0D. Assert.assertTrue(iterator.hasNext()); @@ -1007,7 +1039,8 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentRol final CompactionSegmentIterator iterator = policy.createIterator( ImmutableMap.of(DATA_SOURCE, createCompactionConfig(130000, new Period("P0D"), new UserCompactionTaskGranularityConfig(null, null, true))), ImmutableMap.of(DATA_SOURCE, timeline), - Collections.emptyMap() + Collections.emptyMap(), + statusTracker ); // We should get interval 2017-10-01T00:00:00/2017-10-02T00:00:00 and interval 2017-10-03T00:00:00/2017-10-04T00:00:00. Assert.assertTrue(iterator.hasNext()); @@ -1067,7 +1100,8 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentQue final CompactionSegmentIterator iterator = policy.createIterator( ImmutableMap.of(DATA_SOURCE, createCompactionConfig(130000, new Period("P0D"), new UserCompactionTaskGranularityConfig(null, Granularities.MINUTE, null))), ImmutableMap.of(DATA_SOURCE, timeline), - Collections.emptyMap() + Collections.emptyMap(), + statusTracker ); // We should get interval 2017-10-01T00:00:00/2017-10-02T00:00:00 and interval 2017-10-03T00:00:00/2017-10-04T00:00:00. Assert.assertTrue(iterator.hasNext()); @@ -1141,7 +1175,8 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentDim null )), ImmutableMap.of(DATA_SOURCE, timeline), - Collections.emptyMap() + Collections.emptyMap(), + statusTracker ); // We should get interval 2017-10-01T00:00:00/2017-10-02T00:00:00, interval 2017-10-04T00:00:00/2017-10-05T00:00:00, and interval 2017-10-03T00:00:00/2017-10-04T00:00:00. Assert.assertTrue(iterator.hasNext()); @@ -1182,7 +1217,8 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentDim null )), ImmutableMap.of(DATA_SOURCE, timeline), - Collections.emptyMap() + Collections.emptyMap(), + statusTracker ); // No more Assert.assertFalse(iterator.hasNext()); @@ -1261,7 +1297,8 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentFil null )), ImmutableMap.of(DATA_SOURCE, timeline), - Collections.emptyMap() + Collections.emptyMap(), + statusTracker ); // We should get interval 2017-10-01T00:00:00/2017-10-02T00:00:00, interval 2017-10-04T00:00:00/2017-10-05T00:00:00, and interval 2017-10-03T00:00:00/2017-10-04T00:00:00. Assert.assertTrue(iterator.hasNext()); @@ -1302,7 +1339,8 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentFil null )), ImmutableMap.of(DATA_SOURCE, timeline), - Collections.emptyMap() + Collections.emptyMap(), + statusTracker ); // No more Assert.assertFalse(iterator.hasNext()); @@ -1385,7 +1423,8 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentMet new AggregatorFactory[] {new CountAggregatorFactory("cnt"), new LongSumAggregatorFactory("val", "val")} )), ImmutableMap.of(DATA_SOURCE, timeline), - Collections.emptyMap() + Collections.emptyMap(), + statusTracker ); // We should get interval 2017-10-01T00:00:00/2017-10-02T00:00:00, interval 2017-10-04T00:00:00/2017-10-05T00:00:00, and interval 2017-10-03T00:00:00/2017-10-04T00:00:00. Assert.assertTrue(iterator.hasNext()); @@ -1426,7 +1465,8 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentMet null )), ImmutableMap.of(DATA_SOURCE, timeline), - Collections.emptyMap() + Collections.emptyMap(), + statusTracker ); // No more Assert.assertFalse(iterator.hasNext()); @@ -1443,7 +1483,8 @@ public void testIteratorReturnsSegmentsSmallerSegmentGranularityCoveringMultiple final CompactionSegmentIterator iterator = policy.createIterator( ImmutableMap.of(DATA_SOURCE, createCompactionConfig(130000, new Period("P0D"), new UserCompactionTaskGranularityConfig(Granularities.HOUR, null, null))), ImmutableMap.of(DATA_SOURCE, timeline), - Collections.emptyMap() + Collections.emptyMap(), + statusTracker ); // We should get all segments in timeline back since skip offset is P0D. @@ -1498,7 +1539,8 @@ public void testIteratorReturnsSegmentsAsCompactionStateChangedWithCompactedStat ) ), ImmutableMap.of(DATA_SOURCE, timeline), - Collections.emptyMap() + Collections.emptyMap(), + statusTracker ); // We should get all segments in timeline back since indexSpec changed Assert.assertTrue(iterator.hasNext()); @@ -1565,7 +1607,8 @@ public void testIteratorDoesNotReturnSegmentWithChangingAppendableIndexSpec() null )), ImmutableMap.of(DATA_SOURCE, timeline), - Collections.emptyMap() + Collections.emptyMap(), + statusTracker ); Assert.assertFalse(iterator.hasNext()); @@ -1600,7 +1643,8 @@ public void testIteratorDoesNotReturnSegmentWithChangingAppendableIndexSpec() null )), ImmutableMap.of(DATA_SOURCE, timeline), - Collections.emptyMap() + Collections.emptyMap(), + statusTracker ); Assert.assertFalse(iterator.hasNext()); } @@ -1631,7 +1675,8 @@ public void testSkipAllGranularityToDefault() ) ) ), - Collections.emptyMap() + Collections.emptyMap(), + statusTracker ); Assert.assertFalse(iterator.hasNext()); @@ -1663,7 +1708,8 @@ public void testSkipFirstHalfEternityToDefault() ) ) ), - Collections.emptyMap() + Collections.emptyMap(), + statusTracker ); Assert.assertFalse(iterator.hasNext()); @@ -1695,7 +1741,8 @@ public void testSkipSecondHalfOfEternityToDefault() ) ) ), - Collections.emptyMap() + Collections.emptyMap(), + statusTracker ); Assert.assertFalse(iterator.hasNext()); @@ -1727,7 +1774,8 @@ public void testSkipAllToAllGranularity() ) ) ), - Collections.emptyMap() + Collections.emptyMap(), + statusTracker ); Assert.assertFalse(iterator.hasNext()); @@ -1759,7 +1807,8 @@ public void testSkipAllToFinerGranularity() ) ) ), - Collections.emptyMap() + Collections.emptyMap(), + statusTracker ); Assert.assertFalse(iterator.hasNext()); @@ -1810,7 +1859,8 @@ public void testSkipCompactionForIntervalsContainingSingleTombstone() DATA_SOURCE, SegmentTimeline.forSegments(ImmutableSet.of(tombstone2023, dataSegment2023, tombstone2024)) ), - Collections.emptyMap() + Collections.emptyMap(), + statusTracker ); // Skips 2024/2025 since it has a single tombstone and no data. @@ -1868,7 +1918,8 @@ public void testSkipCompactionForIntervalsContainingSingleTombstone() tombstone2025Mar )) ), - Collections.emptyMap() + Collections.emptyMap(), + statusTracker ); // Does not skip the tombstones in 2025 since there are multiple of them which could potentially be condensed to one Assert.assertEquals( 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 236cfaf7da54..e937a8eb25dc 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 @@ -77,8 +77,7 @@ import org.apache.druid.server.coordinator.UserCompactionTaskIOConfig; import org.apache.druid.server.coordinator.UserCompactionTaskQueryTuningConfig; import org.apache.druid.server.coordinator.UserCompactionTaskTransformConfig; -import org.apache.druid.server.coordinator.compact.CompactionSegmentSearchPolicy; -import org.apache.druid.server.coordinator.compact.NewestSegmentFirstPolicy; +import org.apache.druid.server.coordinator.compact.CompactionStatusTracker; import org.apache.druid.server.coordinator.config.DruidCoordinatorConfig; import org.apache.druid.server.coordinator.stats.CoordinatorRunStats; import org.apache.druid.server.coordinator.stats.Stats; @@ -130,7 +129,6 @@ public class CompactSegmentsTest private static final int TOTAL_SEGMENT_PER_DATASOURCE = 44; private static final int TOTAL_INTERVAL_PER_DATASOURCE = 11; private static final int MAXIMUM_CAPACITY_WITH_AUTO_SCALE = 10; - private static final NewestSegmentFirstPolicy SEARCH_POLICY = new NewestSegmentFirstPolicy(JSON_MAPPER); @Parameterized.Parameters(name = "scenario: {0}, engine: {2}") public static Collection constructorFeeder() @@ -178,6 +176,7 @@ public static Collection constructorFeeder() private final CompactionEngine engine; private DataSourcesSnapshot dataSources; + private CompactionStatusTracker statusTracker; Map> datasourceToSegments = new HashMap<>(); public CompactSegmentsTest( @@ -210,6 +209,7 @@ public void setup() } } dataSources = DataSourcesSnapshot.fromUsedSegments(allSegments, ImmutableMap.of()); + statusTracker = new CompactionStatusTracker(JSON_MAPPER); } private DataSegment createSegment(String dataSource, int startDay, boolean beforeNoon, int partition) @@ -252,10 +252,10 @@ public void testSerde() throws Exception new InjectableValues.Std() .addValue(DruidCoordinatorConfig.class, COORDINATOR_CONFIG) .addValue(OverlordClient.class, overlordClient) - .addValue(CompactionSegmentSearchPolicy.class, SEARCH_POLICY) + .addValue(CompactionStatusTracker.class, statusTracker) ); - final CompactSegments compactSegments = new CompactSegments(SEARCH_POLICY, overlordClient); + final CompactSegments compactSegments = new CompactSegments(statusTracker, overlordClient); String compactSegmentString = JSON_MAPPER.writeValueAsString(compactSegments); CompactSegments serdeCompactSegments = JSON_MAPPER.readValue(compactSegmentString, CompactSegments.class); @@ -267,7 +267,7 @@ public void testSerde() throws Exception public void testRun() { final TestOverlordClient overlordClient = new TestOverlordClient(JSON_MAPPER); - final CompactSegments compactSegments = new CompactSegments(SEARCH_POLICY, overlordClient); + final CompactSegments compactSegments = new CompactSegments(statusTracker, overlordClient); final Supplier expectedVersionSupplier = new Supplier() { @@ -343,7 +343,7 @@ public String get() public void testMakeStats() { final TestOverlordClient overlordClient = new TestOverlordClient(JSON_MAPPER); - final CompactSegments compactSegments = new CompactSegments(SEARCH_POLICY, overlordClient); + final CompactSegments compactSegments = new CompactSegments(statusTracker, overlordClient); // Before any compaction, we do not have any snapshot of compactions Map autoCompactionSnapshots = compactSegments.getAutoCompactionSnapshot(); @@ -437,7 +437,7 @@ public void testMakeStatsForDataSourceWithCompactedIntervalBetweenNonCompactedIn dataSources = DataSourcesSnapshot.fromUsedSegments(segments, ImmutableMap.of()); final TestOverlordClient overlordClient = new TestOverlordClient(JSON_MAPPER); - final CompactSegments compactSegments = new CompactSegments(SEARCH_POLICY, overlordClient); + final CompactSegments compactSegments = new CompactSegments(statusTracker, overlordClient); // Before any compaction, we do not have any snapshot of compactions Map autoCompactionSnapshots = compactSegments.getAutoCompactionSnapshot(); @@ -499,7 +499,7 @@ public void testMakeStatsForDataSourceWithCompactedIntervalBetweenNonCompactedIn public void testMakeStatsWithDeactivatedDatasource() { final TestOverlordClient overlordClient = new TestOverlordClient(JSON_MAPPER); - final CompactSegments compactSegments = new CompactSegments(SEARCH_POLICY, overlordClient); + final CompactSegments compactSegments = new CompactSegments(statusTracker, overlordClient); // Before any compaction, we do not have any snapshot of compactions Map autoCompactionSnapshots = compactSegments.getAutoCompactionSnapshot(); @@ -591,7 +591,7 @@ public void testMakeStatsForDataSourceWithSkipped() dataSources = DataSourcesSnapshot.fromUsedSegments(segments, ImmutableMap.of()); final TestOverlordClient overlordClient = new TestOverlordClient(JSON_MAPPER); - final CompactSegments compactSegments = new CompactSegments(SEARCH_POLICY, overlordClient); + final CompactSegments compactSegments = new CompactSegments(statusTracker, overlordClient); // Before any compaction, we do not have any snapshot of compactions Map autoCompactionSnapshots = compactSegments.getAutoCompactionSnapshot(); @@ -650,7 +650,7 @@ public void testMakeStatsForDataSourceWithSkipped() public void testRunMultipleCompactionTaskSlots() { final TestOverlordClient overlordClient = new TestOverlordClient(JSON_MAPPER); - final CompactSegments compactSegments = new CompactSegments(SEARCH_POLICY, overlordClient); + final CompactSegments compactSegments = new CompactSegments(statusTracker, overlordClient); final CoordinatorRunStats stats = doCompactSegments(compactSegments, 3); Assert.assertEquals(3, stats.get(Stats.Compaction.AVAILABLE_SLOTS)); @@ -670,7 +670,7 @@ public void testRunMultipleCompactionTaskSlotsWithUseAutoScaleSlotsOverMaxSlot() int maxCompactionSlot = 3; Assert.assertTrue(maxCompactionSlot < MAXIMUM_CAPACITY_WITH_AUTO_SCALE); final TestOverlordClient overlordClient = new TestOverlordClient(JSON_MAPPER); - final CompactSegments compactSegments = new CompactSegments(SEARCH_POLICY, overlordClient); + final CompactSegments compactSegments = new CompactSegments(statusTracker, overlordClient); final CoordinatorRunStats stats = doCompactSegments(compactSegments, createCompactionConfigs(), maxCompactionSlot, true); Assert.assertEquals(maxCompactionSlot, stats.get(Stats.Compaction.AVAILABLE_SLOTS)); @@ -690,7 +690,7 @@ public void testRunMultipleCompactionTaskSlotsWithUseAutoScaleSlotsUnderMaxSlot( int maxCompactionSlot = 100; Assert.assertFalse(maxCompactionSlot < MAXIMUM_CAPACITY_WITH_AUTO_SCALE); final TestOverlordClient overlordClient = new TestOverlordClient(JSON_MAPPER); - final CompactSegments compactSegments = new CompactSegments(SEARCH_POLICY, overlordClient); + final CompactSegments compactSegments = new CompactSegments(statusTracker, overlordClient); final CoordinatorRunStats stats = doCompactSegments(compactSegments, createCompactionConfigs(), maxCompactionSlot, true); Assert.assertEquals(MAXIMUM_CAPACITY_WITH_AUTO_SCALE, stats.get(Stats.Compaction.AVAILABLE_SLOTS)); @@ -712,7 +712,7 @@ public void testCompactWithoutGranularitySpec() { final OverlordClient mockClient = Mockito.mock(OverlordClient.class); final ArgumentCaptor payloadCaptor = setUpMockClient(mockClient); - final CompactSegments compactSegments = new CompactSegments(SEARCH_POLICY, mockClient); + final CompactSegments compactSegments = new CompactSegments(statusTracker, mockClient); final List compactionConfigs = new ArrayList<>(); final String dataSource = DATA_SOURCE_PREFIX + 0; compactionConfigs.add( @@ -770,7 +770,7 @@ public void testCompactWithNotNullIOConfig() { final OverlordClient mockClient = Mockito.mock(OverlordClient.class); final ArgumentCaptor payloadCaptor = setUpMockClient(mockClient); - final CompactSegments compactSegments = new CompactSegments(SEARCH_POLICY, mockClient); + final CompactSegments compactSegments = new CompactSegments(statusTracker, mockClient); final List compactionConfigs = new ArrayList<>(); final String dataSource = DATA_SOURCE_PREFIX + 0; compactionConfigs.add( @@ -820,7 +820,7 @@ public void testCompactWithNullIOConfig() { final OverlordClient mockClient = Mockito.mock(OverlordClient.class); final ArgumentCaptor payloadCaptor = setUpMockClient(mockClient); - final CompactSegments compactSegments = new CompactSegments(SEARCH_POLICY, mockClient); + final CompactSegments compactSegments = new CompactSegments(statusTracker, mockClient); final List compactionConfigs = new ArrayList<>(); final String dataSource = DATA_SOURCE_PREFIX + 0; compactionConfigs.add( @@ -870,7 +870,7 @@ public void testCompactWithGranularitySpec() { final OverlordClient mockClient = Mockito.mock(OverlordClient.class); final ArgumentCaptor payloadCaptor = setUpMockClient(mockClient); - final CompactSegments compactSegments = new CompactSegments(SEARCH_POLICY, mockClient); + final CompactSegments compactSegments = new CompactSegments(statusTracker, mockClient); final List compactionConfigs = new ArrayList<>(); final String dataSource = DATA_SOURCE_PREFIX + 0; compactionConfigs.add( @@ -931,7 +931,7 @@ public void testCompactWithDimensionSpec() { final OverlordClient mockClient = Mockito.mock(OverlordClient.class); final ArgumentCaptor payloadCaptor = setUpMockClient(mockClient); - final CompactSegments compactSegments = new CompactSegments(SEARCH_POLICY, mockClient); + final CompactSegments compactSegments = new CompactSegments(statusTracker, mockClient); final List compactionConfigs = new ArrayList<>(); final String dataSource = DATA_SOURCE_PREFIX + 0; compactionConfigs.add( @@ -984,7 +984,7 @@ public void testCompactWithoutDimensionSpec() { final OverlordClient mockClient = Mockito.mock(OverlordClient.class); final ArgumentCaptor payloadCaptor = setUpMockClient(mockClient); - final CompactSegments compactSegments = new CompactSegments(SEARCH_POLICY, mockClient); + final CompactSegments compactSegments = new CompactSegments(statusTracker, mockClient); final List compactionConfigs = new ArrayList<>(); final String dataSource = DATA_SOURCE_PREFIX + 0; compactionConfigs.add( @@ -1034,7 +1034,7 @@ public void testCompactWithRollupInGranularitySpec() { final OverlordClient mockClient = Mockito.mock(OverlordClient.class); final ArgumentCaptor payloadCaptor = setUpMockClient(mockClient); - final CompactSegments compactSegments = new CompactSegments(SEARCH_POLICY, mockClient); + final CompactSegments compactSegments = new CompactSegments(statusTracker, mockClient); final List compactionConfigs = new ArrayList<>(); final String dataSource = DATA_SOURCE_PREFIX + 0; compactionConfigs.add( @@ -1146,7 +1146,7 @@ public void testCompactWithGranularitySpecConflictWithActiveCompactionTask() Mockito.when(mockClient.taskPayload(ArgumentMatchers.eq(conflictTaskId))) .thenReturn(Futures.immediateFuture(runningConflictCompactionTaskPayload)); - final CompactSegments compactSegments = new CompactSegments(SEARCH_POLICY, mockClient); + final CompactSegments compactSegments = new CompactSegments(statusTracker, mockClient); final List compactionConfigs = new ArrayList<>(); compactionConfigs.add( new DataSourceCompactionConfig( @@ -1209,7 +1209,7 @@ public void testCompactWithGranularitySpecConflictWithActiveCompactionTask() public void testRunParallelCompactionMultipleCompactionTaskSlots() { final TestOverlordClient overlordClient = new TestOverlordClient(JSON_MAPPER); - final CompactSegments compactSegments = new CompactSegments(SEARCH_POLICY, overlordClient); + final CompactSegments compactSegments = new CompactSegments(statusTracker, overlordClient); final CoordinatorRunStats stats; // Native uses maxNumConcurrentSubTasks for task slots whereas MSQ uses maxNumTasks. if (engine == CompactionEngine.NATIVE) { @@ -1246,7 +1246,7 @@ public void testRunWithLockedIntervals() // Verify that locked intervals are skipped and only one compaction task // is submitted for dataSource_0 - CompactSegments compactSegments = new CompactSegments(SEARCH_POLICY, overlordClient); + CompactSegments compactSegments = new CompactSegments(statusTracker, overlordClient); final CoordinatorRunStats stats = doCompactSegments(compactSegments, createcompactionConfigsForNative(2), 4); Assert.assertEquals(1, stats.get(Stats.Compaction.SUBMITTED_TASKS)); @@ -1266,7 +1266,7 @@ public void testCompactWithTransformSpec() NullHandling.initializeForTests(); final OverlordClient mockClient = Mockito.mock(OverlordClient.class); final ArgumentCaptor payloadCaptor = setUpMockClient(mockClient); - final CompactSegments compactSegments = new CompactSegments(SEARCH_POLICY, mockClient); + final CompactSegments compactSegments = new CompactSegments(statusTracker, mockClient); final List compactionConfigs = new ArrayList<>(); final String dataSource = DATA_SOURCE_PREFIX + 0; compactionConfigs.add( @@ -1317,7 +1317,7 @@ public void testCompactWithoutCustomSpecs() { final OverlordClient mockClient = Mockito.mock(OverlordClient.class); final ArgumentCaptor payloadCaptor = setUpMockClient(mockClient); - final CompactSegments compactSegments = new CompactSegments(SEARCH_POLICY, mockClient); + final CompactSegments compactSegments = new CompactSegments(statusTracker, mockClient); final List compactionConfigs = new ArrayList<>(); final String dataSource = DATA_SOURCE_PREFIX + 0; compactionConfigs.add( @@ -1370,7 +1370,7 @@ public void testCompactWithMetricsSpec() AggregatorFactory[] aggregatorFactories = new AggregatorFactory[] {new CountAggregatorFactory("cnt")}; final OverlordClient mockClient = Mockito.mock(OverlordClient.class); final ArgumentCaptor payloadCaptor = setUpMockClient(mockClient); - final CompactSegments compactSegments = new CompactSegments(SEARCH_POLICY, mockClient); + final CompactSegments compactSegments = new CompactSegments(statusTracker, mockClient); final List compactionConfigs = new ArrayList<>(); final String dataSource = DATA_SOURCE_PREFIX + 0; compactionConfigs.add( @@ -1452,7 +1452,7 @@ public void testDetermineSegmentGranularityFromSegmentsToCompact() final OverlordClient mockClient = Mockito.mock(OverlordClient.class); final ArgumentCaptor payloadCaptor = setUpMockClient(mockClient); - final CompactSegments compactSegments = new CompactSegments(SEARCH_POLICY, mockClient); + final CompactSegments compactSegments = new CompactSegments(statusTracker, mockClient); final List compactionConfigs = new ArrayList<>(); compactionConfigs.add( new DataSourceCompactionConfig( @@ -1539,7 +1539,7 @@ public void testDetermineSegmentGranularityFromSegmentGranularityInCompactionCon final OverlordClient mockClient = Mockito.mock(OverlordClient.class); final ArgumentCaptor payloadCaptor = setUpMockClient(mockClient); - final CompactSegments compactSegments = new CompactSegments(SEARCH_POLICY, mockClient); + final CompactSegments compactSegments = new CompactSegments(statusTracker, mockClient); final List compactionConfigs = new ArrayList<>(); compactionConfigs.add( new DataSourceCompactionConfig( @@ -1596,7 +1596,7 @@ public void testCompactWithMetricsSpecShouldSetPreserveExistingMetricsTrue() { final OverlordClient mockClient = Mockito.mock(OverlordClient.class); final ArgumentCaptor payloadCaptor = setUpMockClient(mockClient); - final CompactSegments compactSegments = new CompactSegments(SEARCH_POLICY, mockClient); + final CompactSegments compactSegments = new CompactSegments(statusTracker, mockClient); final List compactionConfigs = new ArrayList<>(); final String dataSource = DATA_SOURCE_PREFIX + 0; compactionConfigs.add( @@ -1649,7 +1649,7 @@ public void testCompactWithoutMetricsSpecShouldSetPreserveExistingMetricsFalse() { final OverlordClient mockClient = Mockito.mock(OverlordClient.class); final ArgumentCaptor payloadCaptor = setUpMockClient(mockClient); - final CompactSegments compactSegments = new CompactSegments(SEARCH_POLICY, mockClient); + final CompactSegments compactSegments = new CompactSegments(statusTracker, mockClient); final List compactionConfigs = new ArrayList<>(); final String dataSource = DATA_SOURCE_PREFIX + 0; compactionConfigs.add( @@ -1839,6 +1839,7 @@ private CoordinatorRunStats doCompactSegments( numCompactionTaskSlots == null ? null : 1.0, // 100% when numCompactionTaskSlots is not null numCompactionTaskSlots, useAutoScaleSlots, + null, null ) ) diff --git a/server/src/test/java/org/apache/druid/server/coordinator/simulate/CoordinatorSimulationBuilder.java b/server/src/test/java/org/apache/druid/server/coordinator/simulate/CoordinatorSimulationBuilder.java index 7b6eb280a398..d252d6e6f7f9 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/simulate/CoordinatorSimulationBuilder.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/simulate/CoordinatorSimulationBuilder.java @@ -37,6 +37,7 @@ import org.apache.druid.java.util.metrics.MetricsVerifier; import org.apache.druid.java.util.metrics.StubServiceEmitter; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; +import org.apache.druid.server.coordinator.CompactionSchedulerConfig; import org.apache.druid.server.coordinator.CoordinatorCompactionConfig; import org.apache.druid.server.coordinator.CoordinatorConfigManager; import org.apache.druid.server.coordinator.CoordinatorDynamicConfig; @@ -48,8 +49,7 @@ import org.apache.druid.server.coordinator.balancer.CostBalancerStrategyFactory; import org.apache.druid.server.coordinator.balancer.DiskNormalizedCostBalancerStrategyFactory; import org.apache.druid.server.coordinator.balancer.RandomBalancerStrategyFactory; -import org.apache.druid.server.coordinator.compact.CompactionSegmentSearchPolicy; -import org.apache.druid.server.coordinator.compact.NewestSegmentFirstPolicy; +import org.apache.druid.server.coordinator.compact.CompactionStatusTracker; import org.apache.druid.server.coordinator.config.CoordinatorKillConfigs; import org.apache.druid.server.coordinator.config.CoordinatorPeriodConfig; import org.apache.druid.server.coordinator.config.CoordinatorRunConfig; @@ -87,8 +87,6 @@ public class CoordinatorSimulationBuilder DataSegment.PruneSpecsHolder.DEFAULT ) ); - private static final CompactionSegmentSearchPolicy COMPACTION_SEGMENT_SEARCH_POLICY = - new NewestSegmentFirstPolicy(OBJECT_MAPPER); private String balancerStrategy; private CoordinatorDynamicConfig dynamicConfig = CoordinatorDynamicConfig.builder().build(); private List servers; @@ -212,9 +210,10 @@ public CoordinatorSimulation build() new CoordinatorCustomDutyGroups(Collections.emptySet()), env.lookupCoordinatorManager, env.leaderSelector, - COMPACTION_SEGMENT_SEARCH_POLICY, null, - CentralizedDatasourceSchemaConfig.create() + CentralizedDatasourceSchemaConfig.create(), + CompactionSchedulerConfig.defaultConfig(), + new CompactionStatusTracker(OBJECT_MAPPER) ); return new SimulationImpl(coordinator, env); diff --git a/server/src/test/java/org/apache/druid/server/coordinator/simulate/TestSegmentsMetadataManager.java b/server/src/test/java/org/apache/druid/server/coordinator/simulate/TestSegmentsMetadataManager.java index d255d0abc7d4..4793c25f9167 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/simulate/TestSegmentsMetadataManager.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/simulate/TestSegmentsMetadataManager.java @@ -227,12 +227,6 @@ public List getUnusedSegmentIntervals( return null; } - @Override - public void poll() - { - - } - @Override public void populateUsedFlagLastUpdatedAsync() { diff --git a/services/src/main/java/org/apache/druid/cli/CliCoordinator.java b/services/src/main/java/org/apache/druid/cli/CliCoordinator.java index 7adaac5b88da..b091bffad873 100644 --- a/services/src/main/java/org/apache/druid/cli/CliCoordinator.java +++ b/services/src/main/java/org/apache/druid/cli/CliCoordinator.java @@ -97,13 +97,13 @@ import org.apache.druid.segment.metadata.SegmentSchemaCache; import org.apache.druid.server.QueryScheduler; import org.apache.druid.server.QuerySchedulerProvider; +import org.apache.druid.server.coordinator.CompactionSchedulerConfig; import org.apache.druid.server.coordinator.CoordinatorConfigManager; import org.apache.druid.server.coordinator.DruidCoordinator; import org.apache.druid.server.coordinator.MetadataManager; import org.apache.druid.server.coordinator.balancer.BalancerStrategyFactory; import org.apache.druid.server.coordinator.balancer.CachingCostBalancerStrategyConfig; -import org.apache.druid.server.coordinator.compact.CompactionSegmentSearchPolicy; -import org.apache.druid.server.coordinator.compact.NewestSegmentFirstPolicy; +import org.apache.druid.server.coordinator.compact.CompactionStatusTracker; import org.apache.druid.server.coordinator.config.CoordinatorKillConfigs; import org.apache.druid.server.coordinator.config.CoordinatorPeriodConfig; import org.apache.druid.server.coordinator.config.CoordinatorRunConfig; @@ -229,6 +229,7 @@ public void configure(Binder binder) CachingCostBalancerStrategyConfig.class ); JsonConfigProvider.bind(binder, "druid.coordinator.segmentMetadataCache", SegmentMetadataCacheConfig.class); + JsonConfigProvider.bind(binder, "druid.compaction.scheduler", CompactionSchedulerConfig.class); binder.bind(DruidCoordinatorConfig.class); binder.bind(RedirectFilter.class).in(LazySingleton.class); @@ -255,6 +256,7 @@ public void configure(Binder binder) binder.bind(LookupCoordinatorManager.class).in(LazySingleton.class); + binder.bind(CompactionStatusTracker.class).in(LazySingleton.class); binder.bind(CoordinatorConfigManager.class); binder.bind(MetadataManager.class); binder.bind(DruidCoordinator.class); @@ -292,9 +294,6 @@ public void configure(Binder binder) ); } - //TODO: make this configurable when there are multiple search policies - binder.bind(CompactionSegmentSearchPolicy.class).to(NewestSegmentFirstPolicy.class); - bindAnnouncer( binder, Coordinator.class, diff --git a/services/src/main/java/org/apache/druid/cli/CliOverlord.java b/services/src/main/java/org/apache/druid/cli/CliOverlord.java index 636d775f030f..21d9f30c7d43 100644 --- a/services/src/main/java/org/apache/druid/cli/CliOverlord.java +++ b/services/src/main/java/org/apache/druid/cli/CliOverlord.java @@ -66,6 +66,8 @@ import org.apache.druid.indexing.common.task.batch.parallel.ShuffleClient; import org.apache.druid.indexing.common.tasklogs.SwitchingTaskLogStreamer; import org.apache.druid.indexing.common.tasklogs.TaskRunnerTaskLogStreamer; +import org.apache.druid.indexing.compact.CompactionScheduler; +import org.apache.druid.indexing.compact.CompactionSchedulerImpl; import org.apache.druid.indexing.overlord.DruidOverlord; import org.apache.druid.indexing.overlord.ForkingTaskRunnerFactory; import org.apache.druid.indexing.overlord.HeapMemoryTaskStorage; @@ -91,6 +93,7 @@ import org.apache.druid.indexing.overlord.duty.TaskLogAutoCleanerConfig; import org.apache.druid.indexing.overlord.hrtr.HttpRemoteTaskRunnerFactory; import org.apache.druid.indexing.overlord.hrtr.HttpRemoteTaskRunnerResource; +import org.apache.druid.indexing.overlord.http.OverlordCompactionResource; import org.apache.druid.indexing.overlord.http.OverlordRedirectInfo; import org.apache.druid.indexing.overlord.http.OverlordResource; import org.apache.druid.indexing.overlord.sampler.SamplerModule; @@ -103,6 +106,8 @@ import org.apache.druid.indexing.worker.shuffle.IntermediaryDataManager; import org.apache.druid.indexing.worker.shuffle.LocalIntermediaryDataManager; import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.metadata.SegmentsMetadataManager; +import org.apache.druid.metadata.SegmentsMetadataManagerProvider; import org.apache.druid.metadata.input.InputSourceModule; import org.apache.druid.query.lookup.LookupSerdeModule; import org.apache.druid.segment.incremental.RowIngestionMetersFactory; @@ -111,7 +116,9 @@ import org.apache.druid.segment.realtime.NoopChatHandlerProvider; import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager; import org.apache.druid.segment.realtime.appenderator.DummyForInjectionAppenderatorsManager; +import org.apache.druid.server.coordinator.CompactionSchedulerConfig; import org.apache.druid.server.coordinator.CoordinatorOverlordServiceConfig; +import org.apache.druid.server.coordinator.compact.CompactionStatusTracker; import org.apache.druid.server.http.RedirectFilter; import org.apache.druid.server.http.RedirectInfo; import org.apache.druid.server.http.SelfDiscoveryResource; @@ -201,6 +208,12 @@ public void configure(Binder binder) binder.bindConstant().annotatedWith(Names.named("tlsServicePort")).to(8290); JsonConfigProvider.bind(binder, CentralizedDatasourceSchemaConfig.PROPERTY_PREFIX, CentralizedDatasourceSchemaConfig.class); + + binder.bind(SegmentsMetadataManager.class) + .toProvider(SegmentsMetadataManagerProvider.class) + .in(ManageLifecycle.class); + binder.bind(CompactionStatusTracker.class).in(LazySingleton.class); + JsonConfigProvider.bind(binder, "druid.compaction.scheduler", CompactionSchedulerConfig.class); } JsonConfigProvider.bind(binder, "druid.coordinator.asOverlord", CoordinatorOverlordServiceConfig.class); @@ -233,6 +246,7 @@ public void configure(Binder binder) binder.bind(TaskQueryTool.class).in(LazySingleton.class); binder.bind(IndexerMetadataStorageAdapter.class).in(LazySingleton.class); binder.bind(SupervisorManager.class).in(LazySingleton.class); + binder.bind(CompactionScheduler.class).to(CompactionSchedulerImpl.class).in(LazySingleton.class); binder.bind(ParallelIndexSupervisorTaskClientProvider.class).toProvider(Providers.of(null)); binder.bind(ShuffleClient.class).toProvider(Providers.of(null)); @@ -281,6 +295,7 @@ public void configure(Binder binder) Jerseys.addResource(binder, OverlordResource.class); Jerseys.addResource(binder, SupervisorResource.class); Jerseys.addResource(binder, HttpRemoteTaskRunnerResource.class); + Jerseys.addResource(binder, OverlordCompactionResource.class); binder.bind(AppenderatorsManager.class) diff --git a/services/src/main/java/org/apache/druid/cli/CliRouter.java b/services/src/main/java/org/apache/druid/cli/CliRouter.java index af0ca7c6eb24..4a9266fd40b2 100644 --- a/services/src/main/java/org/apache/druid/cli/CliRouter.java +++ b/services/src/main/java/org/apache/druid/cli/CliRouter.java @@ -43,6 +43,7 @@ import org.apache.druid.query.lookup.LookupSerdeModule; import org.apache.druid.server.AsyncQueryForwardingServlet; import org.apache.druid.server.NoopQuerySegmentWalker; +import org.apache.druid.server.coordinator.CompactionSchedulerConfig; import org.apache.druid.server.http.RouterResource; import org.apache.druid.server.http.SelfDiscoveryResource; import org.apache.druid.server.initialization.jetty.JettyServerInitializer; @@ -101,6 +102,7 @@ protected List getModules() JsonConfigProvider.bind(binder, "druid.router", TieredBrokerConfig.class); JsonConfigProvider.bind(binder, "druid.router.avatica.balancer", AvaticaConnectionBalancer.class); JsonConfigProvider.bind(binder, "druid.router.managementProxy", ManagementProxyConfig.class); + JsonConfigProvider.bind(binder, "druid.compaction.scheduler", CompactionSchedulerConfig.class); binder.bind(QuerySegmentWalker.class).to(NoopQuerySegmentWalker.class).in(LazySingleton.class); From bea1078f6bee0f686a464b796d69126a1475f6dd Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Mon, 22 Jul 2024 11:32:52 +0530 Subject: [PATCH 02/26] Fix checkstyle, forbidden API --- .../druid/indexing/compact/CompactionSchedulerImpl.java | 6 ++++-- .../indexing/overlord/http/OverlordCompactionResource.java | 4 ++++ .../druid/server/AsyncManagementForwardingServlet.java | 2 +- .../coordinator/compact/NewestSegmentFirstPolicy.java | 1 - 4 files changed, 9 insertions(+), 4 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionSchedulerImpl.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionSchedulerImpl.java index 02af6727f64a..2d4196cbc3cb 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionSchedulerImpl.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionSchedulerImpl.java @@ -80,8 +80,10 @@ * - [x] task state listener * - [x] handle success and failure inside CompactionStatusTracker * - [x] make policy serializable - * - [ ] handle priority datasource in policy - * - [ ] add another policy - newestSegmentFirst, smallestSegmentFirst, auto + * - [x] handle priority datasource in policy + * - [x] add another policy - smallestSegmentFirst + * - [ ] add policy auto + * - [ ] handle skipping turns if no more intervals in datasource * - [x] enable segments polling if overlord is standalone * - [ ] test on cluster - standalone, coordinator-overlord * - [ ] unit tests diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordCompactionResource.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordCompactionResource.java index 6676a9199d69..47477b4357c9 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordCompactionResource.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordCompactionResource.java @@ -35,6 +35,10 @@ import java.util.Collection; import java.util.Collections; +/** + * Contains the same logic as {@code CompactionResource} but the APIs are served + * by {@link CompactionScheduler} instead of {@code DruidCoordinator}. + */ @Path("/druid/indexer/v1/compaction") public class OverlordCompactionResource { diff --git a/server/src/main/java/org/apache/druid/server/AsyncManagementForwardingServlet.java b/server/src/main/java/org/apache/druid/server/AsyncManagementForwardingServlet.java index a85268f52f23..d8655fcb6575 100644 --- a/server/src/main/java/org/apache/druid/server/AsyncManagementForwardingServlet.java +++ b/server/src/main/java/org/apache/druid/server/AsyncManagementForwardingServlet.java @@ -115,7 +115,7 @@ protected void service(HttpServletRequest request, HttpServletResponse response) currentLeader = overlordLeaderSelector.getCurrentLeader(); request.setAttribute( MODIFIED_PATH_ATTRIBUTE, - request.getRequestURI().replace(COMPACTION_COORDINATOR_PATH, COMPACTION_OVERLORD_PATH) + StringUtils.replace(request.getRequestURI(), COMPACTION_COORDINATOR_PATH, COMPACTION_OVERLORD_PATH) ); } else if (requestURI.startsWith(STANDARD_COORDINATOR_BASE_PATH)) { currentLeader = coordLeaderSelector.getCurrentLeader(); diff --git a/server/src/main/java/org/apache/druid/server/coordinator/compact/NewestSegmentFirstPolicy.java b/server/src/main/java/org/apache/druid/server/coordinator/compact/NewestSegmentFirstPolicy.java index 55ca55eafc42..0de7602e577a 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/compact/NewestSegmentFirstPolicy.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/compact/NewestSegmentFirstPolicy.java @@ -21,7 +21,6 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import org.apache.curator.shaded.com.google.common.collect.Ordering; import org.apache.druid.java.util.common.guava.Comparators; import javax.annotation.Nullable; From 9455547974eedf24ea8752a5d1709662e7e176f6 Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Mon, 22 Jul 2024 21:12:02 +0530 Subject: [PATCH 03/26] Add some tests --- .../compact/CompactionStatusTracker.java | 42 ++- ...riorityBasedCompactionSegmentIterator.java | 4 +- .../compact/SegmentsToCompact.java | 11 +- .../coordinator/duty/CompactSegments.java | 4 +- .../coordinator/CreateDataSegments.java | 2 +- .../coordinator/DruidCoordinatorTest.java | 4 +- .../compact/CompactionStatusTrackerTest.java | 258 ++++++++++++++++++ 7 files changed, 305 insertions(+), 20 deletions(-) create mode 100644 server/src/test/java/org/apache/druid/server/coordinator/compact/CompactionStatusTrackerTest.java diff --git a/server/src/main/java/org/apache/druid/server/coordinator/compact/CompactionStatusTracker.java b/server/src/main/java/org/apache/druid/server/coordinator/compact/CompactionStatusTracker.java index fbe74ec6142d..4db729e970c1 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/compact/CompactionStatusTracker.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/compact/CompactionStatusTracker.java @@ -42,9 +42,13 @@ public class CompactionStatusTracker { private static final Logger log = new Logger(CompactionStatusTracker.class); + private static final int MAX_FAILURE_RETRIES = 3; + private static final int MAX_SKIPS_AFTER_SUCCESS = 5; + private static final int MAX_SKIPS_AFTER_FAILURE = 5; + private final ObjectMapper objectMapper; private final Map datasourceStatuses = new HashMap<>(); - private final Map submittedTaskIdToPayload = new HashMap<>(); + private final Map submittedTaskIdToSegments = new HashMap<>(); @Inject public CompactionStatusTracker( @@ -121,7 +125,7 @@ public void onTaskSubmitted( SegmentsToCompact candidateSegments ) { - submittedTaskIdToPayload.put(taskPayload.getId(), taskPayload); + submittedTaskIdToSegments.put(taskPayload.getId(), candidateSegments); getOrComputeDatasourceStatus(taskPayload.getDataSource()) .handleSubmittedTask(candidateSegments); } @@ -132,14 +136,14 @@ public void onTaskFinished(String taskId, TaskStatus taskStatus) return; } - final ClientCompactionTaskQuery taskPayload = submittedTaskIdToPayload.remove(taskId); - if (taskPayload == null) { + final SegmentsToCompact candidateSegments = submittedTaskIdToSegments.remove(taskId); + if (candidateSegments == null) { // Nothing to do since we don't know the corresponding datasource or interval return; } - final Interval compactionInterval = taskPayload.getIoConfig().getInputSpec().getInterval(); - getOrComputeDatasourceStatus(taskPayload.getDataSource()) + final Interval compactionInterval = candidateSegments.getUmbrellaInterval(); + getOrComputeDatasourceStatus(candidateSegments.getDataSource()) .handleTaskStatus(compactionInterval, taskStatus); } @@ -164,19 +168,28 @@ void handleTaskStatus(Interval compactionInterval, TaskStatus taskStatus) final IntervalStatus lastKnownStatus = intervalStatus.get(compactionInterval); if (taskStatus.isSuccess()) { - intervalStatus.put(compactionInterval, new IntervalStatus(IntervalState.COMPACTED, 10)); - } else if (lastKnownStatus == null) { + intervalStatus.put( + compactionInterval, + new IntervalStatus(IntervalState.COMPACTED, MAX_SKIPS_AFTER_SUCCESS) + ); + } else if (lastKnownStatus == null || !lastKnownStatus.isFailed()) { // This is the first failure - intervalStatus.put(compactionInterval, new IntervalStatus(IntervalState.FAILED, 0)); - } else if (lastKnownStatus.state == IntervalState.FAILED && ++lastKnownStatus.retryCount > 10) { + intervalStatus.put( + compactionInterval, + new IntervalStatus(IntervalState.FAILED, 0) + ); + } else if (++lastKnownStatus.retryCount >= MAX_FAILURE_RETRIES) { // Failure retries have been exhausted - intervalStatus.put(compactionInterval, new IntervalStatus(IntervalState.FAILED_ALL_RETRIES, 10)); + intervalStatus.put( + compactionInterval, + new IntervalStatus(IntervalState.FAILED_ALL_RETRIES, MAX_SKIPS_AFTER_FAILURE) + ); } } void handleSubmittedTask(SegmentsToCompact candidateSegments) { - getIntervalStatuses().computeIfAbsent( + intervalStatus.computeIfAbsent( candidateSegments.getUmbrellaInterval(), i -> new IntervalStatus(IntervalState.TASK_SUBMITTED, 0) ); @@ -215,6 +228,11 @@ boolean isReady() return turnsToSkip <= 0 && (state == IntervalState.COMPACTED || state == IntervalState.FAILED_ALL_RETRIES); } + + boolean isFailed() + { + return state == IntervalState.FAILED || state == IntervalState.FAILED_ALL_RETRIES; + } } private enum IntervalState diff --git a/server/src/main/java/org/apache/druid/server/coordinator/compact/PriorityBasedCompactionSegmentIterator.java b/server/src/main/java/org/apache/druid/server/coordinator/compact/PriorityBasedCompactionSegmentIterator.java index bb0006aef2d0..e5a0264fdb84 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/compact/PriorityBasedCompactionSegmentIterator.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/compact/PriorityBasedCompactionSegmentIterator.java @@ -63,7 +63,7 @@ public PriorityBasedCompactionSegmentIterator( } else { comparator = Ordering.compound( Arrays.asList( - Ordering.explicit(priorityDatasource).onResultOf(entry -> entry.getFirst().getDataSource()), + Ordering.explicit(priorityDatasource).onResultOf(SegmentsToCompact::getDataSource), segmentPriority ) ); @@ -131,7 +131,7 @@ public SegmentsToCompact next() throw new NoSuchElementException(); } - addNextItemForDatasourceToQueue(entry.getFirst().getDataSource()); + addNextItemForDatasourceToQueue(entry.getDataSource()); return entry; } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/compact/SegmentsToCompact.java b/server/src/main/java/org/apache/druid/server/coordinator/compact/SegmentsToCompact.java index 27ce9beab81f..68031a1305cd 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/compact/SegmentsToCompact.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/compact/SegmentsToCompact.java @@ -38,6 +38,7 @@ public class SegmentsToCompact private final List segments; private final Interval umbrellaInterval; + private final String datasource; private final long totalBytes; private final int numIntervals; @@ -61,6 +62,7 @@ private SegmentsToCompact() this.totalBytes = 0L; this.numIntervals = 0; this.umbrellaInterval = null; + this.datasource = null; } private SegmentsToCompact(List segments) @@ -71,6 +73,7 @@ private SegmentsToCompact(List segments) segments.stream().map(DataSegment::getInterval).collect(Collectors.toList()) ); this.numIntervals = (int) segments.stream().map(DataSegment::getInterval).distinct().count(); + this.datasource = segments.get(0).getDataSource(); } public List getSegments() @@ -107,6 +110,11 @@ public Interval getUmbrellaInterval() return umbrellaInterval; } + public String getDataSource() + { + return datasource; + } + public CompactionStatistics getStats() { return CompactionStatistics.create(totalBytes, size(), numIntervals); @@ -116,7 +124,8 @@ public CompactionStatistics getStats() public String toString() { return "SegmentsToCompact{" + - "segments=" + SegmentUtils.commaSeparatedIdentifiers(segments) + + "datasource=" + datasource + + ", segments=" + SegmentUtils.commaSeparatedIdentifiers(segments) + ", totalSize=" + totalBytes + '}'; } 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 cecc30672154..e9f4c2110799 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 @@ -406,7 +406,7 @@ private int submitCompactionTasks( throw new ISE("segmentsToCompact is empty?"); } - final String dataSourceName = entry.getFirst().getDataSource(); + final String dataSourceName = entry.getDataSource(); LOG.info("Compacting version[%s].", entry.getFirst().getVersion()); // As these segments will be compacted, we will aggregate the statistic to the Compacted statistics @@ -564,7 +564,7 @@ private void updateCompactionSnapshotStats( while (iterator.hasNext()) { final SegmentsToCompact entry = iterator.next(); if (!entry.isEmpty()) { - final String dataSourceName = entry.getFirst().getDataSource(); + final String dataSourceName = entry.getDataSource(); currentRunAutoCompactionSnapshotBuilders .computeIfAbsent(dataSourceName, AutoCompactionSnapshot::builder) .incrementWaitingStats(entry.getStats()); diff --git a/server/src/test/java/org/apache/druid/server/coordinator/CreateDataSegments.java b/server/src/test/java/org/apache/druid/server/coordinator/CreateDataSegments.java index e3c31f6471ce..7afa880927b0 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/CreateDataSegments.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/CreateDataSegments.java @@ -107,7 +107,7 @@ public List eachOfSizeInMb(long sizeMb) nextInterval, new NumberedShardSpec(numPartition, numPartitions), ++uniqueIdInInterval, - sizeMb << 20 + sizeMb * 1_000_000 ) ); } diff --git a/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorTest.java b/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorTest.java index 6111ba2f9d72..3d2e21b11ce5 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorTest.java @@ -191,7 +191,7 @@ public void setUp() throws Exception new TestDruidLeaderSelector(), null, CentralizedDatasourceSchemaConfig.create(), - CompactionSchedulerConfig.defaultConfig(), + new CompactionSchedulerConfig(true), statusTracker ); } @@ -811,7 +811,7 @@ public void testCoordinatorCustomDutyGroupsRunAsExpected() throws Exception new TestDruidLeaderSelector(), null, CentralizedDatasourceSchemaConfig.create(), - CompactionSchedulerConfig.defaultConfig(), + new CompactionSchedulerConfig(true), statusTracker ); coordinator.start(); diff --git a/server/src/test/java/org/apache/druid/server/coordinator/compact/CompactionStatusTrackerTest.java b/server/src/test/java/org/apache/druid/server/coordinator/compact/CompactionStatusTrackerTest.java new file mode 100644 index 000000000000..ae4ce7b37c5e --- /dev/null +++ b/server/src/test/java/org/apache/druid/server/coordinator/compact/CompactionStatusTrackerTest.java @@ -0,0 +1,258 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.server.coordinator.compact; + +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.client.indexing.ClientCompactionTaskQuery; +import org.apache.druid.indexer.TaskStatus; +import org.apache.druid.jackson.DefaultObjectMapper; +import org.apache.druid.server.coordinator.CreateDataSegments; +import org.apache.druid.server.coordinator.DataSourceCompactionConfig; +import org.apache.druid.timeline.DataSegment; +import org.joda.time.Duration; +import org.joda.time.Interval; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.util.Collections; + +public class CompactionStatusTrackerTest +{ + private static final ObjectMapper MAPPER = new DefaultObjectMapper(); + private static final DataSegment WIKI_SEGMENT + = CreateDataSegments.ofDatasource(DS.WIKI).eachOfSizeInMb(100).get(0); + + private CompactionStatusTracker statusTracker; + + @Before + public void setup() + { + statusTracker = new CompactionStatusTracker(MAPPER); + } + + @Test + public void testIntervalIsSkippedIfSegmentSizeExceedsLimit() + { + CompactionStatus status = statusTracker.computeCompactionStatus( + SegmentsToCompact.from(Collections.singletonList(WIKI_SEGMENT)), + createCompactionConfig(DS.WIKI, 100L) + ); + Assert.assertTrue(status.isSkipped()); + Assert.assertEquals( + "Total segment size[100000000] is larger than allowed inputSegmentSize[100]", + status.getReason() + ); + } + + @Test + public void testIntervalIsSkippedIfRecentlySubmitted() + { + final SegmentsToCompact candidateSegments + = SegmentsToCompact.from(Collections.singletonList(WIKI_SEGMENT)); + statusTracker.onTaskSubmitted(createCompactionTask("task1", DS.WIKI), candidateSegments); + + CompactionStatus status = statusTracker.computeCompactionStatus( + candidateSegments, + createCompactionConfig(DS.WIKI, null) + ); + Assert.assertTrue(status.isSkipped()); + Assert.assertEquals( + "Interval[2012-10-24T00:00:00.000Z/2012-10-25T00:00:00.000Z] was recently" + + " submitted for compaction and has state[TASK_SUBMITTED].", + status.getReason() + ); + } + + @Test + public void testIntervalIsSkippedIfRecentlyCompacted() + { + final SegmentsToCompact candidateSegments + = SegmentsToCompact.from(Collections.singletonList(WIKI_SEGMENT)); + + final String taskId = "task1"; + statusTracker.onTaskSubmitted(createCompactionTask(taskId, DS.WIKI), candidateSegments); + statusTracker.onTaskFinished(taskId, TaskStatus.success(taskId)); + + CompactionStatus status = statusTracker.computeCompactionStatus( + candidateSegments, + createCompactionConfig(DS.WIKI, null) + ); + Assert.assertTrue(status.isSkipped()); + Assert.assertEquals( + "Interval[2012-10-24T00:00:00.000Z/2012-10-25T00:00:00.000Z] was recently" + + " submitted for compaction and has state[COMPACTED].", + status.getReason() + ); + } + + @Test + public void testIntervalIsNotSkippedIfFailedOnce() + { + final SegmentsToCompact candidateSegments + = SegmentsToCompact.from(Collections.singletonList(WIKI_SEGMENT)); + + final String taskId = "task1"; + statusTracker.onTaskSubmitted(createCompactionTask(taskId, DS.WIKI), candidateSegments); + statusTracker.onTaskFinished(taskId, TaskStatus.failure(taskId, "first error")); + + CompactionStatus status = statusTracker.computeCompactionStatus( + candidateSegments, + createCompactionConfig(DS.WIKI, null) + ); + Assert.assertFalse(status.isComplete() || status.isSkipped()); + Assert.assertEquals("Not compacted yet", status.getReason()); + } + + @Test + public void testIntervalIsSkippedIfFailedForMaxRetries() + { + final SegmentsToCompact candidateSegments + = SegmentsToCompact.from(Collections.singletonList(WIKI_SEGMENT)); + final DataSourceCompactionConfig compactionConfig + = createCompactionConfig(DS.WIKI, null); + + // Verify that interval remains eligible for compaction until max failure retries + for (int i = 0; i < 4; ++i) { + CompactionStatus status = statusTracker.computeCompactionStatus( + candidateSegments, + compactionConfig + ); + Assert.assertFalse(status.isComplete() || status.isSkipped()); + Assert.assertEquals("Not compacted yet", status.getReason()); + + final String taskId = "task_" + i; + statusTracker.onTaskSubmitted(createCompactionTask(taskId, DS.WIKI), candidateSegments); + statusTracker.onTaskFinished(taskId, TaskStatus.failure(taskId, "error number " + i)); + } + + CompactionStatus status = statusTracker.computeCompactionStatus( + candidateSegments, + createCompactionConfig(DS.WIKI, null) + ); + Assert.assertTrue(status.isSkipped()); + Assert.assertEquals( + "Interval[2012-10-24T00:00:00.000Z/2012-10-25T00:00:00.000Z] was recently" + + " submitted for compaction and has state[FAILED_ALL_RETRIES].", + status.getReason() + ); + } + + @Test + public void testIntervalBecomesEligibleAgainAfterMaxSkips() + { + final SegmentsToCompact candidateSegments + = SegmentsToCompact.from(Collections.singletonList(WIKI_SEGMENT)); + + final String taskId = "task1"; + statusTracker.onTaskSubmitted(createCompactionTask(taskId, DS.WIKI), candidateSegments); + statusTracker.onTaskFinished(taskId, TaskStatus.success(taskId)); + + // Verify that interval is skipped until skip count is exhausted + for (int i = 0; i < 5; ++i) { + CompactionStatus status = statusTracker.computeCompactionStatus( + candidateSegments, + createCompactionConfig(DS.WIKI, null) + ); + Assert.assertTrue(status.isSkipped()); + Assert.assertEquals( + "Interval[2012-10-24T00:00:00.000Z/2012-10-25T00:00:00.000Z] was recently" + + " submitted for compaction and has state[COMPACTED].", + status.getReason() + ); + + // Submit task for a different interval to decrement skip count + final DataSegment segment + = DataSegment.builder(WIKI_SEGMENT) + .interval(shift(WIKI_SEGMENT.getInterval(), Duration.standardDays(i + 1))) + .build(); + statusTracker.onTaskSubmitted( + createCompactionTask("task_" + i, DS.WIKI), + SegmentsToCompact.from(Collections.singletonList(segment)) + ); + } + + // Verify that interval is now eligible again + CompactionStatus status = statusTracker.computeCompactionStatus( + candidateSegments, + createCompactionConfig(DS.WIKI, null) + ); + Assert.assertFalse(status.isComplete() || status.isSkipped()); + Assert.assertEquals("Not compacted yet", status.getReason()); + } + + @Test + public void testResetClearsAllStatuses() + { + + } + + private static Interval shift(Interval interval, Duration duration) + { + return interval.withEnd(interval.getEnd().plus(duration)) + .withStart(interval.getStart().plus(duration)); + } + + private DataSourceCompactionConfig createCompactionConfig( + String datasource, + Long inputSegmentSizeBytes + ) + { + return new DataSourceCompactionConfig( + datasource, + null, + inputSegmentSizeBytes, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null + ); + } + + private ClientCompactionTaskQuery createCompactionTask( + String taskId, + String datasource + ) + { + return new ClientCompactionTaskQuery( + taskId, + datasource, + null, + null, + null, + null, + null, + null, + null, + null + ); + } + + private static class DS + { + static final String WIKI = "wiki"; + } +} From 26d4bdcb66f917ac6180e19b18e26c333aa9ce81 Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Wed, 24 Jul 2024 12:05:40 +0530 Subject: [PATCH 04/26] Add CompactionDutySimulator --- .../compact/CompactionSchedulerImpl.java | 152 +---------- .../indexing/compact/LocalOverlordClient.java | 244 ++++++++++++++++++ .../indexing/compact/NoopOverlordClient.java | 135 ---------- .../indexing/overlord/TaskQueryTool.java | 23 +- .../overlord/http/OverlordResource.java | 13 +- .../java/util/common/guava/Comparators.java | 18 ++ .../util/common/guava/ComparatorsTest.java | 19 ++ .../compact/CompactionDutySimulator.java | 233 +++++++++++++++++ .../compact/CompactionSimulateResult.java | 44 ++++ ...riorityBasedCompactionSegmentIterator.java | 14 +- .../coordinator/duty/CompactSegments.java | 14 +- .../CoordinatorCompactionConfigsResource.java | 15 ++ .../compact/NewestSegmentFirstPolicyTest.java | 49 +++- ...rdinatorCompactionConfigsResourceTest.java | 80 +++++- .../org/apache/druid/cli/CliCoordinator.java | 2 + 15 files changed, 733 insertions(+), 322 deletions(-) create mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/compact/LocalOverlordClient.java delete mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/compact/NoopOverlordClient.java create mode 100644 server/src/main/java/org/apache/druid/server/coordinator/compact/CompactionDutySimulator.java create mode 100644 server/src/main/java/org/apache/druid/server/coordinator/compact/CompactionSimulateResult.java diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionSchedulerImpl.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionSchedulerImpl.java index 2d4196cbc3cb..cca74e706a0b 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionSchedulerImpl.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionSchedulerImpl.java @@ -21,34 +21,20 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Optional; -import com.google.common.util.concurrent.Futures; -import com.google.common.util.concurrent.ListenableFuture; import com.google.inject.Inject; import org.apache.druid.client.DataSourcesSnapshot; -import org.apache.druid.client.indexing.ClientCompactionTaskQuery; -import org.apache.druid.client.indexing.IndexingTotalWorkerCapacityInfo; -import org.apache.druid.client.indexing.TaskPayloadResponse; import org.apache.druid.common.config.JacksonConfigManager; -import org.apache.druid.error.DruidException; import org.apache.druid.indexer.TaskLocation; import org.apache.druid.indexer.TaskStatus; -import org.apache.druid.indexer.TaskStatusPlus; -import org.apache.druid.indexing.common.task.CompactionTask; import org.apache.druid.indexing.overlord.TaskMaster; import org.apache.druid.indexing.overlord.TaskQueryTool; -import org.apache.druid.indexing.overlord.TaskQueue; import org.apache.druid.indexing.overlord.TaskRunner; import org.apache.druid.indexing.overlord.TaskRunnerListener; -import org.apache.druid.indexing.overlord.http.TotalWorkerCapacityResponse; -import org.apache.druid.java.util.common.CloseableIterators; import org.apache.druid.java.util.common.Stopwatch; -import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.common.concurrent.ScheduledExecutorFactory; import org.apache.druid.java.util.common.logger.Logger; -import org.apache.druid.java.util.common.parsers.CloseableIterator; import org.apache.druid.java.util.emitter.service.ServiceEmitter; import org.apache.druid.java.util.emitter.service.ServiceMetricEvent; -import org.apache.druid.metadata.LockFilterPolicy; import org.apache.druid.metadata.SegmentsMetadataManager; import org.apache.druid.server.coordinator.AutoCompactionSnapshot; import org.apache.druid.server.coordinator.CompactionSchedulerConfig; @@ -60,16 +46,11 @@ import org.apache.druid.server.coordinator.stats.CoordinatorStat; import org.apache.druid.server.coordinator.stats.Dimension; import org.joda.time.Duration; -import org.joda.time.Interval; -import javax.annotation.Nullable; -import java.io.IOException; -import java.util.List; import java.util.Map; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; -import java.util.function.Supplier; /** * TODO: pending items @@ -94,11 +75,9 @@ public class CompactionSchedulerImpl implements CompactionScheduler private static final Logger log = new Logger(CompactionSchedulerImpl.class); private final TaskMaster taskMaster; - private final TaskQueryTool taskQueryTool; private final JacksonConfigManager configManager; private final SegmentsMetadataManager segmentManager; private final ServiceEmitter emitter; - private final ObjectMapper objectMapper; /** * Single-threaded executor to process the compaction queue. @@ -135,17 +114,19 @@ public CompactionSchedulerImpl( ) { this.taskMaster = taskMaster; - this.taskQueryTool = taskQueryTool; this.configManager = configManager; this.segmentManager = segmentManager; this.statusTracker = statusTracker; - this.objectMapper = objectMapper; this.emitter = emitter; this.schedulerConfig = schedulerConfig; this.executor = executorFactory.create(1, "CompactionScheduler-%s"); this.shouldPollSegments = segmentManager != null && !coordinatorOverlordServiceConfig.isEnabled(); - this.duty = new CompactSegments(statusTracker, new LocalOverlordClient()); + this.duty = new CompactSegments( + statusTracker, + new LocalOverlordClient(taskMaster, taskQueryTool, objectMapper) + ); + this.taskStateListener = new TaskRunnerListener() { @Override @@ -229,16 +210,6 @@ private synchronized void cleanupState() } } - private TaskQueue getValidTaskQueue() - { - Optional taskQueue = taskMaster.getTaskQueue(); - if (taskQueue.isPresent()) { - return taskQueue.get(); - } else { - throw DruidException.defensive("No TaskQueue. Cannot proceed."); - } - } - public boolean isEnabled() { return schedulerConfig.isEnabled(); @@ -322,117 +293,4 @@ public Map getAllCompactionSnapshots() return duty.getAutoCompactionSnapshot(); } - /** - * Dummy Overlord client used by the {@link #duty} to fetch task related info. - * This client simply redirects all queries to the {@link TaskQueryTool}. - */ - private class LocalOverlordClient extends NoopOverlordClient - { - @Override - public ListenableFuture runTask(String taskId, Object clientTaskQuery) - { - return futureOf(() -> { - getValidTaskQueue().add( - convertTask(clientTaskQuery, ClientCompactionTaskQuery.class, CompactionTask.class) - ); - return null; - }); - } - - @Override - public ListenableFuture cancelTask(String taskId) - { - return futureOf(() -> { - getValidTaskQueue().shutdown(taskId, "Shutdown by Compaction Scheduler"); - return null; - }); - } - - @Override - public ListenableFuture taskPayload(String taskId) - { - ClientCompactionTaskQuery taskPayload = taskQueryTool.getTask(taskId).transform( - task -> convertTask(task, CompactionTask.class, ClientCompactionTaskQuery.class) - ).orNull(); - return futureOf( - () -> new TaskPayloadResponse(taskId, taskPayload) - ); - } - - @Override - public ListenableFuture> taskStatuses( - @Nullable String state, - @Nullable String dataSource, - @Nullable Integer maxCompletedTasks - ) - { - final ListenableFuture> tasksFuture - = futureOf(taskQueryTool::getAllActiveTasks); - return Futures.transform( - tasksFuture, - taskList -> CloseableIterators.withEmptyBaggage(taskList.iterator()), - Execs.directExecutor() - ); - } - - @Override - public ListenableFuture>> findLockedIntervals(List lockFilterPolicies) - { - return futureOf(() -> taskQueryTool.getLockedIntervals(lockFilterPolicies)); - } - - @Override - public ListenableFuture getTotalWorkerCapacity() - { - return futureOf(() -> dutyCompatible(taskQueryTool.getTotalWorkerCapacity())); - } - - private ListenableFuture futureOf(Supplier supplier) - { - try { - return Futures.immediateFuture(supplier.get()); - } - catch (Exception e) { - return Futures.immediateFailedFuture(e); - } - } - - private IndexingTotalWorkerCapacityInfo dutyCompatible(TotalWorkerCapacityResponse capacity) - { - if (capacity == null) { - return null; - } else { - return new IndexingTotalWorkerCapacityInfo( - capacity.getCurrentClusterCapacity(), - capacity.getMaximumCapacityWithAutoScale() - ); - } - } - - private V convertTask(Object taskPayload, Class inputType, Class outputType) - { - if (taskPayload == null) { - return null; - } else if (inputType.isAssignableFrom(taskPayload.getClass())) { - throw DruidException.defensive( - "Unknown type[%s] for compaction task. Expected type[%s].", - taskPayload.getClass().getSimpleName(), inputType.getSimpleName() - ); - } - - try { - return objectMapper.readValue( - objectMapper.writeValueAsBytes(taskPayload), - outputType - ); - } - catch (IOException e) { - log.warn(e, "Could not convert task[%s] to client compatible object", taskPayload); - throw DruidException.defensive( - "Could not convert task[%s] to compatible object.", - taskPayload - ); - } - } - } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/LocalOverlordClient.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/LocalOverlordClient.java new file mode 100644 index 000000000000..70cbb520f716 --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/LocalOverlordClient.java @@ -0,0 +1,244 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.compact; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Optional; +import com.google.common.util.concurrent.Futures; +import com.google.common.util.concurrent.ListenableFuture; +import org.apache.druid.client.indexing.ClientCompactionTaskQuery; +import org.apache.druid.client.indexing.IndexingTotalWorkerCapacityInfo; +import org.apache.druid.client.indexing.IndexingWorkerInfo; +import org.apache.druid.client.indexing.TaskPayloadResponse; +import org.apache.druid.client.indexing.TaskStatusResponse; +import org.apache.druid.error.DruidException; +import org.apache.druid.indexer.TaskStatus; +import org.apache.druid.indexer.TaskStatusPlus; +import org.apache.druid.indexer.report.TaskReport; +import org.apache.druid.indexing.common.task.CompactionTask; +import org.apache.druid.indexing.overlord.TaskMaster; +import org.apache.druid.indexing.overlord.TaskQueryTool; +import org.apache.druid.indexing.overlord.TaskQueue; +import org.apache.druid.indexing.overlord.http.TotalWorkerCapacityResponse; +import org.apache.druid.indexing.overlord.supervisor.SupervisorStatus; +import org.apache.druid.java.util.common.CloseableIterators; +import org.apache.druid.java.util.common.concurrent.Execs; +import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.java.util.common.parsers.CloseableIterator; +import org.apache.druid.metadata.LockFilterPolicy; +import org.apache.druid.rpc.ServiceRetryPolicy; +import org.apache.druid.rpc.indexing.OverlordClient; +import org.joda.time.Interval; + +import javax.annotation.Nullable; +import java.io.IOException; +import java.net.URI; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.function.Supplier; + +/** + * Dummy Overlord client used by the {@link CompactionSchedulerImpl} to fetch + * task related info. This client simply redirects all queries to the + * {@link TaskQueryTool} and all updates to the {@link TaskQueue}. + */ +class LocalOverlordClient implements OverlordClient +{ + private static final Logger log = new Logger(LocalOverlordClient.class); + + private final TaskMaster taskMaster; + private final TaskQueryTool taskQueryTool; + private final ObjectMapper objectMapper; + + LocalOverlordClient(TaskMaster taskMaster, TaskQueryTool taskQueryTool, ObjectMapper objectMapper) + { + this.taskMaster = taskMaster; + this.taskQueryTool = taskQueryTool; + this.objectMapper = objectMapper; + } + + @Override + public ListenableFuture runTask(String taskId, Object clientTaskQuery) + { + return futureOf(() -> { + getValidTaskQueue().add( + convertTask(clientTaskQuery, ClientCompactionTaskQuery.class, CompactionTask.class) + ); + return null; + }); + } + + @Override + public ListenableFuture cancelTask(String taskId) + { + return futureOf(() -> { + getValidTaskQueue().shutdown(taskId, "Shutdown by Compaction Scheduler"); + return null; + }); + } + + @Override + public ListenableFuture taskPayload(String taskId) + { + ClientCompactionTaskQuery taskPayload = taskQueryTool.getTask(taskId).transform( + task -> convertTask(task, CompactionTask.class, ClientCompactionTaskQuery.class) + ).orNull(); + return futureOf(() -> new TaskPayloadResponse(taskId, taskPayload)); + } + + @Override + public ListenableFuture> taskStatuses( + @Nullable String state, + @Nullable String dataSource, + @Nullable Integer maxCompletedTasks + ) + { + final ListenableFuture> tasksFuture + = futureOf(taskQueryTool::getAllActiveTasks); + return Futures.transform( + tasksFuture, + taskList -> CloseableIterators.withEmptyBaggage(taskList.iterator()), + Execs.directExecutor() + ); + } + + @Override + public ListenableFuture> taskStatuses(Set taskIds) + { + return futureOf(() -> taskQueryTool.getMultipleTaskStatuses(taskIds)); + } + + @Override + public ListenableFuture>> findLockedIntervals( + List lockFilterPolicies + ) + { + return futureOf(() -> taskQueryTool.getLockedIntervals(lockFilterPolicies)); + } + + @Override + public ListenableFuture getTotalWorkerCapacity() + { + return futureOf(() -> convert(taskQueryTool.getTotalWorkerCapacity())); + } + + private TaskQueue getValidTaskQueue() + { + Optional taskQueue = taskMaster.getTaskQueue(); + if (taskQueue.isPresent()) { + return taskQueue.get(); + } else { + throw DruidException.defensive("No TaskQueue. Cannot proceed."); + } + } + + private ListenableFuture futureOf(Supplier supplier) + { + try { + return Futures.immediateFuture(supplier.get()); + } + catch (Exception e) { + return Futures.immediateFailedFuture(e); + } + } + + private IndexingTotalWorkerCapacityInfo convert(TotalWorkerCapacityResponse capacity) + { + if (capacity == null) { + return null; + } else { + return new IndexingTotalWorkerCapacityInfo( + capacity.getCurrentClusterCapacity(), + capacity.getMaximumCapacityWithAutoScale() + ); + } + } + + private V convertTask(Object taskPayload, Class inputType, Class outputType) + { + if (taskPayload == null) { + return null; + } else if (inputType.isAssignableFrom(taskPayload.getClass())) { + throw DruidException.defensive( + "Unknown type[%s] for compaction task. Expected type[%s].", + taskPayload.getClass().getSimpleName(), inputType.getSimpleName() + ); + } + + try { + return objectMapper.readValue( + objectMapper.writeValueAsBytes(taskPayload), + outputType + ); + } + catch (IOException e) { + log.warn(e, "Could not convert task[%s] to client compatible object", taskPayload); + throw DruidException.defensive( + "Could not convert task[%s] to compatible object.", + taskPayload + ); + } + } + + // Unsupported methods as these are not used by the CompactionScheduler / CompactSegments duty + + @Override + public ListenableFuture findCurrentLeader() + { + throw new UnsupportedOperationException(); + } + + @Override + public ListenableFuture taskStatus(String taskId) + { + throw new UnsupportedOperationException(); + } + + @Override + public ListenableFuture taskReportAsMap(String taskId) + { + throw new UnsupportedOperationException(); + } + + @Override + public ListenableFuture> supervisorStatuses() + { + throw new UnsupportedOperationException(); + } + + @Override + public ListenableFuture killPendingSegments(String dataSource, Interval interval) + { + throw new UnsupportedOperationException(); + } + + @Override + public ListenableFuture> getWorkers() + { + throw new UnsupportedOperationException(); + } + + @Override + public OverlordClient withRetryPolicy(ServiceRetryPolicy retryPolicy) + { + return this; + } +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/NoopOverlordClient.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/NoopOverlordClient.java deleted file mode 100644 index 38343568c105..000000000000 --- a/indexing-service/src/main/java/org/apache/druid/indexing/compact/NoopOverlordClient.java +++ /dev/null @@ -1,135 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.druid.indexing.compact; - -import com.google.common.util.concurrent.ListenableFuture; -import org.apache.druid.client.indexing.IndexingTotalWorkerCapacityInfo; -import org.apache.druid.client.indexing.IndexingWorkerInfo; -import org.apache.druid.client.indexing.TaskPayloadResponse; -import org.apache.druid.client.indexing.TaskStatusResponse; -import org.apache.druid.indexer.TaskStatus; -import org.apache.druid.indexer.TaskStatusPlus; -import org.apache.druid.indexer.report.TaskReport; -import org.apache.druid.indexing.overlord.supervisor.SupervisorStatus; -import org.apache.druid.java.util.common.parsers.CloseableIterator; -import org.apache.druid.metadata.LockFilterPolicy; -import org.apache.druid.rpc.ServiceRetryPolicy; -import org.apache.druid.rpc.indexing.OverlordClient; -import org.joda.time.Interval; - -import javax.annotation.Nullable; -import java.net.URI; -import java.util.List; -import java.util.Map; -import java.util.Set; - -public class NoopOverlordClient implements OverlordClient -{ - @Override - public ListenableFuture findCurrentLeader() - { - throw new UnsupportedOperationException(); - } - - @Override - public ListenableFuture runTask(String taskId, Object taskObject) - { - throw new UnsupportedOperationException(); - } - - @Override - public ListenableFuture cancelTask(String taskId) - { - throw new UnsupportedOperationException(); - } - - @Override - public ListenableFuture> taskStatuses( - @Nullable String state, - @Nullable String dataSource, - @Nullable Integer maxCompletedTasks - ) - { - throw new UnsupportedOperationException(); - } - - @Override - public ListenableFuture> taskStatuses(Set taskIds) - { - throw new UnsupportedOperationException(); - } - - @Override - public ListenableFuture taskStatus(String taskId) - { - throw new UnsupportedOperationException(); - } - - @Override - public ListenableFuture taskPayload(String taskId) - { - throw new UnsupportedOperationException(); - } - - @Override - public ListenableFuture taskReportAsMap(String taskId) - { - throw new UnsupportedOperationException(); - } - - @Override - public ListenableFuture> supervisorStatuses() - { - throw new UnsupportedOperationException(); - } - - @Override - public ListenableFuture>> findLockedIntervals( - List lockFilterPolicies - ) - { - throw new UnsupportedOperationException(); - } - - @Override - public ListenableFuture killPendingSegments(String dataSource, Interval interval) - { - throw new UnsupportedOperationException(); - } - - @Override - public ListenableFuture> getWorkers() - { - throw new UnsupportedOperationException(); - } - - @Override - public ListenableFuture getTotalWorkerCapacity() - { - throw new UnsupportedOperationException(); - } - - @Override - public OverlordClient withRetryPolicy(ServiceRetryPolicy retryPolicy) - { - // Ignore retryPolicy for the test client. - return this; - } -} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskQueryTool.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskQueryTool.java index c092b62190e4..331680f9a230 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskQueryTool.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskQueryTool.java @@ -22,6 +22,7 @@ import com.google.common.base.Optional; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Maps; import com.google.inject.Inject; import org.apache.druid.common.config.JacksonConfigManager; import org.apache.druid.indexer.TaskInfo; @@ -49,6 +50,7 @@ import java.util.Collections; import java.util.List; import java.util.Map; +import java.util.Set; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -114,12 +116,17 @@ public List> getActiveTaskInfo(@Nullable String dataS return storage.getTaskInfos(TaskLookup.activeTasksOnly(), dataSource); } - private List getTaskStatusPlusList( - Map taskLookups, - @Nullable String dataSource - ) + public Map getMultipleTaskStatuses(Set taskIds) { - return storage.getTaskStatusPlusList(taskLookups, dataSource); + final Map result = Maps.newHashMapWithExpectedSize(taskIds.size()); + for (String taskId : taskIds) { + final Optional optional = getTaskStatus(taskId); + if (optional.isPresent()) { + result.put(taskId, optional.get()); + } + } + + return result; } public Optional getTask(final String taskId) @@ -285,10 +292,8 @@ private Stream getTaskStatusPlusStream( throw new IAE("Unknown state: [%s]", state); } - final Stream taskStatusPlusStream = getTaskStatusPlusList( - taskLookups, - dataSource - ).stream(); + final Stream taskStatusPlusStream + = storage.getTaskStatusPlusList(taskLookups, dataSource).stream(); if (type != null) { return taskStatusPlusStream.filter( statusPlus -> type.equals(statusPlus == null ? null : statusPlus.getType()) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordResource.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordResource.java index 54ada7cb2b43..8a7cd599e37a 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordResource.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordResource.java @@ -26,7 +26,6 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Lists; -import com.google.common.collect.Maps; import com.google.inject.Inject; import com.sun.jersey.spi.container.ResourceFilters; import org.apache.druid.audit.AuditEntry; @@ -419,17 +418,9 @@ public Response getMultipleTaskStatuses(Set taskIds) { if (CollectionUtils.isNullOrEmpty(taskIds)) { return Response.status(Response.Status.BAD_REQUEST).entity("No Task IDs provided.").build(); + } else { + return Response.ok().entity(taskQueryTool.getMultipleTaskStatuses(taskIds)).build(); } - - final Map result = Maps.newHashMapWithExpectedSize(taskIds.size()); - for (String taskId : taskIds) { - final Optional optional = taskQueryTool.getTaskStatus(taskId); - if (optional.isPresent()) { - result.put(taskId, optional.get()); - } - } - - return Response.ok().entity(result).build(); } @GET diff --git a/processing/src/main/java/org/apache/druid/java/util/common/guava/Comparators.java b/processing/src/main/java/org/apache/druid/java/util/common/guava/Comparators.java index 61e1912d2eb0..ad4c161c142c 100644 --- a/processing/src/main/java/org/apache/druid/java/util/common/guava/Comparators.java +++ b/processing/src/main/java/org/apache/druid/java/util/common/guava/Comparators.java @@ -19,12 +19,14 @@ package org.apache.druid.java.util.common.guava; +import com.google.common.base.Preconditions; import com.google.common.collect.Ordering; import org.joda.time.DateTimeComparator; import org.joda.time.Interval; import javax.annotation.Nullable; import java.util.Comparator; +import java.util.Objects; /** */ @@ -51,6 +53,22 @@ public static Ordering alwaysEqual() return (Ordering) ALWAYS_EQUAL; } + /** + * Creates an ordering which always gives priority to the passed value. + */ + public static Ordering alwaysFirst(T value) + { + Preconditions.checkNotNull(value, "value cannot be null"); + + return Ordering.from((o1, o2) -> { + if (value.equals(o1)) { + return value.equals(o2) ? 0 : -1; + } else { + return value.equals(o2) ? 1 : 0; + } + }); + } + @SuppressWarnings("unchecked") public static > Ordering naturalNullsFirst() { diff --git a/processing/src/test/java/org/apache/druid/java/util/common/guava/ComparatorsTest.java b/processing/src/test/java/org/apache/druid/java/util/common/guava/ComparatorsTest.java index 816e2d5a32a9..53530852653b 100644 --- a/processing/src/test/java/org/apache/druid/java/util/common/guava/ComparatorsTest.java +++ b/processing/src/test/java/org/apache/druid/java/util/common/guava/ComparatorsTest.java @@ -102,4 +102,23 @@ public void testIntervalsByEndThenStart() intervals ); } + + @Test + public void testAlwaysFirst() + { + final Comparator comparator = Comparators.alwaysFirst(25); + Assert.assertEquals(0, comparator.compare(25, 25)); + Assert.assertEquals(-1, comparator.compare(25, 1000)); + Assert.assertEquals(1, comparator.compare(1000, 25)); + Assert.assertEquals(-1, comparator.compare(25, 1)); + Assert.assertEquals(1, comparator.compare(1, 25)); + Assert.assertEquals(0, comparator.compare(10, 12)); + + Integer[] array = {25, 0, 1, 25, -10, 100, 31, 77, 89, 25}; + Arrays.sort(array, comparator); + Assert.assertArrayEquals( + new Integer[]{25, 25, 25, 0, 1, -10, 100, 31, 77, 89}, + array + ); + } } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/compact/CompactionDutySimulator.java b/server/src/main/java/org/apache/druid/server/coordinator/compact/CompactionDutySimulator.java new file mode 100644 index 000000000000..b12ebce0aabb --- /dev/null +++ b/server/src/main/java/org/apache/druid/server/coordinator/compact/CompactionDutySimulator.java @@ -0,0 +1,233 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.server.coordinator.compact; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.util.concurrent.Futures; +import com.google.common.util.concurrent.ListenableFuture; +import com.google.inject.Inject; +import org.apache.druid.client.indexing.ClientCompactionTaskQuery; +import org.apache.druid.client.indexing.IndexingTotalWorkerCapacityInfo; +import org.apache.druid.client.indexing.IndexingWorkerInfo; +import org.apache.druid.client.indexing.TaskPayloadResponse; +import org.apache.druid.client.indexing.TaskStatusResponse; +import org.apache.druid.indexer.TaskStatus; +import org.apache.druid.indexer.TaskStatusPlus; +import org.apache.druid.indexer.report.TaskReport; +import org.apache.druid.indexing.overlord.supervisor.SupervisorStatus; +import org.apache.druid.java.util.common.CloseableIterators; +import org.apache.druid.java.util.common.parsers.CloseableIterator; +import org.apache.druid.metadata.LockFilterPolicy; +import org.apache.druid.rpc.ServiceRetryPolicy; +import org.apache.druid.rpc.indexing.OverlordClient; +import org.apache.druid.server.coordinator.CoordinatorCompactionConfig; +import org.apache.druid.server.coordinator.MetadataManager; +import org.apache.druid.server.coordinator.duty.CompactSegments; +import org.apache.druid.server.coordinator.stats.CoordinatorRunStats; +import org.apache.druid.server.http.CompactionConfigUpdateRequest; +import org.apache.druid.timeline.SegmentTimeline; +import org.joda.time.Interval; + +import javax.annotation.Nullable; +import java.net.URI; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Set; + +/** + * Simulates runs of auto-compaction duty to verify the expected list of + * compaction tasks that would be submitted by the actual compaction duty. + */ +public class CompactionDutySimulator +{ + private final ObjectMapper objectMapper; + private final MetadataManager metadataManager; + private final OverlordClient emptyOverlordClient = new EmptyOverlordClient(); + + @Inject + public CompactionDutySimulator( + MetadataManager metadataManager, + ObjectMapper objectMapper + ) + { + this.objectMapper = objectMapper; + this.metadataManager = metadataManager; + } + + /** + * Simulates a run of the compact segments duty with the given config update + * assuming unlimited compaction task slots. + */ + public CompactionSimulateResult simulateRunWithConfigUpdate( + CompactionConfigUpdateRequest updateRequest + ) + { + final CompactionConfigUpdateRequest updateWithUnlimitedSlots = new CompactionConfigUpdateRequest( + 1.0, + Integer.MAX_VALUE, + updateRequest.getUseAutoScaleSlots(), + updateRequest.getCompactionEngine(), + updateRequest.getCompactionPolicy() + ); + final CoordinatorCompactionConfig configWithUnlimitedTaskSlots = CoordinatorCompactionConfig.from( + metadataManager.configs().getCurrentCompactionConfig(), + updateWithUnlimitedSlots + ); + final Map datasourceTimelines + = metadataManager.segments() + .getSnapshotOfDataSourcesWithAllUsedSegments() + .getUsedSegmentsTimelinesPerDataSource(); + + final List> tableOfSubmittedTasks = new ArrayList<>(); + final CompactionStatusTracker statusTracker = new CompactionStatusTracker(objectMapper) { + @Override + public void onTaskSubmitted(ClientCompactionTaskQuery taskPayload, SegmentsToCompact candidateSegments) + { + // Add a row for each task in order of submission + tableOfSubmittedTasks.add( + Arrays.asList( + candidateSegments.getDataSource(), + candidateSegments.getUmbrellaInterval(), + candidateSegments.size(), + candidateSegments.getTotalBytes(), + CompactSegments.findMaxNumTaskSlotsUsedByOneNativeCompactionTask(taskPayload.getTuningConfig()), + "" + ) + ); + } + }; + + final CoordinatorRunStats stats = new CoordinatorRunStats(); + new CompactSegments(statusTracker, emptyOverlordClient).run( + configWithUnlimitedTaskSlots, + datasourceTimelines, + stats + ); + + if (!tableOfSubmittedTasks.isEmpty()) { + // Add header row + tableOfSubmittedTasks.add( + 0, + Arrays.asList("dataSource", "interval", "numSegments", "bytes", "maxTaskSlots", "reasonToCompact") + ); + } + + return new CompactionSimulateResult(tableOfSubmittedTasks); + } + + /** + * Dummy overlord client that returns empty results for all APIs. + */ + private static class EmptyOverlordClient implements OverlordClient + { + @Override + public ListenableFuture findCurrentLeader() + { + return null; + } + + @Override + public ListenableFuture runTask(String taskId, Object taskObject) + { + return Futures.immediateVoidFuture(); + } + + @Override + public ListenableFuture cancelTask(String taskId) + { + return Futures.immediateVoidFuture(); + } + + @Override + public ListenableFuture> taskStatuses( + @Nullable String state, + @Nullable String dataSource, + @Nullable Integer maxCompletedTasks + ) + { + return Futures.immediateFuture(CloseableIterators.withEmptyBaggage(Collections.emptyIterator())); + } + + @Override + public ListenableFuture> taskStatuses(Set taskIds) + { + return Futures.immediateFuture(Collections.emptyMap()); + } + + @Override + public ListenableFuture taskStatus(String taskId) + { + return null; + } + + @Override + public ListenableFuture taskReportAsMap(String taskId) + { + return null; + } + + @Override + public ListenableFuture taskPayload(String taskId) + { + return Futures.immediateFuture(null); + } + + @Override + public ListenableFuture> supervisorStatuses() + { + return null; + } + + @Override + public ListenableFuture>> findLockedIntervals(List lockFilterPolicies) + { + return Futures.immediateFuture(Collections.emptyMap()); + } + + @Override + public ListenableFuture killPendingSegments(String dataSource, Interval interval) + { + return null; + } + + @Override + public ListenableFuture> getWorkers() + { + return Futures.immediateFuture(Collections.emptyList()); + } + + @Override + public ListenableFuture getTotalWorkerCapacity() + { + return Futures.immediateFuture( + new IndexingTotalWorkerCapacityInfo(Integer.MAX_VALUE, Integer.MAX_VALUE) + ); + } + + @Override + public OverlordClient withRetryPolicy(ServiceRetryPolicy retryPolicy) + { + return this; + } + } +} diff --git a/server/src/main/java/org/apache/druid/server/coordinator/compact/CompactionSimulateResult.java b/server/src/main/java/org/apache/druid/server/coordinator/compact/CompactionSimulateResult.java new file mode 100644 index 000000000000..9da5226cfbea --- /dev/null +++ b/server/src/main/java/org/apache/druid/server/coordinator/compact/CompactionSimulateResult.java @@ -0,0 +1,44 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.server.coordinator.compact; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +import java.util.List; + +public class CompactionSimulateResult +{ + private final List> submittedTasks; + + @JsonCreator + public CompactionSimulateResult( + @JsonProperty("submittedTasks") List> submittedTasks + ) + { + this.submittedTasks = submittedTasks; + } + + @JsonProperty + public List> getSubmittedTasks() + { + return submittedTasks; + } +} diff --git a/server/src/main/java/org/apache/druid/server/coordinator/compact/PriorityBasedCompactionSegmentIterator.java b/server/src/main/java/org/apache/druid/server/coordinator/compact/PriorityBasedCompactionSegmentIterator.java index e5a0264fdb84..a8526b011722 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/compact/PriorityBasedCompactionSegmentIterator.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/compact/PriorityBasedCompactionSegmentIterator.java @@ -20,8 +20,8 @@ package org.apache.druid.server.coordinator.compact; import com.google.common.collect.Maps; -import com.google.common.collect.Ordering; import org.apache.druid.error.DruidException; +import org.apache.druid.java.util.common.guava.Comparators; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; import org.apache.druid.timeline.SegmentTimeline; @@ -29,7 +29,6 @@ import org.joda.time.Interval; import javax.annotation.Nullable; -import java.util.Arrays; import java.util.Collections; import java.util.Comparator; import java.util.List; @@ -58,15 +57,12 @@ public PriorityBasedCompactionSegmentIterator( ) { final Comparator comparator; - if (priorityDatasource == null) { + if (priorityDatasource == null || priorityDatasource.isEmpty()) { comparator = segmentPriority; } else { - comparator = Ordering.compound( - Arrays.asList( - Ordering.explicit(priorityDatasource).onResultOf(SegmentsToCompact::getDataSource), - segmentPriority - ) - ); + comparator = Comparators.alwaysFirst(priorityDatasource) + .onResultOf(SegmentsToCompact::getDataSource) + .thenComparing(segmentPriority); } this.queue = new PriorityQueue<>(comparator); 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 e9f4c2110799..bed743ea07fc 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 @@ -116,7 +116,7 @@ public OverlordClient getOverlordClient() @Override public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params) { - statusTracker.reset(); + // statusTracker.reset(); run( params.getCoordinatorCompactionConfig(), params.getUsedSegmentsTimelinesPerDataSource(), @@ -304,8 +304,7 @@ private Map> getLockedIntervals( * Returns the maximum number of task slots used by one native compaction task at any time when the task is * issued with the given tuningConfig. */ - @VisibleForTesting - static int findMaxNumTaskSlotsUsedByOneNativeCompactionTask( + public static int findMaxNumTaskSlotsUsedByOneNativeCompactionTask( @Nullable ClientCompactionTaskQueryTuningConfig tuningConfig ) { @@ -418,7 +417,6 @@ private int submitCompactionTasks( final List segmentsToCompact = entry.getSegments(); // Create granularitySpec to send to compaction task - ClientCompactionTaskGranularitySpec granularitySpec; Granularity segmentGranularityToUse = null; if (config.getGranularitySpec() == null || config.getGranularitySpec().getSegmentGranularity() == null) { // Determines segmentGranularity from the segmentsToCompact @@ -443,14 +441,14 @@ private int submitCompactionTasks( } else { segmentGranularityToUse = config.getGranularitySpec().getSegmentGranularity(); } - granularitySpec = new ClientCompactionTaskGranularitySpec( + final ClientCompactionTaskGranularitySpec granularitySpec = new ClientCompactionTaskGranularitySpec( segmentGranularityToUse, config.getGranularitySpec() != null ? config.getGranularitySpec().getQueryGranularity() : null, config.getGranularitySpec() != null ? config.getGranularitySpec().isRollup() : null ); // Create dimensionsSpec to send to compaction task - ClientCompactionTaskDimensionsSpec dimensionsSpec; + final ClientCompactionTaskDimensionsSpec dimensionsSpec; if (config.getDimensionsSpec() != null) { dimensionsSpec = new ClientCompactionTaskDimensionsSpec( config.getDimensionsSpec().getDimensions() @@ -638,8 +636,8 @@ public Map getAutoCompactionSnapshot() private String compactSegments( SegmentsToCompact entry, int compactionTaskPriority, - @Nullable ClientCompactionTaskQueryTuningConfig tuningConfig, - @Nullable ClientCompactionTaskGranularitySpec granularitySpec, + ClientCompactionTaskQueryTuningConfig tuningConfig, + ClientCompactionTaskGranularitySpec granularitySpec, @Nullable ClientCompactionTaskDimensionsSpec dimensionsSpec, @Nullable AggregatorFactory[] metricsSpec, @Nullable ClientCompactionTaskTransformSpec transformSpec, diff --git a/server/src/main/java/org/apache/druid/server/http/CoordinatorCompactionConfigsResource.java b/server/src/main/java/org/apache/druid/server/http/CoordinatorCompactionConfigsResource.java index 9549c1d51d66..e31c9604ce5b 100644 --- a/server/src/main/java/org/apache/druid/server/http/CoordinatorCompactionConfigsResource.java +++ b/server/src/main/java/org/apache/druid/server/http/CoordinatorCompactionConfigsResource.java @@ -38,6 +38,7 @@ import org.apache.druid.server.coordinator.CoordinatorConfigManager; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; import org.apache.druid.server.coordinator.DataSourceCompactionConfigHistory; +import org.apache.druid.server.coordinator.compact.CompactionDutySimulator; import org.apache.druid.server.http.security.ConfigResourceFilter; import org.apache.druid.server.security.AuthorizationUtils; import org.joda.time.Interval; @@ -72,15 +73,18 @@ public class CoordinatorCompactionConfigsResource static final int UPDATE_NUM_RETRY = 5; private final CoordinatorConfigManager configManager; + private final CompactionDutySimulator simulator; private final AuditManager auditManager; @Inject public CoordinatorCompactionConfigsResource( CoordinatorConfigManager configManager, + CompactionDutySimulator simulator, AuditManager auditManager ) { this.configManager = configManager; + this.simulator = simulator; this.auditManager = auditManager; } @@ -104,6 +108,17 @@ public Response updateCompactionDynamicConfig( return updateConfigHelper(operator, AuthorizationUtils.buildAuditInfo(req)); } + @POST + @Path("/simulate") + @Consumes(MediaType.APPLICATION_JSON) + public Response simulateCompactionDynamicConfig( + CompactionConfigUpdateRequest updatePayload, + @Context HttpServletRequest req + ) + { + return Response.ok().entity(simulator.simulateRunWithConfigUpdate(updatePayload)).build(); + } + @POST @Path("/taskslots") @Consumes(MediaType.APPLICATION_JSON) diff --git a/server/src/test/java/org/apache/druid/server/coordinator/compact/NewestSegmentFirstPolicyTest.java b/server/src/test/java/org/apache/druid/server/coordinator/compact/NewestSegmentFirstPolicyTest.java index 5d5d10cec67d..84b2c37d0430 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/compact/NewestSegmentFirstPolicyTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/compact/NewestSegmentFirstPolicyTest.java @@ -46,6 +46,7 @@ import org.apache.druid.segment.data.ConciseBitmapSerdeFactory; import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.segment.transform.TransformSpec; +import org.apache.druid.server.coordinator.CreateDataSegments; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; import org.apache.druid.server.coordinator.UserCompactionTaskDimensionsConfig; import org.apache.druid.server.coordinator.UserCompactionTaskGranularityConfig; @@ -79,7 +80,8 @@ public class NewestSegmentFirstPolicyTest { - private static final String DATA_SOURCE = "dataSource"; + private static final String DATA_SOURCE = "wikipedia"; + private static final String DATASOURCE_KOALA = "koala"; private static final long DEFAULT_SEGMENT_SIZE = 1000; private static final int DEFAULT_NUM_SEGMENTS_PER_SHARD = 4; private final ObjectMapper mapper = new DefaultObjectMapper(); @@ -1928,6 +1930,51 @@ public void testSkipCompactionForIntervalsContainingSingleTombstone() ); } + @Test + public void testPriorityDatasource() + { + final List wikiSegments + = CreateDataSegments.ofDatasource(DATA_SOURCE) + .forIntervals(1, Granularities.DAY) + .startingAt("2012-01-01") + .withNumPartitions(10) + .eachOfSizeInMb(100); + final List koalaSegments + = CreateDataSegments.ofDatasource(DATASOURCE_KOALA) + .forIntervals(1, Granularities.DAY) + .startingAt("2013-01-01") + .withNumPartitions(10) + .eachOfSizeInMb(100); + + // Setup policy and iterator with priorityDatasource = wikipedia + final NewestSegmentFirstPolicy policy = new NewestSegmentFirstPolicy(DATA_SOURCE); + CompactionSegmentIterator iterator = policy.createIterator( + ImmutableMap.of( + DATA_SOURCE, createCompactionConfig(Long.MAX_VALUE, Period.seconds(0), null), + DATASOURCE_KOALA, createCompactionConfig(Long.MAX_VALUE, Period.seconds(0), null) + ), + ImmutableMap.of( + DATA_SOURCE, SegmentTimeline.forSegments(wikiSegments), + DATASOURCE_KOALA, SegmentTimeline.forSegments(koalaSegments) + ), + Collections.emptyMap(), + statusTracker + ); + + // Verify that the segments of "wikipedia" are preferred even though they are older + Assert.assertTrue(iterator.hasNext()); + SegmentsToCompact next = iterator.next(); + Assert.assertFalse(next.isEmpty()); + Assert.assertEquals(DATA_SOURCE, next.getDataSource()); + Assert.assertEquals(Intervals.of("2012-01-01/P1D"), next.getUmbrellaInterval()); + + Assert.assertTrue(iterator.hasNext()); + next = iterator.next(); + Assert.assertFalse(next.isEmpty()); + Assert.assertEquals(DATASOURCE_KOALA, next.getDataSource()); + Assert.assertEquals(Intervals.of("2013-01-01/P1D"), next.getUmbrellaInterval()); + } + private static void assertCompactSegmentIntervals( CompactionSegmentIterator iterator, Period segmentPeriod, diff --git a/server/src/test/java/org/apache/druid/server/http/CoordinatorCompactionConfigsResourceTest.java b/server/src/test/java/org/apache/druid/server/http/CoordinatorCompactionConfigsResourceTest.java index 17db22854779..bdeeba296b79 100644 --- a/server/src/test/java/org/apache/druid/server/http/CoordinatorCompactionConfigsResourceTest.java +++ b/server/src/test/java/org/apache/druid/server/http/CoordinatorCompactionConfigsResourceTest.java @@ -19,6 +19,7 @@ package org.apache.druid.server.http; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import org.apache.druid.audit.AuditManager; @@ -28,14 +29,22 @@ import org.apache.druid.error.DruidException; import org.apache.druid.error.ErrorResponse; import org.apache.druid.indexer.CompactionEngine; +import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.metadata.MetadataStorageConnector; import org.apache.druid.metadata.MetadataStorageTablesConfig; import org.apache.druid.server.coordinator.CoordinatorCompactionConfig; import org.apache.druid.server.coordinator.CoordinatorConfigManager; +import org.apache.druid.server.coordinator.CreateDataSegments; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; +import org.apache.druid.server.coordinator.MetadataManager; import org.apache.druid.server.coordinator.UserCompactionTaskGranularityConfig; +import org.apache.druid.server.coordinator.compact.CompactionDutySimulator; +import org.apache.druid.server.coordinator.compact.CompactionSimulateResult; +import org.apache.druid.server.coordinator.simulate.TestSegmentsMetadataManager; +import org.apache.druid.timeline.DataSegment; import org.joda.time.Period; import org.junit.Assert; import org.junit.Before; @@ -49,11 +58,15 @@ import javax.servlet.http.HttpServletRequest; import javax.ws.rs.core.Response; +import java.util.Arrays; import java.util.Collection; +import java.util.Collections; +import java.util.List; @RunWith(MockitoJUnitRunner.class) public class CoordinatorCompactionConfigsResourceTest { + private static final ObjectMapper OBJECT_MAPPER = new DefaultObjectMapper(); private static final DataSourceCompactionConfig OLD_CONFIG = new DataSourceCompactionConfig( "oldDataSource", null, @@ -133,6 +146,7 @@ public void setup() ).thenReturn(ImmutableList.of()); coordinatorCompactionConfigsResource = new CoordinatorCompactionConfigsResource( new CoordinatorConfigManager(mockJacksonConfigManager, mockConnector, mockConnectorConfig), + null, mockAuditManager ); Mockito.when(mockHttpServletRequest.getRemoteAddr()).thenReturn("123"); @@ -400,8 +414,6 @@ public void testAddOrUpdateCompactionConfigWithoutExistingConfig() CompactionEngine.MSQ, ImmutableMap.of("key", "val") ); - String author = "maytas"; - String comment = "hello"; Response result = coordinatorCompactionConfigsResource.addOrUpdateCompactionConfig( newConfig, mockHttpServletRequest @@ -543,4 +555,68 @@ public void testGetCompactionConfigHistoryForUnknownDataSourceShouldReturnEmptyL Assert.assertEquals(Response.Status.OK.getStatusCode(), response.getStatus()); Assert.assertTrue(((Collection) response.getEntity()).isEmpty()); } + + @Test + public void testSimulateCompactionDynamicConfig() + { + final TestSegmentsMetadataManager segmentsMetadataManager = new TestSegmentsMetadataManager(); + final CoordinatorConfigManager mockConfigManager = Mockito.mock(CoordinatorConfigManager.class); + Mockito.when(mockConfigManager.getCurrentCompactionConfig()).thenReturn( + new CoordinatorCompactionConfig( + Collections.singletonList(createDatasourceConfig("wiki")), + null, null, null, null, null + ) + ); + + final MetadataManager metadataManager = new MetadataManager( + mockAuditManager, + mockConfigManager, segmentsMetadataManager, null, null, null, null + ); + + // Add some segments to the timeline + final List wikiSegments + = CreateDataSegments.ofDatasource("wiki") + .forIntervals(10, Granularities.DAY) + .withNumPartitions(10) + .startingAt("2013-01-01") + .eachOfSizeInMb(100); + wikiSegments.forEach(segmentsMetadataManager::addSegment); + + coordinatorCompactionConfigsResource = new CoordinatorCompactionConfigsResource( + null, + new CompactionDutySimulator(metadataManager, OBJECT_MAPPER), + null + ); + Response response = coordinatorCompactionConfigsResource.simulateCompactionDynamicConfig( + new CompactionConfigUpdateRequest(null, null, null, null, null), + mockHttpServletRequest + ); + Assert.assertEquals(Response.Status.OK.getStatusCode(), response.getStatus()); + Assert.assertTrue(response.getEntity() instanceof CompactionSimulateResult); + + CompactionSimulateResult simulateResult = (CompactionSimulateResult) response.getEntity(); + Assert.assertEquals( + Arrays.asList( + Arrays.asList("dataSource", "interval", "numSegments", "bytes", "reasonToCompact"), + Arrays.asList("wiki", Intervals.of("2013-01-09/P1D"), 10, 1_000_000_000L, 2, ""), + Arrays.asList("wiki", Intervals.of("2013-01-08/P1D"), 10, 1_000_000_000L, 2, ""), + Arrays.asList("wiki", Intervals.of("2013-01-07/P1D"), 10, 1_000_000_000L, 2, ""), + Arrays.asList("wiki", Intervals.of("2013-01-06/P1D"), 10, 1_000_000_000L, 2, ""), + Arrays.asList("wiki", Intervals.of("2013-01-05/P1D"), 10, 1_000_000_000L, 2, ""), + Arrays.asList("wiki", Intervals.of("2013-01-04/P1D"), 10, 1_000_000_000L, 2, ""), + Arrays.asList("wiki", Intervals.of("2013-01-03/P1D"), 10, 1_000_000_000L, 2, ""), + Arrays.asList("wiki", Intervals.of("2013-01-02/P1D"), 10, 1_000_000_000L, 2, ""), + Arrays.asList("wiki", Intervals.of("2013-01-01/P1D"), 10, 1_000_000_000L, 2, "") + ), + simulateResult.getSubmittedTasks() + ); + } + + private static DataSourceCompactionConfig createDatasourceConfig(String datasource) + { + return new DataSourceCompactionConfig( + "wiki", + null, null, null, null, null, null, null, null, null, null, null, null + ); + } } diff --git a/services/src/main/java/org/apache/druid/cli/CliCoordinator.java b/services/src/main/java/org/apache/druid/cli/CliCoordinator.java index b091bffad873..dc6298e4f232 100644 --- a/services/src/main/java/org/apache/druid/cli/CliCoordinator.java +++ b/services/src/main/java/org/apache/druid/cli/CliCoordinator.java @@ -103,6 +103,7 @@ import org.apache.druid.server.coordinator.MetadataManager; import org.apache.druid.server.coordinator.balancer.BalancerStrategyFactory; import org.apache.druid.server.coordinator.balancer.CachingCostBalancerStrategyConfig; +import org.apache.druid.server.coordinator.compact.CompactionDutySimulator; import org.apache.druid.server.coordinator.compact.CompactionStatusTracker; import org.apache.druid.server.coordinator.config.CoordinatorKillConfigs; import org.apache.druid.server.coordinator.config.CoordinatorPeriodConfig; @@ -260,6 +261,7 @@ public void configure(Binder binder) binder.bind(CoordinatorConfigManager.class); binder.bind(MetadataManager.class); binder.bind(DruidCoordinator.class); + binder.bind(CompactionDutySimulator.class).in(LazySingleton.class); LifecycleModule.register(binder, MetadataStorage.class); LifecycleModule.register(binder, DruidCoordinator.class); From caefb27999c98bfe766bb235744f9491477ba05a Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Wed, 24 Jul 2024 13:34:39 +0530 Subject: [PATCH 05/26] Fix tests and checkstyle --- .../java/util/common/guava/Comparators.java | 4 +-- .../CoordinatorCompactionConfigsResource.java | 3 +- ...rdinatorCompactionConfigsResourceTest.java | 28 +++++++++---------- 3 files changed, 17 insertions(+), 18 deletions(-) diff --git a/processing/src/main/java/org/apache/druid/java/util/common/guava/Comparators.java b/processing/src/main/java/org/apache/druid/java/util/common/guava/Comparators.java index ad4c161c142c..9eb4578820b1 100644 --- a/processing/src/main/java/org/apache/druid/java/util/common/guava/Comparators.java +++ b/processing/src/main/java/org/apache/druid/java/util/common/guava/Comparators.java @@ -26,7 +26,6 @@ import javax.annotation.Nullable; import java.util.Comparator; -import java.util.Objects; /** */ @@ -54,7 +53,8 @@ public static Ordering alwaysEqual() } /** - * Creates an ordering which always gives priority to the passed value. + * Creates an ordering which always gives priority to the specified value. + * Other values are considered equal to each other. */ public static Ordering alwaysFirst(T value) { diff --git a/server/src/main/java/org/apache/druid/server/http/CoordinatorCompactionConfigsResource.java b/server/src/main/java/org/apache/druid/server/http/CoordinatorCompactionConfigsResource.java index e31c9604ce5b..82c4a8af2ee1 100644 --- a/server/src/main/java/org/apache/druid/server/http/CoordinatorCompactionConfigsResource.java +++ b/server/src/main/java/org/apache/druid/server/http/CoordinatorCompactionConfigsResource.java @@ -112,8 +112,7 @@ public Response updateCompactionDynamicConfig( @Path("/simulate") @Consumes(MediaType.APPLICATION_JSON) public Response simulateCompactionDynamicConfig( - CompactionConfigUpdateRequest updatePayload, - @Context HttpServletRequest req + CompactionConfigUpdateRequest updatePayload ) { return Response.ok().entity(simulator.simulateRunWithConfigUpdate(updatePayload)).build(); diff --git a/server/src/test/java/org/apache/druid/server/http/CoordinatorCompactionConfigsResourceTest.java b/server/src/test/java/org/apache/druid/server/http/CoordinatorCompactionConfigsResourceTest.java index bdeeba296b79..2ecbd62704c1 100644 --- a/server/src/test/java/org/apache/druid/server/http/CoordinatorCompactionConfigsResourceTest.java +++ b/server/src/test/java/org/apache/druid/server/http/CoordinatorCompactionConfigsResourceTest.java @@ -574,8 +574,9 @@ public void testSimulateCompactionDynamicConfig() ); // Add some segments to the timeline + final String datasource = "wiki"; final List wikiSegments - = CreateDataSegments.ofDatasource("wiki") + = CreateDataSegments.ofDatasource(datasource) .forIntervals(10, Granularities.DAY) .withNumPartitions(10) .startingAt("2013-01-01") @@ -588,8 +589,7 @@ public void testSimulateCompactionDynamicConfig() null ); Response response = coordinatorCompactionConfigsResource.simulateCompactionDynamicConfig( - new CompactionConfigUpdateRequest(null, null, null, null, null), - mockHttpServletRequest + new CompactionConfigUpdateRequest(null, null, null, null, null) ); Assert.assertEquals(Response.Status.OK.getStatusCode(), response.getStatus()); Assert.assertTrue(response.getEntity() instanceof CompactionSimulateResult); @@ -597,16 +597,16 @@ public void testSimulateCompactionDynamicConfig() CompactionSimulateResult simulateResult = (CompactionSimulateResult) response.getEntity(); Assert.assertEquals( Arrays.asList( - Arrays.asList("dataSource", "interval", "numSegments", "bytes", "reasonToCompact"), - Arrays.asList("wiki", Intervals.of("2013-01-09/P1D"), 10, 1_000_000_000L, 2, ""), - Arrays.asList("wiki", Intervals.of("2013-01-08/P1D"), 10, 1_000_000_000L, 2, ""), - Arrays.asList("wiki", Intervals.of("2013-01-07/P1D"), 10, 1_000_000_000L, 2, ""), - Arrays.asList("wiki", Intervals.of("2013-01-06/P1D"), 10, 1_000_000_000L, 2, ""), - Arrays.asList("wiki", Intervals.of("2013-01-05/P1D"), 10, 1_000_000_000L, 2, ""), - Arrays.asList("wiki", Intervals.of("2013-01-04/P1D"), 10, 1_000_000_000L, 2, ""), - Arrays.asList("wiki", Intervals.of("2013-01-03/P1D"), 10, 1_000_000_000L, 2, ""), - Arrays.asList("wiki", Intervals.of("2013-01-02/P1D"), 10, 1_000_000_000L, 2, ""), - Arrays.asList("wiki", Intervals.of("2013-01-01/P1D"), 10, 1_000_000_000L, 2, "") + Arrays.asList("dataSource", "interval", "numSegments", "bytes", "maxTaskSlots", "reasonToCompact"), + Arrays.asList("wiki", Intervals.of("2013-01-09/P1D"), 10, 1_000_000_000L, 1, ""), + Arrays.asList("wiki", Intervals.of("2013-01-08/P1D"), 10, 1_000_000_000L, 1, ""), + Arrays.asList("wiki", Intervals.of("2013-01-07/P1D"), 10, 1_000_000_000L, 1, ""), + Arrays.asList("wiki", Intervals.of("2013-01-06/P1D"), 10, 1_000_000_000L, 1, ""), + Arrays.asList("wiki", Intervals.of("2013-01-05/P1D"), 10, 1_000_000_000L, 1, ""), + Arrays.asList("wiki", Intervals.of("2013-01-04/P1D"), 10, 1_000_000_000L, 1, ""), + Arrays.asList("wiki", Intervals.of("2013-01-03/P1D"), 10, 1_000_000_000L, 1, ""), + Arrays.asList("wiki", Intervals.of("2013-01-02/P1D"), 10, 1_000_000_000L, 1, ""), + Arrays.asList("wiki", Intervals.of("2013-01-01/P1D"), 10, 1_000_000_000L, 1, "") ), simulateResult.getSubmittedTasks() ); @@ -615,7 +615,7 @@ public void testSimulateCompactionDynamicConfig() private static DataSourceCompactionConfig createDatasourceConfig(String datasource) { return new DataSourceCompactionConfig( - "wiki", + datasource, null, null, null, null, null, null, null, null, null, null, null, null ); } From bbc2eb64545530a000144e6f1ae1eed1a7d7ed70 Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Sat, 27 Jul 2024 09:26:13 +0530 Subject: [PATCH 06/26] More changes for simulator --- .../indexing/compact/LocalOverlordClient.java | 2 +- .../compact/CompactionSchedulerTest.java | 49 ++++++++++ .../compact/CompactionDutySimulator.java | 55 +++++++++-- .../compact/CompactionSimulateResult.java | 19 +++- .../coordinator/compact/CompactionStatus.java | 96 +++++++++++++++---- .../compact/CompactionStatusTracker.java | 25 ++++- .../DataSourceCompactibleSegmentIterator.java | 58 +++++++---- .../compact/SegmentsToCompact.java | 35 ++++--- .../compact/CompactionStatusTest.java | 62 ++++++++++++ ...aSourceCompactibleSegmentIteratorTest.java | 6 +- ...rdinatorCompactionConfigsResourceTest.java | 22 ++--- 11 files changed, 344 insertions(+), 85 deletions(-) create mode 100644 indexing-service/src/test/java/org/apache/druid/indexing/compact/CompactionSchedulerTest.java diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/LocalOverlordClient.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/LocalOverlordClient.java index 70cbb520f716..b9358fd86f86 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/compact/LocalOverlordClient.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/LocalOverlordClient.java @@ -176,7 +176,7 @@ private V convertTask(Object taskPayload, Class inputType, Class ou { if (taskPayload == null) { return null; - } else if (inputType.isAssignableFrom(taskPayload.getClass())) { + } else if (!inputType.isInstance(taskPayload)) { throw DruidException.defensive( "Unknown type[%s] for compaction task. Expected type[%s].", taskPayload.getClass().getSimpleName(), inputType.getSimpleName() diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/compact/CompactionSchedulerTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/compact/CompactionSchedulerTest.java new file mode 100644 index 000000000000..ca7e8aeac640 --- /dev/null +++ b/indexing-service/src/test/java/org/apache/druid/indexing/compact/CompactionSchedulerTest.java @@ -0,0 +1,49 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.compact; + +import org.junit.Test; + +/** + * Tests the scheduling behaviour of the Compaction Scheduler and not the + * compaction of segments itself. There are other tests which already verify + * the compaction of segments with different configs and datasources. + */ +public class CompactionSchedulerTest +{ + // what are the different aspects we would like to test + // task status updates + // + // config + // + + // there should also be a test where we can do some sort of comparison of the two things + + // CompactSegmentsTest is not the right place for that because the entry point is CompactSegments + + // For us, the entry point is DruidCoordinator (i.e. sim) vs Compaction Scheduler. + + @Test + public void test() + { + + } + +} diff --git a/server/src/main/java/org/apache/druid/server/coordinator/compact/CompactionDutySimulator.java b/server/src/main/java/org/apache/druid/server/coordinator/compact/CompactionDutySimulator.java index b12ebce0aabb..14cf27456c3e 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/compact/CompactionDutySimulator.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/compact/CompactionDutySimulator.java @@ -38,6 +38,7 @@ import org.apache.druid.rpc.ServiceRetryPolicy; import org.apache.druid.rpc.indexing.OverlordClient; import org.apache.druid.server.coordinator.CoordinatorCompactionConfig; +import org.apache.druid.server.coordinator.DataSourceCompactionConfig; import org.apache.druid.server.coordinator.MetadataManager; import org.apache.druid.server.coordinator.duty.CompactSegments; import org.apache.druid.server.coordinator.stats.CoordinatorRunStats; @@ -62,14 +63,17 @@ public class CompactionDutySimulator { private final ObjectMapper objectMapper; private final MetadataManager metadataManager; + private final CompactionStatusTracker statusTracker; private final OverlordClient emptyOverlordClient = new EmptyOverlordClient(); @Inject public CompactionDutySimulator( + CompactionStatusTracker statusTracker, MetadataManager metadataManager, ObjectMapper objectMapper ) { + this.statusTracker = statusTracker; this.objectMapper = objectMapper; this.metadataManager = metadataManager; } @@ -98,41 +102,74 @@ public CompactionSimulateResult simulateRunWithConfigUpdate( .getSnapshotOfDataSourcesWithAllUsedSegments() .getUsedSegmentsTimelinesPerDataSource(); - final List> tableOfSubmittedTasks = new ArrayList<>(); - final CompactionStatusTracker statusTracker = new CompactionStatusTracker(objectMapper) { + final List> tableOfCompactibleIntervals = new ArrayList<>(); + final List> tableOfSkippedIntervals = new ArrayList<>(); + final CompactionStatusTracker simulationStatusTracker = new CompactionStatusTracker(objectMapper) { + @Override + public CompactionStatus computeCompactionStatus( + SegmentsToCompact candidateSegments, + DataSourceCompactionConfig config + ) + { + return statusTracker.computeCompactionStatus(candidateSegments, config); + } + + @Override + public void onIntervalSkipped(SegmentsToCompact candidateSegments, CompactionStatus status) + { + // Add a row for each skipped interval + tableOfSkippedIntervals.add( + Arrays.asList( + candidateSegments.getDataSource(), + candidateSegments.getUmbrellaInterval(), + candidateSegments.size(), + candidateSegments.getTotalBytes(), + status.getReason() + ) + ); + } + @Override public void onTaskSubmitted(ClientCompactionTaskQuery taskPayload, SegmentsToCompact candidateSegments) { // Add a row for each task in order of submission - tableOfSubmittedTasks.add( + final CompactionStatus status = candidateSegments.getCompactionStatus(); + final String reason = status == null ? "" : status.getReason(); + tableOfCompactibleIntervals.add( Arrays.asList( candidateSegments.getDataSource(), candidateSegments.getUmbrellaInterval(), candidateSegments.size(), candidateSegments.getTotalBytes(), CompactSegments.findMaxNumTaskSlotsUsedByOneNativeCompactionTask(taskPayload.getTuningConfig()), - "" + reason ) ); } }; final CoordinatorRunStats stats = new CoordinatorRunStats(); - new CompactSegments(statusTracker, emptyOverlordClient).run( + new CompactSegments(simulationStatusTracker, emptyOverlordClient).run( configWithUnlimitedTaskSlots, datasourceTimelines, stats ); - if (!tableOfSubmittedTasks.isEmpty()) { - // Add header row - tableOfSubmittedTasks.add( + // Add header rows + if (!tableOfCompactibleIntervals.isEmpty()) { + tableOfCompactibleIntervals.add( 0, Arrays.asList("dataSource", "interval", "numSegments", "bytes", "maxTaskSlots", "reasonToCompact") ); } + if (!tableOfSkippedIntervals.isEmpty()) { + tableOfSkippedIntervals.add( + 0, + Arrays.asList("dataSource", "interval", "numSegments", "bytes", "reasonToSkip") + ); + } - return new CompactionSimulateResult(tableOfSubmittedTasks); + return new CompactionSimulateResult(tableOfCompactibleIntervals, tableOfSkippedIntervals); } /** diff --git a/server/src/main/java/org/apache/druid/server/coordinator/compact/CompactionSimulateResult.java b/server/src/main/java/org/apache/druid/server/coordinator/compact/CompactionSimulateResult.java index 9da5226cfbea..c21a8e227020 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/compact/CompactionSimulateResult.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/compact/CompactionSimulateResult.java @@ -26,19 +26,28 @@ public class CompactionSimulateResult { - private final List> submittedTasks; + private final List> intervalsToCompact; + private final List> skippedIntervals; @JsonCreator public CompactionSimulateResult( - @JsonProperty("submittedTasks") List> submittedTasks + @JsonProperty("intervalsToCompact") List> intervalsToCompact, + @JsonProperty("skippedIntervals") List> skippedIntervals ) { - this.submittedTasks = submittedTasks; + this.intervalsToCompact = intervalsToCompact; + this.skippedIntervals = skippedIntervals; } @JsonProperty - public List> getSubmittedTasks() + public List> getIntervalsToCompact() { - return submittedTasks; + return intervalsToCompact; + } + + @JsonProperty + public List> getSkippedIntervals() + { + return skippedIntervals; } } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/compact/CompactionStatus.java b/server/src/main/java/org/apache/druid/server/coordinator/compact/CompactionStatus.java index a63aef39315d..490bd377658f 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/compact/CompactionStatus.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/compact/CompactionStatus.java @@ -27,10 +27,13 @@ import org.apache.druid.client.indexing.ClientCompactionTaskTransformSpec; import org.apache.druid.common.config.Configs; import org.apache.druid.data.input.impl.DimensionsSpec; +import org.apache.druid.indexer.partitions.DimensionRangePartitionsSpec; import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; +import org.apache.druid.indexer.partitions.HashedPartitionsSpec; import org.apache.druid.indexer.partitions.PartitionsSpec; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.Granularity; +import org.apache.druid.java.util.common.granularity.GranularityType; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.segment.IndexSpec; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; @@ -102,23 +105,72 @@ private static CompactionStatus incomplete(String reasonFormat, Object... args) return new CompactionStatus(State.PENDING, StringUtils.format(reasonFormat, args)); } - private static CompactionStatus completeIfEqual(String field, Object configured, Object current) + private static CompactionStatus completeIfEqual( + String field, + T configured, + T current, + Function stringFunction + ) { if (configured == null || configured.equals(current)) { return COMPLETE; } else { - return configChanged(field, configured, current); + return configChanged(field, configured, current, stringFunction); } } - private static CompactionStatus configChanged(String field, Object configured, Object current) + private static CompactionStatus configChanged( + String field, + T target, + T current, + Function stringFunction + ) { return CompactionStatus.incomplete( - "Configured %s[%s] is different from current %s[%s]", - field, configured, field, current + "'%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) + { + if (granularity == null) { + return null; + } + for (GranularityType type : GranularityType.values()) { + if (type.getDefaultGranularity().equals(granularity)) { + return type.toString(); + } + } + return granularity.toString(); + } + + private static String asString(PartitionsSpec partitionsSpec) { + if (partitionsSpec instanceof DimensionRangePartitionsSpec) { + DimensionRangePartitionsSpec rangeSpec = (DimensionRangePartitionsSpec) partitionsSpec; + return StringUtils.format( + "'range' on %s with %,d rows", + rangeSpec.getPartitionDimensions(), rangeSpec.getTargetRowsPerSegment() + ); + } else if (partitionsSpec instanceof HashedPartitionsSpec) { + HashedPartitionsSpec hashedSpec = (HashedPartitionsSpec) partitionsSpec; + return StringUtils.format( + "'hashed' on %s with %,d rows", + hashedSpec.getPartitionDimensions(), hashedSpec.getTargetRowsPerSegment() + ); + } else if (partitionsSpec instanceof DynamicPartitionsSpec) { + DynamicPartitionsSpec dynamicSpec = (DynamicPartitionsSpec) partitionsSpec; + return StringUtils.format( + "'dynamic' with %,d rows", + dynamicSpec.getMaxRowsPerSegment() + ); + } else { + return partitionsSpec.toString(); + } + } + static CompactionStatus skipped(String reasonFormat, Object... args) { return new CompactionStatus(State.SKIPPED, StringUtils.format(reasonFormat, args)); @@ -197,7 +249,7 @@ private Evaluator( private CompactionStatus segmentsHaveBeenCompactedAtLeastOnce() { if (lastCompactionState == null) { - return CompactionStatus.incomplete("Not compacted yet"); + return CompactionStatus.incomplete("not compacted yet"); } else { return COMPLETE; } @@ -211,7 +263,7 @@ private CompactionStatus allCandidatesHaveSameCompactionState() if (allHaveSameCompactionState) { return COMPLETE; } else { - return CompactionStatus.incomplete("Candidate segments have different last compaction states."); + return CompactionStatus.incomplete("segments have different last compaction states"); } } @@ -220,7 +272,8 @@ private CompactionStatus partitionsSpecIsUpToDate() return CompactionStatus.completeIfEqual( "partitionsSpec", findPartitionsSpecFromConfig(tuningConfig), - lastCompactionState.getPartitionsSpec() + lastCompactionState.getPartitionsSpec(), + CompactionStatus::asString ); } @@ -229,7 +282,8 @@ private CompactionStatus indexSpecIsUpToDate() return CompactionStatus.completeIfEqual( "indexSpec", Configs.valueOrDefault(tuningConfig.getIndexSpec(), IndexSpec.DEFAULT), - objectMapper.convertValue(lastCompactionState.getIndexSpec(), IndexSpec.class) + objectMapper.convertValue(lastCompactionState.getIndexSpec(), IndexSpec.class), + String::valueOf ); } @@ -254,15 +308,16 @@ private CompactionStatus segmentGranularityIsUpToDate() ); if (needsCompaction) { return CompactionStatus.incomplete( - "Configured segmentGranularity[%s] does not align with segment intervals.", - configuredSegmentGranularity + "segmentGranularity: segments do not align with target[%s]", + asString(configuredSegmentGranularity) ); } } else { return CompactionStatus.configChanged( "segmentGranularity", configuredSegmentGranularity, - existingSegmentGranularity + existingSegmentGranularity, + CompactionStatus::asString ); } @@ -277,7 +332,8 @@ private CompactionStatus rollupIsUpToDate() return CompactionStatus.completeIfEqual( "rollup", configuredGranularitySpec.isRollup(), - existingGranularitySpec == null ? null : existingGranularitySpec.isRollup() + existingGranularitySpec == null ? null : existingGranularitySpec.isRollup(), + String::valueOf ); } } @@ -290,7 +346,8 @@ private CompactionStatus queryGranularityIsUpToDate() return CompactionStatus.completeIfEqual( "queryGranularity", configuredGranularitySpec.getQueryGranularity(), - existingGranularitySpec == null ? null : existingGranularitySpec.getQueryGranularity() + existingGranularitySpec == null ? null : existingGranularitySpec.getQueryGranularity(), + CompactionStatus::asString ); } } @@ -304,7 +361,8 @@ private CompactionStatus dimensionsSpecIsUpToDate() return CompactionStatus.completeIfEqual( "dimensionsSpec", compactionConfig.getDimensionsSpec().getDimensions(), - existingDimensionsSpec == null ? null : existingDimensionsSpec.getDimensions() + existingDimensionsSpec == null ? null : existingDimensionsSpec.getDimensions(), + String::valueOf ); } } @@ -324,8 +382,9 @@ private CompactionStatus metricsSpecIsUpToDate() if (existingMetricsSpec == null || !Arrays.deepEquals(configuredMetricsSpec, existingMetricsSpec)) { return CompactionStatus.configChanged( "metricsSpec", - Arrays.toString(configuredMetricsSpec), - Arrays.toString(existingMetricsSpec) + configuredMetricsSpec, + existingMetricsSpec, + Arrays::toString ); } else { return COMPLETE; @@ -345,7 +404,8 @@ private CompactionStatus transformSpecFilterIsUpToDate() return CompactionStatus.completeIfEqual( "transformSpec filter", compactionConfig.getTransformSpec().getFilter(), - existingTransformSpec == null ? null : existingTransformSpec.getFilter() + existingTransformSpec == null ? null : existingTransformSpec.getFilter(), + String::valueOf ); } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/compact/CompactionStatusTracker.java b/server/src/main/java/org/apache/druid/server/coordinator/compact/CompactionStatusTracker.java index 4db729e970c1..079bfd448c60 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/compact/CompactionStatusTracker.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/compact/CompactionStatusTracker.java @@ -37,6 +37,17 @@ * Tracks status of both recently submitted compaction tasks and the compaction * state of segments. Can be used to check if a set of segments is currently * eligible for compaction. + *

+ * TODO: Keep the interval sidelined until + * - no other interval to compact for the last 1 hour + * - some other intervals have been submitted + * - threshold has been crossed - 30% uncompacted bytes or 50% uncompacted segments + * - interval is uncompacted and has just 1 uncompacted segment out of 100 + * - level of uncompaction is important to know + * - should this be a part of policy + * + * + * - A policy that picks up stuff only if it meets some thresholds and then */ public class CompactionStatusTracker { @@ -71,7 +82,7 @@ public CompactionStatus computeCompactionStatus( final long inputSegmentSize = config.getInputSegmentSizeBytes(); if (candidate.getTotalBytes() > inputSegmentSize) { return CompactionStatus.skipped( - "Total segment size[%d] is larger than allowed inputSegmentSize[%d]", + "'inputSegmentSize' exceeded: Total segment size[%d] is larger than allowed inputSegmentSize[%d]", candidate.getTotalBytes(), inputSegmentSize ); } @@ -92,8 +103,8 @@ public CompactionStatus computeCompactionStatus( case COMPACTED: case FAILED_ALL_RETRIES: return CompactionStatus.skipped( - "Interval[%s] was recently submitted for compaction and has state[%s].", - compactionInterval, intervalStatus.state + "recently submitted: current compaction state[%s]", + intervalStatus.state ); default: break; @@ -120,6 +131,14 @@ public void onCompactionConfigUpdated(CoordinatorCompactionConfig compactionConf }); } + public void onIntervalSkipped( + SegmentsToCompact candidateSegments, + CompactionStatus status + ) + { + // do nothing + } + public void onTaskSubmitted( ClientCompactionTaskQuery taskPayload, SegmentsToCompact candidateSegments diff --git a/server/src/main/java/org/apache/druid/server/coordinator/compact/DataSourceCompactibleSegmentIterator.java b/server/src/main/java/org/apache/druid/server/coordinator/compact/DataSourceCompactibleSegmentIterator.java index e0e00a25a624..17ec6115b8d6 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/compact/DataSourceCompactibleSegmentIterator.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/compact/DataSourceCompactibleSegmentIterator.java @@ -36,6 +36,7 @@ import org.apache.druid.timeline.partition.NumberedPartitionChunk; import org.apache.druid.timeline.partition.NumberedShardSpec; import org.apache.druid.timeline.partition.PartitionChunk; +import org.apache.druid.utils.CollectionUtils; import org.apache.druid.utils.Streams; import org.joda.time.DateTime; import org.joda.time.Interval; @@ -297,6 +298,9 @@ private void findAndEnqueueSegmentsToCompact(CompactibleSegmentIterator compacti { while (compactibleSegmentIterator.hasNext()) { List segments = compactibleSegmentIterator.next(); + if (CollectionUtils.isNullOrEmpty(segments)) { + continue; + } // Do not compact an interval which comprises of a single tombstone // If there are multiple tombstones in the interval, we may still want to compact them @@ -319,6 +323,7 @@ private void findAndEnqueueSegmentsToCompact(CompactibleSegmentIterator compacti compactedSegmentStats.increment(candidates.getStats()); } else if (compactionStatus.isSkipped()) { skippedSegmentStats.increment(candidates.getStats()); + statusTracker.onIntervalSkipped(candidates, compactionStatus); log.warn( "Skipping compaction for datasource[%s], interval[%s] due to reason[%s].", dataSource, interval, compactionStatus.getReason() @@ -329,10 +334,10 @@ private void findAndEnqueueSegmentsToCompact(CompactibleSegmentIterator compacti // Skip these candidate segments as we have already queued this interval } else { queuedIntervals.add(interval); - queue.add(candidates); + queue.add(candidates.withStatus(compactionStatus)); } } else { - queue.add(candidates); + queue.add(candidates.withStatus(compactionStatus)); } } @@ -360,10 +365,13 @@ private List findInitialSearchInterval( final TimelineObjectHolder first = Preconditions.checkNotNull(timeline.first(), "first"); final TimelineObjectHolder last = Preconditions.checkNotNull(timeline.last(), "last"); - final List fullSkipIntervals = sortAndAddSkipIntervalFromLatest( - last.getInterval().getEnd(), - skipOffset, + final Interval latestSkipInterval = computeLatestSkipInterval( configuredSegmentGranularity, + last.getInterval().getEnd(), + skipOffset + ); + final List fullSkipIntervals = sortAndAddSkipIntervalFromLatest( + latestSkipInterval, skipIntervals ); @@ -372,7 +380,18 @@ private List findInitialSearchInterval( final List segments = new ArrayList<>( timeline.findNonOvershadowedObjectsInInterval(skipInterval, Partitions.ONLY_COMPLETE) ); - skippedSegmentStats.increment(SegmentsToCompact.from(segments).getStats()); + if (!CollectionUtils.isNullOrEmpty(segments)) { + SegmentsToCompact candidates = SegmentsToCompact.from(segments); + skippedSegmentStats.increment(candidates.getStats()); + + final CompactionStatus reason; + if (skipInterval.overlaps(latestSkipInterval)) { + reason = CompactionStatus.skipped("skip offset from latest[%s]", skipOffset); + } else { + reason = CompactionStatus.skipped("interval locked by another task"); + } + statusTracker.onIntervalSkipped(candidates, reason); + } } final Interval totalInterval = new Interval(first.getInterval().getStart(), last.getInterval().getEnd()); @@ -413,25 +432,32 @@ private List findInitialSearchInterval( return searchIntervals; } - @VisibleForTesting - static List sortAndAddSkipIntervalFromLatest( - DateTime latest, - Period skipOffset, + Interval computeLatestSkipInterval( Granularity configuredSegmentGranularity, - @Nullable List skipIntervals + DateTime latest, + Period skipOffsetFromLatest ) { - final List nonNullSkipIntervals = skipIntervals == null - ? new ArrayList<>(1) - : new ArrayList<>(skipIntervals.size()); final Interval skipFromLatest; if (configuredSegmentGranularity != null) { - DateTime skipFromLastest = new DateTime(latest, latest.getZone()).minus(skipOffset); + DateTime skipFromLastest = new DateTime(latest, latest.getZone()).minus(skipOffsetFromLatest); DateTime skipOffsetBucketToSegmentGranularity = configuredSegmentGranularity.bucketStart(skipFromLastest); skipFromLatest = new Interval(skipOffsetBucketToSegmentGranularity, latest); } else { - skipFromLatest = new Interval(skipOffset, latest); + skipFromLatest = new Interval(skipOffsetFromLatest, latest); } + return skipFromLatest; + } + + @VisibleForTesting + static List sortAndAddSkipIntervalFromLatest( + Interval skipFromLatest, + @Nullable List skipIntervals + ) + { + final List nonNullSkipIntervals = skipIntervals == null + ? new ArrayList<>(1) + : new ArrayList<>(skipIntervals.size()); if (skipIntervals != null) { final List sortedSkipIntervals = new ArrayList<>(skipIntervals); diff --git a/server/src/main/java/org/apache/druid/server/coordinator/compact/SegmentsToCompact.java b/server/src/main/java/org/apache/druid/server/coordinator/compact/SegmentsToCompact.java index 68031a1305cd..c0c8d8508c59 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/compact/SegmentsToCompact.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/compact/SegmentsToCompact.java @@ -19,12 +19,12 @@ package org.apache.druid.server.coordinator.compact; +import org.apache.druid.error.InvalidInput; import org.apache.druid.java.util.common.JodaUtils; import org.apache.druid.segment.SegmentUtils; import org.apache.druid.timeline.DataSegment; import org.joda.time.Interval; -import java.util.Collections; import java.util.List; import java.util.NoSuchElementException; import java.util.stream.Collectors; @@ -34,38 +34,24 @@ */ public class SegmentsToCompact { - private static final SegmentsToCompact EMPTY_INSTANCE = new SegmentsToCompact(); - private final List segments; private final Interval umbrellaInterval; private final String datasource; private final long totalBytes; private final int numIntervals; - static SegmentsToCompact empty() - { - return EMPTY_INSTANCE; - } + private final CompactionStatus compactionStatus; public static SegmentsToCompact from(List segments) { if (segments == null || segments.isEmpty()) { - return empty(); + throw InvalidInput.exception("Segments to compact must be non-empty"); } else { - return new SegmentsToCompact(segments); + return new SegmentsToCompact(segments, null); } } - private SegmentsToCompact() - { - this.segments = Collections.emptyList(); - this.totalBytes = 0L; - this.numIntervals = 0; - this.umbrellaInterval = null; - this.datasource = null; - } - - private SegmentsToCompact(List segments) + private SegmentsToCompact(List segments, CompactionStatus status) { this.segments = segments; this.totalBytes = segments.stream().mapToLong(DataSegment::getSize).sum(); @@ -74,6 +60,7 @@ private SegmentsToCompact(List segments) ); this.numIntervals = (int) segments.stream().map(DataSegment::getInterval).distinct().count(); this.datasource = segments.get(0).getDataSource(); + this.compactionStatus = status; } public List getSegments() @@ -120,6 +107,16 @@ public CompactionStatistics getStats() return CompactionStatistics.create(totalBytes, size(), numIntervals); } + public CompactionStatus getCompactionStatus() + { + return compactionStatus; + } + + public SegmentsToCompact withStatus(CompactionStatus status) + { + return new SegmentsToCompact(this.segments, status); + } + @Override public String toString() { diff --git a/server/src/test/java/org/apache/druid/server/coordinator/compact/CompactionStatusTest.java b/server/src/test/java/org/apache/druid/server/coordinator/compact/CompactionStatusTest.java index 0e13f8cd0e1e..15caa77f22d9 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/compact/CompactionStatusTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/compact/CompactionStatusTest.java @@ -19,20 +19,32 @@ package org.apache.druid.server.coordinator.compact; +import com.fasterxml.jackson.databind.ObjectMapper; import org.apache.druid.client.indexing.ClientCompactionTaskQueryTuningConfig; import org.apache.druid.indexer.partitions.DimensionRangePartitionsSpec; import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; import org.apache.druid.indexer.partitions.HashedPartitionsSpec; import org.apache.druid.indexer.partitions.PartitionsSpec; +import org.apache.druid.jackson.DefaultObjectMapper; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.granularity.Granularities; +import org.apache.druid.java.util.common.granularity.Granularity; +import org.apache.druid.segment.indexing.granularity.GranularitySpec; +import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; +import org.apache.druid.server.coordinator.UserCompactionTaskGranularityConfig; import org.apache.druid.server.coordinator.UserCompactionTaskQueryTuningConfig; +import org.apache.druid.timeline.CompactionState; +import org.apache.druid.timeline.DataSegment; import org.junit.Assert; import org.junit.Test; +import java.util.Arrays; import java.util.Collections; public class CompactionStatusTest { + private static final ObjectMapper OBJECT_MAPPER = new DefaultObjectMapper(); private static final String DS_WIKI = "wiki"; @Test @@ -154,6 +166,56 @@ public void testFindPartitionsSpecWhenGivenIsRange() ); } + @Test + public void testStatusOnSegmentGranularityMismatch() + { + final GranularitySpec segmentGranularitySpec + = new UniformGranularitySpec(Granularities.HOUR, null, null, null); + final CompactionState segmentLastCompactionState = new CompactionState( + null, + null, + null, + null, + null, + segmentGranularitySpec.asMap(OBJECT_MAPPER) + ); + final DataSegment segment + = DataSegment.builder() + .dataSource(DS_WIKI) + .interval(Intervals.of("2013-01-01/PT1H")) + .size(100_000_000L) + .version("v1") + .lastCompactionState(segmentLastCompactionState) + .build(); + + final CompactionStatus status = CompactionStatus.compute( + SegmentsToCompact.from(Collections.singletonList(segment)), + createCompactionConfig(Granularities.DAY), + OBJECT_MAPPER + ); + + Assert.assertFalse(status.isComplete()); + Assert.assertEquals( + "segmentGranularity", + status.getReason() + ); + } + + private static DataSourceCompactionConfig createCompactionConfig( + Granularity segmentGranularity + ) + { + return new DataSourceCompactionConfig( + DS_WIKI, + null, null, null, null, + createTuningConfig( + new DimensionRangePartitionsSpec(1_000_000, null, Arrays.asList("countryName", "cityName"), false) + ), + new UserCompactionTaskGranularityConfig(segmentGranularity, null, null), + null, null, null, null, null, null + ); + } + private static DataSourceCompactionConfig createCompactionConfig( PartitionsSpec partitionsSpec ) diff --git a/server/src/test/java/org/apache/druid/server/coordinator/compact/DataSourceCompactibleSegmentIteratorTest.java b/server/src/test/java/org/apache/druid/server/coordinator/compact/DataSourceCompactibleSegmentIteratorTest.java index a2765ccfc09c..a7ebe6fb2aa1 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/compact/DataSourceCompactibleSegmentIteratorTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/compact/DataSourceCompactibleSegmentIteratorTest.java @@ -62,7 +62,7 @@ public void testAddSkipIntervalFromLatestAndSort() Intervals.of("2018-12-24/2018-12-25"), Intervals.of("2018-12-29/2019-01-01") ); - final List fullSkipIntervals = DataSourceCompactibleSegmentIterator.sortAndAddSkipIntervalFromLatest( + /*final List fullSkipIntervals = DataSourceCompactibleSegmentIterator.sortAndAddSkipIntervalFromLatest( DateTimes.of("2019-01-01"), new Period(72, 0, 0, 0), null, @@ -70,8 +70,8 @@ public void testAddSkipIntervalFromLatestAndSort() Intervals.of("2018-12-30/2018-12-31"), Intervals.of("2018-12-24/2018-12-25") ) - ); + );*/ - Assert.assertEquals(expectedIntervals, fullSkipIntervals); + //Assert.assertEquals(expectedIntervals, fullSkipIntervals); } } diff --git a/server/src/test/java/org/apache/druid/server/http/CoordinatorCompactionConfigsResourceTest.java b/server/src/test/java/org/apache/druid/server/http/CoordinatorCompactionConfigsResourceTest.java index 2ecbd62704c1..9bf88fc83787 100644 --- a/server/src/test/java/org/apache/druid/server/http/CoordinatorCompactionConfigsResourceTest.java +++ b/server/src/test/java/org/apache/druid/server/http/CoordinatorCompactionConfigsResourceTest.java @@ -585,7 +585,7 @@ public void testSimulateCompactionDynamicConfig() coordinatorCompactionConfigsResource = new CoordinatorCompactionConfigsResource( null, - new CompactionDutySimulator(metadataManager, OBJECT_MAPPER), + new CompactionDutySimulator(null, metadataManager, OBJECT_MAPPER), null ); Response response = coordinatorCompactionConfigsResource.simulateCompactionDynamicConfig( @@ -598,17 +598,17 @@ public void testSimulateCompactionDynamicConfig() Assert.assertEquals( Arrays.asList( Arrays.asList("dataSource", "interval", "numSegments", "bytes", "maxTaskSlots", "reasonToCompact"), - Arrays.asList("wiki", Intervals.of("2013-01-09/P1D"), 10, 1_000_000_000L, 1, ""), - Arrays.asList("wiki", Intervals.of("2013-01-08/P1D"), 10, 1_000_000_000L, 1, ""), - Arrays.asList("wiki", Intervals.of("2013-01-07/P1D"), 10, 1_000_000_000L, 1, ""), - Arrays.asList("wiki", Intervals.of("2013-01-06/P1D"), 10, 1_000_000_000L, 1, ""), - Arrays.asList("wiki", Intervals.of("2013-01-05/P1D"), 10, 1_000_000_000L, 1, ""), - Arrays.asList("wiki", Intervals.of("2013-01-04/P1D"), 10, 1_000_000_000L, 1, ""), - Arrays.asList("wiki", Intervals.of("2013-01-03/P1D"), 10, 1_000_000_000L, 1, ""), - Arrays.asList("wiki", Intervals.of("2013-01-02/P1D"), 10, 1_000_000_000L, 1, ""), - Arrays.asList("wiki", Intervals.of("2013-01-01/P1D"), 10, 1_000_000_000L, 1, "") + Arrays.asList("wiki", Intervals.of("2013-01-09/P1D"), 10, 1_000_000_000L, 1, "not compacted yet"), + Arrays.asList("wiki", Intervals.of("2013-01-08/P1D"), 10, 1_000_000_000L, 1, "not compacted yet"), + Arrays.asList("wiki", Intervals.of("2013-01-07/P1D"), 10, 1_000_000_000L, 1, "not compacted yet"), + Arrays.asList("wiki", Intervals.of("2013-01-06/P1D"), 10, 1_000_000_000L, 1, "not compacted yet"), + Arrays.asList("wiki", Intervals.of("2013-01-05/P1D"), 10, 1_000_000_000L, 1, "not compacted yet"), + Arrays.asList("wiki", Intervals.of("2013-01-04/P1D"), 10, 1_000_000_000L, 1, "not compacted yet"), + Arrays.asList("wiki", Intervals.of("2013-01-03/P1D"), 10, 1_000_000_000L, 1, "not compacted yet"), + Arrays.asList("wiki", Intervals.of("2013-01-02/P1D"), 10, 1_000_000_000L, 1, "not compacted yet"), + Arrays.asList("wiki", Intervals.of("2013-01-01/P1D"), 10, 1_000_000_000L, 1, "not compacted yet") ), - simulateResult.getSubmittedTasks() + simulateResult.getIntervalsToCompact() ); } From 23899b883cb8c4b1bdfda87765233e0bd07c88b6 Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Tue, 30 Jul 2024 14:03:42 +0530 Subject: [PATCH 07/26] Use compaction simulator on both coordinator and overlord --- .../NewestSegmentFirstPolicyBenchmark.java | 8 +- .../indexing/compact/LocalOverlordClient.java | 2 +- ....java => OverlordCompactionScheduler.java} | 62 ++++++++----- .../indexing/overlord/DruidOverlord.java | 4 +- .../http/OverlordCompactionResource.java | 29 +++++-- .../indexing/overlord/http/OverlordTest.java | 4 +- .../BaseSegmentSearchPolicy.java | 2 +- .../CompactionRunSimulator.java} | 39 ++++----- .../compaction}/CompactionScheduler.java | 11 ++- .../CompactionSegmentIterator.java | 2 +- .../CompactionSegmentSearchPolicy.java | 2 +- .../CompactionSimulateResult.java | 2 +- .../CompactionStatistics.java | 2 +- .../CompactionStatus.java | 2 +- .../CompactionStatusTracker.java | 28 +++--- .../DataSourceCompactibleSegmentIterator.java | 6 +- .../NewestSegmentFirstPolicy.java | 2 +- ...riorityBasedCompactionSegmentIterator.java | 2 +- .../SegmentsToCompact.java | 2 +- .../SmallestSegmentFirstPolicy.java | 2 +- .../coordinator/AutoCompactionSnapshot.java | 2 +- .../CoordinatorCompactionConfig.java | 4 +- .../server/coordinator/DruidCoordinator.java | 41 +++++++-- .../coordinator/duty/CompactSegments.java | 8 +- .../http/CompactionConfigUpdateRequest.java | 2 +- .../druid/server/http/CompactionResource.java | 17 +++- .../CoordinatorCompactionConfigsResource.java | 14 --- .../CompactionRunSimulatorTest.java | 87 +++++++++++++++++++ .../CompactionStatusTest.java | 2 +- .../CompactionStatusTrackerTest.java | 2 +- ...aSourceCompactibleSegmentIteratorTest.java | 2 +- .../NewestSegmentFirstPolicyTest.java | 2 +- .../AutoCompactionSnapshotTest.java | 2 +- .../coordinator/DruidCoordinatorTest.java | 13 +-- .../coordinator/duty/CompactSegmentsTest.java | 2 +- .../CoordinatorSimulationBuilder.java | 3 +- .../server/http/CompactionResourceTest.java | 8 +- ...rdinatorCompactionConfigsResourceTest.java | 81 +++-------------- .../org/apache/druid/cli/CliCoordinator.java | 4 - .../org/apache/druid/cli/CliOverlord.java | 13 +-- 40 files changed, 293 insertions(+), 229 deletions(-) rename indexing-service/src/main/java/org/apache/druid/indexing/compact/{CompactionSchedulerImpl.java => OverlordCompactionScheduler.java} (82%) rename server/src/main/java/org/apache/druid/server/{coordinator/compact => compaction}/BaseSegmentSearchPolicy.java (97%) rename server/src/main/java/org/apache/druid/server/{coordinator/compact/CompactionDutySimulator.java => compaction/CompactionRunSimulator.java} (88%) rename {indexing-service/src/main/java/org/apache/druid/indexing/compact => server/src/main/java/org/apache/druid/server/compaction}/CompactionScheduler.java (80%) rename server/src/main/java/org/apache/druid/server/{coordinator/compact => compaction}/CompactionSegmentIterator.java (97%) rename server/src/main/java/org/apache/druid/server/{coordinator/compact => compaction}/CompactionSegmentSearchPolicy.java (97%) rename server/src/main/java/org/apache/druid/server/{coordinator/compact => compaction}/CompactionSimulateResult.java (96%) rename server/src/main/java/org/apache/druid/server/{coordinator/compact => compaction}/CompactionStatistics.java (96%) rename server/src/main/java/org/apache/druid/server/{coordinator/compact => compaction}/CompactionStatus.java (99%) rename server/src/main/java/org/apache/druid/server/{coordinator/compact => compaction}/CompactionStatusTracker.java (93%) rename server/src/main/java/org/apache/druid/server/{coordinator/compact => compaction}/DataSourceCompactibleSegmentIterator.java (99%) rename server/src/main/java/org/apache/druid/server/{coordinator/compact => compaction}/NewestSegmentFirstPolicy.java (96%) rename server/src/main/java/org/apache/druid/server/{coordinator/compact => compaction}/PriorityBasedCompactionSegmentIterator.java (98%) rename server/src/main/java/org/apache/druid/server/{coordinator/compact => compaction}/SegmentsToCompact.java (98%) rename server/src/main/java/org/apache/druid/server/{coordinator/compact => compaction}/SmallestSegmentFirstPolicy.java (96%) create mode 100644 server/src/test/java/org/apache/druid/server/compaction/CompactionRunSimulatorTest.java rename server/src/test/java/org/apache/druid/server/{coordinator/compact => compaction}/CompactionStatusTest.java (99%) rename server/src/test/java/org/apache/druid/server/{coordinator/compact => compaction}/CompactionStatusTrackerTest.java (99%) rename server/src/test/java/org/apache/druid/server/{coordinator/compact => compaction}/DataSourceCompactibleSegmentIteratorTest.java (97%) rename server/src/test/java/org/apache/druid/server/{coordinator/compact => compaction}/NewestSegmentFirstPolicyTest.java (99%) diff --git a/benchmarks/src/test/java/org/apache/druid/server/coordinator/NewestSegmentFirstPolicyBenchmark.java b/benchmarks/src/test/java/org/apache/druid/server/coordinator/NewestSegmentFirstPolicyBenchmark.java index 2e6879d01086..a8fae1c97474 100644 --- a/benchmarks/src/test/java/org/apache/druid/server/coordinator/NewestSegmentFirstPolicyBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/server/coordinator/NewestSegmentFirstPolicyBenchmark.java @@ -24,10 +24,10 @@ import org.apache.druid.client.DataSourcesSnapshot; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.DateTimes; -import org.apache.druid.server.coordinator.compact.CompactionSegmentIterator; -import org.apache.druid.server.coordinator.compact.CompactionSegmentSearchPolicy; -import org.apache.druid.server.coordinator.compact.CompactionStatusTracker; -import org.apache.druid.server.coordinator.compact.NewestSegmentFirstPolicy; +import org.apache.druid.server.compaction.CompactionSegmentIterator; +import org.apache.druid.server.compaction.CompactionSegmentSearchPolicy; +import org.apache.druid.server.compaction.CompactionStatusTracker; +import org.apache.druid.server.compaction.NewestSegmentFirstPolicy; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.SegmentTimeline; import org.apache.druid.timeline.partition.NumberedShardSpec; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/LocalOverlordClient.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/LocalOverlordClient.java index b9358fd86f86..c3f917c4d365 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/compact/LocalOverlordClient.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/LocalOverlordClient.java @@ -56,7 +56,7 @@ import java.util.function.Supplier; /** - * Dummy Overlord client used by the {@link CompactionSchedulerImpl} to fetch + * Dummy Overlord client used by the {@link OverlordCompactionScheduler} to fetch * task related info. This client simply redirects all queries to the * {@link TaskQueryTool} and all updates to the {@link TaskQueue}. */ diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionSchedulerImpl.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/OverlordCompactionScheduler.java similarity index 82% rename from indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionSchedulerImpl.java rename to indexing-service/src/main/java/org/apache/druid/indexing/compact/OverlordCompactionScheduler.java index cca74e706a0b..5c88434f2598 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionSchedulerImpl.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/OverlordCompactionScheduler.java @@ -22,7 +22,6 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Optional; import com.google.inject.Inject; -import org.apache.druid.client.DataSourcesSnapshot; import org.apache.druid.common.config.JacksonConfigManager; import org.apache.druid.indexer.TaskLocation; import org.apache.druid.indexer.TaskStatus; @@ -36,15 +35,20 @@ import org.apache.druid.java.util.emitter.service.ServiceEmitter; import org.apache.druid.java.util.emitter.service.ServiceMetricEvent; import org.apache.druid.metadata.SegmentsMetadataManager; +import org.apache.druid.server.compaction.CompactionRunSimulator; +import org.apache.druid.server.compaction.CompactionScheduler; +import org.apache.druid.server.compaction.CompactionSimulateResult; +import org.apache.druid.server.compaction.CompactionStatusTracker; import org.apache.druid.server.coordinator.AutoCompactionSnapshot; import org.apache.druid.server.coordinator.CompactionSchedulerConfig; import org.apache.druid.server.coordinator.CoordinatorCompactionConfig; import org.apache.druid.server.coordinator.CoordinatorOverlordServiceConfig; -import org.apache.druid.server.coordinator.compact.CompactionStatusTracker; import org.apache.druid.server.coordinator.duty.CompactSegments; import org.apache.druid.server.coordinator.stats.CoordinatorRunStats; import org.apache.druid.server.coordinator.stats.CoordinatorStat; import org.apache.druid.server.coordinator.stats.Dimension; +import org.apache.druid.server.http.CompactionConfigUpdateRequest; +import org.apache.druid.timeline.SegmentTimeline; import org.joda.time.Duration; import java.util.Map; @@ -54,25 +58,25 @@ /** * TODO: pending items - * - [x] make config static. - * - [?] bind scheduler only when enabled - * - [x] route compaction status API to overlord if scheduler is enabled - * - [x] skip run on coordinator if scheduler is enabled - * - [x] task state listener - * - [x] handle success and failure inside CompactionStatusTracker - * - [x] make policy serializable - * - [x] handle priority datasource in policy * - [x] add another policy - smallestSegmentFirst - * - [ ] add policy auto - * - [ ] handle skipping turns if no more intervals in datasource - * - [x] enable segments polling if overlord is standalone + * - [ ] come up with better policy names + * - [?] poll status of recently completed tasks in CompactSegments duty + * - [?] BasePolicy.shouldCompact(): locked and skip offset intervals will always be skipped + * - [ ] finalize logic of skipping turns of successful and failed + * - [ ] compaction status should have last compacted time + * - [ ] compaction status should have num uncompacted segments, bytes * - [ ] test on cluster - standalone, coordinator-overlord * - [ ] unit tests * - [ ] integration tests */ -public class CompactionSchedulerImpl implements CompactionScheduler + +/** + * Compaction Scheduler that runs on the Overlord if + * {@code druid.compaction.scheduler.enabled=true}. + */ +public class OverlordCompactionScheduler implements CompactionScheduler { - private static final Logger log = new Logger(CompactionSchedulerImpl.class); + private static final Logger log = new Logger(OverlordCompactionScheduler.class); private final TaskMaster taskMaster; private final JacksonConfigManager configManager; @@ -100,10 +104,9 @@ public class CompactionSchedulerImpl implements CompactionScheduler private final CompactionSchedulerConfig schedulerConfig; @Inject - public CompactionSchedulerImpl( + public OverlordCompactionScheduler( TaskMaster taskMaster, TaskQueryTool taskQueryTool, - CompactionStatusTracker statusTracker, SegmentsMetadataManager segmentManager, JacksonConfigManager configManager, CompactionSchedulerConfig schedulerConfig, @@ -116,7 +119,7 @@ public CompactionSchedulerImpl( this.taskMaster = taskMaster; this.configManager = configManager; this.segmentManager = segmentManager; - this.statusTracker = statusTracker; + this.statusTracker = new CompactionStatusTracker(objectMapper); this.emitter = emitter; this.schedulerConfig = schedulerConfig; this.executor = executorFactory.create(1, "CompactionScheduler-%s"); @@ -210,7 +213,7 @@ private synchronized void cleanupState() } } - public boolean isEnabled() + private boolean isEnabled() { return schedulerConfig.isEnabled(); } @@ -234,13 +237,11 @@ private synchronized void processCompactionQueue( CoordinatorCompactionConfig currentConfig ) { - DataSourcesSnapshot dataSourcesSnapshot - = segmentManager.getSnapshotOfDataSourcesWithAllUsedSegments(); final CoordinatorRunStats stats = new CoordinatorRunStats(); duty.run( currentConfig, - dataSourcesSnapshot.getUsedSegmentsTimelinesPerDataSource(), + getCurrentDatasourceTimelines(), stats ); @@ -275,6 +276,12 @@ private CoordinatorCompactionConfig getLatestConfig() ).get(); } + private Map getCurrentDatasourceTimelines() + { + return segmentManager.getSnapshotOfDataSourcesWithAllUsedSegments() + .getUsedSegmentsTimelinesPerDataSource(); + } + @Override public AutoCompactionSnapshot getCompactionSnapshot(String dataSource) { @@ -282,7 +289,7 @@ public AutoCompactionSnapshot getCompactionSnapshot(String dataSource) } @Override - public Long getSegmentBytesYetToBeCompacted(String dataSource) + public Long getTotalSizeOfSegmentsAwaitingCompaction(String dataSource) { return duty.getTotalSizeOfSegmentsAwaitingCompaction(dataSource); } @@ -293,4 +300,13 @@ public Map getAllCompactionSnapshots() return duty.getAutoCompactionSnapshot(); } + @Override + public CompactionSimulateResult simulateRunWithConfigUpdate(CompactionConfigUpdateRequest updateRequest) + { + return new CompactionRunSimulator( + statusTracker, + getLatestConfig(), + getCurrentDatasourceTimelines() + ).simulateRunWithConfigUpdate(updateRequest); + } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/DruidOverlord.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/DruidOverlord.java index 90ac36a96791..eb4067b05237 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/DruidOverlord.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/DruidOverlord.java @@ -29,7 +29,7 @@ import org.apache.druid.indexing.common.actions.SegmentAllocationQueue; import org.apache.druid.indexing.common.actions.TaskActionClientFactory; import org.apache.druid.indexing.common.task.TaskContextEnricher; -import org.apache.druid.indexing.compact.CompactionScheduler; +import org.apache.druid.indexing.compact.OverlordCompactionScheduler; import org.apache.druid.indexing.overlord.config.DefaultTaskConfig; import org.apache.druid.indexing.overlord.config.TaskLockConfig; import org.apache.druid.indexing.overlord.config.TaskQueueConfig; @@ -88,7 +88,7 @@ public DruidOverlord( final OverlordDutyExecutor overlordDutyExecutor, @IndexingService final DruidLeaderSelector overlordLeaderSelector, final SegmentAllocationQueue segmentAllocationQueue, - final CompactionScheduler compactionScheduler, + final OverlordCompactionScheduler compactionScheduler, final ObjectMapper mapper, final TaskContextEnricher taskContextEnricher ) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordCompactionResource.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordCompactionResource.java index 47477b4357c9..089d085e4b20 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordCompactionResource.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordCompactionResource.java @@ -21,12 +21,14 @@ import com.google.inject.Inject; import com.sun.jersey.spi.container.ResourceFilters; -import org.apache.druid.indexing.compact.CompactionScheduler; -import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.indexing.compact.OverlordCompactionScheduler; import org.apache.druid.server.coordinator.AutoCompactionSnapshot; +import org.apache.druid.server.http.CompactionConfigUpdateRequest; import org.apache.druid.server.http.security.StateResourceFilter; +import javax.ws.rs.Consumes; import javax.ws.rs.GET; +import javax.ws.rs.POST; import javax.ws.rs.Path; import javax.ws.rs.Produces; import javax.ws.rs.QueryParam; @@ -37,22 +39,19 @@ /** * Contains the same logic as {@code CompactionResource} but the APIs are served - * by {@link CompactionScheduler} instead of {@code DruidCoordinator}. + * by {@link OverlordCompactionScheduler} instead of {@code DruidCoordinator}. */ @Path("/druid/indexer/v1/compaction") public class OverlordCompactionResource { - private static final Logger log = new Logger(OverlordCompactionResource.class); - - private final CompactionScheduler scheduler; + private final OverlordCompactionScheduler scheduler; @Inject public OverlordCompactionResource( - CompactionScheduler scheduler + OverlordCompactionScheduler scheduler ) { this.scheduler = scheduler; - log.info("Creating the new overlord compaction resource."); } @GET @@ -63,7 +62,7 @@ public Response getCompactionProgress( @QueryParam("dataSource") String dataSource ) { - final Long notCompactedSegmentSizeBytes = scheduler.getSegmentBytesYetToBeCompacted(dataSource); + final Long notCompactedSegmentSizeBytes = scheduler.getTotalSizeOfSegmentsAwaitingCompaction(dataSource); if (notCompactedSegmentSizeBytes == null) { return Response.status(Response.Status.NOT_FOUND) .entity(Collections.singletonMap("error", "Unknown DataSource")) @@ -96,4 +95,16 @@ public Response getCompactionSnapshotForDataSource( } return Response.ok(Collections.singletonMap("latestStatus", snapshots)).build(); } + + @POST + @Path("/simulate") + @Consumes(MediaType.APPLICATION_JSON) + public Response simulateClusterCompactionConfigUpdate( + CompactionConfigUpdateRequest updatePayload + ) + { + return Response.ok().entity( + scheduler.simulateRunWithConfigUpdate(updatePayload) + ).build(); + } } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/http/OverlordTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/http/OverlordTest.java index f1a8b65a509f..6bc6643dd11e 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/http/OverlordTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/http/OverlordTest.java @@ -47,7 +47,7 @@ import org.apache.druid.indexing.common.task.NoopTask; import org.apache.druid.indexing.common.task.NoopTaskContextEnricher; import org.apache.druid.indexing.common.task.Task; -import org.apache.druid.indexing.compact.CompactionScheduler; +import org.apache.druid.indexing.compact.OverlordCompactionScheduler; import org.apache.druid.indexing.overlord.DruidOverlord; import org.apache.druid.indexing.overlord.HeapMemoryTaskStorage; import org.apache.druid.indexing.overlord.IndexerMetadataStorageAdapter; @@ -258,7 +258,7 @@ public MockTaskRunner get() EasyMock.createNiceMock(OverlordDutyExecutor.class), new TestDruidLeaderSelector(), EasyMock.createNiceMock(SegmentAllocationQueue.class), - EasyMock.createNiceMock(CompactionScheduler.class), + EasyMock.createNiceMock(OverlordCompactionScheduler.class), new DefaultObjectMapper(), new NoopTaskContextEnricher() ); diff --git a/server/src/main/java/org/apache/druid/server/coordinator/compact/BaseSegmentSearchPolicy.java b/server/src/main/java/org/apache/druid/server/compaction/BaseSegmentSearchPolicy.java similarity index 97% rename from server/src/main/java/org/apache/druid/server/coordinator/compact/BaseSegmentSearchPolicy.java rename to server/src/main/java/org/apache/druid/server/compaction/BaseSegmentSearchPolicy.java index 0e74f4299c7e..838227d6b2dc 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/compact/BaseSegmentSearchPolicy.java +++ b/server/src/main/java/org/apache/druid/server/compaction/BaseSegmentSearchPolicy.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.druid.server.coordinator.compact; +package org.apache.druid.server.compaction; import com.fasterxml.jackson.annotation.JsonProperty; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; diff --git a/server/src/main/java/org/apache/druid/server/coordinator/compact/CompactionDutySimulator.java b/server/src/main/java/org/apache/druid/server/compaction/CompactionRunSimulator.java similarity index 88% rename from server/src/main/java/org/apache/druid/server/coordinator/compact/CompactionDutySimulator.java rename to server/src/main/java/org/apache/druid/server/compaction/CompactionRunSimulator.java index 14cf27456c3e..d4d999e9c0c5 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/compact/CompactionDutySimulator.java +++ b/server/src/main/java/org/apache/druid/server/compaction/CompactionRunSimulator.java @@ -17,12 +17,10 @@ * under the License. */ -package org.apache.druid.server.coordinator.compact; +package org.apache.druid.server.compaction; -import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; -import com.google.inject.Inject; import org.apache.druid.client.indexing.ClientCompactionTaskQuery; import org.apache.druid.client.indexing.IndexingTotalWorkerCapacityInfo; import org.apache.druid.client.indexing.IndexingWorkerInfo; @@ -39,7 +37,6 @@ import org.apache.druid.rpc.indexing.OverlordClient; import org.apache.druid.server.coordinator.CoordinatorCompactionConfig; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; -import org.apache.druid.server.coordinator.MetadataManager; import org.apache.druid.server.coordinator.duty.CompactSegments; import org.apache.druid.server.coordinator.stats.CoordinatorRunStats; import org.apache.druid.server.http.CompactionConfigUpdateRequest; @@ -56,26 +53,25 @@ import java.util.Set; /** - * Simulates runs of auto-compaction duty to verify the expected list of + * Simulates runs of auto-compaction duty to obtain the expected list of * compaction tasks that would be submitted by the actual compaction duty. */ -public class CompactionDutySimulator +public class CompactionRunSimulator { - private final ObjectMapper objectMapper; - private final MetadataManager metadataManager; + private final CoordinatorCompactionConfig compactionConfig; private final CompactionStatusTracker statusTracker; + private final Map datasourceTimelines; private final OverlordClient emptyOverlordClient = new EmptyOverlordClient(); - @Inject - public CompactionDutySimulator( + public CompactionRunSimulator( CompactionStatusTracker statusTracker, - MetadataManager metadataManager, - ObjectMapper objectMapper + CoordinatorCompactionConfig compactionConfig, + Map datasourceTimelines ) { this.statusTracker = statusTracker; - this.objectMapper = objectMapper; - this.metadataManager = metadataManager; + this.datasourceTimelines = datasourceTimelines; + this.compactionConfig = compactionConfig; } /** @@ -93,18 +89,13 @@ public CompactionSimulateResult simulateRunWithConfigUpdate( updateRequest.getCompactionEngine(), updateRequest.getCompactionPolicy() ); - final CoordinatorCompactionConfig configWithUnlimitedTaskSlots = CoordinatorCompactionConfig.from( - metadataManager.configs().getCurrentCompactionConfig(), - updateWithUnlimitedSlots - ); - final Map datasourceTimelines - = metadataManager.segments() - .getSnapshotOfDataSourcesWithAllUsedSegments() - .getUsedSegmentsTimelinesPerDataSource(); + final CoordinatorCompactionConfig configWithUnlimitedTaskSlots + = CoordinatorCompactionConfig.from(compactionConfig, updateWithUnlimitedSlots); final List> tableOfCompactibleIntervals = new ArrayList<>(); final List> tableOfSkippedIntervals = new ArrayList<>(); - final CompactionStatusTracker simulationStatusTracker = new CompactionStatusTracker(objectMapper) { + final CompactionStatusTracker simulationStatusTracker = new CompactionStatusTracker(null) + { @Override public CompactionStatus computeCompactionStatus( SegmentsToCompact candidateSegments, @@ -115,7 +106,7 @@ public CompactionStatus computeCompactionStatus( } @Override - public void onIntervalSkipped(SegmentsToCompact candidateSegments, CompactionStatus status) + public void onSegmentsSkipped(SegmentsToCompact candidateSegments, CompactionStatus status) { // Add a row for each skipped interval tableOfSkippedIntervals.add( diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionScheduler.java b/server/src/main/java/org/apache/druid/server/compaction/CompactionScheduler.java similarity index 80% rename from indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionScheduler.java rename to server/src/main/java/org/apache/druid/server/compaction/CompactionScheduler.java index 85db08aa3b34..38640974edf7 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionScheduler.java +++ b/server/src/main/java/org/apache/druid/server/compaction/CompactionScheduler.java @@ -17,16 +17,13 @@ * under the License. */ -package org.apache.druid.indexing.compact; +package org.apache.druid.server.compaction; import org.apache.druid.server.coordinator.AutoCompactionSnapshot; +import org.apache.druid.server.http.CompactionConfigUpdateRequest; import java.util.Map; -/** - * Compaction Scheduler that runs on the Overlord if - * {@code druid.compaction.scheduler.enabled=true}. - */ public interface CompactionScheduler { void becomeLeader(); @@ -37,6 +34,8 @@ public interface CompactionScheduler AutoCompactionSnapshot getCompactionSnapshot(String dataSource); - Long getSegmentBytesYetToBeCompacted(String dataSource); + Long getTotalSizeOfSegmentsAwaitingCompaction(String dataSource); + + CompactionSimulateResult simulateRunWithConfigUpdate(CompactionConfigUpdateRequest updateRequest); } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/compact/CompactionSegmentIterator.java b/server/src/main/java/org/apache/druid/server/compaction/CompactionSegmentIterator.java similarity index 97% rename from server/src/main/java/org/apache/druid/server/coordinator/compact/CompactionSegmentIterator.java rename to server/src/main/java/org/apache/druid/server/compaction/CompactionSegmentIterator.java index bab7ca8f92fb..8e01d054b19e 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/compact/CompactionSegmentIterator.java +++ b/server/src/main/java/org/apache/druid/server/compaction/CompactionSegmentIterator.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.druid.server.coordinator.compact; +package org.apache.druid.server.compaction; import org.apache.druid.timeline.DataSegment; diff --git a/server/src/main/java/org/apache/druid/server/coordinator/compact/CompactionSegmentSearchPolicy.java b/server/src/main/java/org/apache/druid/server/compaction/CompactionSegmentSearchPolicy.java similarity index 97% rename from server/src/main/java/org/apache/druid/server/coordinator/compact/CompactionSegmentSearchPolicy.java rename to server/src/main/java/org/apache/druid/server/compaction/CompactionSegmentSearchPolicy.java index 5d71f8f6edd1..46cacf844df2 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/compact/CompactionSegmentSearchPolicy.java +++ b/server/src/main/java/org/apache/druid/server/compaction/CompactionSegmentSearchPolicy.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.druid.server.coordinator.compact; +package org.apache.druid.server.compaction; import com.fasterxml.jackson.annotation.JsonSubTypes; import com.fasterxml.jackson.annotation.JsonTypeInfo; diff --git a/server/src/main/java/org/apache/druid/server/coordinator/compact/CompactionSimulateResult.java b/server/src/main/java/org/apache/druid/server/compaction/CompactionSimulateResult.java similarity index 96% rename from server/src/main/java/org/apache/druid/server/coordinator/compact/CompactionSimulateResult.java rename to server/src/main/java/org/apache/druid/server/compaction/CompactionSimulateResult.java index c21a8e227020..9d840ae5eb16 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/compact/CompactionSimulateResult.java +++ b/server/src/main/java/org/apache/druid/server/compaction/CompactionSimulateResult.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.druid.server.coordinator.compact; +package org.apache.druid.server.compaction; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; diff --git a/server/src/main/java/org/apache/druid/server/coordinator/compact/CompactionStatistics.java b/server/src/main/java/org/apache/druid/server/compaction/CompactionStatistics.java similarity index 96% rename from server/src/main/java/org/apache/druid/server/coordinator/compact/CompactionStatistics.java rename to server/src/main/java/org/apache/druid/server/compaction/CompactionStatistics.java index 6997dec47c01..23f1b7fe9ef5 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/compact/CompactionStatistics.java +++ b/server/src/main/java/org/apache/druid/server/compaction/CompactionStatistics.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.druid.server.coordinator.compact; +package org.apache.druid.server.compaction; /** * Used to track statistics for segments in different states of compaction. diff --git a/server/src/main/java/org/apache/druid/server/coordinator/compact/CompactionStatus.java b/server/src/main/java/org/apache/druid/server/compaction/CompactionStatus.java similarity index 99% rename from server/src/main/java/org/apache/druid/server/coordinator/compact/CompactionStatus.java rename to server/src/main/java/org/apache/druid/server/compaction/CompactionStatus.java index 807c8e534b42..a6078d880af5 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/compact/CompactionStatus.java +++ b/server/src/main/java/org/apache/druid/server/compaction/CompactionStatus.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.druid.server.coordinator.compact; +package org.apache.druid.server.compaction; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Preconditions; diff --git a/server/src/main/java/org/apache/druid/server/coordinator/compact/CompactionStatusTracker.java b/server/src/main/java/org/apache/druid/server/compaction/CompactionStatusTracker.java similarity index 93% rename from server/src/main/java/org/apache/druid/server/coordinator/compact/CompactionStatusTracker.java rename to server/src/main/java/org/apache/druid/server/compaction/CompactionStatusTracker.java index 079bfd448c60..494c977ea349 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/compact/CompactionStatusTracker.java +++ b/server/src/main/java/org/apache/druid/server/compaction/CompactionStatusTracker.java @@ -17,10 +17,9 @@ * under the License. */ -package org.apache.druid.server.coordinator.compact; +package org.apache.druid.server.compaction; import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.inject.Inject; import org.apache.druid.client.indexing.ClientCompactionTaskQuery; import org.apache.druid.indexer.TaskStatus; import org.apache.druid.java.util.common.logger.Logger; @@ -61,10 +60,7 @@ public class CompactionStatusTracker private final Map datasourceStatuses = new HashMap<>(); private final Map submittedTaskIdToSegments = new HashMap<>(); - @Inject - public CompactionStatusTracker( - ObjectMapper objectMapper - ) + public CompactionStatusTracker(ObjectMapper objectMapper) { this.objectMapper = objectMapper; } @@ -89,10 +85,10 @@ public CompactionStatus computeCompactionStatus( final Interval compactionInterval = candidate.getUmbrellaInterval(); - final IntervalStatus intervalStatus - = datasourceStatuses.getOrDefault(config.getDataSource(), DatasourceStatus.EMPTY) - .getIntervalStatuses() - .get(compactionInterval); + final IntervalStatus intervalStatus = datasourceStatuses + .getOrDefault(config.getDataSource(), DatasourceStatus.EMPTY) + .intervalStatus + .get(compactionInterval); if (intervalStatus == null) { return compactionStatus; @@ -131,7 +127,12 @@ public void onCompactionConfigUpdated(CoordinatorCompactionConfig compactionConf }); } - public void onIntervalSkipped( + public void onSegmentsCompacted(SegmentsToCompact candidateSegments) + { + // do nothing + } + + public void onSegmentsSkipped( SegmentsToCompact candidateSegments, CompactionStatus status ) @@ -223,11 +224,6 @@ void handleSubmittedTask(SegmentsToCompact candidateSegments) readyIntervals.forEach(intervalStatus::remove); } - - Map getIntervalStatuses() - { - return intervalStatus; - } } private static class IntervalStatus diff --git a/server/src/main/java/org/apache/druid/server/coordinator/compact/DataSourceCompactibleSegmentIterator.java b/server/src/main/java/org/apache/druid/server/compaction/DataSourceCompactibleSegmentIterator.java similarity index 99% rename from server/src/main/java/org/apache/druid/server/coordinator/compact/DataSourceCompactibleSegmentIterator.java rename to server/src/main/java/org/apache/druid/server/compaction/DataSourceCompactibleSegmentIterator.java index 17ec6115b8d6..98a83976c6ff 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/compact/DataSourceCompactibleSegmentIterator.java +++ b/server/src/main/java/org/apache/druid/server/compaction/DataSourceCompactibleSegmentIterator.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.druid.server.coordinator.compact; +package org.apache.druid.server.compaction; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; @@ -323,7 +323,7 @@ private void findAndEnqueueSegmentsToCompact(CompactibleSegmentIterator compacti compactedSegmentStats.increment(candidates.getStats()); } else if (compactionStatus.isSkipped()) { skippedSegmentStats.increment(candidates.getStats()); - statusTracker.onIntervalSkipped(candidates, compactionStatus); + statusTracker.onSegmentsSkipped(candidates, compactionStatus); log.warn( "Skipping compaction for datasource[%s], interval[%s] due to reason[%s].", dataSource, interval, compactionStatus.getReason() @@ -390,7 +390,7 @@ private List findInitialSearchInterval( } else { reason = CompactionStatus.skipped("interval locked by another task"); } - statusTracker.onIntervalSkipped(candidates, reason); + statusTracker.onSegmentsSkipped(candidates, reason); } } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/compact/NewestSegmentFirstPolicy.java b/server/src/main/java/org/apache/druid/server/compaction/NewestSegmentFirstPolicy.java similarity index 96% rename from server/src/main/java/org/apache/druid/server/coordinator/compact/NewestSegmentFirstPolicy.java rename to server/src/main/java/org/apache/druid/server/compaction/NewestSegmentFirstPolicy.java index 0de7602e577a..3e2c341bfd7c 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/compact/NewestSegmentFirstPolicy.java +++ b/server/src/main/java/org/apache/druid/server/compaction/NewestSegmentFirstPolicy.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.druid.server.coordinator.compact; +package org.apache.druid.server.compaction; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; diff --git a/server/src/main/java/org/apache/druid/server/coordinator/compact/PriorityBasedCompactionSegmentIterator.java b/server/src/main/java/org/apache/druid/server/compaction/PriorityBasedCompactionSegmentIterator.java similarity index 98% rename from server/src/main/java/org/apache/druid/server/coordinator/compact/PriorityBasedCompactionSegmentIterator.java rename to server/src/main/java/org/apache/druid/server/compaction/PriorityBasedCompactionSegmentIterator.java index a8526b011722..e8838432bd86 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/compact/PriorityBasedCompactionSegmentIterator.java +++ b/server/src/main/java/org/apache/druid/server/compaction/PriorityBasedCompactionSegmentIterator.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.druid.server.coordinator.compact; +package org.apache.druid.server.compaction; import com.google.common.collect.Maps; import org.apache.druid.error.DruidException; diff --git a/server/src/main/java/org/apache/druid/server/coordinator/compact/SegmentsToCompact.java b/server/src/main/java/org/apache/druid/server/compaction/SegmentsToCompact.java similarity index 98% rename from server/src/main/java/org/apache/druid/server/coordinator/compact/SegmentsToCompact.java rename to server/src/main/java/org/apache/druid/server/compaction/SegmentsToCompact.java index c0c8d8508c59..d90d8fceb921 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/compact/SegmentsToCompact.java +++ b/server/src/main/java/org/apache/druid/server/compaction/SegmentsToCompact.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.druid.server.coordinator.compact; +package org.apache.druid.server.compaction; import org.apache.druid.error.InvalidInput; import org.apache.druid.java.util.common.JodaUtils; diff --git a/server/src/main/java/org/apache/druid/server/coordinator/compact/SmallestSegmentFirstPolicy.java b/server/src/main/java/org/apache/druid/server/compaction/SmallestSegmentFirstPolicy.java similarity index 96% rename from server/src/main/java/org/apache/druid/server/coordinator/compact/SmallestSegmentFirstPolicy.java rename to server/src/main/java/org/apache/druid/server/compaction/SmallestSegmentFirstPolicy.java index 1ab0f7d8e698..56a6d30cf9e5 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/compact/SmallestSegmentFirstPolicy.java +++ b/server/src/main/java/org/apache/druid/server/compaction/SmallestSegmentFirstPolicy.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.druid.server.coordinator.compact; +package org.apache.druid.server.compaction; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; diff --git a/server/src/main/java/org/apache/druid/server/coordinator/AutoCompactionSnapshot.java b/server/src/main/java/org/apache/druid/server/coordinator/AutoCompactionSnapshot.java index d52d4e9eba0d..ec82a2fbdb20 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/AutoCompactionSnapshot.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/AutoCompactionSnapshot.java @@ -22,7 +22,7 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import org.apache.druid.java.util.common.ISE; -import org.apache.druid.server.coordinator.compact.CompactionStatistics; +import org.apache.druid.server.compaction.CompactionStatistics; import javax.validation.constraints.NotNull; import java.util.Objects; diff --git a/server/src/main/java/org/apache/druid/server/coordinator/CoordinatorCompactionConfig.java b/server/src/main/java/org/apache/druid/server/coordinator/CoordinatorCompactionConfig.java index f686174e3c9c..21fc134d1a38 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/CoordinatorCompactionConfig.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/CoordinatorCompactionConfig.java @@ -24,8 +24,8 @@ import com.google.common.collect.ImmutableList; import org.apache.druid.common.config.Configs; import org.apache.druid.indexer.CompactionEngine; -import org.apache.druid.server.coordinator.compact.CompactionSegmentSearchPolicy; -import org.apache.druid.server.coordinator.compact.NewestSegmentFirstPolicy; +import org.apache.druid.server.compaction.CompactionSegmentSearchPolicy; +import org.apache.druid.server.compaction.NewestSegmentFirstPolicy; import org.apache.druid.server.http.CompactionConfigUpdateRequest; import javax.annotation.Nullable; diff --git a/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java b/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java index 4264fd9bf2ab..9c1586f8f9fe 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java @@ -19,6 +19,7 @@ package org.apache.druid.server.coordinator; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.ImmutableList; import com.google.common.collect.Ordering; @@ -54,8 +55,11 @@ import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; import org.apache.druid.segment.metadata.CoordinatorSegmentMetadataCache; import org.apache.druid.server.DruidNode; +import org.apache.druid.server.compaction.CompactionRunSimulator; +import org.apache.druid.server.compaction.CompactionScheduler; +import org.apache.druid.server.compaction.CompactionSimulateResult; +import org.apache.druid.server.compaction.CompactionStatusTracker; import org.apache.druid.server.coordinator.balancer.BalancerStrategyFactory; -import org.apache.druid.server.coordinator.compact.CompactionStatusTracker; import org.apache.druid.server.coordinator.config.CoordinatorKillConfigs; import org.apache.druid.server.coordinator.config.DruidCoordinatorConfig; import org.apache.druid.server.coordinator.config.KillUnusedSegmentsConfig; @@ -88,6 +92,7 @@ import org.apache.druid.server.coordinator.stats.Dimension; import org.apache.druid.server.coordinator.stats.RowKey; import org.apache.druid.server.coordinator.stats.Stats; +import org.apache.druid.server.http.CompactionConfigUpdateRequest; import org.apache.druid.server.lookup.cache.LookupCoordinatorManager; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.SegmentId; @@ -111,7 +116,7 @@ * */ @ManageLifecycle -public class DruidCoordinator +public class DruidCoordinator implements CompactionScheduler { /** * Orders newest segments (i.e. segments with most recent intervals) first. @@ -153,6 +158,7 @@ public class DruidCoordinator private final BalancerStrategyFactory balancerStrategyFactory; private final LookupCoordinatorManager lookupCoordinatorManager; private final DruidLeaderSelector coordLeaderSelector; + private final CompactionStatusTracker compactionStatusTracker; private final CompactSegments compactSegments; @Nullable private final CoordinatorSegmentMetadataCache coordinatorSegmentMetadataCache; @@ -204,7 +210,7 @@ public DruidCoordinator( @Nullable CoordinatorSegmentMetadataCache coordinatorSegmentMetadataCache, CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig, CompactionSchedulerConfig compactionSchedulerConfig, - CompactionStatusTracker compactionStatusTracker + ObjectMapper objectMapper ) { this.config = config; @@ -222,7 +228,8 @@ public DruidCoordinator( this.balancerStrategyFactory = config.getBalancerStrategyFactory(); this.lookupCoordinatorManager = lookupCoordinatorManager; this.coordLeaderSelector = coordLeaderSelector; - this.compactSegments = initializeCompactSegmentsDuty(compactionStatusTracker); + this.compactionStatusTracker = new CompactionStatusTracker(objectMapper); + this.compactSegments = initializeCompactSegmentsDuty(this.compactionStatusTracker); this.loadQueueManager = loadQueueManager; this.coordinatorSegmentMetadataCache = coordinatorSegmentMetadataCache; this.centralizedDatasourceSchemaConfig = centralizedDatasourceSchemaConfig; @@ -344,23 +351,38 @@ public Integer getReplicationFactor(SegmentId segmentId) return replicaCountsInCluster == null ? null : replicaCountsInCluster.required(); } + @Override @Nullable public Long getTotalSizeOfSegmentsAwaitingCompaction(String dataSource) { return compactSegments.getTotalSizeOfSegmentsAwaitingCompaction(dataSource); } + @Override @Nullable - public AutoCompactionSnapshot getAutoCompactionSnapshotForDataSource(String dataSource) + public AutoCompactionSnapshot getCompactionSnapshot(String dataSource) { return compactSegments.getAutoCompactionSnapshot(dataSource); } - public Map getAutoCompactionSnapshot() + @Override + public Map getAllCompactionSnapshots() { return compactSegments.getAutoCompactionSnapshot(); } + @Override + public CompactionSimulateResult simulateRunWithConfigUpdate(CompactionConfigUpdateRequest updateRequest) + { + return new CompactionRunSimulator( + compactionStatusTracker, + metadataManager.configs().getCurrentCompactionConfig(), + metadataManager.segments() + .getSnapshotOfDataSourcesWithAllUsedSegments() + .getUsedSegmentsTimelinesPerDataSource() + ).simulateRunWithConfigUpdate(updateRequest); + } + public String getCurrentLeader() { return coordLeaderSelector.getCurrentLeader(); @@ -435,7 +457,8 @@ private Map> computeUnderReplicated( } } - private void becomeLeader() + @Override + public void becomeLeader() { synchronized (lock) { if (!started) { @@ -527,7 +550,8 @@ private void becomeLeader() } } - private void stopBeingLeader() + @Override + public void stopBeingLeader() { synchronized (lock) { @@ -536,6 +560,7 @@ private void stopBeingLeader() if (coordinatorSegmentMetadataCache != null) { coordinatorSegmentMetadataCache.onLeaderStop(); } + compactionStatusTracker.reset(); taskMaster.onLeaderStop(); serviceAnnouncer.unannounce(self); lookupCoordinatorManager.stop(); 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 bed743ea07fc..582ef70245e0 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 @@ -46,14 +46,14 @@ import org.apache.druid.metadata.LockFilterPolicy; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.rpc.indexing.OverlordClient; +import org.apache.druid.server.compaction.CompactionSegmentIterator; +import org.apache.druid.server.compaction.CompactionSegmentSearchPolicy; +import org.apache.druid.server.compaction.CompactionStatusTracker; +import org.apache.druid.server.compaction.SegmentsToCompact; import org.apache.druid.server.coordinator.AutoCompactionSnapshot; import org.apache.druid.server.coordinator.CoordinatorCompactionConfig; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; import org.apache.druid.server.coordinator.DruidCoordinatorRuntimeParams; -import org.apache.druid.server.coordinator.compact.CompactionSegmentIterator; -import org.apache.druid.server.coordinator.compact.CompactionSegmentSearchPolicy; -import org.apache.druid.server.coordinator.compact.CompactionStatusTracker; -import org.apache.druid.server.coordinator.compact.SegmentsToCompact; import org.apache.druid.server.coordinator.stats.CoordinatorRunStats; import org.apache.druid.server.coordinator.stats.Dimension; import org.apache.druid.server.coordinator.stats.RowKey; diff --git a/server/src/main/java/org/apache/druid/server/http/CompactionConfigUpdateRequest.java b/server/src/main/java/org/apache/druid/server/http/CompactionConfigUpdateRequest.java index 5a0a716472f5..d0d6b9744fe5 100644 --- a/server/src/main/java/org/apache/druid/server/http/CompactionConfigUpdateRequest.java +++ b/server/src/main/java/org/apache/druid/server/http/CompactionConfigUpdateRequest.java @@ -22,7 +22,7 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import org.apache.druid.indexer.CompactionEngine; -import org.apache.druid.server.coordinator.compact.CompactionSegmentSearchPolicy; +import org.apache.druid.server.compaction.CompactionSegmentSearchPolicy; import javax.annotation.Nullable; diff --git a/server/src/main/java/org/apache/druid/server/http/CompactionResource.java b/server/src/main/java/org/apache/druid/server/http/CompactionResource.java index e88d0cdacfb2..fc41a8b16b30 100644 --- a/server/src/main/java/org/apache/druid/server/http/CompactionResource.java +++ b/server/src/main/java/org/apache/druid/server/http/CompactionResource.java @@ -29,6 +29,7 @@ import org.apache.druid.server.http.security.ConfigResourceFilter; import org.apache.druid.server.http.security.StateResourceFilter; +import javax.ws.rs.Consumes; import javax.ws.rs.GET; import javax.ws.rs.POST; import javax.ws.rs.Path; @@ -90,9 +91,9 @@ public Response getCompactionSnapshotForDataSource( { final Collection snapshots; if (dataSource == null || dataSource.isEmpty()) { - snapshots = coordinator.getAutoCompactionSnapshot().values(); + snapshots = coordinator.getAllCompactionSnapshots().values(); } else { - AutoCompactionSnapshot autoCompactionSnapshot = coordinator.getAutoCompactionSnapshotForDataSource(dataSource); + AutoCompactionSnapshot autoCompactionSnapshot = coordinator.getCompactionSnapshot(dataSource); if (autoCompactionSnapshot == null) { return Response.status(Response.Status.NOT_FOUND).entity(ImmutableMap.of("error", "unknown dataSource")).build(); } @@ -100,4 +101,16 @@ public Response getCompactionSnapshotForDataSource( } return Response.ok(ImmutableMap.of("latestStatus", snapshots)).build(); } + + @POST + @Path("/simulate") + @Consumes(MediaType.APPLICATION_JSON) + public Response simulateClusterCompactionConfigUpdate( + CompactionConfigUpdateRequest updatePayload + ) + { + return Response.ok().entity( + coordinator.simulateRunWithConfigUpdate(updatePayload) + ).build(); + } } diff --git a/server/src/main/java/org/apache/druid/server/http/CoordinatorCompactionConfigsResource.java b/server/src/main/java/org/apache/druid/server/http/CoordinatorCompactionConfigsResource.java index 721121a6de9f..18dc5bd2a169 100644 --- a/server/src/main/java/org/apache/druid/server/http/CoordinatorCompactionConfigsResource.java +++ b/server/src/main/java/org/apache/druid/server/http/CoordinatorCompactionConfigsResource.java @@ -38,7 +38,6 @@ import org.apache.druid.server.coordinator.CoordinatorConfigManager; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; import org.apache.druid.server.coordinator.DataSourceCompactionConfigHistory; -import org.apache.druid.server.coordinator.compact.CompactionDutySimulator; import org.apache.druid.server.http.security.ConfigResourceFilter; import org.apache.druid.server.security.AuthorizationUtils; import org.apache.druid.utils.CollectionUtils; @@ -74,18 +73,15 @@ public class CoordinatorCompactionConfigsResource static final int MAX_UPDATE_RETRIES = 5; private final CoordinatorConfigManager configManager; - private final CompactionDutySimulator simulator; private final AuditManager auditManager; @Inject public CoordinatorCompactionConfigsResource( CoordinatorConfigManager configManager, - CompactionDutySimulator simulator, AuditManager auditManager ) { this.configManager = configManager; - this.simulator = simulator; this.auditManager = auditManager; } @@ -131,16 +127,6 @@ public Response updateClusterCompactionConfig( return updateConfigHelper(operator, AuthorizationUtils.buildAuditInfo(req)); } - @POST - @Path("/simulate") - @Consumes(MediaType.APPLICATION_JSON) - public Response simulateClusterCompactionConfigUpdate( - CompactionConfigUpdateRequest updatePayload - ) - { - return Response.ok().entity(simulator.simulateRunWithConfigUpdate(updatePayload)).build(); - } - @POST @Path("/taskslots") @Consumes(MediaType.APPLICATION_JSON) 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 new file mode 100644 index 000000000000..bbca315a9578 --- /dev/null +++ b/server/src/test/java/org/apache/druid/server/compaction/CompactionRunSimulatorTest.java @@ -0,0 +1,87 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.server.compaction; + +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.jackson.DefaultObjectMapper; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.granularity.Granularities; +import org.apache.druid.server.coordinator.CoordinatorCompactionConfig; +import org.apache.druid.server.coordinator.CreateDataSegments; +import org.apache.druid.server.coordinator.DataSourceCompactionConfig; +import org.apache.druid.server.coordinator.simulate.TestSegmentsMetadataManager; +import org.apache.druid.server.http.CompactionConfigUpdateRequest; +import org.apache.druid.timeline.DataSegment; +import org.junit.Assert; +import org.junit.Test; + +import java.util.Arrays; +import java.util.Collections; +import java.util.List; + +public class CompactionRunSimulatorTest +{ + private static final ObjectMapper OBJECT_MAPPER = new DefaultObjectMapper(); + + @Test + public void testSimulateClusterCompactionConfigUpdate() + { + final TestSegmentsMetadataManager segmentsMetadataManager = new TestSegmentsMetadataManager(); + + // Add some segments to the timeline + final List wikiSegments + = CreateDataSegments.ofDatasource("wiki") + .forIntervals(10, Granularities.DAY) + .withNumPartitions(10) + .startingAt("2013-01-01") + .eachOfSizeInMb(100); + wikiSegments.forEach(segmentsMetadataManager::addSegment); + + final CompactionSimulateResult simulateResult = new CompactionRunSimulator( + new CompactionStatusTracker(OBJECT_MAPPER), + CoordinatorCompactionConfig.from( + Collections.singletonList( + DataSourceCompactionConfig.builder().forDataSource("wiki").build() + ) + ), + segmentsMetadataManager.getSnapshotOfDataSourcesWithAllUsedSegments() + .getUsedSegmentsTimelinesPerDataSource() + ).simulateRunWithConfigUpdate( + new CompactionConfigUpdateRequest(null, null, null, null, null) + ); + Assert.assertNotNull(simulateResult); + + Assert.assertEquals( + Arrays.asList( + Arrays.asList("dataSource", "interval", "numSegments", "bytes", "maxTaskSlots", "reasonToCompact"), + Arrays.asList("wiki", Intervals.of("2013-01-09/P1D"), 10, 1_000_000_000L, 1, "not compacted yet"), + Arrays.asList("wiki", Intervals.of("2013-01-08/P1D"), 10, 1_000_000_000L, 1, "not compacted yet"), + Arrays.asList("wiki", Intervals.of("2013-01-07/P1D"), 10, 1_000_000_000L, 1, "not compacted yet"), + Arrays.asList("wiki", Intervals.of("2013-01-06/P1D"), 10, 1_000_000_000L, 1, "not compacted yet"), + Arrays.asList("wiki", Intervals.of("2013-01-05/P1D"), 10, 1_000_000_000L, 1, "not compacted yet"), + Arrays.asList("wiki", Intervals.of("2013-01-04/P1D"), 10, 1_000_000_000L, 1, "not compacted yet"), + Arrays.asList("wiki", Intervals.of("2013-01-03/P1D"), 10, 1_000_000_000L, 1, "not compacted yet"), + Arrays.asList("wiki", Intervals.of("2013-01-02/P1D"), 10, 1_000_000_000L, 1, "not compacted yet"), + Arrays.asList("wiki", Intervals.of("2013-01-01/P1D"), 10, 1_000_000_000L, 1, "not compacted yet") + ), + simulateResult.getIntervalsToCompact() + ); + } +} diff --git a/server/src/test/java/org/apache/druid/server/coordinator/compact/CompactionStatusTest.java b/server/src/test/java/org/apache/druid/server/compaction/CompactionStatusTest.java similarity index 99% rename from server/src/test/java/org/apache/druid/server/coordinator/compact/CompactionStatusTest.java rename to server/src/test/java/org/apache/druid/server/compaction/CompactionStatusTest.java index 15caa77f22d9..ca11121efbbe 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/compact/CompactionStatusTest.java +++ b/server/src/test/java/org/apache/druid/server/compaction/CompactionStatusTest.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.druid.server.coordinator.compact; +package org.apache.druid.server.compaction; import com.fasterxml.jackson.databind.ObjectMapper; import org.apache.druid.client.indexing.ClientCompactionTaskQueryTuningConfig; diff --git a/server/src/test/java/org/apache/druid/server/coordinator/compact/CompactionStatusTrackerTest.java b/server/src/test/java/org/apache/druid/server/compaction/CompactionStatusTrackerTest.java similarity index 99% rename from server/src/test/java/org/apache/druid/server/coordinator/compact/CompactionStatusTrackerTest.java rename to server/src/test/java/org/apache/druid/server/compaction/CompactionStatusTrackerTest.java index ae4ce7b37c5e..9e4d5561f87d 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/compact/CompactionStatusTrackerTest.java +++ b/server/src/test/java/org/apache/druid/server/compaction/CompactionStatusTrackerTest.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.druid.server.coordinator.compact; +package org.apache.druid.server.compaction; import com.fasterxml.jackson.databind.ObjectMapper; import org.apache.druid.client.indexing.ClientCompactionTaskQuery; diff --git a/server/src/test/java/org/apache/druid/server/coordinator/compact/DataSourceCompactibleSegmentIteratorTest.java b/server/src/test/java/org/apache/druid/server/compaction/DataSourceCompactibleSegmentIteratorTest.java similarity index 97% rename from server/src/test/java/org/apache/druid/server/coordinator/compact/DataSourceCompactibleSegmentIteratorTest.java rename to server/src/test/java/org/apache/druid/server/compaction/DataSourceCompactibleSegmentIteratorTest.java index f2eb3d82cfa4..eea081b9f51b 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/compact/DataSourceCompactibleSegmentIteratorTest.java +++ b/server/src/test/java/org/apache/druid/server/compaction/DataSourceCompactibleSegmentIteratorTest.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.druid.server.coordinator.compact; +package org.apache.druid.server.compaction; import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; diff --git a/server/src/test/java/org/apache/druid/server/coordinator/compact/NewestSegmentFirstPolicyTest.java b/server/src/test/java/org/apache/druid/server/compaction/NewestSegmentFirstPolicyTest.java similarity index 99% rename from server/src/test/java/org/apache/druid/server/coordinator/compact/NewestSegmentFirstPolicyTest.java rename to server/src/test/java/org/apache/druid/server/compaction/NewestSegmentFirstPolicyTest.java index 84b2c37d0430..4dadccac9dba 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/compact/NewestSegmentFirstPolicyTest.java +++ b/server/src/test/java/org/apache/druid/server/compaction/NewestSegmentFirstPolicyTest.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.druid.server.coordinator.compact; +package org.apache.druid.server.compaction; import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.InjectableValues; diff --git a/server/src/test/java/org/apache/druid/server/coordinator/AutoCompactionSnapshotTest.java b/server/src/test/java/org/apache/druid/server/coordinator/AutoCompactionSnapshotTest.java index 5517bf9e6a4e..9a8cd3cc8772 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/AutoCompactionSnapshotTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/AutoCompactionSnapshotTest.java @@ -19,7 +19,7 @@ package org.apache.druid.server.coordinator; -import org.apache.druid.server.coordinator.compact.CompactionStatistics; +import org.apache.druid.server.compaction.CompactionStatistics; import org.junit.Assert; import org.junit.Test; diff --git a/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorTest.java b/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorTest.java index 3d2e21b11ce5..88d2051e0488 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorTest.java @@ -50,9 +50,9 @@ import org.apache.druid.metadata.SegmentsMetadataManager; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; import org.apache.druid.server.DruidNode; +import org.apache.druid.server.compaction.CompactionStatusTracker; import org.apache.druid.server.coordination.ServerType; import org.apache.druid.server.coordinator.balancer.CostBalancerStrategyFactory; -import org.apache.druid.server.coordinator.compact.CompactionStatusTracker; import org.apache.druid.server.coordinator.config.CoordinatorKillConfigs; import org.apache.druid.server.coordinator.config.CoordinatorPeriodConfig; import org.apache.druid.server.coordinator.config.CoordinatorRunConfig; @@ -96,6 +96,7 @@ public class DruidCoordinatorTest extends CuratorTestBase private static final Duration LOAD_TIMEOUT = Duration.standardMinutes(15); private static final long COORDINATOR_START_DELAY = 1; private static final long COORDINATOR_PERIOD = 100; + private static final ObjectMapper OBJECT_MAPPER = new DefaultObjectMapper(); private DruidCoordinator coordinator; private SegmentsMetadataManager segmentsMetadataManager; @@ -192,7 +193,7 @@ public void setUp() throws Exception null, CentralizedDatasourceSchemaConfig.create(), new CompactionSchedulerConfig(true), - statusTracker + OBJECT_MAPPER ); } @@ -624,7 +625,7 @@ public void testCompactSegmentsDutyWhenCustomDutyGroupEmpty() null, CentralizedDatasourceSchemaConfig.create(), CompactionSchedulerConfig.defaultConfig(), - statusTracker + OBJECT_MAPPER ); // Since CompactSegments is not enabled in Custom Duty Group, then CompactSegments must be created in IndexingServiceDuties List indexingDuties = coordinator.makeIndexingServiceDuties(); @@ -665,7 +666,7 @@ public void testInitializeCompactSegmentsDutyWhenCustomDutyGroupDoesNotContainsC null, CentralizedDatasourceSchemaConfig.create(), CompactionSchedulerConfig.defaultConfig(), - statusTracker + OBJECT_MAPPER ); // Since CompactSegments is not enabled in Custom Duty Group, then CompactSegments must be created in IndexingServiceDuties List indexingDuties = coordinator.makeIndexingServiceDuties(); @@ -706,7 +707,7 @@ public void testInitializeCompactSegmentsDutyWhenCustomDutyGroupContainsCompactS null, CentralizedDatasourceSchemaConfig.create(), CompactionSchedulerConfig.defaultConfig(), - statusTracker + OBJECT_MAPPER ); // Since CompactSegments is enabled in Custom Duty Group, then CompactSegments must not be created in IndexingServiceDuties List indexingDuties = coordinator.makeIndexingServiceDuties(); @@ -812,7 +813,7 @@ public void testCoordinatorCustomDutyGroupsRunAsExpected() throws Exception null, CentralizedDatasourceSchemaConfig.create(), new CompactionSchedulerConfig(true), - statusTracker + OBJECT_MAPPER ); coordinator.start(); diff --git a/server/src/test/java/org/apache/druid/server/coordinator/duty/CompactSegmentsTest.java b/server/src/test/java/org/apache/druid/server/coordinator/duty/CompactSegmentsTest.java index e937a8eb25dc..722787727a76 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 @@ -68,6 +68,7 @@ import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.segment.indexing.BatchIOConfig; import org.apache.druid.segment.transform.TransformSpec; +import org.apache.druid.server.compaction.CompactionStatusTracker; import org.apache.druid.server.coordinator.AutoCompactionSnapshot; import org.apache.druid.server.coordinator.CoordinatorCompactionConfig; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; @@ -77,7 +78,6 @@ import org.apache.druid.server.coordinator.UserCompactionTaskIOConfig; import org.apache.druid.server.coordinator.UserCompactionTaskQueryTuningConfig; import org.apache.druid.server.coordinator.UserCompactionTaskTransformConfig; -import org.apache.druid.server.coordinator.compact.CompactionStatusTracker; import org.apache.druid.server.coordinator.config.DruidCoordinatorConfig; import org.apache.druid.server.coordinator.stats.CoordinatorRunStats; import org.apache.druid.server.coordinator.stats.Stats; diff --git a/server/src/test/java/org/apache/druid/server/coordinator/simulate/CoordinatorSimulationBuilder.java b/server/src/test/java/org/apache/druid/server/coordinator/simulate/CoordinatorSimulationBuilder.java index d252d6e6f7f9..9c1fa19689ca 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/simulate/CoordinatorSimulationBuilder.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/simulate/CoordinatorSimulationBuilder.java @@ -49,7 +49,6 @@ import org.apache.druid.server.coordinator.balancer.CostBalancerStrategyFactory; import org.apache.druid.server.coordinator.balancer.DiskNormalizedCostBalancerStrategyFactory; import org.apache.druid.server.coordinator.balancer.RandomBalancerStrategyFactory; -import org.apache.druid.server.coordinator.compact.CompactionStatusTracker; import org.apache.druid.server.coordinator.config.CoordinatorKillConfigs; import org.apache.druid.server.coordinator.config.CoordinatorPeriodConfig; import org.apache.druid.server.coordinator.config.CoordinatorRunConfig; @@ -213,7 +212,7 @@ public CoordinatorSimulation build() null, CentralizedDatasourceSchemaConfig.create(), CompactionSchedulerConfig.defaultConfig(), - new CompactionStatusTracker(OBJECT_MAPPER) + OBJECT_MAPPER ); return new SimulationImpl(coordinator, env); diff --git a/server/src/test/java/org/apache/druid/server/http/CompactionResourceTest.java b/server/src/test/java/org/apache/druid/server/http/CompactionResourceTest.java index fe16d2f8c1f2..fdd34515395f 100644 --- a/server/src/test/java/org/apache/druid/server/http/CompactionResourceTest.java +++ b/server/src/test/java/org/apache/druid/server/http/CompactionResourceTest.java @@ -70,7 +70,7 @@ public void testGetCompactionSnapshotForDataSourceWithEmptyQueryParameter() expectedSnapshot ); - EasyMock.expect(mock.getAutoCompactionSnapshot()).andReturn(expected).once(); + EasyMock.expect(mock.getAllCompactionSnapshots()).andReturn(expected).once(); EasyMock.replay(mock); final Response response = new CompactionResource(mock).getCompactionSnapshotForDataSource(""); @@ -87,7 +87,7 @@ public void testGetCompactionSnapshotForDataSourceWithNullQueryParameter() expectedSnapshot ); - EasyMock.expect(mock.getAutoCompactionSnapshot()).andReturn(expected).once(); + EasyMock.expect(mock.getAllCompactionSnapshots()).andReturn(expected).once(); EasyMock.replay(mock); final Response response = new CompactionResource(mock).getCompactionSnapshotForDataSource(null); @@ -100,7 +100,7 @@ public void testGetCompactionSnapshotForDataSourceWithValidQueryParameter() { String dataSourceName = "datasource_1"; - EasyMock.expect(mock.getAutoCompactionSnapshotForDataSource(dataSourceName)).andReturn(expectedSnapshot).once(); + EasyMock.expect(mock.getCompactionSnapshot(dataSourceName)).andReturn(expectedSnapshot).once(); EasyMock.replay(mock); final Response response = new CompactionResource(mock).getCompactionSnapshotForDataSource(dataSourceName); @@ -113,7 +113,7 @@ public void testGetCompactionSnapshotForDataSourceWithInvalidQueryParameter() { String dataSourceName = "invalid_datasource"; - EasyMock.expect(mock.getAutoCompactionSnapshotForDataSource(dataSourceName)).andReturn(null).once(); + EasyMock.expect(mock.getCompactionSnapshot(dataSourceName)).andReturn(null).once(); EasyMock.replay(mock); final Response response = new CompactionResource(mock).getCompactionSnapshotForDataSource(dataSourceName); diff --git a/server/src/test/java/org/apache/druid/server/http/CoordinatorCompactionConfigsResourceTest.java b/server/src/test/java/org/apache/druid/server/http/CoordinatorCompactionConfigsResourceTest.java index cad94cc54265..efad8d622da8 100644 --- a/server/src/test/java/org/apache/druid/server/http/CoordinatorCompactionConfigsResourceTest.java +++ b/server/src/test/java/org/apache/druid/server/http/CoordinatorCompactionConfigsResourceTest.java @@ -32,7 +32,6 @@ import org.apache.druid.error.ErrorResponse; import org.apache.druid.indexer.CompactionEngine; import org.apache.druid.jackson.DefaultObjectMapper; -import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.metadata.MetadataCASUpdate; import org.apache.druid.metadata.MetadataStorageTablesConfig; @@ -40,17 +39,10 @@ import org.apache.druid.metadata.TestMetadataStorageTablesConfig; import org.apache.druid.server.coordinator.CoordinatorCompactionConfig; import org.apache.druid.server.coordinator.CoordinatorConfigManager; -import org.apache.druid.server.coordinator.CreateDataSegments; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; import org.apache.druid.server.coordinator.DataSourceCompactionConfigAuditEntry; -import org.apache.druid.server.coordinator.MetadataManager; import org.apache.druid.server.coordinator.UserCompactionTaskGranularityConfig; -import org.apache.druid.server.coordinator.compact.CompactionDutySimulator; -import org.apache.druid.server.coordinator.compact.CompactionSimulateResult; -import org.apache.druid.server.coordinator.compact.CompactionStatusTracker; import org.apache.druid.server.coordinator.config.DataSourceCompactionConfigBuilder; -import org.apache.druid.server.coordinator.simulate.TestSegmentsMetadataManager; -import org.apache.druid.timeline.DataSegment; import org.joda.time.Interval; import org.joda.time.Period; import org.junit.After; @@ -92,7 +84,7 @@ public void setup() Mockito.when(mockHttpServletRequest.getRemoteAddr()).thenReturn("123"); final AuditManager auditManager = new TestAuditManager(); configManager = TestCoordinatorConfigManager.create(auditManager); - resource = new CoordinatorCompactionConfigsResource(configManager, null, auditManager); + resource = new CoordinatorCompactionConfigsResource(configManager, auditManager); configManager.delegate.start(); } @@ -447,6 +439,7 @@ public int removeAuditLogsOlderThan(long timestamp) private static class TestCoordinatorConfigManager extends CoordinatorConfigManager { private final ConfigManager delegate; + private final JacksonConfigManager jackson; private int numUpdateAttempts; private ConfigManager.SetResult configUpdateResult; @@ -468,22 +461,24 @@ public String getConfigTable() Suppliers.ofInstance(new TestConfigManagerConfig()) ); - return new TestCoordinatorConfigManager(configManager, dbConnector, tablesConfig, auditManager); + return new TestCoordinatorConfigManager( + new JacksonConfigManager(configManager, OBJECT_MAPPER, auditManager), + configManager, + dbConnector, + tablesConfig + ); } TestCoordinatorConfigManager( + JacksonConfigManager jackson, ConfigManager configManager, TestDBConnector dbConnector, - MetadataStorageTablesConfig tablesConfig, - AuditManager auditManager + MetadataStorageTablesConfig tablesConfig ) { - super( - new JacksonConfigManager(configManager, OBJECT_MAPPER, auditManager), - dbConnector, - tablesConfig - ); + super(jackson, dbConnector, tablesConfig); this.delegate = configManager; + this.jackson = jackson; } @Override @@ -552,56 +547,4 @@ private static class DS { static final String WIKI = "wiki"; } - - @Test - public void testSimulateClusterCompactionConfigUpdate() - { - resource.addOrUpdateDatasourceCompactionConfig( - DataSourceCompactionConfig.builder().forDataSource(DS.WIKI).build(), - mockHttpServletRequest - ); - - final TestSegmentsMetadataManager segmentsMetadataManager = new TestSegmentsMetadataManager(); - final MetadataManager metadataManager = new MetadataManager( - null, - configManager, segmentsMetadataManager, null, null, null, null - ); - - // Add some segments to the timeline - final List wikiSegments - = CreateDataSegments.ofDatasource(DS.WIKI) - .forIntervals(10, Granularities.DAY) - .withNumPartitions(10) - .startingAt("2013-01-01") - .eachOfSizeInMb(100); - wikiSegments.forEach(segmentsMetadataManager::addSegment); - - resource = new CoordinatorCompactionConfigsResource( - null, - new CompactionDutySimulator(new CompactionStatusTracker(OBJECT_MAPPER), metadataManager, OBJECT_MAPPER), - null - ); - Response response = resource.simulateClusterCompactionConfigUpdate( - new CompactionConfigUpdateRequest(null, null, null, null, null) - ); - Assert.assertEquals(Response.Status.OK.getStatusCode(), response.getStatus()); - Assert.assertTrue(response.getEntity() instanceof CompactionSimulateResult); - - CompactionSimulateResult simulateResult = (CompactionSimulateResult) response.getEntity(); - Assert.assertEquals( - Arrays.asList( - Arrays.asList("dataSource", "interval", "numSegments", "bytes", "maxTaskSlots", "reasonToCompact"), - Arrays.asList("wiki", Intervals.of("2013-01-09/P1D"), 10, 1_000_000_000L, 1, "not compacted yet"), - Arrays.asList("wiki", Intervals.of("2013-01-08/P1D"), 10, 1_000_000_000L, 1, "not compacted yet"), - Arrays.asList("wiki", Intervals.of("2013-01-07/P1D"), 10, 1_000_000_000L, 1, "not compacted yet"), - Arrays.asList("wiki", Intervals.of("2013-01-06/P1D"), 10, 1_000_000_000L, 1, "not compacted yet"), - Arrays.asList("wiki", Intervals.of("2013-01-05/P1D"), 10, 1_000_000_000L, 1, "not compacted yet"), - Arrays.asList("wiki", Intervals.of("2013-01-04/P1D"), 10, 1_000_000_000L, 1, "not compacted yet"), - Arrays.asList("wiki", Intervals.of("2013-01-03/P1D"), 10, 1_000_000_000L, 1, "not compacted yet"), - Arrays.asList("wiki", Intervals.of("2013-01-02/P1D"), 10, 1_000_000_000L, 1, "not compacted yet"), - Arrays.asList("wiki", Intervals.of("2013-01-01/P1D"), 10, 1_000_000_000L, 1, "not compacted yet") - ), - simulateResult.getIntervalsToCompact() - ); - } } diff --git a/services/src/main/java/org/apache/druid/cli/CliCoordinator.java b/services/src/main/java/org/apache/druid/cli/CliCoordinator.java index dc6298e4f232..57ede3d42502 100644 --- a/services/src/main/java/org/apache/druid/cli/CliCoordinator.java +++ b/services/src/main/java/org/apache/druid/cli/CliCoordinator.java @@ -103,8 +103,6 @@ import org.apache.druid.server.coordinator.MetadataManager; import org.apache.druid.server.coordinator.balancer.BalancerStrategyFactory; import org.apache.druid.server.coordinator.balancer.CachingCostBalancerStrategyConfig; -import org.apache.druid.server.coordinator.compact.CompactionDutySimulator; -import org.apache.druid.server.coordinator.compact.CompactionStatusTracker; import org.apache.druid.server.coordinator.config.CoordinatorKillConfigs; import org.apache.druid.server.coordinator.config.CoordinatorPeriodConfig; import org.apache.druid.server.coordinator.config.CoordinatorRunConfig; @@ -257,11 +255,9 @@ public void configure(Binder binder) binder.bind(LookupCoordinatorManager.class).in(LazySingleton.class); - binder.bind(CompactionStatusTracker.class).in(LazySingleton.class); binder.bind(CoordinatorConfigManager.class); binder.bind(MetadataManager.class); binder.bind(DruidCoordinator.class); - binder.bind(CompactionDutySimulator.class).in(LazySingleton.class); LifecycleModule.register(binder, MetadataStorage.class); LifecycleModule.register(binder, DruidCoordinator.class); diff --git a/services/src/main/java/org/apache/druid/cli/CliOverlord.java b/services/src/main/java/org/apache/druid/cli/CliOverlord.java index 21d9f30c7d43..52f8665b5487 100644 --- a/services/src/main/java/org/apache/druid/cli/CliOverlord.java +++ b/services/src/main/java/org/apache/druid/cli/CliOverlord.java @@ -66,8 +66,7 @@ import org.apache.druid.indexing.common.task.batch.parallel.ShuffleClient; import org.apache.druid.indexing.common.tasklogs.SwitchingTaskLogStreamer; import org.apache.druid.indexing.common.tasklogs.TaskRunnerTaskLogStreamer; -import org.apache.druid.indexing.compact.CompactionScheduler; -import org.apache.druid.indexing.compact.CompactionSchedulerImpl; +import org.apache.druid.indexing.compact.OverlordCompactionScheduler; import org.apache.druid.indexing.overlord.DruidOverlord; import org.apache.druid.indexing.overlord.ForkingTaskRunnerFactory; import org.apache.druid.indexing.overlord.HeapMemoryTaskStorage; @@ -118,7 +117,6 @@ import org.apache.druid.segment.realtime.appenderator.DummyForInjectionAppenderatorsManager; import org.apache.druid.server.coordinator.CompactionSchedulerConfig; import org.apache.druid.server.coordinator.CoordinatorOverlordServiceConfig; -import org.apache.druid.server.coordinator.compact.CompactionStatusTracker; import org.apache.druid.server.http.RedirectFilter; import org.apache.druid.server.http.RedirectInfo; import org.apache.druid.server.http.SelfDiscoveryResource; @@ -207,12 +205,15 @@ public void configure(Binder binder) binder.bindConstant().annotatedWith(Names.named("servicePort")).to(8090); binder.bindConstant().annotatedWith(Names.named("tlsServicePort")).to(8290); - JsonConfigProvider.bind(binder, CentralizedDatasourceSchemaConfig.PROPERTY_PREFIX, CentralizedDatasourceSchemaConfig.class); + JsonConfigProvider.bind( + binder, + CentralizedDatasourceSchemaConfig.PROPERTY_PREFIX, + CentralizedDatasourceSchemaConfig.class + ); binder.bind(SegmentsMetadataManager.class) .toProvider(SegmentsMetadataManagerProvider.class) .in(ManageLifecycle.class); - binder.bind(CompactionStatusTracker.class).in(LazySingleton.class); JsonConfigProvider.bind(binder, "druid.compaction.scheduler", CompactionSchedulerConfig.class); } @@ -246,7 +247,7 @@ public void configure(Binder binder) binder.bind(TaskQueryTool.class).in(LazySingleton.class); binder.bind(IndexerMetadataStorageAdapter.class).in(LazySingleton.class); binder.bind(SupervisorManager.class).in(LazySingleton.class); - binder.bind(CompactionScheduler.class).to(CompactionSchedulerImpl.class).in(LazySingleton.class); + binder.bind(OverlordCompactionScheduler.class).in(LazySingleton.class); binder.bind(ParallelIndexSupervisorTaskClientProvider.class).toProvider(Providers.of(null)); binder.bind(ShuffleClient.class).toProvider(Providers.of(null)); From f972ea2c020f2d745bf356ddc1a938cb0e508679 Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Tue, 30 Jul 2024 15:01:39 +0530 Subject: [PATCH 08/26] Handle master changes --- .../compact/OverlordCompactionScheduler.java | 16 ++++++++-------- .../http/OverlordCompactionResource.java | 4 ++-- .../compaction/CompactionRunSimulator.java | 18 +++++++++--------- .../server/compaction/CompactionScheduler.java | 4 ++-- .../server/coordinator/DruidCoordinator.java | 3 +-- .../druid/server/http/CompactionResource.java | 3 ++- .../compaction/CompactionRunSimulatorTest.java | 13 +++++-------- ...taSourceCompactionConfigAuditEntryTest.java | 12 ++++++------ .../DataSourceCompactionConfigHistoryTest.java | 2 +- .../coordinator/DruidCompactionConfigTest.java | 6 ++++-- 10 files changed, 40 insertions(+), 41 deletions(-) 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 5c88434f2598..525d34e4f277 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 @@ -40,14 +40,14 @@ import org.apache.druid.server.compaction.CompactionSimulateResult; import org.apache.druid.server.compaction.CompactionStatusTracker; import org.apache.druid.server.coordinator.AutoCompactionSnapshot; +import org.apache.druid.server.coordinator.ClusterCompactionConfig; import org.apache.druid.server.coordinator.CompactionSchedulerConfig; -import org.apache.druid.server.coordinator.CoordinatorCompactionConfig; import org.apache.druid.server.coordinator.CoordinatorOverlordServiceConfig; +import org.apache.druid.server.coordinator.DruidCompactionConfig; import org.apache.druid.server.coordinator.duty.CompactSegments; import org.apache.druid.server.coordinator.stats.CoordinatorRunStats; import org.apache.druid.server.coordinator.stats.CoordinatorStat; import org.apache.druid.server.coordinator.stats.Dimension; -import org.apache.druid.server.http.CompactionConfigUpdateRequest; import org.apache.druid.timeline.SegmentTimeline; import org.joda.time.Duration; @@ -234,7 +234,7 @@ private synchronized void checkSchedulingStatus() } private synchronized void processCompactionQueue( - CoordinatorCompactionConfig currentConfig + DruidCompactionConfig currentConfig ) { final CoordinatorRunStats stats = new CoordinatorRunStats(); @@ -267,12 +267,12 @@ private void emitStat(CoordinatorStat stat, Map dimensionValu emitter.emit(eventBuilder.setMetric(stat.getMetricName(), value)); } - private CoordinatorCompactionConfig getLatestConfig() + private DruidCompactionConfig getLatestConfig() { return configManager.watch( - CoordinatorCompactionConfig.CONFIG_KEY, - CoordinatorCompactionConfig.class, - CoordinatorCompactionConfig.empty() + DruidCompactionConfig.CONFIG_KEY, + DruidCompactionConfig.class, + DruidCompactionConfig.empty() ).get(); } @@ -301,7 +301,7 @@ public Map getAllCompactionSnapshots() } @Override - public CompactionSimulateResult simulateRunWithConfigUpdate(CompactionConfigUpdateRequest updateRequest) + public CompactionSimulateResult simulateRunWithConfigUpdate(ClusterCompactionConfig updateRequest) { return new CompactionRunSimulator( statusTracker, diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordCompactionResource.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordCompactionResource.java index 089d085e4b20..98719474411d 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordCompactionResource.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordCompactionResource.java @@ -23,7 +23,7 @@ import com.sun.jersey.spi.container.ResourceFilters; import org.apache.druid.indexing.compact.OverlordCompactionScheduler; import org.apache.druid.server.coordinator.AutoCompactionSnapshot; -import org.apache.druid.server.http.CompactionConfigUpdateRequest; +import org.apache.druid.server.coordinator.ClusterCompactionConfig; import org.apache.druid.server.http.security.StateResourceFilter; import javax.ws.rs.Consumes; @@ -100,7 +100,7 @@ public Response getCompactionSnapshotForDataSource( @Path("/simulate") @Consumes(MediaType.APPLICATION_JSON) public Response simulateClusterCompactionConfigUpdate( - CompactionConfigUpdateRequest updatePayload + ClusterCompactionConfig updatePayload ) { return Response.ok().entity( 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 d4d999e9c0c5..f7853204a122 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 @@ -35,11 +35,11 @@ import org.apache.druid.metadata.LockFilterPolicy; import org.apache.druid.rpc.ServiceRetryPolicy; import org.apache.druid.rpc.indexing.OverlordClient; -import org.apache.druid.server.coordinator.CoordinatorCompactionConfig; +import org.apache.druid.server.coordinator.ClusterCompactionConfig; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; +import org.apache.druid.server.coordinator.DruidCompactionConfig; import org.apache.druid.server.coordinator.duty.CompactSegments; import org.apache.druid.server.coordinator.stats.CoordinatorRunStats; -import org.apache.druid.server.http.CompactionConfigUpdateRequest; import org.apache.druid.timeline.SegmentTimeline; import org.joda.time.Interval; @@ -58,14 +58,14 @@ */ public class CompactionRunSimulator { - private final CoordinatorCompactionConfig compactionConfig; + private final DruidCompactionConfig compactionConfig; private final CompactionStatusTracker statusTracker; private final Map datasourceTimelines; private final OverlordClient emptyOverlordClient = new EmptyOverlordClient(); public CompactionRunSimulator( CompactionStatusTracker statusTracker, - CoordinatorCompactionConfig compactionConfig, + DruidCompactionConfig compactionConfig, Map datasourceTimelines ) { @@ -79,18 +79,18 @@ public CompactionRunSimulator( * assuming unlimited compaction task slots. */ public CompactionSimulateResult simulateRunWithConfigUpdate( - CompactionConfigUpdateRequest updateRequest + ClusterCompactionConfig updateRequest ) { - final CompactionConfigUpdateRequest updateWithUnlimitedSlots = new CompactionConfigUpdateRequest( + final ClusterCompactionConfig updateWithUnlimitedSlots = new ClusterCompactionConfig( 1.0, Integer.MAX_VALUE, updateRequest.getUseAutoScaleSlots(), - updateRequest.getCompactionEngine(), + updateRequest.getEngine(), updateRequest.getCompactionPolicy() ); - final CoordinatorCompactionConfig configWithUnlimitedTaskSlots - = CoordinatorCompactionConfig.from(compactionConfig, updateWithUnlimitedSlots); + final DruidCompactionConfig configWithUnlimitedTaskSlots + = compactionConfig.withClusterConfig(updateWithUnlimitedSlots); final List> tableOfCompactibleIntervals = new ArrayList<>(); final List> tableOfSkippedIntervals = new ArrayList<>(); diff --git a/server/src/main/java/org/apache/druid/server/compaction/CompactionScheduler.java b/server/src/main/java/org/apache/druid/server/compaction/CompactionScheduler.java index 38640974edf7..10989bb45d4b 100644 --- a/server/src/main/java/org/apache/druid/server/compaction/CompactionScheduler.java +++ b/server/src/main/java/org/apache/druid/server/compaction/CompactionScheduler.java @@ -20,7 +20,7 @@ package org.apache.druid.server.compaction; import org.apache.druid.server.coordinator.AutoCompactionSnapshot; -import org.apache.druid.server.http.CompactionConfigUpdateRequest; +import org.apache.druid.server.coordinator.ClusterCompactionConfig; import java.util.Map; @@ -36,6 +36,6 @@ public interface CompactionScheduler Long getTotalSizeOfSegmentsAwaitingCompaction(String dataSource); - CompactionSimulateResult simulateRunWithConfigUpdate(CompactionConfigUpdateRequest updateRequest); + CompactionSimulateResult simulateRunWithConfigUpdate(ClusterCompactionConfig updateRequest); } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java b/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java index a32714b304e6..3b5bb9b17df2 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java @@ -92,7 +92,6 @@ import org.apache.druid.server.coordinator.stats.Dimension; import org.apache.druid.server.coordinator.stats.RowKey; import org.apache.druid.server.coordinator.stats.Stats; -import org.apache.druid.server.http.CompactionConfigUpdateRequest; import org.apache.druid.server.lookup.cache.LookupCoordinatorManager; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.SegmentId; @@ -372,7 +371,7 @@ public Map getAllCompactionSnapshots() } @Override - public CompactionSimulateResult simulateRunWithConfigUpdate(CompactionConfigUpdateRequest updateRequest) + public CompactionSimulateResult simulateRunWithConfigUpdate(ClusterCompactionConfig updateRequest) { return new CompactionRunSimulator( compactionStatusTracker, diff --git a/server/src/main/java/org/apache/druid/server/http/CompactionResource.java b/server/src/main/java/org/apache/druid/server/http/CompactionResource.java index fc41a8b16b30..9dd569822521 100644 --- a/server/src/main/java/org/apache/druid/server/http/CompactionResource.java +++ b/server/src/main/java/org/apache/druid/server/http/CompactionResource.java @@ -25,6 +25,7 @@ import com.google.inject.Inject; import com.sun.jersey.spi.container.ResourceFilters; import org.apache.druid.server.coordinator.AutoCompactionSnapshot; +import org.apache.druid.server.coordinator.ClusterCompactionConfig; import org.apache.druid.server.coordinator.DruidCoordinator; import org.apache.druid.server.http.security.ConfigResourceFilter; import org.apache.druid.server.http.security.StateResourceFilter; @@ -106,7 +107,7 @@ public Response getCompactionSnapshotForDataSource( @Path("/simulate") @Consumes(MediaType.APPLICATION_JSON) public Response simulateClusterCompactionConfigUpdate( - CompactionConfigUpdateRequest updatePayload + ClusterCompactionConfig updatePayload ) { return Response.ok().entity( 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 bbca315a9578..7fb6bc5d7f31 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,17 +23,16 @@ import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.granularity.Granularities; -import org.apache.druid.server.coordinator.CoordinatorCompactionConfig; +import org.apache.druid.server.coordinator.ClusterCompactionConfig; import org.apache.druid.server.coordinator.CreateDataSegments; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; +import org.apache.druid.server.coordinator.DruidCompactionConfig; import org.apache.druid.server.coordinator.simulate.TestSegmentsMetadataManager; -import org.apache.druid.server.http.CompactionConfigUpdateRequest; import org.apache.druid.timeline.DataSegment; import org.junit.Assert; import org.junit.Test; import java.util.Arrays; -import java.util.Collections; import java.util.List; public class CompactionRunSimulatorTest @@ -56,15 +55,13 @@ public void testSimulateClusterCompactionConfigUpdate() final CompactionSimulateResult simulateResult = new CompactionRunSimulator( new CompactionStatusTracker(OBJECT_MAPPER), - CoordinatorCompactionConfig.from( - Collections.singletonList( - DataSourceCompactionConfig.builder().forDataSource("wiki").build() - ) + DruidCompactionConfig.empty().withDatasourceConfig( + DataSourceCompactionConfig.builder().forDataSource("wiki").build() ), segmentsMetadataManager.getSnapshotOfDataSourcesWithAllUsedSegments() .getUsedSegmentsTimelinesPerDataSource() ).simulateRunWithConfigUpdate( - new CompactionConfigUpdateRequest(null, null, null, null, null) + new ClusterCompactionConfig(null, null, null, null, null) ); Assert.assertNotNull(simulateResult); diff --git a/server/src/test/java/org/apache/druid/server/coordinator/DataSourceCompactionConfigAuditEntryTest.java b/server/src/test/java/org/apache/druid/server/coordinator/DataSourceCompactionConfigAuditEntryTest.java index 27d80e3b5683..ddb495572ab4 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/DataSourceCompactionConfigAuditEntryTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/DataSourceCompactionConfigAuditEntryTest.java @@ -34,7 +34,7 @@ public class DataSourceCompactionConfigAuditEntryTest private final AuditInfo auditInfo = new AuditInfo("author", "identity", "comment", "ip"); private final DataSourceCompactionConfigAuditEntry firstEntry = new DataSourceCompactionConfigAuditEntry( - new ClusterCompactionConfig(0.1, 9, true, CompactionEngine.MSQ), + new ClusterCompactionConfig(0.1, 9, true, CompactionEngine.MSQ, null), DataSourceCompactionConfig.builder().forDataSource(DS_WIKI).build(), auditInfo, DateTimes.nowUtc() @@ -44,7 +44,7 @@ public class DataSourceCompactionConfigAuditEntryTest public void testhasSameConfigWithSameBaseConfigIsTrue() { final DataSourceCompactionConfigAuditEntry secondEntry = new DataSourceCompactionConfigAuditEntry( - new ClusterCompactionConfig(0.1, 9, true, CompactionEngine.MSQ), + new ClusterCompactionConfig(0.1, 9, true, CompactionEngine.MSQ, null), DataSourceCompactionConfig.builder().forDataSource(DS_WIKI).build(), auditInfo, DateTimes.nowUtc() @@ -57,7 +57,7 @@ public void testhasSameConfigWithSameBaseConfigIsTrue() public void testhasSameConfigWithDifferentClusterConfigIsFalse() { DataSourceCompactionConfigAuditEntry secondEntry = new DataSourceCompactionConfigAuditEntry( - new ClusterCompactionConfig(0.1, 9, false, CompactionEngine.MSQ), + new ClusterCompactionConfig(0.1, 9, false, CompactionEngine.MSQ, null), DataSourceCompactionConfig.builder().forDataSource(DS_WIKI).build(), auditInfo, DateTimes.nowUtc() @@ -66,7 +66,7 @@ public void testhasSameConfigWithDifferentClusterConfigIsFalse() Assert.assertFalse(secondEntry.hasSameConfig(firstEntry)); secondEntry = new DataSourceCompactionConfigAuditEntry( - new ClusterCompactionConfig(0.1, 9, true, CompactionEngine.NATIVE), + new ClusterCompactionConfig(0.1, 9, true, CompactionEngine.NATIVE, null), DataSourceCompactionConfig.builder().forDataSource(DS_WIKI).build(), auditInfo, DateTimes.nowUtc() @@ -79,7 +79,7 @@ public void testhasSameConfigWithDifferentClusterConfigIsFalse() public void testhasSameConfigWithDifferentDatasourceConfigIsFalse() { DataSourceCompactionConfigAuditEntry secondEntry = new DataSourceCompactionConfigAuditEntry( - new ClusterCompactionConfig(0.1, 9, true, CompactionEngine.NATIVE), + new ClusterCompactionConfig(0.1, 9, true, CompactionEngine.NATIVE, null), DataSourceCompactionConfig.builder().forDataSource(DS_WIKI).build(), auditInfo, DateTimes.nowUtc() @@ -92,7 +92,7 @@ public void testhasSameConfigWithDifferentDatasourceConfigIsFalse() public void testhasSameConfigWithNullDatasourceConfigIsFalse() { final DataSourceCompactionConfigAuditEntry secondEntry = new DataSourceCompactionConfigAuditEntry( - new ClusterCompactionConfig(0.1, 9, true, CompactionEngine.NATIVE), + new ClusterCompactionConfig(0.1, 9, true, CompactionEngine.NATIVE, null), null, auditInfo, DateTimes.nowUtc() diff --git a/server/src/test/java/org/apache/druid/server/coordinator/DataSourceCompactionConfigHistoryTest.java b/server/src/test/java/org/apache/druid/server/coordinator/DataSourceCompactionConfigHistoryTest.java index 60a638974f46..1809a5bc54b1 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/DataSourceCompactionConfigHistoryTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/DataSourceCompactionConfigHistoryTest.java @@ -177,7 +177,7 @@ public void testAddAndModifyClusterConfigShouldAddTwice() wikiAuditHistory.add(originalConfig, auditInfo, DateTimes.nowUtc()); final DruidCompactionConfig updatedConfig = originalConfig.withClusterConfig( - new ClusterCompactionConfig(null, null, null, CompactionEngine.MSQ) + new ClusterCompactionConfig(null, null, null, CompactionEngine.MSQ, null) ); wikiAuditHistory.add(updatedConfig, auditInfo, DateTimes.nowUtc()); diff --git a/server/src/test/java/org/apache/druid/server/coordinator/DruidCompactionConfigTest.java b/server/src/test/java/org/apache/druid/server/coordinator/DruidCompactionConfigTest.java index bf13a94e0080..2e58f740d63c 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/DruidCompactionConfigTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/DruidCompactionConfigTest.java @@ -62,7 +62,8 @@ public void testSerdeWithDatasourceConfigs() throws Exception null, null, null, - CompactionEngine.MSQ + CompactionEngine.MSQ, + null ); final String json = MAPPER.writeValueAsString(config); @@ -75,7 +76,8 @@ public void testCopyWithClusterConfig() { final DruidCompactionConfig config = DruidCompactionConfig.empty(); - final ClusterCompactionConfig clusterConfig = new ClusterCompactionConfig(0.5, 10, false, CompactionEngine.MSQ); + final ClusterCompactionConfig clusterConfig + = new ClusterCompactionConfig(0.5, 10, false, CompactionEngine.MSQ, null); final DruidCompactionConfig copy = config.withClusterConfig(clusterConfig); Assert.assertEquals(clusterConfig, copy.clusterConfig()); From fdcfdf19f77cf9a6b5a49f16a35e00487c4b120b Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Thu, 8 Aug 2024 10:44:09 +0530 Subject: [PATCH 09/26] Add CompactionSupervisor --- .../apache/druid/guice/SupervisorModule.java | 52 ++++ .../compact/CompactionStatusReport.java | 18 +- .../compact/CompactionSupervisor.java | 162 +++++++++++++ .../compact/CompactionSupervisorSpec.java | 115 +++++++++ .../compact/OverlordCompactionScheduler.java | 222 +++++++++++------- .../indexing/overlord/DruidOverlord.java | 8 +- .../http/OverlordCompactionResource.java | 5 +- .../indexing/overlord/http/OverlordTest.java | 4 +- .../compaction/CompactionRunSimulator.java | 218 +++++++++-------- .../compaction/CompactionScheduler.java | 11 +- .../compaction/CompactionSegmentIterator.java | 24 +- .../compaction/CompactionSimulateResult.java | 21 +- .../server/compaction/CompactionStatus.java | 14 +- .../compaction/CompactionStatusTracker.java | 196 +++++----------- .../compaction/CompactionTaskStatus.java | 56 +++++ .../DataSourceCompactibleSegmentIterator.java | 211 ++++++++++------- .../compaction/NewestSegmentFirstPolicy.java | 7 +- ...riorityBasedCompactionSegmentIterator.java | 31 ++- ... => PriorityBasedSegmentSearchPolicy.java} | 32 ++- .../SmallestSegmentFirstPolicy.java | 5 +- .../apache/druid/server/compaction/Table.java | 75 ++++++ .../coordinator/AutoCompactionSnapshot.java | 19 +- .../coordinator/DruidCompactionConfig.java | 4 +- .../server/coordinator/DruidCoordinator.java | 25 +- .../coordinator/duty/CompactSegments.java | 20 +- .../druid/server/http/CompactionResource.java | 29 ++- .../CompactionRunSimulatorTest.java | 95 +++++++- .../CompactionStatusTrackerTest.java | 181 ++------------ .../DruidCompactionConfigTest.java | 10 +- .../coordinator/DruidCoordinatorTest.java | 20 +- .../CoordinatorSimulationBuilder.java | 5 +- .../server/http/CompactionResourceTest.java | 32 ++- .../org/apache/druid/cli/CliCoordinator.java | 2 + .../org/apache/druid/cli/CliOverlord.java | 26 +- 34 files changed, 1243 insertions(+), 712 deletions(-) create mode 100644 indexing-service/src/main/java/org/apache/druid/guice/SupervisorModule.java rename server/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorModule.java => indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionStatusReport.java (68%) create mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionSupervisor.java create mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionSupervisorSpec.java create mode 100644 server/src/main/java/org/apache/druid/server/compaction/CompactionTaskStatus.java rename server/src/main/java/org/apache/druid/server/compaction/{BaseSegmentSearchPolicy.java => PriorityBasedSegmentSearchPolicy.java} (71%) create mode 100644 server/src/main/java/org/apache/druid/server/compaction/Table.java diff --git a/indexing-service/src/main/java/org/apache/druid/guice/SupervisorModule.java b/indexing-service/src/main/java/org/apache/druid/guice/SupervisorModule.java new file mode 100644 index 000000000000..f08f36e0682e --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/guice/SupervisorModule.java @@ -0,0 +1,52 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + + +package org.apache.druid.guice; + +import com.fasterxml.jackson.databind.Module; +import com.fasterxml.jackson.databind.jsontype.NamedType; +import com.fasterxml.jackson.databind.module.SimpleModule; +import com.google.common.collect.ImmutableList; +import com.google.inject.Binder; +import org.apache.druid.indexing.compact.CompactionSupervisorSpec; +import org.apache.druid.indexing.overlord.supervisor.SupervisorStateManagerConfig; +import org.apache.druid.initialization.DruidModule; + +import java.util.List; + +public class SupervisorModule implements DruidModule +{ + @Override + public void configure(Binder binder) + { + JsonConfigProvider.bind(binder, "druid.supervisor", SupervisorStateManagerConfig.class); + } + + @Override + public List getJacksonModules() + { + return ImmutableList.of( + new SimpleModule(getClass().getSimpleName()) + .registerSubtypes( + new NamedType(CompactionSupervisorSpec.class, CompactionSupervisorSpec.TYPE) + ) + ); + } +} diff --git a/server/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorModule.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionStatusReport.java similarity index 68% rename from server/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorModule.java rename to indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionStatusReport.java index d281f55e52c1..9096cfad64f5 100644 --- a/server/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorModule.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionStatusReport.java @@ -17,18 +17,12 @@ * under the License. */ +package org.apache.druid.indexing.compact; -package org.apache.druid.indexing.overlord.supervisor; - -import com.google.inject.Binder; -import com.google.inject.Module; -import org.apache.druid.guice.JsonConfigProvider; - -public class SupervisorModule implements Module +/** + * This can contain stats and progress and stuff. + * For simulation, we will still have the separate API. + */ +public class CompactionStatusReport { - @Override - public void configure(Binder binder) - { - JsonConfigProvider.bind(binder, "druid.supervisor", SupervisorStateManagerConfig.class); - } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionSupervisor.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionSupervisor.java new file mode 100644 index 000000000000..e924ee186115 --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionSupervisor.java @@ -0,0 +1,162 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.compact; + +import org.apache.druid.indexing.overlord.DataSourceMetadata; +import org.apache.druid.indexing.overlord.supervisor.Supervisor; +import org.apache.druid.indexing.overlord.supervisor.SupervisorReport; +import org.apache.druid.indexing.overlord.supervisor.SupervisorStateManager; +import org.apache.druid.indexing.overlord.supervisor.autoscaler.LagStats; +import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.server.compaction.CompactionScheduler; +import org.apache.druid.server.coordinator.AutoCompactionSnapshot; + +/** + * Supervisor for compaction of a single datasource. + */ +public class CompactionSupervisor implements Supervisor +{ + private static final Logger log = new Logger(CompactionSupervisor.class); + + private final CompactionScheduler scheduler; + private final CompactionSupervisorSpec supervisorSpec; + + public CompactionSupervisor( + CompactionSupervisorSpec supervisorSpec, + CompactionScheduler scheduler + ) + { + this.supervisorSpec = supervisorSpec; + this.scheduler = scheduler; + } + + @Override + public void start() + { + final String dataSource = getDataSource(); + if (supervisorSpec.isSuspended()) { + log.info("Suspending compaction for dataSource[%s].", dataSource); + scheduler.stopCompaction(dataSource); + } else { + log.info("Starting compaction for dataSource[%s].", dataSource); + scheduler.startCompaction(dataSource, supervisorSpec.getSpec()); + } + } + + @Override + public void stop(boolean stopGracefully) + { + final String dataSource = getDataSource(); + log.info("Stopping compaction for dataSource[%s].", dataSource); + scheduler.stopCompaction(dataSource); + } + + @Override + public SupervisorReport getStatus() + { + final AutoCompactionSnapshot snapshot; + if (supervisorSpec.isSuspended()) { + snapshot = AutoCompactionSnapshot.builder(getDataSource()) + .withStatus(AutoCompactionSnapshot.AutoCompactionScheduleStatus.NOT_ENABLED) + .build(); + } else { + snapshot = scheduler.getCompactionSnapshot(getDataSource()); + } + + return new SupervisorReport<>(supervisorSpec.getId(), DateTimes.nowUtc(), snapshot); + } + + @Override + public SupervisorStateManager.State getState() + { + if (!scheduler.isRunning()) { + return State.SCHEDULER_DISABLED; + } else if (supervisorSpec.isSuspended()) { + return State.SUSPENDED; + } else { + return State.RUNNING; + } + } + + private String getDataSource() + { + return supervisorSpec.getSpec().getDataSource(); + } + + // Un-implemented methods used only by streaming supervisors + + @Override + public void reset(DataSourceMetadata dataSourceMetadata) + { + // Do nothing + } + + @Override + public void resetOffsets(DataSourceMetadata resetDataSourceMetadata) + { + // Do nothing + } + + @Override + public void checkpoint(int taskGroupId, DataSourceMetadata checkpointMetadata) + { + // Do nothing + } + + @Override + public LagStats computeLagStats() + { + return new LagStats(0L, 0L, 0L); + } + + @Override + public int getActiveTaskGroupsCount() + { + return 0; + } + + public enum State implements SupervisorStateManager.State + { + SCHEDULER_DISABLED(true), + RUNNING(true), + SUSPENDED(true), + UNHEALTHY(false); + + private final boolean healthy; + + State(boolean healthy) + { + this.healthy = healthy; + } + + @Override + public boolean isFirstRunOnly() + { + return false; + } + + @Override + public boolean isHealthy() + { + return healthy; + } + } +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionSupervisorSpec.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionSupervisorSpec.java new file mode 100644 index 000000000000..f263e925f864 --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionSupervisorSpec.java @@ -0,0 +1,115 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.compact; + +import com.fasterxml.jackson.annotation.JacksonInject; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.druid.common.config.Configs; +import org.apache.druid.error.InvalidInput; +import org.apache.druid.indexing.overlord.supervisor.Supervisor; +import org.apache.druid.indexing.overlord.supervisor.SupervisorSpec; +import org.apache.druid.server.compaction.CompactionScheduler; +import org.apache.druid.server.coordinator.DataSourceCompactionConfig; + +import javax.annotation.Nullable; +import java.util.Collections; +import java.util.List; + +public class CompactionSupervisorSpec implements SupervisorSpec +{ + public static final String TYPE = "autocompact"; + public static final String ID_PREFIX = "autocompact__"; + + private final boolean suspended; + private final DataSourceCompactionConfig spec; + private final CompactionScheduler scheduler; + + @JsonCreator + public CompactionSupervisorSpec( + @JsonProperty("spec") DataSourceCompactionConfig spec, + @JsonProperty("suspended") @Nullable Boolean suspended, + @JacksonInject CompactionScheduler scheduler + ) + { + if (spec == null) { + throw InvalidInput.exception("'spec' must be specified for a compaction supervisor."); + } + + this.spec = spec; + this.suspended = Configs.valueOrDefault(suspended, false); + this.scheduler = scheduler; + } + + @JsonProperty + public DataSourceCompactionConfig getSpec() + { + return spec; + } + + @Override + @JsonProperty + public boolean isSuspended() + { + return suspended; + } + + @Override + public String getId() + { + return ID_PREFIX + spec.getDataSource(); + } + + @Override + public Supervisor createSupervisor() + { + return new CompactionSupervisor(this, scheduler); + } + + @Override + public List getDataSources() + { + return Collections.singletonList(spec.getDataSource()); + } + + @Override + public SupervisorSpec createSuspendedSpec() + { + return new CompactionSupervisorSpec(spec, true, scheduler); + } + + @Override + public SupervisorSpec createRunningSpec() + { + return new CompactionSupervisorSpec(spec, false, scheduler); + } + + @Override + public String getType() + { + return TYPE; + } + + @Override + public String getSource() + { + return ""; + } +} 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 525d34e4f277..ea563429d58a 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 @@ -21,8 +21,8 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Optional; +import com.google.common.base.Supplier; import com.google.inject.Inject; -import org.apache.druid.common.config.JacksonConfigManager; import org.apache.druid.indexer.TaskLocation; import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexing.overlord.TaskMaster; @@ -35,6 +35,7 @@ import org.apache.druid.java.util.emitter.service.ServiceEmitter; import org.apache.druid.java.util.emitter.service.ServiceMetricEvent; import org.apache.druid.metadata.SegmentsMetadataManager; +import org.apache.druid.rpc.indexing.OverlordClient; import org.apache.druid.server.compaction.CompactionRunSimulator; import org.apache.druid.server.compaction.CompactionScheduler; import org.apache.druid.server.compaction.CompactionSimulateResult; @@ -43,6 +44,7 @@ import org.apache.druid.server.coordinator.ClusterCompactionConfig; import org.apache.druid.server.coordinator.CompactionSchedulerConfig; import org.apache.druid.server.coordinator.CoordinatorOverlordServiceConfig; +import org.apache.druid.server.coordinator.DataSourceCompactionConfig; import org.apache.druid.server.coordinator.DruidCompactionConfig; import org.apache.druid.server.coordinator.duty.CompactSegments; import org.apache.druid.server.coordinator.stats.CoordinatorRunStats; @@ -51,38 +53,44 @@ import org.apache.druid.timeline.SegmentTimeline; import org.joda.time.Duration; +import java.util.ArrayList; +import java.util.Collections; import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; /** - * TODO: pending items - * - [x] add another policy - smallestSegmentFirst - * - [ ] come up with better policy names - * - [?] poll status of recently completed tasks in CompactSegments duty - * - [?] BasePolicy.shouldCompact(): locked and skip offset intervals will always be skipped - * - [ ] finalize logic of skipping turns of successful and failed - * - [ ] compaction status should have last compacted time - * - [ ] compaction status should have num uncompacted segments, bytes - * - [ ] test on cluster - standalone, coordinator-overlord - * - [ ] unit tests - * - [ ] integration tests - */ - -/** - * Compaction Scheduler that runs on the Overlord if - * {@code druid.compaction.scheduler.enabled=true}. + * Compaction scheduler that runs on the Overlord if {@link CompactionSchedulerConfig} + * is enabled. + *

+ * Usage: + *

    + *
  • When an active {@link CompactionSupervisor} starts, it should register + * itself by calling {@link #startCompaction}.
  • + *
  • When a suspended {@link CompactionSupervisor} starts, it should stop + * compaction by calling {@link #stopCompaction}.
  • + *
  • When stopping, any {@link CompactionSupervisor} (active or suspended) + * should call {@link #stopCompaction}.
  • + *
*/ public class OverlordCompactionScheduler implements CompactionScheduler { private static final Logger log = new Logger(OverlordCompactionScheduler.class); + private static final long SCHEDULE_PERIOD_SECONDS = 5; + private static final Duration METRIC_EMISSION_PERIOD = Duration.standardMinutes(5); + private final TaskMaster taskMaster; - private final JacksonConfigManager configManager; private final SegmentsMetadataManager segmentManager; + private final OverlordClient overlordClient; private final ServiceEmitter emitter; + private final CompactionSchedulerConfig schedulerConfig; + private final Supplier clusterCompactionConfigSupplier; + private final ConcurrentHashMap activeDatasourceConfigs; + /** * Single-threaded executor to process the compaction queue. */ @@ -90,7 +98,7 @@ public class OverlordCompactionScheduler implements CompactionScheduler private final TaskRunnerListener taskStateListener; private final CompactionStatusTracker statusTracker; - private final AtomicBoolean isLeader = new AtomicBoolean(false); + private final AtomicBoolean started = new AtomicBoolean(false); private final CompactSegments duty; /** @@ -101,14 +109,14 @@ public class OverlordCompactionScheduler implements CompactionScheduler private final boolean shouldPollSegments; private final Stopwatch sinceStatsEmitted = Stopwatch.createStarted(); - private final CompactionSchedulerConfig schedulerConfig; @Inject public OverlordCompactionScheduler( TaskMaster taskMaster, TaskQueryTool taskQueryTool, SegmentsMetadataManager segmentManager, - JacksonConfigManager configManager, + Supplier clusterCompactionConfigSupplier, + CompactionStatusTracker statusTracker, CompactionSchedulerConfig schedulerConfig, CoordinatorOverlordServiceConfig coordinatorOverlordServiceConfig, ScheduledExecutorFactory executorFactory, @@ -117,25 +125,25 @@ public OverlordCompactionScheduler( ) { this.taskMaster = taskMaster; - this.configManager = configManager; this.segmentManager = segmentManager; - this.statusTracker = new CompactionStatusTracker(objectMapper); this.emitter = emitter; this.schedulerConfig = schedulerConfig; + this.clusterCompactionConfigSupplier = clusterCompactionConfigSupplier; + this.executor = executorFactory.create(1, "CompactionScheduler-%s"); + this.statusTracker = statusTracker; this.shouldPollSegments = segmentManager != null && !coordinatorOverlordServiceConfig.isEnabled(); - this.duty = new CompactSegments( - statusTracker, - new LocalOverlordClient(taskMaster, taskQueryTool, objectMapper) - ); + this.overlordClient = new LocalOverlordClient(taskMaster, taskQueryTool, objectMapper); + this.duty = new CompactSegments(this.statusTracker, overlordClient); + this.activeDatasourceConfigs = new ConcurrentHashMap<>(); this.taskStateListener = new TaskRunnerListener() { @Override public String getListenerId() { - return "CompactionScheduler"; + return "CompactionSupervisorManager"; } @Override @@ -147,42 +155,52 @@ public void locationChanged(String taskId, TaskLocation newLocation) @Override public void statusChanged(String taskId, TaskStatus status) { - runOnExecutor(() -> statusTracker.onTaskFinished(taskId, status)); + runOnExecutor(() -> OverlordCompactionScheduler.this.statusTracker.onTaskFinished(taskId, status)); } }; } @Override - public void becomeLeader() + public void start() { - if (isEnabled() && isLeader.compareAndSet(false, true)) { - log.info("Starting compaction scheduler as we are now the leader."); - runOnExecutor(() -> { - initState(); - checkSchedulingStatus(); - }); + if (isEnabled() && started.compareAndSet(false, true)) { + log.info("Starting compaction scheduler."); + scheduleOnExecutor( + () -> { + initState(); + checkSchedulingStatus(); + }, + SCHEDULE_PERIOD_SECONDS + ); } } @Override - public void stopBeingLeader() + public void stop() { - if (isEnabled() && isLeader.compareAndSet(true, false)) { - log.info("Stopping compaction scheduler as we are not the leader anymore."); + if (isEnabled() && started.compareAndSet(true, false)) { + log.info("Stopping compaction scheduler."); runOnExecutor(this::cleanupState); } } - private void runOnExecutor(Runnable runnable) + @Override + public boolean isRunning() { - executor.submit(() -> { - try { - runnable.run(); - } - catch (Throwable t) { - log.error(t, "Error while executing runnable"); - } - }); + return isEnabled() && started.get(); + } + + @Override + public void startCompaction(String dataSourceName, DataSourceCompactionConfig config) + { + activeDatasourceConfigs.put(dataSourceName, config); + } + + @Override + public void stopCompaction(String dataSourceName) + { + activeDatasourceConfigs.remove(dataSourceName); + statusTracker.removeDatasource(dataSourceName); } private synchronized void initState() @@ -201,7 +219,8 @@ private synchronized void initState() private synchronized void cleanupState() { - statusTracker.reset(); + statusTracker.stop(); + activeDatasourceConfigs.clear(); Optional taskRunner = taskMaster.getTaskRunner(); if (taskRunner.isPresent()) { @@ -220,33 +239,26 @@ private boolean isEnabled() private synchronized void checkSchedulingStatus() { - if (isLeader.get() && isEnabled()) { + if (started.get() && isEnabled()) { try { - processCompactionQueue(getLatestConfig()); + runCompactionDuty(); } catch (Exception e) { log.error(e, "Error processing compaction queue. Continuing schedule."); } - executor.schedule(this::checkSchedulingStatus, 5, TimeUnit.SECONDS); + scheduleOnExecutor(this::checkSchedulingStatus, SCHEDULE_PERIOD_SECONDS); } else { cleanupState(); } } - private synchronized void processCompactionQueue( - DruidCompactionConfig currentConfig - ) + private synchronized void runCompactionDuty() { final CoordinatorRunStats stats = new CoordinatorRunStats(); + duty.run(getLatestConfig(), getCurrentDatasourceTimelines(), stats); - duty.run( - currentConfig, - getCurrentDatasourceTimelines(), - stats - ); - - // Emit stats only every 5 minutes - if (sinceStatsEmitted.hasElapsed(Duration.standardMinutes(5))) { + // Emit stats only if emission period has elapsed + if (sinceStatsEmitted.hasElapsed(METRIC_EMISSION_PERIOD)) { stats.forEachStat( (stat, dimensions, value) -> { if (stat.shouldEmit()) { @@ -258,6 +270,37 @@ private synchronized void processCompactionQueue( } } + @Override + public AutoCompactionSnapshot getCompactionSnapshot(String dataSource) + { + return duty.getAutoCompactionSnapshot(dataSource); + } + + @Override + public Long getTotalSizeOfSegmentsAwaitingCompaction(String dataSource) + { + return duty.getTotalSizeOfSegmentsAwaitingCompaction(dataSource); + } + + @Override + public Map getAllCompactionSnapshots() + { + return duty.getAutoCompactionSnapshot(); + } + + @Override + public CompactionSimulateResult simulateRunWithConfigUpdate(ClusterCompactionConfig updateRequest) + { + if (isRunning()) { + return new CompactionRunSimulator(statusTracker, overlordClient).simulateRunWithConfig( + getLatestConfig().withClusterConfig(updateRequest), + getCurrentDatasourceTimelines() + ); + } else { + return new CompactionSimulateResult(Collections.emptyMap()); + } + } + private void emitStat(CoordinatorStat stat, Map dimensionValues, long value) { ServiceMetricEvent.Builder eventBuilder = new ServiceMetricEvent.Builder(); @@ -269,11 +312,10 @@ private void emitStat(CoordinatorStat stat, Map dimensionValu private DruidCompactionConfig getLatestConfig() { - return configManager.watch( - DruidCompactionConfig.CONFIG_KEY, - DruidCompactionConfig.class, - DruidCompactionConfig.empty() - ).get(); + return DruidCompactionConfig + .empty() + .withClusterConfig(clusterCompactionConfigSupplier.get()) + .withDatasourceConfigs(new ArrayList<>(activeDatasourceConfigs.values())); } private Map getCurrentDatasourceTimelines() @@ -282,31 +324,31 @@ private Map getCurrentDatasourceTimelines() .getUsedSegmentsTimelinesPerDataSource(); } - @Override - public AutoCompactionSnapshot getCompactionSnapshot(String dataSource) - { - return duty.getAutoCompactionSnapshot(dataSource); - } - - @Override - public Long getTotalSizeOfSegmentsAwaitingCompaction(String dataSource) + private void scheduleOnExecutor(Runnable runnable, long delaySeconds) { - return duty.getTotalSizeOfSegmentsAwaitingCompaction(dataSource); - } - - @Override - public Map getAllCompactionSnapshots() - { - return duty.getAutoCompactionSnapshot(); + executor.schedule( + () -> { + try { + runnable.run(); + } + catch (Throwable t) { + log.error(t, "Error while executing runnable"); + } + }, + delaySeconds, + TimeUnit.SECONDS + ); } - @Override - public CompactionSimulateResult simulateRunWithConfigUpdate(ClusterCompactionConfig updateRequest) + private void runOnExecutor(Runnable runnable) { - return new CompactionRunSimulator( - statusTracker, - getLatestConfig(), - getCurrentDatasourceTimelines() - ).simulateRunWithConfigUpdate(updateRequest); + executor.submit(() -> { + try { + runnable.run(); + } + catch (Throwable t) { + log.error(t, "Error while executing runnable"); + } + }); } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/DruidOverlord.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/DruidOverlord.java index eb4067b05237..11705fed32f8 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/DruidOverlord.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/DruidOverlord.java @@ -29,7 +29,6 @@ import org.apache.druid.indexing.common.actions.SegmentAllocationQueue; import org.apache.druid.indexing.common.actions.TaskActionClientFactory; import org.apache.druid.indexing.common.task.TaskContextEnricher; -import org.apache.druid.indexing.compact.OverlordCompactionScheduler; import org.apache.druid.indexing.overlord.config.DefaultTaskConfig; import org.apache.druid.indexing.overlord.config.TaskLockConfig; import org.apache.druid.indexing.overlord.config.TaskQueueConfig; @@ -41,6 +40,7 @@ import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.java.util.emitter.service.ServiceEmitter; import org.apache.druid.server.DruidNode; +import org.apache.druid.server.compaction.CompactionScheduler; import org.apache.druid.server.coordinator.CoordinatorOverlordServiceConfig; import java.util.concurrent.atomic.AtomicReference; @@ -88,7 +88,7 @@ public DruidOverlord( final OverlordDutyExecutor overlordDutyExecutor, @IndexingService final DruidLeaderSelector overlordLeaderSelector, final SegmentAllocationQueue segmentAllocationQueue, - final OverlordCompactionScheduler compactionScheduler, + final CompactionScheduler compactionScheduler, final ObjectMapper mapper, final TaskContextEnricher taskContextEnricher ) @@ -142,7 +142,7 @@ public void start() { segmentAllocationQueue.becomeLeader(); taskMaster.becomeLeader(taskRunner, taskQueue); - compactionScheduler.becomeLeader(); + compactionScheduler.start(); // Announce the node only after all the services have been initialized initialized = true; @@ -153,7 +153,7 @@ public void start() public void stop() { serviceAnnouncer.unannounce(node); - compactionScheduler.stopBeingLeader(); + compactionScheduler.stop(); taskMaster.stopBeingLeader(); segmentAllocationQueue.stopBeingLeader(); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordCompactionResource.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordCompactionResource.java index 98719474411d..30b655650d90 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordCompactionResource.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordCompactionResource.java @@ -22,6 +22,7 @@ import com.google.inject.Inject; import com.sun.jersey.spi.container.ResourceFilters; import org.apache.druid.indexing.compact.OverlordCompactionScheduler; +import org.apache.druid.server.compaction.CompactionScheduler; import org.apache.druid.server.coordinator.AutoCompactionSnapshot; import org.apache.druid.server.coordinator.ClusterCompactionConfig; import org.apache.druid.server.http.security.StateResourceFilter; @@ -44,11 +45,11 @@ @Path("/druid/indexer/v1/compaction") public class OverlordCompactionResource { - private final OverlordCompactionScheduler scheduler; + private final CompactionScheduler scheduler; @Inject public OverlordCompactionResource( - OverlordCompactionScheduler scheduler + CompactionScheduler scheduler ) { this.scheduler = scheduler; diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/http/OverlordTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/http/OverlordTest.java index 6bc6643dd11e..05a65bb32050 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/http/OverlordTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/http/OverlordTest.java @@ -47,7 +47,6 @@ import org.apache.druid.indexing.common.task.NoopTask; import org.apache.druid.indexing.common.task.NoopTaskContextEnricher; import org.apache.druid.indexing.common.task.Task; -import org.apache.druid.indexing.compact.OverlordCompactionScheduler; import org.apache.druid.indexing.overlord.DruidOverlord; import org.apache.druid.indexing.overlord.HeapMemoryTaskStorage; import org.apache.druid.indexing.overlord.IndexerMetadataStorageAdapter; @@ -75,6 +74,7 @@ import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.java.util.emitter.service.ServiceEmitter; import org.apache.druid.server.DruidNode; +import org.apache.druid.server.compaction.CompactionScheduler; import org.apache.druid.server.coordinator.CoordinatorOverlordServiceConfig; import org.apache.druid.server.metrics.NoopServiceEmitter; import org.apache.druid.server.security.AuthConfig; @@ -258,7 +258,7 @@ public MockTaskRunner get() EasyMock.createNiceMock(OverlordDutyExecutor.class), new TestDruidLeaderSelector(), EasyMock.createNiceMock(SegmentAllocationQueue.class), - EasyMock.createNiceMock(OverlordCompactionScheduler.class), + EasyMock.createNiceMock(CompactionScheduler.class), new DefaultObjectMapper(), new NoopTaskContextEnricher() ); 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 f7853204a122..7c7fa9c2b74b 100644 --- a/server/src/main/java/org/apache/druid/server/compaction/CompactionRunSimulator.java +++ b/server/src/main/java/org/apache/druid/server/compaction/CompactionRunSimulator.java @@ -22,6 +22,7 @@ import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; import org.apache.druid.client.indexing.ClientCompactionTaskQuery; +import org.apache.druid.client.indexing.ClientCompactionTaskQueryTuningConfig; import org.apache.druid.client.indexing.IndexingTotalWorkerCapacityInfo; import org.apache.druid.client.indexing.IndexingWorkerInfo; import org.apache.druid.client.indexing.TaskPayloadResponse; @@ -30,7 +31,6 @@ import org.apache.druid.indexer.TaskStatusPlus; import org.apache.druid.indexer.report.TaskReport; import org.apache.druid.indexing.overlord.supervisor.SupervisorStatus; -import org.apache.druid.java.util.common.CloseableIterators; import org.apache.druid.java.util.common.parsers.CloseableIterator; import org.apache.druid.metadata.LockFilterPolicy; import org.apache.druid.rpc.ServiceRetryPolicy; @@ -46,8 +46,7 @@ import javax.annotation.Nullable; import java.net.URI; import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; +import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Set; @@ -58,66 +57,65 @@ */ public class CompactionRunSimulator { - private final DruidCompactionConfig compactionConfig; private final CompactionStatusTracker statusTracker; - private final Map datasourceTimelines; - private final OverlordClient emptyOverlordClient = new EmptyOverlordClient(); + private final OverlordClient readOnlyOverlordClient; public CompactionRunSimulator( CompactionStatusTracker statusTracker, - DruidCompactionConfig compactionConfig, - Map datasourceTimelines + OverlordClient overlordClient ) { this.statusTracker = statusTracker; - this.datasourceTimelines = datasourceTimelines; - this.compactionConfig = compactionConfig; + this.readOnlyOverlordClient = new ReadOnlyOverlordClient(overlordClient); } /** - * Simulates a run of the compact segments duty with the given config update + * Simulates a run of the compact segments duty with the given compaction config * assuming unlimited compaction task slots. */ - public CompactionSimulateResult simulateRunWithConfigUpdate( - ClusterCompactionConfig updateRequest + public CompactionSimulateResult simulateRunWithConfig( + DruidCompactionConfig compactionConfig, + Map datasourceTimelines ) { - final ClusterCompactionConfig updateWithUnlimitedSlots = new ClusterCompactionConfig( - 1.0, - Integer.MAX_VALUE, - updateRequest.getUseAutoScaleSlots(), - updateRequest.getEngine(), - updateRequest.getCompactionPolicy() - ); - final DruidCompactionConfig configWithUnlimitedTaskSlots - = compactionConfig.withClusterConfig(updateWithUnlimitedSlots); + final Table compactedIntervals + = Table.withColumnNames("dataSource", "interval", "numSegments", "bytes"); + final Table runningIntervals + = Table.withColumnNames("dataSource", "interval", "numSegments", "bytes", "maxTaskSlots", "reasonToCompact"); + final Table queuedIntervals + = Table.withColumnNames("dataSource", "interval", "numSegments", "bytes", "maxTaskSlots", "reasonToCompact"); + final Table skippedIntervals + = Table.withColumnNames("dataSource", "interval", "numSegments", "bytes", "reasonToSkip"); - final List> tableOfCompactibleIntervals = new ArrayList<>(); - final List> tableOfSkippedIntervals = new ArrayList<>(); + // Add a wrapper over the status tracker to add intervals to respective tables final CompactionStatusTracker simulationStatusTracker = new CompactionStatusTracker(null) { @Override - public CompactionStatus computeCompactionStatus( - SegmentsToCompact candidateSegments, - DataSourceCompactionConfig config - ) + public CompactionTaskStatus getLatestTaskStatus(SegmentsToCompact candidates) { - return statusTracker.computeCompactionStatus(candidateSegments, config); + return statusTracker.getLatestTaskStatus(candidates); } @Override - public void onSegmentsSkipped(SegmentsToCompact candidateSegments, CompactionStatus status) + public void onCompactionStatusComputed( + SegmentsToCompact candidateSegments, + DataSourceCompactionConfig config + ) { - // Add a row for each skipped interval - tableOfSkippedIntervals.add( - Arrays.asList( - candidateSegments.getDataSource(), - candidateSegments.getUmbrellaInterval(), - candidateSegments.size(), - candidateSegments.getTotalBytes(), - status.getReason() - ) - ); + final CompactionStatus status = candidateSegments.getCompactionStatus(); + 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()) + ); + } } @Override @@ -125,131 +123,159 @@ public void onTaskSubmitted(ClientCompactionTaskQuery taskPayload, SegmentsToCom { // Add a row for each task in order of submission final CompactionStatus status = candidateSegments.getCompactionStatus(); - final String reason = status == null ? "" : status.getReason(); - tableOfCompactibleIntervals.add( - Arrays.asList( - candidateSegments.getDataSource(), - candidateSegments.getUmbrellaInterval(), - candidateSegments.size(), - candidateSegments.getTotalBytes(), - CompactSegments.findMaxNumTaskSlotsUsedByOneNativeCompactionTask(taskPayload.getTuningConfig()), - reason - ) + queuedIntervals.addRow( + createRow(candidateSegments, taskPayload.getTuningConfig(), status.getReason()) ); } }; + // Unlimited task slots to ensure that simulator does not skip any interval + final DruidCompactionConfig configWithUnlimitedTaskSlots = compactionConfig.withClusterConfig( + new ClusterCompactionConfig(1.0, Integer.MAX_VALUE, null, null, null) + ); + final CoordinatorRunStats stats = new CoordinatorRunStats(); - new CompactSegments(simulationStatusTracker, emptyOverlordClient).run( + new CompactSegments(simulationStatusTracker, readOnlyOverlordClient).run( configWithUnlimitedTaskSlots, datasourceTimelines, stats ); - // Add header rows - if (!tableOfCompactibleIntervals.isEmpty()) { - tableOfCompactibleIntervals.add( - 0, - Arrays.asList("dataSource", "interval", "numSegments", "bytes", "maxTaskSlots", "reasonToCompact") - ); + final Map compactionStates = new HashMap<>(); + if (!compactedIntervals.isEmpty()) { + compactionStates.put(CompactionStatus.State.COMPLETE, compactedIntervals); } - if (!tableOfSkippedIntervals.isEmpty()) { - tableOfSkippedIntervals.add( - 0, - Arrays.asList("dataSource", "interval", "numSegments", "bytes", "reasonToSkip") - ); + 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); } - return new CompactionSimulateResult(tableOfCompactibleIntervals, tableOfSkippedIntervals); + return new CompactionSimulateResult(compactionStates); + } + + private Object[] createRow( + SegmentsToCompact candidate, + ClientCompactionTaskQueryTuningConfig tuningConfig, + String reason + ) + { + final List row = new ArrayList<>(); + row.add(candidate.getDataSource()); + row.add(candidate.getUmbrellaInterval()); + row.add(candidate.size()); + row.add(candidate.getTotalBytes()); + if (tuningConfig != null) { + row.add(CompactSegments.findMaxNumTaskSlotsUsedByOneNativeCompactionTask(tuningConfig)); + } + if (reason != null) { + row.add(reason); + } + + return row.toArray(new Object[0]); } /** * Dummy overlord client that returns empty results for all APIs. */ - private static class EmptyOverlordClient implements OverlordClient + private static class ReadOnlyOverlordClient implements OverlordClient { + final OverlordClient delegate; + + ReadOnlyOverlordClient(OverlordClient delegate) + { + this.delegate = delegate; + } + @Override - public ListenableFuture findCurrentLeader() + public ListenableFuture> taskStatuses( + @Nullable String state, + @Nullable String dataSource, + @Nullable Integer maxCompletedTasks + ) { - return null; + return delegate.taskStatuses(state, dataSource, maxCompletedTasks); } @Override - public ListenableFuture runTask(String taskId, Object taskObject) + public ListenableFuture> taskStatuses(Set taskIds) { - return Futures.immediateVoidFuture(); + return delegate.taskStatuses(taskIds); } @Override - public ListenableFuture cancelTask(String taskId) + public ListenableFuture taskPayload(String taskId) { - return Futures.immediateVoidFuture(); + return delegate.taskPayload(taskId); } @Override - public ListenableFuture> taskStatuses( - @Nullable String state, - @Nullable String dataSource, - @Nullable Integer maxCompletedTasks - ) + public ListenableFuture>> findLockedIntervals(List lockFilterPolicies) { - return Futures.immediateFuture(CloseableIterators.withEmptyBaggage(Collections.emptyIterator())); + return delegate.findLockedIntervals(lockFilterPolicies); } @Override - public ListenableFuture> taskStatuses(Set taskIds) + public ListenableFuture getTotalWorkerCapacity() { - return Futures.immediateFuture(Collections.emptyMap()); + // Unlimited worker capacity to ensure that simulator does not skip any interval + return Futures.immediateFuture( + new IndexingTotalWorkerCapacityInfo(Integer.MAX_VALUE, Integer.MAX_VALUE) + ); } @Override - public ListenableFuture taskStatus(String taskId) + public ListenableFuture runTask(String taskId, Object taskObject) { - return null; + return Futures.immediateVoidFuture(); } @Override - public ListenableFuture taskReportAsMap(String taskId) + public ListenableFuture cancelTask(String taskId) { - return null; + return Futures.immediateVoidFuture(); } + // Unsupported methods as these are not used by the CompactionScheduler / CompactSegments duty + @Override - public ListenableFuture taskPayload(String taskId) + public ListenableFuture findCurrentLeader() { - return Futures.immediateFuture(null); + throw new UnsupportedOperationException(); } @Override - public ListenableFuture> supervisorStatuses() + public ListenableFuture taskStatus(String taskId) { - return null; + throw new UnsupportedOperationException(); } @Override - public ListenableFuture>> findLockedIntervals(List lockFilterPolicies) + public ListenableFuture taskReportAsMap(String taskId) { - return Futures.immediateFuture(Collections.emptyMap()); + throw new UnsupportedOperationException(); } @Override - public ListenableFuture killPendingSegments(String dataSource, Interval interval) + public ListenableFuture> supervisorStatuses() { - return null; + throw new UnsupportedOperationException(); } @Override - public ListenableFuture> getWorkers() + public ListenableFuture killPendingSegments(String dataSource, Interval interval) { - return Futures.immediateFuture(Collections.emptyList()); + throw new UnsupportedOperationException(); } @Override - public ListenableFuture getTotalWorkerCapacity() + public ListenableFuture> getWorkers() { - return Futures.immediateFuture( - new IndexingTotalWorkerCapacityInfo(Integer.MAX_VALUE, Integer.MAX_VALUE) - ); + throw new UnsupportedOperationException(); } @Override diff --git a/server/src/main/java/org/apache/druid/server/compaction/CompactionScheduler.java b/server/src/main/java/org/apache/druid/server/compaction/CompactionScheduler.java index 10989bb45d4b..3807ea634356 100644 --- a/server/src/main/java/org/apache/druid/server/compaction/CompactionScheduler.java +++ b/server/src/main/java/org/apache/druid/server/compaction/CompactionScheduler.java @@ -21,14 +21,21 @@ import org.apache.druid.server.coordinator.AutoCompactionSnapshot; import org.apache.druid.server.coordinator.ClusterCompactionConfig; +import org.apache.druid.server.coordinator.DataSourceCompactionConfig; import java.util.Map; public interface CompactionScheduler { - void becomeLeader(); + void start(); - void stopBeingLeader(); + void stop(); + + boolean isRunning(); + + void startCompaction(String dataSourceName, DataSourceCompactionConfig compactionConfig); + + void stopCompaction(String dataSourceName); Map getAllCompactionSnapshots(); 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 8e01d054b19e..30b5390bff63 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 @@ -19,31 +19,25 @@ package org.apache.druid.server.compaction; -import org.apache.druid.timeline.DataSegment; - import java.util.Iterator; -import java.util.Map; +import java.util.List; /** - * Segments in the lists which are the elements of this iterator are sorted according to the natural segment order - * (see {@link DataSegment#compareTo}). + * Iterator over compactible segments. */ public interface CompactionSegmentIterator extends Iterator { /** - * Return a map of dataSourceName to CompactionStatistics. - * This method returns the aggregated statistics of segments that was already compacted and does not need to be compacted - * again. Hence, segment that were not returned by the {@link Iterator#next()} becuase it does not needs compaction. - * Note that the aggregations returned by this method is only up to the current point of the iterator being iterated. + * List of candidate segments that are already compacted and do not need to be + * compacted again. None of these segments are returned by {@link #next()}. */ - Map totalCompactedStatistics(); + List getCompactedSegments(); /** - * Return a map of dataSourceName to CompactionStatistics. - * This method returns the aggregated statistics of segments that was skipped as it cannot be compacted. - * Hence, segment that were not returned by the {@link Iterator#next()} becuase it cannot be compacted. - * Note that the aggregations returned by this method is only up to the current point of the iterator being iterated. + * List of candidate segments that have been skipped for compaction as they + * cannot be compacted due to some reason. None of these segments are returned + * by {@link #next()}. */ - Map totalSkippedStatistics(); + 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 9d840ae5eb16..7a48ccf0e5ba 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 @@ -22,32 +22,23 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import java.util.List; +import java.util.Map; public class CompactionSimulateResult { - private final List> intervalsToCompact; - private final List> skippedIntervals; + private final Map compactionStates; @JsonCreator public CompactionSimulateResult( - @JsonProperty("intervalsToCompact") List> intervalsToCompact, - @JsonProperty("skippedIntervals") List> skippedIntervals + @JsonProperty("compactionStates") Map compactionStates ) { - this.intervalsToCompact = intervalsToCompact; - this.skippedIntervals = skippedIntervals; + this.compactionStates = compactionStates; } @JsonProperty - public List> getIntervalsToCompact() + public Map getCompactionStates() { - return intervalsToCompact; - } - - @JsonProperty - public List> getSkippedIntervals() - { - return skippedIntervals; + return compactionStates; } } 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 a6078d880af5..bbb5cd831cc8 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 @@ -53,9 +53,9 @@ public class CompactionStatus { private static final CompactionStatus COMPLETE = new CompactionStatus(State.COMPLETE, null); - private enum State + public enum State { - COMPLETE, PENDING, SKIPPED + COMPLETE, PENDING, RUNNING, SKIPPED } /** @@ -100,6 +100,11 @@ public String getReason() return reason; } + public State getState() + { + return state; + } + private static CompactionStatus incomplete(String reasonFormat, Object... args) { return new CompactionStatus(State.PENDING, StringUtils.format(reasonFormat, args)); @@ -177,6 +182,11 @@ static CompactionStatus skipped(String reasonFormat, Object... args) return new CompactionStatus(State.SKIPPED, StringUtils.format(reasonFormat, args)); } + static CompactionStatus running(String reasonForCompaction) + { + return new CompactionStatus(State.RUNNING, reasonForCompaction); + } + /** * Determines the CompactionStatus of the given candidate segments by evaluating * the {@link #CHECKS} one by one. If any check returns an incomplete status, 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 de1e506c56d8..930db6f9ae15 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 @@ -20,11 +20,14 @@ package org.apache.druid.server.compaction; import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.inject.Inject; import org.apache.druid.client.indexing.ClientCompactionTaskQuery; +import org.apache.druid.indexer.TaskState; import org.apache.druid.indexer.TaskStatus; -import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; import org.apache.druid.server.coordinator.DruidCompactionConfig; +import org.joda.time.DateTime; import org.joda.time.Interval; import java.util.HashMap; @@ -33,80 +36,51 @@ import java.util.Set; /** - * Tracks status of both recently submitted compaction tasks and the compaction - * state of segments. Can be used to check if a set of segments is currently - * eligible for compaction. - *

- * TODO: Keep the interval sidelined until - * - no other interval to compact for the last 1 hour - * - some other intervals have been submitted - * - threshold has been crossed - 30% uncompacted bytes or 50% uncompacted segments - * - interval is uncompacted and has just 1 uncompacted segment out of 100 - * - level of uncompaction is important to know - * - should this be a part of policy - * - * - * - A policy that picks up stuff only if it meets some thresholds and then + * Tracks status of recently submitted compaction tasks. Can be used by a segment + * search policy to skip an interval if it has been recently compacted or if it + * keeps failing repeatedly. */ public class CompactionStatusTracker { - private static final Logger log = new Logger(CompactionStatusTracker.class); - - private static final int MAX_FAILURE_RETRIES = 3; - private static final int MAX_SKIPS_AFTER_SUCCESS = 5; - private static final int MAX_SKIPS_AFTER_FAILURE = 5; - private final ObjectMapper objectMapper; private final Map datasourceStatuses = new HashMap<>(); private final Map submittedTaskIdToSegments = new HashMap<>(); + @Inject public CompactionStatusTracker(ObjectMapper objectMapper) { this.objectMapper = objectMapper; } - public CompactionStatus computeCompactionStatus( - SegmentsToCompact candidate, - DataSourceCompactionConfig config - ) + public void stop() { - final CompactionStatus compactionStatus = CompactionStatus.compute(candidate, config, objectMapper); - if (compactionStatus.isComplete()) { - return compactionStatus; - } - - final long inputSegmentSize = config.getInputSegmentSizeBytes(); - if (candidate.getTotalBytes() > inputSegmentSize) { - return CompactionStatus.skipped( - "'inputSegmentSize' exceeded: Total segment size[%d] is larger than allowed inputSegmentSize[%d]", - candidate.getTotalBytes(), inputSegmentSize - ); - } - - final Interval compactionInterval = candidate.getUmbrellaInterval(); + datasourceStatuses.clear(); + } - final IntervalStatus intervalStatus = datasourceStatuses - .getOrDefault(config.getDataSource(), DatasourceStatus.EMPTY) - .intervalStatus - .get(compactionInterval); + public ObjectMapper getObjectMapper() + { + return objectMapper; + } - if (intervalStatus == null) { - return compactionStatus; - } + public void removeDatasource(String datasource) + { + datasourceStatuses.remove(datasource); + } - switch (intervalStatus.state) { - case TASK_SUBMITTED: - case COMPACTED: - case FAILED_ALL_RETRIES: - return CompactionStatus.skipped( - "recently submitted: current compaction state[%s]", - intervalStatus.state - ); - default: - break; - } + public CompactionTaskStatus getLatestTaskStatus(SegmentsToCompact candidates) + { + return datasourceStatuses + .getOrDefault(candidates.getDataSource(), DatasourceStatus.EMPTY) + .intervalToTaskStatus + .get(candidates.getUmbrellaInterval()); + } - return compactionStatus; + public void onCompactionStatusComputed( + SegmentsToCompact candidateSegments, + DataSourceCompactionConfig config + ) + { + // Nothing to do, used by simulator } public void onCompactionConfigUpdated(DruidCompactionConfig compactionConfig) @@ -127,19 +101,6 @@ public void onCompactionConfigUpdated(DruidCompactionConfig compactionConfig) }); } - public void onSegmentsCompacted(SegmentsToCompact candidateSegments) - { - // do nothing - } - - public void onSegmentsSkipped( - SegmentsToCompact candidateSegments, - CompactionStatus status - ) - { - // do nothing - } - public void onTaskSubmitted( ClientCompactionTaskQuery taskPayload, SegmentsToCompact candidateSegments @@ -164,12 +125,7 @@ public void onTaskFinished(String taskId, TaskStatus taskStatus) final Interval compactionInterval = candidateSegments.getUmbrellaInterval(); getOrComputeDatasourceStatus(candidateSegments.getDataSource()) - .handleTaskStatus(compactionInterval, taskStatus); - } - - public void reset() - { - datasourceStatuses.clear(); + .handleCompletedTask(compactionInterval, taskStatus); } private DatasourceStatus getOrComputeDatasourceStatus(String datasource) @@ -181,77 +137,43 @@ private static class DatasourceStatus { static final DatasourceStatus EMPTY = new DatasourceStatus(); - final Map intervalStatus = new HashMap<>(); + final Map intervalToTaskStatus = new HashMap<>(); - void handleTaskStatus(Interval compactionInterval, TaskStatus taskStatus) + void handleCompletedTask(Interval compactionInterval, TaskStatus taskStatus) { - final IntervalStatus lastKnownStatus = intervalStatus.get(compactionInterval); + final CompactionTaskStatus lastKnownStatus = intervalToTaskStatus.get(compactionInterval); + final DateTime now = DateTimes.nowUtc(); + final CompactionTaskStatus updatedStatus; if (taskStatus.isSuccess()) { - intervalStatus.put( - compactionInterval, - new IntervalStatus(IntervalState.COMPACTED, MAX_SKIPS_AFTER_SUCCESS) - ); - } else if (lastKnownStatus == null || !lastKnownStatus.isFailed()) { + updatedStatus = new CompactionTaskStatus(TaskState.SUCCESS, now, 0); + } else if (lastKnownStatus == null || lastKnownStatus.getState().isSuccess()) { // This is the first failure - intervalStatus.put( - compactionInterval, - new IntervalStatus(IntervalState.FAILED, 0) - ); - } else if (++lastKnownStatus.retryCount >= MAX_FAILURE_RETRIES) { - // Failure retries have been exhausted - intervalStatus.put( - compactionInterval, - new IntervalStatus(IntervalState.FAILED_ALL_RETRIES, MAX_SKIPS_AFTER_FAILURE) + updatedStatus = new CompactionTaskStatus(TaskState.FAILED, now, 1); + } else { + updatedStatus = new CompactionTaskStatus( + TaskState.FAILED, + now, + lastKnownStatus.getNumConsecutiveFailures() + 1 ); } + intervalToTaskStatus.put(compactionInterval, updatedStatus); } void handleSubmittedTask(SegmentsToCompact candidateSegments) { - intervalStatus.computeIfAbsent( - candidateSegments.getUmbrellaInterval(), - i -> new IntervalStatus(IntervalState.TASK_SUBMITTED, 0) - ); - - final Set readyIntervals = new HashSet<>(); - intervalStatus.forEach((interval, status) -> { - status.turnsToSkip--; - if (status.isReady()) { - readyIntervals.add(interval); - } - }); - - readyIntervals.forEach(intervalStatus::remove); - } - } - - private static class IntervalStatus - { - final IntervalState state; - int turnsToSkip; - int retryCount; - - IntervalStatus(IntervalState state, int turnsToSkip) - { - this.state = state; - this.turnsToSkip = turnsToSkip; - } - - boolean isReady() - { - return turnsToSkip <= 0 - && (state == IntervalState.COMPACTED || state == IntervalState.FAILED_ALL_RETRIES); - } - - boolean isFailed() - { - return state == IntervalState.FAILED || state == IntervalState.FAILED_ALL_RETRIES; + final Interval interval = candidateSegments.getUmbrellaInterval(); + final CompactionTaskStatus lastStatus = intervalToTaskStatus.get(interval); + + final DateTime now = DateTimes.nowUtc(); + if (lastStatus == null || !lastStatus.getState().isFailure()) { + intervalToTaskStatus.put(interval, new CompactionTaskStatus(TaskState.RUNNING, now, 0)); + } else { + intervalToTaskStatus.put( + interval, + new CompactionTaskStatus(TaskState.RUNNING, now, lastStatus.getNumConsecutiveFailures()) + ); + } } } - - private enum IntervalState - { - TASK_SUBMITTED, COMPACTED, FAILED, FAILED_ALL_RETRIES - } } diff --git a/server/src/main/java/org/apache/druid/server/compaction/CompactionTaskStatus.java b/server/src/main/java/org/apache/druid/server/compaction/CompactionTaskStatus.java new file mode 100644 index 000000000000..3431de706f69 --- /dev/null +++ b/server/src/main/java/org/apache/druid/server/compaction/CompactionTaskStatus.java @@ -0,0 +1,56 @@ +/* + * 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.indexer.TaskState; +import org.joda.time.DateTime; + +public class CompactionTaskStatus +{ + private final TaskState state; + private final DateTime updatedTime; + private final int numConsecutiveFailures; + + public CompactionTaskStatus( + TaskState state, + DateTime updatedTime, + int numConsecutiveFailures + ) + { + this.state = state; + this.updatedTime = updatedTime; + this.numConsecutiveFailures = numConsecutiveFailures; + } + + public TaskState getState() + { + return state; + } + + public DateTime getUpdatedTime() + { + return updatedTime; + } + + public int getNumConsecutiveFailures() + { + return numConsecutiveFailures; + } +} 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 98a83976c6ff..6b2cb8099f23 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 @@ -22,6 +22,7 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; import com.google.common.collect.Lists; +import org.apache.druid.indexer.TaskState; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.JodaUtils; @@ -59,15 +60,19 @@ /** * Iterator over compactible segments of a datasource in order of specified priority. */ -public class DataSourceCompactibleSegmentIterator implements Iterator +public class DataSourceCompactibleSegmentIterator implements CompactionSegmentIterator { private static final Logger log = new Logger(DataSourceCompactibleSegmentIterator.class); private final String dataSource; private final DataSourceCompactionConfig config; private final CompactionStatusTracker statusTracker; - private final CompactionStatistics compactedSegmentStats = new CompactionStatistics(); - private final CompactionStatistics skippedSegmentStats = new CompactionStatistics(); + private final PriorityBasedSegmentSearchPolicy searchPolicy; + + private final Granularity configuredSegmentGranularity; + + private final List compactedSegments = 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 @@ -81,45 +86,68 @@ public DataSourceCompactibleSegmentIterator( DataSourceCompactionConfig config, SegmentTimeline timeline, List skipIntervals, - Comparator segmentPriority, + PriorityBasedSegmentSearchPolicy searchPolicy, CompactionStatusTracker statusTracker ) { this.statusTracker = statusTracker; this.config = config; this.dataSource = config.getDataSource(); - this.queue = new PriorityQueue<>(segmentPriority); + this.searchPolicy = searchPolicy; + this.queue = new PriorityQueue<>(searchPolicy.getSegmentComparator()); + + if (config.getGranularitySpec() == null || config.getGranularitySpec().getSegmentGranularity() == null) { + this.configuredSegmentGranularity = null; + } else { + this.configuredSegmentGranularity = config.getGranularitySpec().getSegmentGranularity(); + } + populateQueue(timeline, skipIntervals); } private void populateQueue(SegmentTimeline timeline, List skipIntervals) { if (timeline != null) { - Granularity configuredSegmentGranularity = null; if (!timeline.isEmpty()) { SegmentTimeline originalTimeline = null; - if (config.getGranularitySpec() != null && config.getGranularitySpec().getSegmentGranularity() != null) { - String temporaryVersion = DateTimes.nowUtc().toString(); - Map> intervalToPartitionMap = new HashMap<>(); - configuredSegmentGranularity = config.getGranularitySpec().getSegmentGranularity(); - // Create a new timeline to hold segments in the new configured segment granularity - SegmentTimeline timelineWithConfiguredSegmentGranularity = new SegmentTimeline(); - Set segments = timeline.findNonOvershadowedObjectsInInterval(Intervals.ETERNITY, Partitions.ONLY_COMPLETE); + if (configuredSegmentGranularity != null) { + final Set segments = timeline.findNonOvershadowedObjectsInInterval( + Intervals.ETERNITY, + Partitions.ONLY_COMPLETE + ); + + // Skip compaction if any segment has partial-eternity interval + // See https://github.com/apache/druid/issues/13208 + final List partialEternitySegments = new ArrayList<>(); for (DataSegment segment : segments) { - // Convert original segmentGranularity to new granularities bucket by configuredSegmentGranularity - // For example, if the original is interval of 2020-01-28/2020-02-03 with WEEK granularity - // and the configuredSegmentGranularity is MONTH, the segment will be split to two segments - // of 2020-01/2020-02 and 2020-02/2020-03. if (Intervals.ETERNITY.getStart().equals(segment.getInterval().getStart()) || Intervals.ETERNITY.getEnd().equals(segment.getInterval().getEnd())) { - // This is to prevent the coordinator from crashing as raised in https://github.com/apache/druid/issues/13208 - log.warn("Cannot compact datasource[%s] containing segments with partial-ETERNITY intervals", dataSource); - return; + partialEternitySegments.add(segment); } + } + if (!partialEternitySegments.isEmpty()) { + SegmentsToCompact candidatesWithStatus = SegmentsToCompact.from(partialEternitySegments).withStatus( + CompactionStatus.skipped("Segments have partial-eternity intervals") + ); + skippedSegments.add(candidatesWithStatus); + statusTracker.onCompactionStatusComputed(candidatesWithStatus, config); + return; + } + + // Convert original segmentGranularity to new granularities bucket by configuredSegmentGranularity + // For example, if the original is interval of 2020-01-28/2020-02-03 with WEEK granularity + // and the configuredSegmentGranularity is MONTH, the segment will be split to two segments + // of 2020-01/2020-02 and 2020-02/2020-03. + final SegmentTimeline timelineWithConfiguredSegmentGranularity = new SegmentTimeline(); + final Map> intervalToPartitionMap = new HashMap<>(); + for (DataSegment segment : segments) { for (Interval interval : configuredSegmentGranularity.getIterable(segment.getInterval())) { - intervalToPartitionMap.computeIfAbsent(interval, k -> new HashSet<>()).add(segment); + intervalToPartitionMap.computeIfAbsent(interval, k -> new HashSet<>()) + .add(segment); } } + + final String temporaryVersion = DateTimes.nowUtc().toString(); for (Map.Entry> partitionsPerInterval : intervalToPartitionMap.entrySet()) { Interval interval = partitionsPerInterval.getKey(); int partitionNum = 0; @@ -149,13 +177,7 @@ private void populateQueue(SegmentTimeline timeline, List skipInterval originalTimeline = timeline; timeline = timelineWithConfiguredSegmentGranularity; } - final List searchIntervals = findInitialSearchInterval( - dataSource, - timeline, - config.getSkipOffsetFromLatest(), - configuredSegmentGranularity, - skipIntervals - ); + final List searchIntervals = findInitialSearchInterval(timeline, skipIntervals); if (!searchIntervals.isEmpty()) { findAndEnqueueSegmentsToCompact( new CompactibleSegmentIterator(timeline, searchIntervals, originalTimeline) @@ -167,14 +189,16 @@ private void populateQueue(SegmentTimeline timeline, List skipInterval } } - public CompactionStatistics totalCompactedStatistics() + @Override + public List getCompactedSegments() { - return compactedSegmentStats; + return compactedSegments; } - public CompactionStatistics totalSkippedStatistics() + @Override + public List getSkippedSegments() { - return skippedSegmentStats; + return skippedSegments; } @Override @@ -309,57 +333,69 @@ private void findAndEnqueueSegmentsToCompact(CompactibleSegmentIterator compacti } final SegmentsToCompact candidates = SegmentsToCompact.from(segments); - final Interval interval = candidates.getUmbrellaInterval(); - - final CompactionStatus compactionStatus = statusTracker.computeCompactionStatus(candidates, config); - if (!compactionStatus.isComplete()) { - log.debug( - "Datasource[%s], interval[%s] has [%d] segments that need to be compacted because [%s].", - dataSource, interval, candidates.size(), compactionStatus.getReason() - ); - } + final CompactionStatus compactionStatus = computeCompactionStatus(candidates); + final SegmentsToCompact candidatesWithStatus = candidates.withStatus(compactionStatus); + statusTracker.onCompactionStatusComputed(candidatesWithStatus, config); if (compactionStatus.isComplete()) { - compactedSegmentStats.increment(candidates.getStats()); + compactedSegments.add(candidatesWithStatus); } else if (compactionStatus.isSkipped()) { - skippedSegmentStats.increment(candidates.getStats()); - statusTracker.onSegmentsSkipped(candidates, compactionStatus); - log.warn( - "Skipping compaction for datasource[%s], interval[%s] due to reason[%s].", - dataSource, interval, compactionStatus.getReason() - ); - } else if (config.getGranularitySpec() != null - && config.getGranularitySpec().getSegmentGranularity() != null) { - if (queuedIntervals.contains(interval)) { - // Skip these candidate segments as we have already queued this interval - } else { - queuedIntervals.add(interval); - queue.add(candidates.withStatus(compactionStatus)); - } + skippedSegments.add(candidatesWithStatus); } else { - queue.add(candidates.withStatus(compactionStatus)); + queue.add(candidatesWithStatus); + if (configuredSegmentGranularity != null) { + queuedIntervals.add(candidates.getUmbrellaInterval()); + } } } + } + + private CompactionStatus computeCompactionStatus(SegmentsToCompact candidate) + { + final CompactionStatus compactionStatus + = CompactionStatus.compute(candidate, config, statusTracker.getObjectMapper()); + if (compactionStatus.isComplete()) { + return compactionStatus; + } + + // Skip intervals that violate max allowed input segment size + final long inputSegmentSize = config.getInputSegmentSizeBytes(); + if (candidate.getTotalBytes() > inputSegmentSize) { + return CompactionStatus.skipped( + "'inputSegmentSize' exceeded: Total segment size[%d] is larger than allowed inputSegmentSize[%d]", + candidate.getTotalBytes(), inputSegmentSize + ); + } + + // Skip intervals that are already queued + if (configuredSegmentGranularity != null + && queuedIntervals.contains(candidate.getUmbrellaInterval())) { + return CompactionStatus.skipped("Interval is already queued"); + } + + // Skip intervals that already have a running task + final CompactionTaskStatus lastTaskStatus = statusTracker.getLatestTaskStatus(candidate); + if (lastTaskStatus != null && lastTaskStatus.getState() == TaskState.RUNNING) { + return CompactionStatus.skipped("Task for interval is already running"); + } + + // Skip intervals that have been filtered out by the policy + if (searchPolicy.shouldSkipCompaction(candidate, compactionStatus, lastTaskStatus)) { + return CompactionStatus.skipped("Skipped by policy"); + } - log.debug("No more segments to compact for datasource[%s].", dataSource); + return compactionStatus; } /** * Returns the initial searchInterval which is {@code (timeline.first().start, timeline.last().end - skipOffset)}. - * - * @param timeline timeline of a dataSource - * @param skipIntervals intervals to skip - * - * @return found interval to search or null if it's not found */ private List findInitialSearchInterval( - String dataSourceName, SegmentTimeline timeline, - Period skipOffset, - Granularity configuredSegmentGranularity, @Nullable List skipIntervals ) { + final Period skipOffset = config.getSkipOffsetFromLatest(); Preconditions.checkArgument(timeline != null && !timeline.isEmpty(), "timeline should not be null or empty"); Preconditions.checkNotNull(skipOffset, "skipOffset"); @@ -370,38 +406,41 @@ private List findInitialSearchInterval( last.getInterval().getEnd(), skipOffset ); - final List fullSkipIntervals = sortAndAddSkipIntervalFromLatest( - latestSkipInterval, - skipIntervals - ); + final List allSkipIntervals + = sortAndAddSkipIntervalFromLatest(latestSkipInterval, skipIntervals); // Collect stats for all skipped segments - for (Interval skipInterval : fullSkipIntervals) { + for (Interval skipInterval : allSkipIntervals) { final List segments = new ArrayList<>( timeline.findNonOvershadowedObjectsInInterval(skipInterval, Partitions.ONLY_COMPLETE) ); if (!CollectionUtils.isNullOrEmpty(segments)) { - SegmentsToCompact candidates = SegmentsToCompact.from(segments); - skippedSegmentStats.increment(candidates.getStats()); + final SegmentsToCompact candidates = SegmentsToCompact.from(segments); final CompactionStatus reason; - if (skipInterval.overlaps(latestSkipInterval)) { + if (candidates.getUmbrellaInterval().overlaps(latestSkipInterval)) { reason = CompactionStatus.skipped("skip offset from latest[%s]", skipOffset); } else { reason = CompactionStatus.skipped("interval locked by another task"); } - statusTracker.onSegmentsSkipped(candidates, reason); + + final SegmentsToCompact candidatesWithStatus = candidates.withStatus(reason); + skippedSegments.add(candidatesWithStatus); + statusTracker.onCompactionStatusComputed(candidatesWithStatus, config); } } final Interval totalInterval = new Interval(first.getInterval().getStart(), last.getInterval().getEnd()); - final List filteredInterval = filterSkipIntervals(totalInterval, fullSkipIntervals); + final List filteredInterval = filterSkipIntervals(totalInterval, allSkipIntervals); final List searchIntervals = new ArrayList<>(); for (Interval lookupInterval : filteredInterval) { if (Intervals.ETERNITY.getStart().equals(lookupInterval.getStart()) || Intervals.ETERNITY.getEnd().equals(lookupInterval.getEnd())) { - log.warn("Cannot compact datasource[%s] since interval[%s] coincides with ETERNITY.", dataSourceName, lookupInterval); + log.warn( + "Cannot compact datasource[%s] since interval[%s] coincides with ETERNITY.", + dataSource, lookupInterval + ); return Collections.emptyList(); } final List segments = timeline @@ -432,21 +471,19 @@ private List findInitialSearchInterval( return searchIntervals; } - Interval computeLatestSkipInterval( - Granularity configuredSegmentGranularity, - DateTime latest, + static Interval computeLatestSkipInterval( + @Nullable Granularity configuredSegmentGranularity, + DateTime latestDataTimestamp, Period skipOffsetFromLatest ) { - final Interval skipFromLatest; - if (configuredSegmentGranularity != null) { - DateTime skipFromLastest = new DateTime(latest, latest.getZone()).minus(skipOffsetFromLatest); - DateTime skipOffsetBucketToSegmentGranularity = configuredSegmentGranularity.bucketStart(skipFromLastest); - skipFromLatest = new Interval(skipOffsetBucketToSegmentGranularity, latest); + if (configuredSegmentGranularity == null) { + return new Interval(skipOffsetFromLatest, latestDataTimestamp); } else { - skipFromLatest = new Interval(skipOffsetFromLatest, latest); + DateTime skipFromLastest = new DateTime(latestDataTimestamp, latestDataTimestamp.getZone()).minus(skipOffsetFromLatest); + DateTime skipOffsetBucketToSegmentGranularity = configuredSegmentGranularity.bucketStart(skipFromLastest); + return new Interval(skipOffsetBucketToSegmentGranularity, latestDataTimestamp); } - return skipFromLatest; } @VisibleForTesting 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 3e2c341bfd7c..32b41dcde924 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 @@ -27,9 +27,10 @@ import java.util.Comparator; /** - * This policy searches segments for compaction from newest to oldest. + * Implementation of {@link CompactionSegmentSearchPolicy} that prioritizes + * intervals which have the latest data. */ -public class NewestSegmentFirstPolicy extends BaseSegmentSearchPolicy +public class NewestSegmentFirstPolicy extends PriorityBasedSegmentSearchPolicy { @JsonCreator public NewestSegmentFirstPolicy( @@ -43,6 +44,6 @@ public NewestSegmentFirstPolicy( protected Comparator getSegmentComparator() { return (o1, o2) -> Comparators.intervalsByStartThenEnd() - .compare(o2.getUmbrellaInterval(), o1.getUmbrellaInterval()); + .compare(o2.getUmbrellaInterval(), o1.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 e8838432bd86..999b0938e165 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 @@ -25,16 +25,15 @@ import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; import org.apache.druid.timeline.SegmentTimeline; -import org.apache.druid.utils.CollectionUtils; import org.joda.time.Interval; -import javax.annotation.Nullable; import java.util.Collections; import java.util.Comparator; import java.util.List; import java.util.Map; import java.util.NoSuchElementException; import java.util.PriorityQueue; +import java.util.stream.Collectors; /** * Implementation of {@link CompactionSegmentIterator} that returns segments in @@ -51,18 +50,18 @@ public PriorityBasedCompactionSegmentIterator( Map compactionConfigs, Map datasourceToTimeline, Map> skipIntervals, - @Nullable String priorityDatasource, - Comparator segmentPriority, + PriorityBasedSegmentSearchPolicy searchPolicy, CompactionStatusTracker statusTracker ) { final Comparator comparator; + final String priorityDatasource = searchPolicy.getPriorityDatasource(); if (priorityDatasource == null || priorityDatasource.isEmpty()) { - comparator = segmentPriority; + comparator = searchPolicy.getSegmentComparator(); } else { comparator = Comparators.alwaysFirst(priorityDatasource) .onResultOf(SegmentsToCompact::getDataSource) - .thenComparing(segmentPriority); + .thenComparing(searchPolicy.getSegmentComparator()); } this.queue = new PriorityQueue<>(comparator); @@ -83,7 +82,7 @@ public PriorityBasedCompactionSegmentIterator( compactionConfigs.get(datasource), timeline, skipIntervals.getOrDefault(datasource, Collections.emptyList()), - segmentPriority, + searchPolicy, statusTracker ) ); @@ -92,21 +91,19 @@ public PriorityBasedCompactionSegmentIterator( } @Override - public Map totalCompactedStatistics() + public List getCompactedSegments() { - return CollectionUtils.mapValues( - datasourceIterators, - DataSourceCompactibleSegmentIterator::totalCompactedStatistics - ); + return datasourceIterators.values().stream().flatMap( + iterator -> iterator.getCompactedSegments().stream() + ).collect(Collectors.toList()); } @Override - public Map totalSkippedStatistics() + public List getSkippedSegments() { - return CollectionUtils.mapValues( - datasourceIterators, - DataSourceCompactibleSegmentIterator::totalSkippedStatistics - ); + return datasourceIterators.values().stream().flatMap( + iterator -> iterator.getSkippedSegments().stream() + ).collect(Collectors.toList()); } @Override diff --git a/server/src/main/java/org/apache/druid/server/compaction/BaseSegmentSearchPolicy.java b/server/src/main/java/org/apache/druid/server/compaction/PriorityBasedSegmentSearchPolicy.java similarity index 71% rename from server/src/main/java/org/apache/druid/server/compaction/BaseSegmentSearchPolicy.java rename to server/src/main/java/org/apache/druid/server/compaction/PriorityBasedSegmentSearchPolicy.java index 838227d6b2dc..1134297d4fb2 100644 --- a/server/src/main/java/org/apache/druid/server/compaction/BaseSegmentSearchPolicy.java +++ b/server/src/main/java/org/apache/druid/server/compaction/PriorityBasedSegmentSearchPolicy.java @@ -30,11 +30,15 @@ import java.util.Map; import java.util.Objects; -public abstract class BaseSegmentSearchPolicy implements CompactionSegmentSearchPolicy +/** + * {@link CompactionSegmentSearchPolicy} that selects segments in order of a + * given priority. + */ +public abstract class PriorityBasedSegmentSearchPolicy implements CompactionSegmentSearchPolicy { private final String priorityDatasource; - protected BaseSegmentSearchPolicy( + protected PriorityBasedSegmentSearchPolicy( @Nullable String priorityDatasource ) { @@ -60,12 +64,30 @@ public CompactionSegmentIterator createIterator( compactionConfigs, dataSources, skipIntervals, - getPriorityDatasource(), - getSegmentComparator(), + this, statusTracker ); } + /** + * Checks if compaction of the given candidate segments should be skipped in + * the current iteration. A concrete policy implementation may override this + * method to avoid compacting intervals that do not fulfil some required criteria. + * + * @return false by default + */ + protected boolean shouldSkipCompaction( + SegmentsToCompact candidateSegments, + CompactionStatus currentCompactionStatus, + CompactionTaskStatus latestTaskStatus + ) + { + return false; + } + + /** + * Comparator used to prioritize between compactible segments. + */ protected abstract Comparator getSegmentComparator(); @Override @@ -77,7 +99,7 @@ public boolean equals(Object o) if (o == null || getClass() != o.getClass()) { return false; } - BaseSegmentSearchPolicy that = (BaseSegmentSearchPolicy) o; + PriorityBasedSegmentSearchPolicy that = (PriorityBasedSegmentSearchPolicy) o; return Objects.equals(priorityDatasource, that.priorityDatasource); } diff --git a/server/src/main/java/org/apache/druid/server/compaction/SmallestSegmentFirstPolicy.java b/server/src/main/java/org/apache/druid/server/compaction/SmallestSegmentFirstPolicy.java index 56a6d30cf9e5..73931ccddc8e 100644 --- a/server/src/main/java/org/apache/druid/server/compaction/SmallestSegmentFirstPolicy.java +++ b/server/src/main/java/org/apache/druid/server/compaction/SmallestSegmentFirstPolicy.java @@ -27,9 +27,10 @@ import java.util.Comparator; /** - * This policy searches segments for compaction from smallest to largest. + * Implementation of {@link CompactionSegmentSearchPolicy} that prioritizes + * intervals which have the smallest average size of segments. */ -public class SmallestSegmentFirstPolicy extends BaseSegmentSearchPolicy +public class SmallestSegmentFirstPolicy extends PriorityBasedSegmentSearchPolicy { @JsonCreator public SmallestSegmentFirstPolicy( diff --git a/server/src/main/java/org/apache/druid/server/compaction/Table.java b/server/src/main/java/org/apache/druid/server/compaction/Table.java new file mode 100644 index 000000000000..068a7abacbae --- /dev/null +++ b/server/src/main/java/org/apache/druid/server/compaction/Table.java @@ -0,0 +1,75 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.server.compaction; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; + +/** + * A simple table POJO with any number of rows and specified column names. + */ +public class Table +{ + private final List columnNames; + private final List> rows = new ArrayList<>(); + + public static Table withColumnNames(String... columnNames) + { + return new Table(Arrays.asList(columnNames), null); + } + + @JsonCreator + public Table( + @JsonProperty("columnNames") List columnNames, + @JsonProperty("rows") List> rows + ) + { + this.columnNames = columnNames; + if (rows != null) { + this.rows.addAll(rows); + } + } + + @JsonProperty + public List getColumnNames() + { + return columnNames; + } + + @JsonProperty + public List> getRows() + { + return rows; + } + + public void addRow(Object... values) + { + rows.add(Arrays.asList(values)); + } + + public boolean isEmpty() + { + return rows.isEmpty(); + } +} diff --git a/server/src/main/java/org/apache/druid/server/coordinator/AutoCompactionSnapshot.java b/server/src/main/java/org/apache/druid/server/coordinator/AutoCompactionSnapshot.java index ec82a2fbdb20..7c1dcd1ee9f4 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/AutoCompactionSnapshot.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/AutoCompactionSnapshot.java @@ -21,6 +21,7 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Preconditions; import org.apache.druid.java.util.common.ISE; import org.apache.druid.server.compaction.CompactionStatistics; @@ -60,7 +61,7 @@ public enum AutoCompactionScheduleStatus public static Builder builder(String dataSource) { - return new Builder(dataSource, AutoCompactionScheduleStatus.RUNNING); + return new Builder(dataSource).withStatus(AutoCompactionScheduleStatus.RUNNING); } @JsonCreator @@ -192,26 +193,26 @@ public int hashCode() public static class Builder { private final String dataSource; - private final AutoCompactionScheduleStatus scheduleStatus; + private AutoCompactionScheduleStatus scheduleStatus; private final CompactionStatistics compactedStats = new CompactionStatistics(); private final CompactionStatistics skippedStats = new CompactionStatistics(); private final CompactionStatistics waitingStats = new CompactionStatistics(); private Builder( - @NotNull String dataSource, - @NotNull AutoCompactionScheduleStatus scheduleStatus + @NotNull String dataSource ) { if (dataSource == null || dataSource.isEmpty()) { throw new ISE("Invalid dataSource name"); } - if (scheduleStatus == null) { - throw new ISE("scheduleStatus cannot be null"); - } - this.dataSource = dataSource; - this.scheduleStatus = scheduleStatus; + } + + public Builder withStatus(AutoCompactionScheduleStatus status) + { + this.scheduleStatus = Preconditions.checkNotNull(status, "scheduleStatus cannot be null"); + return this; } public void incrementWaitingStats(CompactionStatistics entry) diff --git a/server/src/main/java/org/apache/druid/server/coordinator/DruidCompactionConfig.java b/server/src/main/java/org/apache/druid/server/coordinator/DruidCompactionConfig.java index 8510dab5a32d..7e69ed2ab6dc 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/DruidCompactionConfig.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/DruidCompactionConfig.java @@ -40,10 +40,10 @@ public class DruidCompactionConfig { public static final String CONFIG_KEY = "coordinator.compaction.config"; - private static final DruidCompactionConfig EMPTY_INSTANCE - = new DruidCompactionConfig(Collections.emptyList(), null, null, null, null, null); private static final CompactionSegmentSearchPolicy DEFAULT_COMPACTION_POLICY = new NewestSegmentFirstPolicy(null); + private static final DruidCompactionConfig EMPTY_INSTANCE + = new DruidCompactionConfig(Collections.emptyList(), null, null, null, null, null); private final List compactionConfigs; private final double compactionTaskSlotRatio; diff --git a/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java b/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java index 3b5bb9b17df2..77d569e608ef 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java @@ -19,7 +19,6 @@ package org.apache.druid.server.coordinator; -import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.ImmutableList; import com.google.common.collect.Ordering; @@ -56,7 +55,6 @@ import org.apache.druid.segment.metadata.CoordinatorSegmentMetadataCache; import org.apache.druid.server.DruidNode; import org.apache.druid.server.compaction.CompactionRunSimulator; -import org.apache.druid.server.compaction.CompactionScheduler; import org.apache.druid.server.compaction.CompactionSimulateResult; import org.apache.druid.server.compaction.CompactionStatusTracker; import org.apache.druid.server.coordinator.balancer.BalancerStrategyFactory; @@ -115,7 +113,7 @@ * */ @ManageLifecycle -public class DruidCoordinator implements CompactionScheduler +public class DruidCoordinator { /** * Orders newest segments (i.e. segments with most recent intervals) first. @@ -208,8 +206,8 @@ public DruidCoordinator( @Coordinator DruidLeaderSelector coordLeaderSelector, @Nullable CoordinatorSegmentMetadataCache coordinatorSegmentMetadataCache, CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig, - CompactionSchedulerConfig compactionSchedulerConfig, - ObjectMapper objectMapper + CompactionStatusTracker compactionStatusTracker, + CompactionSchedulerConfig compactionSchedulerConfig ) { this.config = config; @@ -227,7 +225,7 @@ public DruidCoordinator( this.balancerStrategyFactory = config.getBalancerStrategyFactory(); this.lookupCoordinatorManager = lookupCoordinatorManager; this.coordLeaderSelector = coordLeaderSelector; - this.compactionStatusTracker = new CompactionStatusTracker(objectMapper); + this.compactionStatusTracker = compactionStatusTracker; this.compactSegments = initializeCompactSegmentsDuty(this.compactionStatusTracker); this.loadQueueManager = loadQueueManager; this.coordinatorSegmentMetadataCache = coordinatorSegmentMetadataCache; @@ -350,36 +348,31 @@ public Integer getReplicationFactor(SegmentId segmentId) return replicaCountsInCluster == null ? null : replicaCountsInCluster.required(); } - @Override @Nullable public Long getTotalSizeOfSegmentsAwaitingCompaction(String dataSource) { return compactSegments.getTotalSizeOfSegmentsAwaitingCompaction(dataSource); } - @Override @Nullable public AutoCompactionSnapshot getCompactionSnapshot(String dataSource) { return compactSegments.getAutoCompactionSnapshot(dataSource); } - @Override public Map getAllCompactionSnapshots() { return compactSegments.getAutoCompactionSnapshot(); } - @Override public CompactionSimulateResult simulateRunWithConfigUpdate(ClusterCompactionConfig updateRequest) { - return new CompactionRunSimulator( - compactionStatusTracker, - metadataManager.configs().getCurrentCompactionConfig(), + return new CompactionRunSimulator(compactionStatusTracker, overlordClient).simulateRunWithConfig( + metadataManager.configs().getCurrentCompactionConfig().withClusterConfig(updateRequest), metadataManager.segments() .getSnapshotOfDataSourcesWithAllUsedSegments() .getUsedSegmentsTimelinesPerDataSource() - ).simulateRunWithConfigUpdate(updateRequest); + ); } public String getCurrentLeader() @@ -456,7 +449,6 @@ private Map> computeUnderReplicated( } } - @Override public void becomeLeader() { synchronized (lock) { @@ -549,7 +541,6 @@ public void becomeLeader() } } - @Override public void stopBeingLeader() { synchronized (lock) { @@ -559,7 +550,7 @@ public void stopBeingLeader() if (coordinatorSegmentMetadataCache != null) { coordinatorSegmentMetadataCache.onLeaderStop(); } - compactionStatusTracker.reset(); + compactionStatusTracker.stop(); taskMaster.onLeaderStop(); serviceAnnouncer.unannounce(self); lookupCoordinatorManager.stop(); 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 283117696279..c4f723c3ece9 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 @@ -571,18 +571,18 @@ private void updateCompactionSnapshotStats( } // Statistics of all segments considered compacted after this run - iterator.totalCompactedStatistics().forEach((dataSource, compactedStats) -> { - currentRunAutoCompactionSnapshotBuilders - .computeIfAbsent(dataSource, AutoCompactionSnapshot::builder) - .incrementCompactedStats(compactedStats); - }); + iterator.getCompactedSegments().forEach( + candidateSegments -> currentRunAutoCompactionSnapshotBuilders + .computeIfAbsent(candidateSegments.getDataSource(), AutoCompactionSnapshot::builder) + .incrementCompactedStats(candidateSegments.getStats()) + ); // Statistics of all segments considered skipped after this run - iterator.totalSkippedStatistics().forEach((dataSource, dataSourceSkippedStatistics) -> { - currentRunAutoCompactionSnapshotBuilders - .computeIfAbsent(dataSource, AutoCompactionSnapshot::builder) - .incrementSkippedStats(dataSourceSkippedStatistics); - }); + iterator.getSkippedSegments().forEach( + candidateSegments -> currentRunAutoCompactionSnapshotBuilders + .computeIfAbsent(candidateSegments.getDataSource(), AutoCompactionSnapshot::builder) + .incrementSkippedStats(candidateSegments.getStats()) + ); final Map currentAutoCompactionSnapshotPerDataSource = new HashMap<>(); currentRunAutoCompactionSnapshotBuilders.forEach((dataSource, builder) -> { diff --git a/server/src/main/java/org/apache/druid/server/http/CompactionResource.java b/server/src/main/java/org/apache/druid/server/http/CompactionResource.java index 9dd569822521..5d8151e92344 100644 --- a/server/src/main/java/org/apache/druid/server/http/CompactionResource.java +++ b/server/src/main/java/org/apache/druid/server/http/CompactionResource.java @@ -26,6 +26,7 @@ import com.sun.jersey.spi.container.ResourceFilters; import org.apache.druid.server.coordinator.AutoCompactionSnapshot; import org.apache.druid.server.coordinator.ClusterCompactionConfig; +import org.apache.druid.server.coordinator.CompactionSchedulerConfig; import org.apache.druid.server.coordinator.DruidCoordinator; import org.apache.druid.server.http.security.ConfigResourceFilter; import org.apache.druid.server.http.security.StateResourceFilter; @@ -44,13 +45,16 @@ public class CompactionResource { private final DruidCoordinator coordinator; + private final CompactionSchedulerConfig schedulerConfig; @Inject public CompactionResource( - DruidCoordinator coordinator + DruidCoordinator coordinator, + CompactionSchedulerConfig schedulerConfig ) { this.coordinator = coordinator; + this.schedulerConfig = schedulerConfig; } /** @@ -74,6 +78,10 @@ public Response getCompactionProgress( @QueryParam("dataSource") String dataSource ) { + if (schedulerConfig.isEnabled()) { + buildErrorResponseWhenRunningScheduler(); + } + final Long notCompactedSegmentSizeBytes = coordinator.getTotalSizeOfSegmentsAwaitingCompaction(dataSource); if (notCompactedSegmentSizeBytes == null) { return Response.status(Response.Status.NOT_FOUND).entity(ImmutableMap.of("error", "unknown dataSource")).build(); @@ -90,6 +98,10 @@ public Response getCompactionSnapshotForDataSource( @QueryParam("dataSource") String dataSource ) { + if (schedulerConfig.isEnabled()) { + return buildErrorResponseWhenRunningScheduler(); + } + final Collection snapshots; if (dataSource == null || dataSource.isEmpty()) { snapshots = coordinator.getAllCompactionSnapshots().values(); @@ -110,8 +122,23 @@ public Response simulateClusterCompactionConfigUpdate( ClusterCompactionConfig updatePayload ) { + if (schedulerConfig.isEnabled()) { + return buildErrorResponseWhenRunningScheduler(); + } + return Response.ok().entity( coordinator.simulateRunWithConfigUpdate(updatePayload) ).build(); } + + private Response buildErrorResponseWhenRunningScheduler() + { + return Response.status(Response.Status.SERVICE_UNAVAILABLE).entity( + ImmutableMap.of( + "error", + "Compaction has been disabled on the Coordinator." + + " Use Overlord APIs to fetch compaction status." + ) + ).build(); + } } 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 7fb6bc5d7f31..8b5c6bab6c40 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 @@ -20,25 +20,43 @@ package org.apache.druid.server.compaction; import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.util.concurrent.Futures; +import com.google.common.util.concurrent.ListenableFuture; +import org.apache.druid.client.indexing.NoopOverlordClient; +import org.apache.druid.client.indexing.TaskPayloadResponse; +import org.apache.druid.indexer.TaskStatus; +import org.apache.druid.indexer.TaskStatusPlus; import org.apache.druid.jackson.DefaultObjectMapper; +import org.apache.druid.java.util.common.CloseableIterators; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.granularity.Granularities; -import org.apache.druid.server.coordinator.ClusterCompactionConfig; +import org.apache.druid.java.util.common.parsers.CloseableIterator; +import org.apache.druid.metadata.LockFilterPolicy; import org.apache.druid.server.coordinator.CreateDataSegments; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; import org.apache.druid.server.coordinator.DruidCompactionConfig; import org.apache.druid.server.coordinator.simulate.TestSegmentsMetadataManager; import org.apache.druid.timeline.DataSegment; +import org.joda.time.Interval; import org.junit.Assert; import org.junit.Test; +import javax.annotation.Nullable; import java.util.Arrays; +import java.util.Collections; import java.util.List; +import java.util.Map; +import java.util.Set; public class CompactionRunSimulatorTest { private static final ObjectMapper OBJECT_MAPPER = new DefaultObjectMapper(); + private final CompactionRunSimulator simulator = new CompactionRunSimulator( + new CompactionStatusTracker(OBJECT_MAPPER), + new TestOverlordClient() + ); + @Test public void testSimulateClusterCompactionConfigUpdate() { @@ -53,21 +71,29 @@ public void testSimulateClusterCompactionConfigUpdate() .eachOfSizeInMb(100); wikiSegments.forEach(segmentsMetadataManager::addSegment); - final CompactionSimulateResult simulateResult = new CompactionRunSimulator( - new CompactionStatusTracker(OBJECT_MAPPER), + final CompactionSimulateResult simulateResult = simulator.simulateRunWithConfig( DruidCompactionConfig.empty().withDatasourceConfig( DataSourceCompactionConfig.builder().forDataSource("wiki").build() ), segmentsMetadataManager.getSnapshotOfDataSourcesWithAllUsedSegments() .getUsedSegmentsTimelinesPerDataSource() - ).simulateRunWithConfigUpdate( - new ClusterCompactionConfig(null, null, null, null, null) ); + Assert.assertNotNull(simulateResult); + final Map compactionStates = simulateResult.getCompactionStates(); + Assert.assertNotNull(compactionStates); + + Assert.assertNull(compactionStates.get(CompactionStatus.State.COMPLETE)); + Assert.assertNull(compactionStates.get(CompactionStatus.State.RUNNING)); + + final Table queuedTable = compactionStates.get(CompactionStatus.State.PENDING); + Assert.assertEquals( + Arrays.asList("dataSource", "interval", "numSegments", "bytes", "maxTaskSlots", "reasonToCompact"), + queuedTable.getColumnNames() + ); Assert.assertEquals( Arrays.asList( - Arrays.asList("dataSource", "interval", "numSegments", "bytes", "maxTaskSlots", "reasonToCompact"), Arrays.asList("wiki", Intervals.of("2013-01-09/P1D"), 10, 1_000_000_000L, 1, "not compacted yet"), Arrays.asList("wiki", Intervals.of("2013-01-08/P1D"), 10, 1_000_000_000L, 1, "not compacted yet"), Arrays.asList("wiki", Intervals.of("2013-01-07/P1D"), 10, 1_000_000_000L, 1, "not compacted yet"), @@ -78,7 +104,62 @@ public void testSimulateClusterCompactionConfigUpdate() Arrays.asList("wiki", Intervals.of("2013-01-02/P1D"), 10, 1_000_000_000L, 1, "not compacted yet"), Arrays.asList("wiki", Intervals.of("2013-01-01/P1D"), 10, 1_000_000_000L, 1, "not compacted yet") ), - simulateResult.getIntervalsToCompact() + queuedTable.getRows() + ); + + final Table skippedTable = compactionStates.get(CompactionStatus.State.SKIPPED); + Assert.assertEquals( + Arrays.asList("dataSource", "interval", "numSegments", "bytes", "reasonToSkip"), + skippedTable.getColumnNames() + ); + Assert.assertEquals( + Collections.singletonList( + Arrays.asList("wiki", Intervals.of("2013-01-10/P1D"), 10, 1_000_000_000L, "skip offset from latest[P1D]") + ), + skippedTable.getRows() ); } + + private static class TestOverlordClient extends NoopOverlordClient + { + @Override + public ListenableFuture> taskStatuses( + @Nullable String state, + @Nullable String dataSource, + @Nullable Integer maxCompletedTasks + ) + { + return Futures.immediateFuture(CloseableIterators.withEmptyBaggage(Collections.emptyIterator())); + } + + @Override + public ListenableFuture> taskStatuses(Set taskIds) + { + return Futures.immediateFuture(Collections.emptyMap()); + } + + @Override + public ListenableFuture taskPayload(String taskId) + { + return Futures.immediateFuture(null); + } + + @Override + public ListenableFuture runTask(String taskId, Object taskObject) + { + return Futures.immediateVoidFuture(); + } + + @Override + public ListenableFuture cancelTask(String taskId) + { + return Futures.immediateVoidFuture(); + } + + @Override + public ListenableFuture>> findLockedIntervals(List lockFilterPolicies) + { + return Futures.immediateFuture(Collections.emptyMap()); + } + } } 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 9e4d5561f87d..aa9d1d67c12e 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 @@ -21,13 +21,11 @@ import com.fasterxml.jackson.databind.ObjectMapper; import org.apache.druid.client.indexing.ClientCompactionTaskQuery; +import org.apache.druid.indexer.TaskState; import org.apache.druid.indexer.TaskStatus; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.server.coordinator.CreateDataSegments; -import org.apache.druid.server.coordinator.DataSourceCompactionConfig; import org.apache.druid.timeline.DataSegment; -import org.joda.time.Duration; -import org.joda.time.Interval; import org.junit.Assert; import org.junit.Before; import org.junit.Test; @@ -49,187 +47,60 @@ public void setup() } @Test - public void testIntervalIsSkippedIfSegmentSizeExceedsLimit() - { - CompactionStatus status = statusTracker.computeCompactionStatus( - SegmentsToCompact.from(Collections.singletonList(WIKI_SEGMENT)), - createCompactionConfig(DS.WIKI, 100L) - ); - Assert.assertTrue(status.isSkipped()); - Assert.assertEquals( - "Total segment size[100000000] is larger than allowed inputSegmentSize[100]", - status.getReason() - ); - } - - @Test - public void testIntervalIsSkippedIfRecentlySubmitted() + public void testGetLatestTaskStatusForSubmittedTask() { final SegmentsToCompact candidateSegments = SegmentsToCompact.from(Collections.singletonList(WIKI_SEGMENT)); statusTracker.onTaskSubmitted(createCompactionTask("task1", DS.WIKI), candidateSegments); - CompactionStatus status = statusTracker.computeCompactionStatus( - candidateSegments, - createCompactionConfig(DS.WIKI, null) - ); - Assert.assertTrue(status.isSkipped()); - Assert.assertEquals( - "Interval[2012-10-24T00:00:00.000Z/2012-10-25T00:00:00.000Z] was recently" - + " submitted for compaction and has state[TASK_SUBMITTED].", - status.getReason() - ); + CompactionTaskStatus status = statusTracker.getLatestTaskStatus(candidateSegments); + Assert.assertEquals(TaskState.RUNNING, status.getState()); } @Test - public void testIntervalIsSkippedIfRecentlyCompacted() + public void testGetLatestTaskStatusForSuccessfulTask() { final SegmentsToCompact candidateSegments = SegmentsToCompact.from(Collections.singletonList(WIKI_SEGMENT)); + statusTracker.onTaskSubmitted(createCompactionTask("task1", DS.WIKI), candidateSegments); + statusTracker.onTaskFinished("task1", TaskStatus.success("task1")); - final String taskId = "task1"; - statusTracker.onTaskSubmitted(createCompactionTask(taskId, DS.WIKI), candidateSegments); - statusTracker.onTaskFinished(taskId, TaskStatus.success(taskId)); - - CompactionStatus status = statusTracker.computeCompactionStatus( - candidateSegments, - createCompactionConfig(DS.WIKI, null) - ); - Assert.assertTrue(status.isSkipped()); - Assert.assertEquals( - "Interval[2012-10-24T00:00:00.000Z/2012-10-25T00:00:00.000Z] was recently" - + " submitted for compaction and has state[COMPACTED].", - status.getReason() - ); - } - - @Test - public void testIntervalIsNotSkippedIfFailedOnce() - { - final SegmentsToCompact candidateSegments - = SegmentsToCompact.from(Collections.singletonList(WIKI_SEGMENT)); - - final String taskId = "task1"; - statusTracker.onTaskSubmitted(createCompactionTask(taskId, DS.WIKI), candidateSegments); - statusTracker.onTaskFinished(taskId, TaskStatus.failure(taskId, "first error")); - - CompactionStatus status = statusTracker.computeCompactionStatus( - candidateSegments, - createCompactionConfig(DS.WIKI, null) - ); - Assert.assertFalse(status.isComplete() || status.isSkipped()); - Assert.assertEquals("Not compacted yet", status.getReason()); + CompactionTaskStatus status = statusTracker.getLatestTaskStatus(candidateSegments); + Assert.assertEquals(TaskState.SUCCESS, status.getState()); } @Test - public void testIntervalIsSkippedIfFailedForMaxRetries() + public void testGetLatestTaskStatusForFailedTask() { final SegmentsToCompact candidateSegments = SegmentsToCompact.from(Collections.singletonList(WIKI_SEGMENT)); - final DataSourceCompactionConfig compactionConfig - = createCompactionConfig(DS.WIKI, null); - - // Verify that interval remains eligible for compaction until max failure retries - for (int i = 0; i < 4; ++i) { - CompactionStatus status = statusTracker.computeCompactionStatus( - candidateSegments, - compactionConfig - ); - Assert.assertFalse(status.isComplete() || status.isSkipped()); - Assert.assertEquals("Not compacted yet", status.getReason()); - - final String taskId = "task_" + i; - statusTracker.onTaskSubmitted(createCompactionTask(taskId, DS.WIKI), candidateSegments); - statusTracker.onTaskFinished(taskId, TaskStatus.failure(taskId, "error number " + i)); - } + statusTracker.onTaskSubmitted(createCompactionTask("task1", DS.WIKI), candidateSegments); + statusTracker.onTaskFinished("task1", TaskStatus.failure("task1", "some failure")); - CompactionStatus status = statusTracker.computeCompactionStatus( - candidateSegments, - createCompactionConfig(DS.WIKI, null) - ); - Assert.assertTrue(status.isSkipped()); - Assert.assertEquals( - "Interval[2012-10-24T00:00:00.000Z/2012-10-25T00:00:00.000Z] was recently" - + " submitted for compaction and has state[FAILED_ALL_RETRIES].", - status.getReason() - ); + CompactionTaskStatus status = statusTracker.getLatestTaskStatus(candidateSegments); + Assert.assertEquals(TaskState.FAILED, status.getState()); + Assert.assertEquals(1, status.getNumConsecutiveFailures()); } @Test - public void testIntervalBecomesEligibleAgainAfterMaxSkips() + public void testGetLatestTaskStatusForRepeatedlyFailingTask() { final SegmentsToCompact candidateSegments = SegmentsToCompact.from(Collections.singletonList(WIKI_SEGMENT)); - final String taskId = "task1"; - statusTracker.onTaskSubmitted(createCompactionTask(taskId, DS.WIKI), candidateSegments); - statusTracker.onTaskFinished(taskId, TaskStatus.success(taskId)); - - // Verify that interval is skipped until skip count is exhausted - for (int i = 0; i < 5; ++i) { - CompactionStatus status = statusTracker.computeCompactionStatus( - candidateSegments, - createCompactionConfig(DS.WIKI, null) - ); - Assert.assertTrue(status.isSkipped()); - Assert.assertEquals( - "Interval[2012-10-24T00:00:00.000Z/2012-10-25T00:00:00.000Z] was recently" - + " submitted for compaction and has state[COMPACTED].", - status.getReason() - ); - - // Submit task for a different interval to decrement skip count - final DataSegment segment - = DataSegment.builder(WIKI_SEGMENT) - .interval(shift(WIKI_SEGMENT.getInterval(), Duration.standardDays(i + 1))) - .build(); - statusTracker.onTaskSubmitted( - createCompactionTask("task_" + i, DS.WIKI), - SegmentsToCompact.from(Collections.singletonList(segment)) - ); - } - - // Verify that interval is now eligible again - CompactionStatus status = statusTracker.computeCompactionStatus( - candidateSegments, - createCompactionConfig(DS.WIKI, null) - ); - Assert.assertFalse(status.isComplete() || status.isSkipped()); - Assert.assertEquals("Not compacted yet", status.getReason()); - } - - @Test - public void testResetClearsAllStatuses() - { + statusTracker.onTaskSubmitted(createCompactionTask("task1", DS.WIKI), candidateSegments); + statusTracker.onTaskFinished("task1", TaskStatus.failure("task1", "some failure")); - } + statusTracker.onTaskSubmitted(createCompactionTask("task2", DS.WIKI), candidateSegments); + CompactionTaskStatus status = statusTracker.getLatestTaskStatus(candidateSegments); + Assert.assertEquals(TaskState.RUNNING, status.getState()); + Assert.assertEquals(1, status.getNumConsecutiveFailures()); - private static Interval shift(Interval interval, Duration duration) - { - return interval.withEnd(interval.getEnd().plus(duration)) - .withStart(interval.getStart().plus(duration)); - } + statusTracker.onTaskFinished("task2", TaskStatus.failure("task2", "second failure")); - private DataSourceCompactionConfig createCompactionConfig( - String datasource, - Long inputSegmentSizeBytes - ) - { - return new DataSourceCompactionConfig( - datasource, - null, - inputSegmentSizeBytes, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null - ); + status = statusTracker.getLatestTaskStatus(candidateSegments); + Assert.assertEquals(TaskState.FAILED, status.getState()); + Assert.assertEquals(2, status.getNumConsecutiveFailures()); } private ClientCompactionTaskQuery createCompactionTask( diff --git a/server/src/test/java/org/apache/druid/server/coordinator/DruidCompactionConfigTest.java b/server/src/test/java/org/apache/druid/server/coordinator/DruidCompactionConfigTest.java index 2e58f740d63c..5eadead4e684 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/DruidCompactionConfigTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/DruidCompactionConfigTest.java @@ -22,6 +22,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import org.apache.druid.indexer.CompactionEngine; import org.apache.druid.jackson.DefaultObjectMapper; +import org.apache.druid.server.compaction.NewestSegmentFirstPolicy; import org.joda.time.Period; import org.junit.Assert; import org.junit.Test; @@ -76,8 +77,13 @@ public void testCopyWithClusterConfig() { final DruidCompactionConfig config = DruidCompactionConfig.empty(); - final ClusterCompactionConfig clusterConfig - = new ClusterCompactionConfig(0.5, 10, false, CompactionEngine.MSQ, null); + final ClusterCompactionConfig clusterConfig = new ClusterCompactionConfig( + 0.5, + 10, + false, + CompactionEngine.MSQ, + new NewestSegmentFirstPolicy(null) + ); final DruidCompactionConfig copy = config.withClusterConfig(clusterConfig); Assert.assertEquals(clusterConfig, copy.clusterConfig()); diff --git a/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorTest.java b/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorTest.java index 0dc8563ab300..b5c8f1487cc5 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorTest.java @@ -192,8 +192,8 @@ public void setUp() throws Exception new TestDruidLeaderSelector(), null, CentralizedDatasourceSchemaConfig.create(), - new CompactionSchedulerConfig(true), - OBJECT_MAPPER + new CompactionStatusTracker(OBJECT_MAPPER), + new CompactionSchedulerConfig(true) ); } @@ -624,8 +624,8 @@ public void testCompactSegmentsDutyWhenCustomDutyGroupEmpty() new TestDruidLeaderSelector(), null, CentralizedDatasourceSchemaConfig.create(), - CompactionSchedulerConfig.defaultConfig(), - OBJECT_MAPPER + new CompactionStatusTracker(OBJECT_MAPPER), + CompactionSchedulerConfig.defaultConfig() ); // Since CompactSegments is not enabled in Custom Duty Group, then CompactSegments must be created in IndexingServiceDuties List indexingDuties = coordinator.makeIndexingServiceDuties(); @@ -665,8 +665,8 @@ public void testInitializeCompactSegmentsDutyWhenCustomDutyGroupDoesNotContainsC new TestDruidLeaderSelector(), null, CentralizedDatasourceSchemaConfig.create(), - CompactionSchedulerConfig.defaultConfig(), - OBJECT_MAPPER + new CompactionStatusTracker(OBJECT_MAPPER), + CompactionSchedulerConfig.defaultConfig() ); // Since CompactSegments is not enabled in Custom Duty Group, then CompactSegments must be created in IndexingServiceDuties List indexingDuties = coordinator.makeIndexingServiceDuties(); @@ -706,8 +706,8 @@ public void testInitializeCompactSegmentsDutyWhenCustomDutyGroupContainsCompactS new TestDruidLeaderSelector(), null, CentralizedDatasourceSchemaConfig.create(), - CompactionSchedulerConfig.defaultConfig(), - OBJECT_MAPPER + new CompactionStatusTracker(OBJECT_MAPPER), + CompactionSchedulerConfig.defaultConfig() ); // Since CompactSegments is enabled in Custom Duty Group, then CompactSegments must not be created in IndexingServiceDuties List indexingDuties = coordinator.makeIndexingServiceDuties(); @@ -812,8 +812,8 @@ public void testCoordinatorCustomDutyGroupsRunAsExpected() throws Exception new TestDruidLeaderSelector(), null, CentralizedDatasourceSchemaConfig.create(), - new CompactionSchedulerConfig(true), - OBJECT_MAPPER + new CompactionStatusTracker(OBJECT_MAPPER), + CompactionSchedulerConfig.defaultConfig() ); coordinator.start(); diff --git a/server/src/test/java/org/apache/druid/server/coordinator/simulate/CoordinatorSimulationBuilder.java b/server/src/test/java/org/apache/druid/server/coordinator/simulate/CoordinatorSimulationBuilder.java index a084b12c4004..9e13bebe9df5 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/simulate/CoordinatorSimulationBuilder.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/simulate/CoordinatorSimulationBuilder.java @@ -37,6 +37,7 @@ import org.apache.druid.java.util.metrics.MetricsVerifier; import org.apache.druid.java.util.metrics.StubServiceEmitter; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; +import org.apache.druid.server.compaction.CompactionStatusTracker; import org.apache.druid.server.coordinator.CompactionSchedulerConfig; import org.apache.druid.server.coordinator.CoordinatorConfigManager; import org.apache.druid.server.coordinator.CoordinatorDynamicConfig; @@ -211,8 +212,8 @@ public CoordinatorSimulation build() env.leaderSelector, null, CentralizedDatasourceSchemaConfig.create(), - CompactionSchedulerConfig.defaultConfig(), - OBJECT_MAPPER + new CompactionStatusTracker(OBJECT_MAPPER), + CompactionSchedulerConfig.defaultConfig() ); return new SimulationImpl(coordinator, env); diff --git a/server/src/test/java/org/apache/druid/server/http/CompactionResourceTest.java b/server/src/test/java/org/apache/druid/server/http/CompactionResourceTest.java index fdd34515395f..ea5b1c64293f 100644 --- a/server/src/test/java/org/apache/druid/server/http/CompactionResourceTest.java +++ b/server/src/test/java/org/apache/druid/server/http/CompactionResourceTest.java @@ -22,6 +22,7 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import org.apache.druid.server.coordinator.AutoCompactionSnapshot; +import org.apache.druid.server.coordinator.CompactionSchedulerConfig; import org.apache.druid.server.coordinator.DruidCoordinator; import org.easymock.EasyMock; import org.junit.After; @@ -35,6 +36,7 @@ public class CompactionResourceTest { private DruidCoordinator mock; + private final CompactionSchedulerConfig schedulerConfig = new CompactionSchedulerConfig(false); private String dataSourceName = "datasource_1"; private AutoCompactionSnapshot expectedSnapshot = new AutoCompactionSnapshot( dataSourceName, @@ -73,7 +75,8 @@ public void testGetCompactionSnapshotForDataSourceWithEmptyQueryParameter() EasyMock.expect(mock.getAllCompactionSnapshots()).andReturn(expected).once(); EasyMock.replay(mock); - final Response response = new CompactionResource(mock).getCompactionSnapshotForDataSource(""); + final Response response = new CompactionResource(mock, schedulerConfig) + .getCompactionSnapshotForDataSource(""); Assert.assertEquals(ImmutableMap.of("latestStatus", expected.values()), response.getEntity()); Assert.assertEquals(200, response.getStatus()); } @@ -90,7 +93,8 @@ public void testGetCompactionSnapshotForDataSourceWithNullQueryParameter() EasyMock.expect(mock.getAllCompactionSnapshots()).andReturn(expected).once(); EasyMock.replay(mock); - final Response response = new CompactionResource(mock).getCompactionSnapshotForDataSource(null); + final Response response = new CompactionResource(mock, schedulerConfig) + .getCompactionSnapshotForDataSource(null); Assert.assertEquals(ImmutableMap.of("latestStatus", expected.values()), response.getEntity()); Assert.assertEquals(200, response.getStatus()); } @@ -103,7 +107,8 @@ public void testGetCompactionSnapshotForDataSourceWithValidQueryParameter() EasyMock.expect(mock.getCompactionSnapshot(dataSourceName)).andReturn(expectedSnapshot).once(); EasyMock.replay(mock); - final Response response = new CompactionResource(mock).getCompactionSnapshotForDataSource(dataSourceName); + final Response response = new CompactionResource(mock, schedulerConfig) + .getCompactionSnapshotForDataSource(dataSourceName); Assert.assertEquals(ImmutableMap.of("latestStatus", ImmutableList.of(expectedSnapshot)), response.getEntity()); Assert.assertEquals(200, response.getStatus()); } @@ -116,7 +121,26 @@ public void testGetCompactionSnapshotForDataSourceWithInvalidQueryParameter() EasyMock.expect(mock.getCompactionSnapshot(dataSourceName)).andReturn(null).once(); EasyMock.replay(mock); - final Response response = new CompactionResource(mock).getCompactionSnapshotForDataSource(dataSourceName); + final Response response = new CompactionResource(mock, schedulerConfig) + .getCompactionSnapshotForDataSource(dataSourceName); Assert.assertEquals(404, response.getStatus()); } + + @Test + public void testGetCompactionSnapshotWhenCompactionSchedulerIsEnabled() + { + EasyMock.replay(mock); + + final Response response = new CompactionResource(mock, new CompactionSchedulerConfig(true)) + .getCompactionSnapshotForDataSource("dummy"); + Assert.assertEquals(503, response.getStatus()); + Assert.assertEquals( + ImmutableMap.of( + "error", + "Compaction has been disabled on the Coordinator." + + " Use Overlord APIs to fetch compaction status." + ), + response.getEntity() + ); + } } diff --git a/services/src/main/java/org/apache/druid/cli/CliCoordinator.java b/services/src/main/java/org/apache/druid/cli/CliCoordinator.java index 57ede3d42502..d7e03e24d362 100644 --- a/services/src/main/java/org/apache/druid/cli/CliCoordinator.java +++ b/services/src/main/java/org/apache/druid/cli/CliCoordinator.java @@ -97,6 +97,7 @@ import org.apache.druid.segment.metadata.SegmentSchemaCache; import org.apache.druid.server.QueryScheduler; import org.apache.druid.server.QuerySchedulerProvider; +import org.apache.druid.server.compaction.CompactionStatusTracker; import org.apache.druid.server.coordinator.CompactionSchedulerConfig; import org.apache.druid.server.coordinator.CoordinatorConfigManager; import org.apache.druid.server.coordinator.DruidCoordinator; @@ -258,6 +259,7 @@ public void configure(Binder binder) binder.bind(CoordinatorConfigManager.class); binder.bind(MetadataManager.class); binder.bind(DruidCoordinator.class); + binder.bind(CompactionStatusTracker.class).in(LazySingleton.class); LifecycleModule.register(binder, MetadataStorage.class); LifecycleModule.register(binder, DruidCoordinator.class); diff --git a/services/src/main/java/org/apache/druid/cli/CliOverlord.java b/services/src/main/java/org/apache/druid/cli/CliOverlord.java index 52f8665b5487..f877685765bc 100644 --- a/services/src/main/java/org/apache/druid/cli/CliOverlord.java +++ b/services/src/main/java/org/apache/druid/cli/CliOverlord.java @@ -38,6 +38,8 @@ import com.google.inject.servlet.GuiceFilter; import com.google.inject.util.Providers; import org.apache.druid.client.indexing.IndexingService; +import org.apache.druid.common.config.Configs; +import org.apache.druid.common.config.JacksonConfigManager; import org.apache.druid.discovery.NodeRole; import org.apache.druid.guice.IndexingServiceInputSourceModule; import org.apache.druid.guice.IndexingServiceModuleHelper; @@ -51,6 +53,7 @@ import org.apache.druid.guice.ListProvider; import org.apache.druid.guice.ManageLifecycle; import org.apache.druid.guice.PolyBind; +import org.apache.druid.guice.SupervisorModule; import org.apache.druid.guice.annotations.Json; import org.apache.druid.indexing.common.RetryPolicyFactory; import org.apache.druid.indexing.common.TaskStorageDirTracker; @@ -98,7 +101,6 @@ import org.apache.druid.indexing.overlord.sampler.SamplerModule; import org.apache.druid.indexing.overlord.setup.WorkerBehaviorConfig; import org.apache.druid.indexing.overlord.supervisor.SupervisorManager; -import org.apache.druid.indexing.overlord.supervisor.SupervisorModule; import org.apache.druid.indexing.overlord.supervisor.SupervisorResource; import org.apache.druid.indexing.worker.config.WorkerConfig; import org.apache.druid.indexing.worker.shuffle.DeepStorageIntermediaryDataManager; @@ -115,8 +117,12 @@ import org.apache.druid.segment.realtime.NoopChatHandlerProvider; import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager; import org.apache.druid.segment.realtime.appenderator.DummyForInjectionAppenderatorsManager; +import org.apache.druid.server.compaction.CompactionScheduler; +import org.apache.druid.server.compaction.CompactionStatusTracker; +import org.apache.druid.server.coordinator.ClusterCompactionConfig; import org.apache.druid.server.coordinator.CompactionSchedulerConfig; import org.apache.druid.server.coordinator.CoordinatorOverlordServiceConfig; +import org.apache.druid.server.coordinator.DruidCompactionConfig; import org.apache.druid.server.http.RedirectFilter; import org.apache.druid.server.http.RedirectInfo; import org.apache.druid.server.http.SelfDiscoveryResource; @@ -211,6 +217,7 @@ public void configure(Binder binder) CentralizedDatasourceSchemaConfig.class ); + binder.bind(CompactionStatusTracker.class).in(LazySingleton.class); binder.bind(SegmentsMetadataManager.class) .toProvider(SegmentsMetadataManagerProvider.class) .in(ManageLifecycle.class); @@ -246,8 +253,8 @@ public void configure(Binder binder) binder.bind(TaskLockbox.class).in(LazySingleton.class); binder.bind(TaskQueryTool.class).in(LazySingleton.class); binder.bind(IndexerMetadataStorageAdapter.class).in(LazySingleton.class); + binder.bind(CompactionScheduler.class).to(OverlordCompactionScheduler.class).in(LazySingleton.class); binder.bind(SupervisorManager.class).in(LazySingleton.class); - binder.bind(OverlordCompactionScheduler.class).in(LazySingleton.class); binder.bind(ParallelIndexSupervisorTaskClientProvider.class).toProvider(Providers.of(null)); binder.bind(ShuffleClient.class).toProvider(Providers.of(null)); @@ -408,6 +415,21 @@ public Supplier> getHeartbeatSupplier(DruidOverlord overlord return heartbeatTags; }; } + + @Provides + @LazySingleton + public Supplier getClusterCompactionConfig(JacksonConfigManager configManager) + { + return () -> { + DruidCompactionConfig compactionConfig = configManager.watch( + DruidCompactionConfig.CONFIG_KEY, + DruidCompactionConfig.class, + DruidCompactionConfig.empty() + ).get(); + return Configs.valueOrDefault(compactionConfig, DruidCompactionConfig.empty()) + .clusterConfig(); + }; + } }; } From 4942a08aad390b8e2c18f5f6a05aa3ae6b9a5677 Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Fri, 9 Aug 2024 08:50:12 +0530 Subject: [PATCH 10/26] Add test for OverlordCompactionScheduler --- .../compact}/CompactionScheduler.java | 5 +- .../compact/CompactionSupervisor.java | 1 - .../compact/CompactionSupervisorSpec.java | 1 - .../compact/OverlordCompactionScheduler.java | 18 +- .../indexing/overlord/DruidOverlord.java | 2 +- .../indexing/overlord/TaskQueryTool.java | 13 +- .../http/OverlordCompactionResource.java | 7 +- .../OverlordCompactionSchedulerTest.java | 300 ++++++++++++++++++ .../http/OverlordCompactionResourceTest.java | 25 ++ .../overlord/http/OverlordResourceTest.java | 2 +- .../indexing/overlord/http/OverlordTest.java | 2 +- .../apache/druid/server/compaction/Table.java | 9 + .../coordinator/AutoCompactionSnapshot.java | 18 ++ .../compaction/CompactionStatusTest.java | 189 ++++++++--- .../simulate/TestSegmentsMetadataManager.java | 7 +- .../org/apache/druid/cli/CliOverlord.java | 12 +- 16 files changed, 538 insertions(+), 73 deletions(-) rename {server/src/main/java/org/apache/druid/server/compaction => indexing-service/src/main/java/org/apache/druid/indexing/compact}/CompactionScheduler.java (89%) create mode 100644 indexing-service/src/test/java/org/apache/druid/indexing/compact/OverlordCompactionSchedulerTest.java create mode 100644 indexing-service/src/test/java/org/apache/druid/indexing/overlord/http/OverlordCompactionResourceTest.java diff --git a/server/src/main/java/org/apache/druid/server/compaction/CompactionScheduler.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionScheduler.java similarity index 89% rename from server/src/main/java/org/apache/druid/server/compaction/CompactionScheduler.java rename to indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionScheduler.java index 3807ea634356..9d257867f4cf 100644 --- a/server/src/main/java/org/apache/druid/server/compaction/CompactionScheduler.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionScheduler.java @@ -17,8 +17,9 @@ * under the License. */ -package org.apache.druid.server.compaction; +package org.apache.druid.indexing.compact; +import org.apache.druid.server.compaction.CompactionSimulateResult; import org.apache.druid.server.coordinator.AutoCompactionSnapshot; import org.apache.druid.server.coordinator.ClusterCompactionConfig; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; @@ -41,7 +42,7 @@ public interface CompactionScheduler AutoCompactionSnapshot getCompactionSnapshot(String dataSource); - Long getTotalSizeOfSegmentsAwaitingCompaction(String dataSource); + Long getSegmentBytesAwaitingCompaction(String dataSource); CompactionSimulateResult simulateRunWithConfigUpdate(ClusterCompactionConfig updateRequest); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionSupervisor.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionSupervisor.java index e924ee186115..61023d631bad 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionSupervisor.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionSupervisor.java @@ -26,7 +26,6 @@ import org.apache.druid.indexing.overlord.supervisor.autoscaler.LagStats; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.logger.Logger; -import org.apache.druid.server.compaction.CompactionScheduler; import org.apache.druid.server.coordinator.AutoCompactionSnapshot; /** diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionSupervisorSpec.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionSupervisorSpec.java index f263e925f864..55297f56d89f 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionSupervisorSpec.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionSupervisorSpec.java @@ -26,7 +26,6 @@ import org.apache.druid.error.InvalidInput; import org.apache.druid.indexing.overlord.supervisor.Supervisor; import org.apache.druid.indexing.overlord.supervisor.SupervisorSpec; -import org.apache.druid.server.compaction.CompactionScheduler; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; import javax.annotation.Nullable; 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 ea563429d58a..6021672b14dc 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 @@ -37,7 +37,6 @@ import org.apache.druid.metadata.SegmentsMetadataManager; import org.apache.druid.rpc.indexing.OverlordClient; import org.apache.druid.server.compaction.CompactionRunSimulator; -import org.apache.druid.server.compaction.CompactionScheduler; import org.apache.druid.server.compaction.CompactionSimulateResult; import org.apache.druid.server.compaction.CompactionStatusTracker; import org.apache.druid.server.coordinator.AutoCompactionSnapshot; @@ -108,7 +107,7 @@ public class OverlordCompactionScheduler implements CompactionScheduler */ private final boolean shouldPollSegments; - private final Stopwatch sinceStatsEmitted = Stopwatch.createStarted(); + private final Stopwatch sinceStatsEmitted = Stopwatch.createUnstarted(); @Inject public OverlordCompactionScheduler( @@ -165,13 +164,8 @@ public void start() { if (isEnabled() && started.compareAndSet(false, true)) { log.info("Starting compaction scheduler."); - scheduleOnExecutor( - () -> { - initState(); - checkSchedulingStatus(); - }, - SCHEDULE_PERIOD_SECONDS - ); + initState(); + scheduleOnExecutor(this::checkSchedulingStatus, SCHEDULE_PERIOD_SECONDS); } } @@ -180,7 +174,7 @@ public void stop() { if (isEnabled() && started.compareAndSet(true, false)) { log.info("Stopping compaction scheduler."); - runOnExecutor(this::cleanupState); + cleanupState(); } } @@ -258,7 +252,7 @@ private synchronized void runCompactionDuty() duty.run(getLatestConfig(), getCurrentDatasourceTimelines(), stats); // Emit stats only if emission period has elapsed - if (sinceStatsEmitted.hasElapsed(METRIC_EMISSION_PERIOD)) { + if (!sinceStatsEmitted.isRunning() || sinceStatsEmitted.hasElapsed(METRIC_EMISSION_PERIOD)) { stats.forEachStat( (stat, dimensions, value) -> { if (stat.shouldEmit()) { @@ -277,7 +271,7 @@ public AutoCompactionSnapshot getCompactionSnapshot(String dataSource) } @Override - public Long getTotalSizeOfSegmentsAwaitingCompaction(String dataSource) + public Long getSegmentBytesAwaitingCompaction(String dataSource) { return duty.getTotalSizeOfSegmentsAwaitingCompaction(dataSource); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/DruidOverlord.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/DruidOverlord.java index 11705fed32f8..fb4730e7df40 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/DruidOverlord.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/DruidOverlord.java @@ -29,6 +29,7 @@ import org.apache.druid.indexing.common.actions.SegmentAllocationQueue; import org.apache.druid.indexing.common.actions.TaskActionClientFactory; import org.apache.druid.indexing.common.task.TaskContextEnricher; +import org.apache.druid.indexing.compact.CompactionScheduler; import org.apache.druid.indexing.overlord.config.DefaultTaskConfig; import org.apache.druid.indexing.overlord.config.TaskLockConfig; import org.apache.druid.indexing.overlord.config.TaskQueueConfig; @@ -40,7 +41,6 @@ import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.java.util.emitter.service.ServiceEmitter; import org.apache.druid.server.DruidNode; -import org.apache.druid.server.compaction.CompactionScheduler; import org.apache.druid.server.coordinator.CoordinatorOverlordServiceConfig; import java.util.concurrent.atomic.AtomicReference; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskQueryTool.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskQueryTool.java index c522a8a4a1b2..c51b84b0a762 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskQueryTool.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskQueryTool.java @@ -20,11 +20,11 @@ package org.apache.druid.indexing.overlord; import com.google.common.base.Optional; +import com.google.common.base.Supplier; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Maps; import com.google.inject.Inject; -import org.apache.druid.common.config.JacksonConfigManager; import org.apache.druid.indexer.TaskInfo; import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexer.TaskStatusPlus; @@ -68,7 +68,7 @@ public class TaskQueryTool private final TaskStorage storage; private final TaskLockbox taskLockbox; private final TaskMaster taskMaster; - private final JacksonConfigManager configManager; + private final Supplier workerBehaviorConfigSupplier; private final ProvisioningStrategy provisioningStrategy; @Inject @@ -77,13 +77,13 @@ public TaskQueryTool( TaskLockbox taskLockbox, TaskMaster taskMaster, ProvisioningStrategy provisioningStrategy, - JacksonConfigManager configManager + Supplier workerBehaviorConfigSupplier ) { this.storage = storage; this.taskLockbox = taskLockbox; this.taskMaster = taskMaster; - this.configManager = configManager; + this.workerBehaviorConfigSupplier = workerBehaviorConfigSupplier; this.provisioningStrategy = provisioningStrategy; } @@ -379,10 +379,7 @@ public TotalWorkerCapacityResponse getTotalWorkerCapacity() public WorkerBehaviorConfig getLatestWorkerConfig() { - return configManager.watch( - WorkerBehaviorConfig.CONFIG_KEY, - WorkerBehaviorConfig.class - ).get(); + return workerBehaviorConfigSupplier.get(); } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordCompactionResource.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordCompactionResource.java index 30b655650d90..62f7d1ddce7b 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordCompactionResource.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordCompactionResource.java @@ -21,8 +21,7 @@ import com.google.inject.Inject; import com.sun.jersey.spi.container.ResourceFilters; -import org.apache.druid.indexing.compact.OverlordCompactionScheduler; -import org.apache.druid.server.compaction.CompactionScheduler; +import org.apache.druid.indexing.compact.CompactionScheduler; import org.apache.druid.server.coordinator.AutoCompactionSnapshot; import org.apache.druid.server.coordinator.ClusterCompactionConfig; import org.apache.druid.server.http.security.StateResourceFilter; @@ -40,7 +39,7 @@ /** * Contains the same logic as {@code CompactionResource} but the APIs are served - * by {@link OverlordCompactionScheduler} instead of {@code DruidCoordinator}. + * by {@link CompactionScheduler} instead of {@code DruidCoordinator}. */ @Path("/druid/indexer/v1/compaction") public class OverlordCompactionResource @@ -63,7 +62,7 @@ public Response getCompactionProgress( @QueryParam("dataSource") String dataSource ) { - final Long notCompactedSegmentSizeBytes = scheduler.getTotalSizeOfSegmentsAwaitingCompaction(dataSource); + final Long notCompactedSegmentSizeBytes = scheduler.getSegmentBytesAwaitingCompaction(dataSource); if (notCompactedSegmentSizeBytes == null) { return Response.status(Response.Status.NOT_FOUND) .entity(Collections.singletonMap("error", "Unknown DataSource")) 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 new file mode 100644 index 000000000000..02ee9ea57b64 --- /dev/null +++ b/indexing-service/src/test/java/org/apache/druid/indexing/compact/OverlordCompactionSchedulerTest.java @@ -0,0 +1,300 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.compact; + +import com.fasterxml.jackson.databind.InjectableValues; +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.guice.IndexingServiceTuningConfigModule; +import org.apache.druid.indexing.common.SegmentCacheManagerFactory; +import org.apache.druid.indexing.common.config.TaskStorageConfig; +import org.apache.druid.indexing.common.task.CompactionTask; +import org.apache.druid.indexing.common.task.Task; +import org.apache.druid.indexing.overlord.HeapMemoryTaskStorage; +import org.apache.druid.indexing.overlord.TaskLockbox; +import org.apache.druid.indexing.overlord.TaskMaster; +import org.apache.druid.indexing.overlord.TaskQueryTool; +import org.apache.druid.indexing.overlord.TaskQueue; +import org.apache.druid.indexing.overlord.TaskRunner; +import org.apache.druid.indexing.overlord.setup.DefaultWorkerBehaviorConfig; +import org.apache.druid.indexing.overlord.setup.WorkerBehaviorConfig; +import org.apache.druid.indexing.test.TestIndexerMetadataStorageCoordinator; +import org.apache.druid.jackson.DefaultObjectMapper; +import org.apache.druid.java.util.metrics.StubServiceEmitter; +import org.apache.druid.segment.TestIndex; +import org.apache.druid.server.compaction.CompactionStatistics; +import org.apache.druid.server.compaction.CompactionStatusTracker; +import org.apache.druid.server.coordinator.AutoCompactionSnapshot; +import org.apache.druid.server.coordinator.ClusterCompactionConfig; +import org.apache.druid.server.coordinator.CompactionSchedulerConfig; +import org.apache.druid.server.coordinator.CoordinatorOverlordServiceConfig; +import org.apache.druid.server.coordinator.CreateDataSegments; +import org.apache.druid.server.coordinator.DataSourceCompactionConfig; +import org.apache.druid.server.coordinator.DruidCompactionConfig; +import org.apache.druid.server.coordinator.simulate.BlockingExecutorService; +import org.apache.druid.server.coordinator.simulate.TestSegmentsMetadataManager; +import org.apache.druid.server.coordinator.simulate.WrappingScheduledExecutorService; +import org.apache.druid.server.coordinator.stats.Stats; +import org.apache.druid.timeline.DataSegment; +import org.joda.time.Period; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import org.mockito.ArgumentCaptor; +import org.mockito.ArgumentMatchers; +import org.mockito.Mockito; + +import java.util.Collections; +import java.util.List; + +public class OverlordCompactionSchedulerTest +{ + private static final ObjectMapper OBJECT_MAPPER; + + static { + OBJECT_MAPPER = new DefaultObjectMapper(); + OBJECT_MAPPER.registerModules(new IndexingServiceTuningConfigModule().getJacksonModules()); + OBJECT_MAPPER.setInjectableValues( + new InjectableValues + .Std() + .addValue( + SegmentCacheManagerFactory.class, + new SegmentCacheManagerFactory(TestIndex.INDEX_IO, OBJECT_MAPPER) + ) + ); + } + + private CompactionSchedulerConfig schedulerConfig; + private ClusterCompactionConfig clusterCompactionConfig; + private CoordinatorOverlordServiceConfig coordinatorOverlordServiceConfig; + + private TaskMaster taskMaster; + private TaskRunner taskRunner; + private TaskQueue taskQueue; + private BlockingExecutorService executor; + + private HeapMemoryTaskStorage taskStorage; + private TestSegmentsMetadataManager segmentsMetadataManager; + private StubServiceEmitter serviceEmitter; + + private OverlordCompactionScheduler scheduler; + + @Before + public void setUp() + { + taskRunner = Mockito.mock(TaskRunner.class); + taskQueue = Mockito.mock(TaskQueue.class); + + taskMaster = new TaskMaster(null, null); + taskMaster.becomeLeader(taskRunner, taskQueue); + + taskStorage = new HeapMemoryTaskStorage(new TaskStorageConfig(null)); + + executor = new BlockingExecutorService("test"); + serviceEmitter = new StubServiceEmitter(); + segmentsMetadataManager = new TestSegmentsMetadataManager(); + + schedulerConfig = new CompactionSchedulerConfig(true); + clusterCompactionConfig = DruidCompactionConfig.empty().clusterConfig(); + coordinatorOverlordServiceConfig = new CoordinatorOverlordServiceConfig(false, null); + + initScheduler(); + } + + private void initScheduler() + { + TaskLockbox taskLockbox = new TaskLockbox(taskStorage, new TestIndexerMetadataStorageCoordinator()); + WorkerBehaviorConfig defaultWorkerConfig + = new DefaultWorkerBehaviorConfig(WorkerBehaviorConfig.DEFAULT_STRATEGY, null); + scheduler = new OverlordCompactionScheduler( + taskMaster, + new TaskQueryTool(taskStorage, taskLockbox, taskMaster, null, () -> defaultWorkerConfig), + segmentsMetadataManager, + () -> clusterCompactionConfig, + new CompactionStatusTracker(OBJECT_MAPPER), + schedulerConfig, + coordinatorOverlordServiceConfig, + (nameFormat, numThreads) -> new WrappingScheduledExecutorService("test", executor, false), + serviceEmitter, + OBJECT_MAPPER + ); + } + + @After + public void tearDown() + { + + } + + @Test + public void testStartStopWhenSchedulerIsEnabled() + { + schedulerConfig = new CompactionSchedulerConfig(true); + Assert.assertFalse(scheduler.isRunning()); + + scheduler.start(); + Assert.assertTrue(scheduler.isRunning()); + Assert.assertTrue(executor.hasPendingTasks()); + scheduler.stop(); + Assert.assertFalse(scheduler.isRunning()); + Assert.assertTrue(executor.hasPendingTasks()); + + scheduler.start(); + Assert.assertTrue(scheduler.isRunning()); + scheduler.stop(); + Assert.assertFalse(scheduler.isRunning()); + } + + @Test + public void testStartStopWhenScheduledIsDisabled() + { + schedulerConfig = new CompactionSchedulerConfig(false); + initScheduler(); + + Assert.assertFalse(scheduler.isRunning()); + scheduler.start(); + Assert.assertFalse(scheduler.isRunning()); + Assert.assertFalse(executor.hasPendingTasks()); + scheduler.stop(); + Assert.assertFalse(scheduler.isRunning()); + Assert.assertFalse(executor.hasPendingTasks()); + } + + @Test + public void testSegmentsAreNotPolledWhenSchedulerIsDisabled() + { + schedulerConfig = new CompactionSchedulerConfig(false); + initScheduler(); + + verifySegmentPolling(false); + } + + @Test + public void testSegmentsArePolledWhenRunningInStandaloneMode() + { + coordinatorOverlordServiceConfig = new CoordinatorOverlordServiceConfig(false, null); + initScheduler(); + + verifySegmentPolling(true); + } + + @Test + public void testSegmentsAreNotPolledWhenRunningInCoordinatorMode() + { + coordinatorOverlordServiceConfig = new CoordinatorOverlordServiceConfig(true, "overlord"); + initScheduler(); + + verifySegmentPolling(false); + } + + private void verifySegmentPolling(boolean enabled) + { + scheduler.start(); + Assert.assertEquals(enabled, segmentsMetadataManager.isPollingDatabasePeriodically()); + + scheduler.stop(); + Assert.assertFalse(segmentsMetadataManager.isPollingDatabasePeriodically()); + } + + @Test + public void testStartCompactionForDatasource() + { + final List wikiSegments = CreateDataSegments.ofDatasource(DS.WIKI).eachOfSizeInMb(100); + wikiSegments.forEach(segmentsMetadataManager::addSegment); + + scheduler.start(); + scheduler.startCompaction( + DS.WIKI, + DataSourceCompactionConfig.builder() + .forDataSource(DS.WIKI) + .withSkipOffsetFromLatest(Period.seconds(0)) + .build() + ); + + executor.finishNextPendingTask(); + + ArgumentCaptor taskArgumentCaptor = ArgumentCaptor.forClass(Task.class); + Mockito.verify(taskQueue, Mockito.times(1)).add(taskArgumentCaptor.capture()); + + Task submittedTask = taskArgumentCaptor.getValue(); + Assert.assertNotNull(submittedTask); + Assert.assertTrue(submittedTask instanceof CompactionTask); + + final CompactionTask compactionTask = (CompactionTask) submittedTask; + Assert.assertEquals(DS.WIKI, compactionTask.getDataSource()); + + final AutoCompactionSnapshot.Builder expectedSnapshot = AutoCompactionSnapshot.builder(DS.WIKI); + expectedSnapshot.incrementCompactedStats(CompactionStatistics.create(100_000_000, 1, 1)); + + Assert.assertEquals( + expectedSnapshot.build(), + scheduler.getCompactionSnapshot(DS.WIKI) + ); + Assert.assertEquals( + Collections.singletonMap(DS.WIKI, expectedSnapshot.build()), + scheduler.getAllCompactionSnapshots() + ); + Assert.assertEquals( + 0L, + scheduler.getSegmentBytesAwaitingCompaction(DS.WIKI).longValue() + ); + + serviceEmitter.verifyValue(Stats.Compaction.SUBMITTED_TASKS.getMetricName(), 1L); + serviceEmitter.verifyValue(Stats.Compaction.COMPACTED_BYTES.getMetricName(), 100_000_000L); + + scheduler.stop(); + } + + @Test + public void testStopCompactionForDatasource() + { + final List wikiSegments = CreateDataSegments.ofDatasource(DS.WIKI).eachOfSizeInMb(100); + wikiSegments.forEach(segmentsMetadataManager::addSegment); + + scheduler.start(); + scheduler.startCompaction( + DS.WIKI, + DataSourceCompactionConfig.builder() + .forDataSource(DS.WIKI) + .withSkipOffsetFromLatest(Period.seconds(0)) + .build() + ); + scheduler.stopCompaction(DS.WIKI); + + executor.finishNextPendingTask(); + + Mockito.verify(taskQueue, Mockito.never()).add(ArgumentMatchers.any()); + + Assert.assertNull(scheduler.getCompactionSnapshot(DS.WIKI)); + Assert.assertTrue(scheduler.getAllCompactionSnapshots().isEmpty()); + Assert.assertNull(scheduler.getSegmentBytesAwaitingCompaction(DS.WIKI)); + + serviceEmitter.verifyNotEmitted(Stats.Compaction.SUBMITTED_TASKS.getMetricName()); + serviceEmitter.verifyNotEmitted(Stats.Compaction.COMPACTED_BYTES.getMetricName()); + + scheduler.stop(); + } + + private static class DS + { + static final String WIKI = "wiki"; + } + +} diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/http/OverlordCompactionResourceTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/http/OverlordCompactionResourceTest.java new file mode 100644 index 000000000000..83433a249a2d --- /dev/null +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/http/OverlordCompactionResourceTest.java @@ -0,0 +1,25 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.overlord.http; + +public class OverlordCompactionResourceTest +{ + +} diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/http/OverlordResourceTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/http/OverlordResourceTest.java index 93f80ac97096..c13a15717656 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/http/OverlordResourceTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/http/OverlordResourceTest.java @@ -145,7 +145,7 @@ public void setUp() taskLockbox, taskMaster, provisioningStrategy, - configManager + () -> configManager.watch(WorkerBehaviorConfig.CONFIG_KEY, WorkerBehaviorConfig.class).get() ); indexerMetadataStorageAdapter = EasyMock.createStrictMock(IndexerMetadataStorageAdapter.class); req = EasyMock.createStrictMock(HttpServletRequest.class); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/http/OverlordTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/http/OverlordTest.java index 05a65bb32050..f1a8b65a509f 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/http/OverlordTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/http/OverlordTest.java @@ -47,6 +47,7 @@ import org.apache.druid.indexing.common.task.NoopTask; import org.apache.druid.indexing.common.task.NoopTaskContextEnricher; import org.apache.druid.indexing.common.task.Task; +import org.apache.druid.indexing.compact.CompactionScheduler; import org.apache.druid.indexing.overlord.DruidOverlord; import org.apache.druid.indexing.overlord.HeapMemoryTaskStorage; import org.apache.druid.indexing.overlord.IndexerMetadataStorageAdapter; @@ -74,7 +75,6 @@ import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.java.util.emitter.service.ServiceEmitter; import org.apache.druid.server.DruidNode; -import org.apache.druid.server.compaction.CompactionScheduler; import org.apache.druid.server.coordinator.CoordinatorOverlordServiceConfig; import org.apache.druid.server.metrics.NoopServiceEmitter; import org.apache.druid.server.security.AuthConfig; diff --git a/server/src/main/java/org/apache/druid/server/compaction/Table.java b/server/src/main/java/org/apache/druid/server/compaction/Table.java index 068a7abacbae..9e48818f6499 100644 --- a/server/src/main/java/org/apache/druid/server/compaction/Table.java +++ b/server/src/main/java/org/apache/druid/server/compaction/Table.java @@ -72,4 +72,13 @@ public boolean isEmpty() { return rows.isEmpty(); } + + @Override + public String toString() + { + return "Table{" + + "columnNames=" + columnNames + + ", rows=" + rows + + '}'; + } } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/AutoCompactionSnapshot.java b/server/src/main/java/org/apache/druid/server/coordinator/AutoCompactionSnapshot.java index 7c1dcd1ee9f4..47d6671e0dd6 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/AutoCompactionSnapshot.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/AutoCompactionSnapshot.java @@ -190,6 +190,24 @@ public int hashCode() ); } + @Override + public String toString() + { + return "AutoCompactionSnapshot{" + + "dataSource='" + dataSource + '\'' + + ", scheduleStatus=" + scheduleStatus + + ", bytesAwaitingCompaction=" + bytesAwaitingCompaction + + ", bytesCompacted=" + bytesCompacted + + ", bytesSkipped=" + bytesSkipped + + ", segmentCountAwaitingCompaction=" + segmentCountAwaitingCompaction + + ", segmentCountCompacted=" + segmentCountCompacted + + ", segmentCountSkipped=" + segmentCountSkipped + + ", intervalCountAwaitingCompaction=" + intervalCountAwaitingCompaction + + ", intervalCountCompacted=" + intervalCountCompacted + + ", intervalCountSkipped=" + intervalCountSkipped + + '}'; + } + public static class Builder { private final String dataSource; 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 ca11121efbbe..8f0697bcb408 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 @@ -28,7 +28,8 @@ import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.granularity.Granularities; -import org.apache.druid.java.util.common.granularity.Granularity; +import org.apache.druid.segment.IndexSpec; +import org.apache.druid.segment.data.CompressionStrategy; import org.apache.druid.segment.indexing.granularity.GranularitySpec; import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; @@ -39,7 +40,6 @@ import org.junit.Assert; import org.junit.Test; -import java.util.Arrays; import java.util.Collections; public class CompactionStatusTest @@ -47,6 +47,14 @@ public class CompactionStatusTest private static final ObjectMapper OBJECT_MAPPER = new DefaultObjectMapper(); private static final String DS_WIKI = "wiki"; + private static final DataSegment WIKI_SEGMENT + = DataSegment.builder() + .dataSource(DS_WIKI) + .interval(Intervals.of("2013-01-01/PT1H")) + .size(100_000_000L) + .version("v1") + .build(); + @Test public void testFindPartitionsSpecWhenGivenIsNull() { @@ -166,74 +174,179 @@ public void testFindPartitionsSpecWhenGivenIsRange() ); } + @Test + public void testStatusWhenLastCompactionStateIsNull() + { + verifyCompactionStatusIsPendingBecause( + null, + DataSourceCompactionConfig.builder().forDataSource(DS_WIKI).build(), + "not compacted yet" + ); + } + + @Test + public void testStatusWhenLastCompactionStateIsEmpty() + { + verifyCompactionStatusIsPendingBecause( + new CompactionState(null, null, null, null, null, null), + DataSourceCompactionConfig.builder().forDataSource(DS_WIKI).build(), + "'partitionsSpec' mismatch: required['dynamic' with 5,000,000 rows], current[null]" + ); + } + + @Test + public void testStatusOnPartitionsSpecMismatch() + { + final PartitionsSpec currentPartitionsSpec = new DynamicPartitionsSpec(100, 0L); + + final CompactionState lastCompactionState + = new CompactionState(currentPartitionsSpec, null, null, null, null, null); + final DataSourceCompactionConfig compactionConfig + = DataSourceCompactionConfig.builder().forDataSource(DS_WIKI).build(); + + verifyCompactionStatusIsPendingBecause( + lastCompactionState, + compactionConfig, + "'partitionsSpec' mismatch: required['dynamic' with 5,000,000 rows]," + + " current['dynamic' with 100 rows]" + ); + } + + @Test + public void testStatusOnIndexSpecMismatch() + { + final IndexSpec currentIndexSpec + = IndexSpec.builder().withDimensionCompression(CompressionStrategy.ZSTD).build(); + + final PartitionsSpec currentPartitionsSpec = new DynamicPartitionsSpec(100, 0L); + final CompactionState lastCompactionState = new CompactionState( + currentPartitionsSpec, + null, + null, + null, + currentIndexSpec.asMap(OBJECT_MAPPER), + null + ); + final DataSourceCompactionConfig compactionConfig = DataSourceCompactionConfig + .builder() + .forDataSource(DS_WIKI) + .withTuningConfig(createTuningConfig(currentPartitionsSpec, null)) + .build(); + + verifyCompactionStatusIsPendingBecause( + lastCompactionState, + compactionConfig, + "'indexSpec' mismatch: " + + "required[IndexSpec{bitmapSerdeFactory=RoaringBitmapSerdeFactory{}," + + " dimensionCompression=lz4, stringDictionaryEncoding=Utf8{}," + + " metricCompression=lz4, longEncoding=longs, jsonCompression=null, segmentLoader=null}], " + + "current[IndexSpec{bitmapSerdeFactory=RoaringBitmapSerdeFactory{}," + + " dimensionCompression=zstd, stringDictionaryEncoding=Utf8{}," + + " metricCompression=lz4, longEncoding=longs, jsonCompression=null, segmentLoader=null}]" + ); + } + @Test public void testStatusOnSegmentGranularityMismatch() { - final GranularitySpec segmentGranularitySpec - = new UniformGranularitySpec(Granularities.HOUR, null, null, null); - final CompactionState segmentLastCompactionState = new CompactionState( + final GranularitySpec currentGranularitySpec + = new UniformGranularitySpec(Granularities.HOUR, null, null); + + final PartitionsSpec currentPartitionsSpec = new DynamicPartitionsSpec(100, 0L); + final IndexSpec currentIndexSpec + = IndexSpec.builder().withDimensionCompression(CompressionStrategy.ZSTD).build(); + final CompactionState lastCompactionState = new CompactionState( + currentPartitionsSpec, + null, null, null, + currentIndexSpec.asMap(OBJECT_MAPPER), + currentGranularitySpec.asMap(OBJECT_MAPPER) + ); + final DataSourceCompactionConfig compactionConfig = DataSourceCompactionConfig + .builder() + .forDataSource(DS_WIKI) + .withTuningConfig(createTuningConfig(currentPartitionsSpec, currentIndexSpec)) + .withGranularitySpec(new UserCompactionTaskGranularityConfig(Granularities.DAY, null, null)) + .build(); + + verifyCompactionStatusIsPendingBecause( + lastCompactionState, + compactionConfig, + "'segmentGranularity' mismatch: required[DAY], current[HOUR]" + ); + } + + @Test + public void testStatusWhenLastCompactionStateSameAsRequired() + { + final GranularitySpec currentGranularitySpec + = new UniformGranularitySpec(Granularities.HOUR, null, null); + final PartitionsSpec currentPartitionsSpec = new DynamicPartitionsSpec(100, 0L); + final IndexSpec currentIndexSpec + = IndexSpec.builder().withDimensionCompression(CompressionStrategy.ZSTD).build(); + final CompactionState lastCompactionState = new CompactionState( + currentPartitionsSpec, null, null, null, - segmentGranularitySpec.asMap(OBJECT_MAPPER) + currentIndexSpec.asMap(OBJECT_MAPPER), + currentGranularitySpec.asMap(OBJECT_MAPPER) ); - final DataSegment segment - = DataSegment.builder() - .dataSource(DS_WIKI) - .interval(Intervals.of("2013-01-01/PT1H")) - .size(100_000_000L) - .version("v1") - .lastCompactionState(segmentLastCompactionState) - .build(); + final DataSourceCompactionConfig compactionConfig = DataSourceCompactionConfig + .builder() + .forDataSource(DS_WIKI) + .withTuningConfig(createTuningConfig(currentPartitionsSpec, currentIndexSpec)) + .withGranularitySpec(new UserCompactionTaskGranularityConfig(Granularities.HOUR, null, null)) + .build(); + final DataSegment segment = DataSegment.builder(WIKI_SEGMENT).lastCompactionState(lastCompactionState).build(); final CompactionStatus status = CompactionStatus.compute( SegmentsToCompact.from(Collections.singletonList(segment)), - createCompactionConfig(Granularities.DAY), + compactionConfig, OBJECT_MAPPER ); - - Assert.assertFalse(status.isComplete()); - Assert.assertEquals( - "segmentGranularity", - status.getReason() - ); + Assert.assertTrue(status.isComplete()); } - private static DataSourceCompactionConfig createCompactionConfig( - Granularity segmentGranularity + private void verifyCompactionStatusIsPendingBecause( + CompactionState lastCompactionState, + DataSourceCompactionConfig compactionConfig, + String expectedReason ) { - return new DataSourceCompactionConfig( - DS_WIKI, - null, null, null, null, - createTuningConfig( - new DimensionRangePartitionsSpec(1_000_000, null, Arrays.asList("countryName", "cityName"), false) - ), - new UserCompactionTaskGranularityConfig(segmentGranularity, null, null), - null, null, null, null, null, null + final DataSegment segment + = DataSegment.builder(WIKI_SEGMENT) + .lastCompactionState(lastCompactionState) + .build(); + final CompactionStatus status = CompactionStatus.compute( + SegmentsToCompact.from(Collections.singletonList(segment)), + compactionConfig, + OBJECT_MAPPER ); + + Assert.assertFalse(status.isComplete()); + Assert.assertEquals(expectedReason, status.getReason()); } private static DataSourceCompactionConfig createCompactionConfig( PartitionsSpec partitionsSpec ) { - return new DataSourceCompactionConfig( - DS_WIKI, - null, null, null, null, createTuningConfig(partitionsSpec), - null, null, null, null, null, null, null - ); + return DataSourceCompactionConfig.builder() + .forDataSource(DS_WIKI) + .withTuningConfig(createTuningConfig(partitionsSpec, null)) + .build(); } private static UserCompactionTaskQueryTuningConfig createTuningConfig( - PartitionsSpec partitionsSpec + PartitionsSpec partitionsSpec, + IndexSpec indexSpec ) { return new UserCompactionTaskQueryTuningConfig( null, - null, null, null, null, partitionsSpec, null, null, null, + null, null, null, null, partitionsSpec, indexSpec, null, null, null, null, null, null, null, null, null, null, null, null ); } diff --git a/server/src/test/java/org/apache/druid/server/coordinator/simulate/TestSegmentsMetadataManager.java b/server/src/test/java/org/apache/druid/server/coordinator/simulate/TestSegmentsMetadataManager.java index 4793c25f9167..934f61dbe07e 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/simulate/TestSegmentsMetadataManager.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/simulate/TestSegmentsMetadataManager.java @@ -47,6 +47,7 @@ public class TestSegmentsMetadataManager implements SegmentsMetadataManager private final ConcurrentMap usedSegments = new ConcurrentHashMap<>(); private volatile DataSourcesSnapshot snapshot; + private volatile boolean pollingStarted; public void addSegment(DataSegment segment) { @@ -65,19 +66,19 @@ public void removeSegment(DataSegment segment) @Override public void startPollingDatabasePeriodically() { - + this.pollingStarted = true; } @Override public void stopPollingDatabasePeriodically() { - + this.pollingStarted = false; } @Override public boolean isPollingDatabasePeriodically() { - return true; + return pollingStarted; } @Override diff --git a/services/src/main/java/org/apache/druid/cli/CliOverlord.java b/services/src/main/java/org/apache/druid/cli/CliOverlord.java index f877685765bc..ebe521967fc3 100644 --- a/services/src/main/java/org/apache/druid/cli/CliOverlord.java +++ b/services/src/main/java/org/apache/druid/cli/CliOverlord.java @@ -69,6 +69,7 @@ import org.apache.druid.indexing.common.task.batch.parallel.ShuffleClient; import org.apache.druid.indexing.common.tasklogs.SwitchingTaskLogStreamer; import org.apache.druid.indexing.common.tasklogs.TaskRunnerTaskLogStreamer; +import org.apache.druid.indexing.compact.CompactionScheduler; import org.apache.druid.indexing.compact.OverlordCompactionScheduler; import org.apache.druid.indexing.overlord.DruidOverlord; import org.apache.druid.indexing.overlord.ForkingTaskRunnerFactory; @@ -117,7 +118,6 @@ import org.apache.druid.segment.realtime.NoopChatHandlerProvider; import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager; import org.apache.druid.segment.realtime.appenderator.DummyForInjectionAppenderatorsManager; -import org.apache.druid.server.compaction.CompactionScheduler; import org.apache.druid.server.compaction.CompactionStatusTracker; import org.apache.druid.server.coordinator.ClusterCompactionConfig; import org.apache.druid.server.coordinator.CompactionSchedulerConfig; @@ -430,6 +430,16 @@ public Supplier getClusterCompactionConfig(JacksonConfi .clusterConfig(); }; } + + @Provides + @LazySingleton + public Supplier getWorkerBehaviourConfig(JacksonConfigManager configManager) + { + return () -> configManager.watch( + WorkerBehaviorConfig.CONFIG_KEY, + WorkerBehaviorConfig.class + ).get(); + } }; } From b2ae5608e3ad60718e6326076c9f4753f9d8a729 Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Fri, 9 Aug 2024 15:09:07 +0530 Subject: [PATCH 11/26] Add more tests --- .../compact/CompactionSupervisor.java | 4 +- .../compact/CompactionSupervisorSpec.java | 7 +- .../compact/OverlordCompactionScheduler.java | 8 +- .../compact/CompactionSupervisorSpecTest.java | 205 ++++++++++++++++++ .../OverlordCompactionSchedulerTest.java | 9 +- .../indexing/compact/TestDataSource.java | 26 +++ .../org/apache/druid/cli/CliOverlord.java | 34 +-- 7 files changed, 250 insertions(+), 43 deletions(-) create mode 100644 indexing-service/src/test/java/org/apache/druid/indexing/compact/CompactionSupervisorSpecTest.java create mode 100644 indexing-service/src/test/java/org/apache/druid/indexing/compact/TestDataSource.java diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionSupervisor.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionSupervisor.java index 61023d631bad..fcdad7d0174a 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionSupervisor.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionSupervisor.java @@ -87,7 +87,7 @@ public SupervisorReport getStatus() public SupervisorStateManager.State getState() { if (!scheduler.isRunning()) { - return State.SCHEDULER_DISABLED; + return State.SCHEDULER_STOPPED; } else if (supervisorSpec.isSuspended()) { return State.SUSPENDED; } else { @@ -134,7 +134,7 @@ public int getActiveTaskGroupsCount() public enum State implements SupervisorStateManager.State { - SCHEDULER_DISABLED(true), + SCHEDULER_STOPPED(true), RUNNING(true), SUSPENDED(true), UNHEALTHY(false); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionSupervisorSpec.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionSupervisorSpec.java index 55297f56d89f..02718499e740 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionSupervisorSpec.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionSupervisorSpec.java @@ -24,7 +24,6 @@ import com.fasterxml.jackson.annotation.JsonProperty; import org.apache.druid.common.config.Configs; import org.apache.druid.error.InvalidInput; -import org.apache.druid.indexing.overlord.supervisor.Supervisor; import org.apache.druid.indexing.overlord.supervisor.SupervisorSpec; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; @@ -77,7 +76,7 @@ public String getId() } @Override - public Supervisor createSupervisor() + public CompactionSupervisor createSupervisor() { return new CompactionSupervisor(this, scheduler); } @@ -89,13 +88,13 @@ public List getDataSources() } @Override - public SupervisorSpec createSuspendedSpec() + public CompactionSupervisorSpec createSuspendedSpec() { return new CompactionSupervisorSpec(spec, true, scheduler); } @Override - public SupervisorSpec createRunningSpec() + public CompactionSupervisorSpec createRunningSpec() { return new CompactionSupervisorSpec(spec, false, scheduler); } 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 6021672b14dc..39d52bac9d5a 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 @@ -87,7 +87,7 @@ public class OverlordCompactionScheduler implements CompactionScheduler private final ServiceEmitter emitter; private final CompactionSchedulerConfig schedulerConfig; - private final Supplier clusterCompactionConfigSupplier; + private final Supplier compactionConfigSupplier; private final ConcurrentHashMap activeDatasourceConfigs; /** @@ -114,7 +114,7 @@ public OverlordCompactionScheduler( TaskMaster taskMaster, TaskQueryTool taskQueryTool, SegmentsMetadataManager segmentManager, - Supplier clusterCompactionConfigSupplier, + Supplier compactionConfigSupplier, CompactionStatusTracker statusTracker, CompactionSchedulerConfig schedulerConfig, CoordinatorOverlordServiceConfig coordinatorOverlordServiceConfig, @@ -127,7 +127,7 @@ public OverlordCompactionScheduler( this.segmentManager = segmentManager; this.emitter = emitter; this.schedulerConfig = schedulerConfig; - this.clusterCompactionConfigSupplier = clusterCompactionConfigSupplier; + this.compactionConfigSupplier = compactionConfigSupplier; this.executor = executorFactory.create(1, "CompactionScheduler-%s"); this.statusTracker = statusTracker; @@ -308,7 +308,7 @@ private DruidCompactionConfig getLatestConfig() { return DruidCompactionConfig .empty() - .withClusterConfig(clusterCompactionConfigSupplier.get()) + .withClusterConfig(compactionConfigSupplier.get().clusterConfig()) .withDatasourceConfigs(new ArrayList<>(activeDatasourceConfigs.values())); } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/compact/CompactionSupervisorSpecTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/compact/CompactionSupervisorSpecTest.java new file mode 100644 index 000000000000..bea0e5165488 --- /dev/null +++ b/indexing-service/src/test/java/org/apache/druid/indexing/compact/CompactionSupervisorSpecTest.java @@ -0,0 +1,205 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.compact; + +import com.fasterxml.jackson.databind.InjectableValues; +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.error.DruidException; +import org.apache.druid.guice.SupervisorModule; +import org.apache.druid.indexing.overlord.supervisor.SupervisorSpec; +import org.apache.druid.jackson.DefaultObjectMapper; +import org.apache.druid.server.coordinator.DataSourceCompactionConfig; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import org.mockito.Mockito; + +import java.util.Collections; + +public class CompactionSupervisorSpecTest +{ + private static final ObjectMapper OBJECT_MAPPER = new DefaultObjectMapper(); + private CompactionScheduler scheduler; + + @Before + public void setUp() + { + scheduler = Mockito.mock(CompactionScheduler.class); + OBJECT_MAPPER.setInjectableValues( + new InjectableValues.Std() + .addValue(CompactionScheduler.class, scheduler) + ); + OBJECT_MAPPER.registerModules( + new SupervisorModule().getJacksonModules() + ); + } + + @Test + public void testSerdeOfActiveSpec() + { + testSerde( + new CompactionSupervisorSpec( + DataSourceCompactionConfig.builder().forDataSource(TestDataSource.WIKI).build(), + false, + scheduler + ) + ); + } + + @Test + public void testSerdeOfSuspendedSpec() + { + testSerde( + new CompactionSupervisorSpec( + DataSourceCompactionConfig.builder().forDataSource(TestDataSource.WIKI).build(), + true, + scheduler + ) + ); + } + + @Test + public void testGetIdAndDataSources() + { + final CompactionSupervisorSpec activeSpec = new CompactionSupervisorSpec( + DataSourceCompactionConfig.builder().forDataSource(TestDataSource.WIKI).build(), + false, + scheduler + ); + Assert.assertEquals("autocompact__wiki", activeSpec.getId()); + Assert.assertEquals(Collections.singletonList(TestDataSource.WIKI), activeSpec.getDataSources()); + Assert.assertFalse(activeSpec.isSuspended()); + } + + @Test + public void testStartStopSupervisorForActiveSpec() + { + Mockito.when(scheduler.isRunning()).thenReturn(true); + + final DataSourceCompactionConfig spec + = DataSourceCompactionConfig.builder() + .forDataSource(TestDataSource.WIKI) + .build(); + final CompactionSupervisorSpec activeSpec + = new CompactionSupervisorSpec(spec, false, scheduler); + + final CompactionSupervisor supervisor = activeSpec.createSupervisor(); + Assert.assertEquals(CompactionSupervisor.State.RUNNING, supervisor.getState()); + + supervisor.start(); + supervisor.stop(false); + + Mockito.verify(scheduler, Mockito.times(1)).startCompaction(TestDataSource.WIKI, spec); + Mockito.verify(scheduler, Mockito.times(1)).stopCompaction(TestDataSource.WIKI); + } + + @Test + public void testStartStopSupervisorWhenSchedulerStopped() + { + final DataSourceCompactionConfig spec + = DataSourceCompactionConfig.builder() + .forDataSource(TestDataSource.WIKI) + .build(); + final CompactionSupervisorSpec activeSpec + = new CompactionSupervisorSpec(spec, false, scheduler); + + final CompactionSupervisor supervisor = activeSpec.createSupervisor(); + Assert.assertEquals(CompactionSupervisor.State.SCHEDULER_STOPPED, supervisor.getState()); + + supervisor.start(); + supervisor.stop(false); + + Mockito.verify(scheduler, Mockito.times(1)).startCompaction(TestDataSource.WIKI, spec); + Mockito.verify(scheduler, Mockito.times(1)).stopCompaction(TestDataSource.WIKI); + } + + @Test + public void testStartStopSupervisorForSuspendedSpec() + { + Mockito.when(scheduler.isRunning()).thenReturn(true); + + final DataSourceCompactionConfig spec + = DataSourceCompactionConfig.builder() + .forDataSource(TestDataSource.WIKI) + .build(); + final CompactionSupervisorSpec suspendedSpec + = new CompactionSupervisorSpec(spec, true, scheduler); + + final CompactionSupervisor supervisor = suspendedSpec.createSupervisor(); + Assert.assertEquals(CompactionSupervisor.State.SUSPENDED, supervisor.getState()); + + supervisor.start(); + supervisor.stop(false); + + Mockito.verify(scheduler, Mockito.times(2)).stopCompaction(TestDataSource.WIKI); + } + + @Test + public void testCreateSuspendedSpec() + { + final CompactionSupervisorSpec activeSpec = new CompactionSupervisorSpec( + DataSourceCompactionConfig.builder().forDataSource(TestDataSource.WIKI).build(), + false, + scheduler + ); + Assert.assertFalse(activeSpec.isSuspended()); + + final CompactionSupervisorSpec suspendedSpec = activeSpec.createSuspendedSpec(); + Assert.assertTrue(suspendedSpec.isSuspended()); + Assert.assertEquals(activeSpec.getId(), suspendedSpec.getId()); + Assert.assertEquals(activeSpec.getSpec(), suspendedSpec.getSpec()); + Assert.assertEquals(activeSpec.getDataSources(), suspendedSpec.getDataSources()); + } + + @Test + public void testCreateRunningSpec() + { + final CompactionSupervisorSpec suspendedSpec = new CompactionSupervisorSpec( + DataSourceCompactionConfig.builder().forDataSource(TestDataSource.WIKI).build(), + true, + scheduler + ); + Assert.assertTrue(suspendedSpec.isSuspended()); + + final CompactionSupervisorSpec activeSpec = suspendedSpec.createRunningSpec(); + Assert.assertFalse(activeSpec.isSuspended()); + Assert.assertEquals(activeSpec.getId(), suspendedSpec.getId()); + Assert.assertEquals(activeSpec.getSpec(), suspendedSpec.getSpec()); + Assert.assertEquals(activeSpec.getDataSources(), suspendedSpec.getDataSources()); + } + + private void testSerde(CompactionSupervisorSpec spec) + { + try { + String json = OBJECT_MAPPER.writeValueAsString(spec); + SupervisorSpec deserialized = OBJECT_MAPPER.readValue(json, SupervisorSpec.class); + Assert.assertTrue(deserialized instanceof CompactionSupervisorSpec); + + final CompactionSupervisorSpec observedSpec = (CompactionSupervisorSpec) deserialized; + Assert.assertEquals(spec.isSuspended(), observedSpec.isSuspended()); + Assert.assertEquals(spec.getSpec(), observedSpec.getSpec()); + Assert.assertEquals(spec.getId(), observedSpec.getId()); + Assert.assertEquals(spec.getDataSources(), observedSpec.getDataSources()); + } + catch (Exception e) { + throw DruidException.defensive(e, "Error while performing serde"); + } + } +} 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 02ee9ea57b64..e74c430ae8d9 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 @@ -41,7 +41,6 @@ import org.apache.druid.server.compaction.CompactionStatistics; import org.apache.druid.server.compaction.CompactionStatusTracker; import org.apache.druid.server.coordinator.AutoCompactionSnapshot; -import org.apache.druid.server.coordinator.ClusterCompactionConfig; import org.apache.druid.server.coordinator.CompactionSchedulerConfig; import org.apache.druid.server.coordinator.CoordinatorOverlordServiceConfig; import org.apache.druid.server.coordinator.CreateDataSegments; @@ -67,7 +66,7 @@ public class OverlordCompactionSchedulerTest { private static final ObjectMapper OBJECT_MAPPER; - + static { OBJECT_MAPPER = new DefaultObjectMapper(); OBJECT_MAPPER.registerModules(new IndexingServiceTuningConfigModule().getJacksonModules()); @@ -82,7 +81,7 @@ public class OverlordCompactionSchedulerTest } private CompactionSchedulerConfig schedulerConfig; - private ClusterCompactionConfig clusterCompactionConfig; + private DruidCompactionConfig compactionConfig; private CoordinatorOverlordServiceConfig coordinatorOverlordServiceConfig; private TaskMaster taskMaster; @@ -112,7 +111,7 @@ public void setUp() segmentsMetadataManager = new TestSegmentsMetadataManager(); schedulerConfig = new CompactionSchedulerConfig(true); - clusterCompactionConfig = DruidCompactionConfig.empty().clusterConfig(); + compactionConfig = DruidCompactionConfig.empty(); coordinatorOverlordServiceConfig = new CoordinatorOverlordServiceConfig(false, null); initScheduler(); @@ -127,7 +126,7 @@ private void initScheduler() taskMaster, new TaskQueryTool(taskStorage, taskLockbox, taskMaster, null, () -> defaultWorkerConfig), segmentsMetadataManager, - () -> clusterCompactionConfig, + () -> compactionConfig, new CompactionStatusTracker(OBJECT_MAPPER), schedulerConfig, coordinatorOverlordServiceConfig, diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/compact/TestDataSource.java b/indexing-service/src/test/java/org/apache/druid/indexing/compact/TestDataSource.java new file mode 100644 index 000000000000..f111a6bbbccf --- /dev/null +++ b/indexing-service/src/test/java/org/apache/druid/indexing/compact/TestDataSource.java @@ -0,0 +1,26 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.compact; + +public final class TestDataSource +{ + public static final String WIKI = "wiki"; + public static final String KOALA = "koala"; +} diff --git a/services/src/main/java/org/apache/druid/cli/CliOverlord.java b/services/src/main/java/org/apache/druid/cli/CliOverlord.java index ebe521967fc3..f759701ba77b 100644 --- a/services/src/main/java/org/apache/druid/cli/CliOverlord.java +++ b/services/src/main/java/org/apache/druid/cli/CliOverlord.java @@ -38,8 +38,6 @@ import com.google.inject.servlet.GuiceFilter; import com.google.inject.util.Providers; import org.apache.druid.client.indexing.IndexingService; -import org.apache.druid.common.config.Configs; -import org.apache.druid.common.config.JacksonConfigManager; import org.apache.druid.discovery.NodeRole; import org.apache.druid.guice.IndexingServiceInputSourceModule; import org.apache.druid.guice.IndexingServiceModuleHelper; @@ -119,7 +117,6 @@ import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager; import org.apache.druid.segment.realtime.appenderator.DummyForInjectionAppenderatorsManager; import org.apache.druid.server.compaction.CompactionStatusTracker; -import org.apache.druid.server.coordinator.ClusterCompactionConfig; import org.apache.druid.server.coordinator.CompactionSchedulerConfig; import org.apache.druid.server.coordinator.CoordinatorOverlordServiceConfig; import org.apache.druid.server.coordinator.DruidCompactionConfig; @@ -394,6 +391,12 @@ public void configure(Binder binder) binder.bind(HttpRemoteTaskRunnerFactory.class).in(LazySingleton.class); JacksonConfigProvider.bind(binder, WorkerBehaviorConfig.CONFIG_KEY, WorkerBehaviorConfig.class, null); + JacksonConfigProvider.bind( + binder, + DruidCompactionConfig.CONFIG_KEY, + DruidCompactionConfig.class, + DruidCompactionConfig.empty() + ); } @Provides @@ -415,31 +418,6 @@ public Supplier> getHeartbeatSupplier(DruidOverlord overlord return heartbeatTags; }; } - - @Provides - @LazySingleton - public Supplier getClusterCompactionConfig(JacksonConfigManager configManager) - { - return () -> { - DruidCompactionConfig compactionConfig = configManager.watch( - DruidCompactionConfig.CONFIG_KEY, - DruidCompactionConfig.class, - DruidCompactionConfig.empty() - ).get(); - return Configs.valueOrDefault(compactionConfig, DruidCompactionConfig.empty()) - .clusterConfig(); - }; - } - - @Provides - @LazySingleton - public Supplier getWorkerBehaviourConfig(JacksonConfigManager configManager) - { - return () -> configManager.watch( - WorkerBehaviorConfig.CONFIG_KEY, - WorkerBehaviorConfig.class - ).get(); - } }; } From ad9838333e6feb8b5704ffb50a01a14eabc495cb Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Sat, 10 Aug 2024 14:33:18 +0530 Subject: [PATCH 12/26] Fix flow --- .../compact/OverlordCompactionScheduler.java | 61 +-------------- .../OverlordCompactionSchedulerTest.java | 12 +-- .../AsyncManagementForwardingServlet.java | 10 +-- .../compaction/CompactionRunSimulator.java | 10 ++- .../compaction/CompactionStatusTracker.java | 11 +++ ....java => CompactionSupervisorsConfig.java} | 12 +-- .../server/coordinator/DruidCoordinator.java | 12 +-- .../coordinator/duty/CompactSegments.java | 21 ++++- .../duty/CoordinatorDutyUtils.java | 11 ++- .../coordinator/duty/KillUnusedSegments.java | 2 +- .../druid/server/http/CompactionResource.java | 26 +++---- .../AsyncManagementForwardingServletTest.java | 4 +- .../coordinator/DruidCoordinatorTest.java | 10 +-- .../coordinator/duty/CompactSegmentsTest.java | 76 +++++++++++++++---- .../CoordinatorSimulationBuilder.java | 4 +- .../server/http/CompactionResourceTest.java | 14 ++-- .../org/apache/druid/cli/CliCoordinator.java | 4 +- .../org/apache/druid/cli/CliOverlord.java | 4 +- .../java/org/apache/druid/cli/CliRouter.java | 4 +- 19 files changed, 168 insertions(+), 140 deletions(-) rename server/src/main/java/org/apache/druid/server/coordinator/{CompactionSchedulerConfig.java => CompactionSupervisorsConfig.java} (84%) 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 39d52bac9d5a..d8bd7c96576d 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 @@ -20,15 +20,10 @@ package org.apache.druid.indexing.compact; import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.base.Optional; import com.google.common.base.Supplier; import com.google.inject.Inject; -import org.apache.druid.indexer.TaskLocation; -import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexing.overlord.TaskMaster; import org.apache.druid.indexing.overlord.TaskQueryTool; -import org.apache.druid.indexing.overlord.TaskRunner; -import org.apache.druid.indexing.overlord.TaskRunnerListener; import org.apache.druid.java.util.common.Stopwatch; import org.apache.druid.java.util.common.concurrent.ScheduledExecutorFactory; import org.apache.druid.java.util.common.logger.Logger; @@ -41,7 +36,7 @@ import org.apache.druid.server.compaction.CompactionStatusTracker; import org.apache.druid.server.coordinator.AutoCompactionSnapshot; import org.apache.druid.server.coordinator.ClusterCompactionConfig; -import org.apache.druid.server.coordinator.CompactionSchedulerConfig; +import org.apache.druid.server.coordinator.CompactionSupervisorsConfig; import org.apache.druid.server.coordinator.CoordinatorOverlordServiceConfig; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; import org.apache.druid.server.coordinator.DruidCompactionConfig; @@ -61,7 +56,7 @@ import java.util.concurrent.atomic.AtomicBoolean; /** - * Compaction scheduler that runs on the Overlord if {@link CompactionSchedulerConfig} + * Compaction scheduler that runs on the Overlord if {@link CompactionSupervisorsConfig} * is enabled. *

* Usage: @@ -81,12 +76,11 @@ public class OverlordCompactionScheduler implements CompactionScheduler private static final long SCHEDULE_PERIOD_SECONDS = 5; private static final Duration METRIC_EMISSION_PERIOD = Duration.standardMinutes(5); - private final TaskMaster taskMaster; private final SegmentsMetadataManager segmentManager; private final OverlordClient overlordClient; private final ServiceEmitter emitter; - private final CompactionSchedulerConfig schedulerConfig; + private final CompactionSupervisorsConfig schedulerConfig; private final Supplier compactionConfigSupplier; private final ConcurrentHashMap activeDatasourceConfigs; @@ -95,7 +89,6 @@ public class OverlordCompactionScheduler implements CompactionScheduler */ private final ScheduledExecutorService executor; - private final TaskRunnerListener taskStateListener; private final CompactionStatusTracker statusTracker; private final AtomicBoolean started = new AtomicBoolean(false); private final CompactSegments duty; @@ -116,14 +109,13 @@ public OverlordCompactionScheduler( SegmentsMetadataManager segmentManager, Supplier compactionConfigSupplier, CompactionStatusTracker statusTracker, - CompactionSchedulerConfig schedulerConfig, + CompactionSupervisorsConfig schedulerConfig, CoordinatorOverlordServiceConfig coordinatorOverlordServiceConfig, ScheduledExecutorFactory executorFactory, ServiceEmitter emitter, ObjectMapper objectMapper ) { - this.taskMaster = taskMaster; this.segmentManager = segmentManager; this.emitter = emitter; this.schedulerConfig = schedulerConfig; @@ -136,27 +128,6 @@ public OverlordCompactionScheduler( this.overlordClient = new LocalOverlordClient(taskMaster, taskQueryTool, objectMapper); this.duty = new CompactSegments(this.statusTracker, overlordClient); this.activeDatasourceConfigs = new ConcurrentHashMap<>(); - - this.taskStateListener = new TaskRunnerListener() - { - @Override - public String getListenerId() - { - return "CompactionSupervisorManager"; - } - - @Override - public void locationChanged(String taskId, TaskLocation newLocation) - { - // Do nothing - } - - @Override - public void statusChanged(String taskId, TaskStatus status) - { - runOnExecutor(() -> OverlordCompactionScheduler.this.statusTracker.onTaskFinished(taskId, status)); - } - }; } @Override @@ -199,13 +170,6 @@ public void stopCompaction(String dataSourceName) private synchronized void initState() { - Optional taskRunner = taskMaster.getTaskRunner(); - if (taskRunner.isPresent()) { - taskRunner.get().registerListener(taskStateListener, executor); - } else { - log.warn("No TaskRunner. Unable to register callbacks."); - } - if (shouldPollSegments) { segmentManager.startPollingDatabasePeriodically(); } @@ -216,11 +180,6 @@ private synchronized void cleanupState() statusTracker.stop(); activeDatasourceConfigs.clear(); - Optional taskRunner = taskMaster.getTaskRunner(); - if (taskRunner.isPresent()) { - taskRunner.get().unregisterListener(taskStateListener.getListenerId()); - } - if (shouldPollSegments) { segmentManager.stopPollingDatabasePeriodically(); } @@ -333,16 +292,4 @@ private void scheduleOnExecutor(Runnable runnable, long delaySeconds) TimeUnit.SECONDS ); } - - private void runOnExecutor(Runnable runnable) - { - executor.submit(() -> { - try { - runnable.run(); - } - catch (Throwable t) { - log.error(t, "Error while executing runnable"); - } - }); - } } 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 e74c430ae8d9..af77b0e1c03d 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 @@ -41,7 +41,7 @@ import org.apache.druid.server.compaction.CompactionStatistics; import org.apache.druid.server.compaction.CompactionStatusTracker; import org.apache.druid.server.coordinator.AutoCompactionSnapshot; -import org.apache.druid.server.coordinator.CompactionSchedulerConfig; +import org.apache.druid.server.coordinator.CompactionSupervisorsConfig; import org.apache.druid.server.coordinator.CoordinatorOverlordServiceConfig; import org.apache.druid.server.coordinator.CreateDataSegments; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; @@ -80,7 +80,7 @@ public class OverlordCompactionSchedulerTest ); } - private CompactionSchedulerConfig schedulerConfig; + private CompactionSupervisorsConfig schedulerConfig; private DruidCompactionConfig compactionConfig; private CoordinatorOverlordServiceConfig coordinatorOverlordServiceConfig; @@ -110,7 +110,7 @@ public void setUp() serviceEmitter = new StubServiceEmitter(); segmentsMetadataManager = new TestSegmentsMetadataManager(); - schedulerConfig = new CompactionSchedulerConfig(true); + schedulerConfig = new CompactionSupervisorsConfig(true); compactionConfig = DruidCompactionConfig.empty(); coordinatorOverlordServiceConfig = new CoordinatorOverlordServiceConfig(false, null); @@ -145,7 +145,7 @@ public void tearDown() @Test public void testStartStopWhenSchedulerIsEnabled() { - schedulerConfig = new CompactionSchedulerConfig(true); + schedulerConfig = new CompactionSupervisorsConfig(true); Assert.assertFalse(scheduler.isRunning()); scheduler.start(); @@ -164,7 +164,7 @@ public void testStartStopWhenSchedulerIsEnabled() @Test public void testStartStopWhenScheduledIsDisabled() { - schedulerConfig = new CompactionSchedulerConfig(false); + schedulerConfig = new CompactionSupervisorsConfig(false); initScheduler(); Assert.assertFalse(scheduler.isRunning()); @@ -179,7 +179,7 @@ public void testStartStopWhenScheduledIsDisabled() @Test public void testSegmentsAreNotPolledWhenSchedulerIsDisabled() { - schedulerConfig = new CompactionSchedulerConfig(false); + schedulerConfig = new CompactionSupervisorsConfig(false); initScheduler(); verifySegmentPolling(false); diff --git a/server/src/main/java/org/apache/druid/server/AsyncManagementForwardingServlet.java b/server/src/main/java/org/apache/druid/server/AsyncManagementForwardingServlet.java index d8655fcb6575..458c8b69c1f9 100644 --- a/server/src/main/java/org/apache/druid/server/AsyncManagementForwardingServlet.java +++ b/server/src/main/java/org/apache/druid/server/AsyncManagementForwardingServlet.java @@ -30,7 +30,7 @@ import org.apache.druid.guice.annotations.Json; import org.apache.druid.guice.http.DruidHttpClientConfig; import org.apache.druid.java.util.common.StringUtils; -import org.apache.druid.server.coordinator.CompactionSchedulerConfig; +import org.apache.druid.server.coordinator.CompactionSupervisorsConfig; import org.apache.druid.server.initialization.jetty.StandardResponseHeaderFilterHolder; import org.apache.druid.server.security.AuthConfig; import org.apache.druid.server.security.AuthorizationUtils; @@ -82,7 +82,7 @@ public class AsyncManagementForwardingServlet extends AsyncProxyServlet private final DruidLeaderSelector coordLeaderSelector; private final DruidLeaderSelector overlordLeaderSelector; private final AuthorizerMapper authorizerMapper; - private final CompactionSchedulerConfig compactionSchedulerConfig; + private final CompactionSupervisorsConfig compactionSupervisorsConfig; @Inject public AsyncManagementForwardingServlet( @@ -91,7 +91,7 @@ public AsyncManagementForwardingServlet( @Global DruidHttpClientConfig httpClientConfig, @Coordinator DruidLeaderSelector coordLeaderSelector, @IndexingService DruidLeaderSelector overlordLeaderSelector, - CompactionSchedulerConfig compactionSchedulerConfig, + CompactionSupervisorsConfig compactionSupervisorsConfig, AuthorizerMapper authorizerMapper ) { @@ -100,7 +100,7 @@ public AsyncManagementForwardingServlet( this.httpClientConfig = httpClientConfig; this.coordLeaderSelector = coordLeaderSelector; this.overlordLeaderSelector = overlordLeaderSelector; - this.compactionSchedulerConfig = compactionSchedulerConfig; + this.compactionSupervisorsConfig = compactionSupervisorsConfig; this.authorizerMapper = authorizerMapper; } @@ -109,7 +109,7 @@ protected void service(HttpServletRequest request, HttpServletResponse response) { String currentLeader; String requestURI = StringUtils.toLowerCase(request.getRequestURI()); - if (compactionSchedulerConfig.isEnabled() + if (compactionSupervisorsConfig.isEnabled() && requestURI.startsWith(COMPACTION_COORDINATOR_PATH)) { // If Compaction Scheduler is enabled, compaction APIs must be forwarded to the Overlord currentLeader = overlordLeaderSelector.getCurrentLeader(); 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 7c7fa9c2b74b..3d7eab25ab9b 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 @@ -19,6 +19,7 @@ package org.apache.druid.server.compaction; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; import org.apache.druid.client.indexing.ClientCompactionTaskQuery; @@ -87,7 +88,8 @@ public CompactionSimulateResult simulateRunWithConfig( final Table skippedIntervals = Table.withColumnNames("dataSource", "interval", "numSegments", "bytes", "reasonToSkip"); - // Add a wrapper over the status tracker to add intervals to respective tables + // Add a read-only wrapper over the actual status tracker so that we can + // account for the active tasks final CompactionStatusTracker simulationStatusTracker = new CompactionStatusTracker(null) { @Override @@ -96,6 +98,12 @@ public CompactionTaskStatus getLatestTaskStatus(SegmentsToCompact candidates) return statusTracker.getLatestTaskStatus(candidates); } + @Override + public ObjectMapper getObjectMapper() + { + return statusTracker.getObjectMapper(); + } + @Override public void onCompactionStatusComputed( SegmentsToCompact candidateSegments, 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 930db6f9ae15..9948b801c6e8 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 @@ -75,12 +75,23 @@ public CompactionTaskStatus getLatestTaskStatus(SegmentsToCompact candidates) .get(candidates.getUmbrellaInterval()); } + public Set getSubmittedTaskIds() + { + return submittedTaskIdToSegments.keySet(); + } + public void onCompactionStatusComputed( SegmentsToCompact candidateSegments, DataSourceCompactionConfig config ) { // Nothing to do, used by simulator + System.out.printf( + "Computed status of datasource[%s], interval[%s] = [%s, %s]%n", + candidateSegments.getDataSource(), candidateSegments.getUmbrellaInterval(), + candidateSegments.getCompactionStatus().getState(), + candidateSegments.getCompactionStatus().getReason() + ); } public void onCompactionConfigUpdated(DruidCompactionConfig compactionConfig) diff --git a/server/src/main/java/org/apache/druid/server/coordinator/CompactionSchedulerConfig.java b/server/src/main/java/org/apache/druid/server/coordinator/CompactionSupervisorsConfig.java similarity index 84% rename from server/src/main/java/org/apache/druid/server/coordinator/CompactionSchedulerConfig.java rename to server/src/main/java/org/apache/druid/server/coordinator/CompactionSupervisorsConfig.java index 345e0af6230b..ab13eda3b1dd 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/CompactionSchedulerConfig.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/CompactionSupervisorsConfig.java @@ -29,25 +29,25 @@ /** * This config must be bound on the following services: *

    - *
  • CliOverlord - to run the compaction scheduler on the Overlord
  • + *
  • CliOverlord - to run the compaction supervisors on the Overlord
  • *
  • CliOverlord - to prevent the Coordinator from running auto-compaction duty
  • *
  • CliRouter - to allow the Router to forward compaction stats requests to the Overlord
  • *
*/ -public class CompactionSchedulerConfig +public class CompactionSupervisorsConfig { - private static final CompactionSchedulerConfig DEFAULT = new CompactionSchedulerConfig(null); + private static final CompactionSupervisorsConfig DEFAULT = new CompactionSupervisorsConfig(null); @JsonProperty private final boolean enabled; - public static CompactionSchedulerConfig defaultConfig() + public static CompactionSupervisorsConfig defaultConfig() { return DEFAULT; } @JsonCreator - public CompactionSchedulerConfig( + public CompactionSupervisorsConfig( @JsonProperty("enabled") @Nullable Boolean enabled ) { @@ -68,7 +68,7 @@ public boolean equals(Object o) if (o == null || getClass() != o.getClass()) { return false; } - CompactionSchedulerConfig that = (CompactionSchedulerConfig) o; + CompactionSupervisorsConfig that = (CompactionSupervisorsConfig) o; return enabled == that.enabled; } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java b/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java index 77d569e608ef..99321f152b14 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java @@ -160,7 +160,7 @@ public class DruidCoordinator @Nullable private final CoordinatorSegmentMetadataCache coordinatorSegmentMetadataCache; private final CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig; - private final CompactionSchedulerConfig compactionSchedulerConfig; + private final CompactionSupervisorsConfig compactionSupervisorsConfig; private volatile boolean started = false; @@ -207,7 +207,7 @@ public DruidCoordinator( @Nullable CoordinatorSegmentMetadataCache coordinatorSegmentMetadataCache, CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig, CompactionStatusTracker compactionStatusTracker, - CompactionSchedulerConfig compactionSchedulerConfig + CompactionSupervisorsConfig compactionSupervisorsConfig ) { this.config = config; @@ -230,7 +230,7 @@ public DruidCoordinator( this.loadQueueManager = loadQueueManager; this.coordinatorSegmentMetadataCache = coordinatorSegmentMetadataCache; this.centralizedDatasourceSchemaConfig = centralizedDatasourceSchemaConfig; - this.compactionSchedulerConfig = compactionSchedulerConfig; + this.compactionSupervisorsConfig = compactionSupervisorsConfig; } public boolean isLeader() @@ -355,12 +355,12 @@ public Long getTotalSizeOfSegmentsAwaitingCompaction(String dataSource) } @Nullable - public AutoCompactionSnapshot getCompactionSnapshot(String dataSource) + public AutoCompactionSnapshot getAutoCompactionSnapshotForDataSource(String dataSource) { return compactSegments.getAutoCompactionSnapshot(dataSource); } - public Map getAllCompactionSnapshots() + public Map getAutoCompactionSnapshot() { return compactSegments.getAutoCompactionSnapshot(); } @@ -809,7 +809,7 @@ public void run() */ private boolean shouldSkipAutoCompactDuty(CoordinatorDuty duty) { - final boolean shouldSkipDuty = compactionSchedulerConfig.isEnabled() + final boolean shouldSkipDuty = compactionSupervisorsConfig.isEnabled() && duty instanceof CompactSegments && !COMPACT_SEGMENTS_DUTIES_DUTY_GROUP.equals(dutyGroupName); if (shouldSkipDuty) { 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 c4f723c3ece9..5986c1d57375 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 @@ -37,6 +37,7 @@ import org.apache.druid.common.guava.FutureUtils; import org.apache.druid.common.utils.IdUtils; import org.apache.druid.indexer.CompactionEngine; +import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexer.TaskStatusPlus; import org.apache.druid.indexer.partitions.DimensionRangePartitionsSpec; import org.apache.druid.java.util.common.ISE; @@ -66,8 +67,10 @@ import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; +import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Set; import java.util.concurrent.atomic.AtomicReference; import java.util.function.Function; import java.util.stream.Collectors; @@ -116,7 +119,6 @@ public OverlordClient getOverlordClient() @Override public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params) { - // statusTracker.reset(); run( params.getCompactionConfig(), params.getUsedSegmentsTimelinesPerDataSource(), @@ -157,10 +159,13 @@ public void run( // Fetch currently running compaction tasks int busyCompactionTaskSlots = 0; - final List compactionTasks = CoordinatorDutyUtils.getNumActiveTaskSlots( + final List compactionTasks = CoordinatorDutyUtils.getStatusOfActiveTasks( overlordClient, IS_COMPACTION_TASK ); + + final Set activeTaskIds + = compactionTasks.stream().map(TaskStatusPlus::getId).collect(Collectors.toSet()); for (TaskStatusPlus status : compactionTasks) { final TaskPayloadResponse response = FutureUtils.getUnchecked(overlordClient.taskPayload(status.getId()), true); @@ -206,6 +211,18 @@ public void run( .addAll(intervals) ); + // Get status of all tasks that were submitted recently but are not active anymore + final Set finishedTaskIds = new HashSet<>(statusTracker.getSubmittedTaskIds()); + finishedTaskIds.removeAll(activeTaskIds); + + final Map taskStatusMap + = FutureUtils.getUnchecked(overlordClient.taskStatuses(finishedTaskIds), true); + for (String taskId : finishedTaskIds) { + // Assume unknown task to have finished successfully + final TaskStatus taskStatus = taskStatusMap.getOrDefault(taskId, TaskStatus.success(taskId)); + statusTracker.onTaskFinished(taskId, taskStatus); + } + // Get iterator over segments to compact and submit compaction tasks final CompactionSegmentSearchPolicy policy = dynamicConfig.getCompactionPolicy(); final CompactionSegmentIterator iterator = diff --git a/server/src/main/java/org/apache/druid/server/coordinator/duty/CoordinatorDutyUtils.java b/server/src/main/java/org/apache/druid/server/coordinator/duty/CoordinatorDutyUtils.java index f6f31173fa5b..1c31a28087bb 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/duty/CoordinatorDutyUtils.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/duty/CoordinatorDutyUtils.java @@ -86,17 +86,16 @@ public static int getTotalWorkerCapacity(@Nonnull final OverlordClient overlordC } /** - * Return the number of active tasks that match the task predicate provided. The number of active tasks returned - * may be an overestimate, as tasks that return status's with null types will be conservatively counted to match the - * predicate provided. + * Fetches active task statuses that match the given predicate. + * Task statuses with null types are considered to satisfy the predicate too. * * @param overlordClient The overlord client to use to retrieve the list of active tasks. - * @param taskPredicate The predicate to match against the list of retreived task status. + * @param taskPredicate The predicate to match against the list of retrieved task statuses. * This predicate will never be called with a null task status. * - * @return the number of active tasks that match the task predicate provided + * @return Active task statuses that match the given predicate. */ - public static List getNumActiveTaskSlots( + public static List getStatusOfActiveTasks( @Nonnull final OverlordClient overlordClient, final Predicate taskPredicate ) diff --git a/server/src/main/java/org/apache/druid/server/coordinator/duty/KillUnusedSegments.java b/server/src/main/java/org/apache/druid/server/coordinator/duty/KillUnusedSegments.java index 64b61df5e539..199c3555b377 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/duty/KillUnusedSegments.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/duty/KillUnusedSegments.java @@ -306,7 +306,7 @@ private int getAvailableKillTaskSlots(final CoordinatorDynamicConfig config, fin final int availableKillTaskSlots = Math.max( 0, - killTaskCapacity - CoordinatorDutyUtils.getNumActiveTaskSlots(overlordClient, IS_AUTO_KILL_TASK).size() + killTaskCapacity - CoordinatorDutyUtils.getStatusOfActiveTasks(overlordClient, IS_AUTO_KILL_TASK).size() ); stats.add(Stats.Kill.AVAILABLE_SLOTS, availableKillTaskSlots); diff --git a/server/src/main/java/org/apache/druid/server/http/CompactionResource.java b/server/src/main/java/org/apache/druid/server/http/CompactionResource.java index 5d8151e92344..7bbd82fa84a6 100644 --- a/server/src/main/java/org/apache/druid/server/http/CompactionResource.java +++ b/server/src/main/java/org/apache/druid/server/http/CompactionResource.java @@ -26,7 +26,7 @@ import com.sun.jersey.spi.container.ResourceFilters; import org.apache.druid.server.coordinator.AutoCompactionSnapshot; import org.apache.druid.server.coordinator.ClusterCompactionConfig; -import org.apache.druid.server.coordinator.CompactionSchedulerConfig; +import org.apache.druid.server.coordinator.CompactionSupervisorsConfig; import org.apache.druid.server.coordinator.DruidCoordinator; import org.apache.druid.server.http.security.ConfigResourceFilter; import org.apache.druid.server.http.security.StateResourceFilter; @@ -45,16 +45,16 @@ public class CompactionResource { private final DruidCoordinator coordinator; - private final CompactionSchedulerConfig schedulerConfig; + private final CompactionSupervisorsConfig compactionSupervisorsConfig; @Inject public CompactionResource( DruidCoordinator coordinator, - CompactionSchedulerConfig schedulerConfig + CompactionSupervisorsConfig compactionSupervisorsConfig ) { this.coordinator = coordinator; - this.schedulerConfig = schedulerConfig; + this.compactionSupervisorsConfig = compactionSupervisorsConfig; } /** @@ -78,8 +78,8 @@ public Response getCompactionProgress( @QueryParam("dataSource") String dataSource ) { - if (schedulerConfig.isEnabled()) { - buildErrorResponseWhenRunningScheduler(); + if (compactionSupervisorsConfig.isEnabled()) { + buildErrorResponseWhenRunningAsSupervisor(); } final Long notCompactedSegmentSizeBytes = coordinator.getTotalSizeOfSegmentsAwaitingCompaction(dataSource); @@ -98,15 +98,15 @@ public Response getCompactionSnapshotForDataSource( @QueryParam("dataSource") String dataSource ) { - if (schedulerConfig.isEnabled()) { - return buildErrorResponseWhenRunningScheduler(); + if (compactionSupervisorsConfig.isEnabled()) { + return buildErrorResponseWhenRunningAsSupervisor(); } final Collection snapshots; if (dataSource == null || dataSource.isEmpty()) { - snapshots = coordinator.getAllCompactionSnapshots().values(); + snapshots = coordinator.getAutoCompactionSnapshot().values(); } else { - AutoCompactionSnapshot autoCompactionSnapshot = coordinator.getCompactionSnapshot(dataSource); + AutoCompactionSnapshot autoCompactionSnapshot = coordinator.getAutoCompactionSnapshotForDataSource(dataSource); if (autoCompactionSnapshot == null) { return Response.status(Response.Status.NOT_FOUND).entity(ImmutableMap.of("error", "unknown dataSource")).build(); } @@ -122,8 +122,8 @@ public Response simulateClusterCompactionConfigUpdate( ClusterCompactionConfig updatePayload ) { - if (schedulerConfig.isEnabled()) { - return buildErrorResponseWhenRunningScheduler(); + if (compactionSupervisorsConfig.isEnabled()) { + return buildErrorResponseWhenRunningAsSupervisor(); } return Response.ok().entity( @@ -131,7 +131,7 @@ public Response simulateClusterCompactionConfigUpdate( ).build(); } - private Response buildErrorResponseWhenRunningScheduler() + private Response buildErrorResponseWhenRunningAsSupervisor() { return Response.status(Response.Status.SERVICE_UNAVAILABLE).entity( ImmutableMap.of( diff --git a/server/src/test/java/org/apache/druid/server/AsyncManagementForwardingServletTest.java b/server/src/test/java/org/apache/druid/server/AsyncManagementForwardingServletTest.java index 5ca6fcc9dccb..62053cd555ff 100644 --- a/server/src/test/java/org/apache/druid/server/AsyncManagementForwardingServletTest.java +++ b/server/src/test/java/org/apache/druid/server/AsyncManagementForwardingServletTest.java @@ -35,7 +35,7 @@ import org.apache.druid.guice.http.DruidHttpClientConfig; import org.apache.druid.initialization.Initialization; import org.apache.druid.java.util.common.StringUtils; -import org.apache.druid.server.coordinator.CompactionSchedulerConfig; +import org.apache.druid.server.coordinator.CompactionSupervisorsConfig; import org.apache.druid.server.initialization.BaseJettyTest; import org.apache.druid.server.initialization.ServerConfig; import org.apache.druid.server.initialization.jetty.JettyServerInitUtils; @@ -540,7 +540,7 @@ public String getCurrentLeader() injector.getInstance(DruidHttpClientConfig.class), coordinatorLeaderSelector, overlordLeaderSelector, - new CompactionSchedulerConfig(false) { + new CompactionSupervisorsConfig(false) { @Override public boolean isEnabled() { diff --git a/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorTest.java b/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorTest.java index b5c8f1487cc5..6f6e3b4ad74c 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorTest.java @@ -193,7 +193,7 @@ public void setUp() throws Exception null, CentralizedDatasourceSchemaConfig.create(), new CompactionStatusTracker(OBJECT_MAPPER), - new CompactionSchedulerConfig(true) + new CompactionSupervisorsConfig(true) ); } @@ -625,7 +625,7 @@ public void testCompactSegmentsDutyWhenCustomDutyGroupEmpty() null, CentralizedDatasourceSchemaConfig.create(), new CompactionStatusTracker(OBJECT_MAPPER), - CompactionSchedulerConfig.defaultConfig() + CompactionSupervisorsConfig.defaultConfig() ); // Since CompactSegments is not enabled in Custom Duty Group, then CompactSegments must be created in IndexingServiceDuties List indexingDuties = coordinator.makeIndexingServiceDuties(); @@ -666,7 +666,7 @@ public void testInitializeCompactSegmentsDutyWhenCustomDutyGroupDoesNotContainsC null, CentralizedDatasourceSchemaConfig.create(), new CompactionStatusTracker(OBJECT_MAPPER), - CompactionSchedulerConfig.defaultConfig() + CompactionSupervisorsConfig.defaultConfig() ); // Since CompactSegments is not enabled in Custom Duty Group, then CompactSegments must be created in IndexingServiceDuties List indexingDuties = coordinator.makeIndexingServiceDuties(); @@ -707,7 +707,7 @@ public void testInitializeCompactSegmentsDutyWhenCustomDutyGroupContainsCompactS null, CentralizedDatasourceSchemaConfig.create(), new CompactionStatusTracker(OBJECT_MAPPER), - CompactionSchedulerConfig.defaultConfig() + CompactionSupervisorsConfig.defaultConfig() ); // Since CompactSegments is enabled in Custom Duty Group, then CompactSegments must not be created in IndexingServiceDuties List indexingDuties = coordinator.makeIndexingServiceDuties(); @@ -813,7 +813,7 @@ public void testCoordinatorCustomDutyGroupsRunAsExpected() throws Exception null, CentralizedDatasourceSchemaConfig.create(), new CompactionStatusTracker(OBJECT_MAPPER), - CompactionSchedulerConfig.defaultConfig() + CompactionSupervisorsConfig.defaultConfig() ); coordinator.start(); diff --git a/server/src/test/java/org/apache/druid/server/coordinator/duty/CompactSegmentsTest.java b/server/src/test/java/org/apache/druid/server/coordinator/duty/CompactSegmentsTest.java index 16aead76c9b2..7f1df62f27fb 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 @@ -47,6 +47,7 @@ import org.apache.druid.indexer.RunnerTaskState; import org.apache.druid.indexer.TaskLocation; import org.apache.druid.indexer.TaskState; +import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexer.TaskStatusPlus; import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; import org.apache.druid.indexer.partitions.HashedPartitionsSpec; @@ -57,6 +58,7 @@ import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.IAE; 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.Granularities; import org.apache.druid.java.util.common.parsers.CloseableIterator; @@ -85,14 +87,12 @@ import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.SegmentTimeline; import org.apache.druid.timeline.TimelineObjectHolder; -import org.apache.druid.timeline.VersionedIntervalTimeline; import org.apache.druid.timeline.partition.HashBasedNumberedShardSpec; import org.apache.druid.timeline.partition.NumberedShardSpec; import org.apache.druid.timeline.partition.PartitionChunk; import org.apache.druid.timeline.partition.ShardSpec; import org.apache.druid.timeline.partition.SingleDimensionShardSpec; import org.apache.druid.utils.Streams; -import org.joda.time.DateTime; import org.joda.time.Interval; import org.joda.time.Period; import org.junit.Assert; @@ -112,6 +112,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Set; import java.util.concurrent.atomic.AtomicInteger; import java.util.function.BiFunction; import java.util.function.Supplier; @@ -1205,6 +1206,52 @@ public void testCompactWithGranularitySpecConflictWithActiveCompactionTask() Assert.assertEquals(expectedGranularitySpec, taskPayload.getGranularitySpec()); } + @Test + public void testIntervalIsCompactedAgainWhenSegmentIsAdded() + { + final TestOverlordClient overlordClient = new TestOverlordClient(JSON_MAPPER); + final CompactSegments compactSegments = new CompactSegments(statusTracker, overlordClient); + + final String dataSource = DATA_SOURCE_PREFIX + 0; + final DataSourceCompactionConfig compactionConfig = DataSourceCompactionConfig + .builder() + .forDataSource(dataSource) + .withSkipOffsetFromLatest(Period.seconds(0)) + .withGranularitySpec(new UserCompactionTaskGranularityConfig(Granularities.DAY, null, null)) + .build(); + + CoordinatorRunStats stats = doCompactSegments( + compactSegments, + ImmutableList.of(compactionConfig) + ); + Assert.assertEquals(1, stats.get(Stats.Compaction.SUBMITTED_TASKS)); + Assert.assertEquals(1, overlordClient.submittedCompactionTasks.size()); + + ClientCompactionTaskQuery submittedTask = overlordClient.submittedCompactionTasks.get(0); + Assert.assertEquals(submittedTask.getDataSource(), dataSource); + Assert.assertEquals( + Intervals.of("2017-01-09/P1D"), + submittedTask.getIoConfig().getInputSpec().getInterval() + ); + + // Add more data to the latest interval + addMoreData(dataSource, 8); + stats = doCompactSegments( + compactSegments, + ImmutableList.of(compactionConfig) + ); + Assert.assertEquals(1, stats.get(Stats.Compaction.SUBMITTED_TASKS)); + Assert.assertEquals(2, overlordClient.submittedCompactionTasks.size()); + + // Verify that the latest interval is compacted again + submittedTask = overlordClient.submittedCompactionTasks.get(1); + Assert.assertEquals(submittedTask.getDataSource(), dataSource); + Assert.assertEquals( + Intervals.of("2017-01-09/P1D"), + submittedTask.getIoConfig().getInputSpec().getInterval() + ); + } + @Test public void testRunParallelCompactionMultipleCompactionTaskSlots() { @@ -1806,12 +1853,12 @@ private CoordinatorRunStats doCompactSegments(CompactSegments compactSegments, @ return doCompactSegments(compactSegments, createCompactionConfigs(), numCompactionTaskSlots); } - private void doCompactSegments( + private CoordinatorRunStats doCompactSegments( CompactSegments compactSegments, List compactionConfigs ) { - doCompactSegments(compactSegments, compactionConfigs, null); + return doCompactSegments(compactSegments, compactionConfigs, null); } private CoordinatorRunStats doCompactSegments( @@ -2087,6 +2134,12 @@ public ListenableFuture> taskStatuses( return Futures.immediateFuture(CloseableIterators.withEmptyBaggage(Collections.emptyIterator())); } + @Override + public ListenableFuture> taskStatuses(Set taskIds) + { + return Futures.immediateFuture(Collections.emptyMap()); + } + @Override public ListenableFuture getTotalWorkerCapacity() { @@ -2094,22 +2147,15 @@ public ListenableFuture getTotalWorkerCapacity( } private void compactSegments( - VersionedIntervalTimeline timeline, + SegmentTimeline timeline, List segments, ClientCompactionTaskQuery clientCompactionTaskQuery ) { Preconditions.checkArgument(segments.size() > 1); - DateTime minStart = DateTimes.MAX, maxEnd = DateTimes.MIN; - for (DataSegment segment : segments) { - if (segment.getInterval().getStart().compareTo(minStart) < 0) { - minStart = segment.getInterval().getStart(); - } - if (segment.getInterval().getEnd().compareTo(maxEnd) > 0) { - maxEnd = segment.getInterval().getEnd(); - } - } - Interval compactInterval = new Interval(minStart, maxEnd); + final Interval compactInterval = JodaUtils.umbrellaInterval( + segments.stream().map(DataSegment::getInterval).collect(Collectors.toList()) + ); segments.forEach( segment -> timeline.remove( segment.getInterval(), diff --git a/server/src/test/java/org/apache/druid/server/coordinator/simulate/CoordinatorSimulationBuilder.java b/server/src/test/java/org/apache/druid/server/coordinator/simulate/CoordinatorSimulationBuilder.java index 9e13bebe9df5..3cafd645def8 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/simulate/CoordinatorSimulationBuilder.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/simulate/CoordinatorSimulationBuilder.java @@ -38,7 +38,7 @@ import org.apache.druid.java.util.metrics.StubServiceEmitter; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; import org.apache.druid.server.compaction.CompactionStatusTracker; -import org.apache.druid.server.coordinator.CompactionSchedulerConfig; +import org.apache.druid.server.coordinator.CompactionSupervisorsConfig; import org.apache.druid.server.coordinator.CoordinatorConfigManager; import org.apache.druid.server.coordinator.CoordinatorDynamicConfig; import org.apache.druid.server.coordinator.DruidCompactionConfig; @@ -213,7 +213,7 @@ public CoordinatorSimulation build() null, CentralizedDatasourceSchemaConfig.create(), new CompactionStatusTracker(OBJECT_MAPPER), - CompactionSchedulerConfig.defaultConfig() + CompactionSupervisorsConfig.defaultConfig() ); return new SimulationImpl(coordinator, env); diff --git a/server/src/test/java/org/apache/druid/server/http/CompactionResourceTest.java b/server/src/test/java/org/apache/druid/server/http/CompactionResourceTest.java index ea5b1c64293f..1e88bea2e099 100644 --- a/server/src/test/java/org/apache/druid/server/http/CompactionResourceTest.java +++ b/server/src/test/java/org/apache/druid/server/http/CompactionResourceTest.java @@ -22,7 +22,7 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import org.apache.druid.server.coordinator.AutoCompactionSnapshot; -import org.apache.druid.server.coordinator.CompactionSchedulerConfig; +import org.apache.druid.server.coordinator.CompactionSupervisorsConfig; import org.apache.druid.server.coordinator.DruidCoordinator; import org.easymock.EasyMock; import org.junit.After; @@ -36,7 +36,7 @@ public class CompactionResourceTest { private DruidCoordinator mock; - private final CompactionSchedulerConfig schedulerConfig = new CompactionSchedulerConfig(false); + private final CompactionSupervisorsConfig schedulerConfig = new CompactionSupervisorsConfig(false); private String dataSourceName = "datasource_1"; private AutoCompactionSnapshot expectedSnapshot = new AutoCompactionSnapshot( dataSourceName, @@ -72,7 +72,7 @@ public void testGetCompactionSnapshotForDataSourceWithEmptyQueryParameter() expectedSnapshot ); - EasyMock.expect(mock.getAllCompactionSnapshots()).andReturn(expected).once(); + EasyMock.expect(mock.getAutoCompactionSnapshot()).andReturn(expected).once(); EasyMock.replay(mock); final Response response = new CompactionResource(mock, schedulerConfig) @@ -90,7 +90,7 @@ public void testGetCompactionSnapshotForDataSourceWithNullQueryParameter() expectedSnapshot ); - EasyMock.expect(mock.getAllCompactionSnapshots()).andReturn(expected).once(); + EasyMock.expect(mock.getAutoCompactionSnapshot()).andReturn(expected).once(); EasyMock.replay(mock); final Response response = new CompactionResource(mock, schedulerConfig) @@ -104,7 +104,7 @@ public void testGetCompactionSnapshotForDataSourceWithValidQueryParameter() { String dataSourceName = "datasource_1"; - EasyMock.expect(mock.getCompactionSnapshot(dataSourceName)).andReturn(expectedSnapshot).once(); + EasyMock.expect(mock.getAutoCompactionSnapshotForDataSource(dataSourceName)).andReturn(expectedSnapshot).once(); EasyMock.replay(mock); final Response response = new CompactionResource(mock, schedulerConfig) @@ -118,7 +118,7 @@ public void testGetCompactionSnapshotForDataSourceWithInvalidQueryParameter() { String dataSourceName = "invalid_datasource"; - EasyMock.expect(mock.getCompactionSnapshot(dataSourceName)).andReturn(null).once(); + EasyMock.expect(mock.getAutoCompactionSnapshotForDataSource(dataSourceName)).andReturn(null).once(); EasyMock.replay(mock); final Response response = new CompactionResource(mock, schedulerConfig) @@ -131,7 +131,7 @@ public void testGetCompactionSnapshotWhenCompactionSchedulerIsEnabled() { EasyMock.replay(mock); - final Response response = new CompactionResource(mock, new CompactionSchedulerConfig(true)) + final Response response = new CompactionResource(mock, new CompactionSupervisorsConfig(true)) .getCompactionSnapshotForDataSource("dummy"); Assert.assertEquals(503, response.getStatus()); Assert.assertEquals( diff --git a/services/src/main/java/org/apache/druid/cli/CliCoordinator.java b/services/src/main/java/org/apache/druid/cli/CliCoordinator.java index d7e03e24d362..1fa3bdb1bc67 100644 --- a/services/src/main/java/org/apache/druid/cli/CliCoordinator.java +++ b/services/src/main/java/org/apache/druid/cli/CliCoordinator.java @@ -98,7 +98,7 @@ import org.apache.druid.server.QueryScheduler; import org.apache.druid.server.QuerySchedulerProvider; import org.apache.druid.server.compaction.CompactionStatusTracker; -import org.apache.druid.server.coordinator.CompactionSchedulerConfig; +import org.apache.druid.server.coordinator.CompactionSupervisorsConfig; import org.apache.druid.server.coordinator.CoordinatorConfigManager; import org.apache.druid.server.coordinator.DruidCoordinator; import org.apache.druid.server.coordinator.MetadataManager; @@ -229,7 +229,7 @@ public void configure(Binder binder) CachingCostBalancerStrategyConfig.class ); JsonConfigProvider.bind(binder, "druid.coordinator.segmentMetadataCache", SegmentMetadataCacheConfig.class); - JsonConfigProvider.bind(binder, "druid.compaction.scheduler", CompactionSchedulerConfig.class); + JsonConfigProvider.bind(binder, "druid.compaction.supervisors", CompactionSupervisorsConfig.class); binder.bind(DruidCoordinatorConfig.class); binder.bind(RedirectFilter.class).in(LazySingleton.class); diff --git a/services/src/main/java/org/apache/druid/cli/CliOverlord.java b/services/src/main/java/org/apache/druid/cli/CliOverlord.java index f759701ba77b..d8290df69473 100644 --- a/services/src/main/java/org/apache/druid/cli/CliOverlord.java +++ b/services/src/main/java/org/apache/druid/cli/CliOverlord.java @@ -117,7 +117,7 @@ import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager; import org.apache.druid.segment.realtime.appenderator.DummyForInjectionAppenderatorsManager; import org.apache.druid.server.compaction.CompactionStatusTracker; -import org.apache.druid.server.coordinator.CompactionSchedulerConfig; +import org.apache.druid.server.coordinator.CompactionSupervisorsConfig; import org.apache.druid.server.coordinator.CoordinatorOverlordServiceConfig; import org.apache.druid.server.coordinator.DruidCompactionConfig; import org.apache.druid.server.http.RedirectFilter; @@ -218,7 +218,7 @@ public void configure(Binder binder) binder.bind(SegmentsMetadataManager.class) .toProvider(SegmentsMetadataManagerProvider.class) .in(ManageLifecycle.class); - JsonConfigProvider.bind(binder, "druid.compaction.scheduler", CompactionSchedulerConfig.class); + JsonConfigProvider.bind(binder, "druid.compaction.supervisors", CompactionSupervisorsConfig.class); } JsonConfigProvider.bind(binder, "druid.coordinator.asOverlord", CoordinatorOverlordServiceConfig.class); diff --git a/services/src/main/java/org/apache/druid/cli/CliRouter.java b/services/src/main/java/org/apache/druid/cli/CliRouter.java index 4a9266fd40b2..a18e358603df 100644 --- a/services/src/main/java/org/apache/druid/cli/CliRouter.java +++ b/services/src/main/java/org/apache/druid/cli/CliRouter.java @@ -43,7 +43,7 @@ import org.apache.druid.query.lookup.LookupSerdeModule; import org.apache.druid.server.AsyncQueryForwardingServlet; import org.apache.druid.server.NoopQuerySegmentWalker; -import org.apache.druid.server.coordinator.CompactionSchedulerConfig; +import org.apache.druid.server.coordinator.CompactionSupervisorsConfig; import org.apache.druid.server.http.RouterResource; import org.apache.druid.server.http.SelfDiscoveryResource; import org.apache.druid.server.initialization.jetty.JettyServerInitializer; @@ -102,7 +102,7 @@ protected List getModules() JsonConfigProvider.bind(binder, "druid.router", TieredBrokerConfig.class); JsonConfigProvider.bind(binder, "druid.router.avatica.balancer", AvaticaConnectionBalancer.class); JsonConfigProvider.bind(binder, "druid.router.managementProxy", ManagementProxyConfig.class); - JsonConfigProvider.bind(binder, "druid.compaction.scheduler", CompactionSchedulerConfig.class); + JsonConfigProvider.bind(binder, "druid.compaction.supervisors", CompactionSupervisorsConfig.class); binder.bind(QuerySegmentWalker.class).to(NoopQuerySegmentWalker.class).in(LazySingleton.class); From 2e676be96028cfe4d6f43bc760f46b5b40c80ab6 Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Sat, 10 Aug 2024 14:41:41 +0530 Subject: [PATCH 13/26] Remove extra sout --- .../druid/server/compaction/CompactionStatusTracker.java | 6 ------ 1 file changed, 6 deletions(-) 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 9948b801c6e8..2041a2e4cf81 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,12 +86,6 @@ public void onCompactionStatusComputed( ) { // Nothing to do, used by simulator - System.out.printf( - "Computed status of datasource[%s], interval[%s] = [%s, %s]%n", - candidateSegments.getDataSource(), candidateSegments.getUmbrellaInterval(), - candidateSegments.getCompactionStatus().getState(), - candidateSegments.getCompactionStatus().getReason() - ); } public void onCompactionConfigUpdated(DruidCompactionConfig compactionConfig) From e793f44e6718f1be9834a49552dc6f142ff46793 Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Sat, 10 Aug 2024 15:44:49 +0530 Subject: [PATCH 14/26] Fix CompactSegmentsTest --- .../druid/server/coordinator/duty/CompactSegmentsTest.java | 4 ++++ 1 file changed, 4 insertions(+) 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 7f1df62f27fb..8f84e860deae 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 @@ -1138,6 +1138,8 @@ public void testCompactWithGranularitySpecConflictWithActiveCompactionTask() .thenReturn( Futures.immediateFuture( CloseableIterators.withEmptyBaggage(ImmutableList.of(runningConflictCompactionTask).iterator()))); + Mockito.when(mockClient.taskStatuses(ArgumentMatchers.any())) + .thenReturn(Futures.immediateFuture(Collections.emptyMap())); Mockito.when(mockClient.findLockedIntervals(ArgumentMatchers.any())) .thenReturn(Futures.immediateFuture(Collections.emptyMap())); Mockito.when(mockClient.cancelTask(conflictTaskId)) @@ -2309,6 +2311,8 @@ private static ArgumentCaptor setUpMockClient(final OverlordClient mockC final ArgumentCaptor payloadCaptor = ArgumentCaptor.forClass(Object.class); Mockito.when(mockClient.taskStatuses(null, null, 0)) .thenReturn(Futures.immediateFuture(CloseableIterators.withEmptyBaggage(Collections.emptyIterator()))); + Mockito.when(mockClient.taskStatuses(ArgumentMatchers.any())) + .thenReturn(Futures.immediateFuture(Collections.emptyMap())); Mockito.when(mockClient.findLockedIntervals(ArgumentMatchers.any())) .thenReturn(Futures.immediateFuture(Collections.emptyMap())); Mockito.when(mockClient.getTotalWorkerCapacity()) From 8ece3f0747ab943fcbf26114c73a10f970c374ad Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Sat, 10 Aug 2024 18:59:17 +0530 Subject: [PATCH 15/26] Do not call taskStatus API with empty taskIds --- .../coordinator/duty/CompactSegments.java | 37 +++++++++++++------ 1 file changed, 25 insertions(+), 12 deletions(-) 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 5986c1d57375..446529e4cd9b 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 @@ -166,6 +166,8 @@ public void run( final Set activeTaskIds = compactionTasks.stream().map(TaskStatusPlus::getId).collect(Collectors.toSet()); + trackStatusOfCompletedTasks(activeTaskIds); + for (TaskStatusPlus status : compactionTasks) { final TaskPayloadResponse response = FutureUtils.getUnchecked(overlordClient.taskPayload(status.getId()), true); @@ -211,18 +213,6 @@ public void run( .addAll(intervals) ); - // Get status of all tasks that were submitted recently but are not active anymore - final Set finishedTaskIds = new HashSet<>(statusTracker.getSubmittedTaskIds()); - finishedTaskIds.removeAll(activeTaskIds); - - final Map taskStatusMap - = FutureUtils.getUnchecked(overlordClient.taskStatuses(finishedTaskIds), true); - for (String taskId : finishedTaskIds) { - // Assume unknown task to have finished successfully - final TaskStatus taskStatus = taskStatusMap.getOrDefault(taskId, TaskStatus.success(taskId)); - statusTracker.onTaskFinished(taskId, taskStatus); - } - // Get iterator over segments to compact and submit compaction tasks final CompactionSegmentSearchPolicy policy = dynamicConfig.getCompactionPolicy(); final CompactionSegmentIterator iterator = @@ -252,6 +242,29 @@ private void resetCompactionSnapshot() autoCompactionSnapshotPerDataSource.set(Collections.emptyMap()); } + /** + * Queries the Overlord for the status of all tasks that were submitted + * recently but are not active anymore. The statuses are then updated in the + * {@link #statusTracker}. + */ + private void trackStatusOfCompletedTasks(Set activeTaskIds) + { + final Set finishedTaskIds = new HashSet<>(statusTracker.getSubmittedTaskIds()); + finishedTaskIds.removeAll(activeTaskIds); + + if (finishedTaskIds.isEmpty()) { + return; + } + + final Map taskStatusMap + = FutureUtils.getUnchecked(overlordClient.taskStatuses(finishedTaskIds), true); + for (String taskId : finishedTaskIds) { + // Assume unknown task to have finished successfully + final TaskStatus taskStatus = taskStatusMap.getOrDefault(taskId, TaskStatus.success(taskId)); + statusTracker.onTaskFinished(taskId, taskStatus); + } + } + /** * Cancels a currently running compaction task if the segment granularity * for this datasource has changed in the compaction config. From 382bfdf6a0c7f008a5c0fc8c3e18dfdab4328cf5 Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Fri, 23 Aug 2024 10:34:34 +0530 Subject: [PATCH 16/26] Add javadocs, cleanup, etc. --- .../indexing/compact/CompactionScheduler.java | 17 ++++++- .../compact/CompactionStatusReport.java | 28 ----------- .../compact/CompactionSupervisor.java | 23 ++++----- .../compact/OverlordCompactionScheduler.java | 29 +++++------ .../indexing/overlord/DruidOverlord.java | 3 +- .../http/OverlordCompactionResource.java | 13 +++-- .../compact/CompactionSchedulerTest.java | 49 ------------------- .../OverlordCompactionSchedulerTest.java | 41 +++++----------- .../CompactionSupervisorsConfig.java | 1 + .../server/coordinator/DruidCoordinator.java | 6 --- .../coordinator/duty/CompactSegments.java | 10 ---- ...ava => CoordinatorCompactionResource.java} | 17 +++++-- ...=> CoordinatorCompactionResourceTest.java} | 12 ++--- .../org/apache/druid/cli/CliCoordinator.java | 10 ++-- .../org/apache/druid/cli/CliOverlord.java | 6 ++- .../java/org/apache/druid/cli/CliRouter.java | 6 ++- .../security/SecurityResourceFilterTest.java | 4 +- 17 files changed, 97 insertions(+), 178 deletions(-) delete mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionStatusReport.java delete mode 100644 indexing-service/src/test/java/org/apache/druid/indexing/compact/CompactionSchedulerTest.java rename server/src/main/java/org/apache/druid/server/http/{CompactionResource.java => CoordinatorCompactionResource.java} (89%) rename server/src/test/java/org/apache/druid/server/http/{CompactionResourceTest.java => CoordinatorCompactionResourceTest.java} (89%) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionScheduler.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionScheduler.java index 9d257867f4cf..a4cee72a40d2 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionScheduler.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionScheduler.java @@ -22,10 +22,25 @@ import org.apache.druid.server.compaction.CompactionSimulateResult; import org.apache.druid.server.coordinator.AutoCompactionSnapshot; import org.apache.druid.server.coordinator.ClusterCompactionConfig; +import org.apache.druid.server.coordinator.CompactionSupervisorsConfig; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; import java.util.Map; +/** + * Compaction scheduler that runs on the Overlord if {@link CompactionSupervisorsConfig} + * is enabled. + *

+ * Usage: + *

    + *
  • When an active {@link CompactionSupervisor} starts, it should register + * itself by calling {@link #startCompaction}.
  • + *
  • When a suspended {@link CompactionSupervisor} starts, it should stop + * compaction by calling {@link #stopCompaction}.
  • + *
  • When stopping, any {@link CompactionSupervisor} (active or suspended) + * should call {@link #stopCompaction}.
  • + *
+ */ public interface CompactionScheduler { void start(); @@ -42,8 +57,6 @@ public interface CompactionScheduler AutoCompactionSnapshot getCompactionSnapshot(String dataSource); - Long getSegmentBytesAwaitingCompaction(String dataSource); - CompactionSimulateResult simulateRunWithConfigUpdate(ClusterCompactionConfig updateRequest); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionStatusReport.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionStatusReport.java deleted file mode 100644 index 9096cfad64f5..000000000000 --- a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionStatusReport.java +++ /dev/null @@ -1,28 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.druid.indexing.compact; - -/** - * This can contain stats and progress and stuff. - * For simulation, we will still have the separate API. - */ -public class CompactionStatusReport -{ -} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionSupervisor.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionSupervisor.java index fcdad7d0174a..d4a3c46ba0b1 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionSupervisor.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionSupervisor.java @@ -35,6 +35,7 @@ public class CompactionSupervisor implements Supervisor { private static final Logger log = new Logger(CompactionSupervisor.class); + private final String dataSource; private final CompactionScheduler scheduler; private final CompactionSupervisorSpec supervisorSpec; @@ -45,12 +46,12 @@ public CompactionSupervisor( { this.supervisorSpec = supervisorSpec; this.scheduler = scheduler; + this.dataSource = supervisorSpec.getSpec().getDataSource(); } @Override public void start() { - final String dataSource = getDataSource(); if (supervisorSpec.isSuspended()) { log.info("Suspending compaction for dataSource[%s].", dataSource); scheduler.stopCompaction(dataSource); @@ -63,7 +64,6 @@ public void start() @Override public void stop(boolean stopGracefully) { - final String dataSource = getDataSource(); log.info("Stopping compaction for dataSource[%s].", dataSource); scheduler.stopCompaction(dataSource); } @@ -73,11 +73,11 @@ public SupervisorReport getStatus() { final AutoCompactionSnapshot snapshot; if (supervisorSpec.isSuspended()) { - snapshot = AutoCompactionSnapshot.builder(getDataSource()) + snapshot = AutoCompactionSnapshot.builder(dataSource) .withStatus(AutoCompactionSnapshot.AutoCompactionScheduleStatus.NOT_ENABLED) .build(); } else { - snapshot = scheduler.getCompactionSnapshot(getDataSource()); + snapshot = scheduler.getCompactionSnapshot(dataSource); } return new SupervisorReport<>(supervisorSpec.getId(), DateTimes.nowUtc(), snapshot); @@ -95,41 +95,36 @@ public SupervisorStateManager.State getState() } } - private String getDataSource() - { - return supervisorSpec.getSpec().getDataSource(); - } - // Un-implemented methods used only by streaming supervisors @Override public void reset(DataSourceMetadata dataSourceMetadata) { - // Do nothing + throw new UnsupportedOperationException("Resetting not supported for 'autocompact' supervisors."); } @Override public void resetOffsets(DataSourceMetadata resetDataSourceMetadata) { - // Do nothing + throw new UnsupportedOperationException("Resetting offsets not supported for 'autocompact' supervisors."); } @Override public void checkpoint(int taskGroupId, DataSourceMetadata checkpointMetadata) { - // Do nothing + throw new UnsupportedOperationException("Checkpointing not supported for 'autocompact' supervisors."); } @Override public LagStats computeLagStats() { - return new LagStats(0L, 0L, 0L); + throw new UnsupportedOperationException("Lag stats not supported for 'autocompact' supervisors."); } @Override public int getActiveTaskGroupsCount() { - return 0; + throw new UnsupportedOperationException("Task groups not supported for 'autocompact' supervisors."); } public enum State implements SupervisorStateManager.State 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 d8bd7c96576d..b49766eb120c 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 @@ -56,18 +56,17 @@ import java.util.concurrent.atomic.AtomicBoolean; /** - * Compaction scheduler that runs on the Overlord if {@link CompactionSupervisorsConfig} - * is enabled. + * Implementation of {@link CompactionScheduler}. *

- * Usage: + * When {@link CompactionSupervisorsConfig} is enabled, this class performs the + * following responsibilities on the leader Overlord: *

    - *
  • When an active {@link CompactionSupervisor} starts, it should register - * itself by calling {@link #startCompaction}.
  • - *
  • When a suspended {@link CompactionSupervisor} starts, it should stop - * compaction by calling {@link #stopCompaction}.
  • - *
  • When stopping, any {@link CompactionSupervisor} (active or suspended) - * should call {@link #stopCompaction}.
  • + *
  • Poll segments from metadata
  • + *
  • Check compaction status every 5 seconds
  • + *
  • Submit compaction tasks for active datasources
  • + *
  • Track status of submitted compaction tasks
  • *
+ * Internally, this class uses an instance of {@link CompactSegments} duty. */ public class OverlordCompactionScheduler implements CompactionScheduler { @@ -158,7 +157,9 @@ public boolean isRunning() @Override public void startCompaction(String dataSourceName, DataSourceCompactionConfig config) { - activeDatasourceConfigs.put(dataSourceName, config); + if (isRunning()) { + activeDatasourceConfigs.put(dataSourceName, config); + } } @Override @@ -192,7 +193,7 @@ private boolean isEnabled() private synchronized void checkSchedulingStatus() { - if (started.get() && isEnabled()) { + if (isRunning()) { try { runCompactionDuty(); } @@ -229,12 +230,6 @@ public AutoCompactionSnapshot getCompactionSnapshot(String dataSource) return duty.getAutoCompactionSnapshot(dataSource); } - @Override - public Long getSegmentBytesAwaitingCompaction(String dataSource) - { - return duty.getTotalSizeOfSegmentsAwaitingCompaction(dataSource); - } - @Override public Map getAllCompactionSnapshots() { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/DruidOverlord.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/DruidOverlord.java index fb4730e7df40..844e4922e244 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/DruidOverlord.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/DruidOverlord.java @@ -132,6 +132,7 @@ public void becomeLeader() leaderLifecycle.addManagedInstance(taskRunner); leaderLifecycle.addManagedInstance(taskQueue); + leaderLifecycle.addManagedInstance(compactionScheduler); leaderLifecycle.addManagedInstance(supervisorManager); leaderLifecycle.addManagedInstance(overlordDutyExecutor); leaderLifecycle.addHandler( @@ -142,7 +143,6 @@ public void start() { segmentAllocationQueue.becomeLeader(); taskMaster.becomeLeader(taskRunner, taskQueue); - compactionScheduler.start(); // Announce the node only after all the services have been initialized initialized = true; @@ -153,7 +153,6 @@ public void start() public void stop() { serviceAnnouncer.unannounce(node); - compactionScheduler.stop(); taskMaster.stopBeingLeader(); segmentAllocationQueue.stopBeingLeader(); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordCompactionResource.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordCompactionResource.java index 62f7d1ddce7b..fa0b928f4a6c 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordCompactionResource.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordCompactionResource.java @@ -62,13 +62,19 @@ public Response getCompactionProgress( @QueryParam("dataSource") String dataSource ) { - final Long notCompactedSegmentSizeBytes = scheduler.getSegmentBytesAwaitingCompaction(dataSource); - if (notCompactedSegmentSizeBytes == null) { + if (dataSource == null || dataSource.isEmpty()) { + return Response.status(Response.Status.BAD_REQUEST) + .entity(Collections.singletonMap("error", "No DataSource specified")) + .build(); + } + + final AutoCompactionSnapshot snapshot = scheduler.getCompactionSnapshot(dataSource); + if (snapshot == null) { return Response.status(Response.Status.NOT_FOUND) .entity(Collections.singletonMap("error", "Unknown DataSource")) .build(); } else { - return Response.ok(Collections.singletonMap("remainingSegmentSize", notCompactedSegmentSizeBytes)) + return Response.ok(Collections.singletonMap("remainingSegmentSize", snapshot.getBytesAwaitingCompaction())) .build(); } } @@ -99,6 +105,7 @@ public Response getCompactionSnapshotForDataSource( @POST @Path("/simulate") @Consumes(MediaType.APPLICATION_JSON) + @ResourceFilters(StateResourceFilter.class) public Response simulateClusterCompactionConfigUpdate( ClusterCompactionConfig updatePayload ) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/compact/CompactionSchedulerTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/compact/CompactionSchedulerTest.java deleted file mode 100644 index ca7e8aeac640..000000000000 --- a/indexing-service/src/test/java/org/apache/druid/indexing/compact/CompactionSchedulerTest.java +++ /dev/null @@ -1,49 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.druid.indexing.compact; - -import org.junit.Test; - -/** - * Tests the scheduling behaviour of the Compaction Scheduler and not the - * compaction of segments itself. There are other tests which already verify - * the compaction of segments with different configs and datasources. - */ -public class CompactionSchedulerTest -{ - // what are the different aspects we would like to test - // task status updates - // - // config - // - - // there should also be a test where we can do some sort of comparison of the two things - - // CompactSegmentsTest is not the right place for that because the entry point is CompactSegments - - // For us, the entry point is DruidCoordinator (i.e. sim) vs Compaction Scheduler. - - @Test - public void test() - { - - } - -} 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 af77b0e1c03d..4796654a1509 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 @@ -52,7 +52,6 @@ import org.apache.druid.server.coordinator.stats.Stats; import org.apache.druid.timeline.DataSegment; import org.joda.time.Period; -import org.junit.After; import org.junit.Assert; import org.junit.Before; import org.junit.Test; @@ -136,12 +135,6 @@ private void initScheduler() ); } - @After - public void tearDown() - { - - } - @Test public void testStartStopWhenSchedulerIsEnabled() { @@ -215,14 +208,14 @@ private void verifySegmentPolling(boolean enabled) @Test public void testStartCompactionForDatasource() { - final List wikiSegments = CreateDataSegments.ofDatasource(DS.WIKI).eachOfSizeInMb(100); + final List wikiSegments = CreateDataSegments.ofDatasource(TestDataSource.WIKI).eachOfSizeInMb(100); wikiSegments.forEach(segmentsMetadataManager::addSegment); scheduler.start(); scheduler.startCompaction( - DS.WIKI, + TestDataSource.WIKI, DataSourceCompactionConfig.builder() - .forDataSource(DS.WIKI) + .forDataSource(TestDataSource.WIKI) .withSkipOffsetFromLatest(Period.seconds(0)) .build() ); @@ -237,23 +230,19 @@ public void testStartCompactionForDatasource() Assert.assertTrue(submittedTask instanceof CompactionTask); final CompactionTask compactionTask = (CompactionTask) submittedTask; - Assert.assertEquals(DS.WIKI, compactionTask.getDataSource()); + Assert.assertEquals(TestDataSource.WIKI, compactionTask.getDataSource()); - final AutoCompactionSnapshot.Builder expectedSnapshot = AutoCompactionSnapshot.builder(DS.WIKI); + final AutoCompactionSnapshot.Builder expectedSnapshot = AutoCompactionSnapshot.builder(TestDataSource.WIKI); expectedSnapshot.incrementCompactedStats(CompactionStatistics.create(100_000_000, 1, 1)); Assert.assertEquals( expectedSnapshot.build(), - scheduler.getCompactionSnapshot(DS.WIKI) + scheduler.getCompactionSnapshot(TestDataSource.WIKI) ); Assert.assertEquals( - Collections.singletonMap(DS.WIKI, expectedSnapshot.build()), + Collections.singletonMap(TestDataSource.WIKI, expectedSnapshot.build()), scheduler.getAllCompactionSnapshots() ); - Assert.assertEquals( - 0L, - scheduler.getSegmentBytesAwaitingCompaction(DS.WIKI).longValue() - ); serviceEmitter.verifyValue(Stats.Compaction.SUBMITTED_TASKS.getMetricName(), 1L); serviceEmitter.verifyValue(Stats.Compaction.COMPACTED_BYTES.getMetricName(), 100_000_000L); @@ -264,26 +253,25 @@ public void testStartCompactionForDatasource() @Test public void testStopCompactionForDatasource() { - final List wikiSegments = CreateDataSegments.ofDatasource(DS.WIKI).eachOfSizeInMb(100); + final List wikiSegments = CreateDataSegments.ofDatasource(TestDataSource.WIKI).eachOfSizeInMb(100); wikiSegments.forEach(segmentsMetadataManager::addSegment); scheduler.start(); scheduler.startCompaction( - DS.WIKI, + TestDataSource.WIKI, DataSourceCompactionConfig.builder() - .forDataSource(DS.WIKI) + .forDataSource(TestDataSource.WIKI) .withSkipOffsetFromLatest(Period.seconds(0)) .build() ); - scheduler.stopCompaction(DS.WIKI); + scheduler.stopCompaction(TestDataSource.WIKI); executor.finishNextPendingTask(); Mockito.verify(taskQueue, Mockito.never()).add(ArgumentMatchers.any()); - Assert.assertNull(scheduler.getCompactionSnapshot(DS.WIKI)); + Assert.assertNull(scheduler.getCompactionSnapshot(TestDataSource.WIKI)); Assert.assertTrue(scheduler.getAllCompactionSnapshots().isEmpty()); - Assert.assertNull(scheduler.getSegmentBytesAwaitingCompaction(DS.WIKI)); serviceEmitter.verifyNotEmitted(Stats.Compaction.SUBMITTED_TASKS.getMetricName()); serviceEmitter.verifyNotEmitted(Stats.Compaction.COMPACTED_BYTES.getMetricName()); @@ -291,9 +279,4 @@ public void testStopCompactionForDatasource() scheduler.stop(); } - private static class DS - { - static final String WIKI = "wiki"; - } - } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/CompactionSupervisorsConfig.java b/server/src/main/java/org/apache/druid/server/coordinator/CompactionSupervisorsConfig.java index ab13eda3b1dd..cf4542afd5cb 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/CompactionSupervisorsConfig.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/CompactionSupervisorsConfig.java @@ -36,6 +36,7 @@ */ public class CompactionSupervisorsConfig { + public static final String PROPERTY_PREFIX = "druid.compaction.supervisors"; private static final CompactionSupervisorsConfig DEFAULT = new CompactionSupervisorsConfig(null); @JsonProperty diff --git a/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java b/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java index 99321f152b14..1c53fb9adc7e 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java @@ -348,12 +348,6 @@ public Integer getReplicationFactor(SegmentId segmentId) return replicaCountsInCluster == null ? null : replicaCountsInCluster.required(); } - @Nullable - public Long getTotalSizeOfSegmentsAwaitingCompaction(String dataSource) - { - return compactSegments.getTotalSizeOfSegmentsAwaitingCompaction(dataSource); - } - @Nullable public AutoCompactionSnapshot getAutoCompactionSnapshotForDataSource(String dataSource) { diff --git a/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java b/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java index 446529e4cd9b..c339af78ef3c 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 @@ -643,16 +643,6 @@ private void collectSnapshotStats( stats.add(Stats.Compaction.SKIPPED_INTERVALS, rowKey, autoCompactionSnapshot.getIntervalCountSkipped()); } - @Nullable - public Long getTotalSizeOfSegmentsAwaitingCompaction(String dataSource) - { - AutoCompactionSnapshot autoCompactionSnapshot = autoCompactionSnapshotPerDataSource.get().get(dataSource); - if (autoCompactionSnapshot == null) { - return null; - } - return autoCompactionSnapshot.getBytesAwaitingCompaction(); - } - @Nullable public AutoCompactionSnapshot getAutoCompactionSnapshot(String dataSource) { diff --git a/server/src/main/java/org/apache/druid/server/http/CompactionResource.java b/server/src/main/java/org/apache/druid/server/http/CoordinatorCompactionResource.java similarity index 89% rename from server/src/main/java/org/apache/druid/server/http/CompactionResource.java rename to server/src/main/java/org/apache/druid/server/http/CoordinatorCompactionResource.java index 7bbd82fa84a6..cd5e9ce8d76a 100644 --- a/server/src/main/java/org/apache/druid/server/http/CompactionResource.java +++ b/server/src/main/java/org/apache/druid/server/http/CoordinatorCompactionResource.java @@ -42,13 +42,13 @@ import java.util.Collection; @Path("/druid/coordinator/v1/compaction") -public class CompactionResource +public class CoordinatorCompactionResource { private final DruidCoordinator coordinator; private final CompactionSupervisorsConfig compactionSupervisorsConfig; @Inject - public CompactionResource( + public CoordinatorCompactionResource( DruidCoordinator coordinator, CompactionSupervisorsConfig compactionSupervisorsConfig ) @@ -82,11 +82,17 @@ public Response getCompactionProgress( buildErrorResponseWhenRunningAsSupervisor(); } - final Long notCompactedSegmentSizeBytes = coordinator.getTotalSizeOfSegmentsAwaitingCompaction(dataSource); - if (notCompactedSegmentSizeBytes == null) { + if (dataSource == null || dataSource.isEmpty()) { + return Response.status(Response.Status.BAD_REQUEST) + .entity(ImmutableMap.of("error", "No DataSource specified")) + .build(); + } + + final AutoCompactionSnapshot snapshot = coordinator.getAutoCompactionSnapshotForDataSource(dataSource); + if (snapshot == null) { return Response.status(Response.Status.NOT_FOUND).entity(ImmutableMap.of("error", "unknown dataSource")).build(); } else { - return Response.ok(ImmutableMap.of("remainingSegmentSize", notCompactedSegmentSizeBytes)).build(); + return Response.ok(ImmutableMap.of("remainingSegmentSize", snapshot.getBytesAwaitingCompaction())).build(); } } @@ -118,6 +124,7 @@ public Response getCompactionSnapshotForDataSource( @POST @Path("/simulate") @Consumes(MediaType.APPLICATION_JSON) + @ResourceFilters(StateResourceFilter.class) public Response simulateClusterCompactionConfigUpdate( ClusterCompactionConfig updatePayload ) diff --git a/server/src/test/java/org/apache/druid/server/http/CompactionResourceTest.java b/server/src/test/java/org/apache/druid/server/http/CoordinatorCompactionResourceTest.java similarity index 89% rename from server/src/test/java/org/apache/druid/server/http/CompactionResourceTest.java rename to server/src/test/java/org/apache/druid/server/http/CoordinatorCompactionResourceTest.java index 1e88bea2e099..a43e20206579 100644 --- a/server/src/test/java/org/apache/druid/server/http/CompactionResourceTest.java +++ b/server/src/test/java/org/apache/druid/server/http/CoordinatorCompactionResourceTest.java @@ -33,7 +33,7 @@ import javax.ws.rs.core.Response; import java.util.Map; -public class CompactionResourceTest +public class CoordinatorCompactionResourceTest { private DruidCoordinator mock; private final CompactionSupervisorsConfig schedulerConfig = new CompactionSupervisorsConfig(false); @@ -75,7 +75,7 @@ public void testGetCompactionSnapshotForDataSourceWithEmptyQueryParameter() EasyMock.expect(mock.getAutoCompactionSnapshot()).andReturn(expected).once(); EasyMock.replay(mock); - final Response response = new CompactionResource(mock, schedulerConfig) + final Response response = new CoordinatorCompactionResource(mock, schedulerConfig) .getCompactionSnapshotForDataSource(""); Assert.assertEquals(ImmutableMap.of("latestStatus", expected.values()), response.getEntity()); Assert.assertEquals(200, response.getStatus()); @@ -93,7 +93,7 @@ public void testGetCompactionSnapshotForDataSourceWithNullQueryParameter() EasyMock.expect(mock.getAutoCompactionSnapshot()).andReturn(expected).once(); EasyMock.replay(mock); - final Response response = new CompactionResource(mock, schedulerConfig) + final Response response = new CoordinatorCompactionResource(mock, schedulerConfig) .getCompactionSnapshotForDataSource(null); Assert.assertEquals(ImmutableMap.of("latestStatus", expected.values()), response.getEntity()); Assert.assertEquals(200, response.getStatus()); @@ -107,7 +107,7 @@ public void testGetCompactionSnapshotForDataSourceWithValidQueryParameter() EasyMock.expect(mock.getAutoCompactionSnapshotForDataSource(dataSourceName)).andReturn(expectedSnapshot).once(); EasyMock.replay(mock); - final Response response = new CompactionResource(mock, schedulerConfig) + final Response response = new CoordinatorCompactionResource(mock, schedulerConfig) .getCompactionSnapshotForDataSource(dataSourceName); Assert.assertEquals(ImmutableMap.of("latestStatus", ImmutableList.of(expectedSnapshot)), response.getEntity()); Assert.assertEquals(200, response.getStatus()); @@ -121,7 +121,7 @@ public void testGetCompactionSnapshotForDataSourceWithInvalidQueryParameter() EasyMock.expect(mock.getAutoCompactionSnapshotForDataSource(dataSourceName)).andReturn(null).once(); EasyMock.replay(mock); - final Response response = new CompactionResource(mock, schedulerConfig) + final Response response = new CoordinatorCompactionResource(mock, schedulerConfig) .getCompactionSnapshotForDataSource(dataSourceName); Assert.assertEquals(404, response.getStatus()); } @@ -131,7 +131,7 @@ public void testGetCompactionSnapshotWhenCompactionSchedulerIsEnabled() { EasyMock.replay(mock); - final Response response = new CompactionResource(mock, new CompactionSupervisorsConfig(true)) + final Response response = new CoordinatorCompactionResource(mock, new CompactionSupervisorsConfig(true)) .getCompactionSnapshotForDataSource("dummy"); Assert.assertEquals(503, response.getStatus()); Assert.assertEquals( diff --git a/services/src/main/java/org/apache/druid/cli/CliCoordinator.java b/services/src/main/java/org/apache/druid/cli/CliCoordinator.java index ea019b1c422a..54b44f68dcfc 100644 --- a/services/src/main/java/org/apache/druid/cli/CliCoordinator.java +++ b/services/src/main/java/org/apache/druid/cli/CliCoordinator.java @@ -113,7 +113,7 @@ import org.apache.druid.server.coordinator.duty.CoordinatorCustomDutyGroups; import org.apache.druid.server.coordinator.loading.LoadQueueTaskMaster; import org.apache.druid.server.http.ClusterResource; -import org.apache.druid.server.http.CompactionResource; +import org.apache.druid.server.http.CoordinatorCompactionResource; import org.apache.druid.server.http.CoordinatorCompactionConfigsResource; import org.apache.druid.server.http.CoordinatorDynamicConfigsResource; import org.apache.druid.server.http.CoordinatorRedirectInfo; @@ -223,7 +223,11 @@ public void configure(Binder binder) JsonConfigProvider.bind(binder, "druid.coordinator.balancer", BalancerStrategyFactory.class); JsonConfigProvider.bind(binder, "druid.coordinator.segment", CoordinatorSegmentWatcherConfig.class); JsonConfigProvider.bind(binder, "druid.coordinator.segmentMetadataCache", SegmentMetadataCacheConfig.class); - JsonConfigProvider.bind(binder, "druid.compaction.supervisors", CompactionSupervisorsConfig.class); + JsonConfigProvider.bind( + binder, + CompactionSupervisorsConfig.PROPERTY_PREFIX, + CompactionSupervisorsConfig.class + ); binder.bind(DruidCoordinatorConfig.class); binder.bind(RedirectFilter.class).in(LazySingleton.class); @@ -262,7 +266,7 @@ public void configure(Binder binder) .to(CoordinatorJettyServerInitializer.class); Jerseys.addResource(binder, CoordinatorResource.class); - Jerseys.addResource(binder, CompactionResource.class); + Jerseys.addResource(binder, CoordinatorCompactionResource.class); Jerseys.addResource(binder, CoordinatorDynamicConfigsResource.class); Jerseys.addResource(binder, CoordinatorCompactionConfigsResource.class); Jerseys.addResource(binder, TiersResource.class); diff --git a/services/src/main/java/org/apache/druid/cli/CliOverlord.java b/services/src/main/java/org/apache/druid/cli/CliOverlord.java index d8290df69473..6e81940926bc 100644 --- a/services/src/main/java/org/apache/druid/cli/CliOverlord.java +++ b/services/src/main/java/org/apache/druid/cli/CliOverlord.java @@ -218,7 +218,11 @@ public void configure(Binder binder) binder.bind(SegmentsMetadataManager.class) .toProvider(SegmentsMetadataManagerProvider.class) .in(ManageLifecycle.class); - JsonConfigProvider.bind(binder, "druid.compaction.supervisors", CompactionSupervisorsConfig.class); + JsonConfigProvider.bind( + binder, + CompactionSupervisorsConfig.PROPERTY_PREFIX, + CompactionSupervisorsConfig.class + ); } JsonConfigProvider.bind(binder, "druid.coordinator.asOverlord", CoordinatorOverlordServiceConfig.class); diff --git a/services/src/main/java/org/apache/druid/cli/CliRouter.java b/services/src/main/java/org/apache/druid/cli/CliRouter.java index a18e358603df..e6a257d0ba54 100644 --- a/services/src/main/java/org/apache/druid/cli/CliRouter.java +++ b/services/src/main/java/org/apache/druid/cli/CliRouter.java @@ -102,7 +102,11 @@ protected List getModules() JsonConfigProvider.bind(binder, "druid.router", TieredBrokerConfig.class); JsonConfigProvider.bind(binder, "druid.router.avatica.balancer", AvaticaConnectionBalancer.class); JsonConfigProvider.bind(binder, "druid.router.managementProxy", ManagementProxyConfig.class); - JsonConfigProvider.bind(binder, "druid.compaction.supervisors", CompactionSupervisorsConfig.class); + JsonConfigProvider.bind( + binder, + CompactionSupervisorsConfig.PROPERTY_PREFIX, + CompactionSupervisorsConfig.class + ); binder.bind(QuerySegmentWalker.class).to(NoopQuerySegmentWalker.class).in(LazySingleton.class); diff --git a/services/src/test/java/org/apache/druid/server/http/security/SecurityResourceFilterTest.java b/services/src/test/java/org/apache/druid/server/http/security/SecurityResourceFilterTest.java index c0b83145d352..a23f65ff2b63 100644 --- a/services/src/test/java/org/apache/druid/server/http/security/SecurityResourceFilterTest.java +++ b/services/src/test/java/org/apache/druid/server/http/security/SecurityResourceFilterTest.java @@ -28,7 +28,7 @@ import org.apache.druid.server.QueryResource; import org.apache.druid.server.StatusResource; import org.apache.druid.server.http.BrokerResource; -import org.apache.druid.server.http.CompactionResource; +import org.apache.druid.server.http.CoordinatorCompactionResource; import org.apache.druid.server.http.CoordinatorDynamicConfigsResource; import org.apache.druid.server.http.CoordinatorResource; import org.apache.druid.server.http.DataSourcesResource; @@ -74,7 +74,7 @@ public static Collection data() getRequestPathsWithAuthorizer(SelfDiscoveryResource.class), getRequestPathsWithAuthorizer(BrokerQueryResource.class), getRequestPathsWithAuthorizer(RouterResource.class), - getRequestPathsWithAuthorizer(CompactionResource.class) + getRequestPathsWithAuthorizer(CoordinatorCompactionResource.class) ) ); } From e111d71ebbe2e58162102cb60c520cc9d720c86e Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Fri, 23 Aug 2024 11:03:54 +0530 Subject: [PATCH 17/26] Fix checkstyle, add tests --- .../http/OverlordCompactionResourceTest.java | 25 ------------------- .../CoordinatorCompactionResourceTest.java | 18 +++++++++++-- .../org/apache/druid/cli/CliCoordinator.java | 2 +- 3 files changed, 17 insertions(+), 28 deletions(-) delete mode 100644 indexing-service/src/test/java/org/apache/druid/indexing/overlord/http/OverlordCompactionResourceTest.java diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/http/OverlordCompactionResourceTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/http/OverlordCompactionResourceTest.java deleted file mode 100644 index 83433a249a2d..000000000000 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/http/OverlordCompactionResourceTest.java +++ /dev/null @@ -1,25 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.druid.indexing.overlord.http; - -public class OverlordCompactionResourceTest -{ - -} diff --git a/server/src/test/java/org/apache/druid/server/http/CoordinatorCompactionResourceTest.java b/server/src/test/java/org/apache/druid/server/http/CoordinatorCompactionResourceTest.java index a43e20206579..0cd600d25f6c 100644 --- a/server/src/test/java/org/apache/druid/server/http/CoordinatorCompactionResourceTest.java +++ b/server/src/test/java/org/apache/druid/server/http/CoordinatorCompactionResourceTest.java @@ -37,8 +37,8 @@ public class CoordinatorCompactionResourceTest { private DruidCoordinator mock; private final CompactionSupervisorsConfig schedulerConfig = new CompactionSupervisorsConfig(false); - private String dataSourceName = "datasource_1"; - private AutoCompactionSnapshot expectedSnapshot = new AutoCompactionSnapshot( + private final String dataSourceName = "datasource_1"; + private final AutoCompactionSnapshot expectedSnapshot = new AutoCompactionSnapshot( dataSourceName, AutoCompactionSnapshot.AutoCompactionScheduleStatus.RUNNING, 1, @@ -126,6 +126,20 @@ public void testGetCompactionSnapshotForDataSourceWithInvalidQueryParameter() Assert.assertEquals(404, response.getStatus()); } + @Test + public void testGetProgressForNullDatasourceReturnsBadRequest() + { + EasyMock.replay(mock); + + final Response response = new CoordinatorCompactionResource(mock, schedulerConfig) + .getCompactionProgress(null); + Assert.assertEquals(Response.Status.BAD_REQUEST.getStatusCode(), response.getStatus()); + Assert.assertEquals( + ImmutableMap.of("error", "No DataSource specified"), + response.getEntity() + ); + } + @Test public void testGetCompactionSnapshotWhenCompactionSchedulerIsEnabled() { diff --git a/services/src/main/java/org/apache/druid/cli/CliCoordinator.java b/services/src/main/java/org/apache/druid/cli/CliCoordinator.java index 54b44f68dcfc..2c75b27bb1a8 100644 --- a/services/src/main/java/org/apache/druid/cli/CliCoordinator.java +++ b/services/src/main/java/org/apache/druid/cli/CliCoordinator.java @@ -113,8 +113,8 @@ import org.apache.druid.server.coordinator.duty.CoordinatorCustomDutyGroups; import org.apache.druid.server.coordinator.loading.LoadQueueTaskMaster; import org.apache.druid.server.http.ClusterResource; -import org.apache.druid.server.http.CoordinatorCompactionResource; import org.apache.druid.server.http.CoordinatorCompactionConfigsResource; +import org.apache.druid.server.http.CoordinatorCompactionResource; import org.apache.druid.server.http.CoordinatorDynamicConfigsResource; import org.apache.druid.server.http.CoordinatorRedirectInfo; import org.apache.druid.server.http.CoordinatorResource; From 93284b363c776bdd017022009ff8d0ab3688f5df Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Wed, 28 Aug 2024 17:38:50 +0530 Subject: [PATCH 18/26] Handle concurrency, validation, cleanup stale entries --- .../indexing/compact/CompactionScheduler.java | 3 ++ .../compact/CompactionSupervisorSpec.java | 6 ++-- .../compact/OverlordCompactionScheduler.java | 26 +++++++++++++-- .../compact/CompactionSupervisorSpecTest.java | 5 +++ .../compaction/CompactionStatusTracker.java | 32 ++++++++++++++++--- 5 files changed, 62 insertions(+), 10 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionScheduler.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionScheduler.java index a4cee72a40d2..ed2bb9ff17e6 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionScheduler.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionScheduler.java @@ -22,6 +22,7 @@ import org.apache.druid.server.compaction.CompactionSimulateResult; import org.apache.druid.server.coordinator.AutoCompactionSnapshot; import org.apache.druid.server.coordinator.ClusterCompactionConfig; +import org.apache.druid.server.coordinator.CompactionConfigValidationResult; import org.apache.druid.server.coordinator.CompactionSupervisorsConfig; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; @@ -49,6 +50,8 @@ public interface CompactionScheduler boolean isRunning(); + CompactionConfigValidationResult validateCompactionConfig(DataSourceCompactionConfig compactionConfig); + void startCompaction(String dataSourceName, DataSourceCompactionConfig compactionConfig); void stopCompaction(String dataSourceName); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionSupervisorSpec.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionSupervisorSpec.java index 02718499e740..6911f35f96e3 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionSupervisorSpec.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionSupervisorSpec.java @@ -25,6 +25,7 @@ import org.apache.druid.common.config.Configs; import org.apache.druid.error.InvalidInput; import org.apache.druid.indexing.overlord.supervisor.SupervisorSpec; +import org.apache.druid.server.coordinator.CompactionConfigValidationResult; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; import javax.annotation.Nullable; @@ -47,8 +48,9 @@ public CompactionSupervisorSpec( @JacksonInject CompactionScheduler scheduler ) { - if (spec == null) { - throw InvalidInput.exception("'spec' must be specified for a compaction supervisor."); + final CompactionConfigValidationResult validationResult = scheduler.validateCompactionConfig(spec); + if (!validationResult.isValid()) { + throw InvalidInput.exception("Compaction supervisor 'spec' is invalid. Reason[%s].", validationResult.getReason()); } this.spec = spec; 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 b49766eb120c..6b7a82e9c9c1 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 @@ -22,6 +22,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Supplier; import com.google.inject.Inject; +import org.apache.druid.client.indexing.ClientCompactionRunnerInfo; import org.apache.druid.indexing.overlord.TaskMaster; import org.apache.druid.indexing.overlord.TaskQueryTool; import org.apache.druid.java.util.common.Stopwatch; @@ -36,6 +37,7 @@ import org.apache.druid.server.compaction.CompactionStatusTracker; import org.apache.druid.server.coordinator.AutoCompactionSnapshot; import org.apache.druid.server.coordinator.ClusterCompactionConfig; +import org.apache.druid.server.coordinator.CompactionConfigValidationResult; import org.apache.druid.server.coordinator.CompactionSupervisorsConfig; import org.apache.druid.server.coordinator.CoordinatorOverlordServiceConfig; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; @@ -44,6 +46,7 @@ import org.apache.druid.server.coordinator.stats.CoordinatorRunStats; import org.apache.druid.server.coordinator.stats.CoordinatorStat; import org.apache.druid.server.coordinator.stats.Dimension; +import org.apache.druid.server.coordinator.stats.Stats; import org.apache.druid.timeline.SegmentTimeline; import org.joda.time.Duration; @@ -135,7 +138,7 @@ public void start() if (isEnabled() && started.compareAndSet(false, true)) { log.info("Starting compaction scheduler."); initState(); - scheduleOnExecutor(this::checkSchedulingStatus, SCHEDULE_PERIOD_SECONDS); + scheduleOnExecutor(this::scheduledRun, SCHEDULE_PERIOD_SECONDS); } } @@ -154,6 +157,19 @@ public boolean isRunning() return isEnabled() && started.get(); } + @Override + public CompactionConfigValidationResult validateCompactionConfig(DataSourceCompactionConfig compactionConfig) + { + if (compactionConfig == null) { + return CompactionConfigValidationResult.failure("Cannot be null"); + } else { + return ClientCompactionRunnerInfo.validateCompactionConfig( + compactionConfig, + compactionConfigSupplier.get().getEngine() + ); + } + } + @Override public void startCompaction(String dataSourceName, DataSourceCompactionConfig config) { @@ -191,7 +207,7 @@ private boolean isEnabled() return schedulerConfig.isEnabled(); } - private synchronized void checkSchedulingStatus() + private synchronized void scheduledRun() { if (isRunning()) { try { @@ -200,7 +216,7 @@ private synchronized void checkSchedulingStatus() catch (Exception e) { log.error(e, "Error processing compaction queue. Continuing schedule."); } - scheduleOnExecutor(this::checkSchedulingStatus, SCHEDULE_PERIOD_SECONDS); + scheduleOnExecutor(this::scheduledRun, SCHEDULE_PERIOD_SECONDS); } else { cleanupState(); } @@ -221,6 +237,10 @@ private synchronized void runCompactionDuty() } ); sinceStatsEmitted.restart(); + } else { + // Always emit number of submitted tasks + long numSubmittedTasks = stats.get(Stats.Compaction.SUBMITTED_TASKS); + emitStat(Stats.Compaction.SUBMITTED_TASKS, Collections.emptyMap(), numSubmittedTasks); } } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/compact/CompactionSupervisorSpecTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/compact/CompactionSupervisorSpecTest.java index bea0e5165488..fc75d3190f0d 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/compact/CompactionSupervisorSpecTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/compact/CompactionSupervisorSpecTest.java @@ -25,10 +25,12 @@ import org.apache.druid.guice.SupervisorModule; import org.apache.druid.indexing.overlord.supervisor.SupervisorSpec; import org.apache.druid.jackson.DefaultObjectMapper; +import org.apache.druid.server.coordinator.CompactionConfigValidationResult; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; import org.junit.Assert; import org.junit.Before; import org.junit.Test; +import org.mockito.ArgumentMatchers; import org.mockito.Mockito; import java.util.Collections; @@ -42,6 +44,9 @@ public class CompactionSupervisorSpecTest public void setUp() { scheduler = Mockito.mock(CompactionScheduler.class); + Mockito.when(scheduler.validateCompactionConfig(ArgumentMatchers.any())) + .thenReturn(CompactionConfigValidationResult.success()); + OBJECT_MAPPER.setInjectableValues( new InjectableValues.Std() .addValue(CompactionScheduler.class, scheduler) 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 2041a2e4cf81..753c80d9da50 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 @@ -28,12 +28,12 @@ import org.apache.druid.server.coordinator.DataSourceCompactionConfig; import org.apache.druid.server.coordinator.DruidCompactionConfig; import org.joda.time.DateTime; +import org.joda.time.Duration; import org.joda.time.Interval; -import java.util.HashMap; import java.util.HashSet; -import java.util.Map; import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; /** * Tracks status of recently submitted compaction tasks. Can be used by a segment @@ -42,9 +42,13 @@ */ public class CompactionStatusTracker { + private static final Duration MAX_STATUS_RETAIN_DURATION = Duration.standardHours(12); + private final ObjectMapper objectMapper; - private final Map datasourceStatuses = new HashMap<>(); - private final Map submittedTaskIdToSegments = new HashMap<>(); + private final ConcurrentHashMap datasourceStatuses + = new ConcurrentHashMap<>(); + private final ConcurrentHashMap submittedTaskIdToSegments + = new ConcurrentHashMap<>(); @Inject public CompactionStatusTracker(ObjectMapper objectMapper) @@ -104,6 +108,9 @@ public void onCompactionConfigUpdated(DruidCompactionConfig compactionConfig) datasourceStatuses.remove(datasource); } }); + + // Clean up stale task statuses + datasourceStatuses.values().forEach(DatasourceStatus::cleanupStaleTaskStatuses); } public void onTaskSubmitted( @@ -142,7 +149,8 @@ private static class DatasourceStatus { static final DatasourceStatus EMPTY = new DatasourceStatus(); - final Map intervalToTaskStatus = new HashMap<>(); + final ConcurrentHashMap intervalToTaskStatus + = new ConcurrentHashMap<>(); void handleCompletedTask(Interval compactionInterval, TaskStatus taskStatus) { @@ -180,5 +188,19 @@ void handleSubmittedTask(SegmentsToCompact candidateSegments) ); } } + + void cleanupStaleTaskStatuses() + { + final DateTime now = DateTimes.nowUtc(); + + final Set staleIntervals = new HashSet<>(); + intervalToTaskStatus.forEach((interval, taskStatus) -> { + if (taskStatus.getUpdatedTime().plus(MAX_STATUS_RETAIN_DURATION).isBefore(now)) { + staleIntervals.add(interval); + } + }); + + staleIntervals.forEach(intervalToTaskStatus::remove); + } } } From 117b0ab86a9e5957f44c7ee07a40f34ad5d6f6dc Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Wed, 28 Aug 2024 19:58:38 +0530 Subject: [PATCH 19/26] Fix test, avoid metadata store calls --- .../compact/OverlordCompactionScheduler.java | 49 ++++++++++++++++++- .../indexing/overlord/DruidOverlord.java | 3 +- .../indexing/overlord/TaskQueryTool.java | 38 +++++++++++++- .../compact/CompactionSupervisorSpecTest.java | 2 +- .../compaction/CompactionStatusTracker.java | 4 ++ .../coordinator/duty/CompactSegments.java | 4 +- .../compaction/CompactionStatusTest.java | 6 ++- 7 files changed, 98 insertions(+), 8 deletions(-) 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 6b7a82e9c9c1..f4102ce4c142 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 @@ -20,18 +20,23 @@ package org.apache.druid.indexing.compact; import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Optional; import com.google.common.base.Supplier; import com.google.inject.Inject; import org.apache.druid.client.indexing.ClientCompactionRunnerInfo; +import org.apache.druid.indexer.TaskLocation; +import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexing.overlord.TaskMaster; import org.apache.druid.indexing.overlord.TaskQueryTool; +import org.apache.druid.indexing.overlord.TaskRunner; +import org.apache.druid.indexing.overlord.TaskRunnerListener; import org.apache.druid.java.util.common.Stopwatch; +import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.common.concurrent.ScheduledExecutorFactory; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.java.util.emitter.service.ServiceEmitter; import org.apache.druid.java.util.emitter.service.ServiceMetricEvent; import org.apache.druid.metadata.SegmentsMetadataManager; -import org.apache.druid.rpc.indexing.OverlordClient; import org.apache.druid.server.compaction.CompactionRunSimulator; import org.apache.druid.server.compaction.CompactionSimulateResult; import org.apache.druid.server.compaction.CompactionStatusTracker; @@ -79,8 +84,9 @@ public class OverlordCompactionScheduler implements CompactionScheduler private static final Duration METRIC_EMISSION_PERIOD = Duration.standardMinutes(5); private final SegmentsMetadataManager segmentManager; - private final OverlordClient overlordClient; + private final LocalOverlordClient overlordClient; private final ServiceEmitter emitter; + private final TaskMaster taskMaster; private final CompactionSupervisorsConfig schedulerConfig; private final Supplier compactionConfigSupplier; @@ -92,6 +98,13 @@ public class OverlordCompactionScheduler implements CompactionScheduler private final ScheduledExecutorService executor; private final CompactionStatusTracker statusTracker; + + /** + * Listener to watch task completion events and update CompactionStatusTracker. + * This helps in avoiding unnecessary metadata store calls. + */ + private final TaskRunnerListener taskRunnerListener; + private final AtomicBoolean started = new AtomicBoolean(false); private final CompactSegments duty; @@ -120,6 +133,7 @@ public OverlordCompactionScheduler( { this.segmentManager = segmentManager; this.emitter = emitter; + this.taskMaster = taskMaster; this.schedulerConfig = schedulerConfig; this.compactionConfigSupplier = compactionConfigSupplier; @@ -130,6 +144,29 @@ public OverlordCompactionScheduler( this.overlordClient = new LocalOverlordClient(taskMaster, taskQueryTool, objectMapper); this.duty = new CompactSegments(this.statusTracker, overlordClient); this.activeDatasourceConfigs = new ConcurrentHashMap<>(); + + this.taskRunnerListener = new TaskRunnerListener() + { + @Override + public String getListenerId() + { + return "OverlordCompactionScheduler"; + } + + @Override + public void locationChanged(String taskId, TaskLocation newLocation) + { + // Do nothing + } + + @Override + public void statusChanged(String taskId, TaskStatus status) + { + if (status.isComplete()) { + statusTracker.onTaskFinished(taskId, status); + } + } + }; } @Override @@ -187,6 +224,10 @@ public void stopCompaction(String dataSourceName) private synchronized void initState() { + final Optional taskRunnerOptional = taskMaster.getTaskRunner(); + if (taskRunnerOptional.isPresent()) { + taskRunnerOptional.get().registerListener(taskRunnerListener, Execs.directExecutor()); + } if (shouldPollSegments) { segmentManager.startPollingDatabasePeriodically(); } @@ -194,6 +235,10 @@ private synchronized void initState() private synchronized void cleanupState() { + final Optional taskRunnerOptional = taskMaster.getTaskRunner(); + if (taskRunnerOptional.isPresent()) { + taskRunnerOptional.get().unregisterListener(taskRunnerListener.getListenerId()); + } statusTracker.stop(); activeDatasourceConfigs.clear(); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/DruidOverlord.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/DruidOverlord.java index 844e4922e244..fb4730e7df40 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/DruidOverlord.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/DruidOverlord.java @@ -132,7 +132,6 @@ public void becomeLeader() leaderLifecycle.addManagedInstance(taskRunner); leaderLifecycle.addManagedInstance(taskQueue); - leaderLifecycle.addManagedInstance(compactionScheduler); leaderLifecycle.addManagedInstance(supervisorManager); leaderLifecycle.addManagedInstance(overlordDutyExecutor); leaderLifecycle.addHandler( @@ -143,6 +142,7 @@ public void start() { segmentAllocationQueue.becomeLeader(); taskMaster.becomeLeader(taskRunner, taskQueue); + compactionScheduler.start(); // Announce the node only after all the services have been initialized initialized = true; @@ -153,6 +153,7 @@ public void start() public void stop() { serviceAnnouncer.unannounce(node); + compactionScheduler.stop(); taskMaster.stopBeingLeader(); segmentAllocationQueue.stopBeingLeader(); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskQueryTool.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskQueryTool.java index c51b84b0a762..4c37af7ef169 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskQueryTool.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskQueryTool.java @@ -35,6 +35,7 @@ import org.apache.druid.indexing.overlord.http.TotalWorkerCapacityResponse; import org.apache.druid.indexing.overlord.setup.DefaultWorkerBehaviorConfig; import org.apache.druid.indexing.overlord.setup.WorkerBehaviorConfig; +import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.StringUtils; @@ -42,6 +43,7 @@ import org.apache.druid.metadata.LockFilterPolicy; import org.apache.druid.metadata.TaskLookup; import org.apache.druid.metadata.TaskLookup.TaskLookupType; +import org.joda.time.DateTime; import org.joda.time.Duration; import org.joda.time.Interval; @@ -153,7 +155,41 @@ public TaskInfo getTaskInfo(String taskId) public List getAllActiveTasks() { - return getTaskStatusPlusList(TaskStateLookup.ALL, null, null, 0, null); + final Optional taskQueue = taskMaster.getTaskQueue(); + if (taskQueue.isPresent()) { + // Serve active task statuses from memory + final List taskStatusPlusList = new ArrayList<>(); + + // Use a dummy created time as this is not used by the caller, just needs to be non-null + final DateTime createdTime = DateTimes.nowUtc(); + + final List activeTasks = taskQueue.get().getTasks(); + for (Task task : activeTasks) { + final Optional statusOptional = taskQueue.get().getTaskStatus(task.getId()); + if (statusOptional.isPresent()) { + final TaskStatus status = statusOptional.get(); + taskStatusPlusList.add( + new TaskStatusPlus( + task.getId(), + task.getGroupId(), + task.getType(), + createdTime, + createdTime, + status.getStatusCode(), + null, + status.getDuration(), + status.getLocation(), + task.getDataSource(), + status.getErrorMsg() + ) + ); + } + } + + return taskStatusPlusList; + } else { + return getTaskStatusPlusList(TaskStateLookup.ALL, null, null, 0, null); + } } public List getTaskStatusPlusList( diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/compact/CompactionSupervisorSpecTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/compact/CompactionSupervisorSpecTest.java index fc75d3190f0d..b02f9d42919c 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/compact/CompactionSupervisorSpecTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/compact/CompactionSupervisorSpecTest.java @@ -46,7 +46,7 @@ public void setUp() scheduler = Mockito.mock(CompactionScheduler.class); Mockito.when(scheduler.validateCompactionConfig(ArgumentMatchers.any())) .thenReturn(CompactionConfigValidationResult.success()); - + OBJECT_MAPPER.setInjectableValues( new InjectableValues.Std() .addValue(CompactionScheduler.class, scheduler) 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 753c80d9da50..5d467302bc96 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 @@ -79,6 +79,10 @@ public CompactionTaskStatus getLatestTaskStatus(SegmentsToCompact candidates) .get(candidates.getUmbrellaInterval()); } + /** + * Set of submitted compaction task IDs which have not been marked completed + * via {@link #onTaskFinished} yet. + */ public Set getSubmittedTaskIds() { return submittedTaskIdToSegments.keySet(); 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 c339af78ef3c..6baa3e08661d 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 @@ -261,7 +261,9 @@ private void trackStatusOfCompletedTasks(Set activeTaskIds) for (String taskId : finishedTaskIds) { // Assume unknown task to have finished successfully final TaskStatus taskStatus = taskStatusMap.getOrDefault(taskId, TaskStatus.success(taskId)); - statusTracker.onTaskFinished(taskId, taskStatus); + if (taskStatus.isComplete()) { + statusTracker.onTaskFinished(taskId, taskStatus); + } } } 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 8f0697bcb408..841f49f70ecf 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 @@ -239,10 +239,12 @@ public void testStatusOnIndexSpecMismatch() "'indexSpec' mismatch: " + "required[IndexSpec{bitmapSerdeFactory=RoaringBitmapSerdeFactory{}," + " dimensionCompression=lz4, stringDictionaryEncoding=Utf8{}," - + " metricCompression=lz4, longEncoding=longs, jsonCompression=null, segmentLoader=null}], " + + " metricCompression=lz4, longEncoding=longs, complexMetricCompression=null," + + " jsonCompression=null, segmentLoader=null}], " + "current[IndexSpec{bitmapSerdeFactory=RoaringBitmapSerdeFactory{}," + " dimensionCompression=zstd, stringDictionaryEncoding=Utf8{}," - + " metricCompression=lz4, longEncoding=longs, jsonCompression=null, segmentLoader=null}]" + + " metricCompression=lz4, longEncoding=longs, complexMetricCompression=null," + + " jsonCompression=null, segmentLoader=null}]" ); } From 0a584d429269d2c9b0850363910348b6e23da2b9 Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Fri, 30 Aug 2024 09:43:11 +0530 Subject: [PATCH 20/26] Redirect compaction progress APIs to Overlord --- .../indexing/compact/LocalOverlordClient.java | 19 +++++ .../http/OverlordCompactionResource.java | 33 +++++++- .../compact/CompactionSupervisorSpecTest.java | 15 ++++ .../OverlordCompactionSchedulerTest.java | 82 +++++++++++++++++++ .../druid/rpc/indexing/OverlordClient.java | 26 ++++++ .../rpc/indexing/OverlordClientImpl.java | 47 +++++++++++ .../AsyncManagementForwardingServlet.java | 20 +---- .../compaction/CompactionRunSimulator.java | 19 +++++ .../apache/druid/server/compaction/Table.java | 1 + .../coordinator/AutoCompactionSnapshot.java | 22 ++--- .../server/coordinator/DruidCoordinator.java | 33 +++++--- .../coordinator/duty/CompactSegments.java | 1 - .../http/CoordinatorCompactionResource.java | 71 ++++++++++------ .../client/indexing/NoopOverlordClient.java | 19 +++++ .../rpc/indexing/OverlordClientImplTest.java | 80 ++++++++++++++++++ .../AsyncManagementForwardingServletTest.java | 51 ------------ .../coordinator/DruidCoordinatorTest.java | 15 ++-- .../CoordinatorSimulationBuilder.java | 4 +- .../CoordinatorCompactionResourceTest.java | 61 ++++++++++---- .../org/apache/druid/cli/CliCoordinator.java | 6 -- .../java/org/apache/druid/cli/CliRouter.java | 6 -- 21 files changed, 470 insertions(+), 161 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/LocalOverlordClient.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/LocalOverlordClient.java index c3f917c4d365..eeaf138c1065 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/compact/LocalOverlordClient.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/LocalOverlordClient.java @@ -45,6 +45,7 @@ import org.apache.druid.metadata.LockFilterPolicy; import org.apache.druid.rpc.ServiceRetryPolicy; import org.apache.druid.rpc.indexing.OverlordClient; +import org.apache.druid.server.coordinator.AutoCompactionSnapshot; import org.joda.time.Interval; import javax.annotation.Nullable; @@ -236,6 +237,24 @@ public ListenableFuture> getWorkers() throw new UnsupportedOperationException(); } + @Override + public ListenableFuture> getCompactionSnapshots(@Nullable String dataSource) + { + throw new UnsupportedOperationException(); + } + + @Override + public ListenableFuture getBytesAwaitingCompaction(String dataSource) + { + throw new UnsupportedOperationException(); + } + + @Override + public ListenableFuture isCompactionSupervisorEnabled() + { + throw new UnsupportedOperationException(); + } + @Override public OverlordClient withRetryPolicy(ServiceRetryPolicy retryPolicy) { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordCompactionResource.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordCompactionResource.java index fa0b928f4a6c..77fc2e916cc9 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordCompactionResource.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordCompactionResource.java @@ -19,6 +19,7 @@ package org.apache.druid.indexing.overlord.http; +import com.google.common.collect.ImmutableMap; import com.google.inject.Inject; import com.sun.jersey.spi.container.ResourceFilters; import org.apache.druid.indexing.compact.CompactionScheduler; @@ -54,6 +55,15 @@ public OverlordCompactionResource( this.scheduler = scheduler; } + @GET + @Path("/isSupervisorEnabled") + @Produces(MediaType.APPLICATION_JSON) + @ResourceFilters(StateResourceFilter.class) + public Response isCompactionSupervisorEnabled() + { + return Response.ok(scheduler.isRunning()).build(); + } + @GET @Path("/progress") @Produces(MediaType.APPLICATION_JSON) @@ -62,6 +72,10 @@ public Response getCompactionProgress( @QueryParam("dataSource") String dataSource ) { + if (!scheduler.isRunning()) { + return buildErrorResponseIfSchedulerDisabled(); + } + if (dataSource == null || dataSource.isEmpty()) { return Response.status(Response.Status.BAD_REQUEST) .entity(Collections.singletonMap("error", "No DataSource specified")) @@ -83,10 +97,14 @@ public Response getCompactionProgress( @Path("/status") @Produces(MediaType.APPLICATION_JSON) @ResourceFilters(StateResourceFilter.class) - public Response getCompactionSnapshotForDataSource( + public Response getCompactionSnapshots( @QueryParam("dataSource") String dataSource ) { + if (!scheduler.isRunning()) { + return buildErrorResponseIfSchedulerDisabled(); + } + final Collection snapshots; if (dataSource == null || dataSource.isEmpty()) { snapshots = scheduler.getAllCompactionSnapshots().values(); @@ -106,7 +124,7 @@ public Response getCompactionSnapshotForDataSource( @Path("/simulate") @Consumes(MediaType.APPLICATION_JSON) @ResourceFilters(StateResourceFilter.class) - public Response simulateClusterCompactionConfigUpdate( + public Response simulateRunWithConfigUpdate( ClusterCompactionConfig updatePayload ) { @@ -114,4 +132,15 @@ public Response simulateClusterCompactionConfigUpdate( scheduler.simulateRunWithConfigUpdate(updatePayload) ).build(); } + + private Response buildErrorResponseIfSchedulerDisabled() + { + return Response.status(Response.Status.SERVICE_UNAVAILABLE).entity( + ImmutableMap.of( + "error", + "Compaction Supervisors are disabled on the Overlord." + + " Use Coordinator APIs to fetch compaction status." + ) + ).build(); + } } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/compact/CompactionSupervisorSpecTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/compact/CompactionSupervisorSpecTest.java index b02f9d42919c..1b2c1826c06f 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/compact/CompactionSupervisorSpecTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/compact/CompactionSupervisorSpecTest.java @@ -80,6 +80,21 @@ public void testSerdeOfSuspendedSpec() ); } + @Test + public void testInvalidSpecThrowsException() + { + Mockito.when(scheduler.validateCompactionConfig(ArgumentMatchers.any())) + .thenReturn(CompactionConfigValidationResult.failure("bad spec")); + final DruidException exception = Assert.assertThrows( + DruidException.class, + () -> new CompactionSupervisorSpec(null, false, scheduler) + ); + Assert.assertEquals( + "Compaction supervisor 'spec' is invalid. Reason[bad spec].", + exception.getMessage() + ); + } + @Test public void testGetIdAndDataSources() { 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 4796654a1509..92682e7b0515 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/compact/OverlordCompactionSchedulerTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/compact/OverlordCompactionSchedulerTest.java @@ -21,7 +21,9 @@ import com.fasterxml.jackson.databind.InjectableValues; import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.client.indexing.ClientMSQContext; import org.apache.druid.guice.IndexingServiceTuningConfigModule; +import org.apache.druid.indexer.CompactionEngine; import org.apache.druid.indexing.common.SegmentCacheManagerFactory; import org.apache.druid.indexing.common.config.TaskStorageConfig; import org.apache.druid.indexing.common.task.CompactionTask; @@ -36,11 +38,18 @@ import org.apache.druid.indexing.overlord.setup.WorkerBehaviorConfig; import org.apache.druid.indexing.test.TestIndexerMetadataStorageCoordinator; import org.apache.druid.jackson.DefaultObjectMapper; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.metrics.StubServiceEmitter; import org.apache.druid.segment.TestIndex; +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; +import org.apache.druid.server.coordinator.ClusterCompactionConfig; +import org.apache.druid.server.coordinator.CompactionConfigValidationResult; import org.apache.druid.server.coordinator.CompactionSupervisorsConfig; import org.apache.druid.server.coordinator.CoordinatorOverlordServiceConfig; import org.apache.druid.server.coordinator.CreateDataSegments; @@ -59,6 +68,7 @@ import org.mockito.ArgumentMatchers; import org.mockito.Mockito; +import java.util.Arrays; import java.util.Collections; import java.util.List; @@ -205,6 +215,32 @@ private void verifySegmentPolling(boolean enabled) Assert.assertFalse(segmentsMetadataManager.isPollingDatabasePeriodically()); } + @Test + public void testNullCompactionConfigIsInvalid() + { + final CompactionConfigValidationResult result = scheduler.validateCompactionConfig(null); + Assert.assertFalse(result.isValid()); + Assert.assertEquals("Cannot be null", result.getReason()); + } + + @Test + public void testMsqCompactionConfigWithOneMaxTasksIsInvalid() + { + final DataSourceCompactionConfig datasourceConfig = DataSourceCompactionConfig + .builder() + .forDataSource(TestDataSource.WIKI) + .withEngine(CompactionEngine.MSQ) + .withTaskContext(Collections.singletonMap(ClientMSQContext.CTX_MAX_NUM_TASKS, 1)) + .build(); + + final CompactionConfigValidationResult result = scheduler.validateCompactionConfig(datasourceConfig); + Assert.assertFalse(result.isValid()); + Assert.assertEquals( + "MSQ: Context maxNumTasks[1] must be at least 2 (1 controller + 1 worker)", + result.getReason() + ); + } + @Test public void testStartCompactionForDatasource() { @@ -279,4 +315,50 @@ public void testStopCompactionForDatasource() scheduler.stop(); } + @Test + public void testRunSimulation() + { + final List wikiSegments = CreateDataSegments + .ofDatasource(TestDataSource.WIKI) + .forIntervals(1, Granularities.DAY) + .startingAt("2013-01-01") + .withNumPartitions(10) + .eachOfSizeInMb(100); + wikiSegments.forEach(segmentsMetadataManager::addSegment); + + scheduler.start(); + scheduler.startCompaction( + TestDataSource.WIKI, + DataSourceCompactionConfig.builder() + .forDataSource(TestDataSource.WIKI) + .withSkipOffsetFromLatest(Period.seconds(0)) + .build() + ); + + final CompactionSimulateResult simulateResult = scheduler.simulateRunWithConfigUpdate( + new ClusterCompactionConfig(null, null, null, null, null) + ); + Assert.assertEquals(1, simulateResult.getCompactionStates().size()); + final Table pendingCompactionTable = simulateResult.getCompactionStates().get(CompactionStatus.State.PENDING); + Assert.assertEquals( + Arrays.asList("dataSource", "interval", "numSegments", "bytes", "maxTaskSlots", "reasonToCompact"), + pendingCompactionTable.getColumnNames() + ); + Assert.assertEquals( + Collections.singletonList( + Arrays.asList("wiki", Intervals.of("2013-01-01/P1D"), 10, 1_000_000_000L, 1, "not compacted yet") + ), + pendingCompactionTable.getRows() + ); + + scheduler.stopCompaction(TestDataSource.WIKI); + + final CompactionSimulateResult simulateResultWhenDisabled = scheduler.simulateRunWithConfigUpdate( + new ClusterCompactionConfig(null, null, null, null, null) + ); + Assert.assertTrue(simulateResultWhenDisabled.getCompactionStates().isEmpty()); + + scheduler.stop(); + } + } diff --git a/server/src/main/java/org/apache/druid/rpc/indexing/OverlordClient.java b/server/src/main/java/org/apache/druid/rpc/indexing/OverlordClient.java index 310684206d28..6797a00b432b 100644 --- a/server/src/main/java/org/apache/druid/rpc/indexing/OverlordClient.java +++ b/server/src/main/java/org/apache/druid/rpc/indexing/OverlordClient.java @@ -35,6 +35,7 @@ import org.apache.druid.java.util.common.parsers.CloseableIterator; import org.apache.druid.metadata.LockFilterPolicy; import org.apache.druid.rpc.ServiceRetryPolicy; +import org.apache.druid.server.coordinator.AutoCompactionSnapshot; import org.joda.time.DateTime; import org.joda.time.Interval; @@ -212,6 +213,31 @@ ListenableFuture>> findLockedIntervals( */ ListenableFuture getTotalWorkerCapacity(); + /** + * Checks if compaction supervisors are enabled on the Overlord. + * When this returns true, the Coordinator does not run CompactSegments duty. + *

+ * API: {@code /druid/indexer/v1/compaction/isSupervisorEnabled} + */ + ListenableFuture isCompactionSupervisorEnabled(); + + /** + * Gets the number of bytes yet to be compacted for the given datasource. + *

+ * API: {@code /druid/indexer/v1/compaction/progress} + */ + ListenableFuture getBytesAwaitingCompaction(String dataSource); + + /** + * Gets the latest compaction snapshots of one or all datasources. + *

+ * API: {@code /druid/indexer/v1/compaction/status} + * + * @param dataSource If passed as non-null, then the returned list contains only + * the snapshot for this datasource. + */ + ListenableFuture> getCompactionSnapshots(@Nullable String dataSource); + /** * Returns a copy of this client with a different retry policy. */ diff --git a/server/src/main/java/org/apache/druid/rpc/indexing/OverlordClientImpl.java b/server/src/main/java/org/apache/druid/rpc/indexing/OverlordClientImpl.java index 3e3d86ca5f25..2a48e0ed6928 100644 --- a/server/src/main/java/org/apache/druid/rpc/indexing/OverlordClientImpl.java +++ b/server/src/main/java/org/apache/druid/rpc/indexing/OverlordClientImpl.java @@ -45,6 +45,7 @@ import org.apache.druid.rpc.RequestBuilder; import org.apache.druid.rpc.ServiceClient; import org.apache.druid.rpc.ServiceRetryPolicy; +import org.apache.druid.server.coordinator.AutoCompactionSnapshot; import org.jboss.netty.handler.codec.http.HttpMethod; import org.joda.time.Interval; @@ -312,6 +313,52 @@ public ListenableFuture taskPayload(final String taskId) ); } + @Override + public ListenableFuture> getCompactionSnapshots(@Nullable String dataSource) + { + final StringBuilder pathBuilder = new StringBuilder("/druid/indexer/v1/compaction/status"); + if (dataSource != null && !dataSource.isEmpty()) { + pathBuilder.append("?").append("dataSource=").append(dataSource); + } + + return FutureUtils.transform( + client.asyncRequest( + new RequestBuilder(HttpMethod.GET, pathBuilder.toString()), + new BytesFullResponseHandler() + ), + holder -> JacksonUtils.readValue( + jsonMapper, + holder.getContent(), + new TypeReference>() {} + ) + ); + } + + @Override + public ListenableFuture getBytesAwaitingCompaction(String dataSource) + { + final String path = "/druid/indexer/v1/compaction/progress?dataSource=" + dataSource; + return FutureUtils.transform( + client.asyncRequest( + new RequestBuilder(HttpMethod.GET, path), + new BytesFullResponseHandler() + ), + holder -> JacksonUtils.readValue(jsonMapper, holder.getContent(), Long.class) + ); + } + + @Override + public ListenableFuture isCompactionSupervisorEnabled() + { + return FutureUtils.transform( + client.asyncRequest( + new RequestBuilder(HttpMethod.GET, "/druid/indexer/v1/compaction/isSupervisorEnabled"), + new BytesFullResponseHandler() + ), + holder -> JacksonUtils.readValue(jsonMapper, holder.getContent(), Boolean.class) + ); + } + @Override public OverlordClientImpl withRetryPolicy(ServiceRetryPolicy retryPolicy) { diff --git a/server/src/main/java/org/apache/druid/server/AsyncManagementForwardingServlet.java b/server/src/main/java/org/apache/druid/server/AsyncManagementForwardingServlet.java index 458c8b69c1f9..4c8db118d37b 100644 --- a/server/src/main/java/org/apache/druid/server/AsyncManagementForwardingServlet.java +++ b/server/src/main/java/org/apache/druid/server/AsyncManagementForwardingServlet.java @@ -30,7 +30,6 @@ import org.apache.druid.guice.annotations.Json; import org.apache.druid.guice.http.DruidHttpClientConfig; import org.apache.druid.java.util.common.StringUtils; -import org.apache.druid.server.coordinator.CompactionSupervisorsConfig; import org.apache.druid.server.initialization.jetty.StandardResponseHeaderFilterHolder; import org.apache.druid.server.security.AuthConfig; import org.apache.druid.server.security.AuthorizationUtils; @@ -67,12 +66,6 @@ public class AsyncManagementForwardingServlet extends AsyncProxyServlet private static final String ARBITRARY_COORDINATOR_BASE_PATH = "/proxy/coordinator"; private static final String ARBITRARY_OVERLORD_BASE_PATH = "/proxy/overlord"; - // If Compaction Scheduler is enabled, compaction status APIs must be forwarded to the Overlord - // Client Request: https://{ROUTER_HOST}:9088/druid/coordinator/v1/compaction/status - // Proxy Request: https://{OVERLORD_HOST}:8281/druid/indexer/v1/compaction/status - private static final String COMPACTION_COORDINATOR_PATH = "/druid/coordinator/v1/compaction"; - private static final String COMPACTION_OVERLORD_PATH = "/druid/indexer/v1/compaction"; - // This path is used to check if the managment proxy is enabled, it simply returns {"enabled":true} private static final String ENABLED_PATH = "/proxy/enabled"; @@ -82,7 +75,6 @@ public class AsyncManagementForwardingServlet extends AsyncProxyServlet private final DruidLeaderSelector coordLeaderSelector; private final DruidLeaderSelector overlordLeaderSelector; private final AuthorizerMapper authorizerMapper; - private final CompactionSupervisorsConfig compactionSupervisorsConfig; @Inject public AsyncManagementForwardingServlet( @@ -91,7 +83,6 @@ public AsyncManagementForwardingServlet( @Global DruidHttpClientConfig httpClientConfig, @Coordinator DruidLeaderSelector coordLeaderSelector, @IndexingService DruidLeaderSelector overlordLeaderSelector, - CompactionSupervisorsConfig compactionSupervisorsConfig, AuthorizerMapper authorizerMapper ) { @@ -100,7 +91,6 @@ public AsyncManagementForwardingServlet( this.httpClientConfig = httpClientConfig; this.coordLeaderSelector = coordLeaderSelector; this.overlordLeaderSelector = overlordLeaderSelector; - this.compactionSupervisorsConfig = compactionSupervisorsConfig; this.authorizerMapper = authorizerMapper; } @@ -109,15 +99,7 @@ protected void service(HttpServletRequest request, HttpServletResponse response) { String currentLeader; String requestURI = StringUtils.toLowerCase(request.getRequestURI()); - if (compactionSupervisorsConfig.isEnabled() - && requestURI.startsWith(COMPACTION_COORDINATOR_PATH)) { - // If Compaction Scheduler is enabled, compaction APIs must be forwarded to the Overlord - currentLeader = overlordLeaderSelector.getCurrentLeader(); - request.setAttribute( - MODIFIED_PATH_ATTRIBUTE, - StringUtils.replace(request.getRequestURI(), COMPACTION_COORDINATOR_PATH, COMPACTION_OVERLORD_PATH) - ); - } else if (requestURI.startsWith(STANDARD_COORDINATOR_BASE_PATH)) { + if (requestURI.startsWith(STANDARD_COORDINATOR_BASE_PATH)) { currentLeader = coordLeaderSelector.getCurrentLeader(); } else if (requestURI.startsWith(STANDARD_OVERLORD_BASE_PATH)) { currentLeader = overlordLeaderSelector.getCurrentLeader(); 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 3d7eab25ab9b..91cdbfa3f812 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 @@ -36,6 +36,7 @@ import org.apache.druid.metadata.LockFilterPolicy; import org.apache.druid.rpc.ServiceRetryPolicy; import org.apache.druid.rpc.indexing.OverlordClient; +import org.apache.druid.server.coordinator.AutoCompactionSnapshot; import org.apache.druid.server.coordinator.ClusterCompactionConfig; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; import org.apache.druid.server.coordinator.DruidCompactionConfig; @@ -286,6 +287,24 @@ public ListenableFuture> getWorkers() throw new UnsupportedOperationException(); } + @Override + public ListenableFuture> getCompactionSnapshots(@Nullable String dataSource) + { + throw new UnsupportedOperationException(); + } + + @Override + public ListenableFuture getBytesAwaitingCompaction(String dataSource) + { + throw new UnsupportedOperationException(); + } + + @Override + public ListenableFuture isCompactionSupervisorEnabled() + { + throw new UnsupportedOperationException(); + } + @Override public OverlordClient withRetryPolicy(ServiceRetryPolicy retryPolicy) { diff --git a/server/src/main/java/org/apache/druid/server/compaction/Table.java b/server/src/main/java/org/apache/druid/server/compaction/Table.java index 9e48818f6499..5a620dacc8d9 100644 --- a/server/src/main/java/org/apache/druid/server/compaction/Table.java +++ b/server/src/main/java/org/apache/druid/server/compaction/Table.java @@ -28,6 +28,7 @@ /** * A simple table POJO with any number of rows and specified column names. + * Used in {@link CompactionSimulateResult}. */ public class Table { diff --git a/server/src/main/java/org/apache/druid/server/coordinator/AutoCompactionSnapshot.java b/server/src/main/java/org/apache/druid/server/coordinator/AutoCompactionSnapshot.java index 47d6671e0dd6..d45db24475d2 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/AutoCompactionSnapshot.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/AutoCompactionSnapshot.java @@ -66,17 +66,17 @@ public static Builder builder(String dataSource) @JsonCreator public AutoCompactionSnapshot( - @JsonProperty @NotNull String dataSource, - @JsonProperty @NotNull AutoCompactionScheduleStatus scheduleStatus, - @JsonProperty long bytesAwaitingCompaction, - @JsonProperty long bytesCompacted, - @JsonProperty long bytesSkipped, - @JsonProperty long segmentCountAwaitingCompaction, - @JsonProperty long segmentCountCompacted, - @JsonProperty long segmentCountSkipped, - @JsonProperty long intervalCountAwaitingCompaction, - @JsonProperty long intervalCountCompacted, - @JsonProperty long intervalCountSkipped + @JsonProperty("dataSource") @NotNull String dataSource, + @JsonProperty("scheduleStatus") @NotNull AutoCompactionScheduleStatus scheduleStatus, + @JsonProperty("bytesAwaitingCompaction") long bytesAwaitingCompaction, + @JsonProperty("bytesCompacted") long bytesCompacted, + @JsonProperty("bytesSkipped") long bytesSkipped, + @JsonProperty("segmentCountAwaitingCompaction") long segmentCountAwaitingCompaction, + @JsonProperty("segmentCountCompacted") long segmentCountCompacted, + @JsonProperty("segmentCountSkipped") long segmentCountSkipped, + @JsonProperty("intervalCountAwaitingCompaction") long intervalCountAwaitingCompaction, + @JsonProperty("intervalCountCompacted") long intervalCountCompacted, + @JsonProperty("intervalCountSkipped") long intervalCountSkipped ) { this.dataSource = dataSource; diff --git a/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java b/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java index 1c53fb9adc7e..6f907354f081 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java @@ -35,6 +35,7 @@ import org.apache.druid.client.ImmutableDruidDataSource; import org.apache.druid.client.ServerInventoryView; import org.apache.druid.client.coordinator.Coordinator; +import org.apache.druid.common.guava.FutureUtils; import org.apache.druid.curator.discovery.ServiceAnnouncer; import org.apache.druid.discovery.DruidLeaderSelector; import org.apache.druid.guice.ManageLifecycle; @@ -160,8 +161,6 @@ public class DruidCoordinator @Nullable private final CoordinatorSegmentMetadataCache coordinatorSegmentMetadataCache; private final CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig; - private final CompactionSupervisorsConfig compactionSupervisorsConfig; - private volatile boolean started = false; @@ -206,8 +205,7 @@ public DruidCoordinator( @Coordinator DruidLeaderSelector coordLeaderSelector, @Nullable CoordinatorSegmentMetadataCache coordinatorSegmentMetadataCache, CentralizedDatasourceSchemaConfig centralizedDatasourceSchemaConfig, - CompactionStatusTracker compactionStatusTracker, - CompactionSupervisorsConfig compactionSupervisorsConfig + CompactionStatusTracker compactionStatusTracker ) { this.config = config; @@ -230,7 +228,6 @@ public DruidCoordinator( this.loadQueueManager = loadQueueManager; this.coordinatorSegmentMetadataCache = coordinatorSegmentMetadataCache; this.centralizedDatasourceSchemaConfig = centralizedDatasourceSchemaConfig; - this.compactionSupervisorsConfig = compactionSupervisorsConfig; } public boolean isLeader() @@ -443,7 +440,7 @@ private Map> computeUnderReplicated( } } - public void becomeLeader() + private void becomeLeader() { synchronized (lock) { if (!started) { @@ -535,7 +532,7 @@ public void becomeLeader() } } - public void stopBeingLeader() + private void stopBeingLeader() { synchronized (lock) { @@ -553,6 +550,20 @@ public void stopBeingLeader() } } + /** + * Check if compaction supervisors are enabled on the Overlord. + */ + private boolean isCompactionSupervisorEnabled() + { + try { + return FutureUtils.getUnchecked(overlordClient.isCompactionSupervisorEnabled(), true); + } + catch (Exception e) { + // The Overlord is probably on an older version, assume that compaction supervisor is not enabled + return false; + } + } + @GuardedBy("lock") private ScheduledExecutorService getOrCreateDutyGroupExecutor(String dutyGroup) { @@ -803,13 +814,13 @@ public void run() */ private boolean shouldSkipAutoCompactDuty(CoordinatorDuty duty) { - final boolean shouldSkipDuty = compactionSupervisorsConfig.isEnabled() - && duty instanceof CompactSegments - && !COMPACT_SEGMENTS_DUTIES_DUTY_GROUP.equals(dutyGroupName); + final boolean shouldSkipDuty = duty instanceof CompactSegments + && !COMPACT_SEGMENTS_DUTIES_DUTY_GROUP.equals(dutyGroupName) + && isCompactionSupervisorEnabled(); if (shouldSkipDuty) { log.warn( "Skipping Compact Segments duty in group[%s] since compaction" - + " scheduler is already running on Overlord.", + + " supervisors are already running on Overlord.", dutyGroupName ); } 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 6baa3e08661d..de7529f8d659 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 @@ -439,7 +439,6 @@ private int submitCompactionTasks( } final String dataSourceName = entry.getDataSource(); - LOG.info("Compacting version[%s].", entry.getFirst().getVersion()); // As these segments will be compacted, we will aggregate the statistic to the Compacted statistics currentRunAutoCompactionSnapshotBuilders diff --git a/server/src/main/java/org/apache/druid/server/http/CoordinatorCompactionResource.java b/server/src/main/java/org/apache/druid/server/http/CoordinatorCompactionResource.java index cd5e9ce8d76a..015a27435cf8 100644 --- a/server/src/main/java/org/apache/druid/server/http/CoordinatorCompactionResource.java +++ b/server/src/main/java/org/apache/druid/server/http/CoordinatorCompactionResource.java @@ -22,11 +22,14 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; +import com.google.common.util.concurrent.ListenableFuture; import com.google.inject.Inject; import com.sun.jersey.spi.container.ResourceFilters; +import org.apache.druid.common.guava.FutureUtils; +import org.apache.druid.rpc.HttpResponseException; +import org.apache.druid.rpc.indexing.OverlordClient; import org.apache.druid.server.coordinator.AutoCompactionSnapshot; import org.apache.druid.server.coordinator.ClusterCompactionConfig; -import org.apache.druid.server.coordinator.CompactionSupervisorsConfig; import org.apache.druid.server.coordinator.DruidCoordinator; import org.apache.druid.server.http.security.ConfigResourceFilter; import org.apache.druid.server.http.security.StateResourceFilter; @@ -45,16 +48,16 @@ public class CoordinatorCompactionResource { private final DruidCoordinator coordinator; - private final CompactionSupervisorsConfig compactionSupervisorsConfig; + private final OverlordClient overlordClient; @Inject public CoordinatorCompactionResource( DruidCoordinator coordinator, - CompactionSupervisorsConfig compactionSupervisorsConfig + OverlordClient overlordClient ) { this.coordinator = coordinator; - this.compactionSupervisorsConfig = compactionSupervisorsConfig; + this.overlordClient = overlordClient; } /** @@ -78,19 +81,19 @@ public Response getCompactionProgress( @QueryParam("dataSource") String dataSource ) { - if (compactionSupervisorsConfig.isEnabled()) { - buildErrorResponseWhenRunningAsSupervisor(); - } - if (dataSource == null || dataSource.isEmpty()) { return Response.status(Response.Status.BAD_REQUEST) .entity(ImmutableMap.of("error", "No DataSource specified")) .build(); } + if (isCompactionSupervisorEnabled()) { + return buildResponse(overlordClient.getBytesAwaitingCompaction(dataSource)); + } + final AutoCompactionSnapshot snapshot = coordinator.getAutoCompactionSnapshotForDataSource(dataSource); if (snapshot == null) { - return Response.status(Response.Status.NOT_FOUND).entity(ImmutableMap.of("error", "unknown dataSource")).build(); + return Response.status(Response.Status.NOT_FOUND).entity(ImmutableMap.of("error", "Unknown DataSource")).build(); } else { return Response.ok(ImmutableMap.of("remainingSegmentSize", snapshot.getBytesAwaitingCompaction())).build(); } @@ -104,8 +107,8 @@ public Response getCompactionSnapshotForDataSource( @QueryParam("dataSource") String dataSource ) { - if (compactionSupervisorsConfig.isEnabled()) { - return buildErrorResponseWhenRunningAsSupervisor(); + if (isCompactionSupervisorEnabled()) { + return buildResponse(overlordClient.getCompactionSnapshots(dataSource)); } final Collection snapshots; @@ -114,7 +117,7 @@ public Response getCompactionSnapshotForDataSource( } else { AutoCompactionSnapshot autoCompactionSnapshot = coordinator.getAutoCompactionSnapshotForDataSource(dataSource); if (autoCompactionSnapshot == null) { - return Response.status(Response.Status.NOT_FOUND).entity(ImmutableMap.of("error", "unknown dataSource")).build(); + return Response.status(Response.Status.NOT_FOUND).entity(ImmutableMap.of("error", "Unknown DataSource")).build(); } snapshots = ImmutableList.of(autoCompactionSnapshot); } @@ -125,27 +128,45 @@ public Response getCompactionSnapshotForDataSource( @Path("/simulate") @Consumes(MediaType.APPLICATION_JSON) @ResourceFilters(StateResourceFilter.class) - public Response simulateClusterCompactionConfigUpdate( + public Response simulateWithClusterConfigUpdate( ClusterCompactionConfig updatePayload ) { - if (compactionSupervisorsConfig.isEnabled()) { - return buildErrorResponseWhenRunningAsSupervisor(); - } - return Response.ok().entity( coordinator.simulateRunWithConfigUpdate(updatePayload) ).build(); } - private Response buildErrorResponseWhenRunningAsSupervisor() + private Response buildResponse(ListenableFuture future) { - return Response.status(Response.Status.SERVICE_UNAVAILABLE).entity( - ImmutableMap.of( - "error", - "Compaction has been disabled on the Coordinator." - + " Use Overlord APIs to fetch compaction status." - ) - ).build(); + try { + return Response.ok(FutureUtils.getUnchecked(future, true)).build(); + } + catch (Exception e) { + if (e.getCause() instanceof HttpResponseException) { + final HttpResponseException cause = (HttpResponseException) e.getCause(); + return Response.status(cause.getResponse().getStatus().getCode()) + .entity(cause.getResponse().getContent()) + .build(); + } else { + return Response.status(Response.Status.INTERNAL_SERVER_ERROR) + .entity(ImmutableMap.of("error", e.getMessage())) + .build(); + } + } + } + + /** + * Check if compaction supervisors are enabled on the Overlord. + */ + private boolean isCompactionSupervisorEnabled() + { + try { + return FutureUtils.getUnchecked(overlordClient.isCompactionSupervisorEnabled(), true); + } + catch (Exception e) { + // Overlord is probably on an older version, assume that compaction supervisor is not enabled + return false; + } } } diff --git a/server/src/test/java/org/apache/druid/client/indexing/NoopOverlordClient.java b/server/src/test/java/org/apache/druid/client/indexing/NoopOverlordClient.java index 10ebeb53af26..c73ff1ca0599 100644 --- a/server/src/test/java/org/apache/druid/client/indexing/NoopOverlordClient.java +++ b/server/src/test/java/org/apache/druid/client/indexing/NoopOverlordClient.java @@ -28,6 +28,7 @@ import org.apache.druid.metadata.LockFilterPolicy; import org.apache.druid.rpc.ServiceRetryPolicy; import org.apache.druid.rpc.indexing.OverlordClient; +import org.apache.druid.server.coordinator.AutoCompactionSnapshot; import org.joda.time.Interval; import javax.annotation.Nullable; @@ -122,6 +123,24 @@ public ListenableFuture getTotalWorkerCapacity( throw new UnsupportedOperationException(); } + @Override + public ListenableFuture> getCompactionSnapshots(@Nullable String dataSource) + { + throw new UnsupportedOperationException(); + } + + @Override + public ListenableFuture getBytesAwaitingCompaction(String dataSource) + { + throw new UnsupportedOperationException(); + } + + @Override + public ListenableFuture isCompactionSupervisorEnabled() + { + throw new UnsupportedOperationException(); + } + @Override public OverlordClient withRetryPolicy(ServiceRetryPolicy retryPolicy) { diff --git a/server/src/test/java/org/apache/druid/rpc/indexing/OverlordClientImplTest.java b/server/src/test/java/org/apache/druid/rpc/indexing/OverlordClientImplTest.java index 5f5837462667..18e7e2769500 100644 --- a/server/src/test/java/org/apache/druid/rpc/indexing/OverlordClientImplTest.java +++ b/server/src/test/java/org/apache/druid/rpc/indexing/OverlordClientImplTest.java @@ -47,6 +47,7 @@ import org.apache.druid.rpc.HttpResponseException; import org.apache.druid.rpc.MockServiceClient; import org.apache.druid.rpc.RequestBuilder; +import org.apache.druid.server.coordinator.AutoCompactionSnapshot; import org.hamcrest.CoreMatchers; import org.hamcrest.MatcherAssert; import org.jboss.netty.handler.codec.http.DefaultHttpResponse; @@ -63,6 +64,7 @@ import javax.ws.rs.core.MediaType; import java.net.URI; import java.nio.charset.StandardCharsets; +import java.util.Arrays; import java.util.Collections; import java.util.List; import java.util.Map; @@ -451,4 +453,82 @@ public void test_taskPayload() throws ExecutionException, InterruptedException, overlordClient.taskPayload(taskID).get().getPayload() ); } + + @Test + public void test_isCompactionSupervisorEnabled() + throws JsonProcessingException, ExecutionException, InterruptedException + { + serviceClient.expectAndRespond( + new RequestBuilder(HttpMethod.GET, "/druid/indexer/v1/compaction/isSupervisorEnabled"), + HttpResponseStatus.OK, + Collections.emptyMap(), + DefaultObjectMapper.INSTANCE.writeValueAsBytes(false) + ); + + Assert.assertFalse(overlordClient.isCompactionSupervisorEnabled().get()); + } + + @Test + public void test_getCompactionSnapshots_nullDataSource() + throws JsonProcessingException, ExecutionException, InterruptedException + { + final List compactionSnapshots = Arrays.asList( + AutoCompactionSnapshot.builder("ds1") + .withStatus(AutoCompactionSnapshot.AutoCompactionScheduleStatus.RUNNING) + .build(), + AutoCompactionSnapshot.builder("ds2") + .withStatus(AutoCompactionSnapshot.AutoCompactionScheduleStatus.RUNNING) + .build() + ); + serviceClient.expectAndRespond( + new RequestBuilder(HttpMethod.GET, "/druid/indexer/v1/compaction/status"), + HttpResponseStatus.OK, + Collections.emptyMap(), + DefaultObjectMapper.INSTANCE.writeValueAsBytes(compactionSnapshots) + ); + + Assert.assertEquals( + compactionSnapshots, + overlordClient.getCompactionSnapshots(null).get() + ); + } + + @Test + public void test_getCompactionSnapshots_nonNullDataSource() + throws JsonProcessingException, ExecutionException, InterruptedException + { + final List compactionSnapshots = Collections.singletonList( + AutoCompactionSnapshot.builder("ds1") + .withStatus(AutoCompactionSnapshot.AutoCompactionScheduleStatus.RUNNING) + .build() + ); + serviceClient.expectAndRespond( + new RequestBuilder(HttpMethod.GET, "/druid/indexer/v1/compaction/status?dataSource=ds1"), + HttpResponseStatus.OK, + Collections.emptyMap(), + DefaultObjectMapper.INSTANCE.writeValueAsBytes(compactionSnapshots) + ); + + Assert.assertEquals( + compactionSnapshots, + overlordClient.getCompactionSnapshots("ds1").get() + ); + } + + @Test + public void test_getBytesAwaitingCompaction() + throws JsonProcessingException, ExecutionException, InterruptedException + { + serviceClient.expectAndRespond( + new RequestBuilder(HttpMethod.GET, "/druid/indexer/v1/compaction/progress?dataSource=ds1"), + HttpResponseStatus.OK, + Collections.emptyMap(), + DefaultObjectMapper.INSTANCE.writeValueAsBytes(100_000L) + ); + + Assert.assertEquals( + 100_000L, + overlordClient.getBytesAwaitingCompaction("ds1").get().longValue() + ); + } } diff --git a/server/src/test/java/org/apache/druid/server/AsyncManagementForwardingServletTest.java b/server/src/test/java/org/apache/druid/server/AsyncManagementForwardingServletTest.java index 62053cd555ff..ba7c78b99b0a 100644 --- a/server/src/test/java/org/apache/druid/server/AsyncManagementForwardingServletTest.java +++ b/server/src/test/java/org/apache/druid/server/AsyncManagementForwardingServletTest.java @@ -35,7 +35,6 @@ import org.apache.druid.guice.http.DruidHttpClientConfig; import org.apache.druid.initialization.Initialization; import org.apache.druid.java.util.common.StringUtils; -import org.apache.druid.server.coordinator.CompactionSupervisorsConfig; import org.apache.druid.server.initialization.BaseJettyTest; import org.apache.druid.server.initialization.ServerConfig; import org.apache.druid.server.initialization.jetty.JettyServerInitUtils; @@ -77,7 +76,6 @@ public class AsyncManagementForwardingServletTest extends BaseJettyTest private static int coordinatorPort; private static int overlordPort; private static boolean isValidLeader; - private static boolean isCompactionSchedulerEnabled; private Server coordinator; private Server overlord; @@ -117,7 +115,6 @@ public void setup() throws Exception coordinator.start(); overlord.start(); isValidLeader = true; - isCompactionSchedulerEnabled = false; } @After @@ -261,47 +258,6 @@ public void testCoordinatorProxySegments() throws Exception Assert.assertFalse("overlord called", OVERLORD_EXPECTED_REQUEST.called); } - @Test - public void testCoordinatorCompactionStatus() throws Exception - { - isCompactionSchedulerEnabled = false; - - COORDINATOR_EXPECTED_REQUEST.path = "/druid/coordinator/v1/compaction/status"; - COORDINATOR_EXPECTED_REQUEST.method = "GET"; - COORDINATOR_EXPECTED_REQUEST.headers = ImmutableMap.of("Authorization", "Basic bXl1c2VyOm15cGFzc3dvcmQ="); - - HttpURLConnection connection = ((HttpURLConnection) - new URL(StringUtils.format("http://localhost:%d/druid/coordinator/v1/compaction/status", port)) - .openConnection()); - connection.setRequestMethod(COORDINATOR_EXPECTED_REQUEST.method); - - COORDINATOR_EXPECTED_REQUEST.headers.forEach(connection::setRequestProperty); - - Assert.assertEquals(200, connection.getResponseCode()); - Assert.assertTrue("coordinator called", COORDINATOR_EXPECTED_REQUEST.called); - Assert.assertFalse("overlord called", OVERLORD_EXPECTED_REQUEST.called); - } - - @Test - public void testOverlordCompactionStatus() throws Exception - { - isCompactionSchedulerEnabled = true; - - OVERLORD_EXPECTED_REQUEST.path = "/druid/indexer/v1/compaction/status"; - OVERLORD_EXPECTED_REQUEST.method = "GET"; - OVERLORD_EXPECTED_REQUEST.headers = ImmutableMap.of("Authorization", "Basic bXl1c2VyOm15cGFzc3dvcmQ="); - - HttpURLConnection connection = ((HttpURLConnection) - new URL(StringUtils.format("http://localhost:%d/druid/coordinator/v1/compaction/status", port)) - .openConnection()); - connection.setRequestMethod("GET"); - OVERLORD_EXPECTED_REQUEST.headers.forEach(connection::setRequestProperty); - - Assert.assertEquals(200, connection.getResponseCode()); - Assert.assertFalse("coordinator called", COORDINATOR_EXPECTED_REQUEST.called); - Assert.assertTrue("overlord called", OVERLORD_EXPECTED_REQUEST.called); - } - @Test public void testOverlordPostTask() throws Exception { @@ -540,13 +496,6 @@ public String getCurrentLeader() injector.getInstance(DruidHttpClientConfig.class), coordinatorLeaderSelector, overlordLeaderSelector, - new CompactionSupervisorsConfig(false) { - @Override - public boolean isEnabled() - { - return isCompactionSchedulerEnabled; - } - }, new AuthorizerMapper(ImmutableMap.of("allowAll", new AllowAllAuthorizer())) ) ); diff --git a/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorTest.java b/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorTest.java index 6f6e3b4ad74c..1ebeb991ccbb 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorTest.java @@ -192,8 +192,7 @@ public void setUp() throws Exception new TestDruidLeaderSelector(), null, CentralizedDatasourceSchemaConfig.create(), - new CompactionStatusTracker(OBJECT_MAPPER), - new CompactionSupervisorsConfig(true) + new CompactionStatusTracker(OBJECT_MAPPER) ); } @@ -624,8 +623,7 @@ public void testCompactSegmentsDutyWhenCustomDutyGroupEmpty() new TestDruidLeaderSelector(), null, CentralizedDatasourceSchemaConfig.create(), - new CompactionStatusTracker(OBJECT_MAPPER), - CompactionSupervisorsConfig.defaultConfig() + new CompactionStatusTracker(OBJECT_MAPPER) ); // Since CompactSegments is not enabled in Custom Duty Group, then CompactSegments must be created in IndexingServiceDuties List indexingDuties = coordinator.makeIndexingServiceDuties(); @@ -665,8 +663,7 @@ public void testInitializeCompactSegmentsDutyWhenCustomDutyGroupDoesNotContainsC new TestDruidLeaderSelector(), null, CentralizedDatasourceSchemaConfig.create(), - new CompactionStatusTracker(OBJECT_MAPPER), - CompactionSupervisorsConfig.defaultConfig() + new CompactionStatusTracker(OBJECT_MAPPER) ); // Since CompactSegments is not enabled in Custom Duty Group, then CompactSegments must be created in IndexingServiceDuties List indexingDuties = coordinator.makeIndexingServiceDuties(); @@ -706,8 +703,7 @@ public void testInitializeCompactSegmentsDutyWhenCustomDutyGroupContainsCompactS new TestDruidLeaderSelector(), null, CentralizedDatasourceSchemaConfig.create(), - new CompactionStatusTracker(OBJECT_MAPPER), - CompactionSupervisorsConfig.defaultConfig() + new CompactionStatusTracker(OBJECT_MAPPER) ); // Since CompactSegments is enabled in Custom Duty Group, then CompactSegments must not be created in IndexingServiceDuties List indexingDuties = coordinator.makeIndexingServiceDuties(); @@ -812,8 +808,7 @@ public void testCoordinatorCustomDutyGroupsRunAsExpected() throws Exception new TestDruidLeaderSelector(), null, CentralizedDatasourceSchemaConfig.create(), - new CompactionStatusTracker(OBJECT_MAPPER), - CompactionSupervisorsConfig.defaultConfig() + new CompactionStatusTracker(OBJECT_MAPPER) ); coordinator.start(); diff --git a/server/src/test/java/org/apache/druid/server/coordinator/simulate/CoordinatorSimulationBuilder.java b/server/src/test/java/org/apache/druid/server/coordinator/simulate/CoordinatorSimulationBuilder.java index 3cafd645def8..f82b89c68593 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/simulate/CoordinatorSimulationBuilder.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/simulate/CoordinatorSimulationBuilder.java @@ -38,7 +38,6 @@ import org.apache.druid.java.util.metrics.StubServiceEmitter; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; import org.apache.druid.server.compaction.CompactionStatusTracker; -import org.apache.druid.server.coordinator.CompactionSupervisorsConfig; import org.apache.druid.server.coordinator.CoordinatorConfigManager; import org.apache.druid.server.coordinator.CoordinatorDynamicConfig; import org.apache.druid.server.coordinator.DruidCompactionConfig; @@ -212,8 +211,7 @@ public CoordinatorSimulation build() env.leaderSelector, null, CentralizedDatasourceSchemaConfig.create(), - new CompactionStatusTracker(OBJECT_MAPPER), - CompactionSupervisorsConfig.defaultConfig() + new CompactionStatusTracker(OBJECT_MAPPER) ); return new SimulationImpl(coordinator, env); diff --git a/server/src/test/java/org/apache/druid/server/http/CoordinatorCompactionResourceTest.java b/server/src/test/java/org/apache/druid/server/http/CoordinatorCompactionResourceTest.java index 0cd600d25f6c..8c6c747d3a8b 100644 --- a/server/src/test/java/org/apache/druid/server/http/CoordinatorCompactionResourceTest.java +++ b/server/src/test/java/org/apache/druid/server/http/CoordinatorCompactionResourceTest.java @@ -21,8 +21,12 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; +import com.google.common.util.concurrent.Futures; +import com.google.common.util.concurrent.ListenableFuture; +import org.apache.druid.client.indexing.NoopOverlordClient; +import org.apache.druid.rpc.indexing.OverlordClient; +import org.apache.druid.server.compaction.CompactionStatistics; import org.apache.druid.server.coordinator.AutoCompactionSnapshot; -import org.apache.druid.server.coordinator.CompactionSupervisorsConfig; import org.apache.druid.server.coordinator.DruidCoordinator; import org.easymock.EasyMock; import org.junit.After; @@ -30,13 +34,16 @@ import org.junit.Before; import org.junit.Test; +import javax.annotation.Nullable; import javax.ws.rs.core.Response; +import java.util.Collections; +import java.util.List; import java.util.Map; public class CoordinatorCompactionResourceTest { private DruidCoordinator mock; - private final CompactionSupervisorsConfig schedulerConfig = new CompactionSupervisorsConfig(false); + private OverlordClient overlordClient; private final String dataSourceName = "datasource_1"; private final AutoCompactionSnapshot expectedSnapshot = new AutoCompactionSnapshot( dataSourceName, @@ -56,6 +63,14 @@ public class CoordinatorCompactionResourceTest public void setUp() { mock = EasyMock.createStrictMock(DruidCoordinator.class); + overlordClient = new NoopOverlordClient() + { + @Override + public ListenableFuture isCompactionSupervisorEnabled() + { + return Futures.immediateFuture(false); + } + }; } @After @@ -75,7 +90,7 @@ public void testGetCompactionSnapshotForDataSourceWithEmptyQueryParameter() EasyMock.expect(mock.getAutoCompactionSnapshot()).andReturn(expected).once(); EasyMock.replay(mock); - final Response response = new CoordinatorCompactionResource(mock, schedulerConfig) + final Response response = new CoordinatorCompactionResource(mock, overlordClient) .getCompactionSnapshotForDataSource(""); Assert.assertEquals(ImmutableMap.of("latestStatus", expected.values()), response.getEntity()); Assert.assertEquals(200, response.getStatus()); @@ -93,7 +108,7 @@ public void testGetCompactionSnapshotForDataSourceWithNullQueryParameter() EasyMock.expect(mock.getAutoCompactionSnapshot()).andReturn(expected).once(); EasyMock.replay(mock); - final Response response = new CoordinatorCompactionResource(mock, schedulerConfig) + final Response response = new CoordinatorCompactionResource(mock, overlordClient) .getCompactionSnapshotForDataSource(null); Assert.assertEquals(ImmutableMap.of("latestStatus", expected.values()), response.getEntity()); Assert.assertEquals(200, response.getStatus()); @@ -107,7 +122,7 @@ public void testGetCompactionSnapshotForDataSourceWithValidQueryParameter() EasyMock.expect(mock.getAutoCompactionSnapshotForDataSource(dataSourceName)).andReturn(expectedSnapshot).once(); EasyMock.replay(mock); - final Response response = new CoordinatorCompactionResource(mock, schedulerConfig) + final Response response = new CoordinatorCompactionResource(mock, overlordClient) .getCompactionSnapshotForDataSource(dataSourceName); Assert.assertEquals(ImmutableMap.of("latestStatus", ImmutableList.of(expectedSnapshot)), response.getEntity()); Assert.assertEquals(200, response.getStatus()); @@ -121,7 +136,7 @@ public void testGetCompactionSnapshotForDataSourceWithInvalidQueryParameter() EasyMock.expect(mock.getAutoCompactionSnapshotForDataSource(dataSourceName)).andReturn(null).once(); EasyMock.replay(mock); - final Response response = new CoordinatorCompactionResource(mock, schedulerConfig) + final Response response = new CoordinatorCompactionResource(mock, overlordClient) .getCompactionSnapshotForDataSource(dataSourceName); Assert.assertEquals(404, response.getStatus()); } @@ -131,7 +146,7 @@ public void testGetProgressForNullDatasourceReturnsBadRequest() { EasyMock.replay(mock); - final Response response = new CoordinatorCompactionResource(mock, schedulerConfig) + final Response response = new CoordinatorCompactionResource(mock, overlordClient) .getCompactionProgress(null); Assert.assertEquals(Response.Status.BAD_REQUEST.getStatusCode(), response.getStatus()); Assert.assertEquals( @@ -141,19 +156,33 @@ public void testGetProgressForNullDatasourceReturnsBadRequest() } @Test - public void testGetCompactionSnapshotWhenCompactionSchedulerIsEnabled() + public void testGetSnapshotWhenCompactionSupervisorIsEnabled() { EasyMock.replay(mock); - final Response response = new CoordinatorCompactionResource(mock, new CompactionSupervisorsConfig(true)) - .getCompactionSnapshotForDataSource("dummy"); - Assert.assertEquals(503, response.getStatus()); + AutoCompactionSnapshot.Builder snapshotBuilder = AutoCompactionSnapshot.builder(dataSourceName); + snapshotBuilder.incrementCompactedStats(CompactionStatistics.create(100L, 10L, 1L)); + final AutoCompactionSnapshot snapshotFromOverlord = snapshotBuilder.build(); + + overlordClient = new NoopOverlordClient() { + @Override + public ListenableFuture isCompactionSupervisorEnabled() + { + return Futures.immediateFuture(true); + } + + @Override + public ListenableFuture> getCompactionSnapshots(@Nullable String dataSource) + { + return Futures.immediateFuture(Collections.singletonList(snapshotFromOverlord)); + } + }; + + final Response response = new CoordinatorCompactionResource(mock, overlordClient) + .getCompactionSnapshotForDataSource(dataSourceName); + Assert.assertEquals(200, response.getStatus()); Assert.assertEquals( - ImmutableMap.of( - "error", - "Compaction has been disabled on the Coordinator." - + " Use Overlord APIs to fetch compaction status." - ), + Collections.singletonList(snapshotFromOverlord), response.getEntity() ); } diff --git a/services/src/main/java/org/apache/druid/cli/CliCoordinator.java b/services/src/main/java/org/apache/druid/cli/CliCoordinator.java index 2c75b27bb1a8..68923f21f1c1 100644 --- a/services/src/main/java/org/apache/druid/cli/CliCoordinator.java +++ b/services/src/main/java/org/apache/druid/cli/CliCoordinator.java @@ -98,7 +98,6 @@ import org.apache.druid.server.QueryScheduler; import org.apache.druid.server.QuerySchedulerProvider; import org.apache.druid.server.compaction.CompactionStatusTracker; -import org.apache.druid.server.coordinator.CompactionSupervisorsConfig; import org.apache.druid.server.coordinator.CoordinatorConfigManager; import org.apache.druid.server.coordinator.DruidCoordinator; import org.apache.druid.server.coordinator.MetadataManager; @@ -223,11 +222,6 @@ public void configure(Binder binder) JsonConfigProvider.bind(binder, "druid.coordinator.balancer", BalancerStrategyFactory.class); JsonConfigProvider.bind(binder, "druid.coordinator.segment", CoordinatorSegmentWatcherConfig.class); JsonConfigProvider.bind(binder, "druid.coordinator.segmentMetadataCache", SegmentMetadataCacheConfig.class); - JsonConfigProvider.bind( - binder, - CompactionSupervisorsConfig.PROPERTY_PREFIX, - CompactionSupervisorsConfig.class - ); binder.bind(DruidCoordinatorConfig.class); binder.bind(RedirectFilter.class).in(LazySingleton.class); diff --git a/services/src/main/java/org/apache/druid/cli/CliRouter.java b/services/src/main/java/org/apache/druid/cli/CliRouter.java index e6a257d0ba54..af0ca7c6eb24 100644 --- a/services/src/main/java/org/apache/druid/cli/CliRouter.java +++ b/services/src/main/java/org/apache/druid/cli/CliRouter.java @@ -43,7 +43,6 @@ import org.apache.druid.query.lookup.LookupSerdeModule; import org.apache.druid.server.AsyncQueryForwardingServlet; import org.apache.druid.server.NoopQuerySegmentWalker; -import org.apache.druid.server.coordinator.CompactionSupervisorsConfig; import org.apache.druid.server.http.RouterResource; import org.apache.druid.server.http.SelfDiscoveryResource; import org.apache.druid.server.initialization.jetty.JettyServerInitializer; @@ -102,11 +101,6 @@ protected List getModules() JsonConfigProvider.bind(binder, "druid.router", TieredBrokerConfig.class); JsonConfigProvider.bind(binder, "druid.router.avatica.balancer", AvaticaConnectionBalancer.class); JsonConfigProvider.bind(binder, "druid.router.managementProxy", ManagementProxyConfig.class); - JsonConfigProvider.bind( - binder, - CompactionSupervisorsConfig.PROPERTY_PREFIX, - CompactionSupervisorsConfig.class - ); binder.bind(QuerySegmentWalker.class).to(NoopQuerySegmentWalker.class).in(LazySingleton.class); From ab711edeb5663aecce6f6f1eede8b54b3cd9bd33 Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Fri, 30 Aug 2024 12:55:37 +0530 Subject: [PATCH 21/26] Bind CompactionSupervisorConfig in SupervisorModule --- .../apache/druid/guice/SupervisorModule.java | 2 ++ .../indexing/compact/CompactionScheduler.java | 4 ++-- .../compact/OverlordCompactionScheduler.java | 12 +++++------ .../http/OverlordCompactionResource.java | 10 +++++++--- .../OverlordCompactionSchedulerTest.java | 14 ++++++------- ...g.java => CompactionSupervisorConfig.java} | 20 ++++++++----------- .../org/apache/druid/cli/CliOverlord.java | 6 ------ 7 files changed, 32 insertions(+), 36 deletions(-) rename server/src/main/java/org/apache/druid/server/coordinator/{CompactionSupervisorsConfig.java => CompactionSupervisorConfig.java} (71%) diff --git a/indexing-service/src/main/java/org/apache/druid/guice/SupervisorModule.java b/indexing-service/src/main/java/org/apache/druid/guice/SupervisorModule.java index f08f36e0682e..f855459159ac 100644 --- a/indexing-service/src/main/java/org/apache/druid/guice/SupervisorModule.java +++ b/indexing-service/src/main/java/org/apache/druid/guice/SupervisorModule.java @@ -28,6 +28,7 @@ import org.apache.druid.indexing.compact.CompactionSupervisorSpec; import org.apache.druid.indexing.overlord.supervisor.SupervisorStateManagerConfig; import org.apache.druid.initialization.DruidModule; +import org.apache.druid.server.coordinator.CompactionSupervisorConfig; import java.util.List; @@ -37,6 +38,7 @@ public class SupervisorModule implements DruidModule public void configure(Binder binder) { JsonConfigProvider.bind(binder, "druid.supervisor", SupervisorStateManagerConfig.class); + JsonConfigProvider.bind(binder, "druid.supervisor.compaction", CompactionSupervisorConfig.class); } @Override diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionScheduler.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionScheduler.java index ed2bb9ff17e6..83a87b358c91 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionScheduler.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionScheduler.java @@ -23,13 +23,13 @@ import org.apache.druid.server.coordinator.AutoCompactionSnapshot; import org.apache.druid.server.coordinator.ClusterCompactionConfig; import org.apache.druid.server.coordinator.CompactionConfigValidationResult; -import org.apache.druid.server.coordinator.CompactionSupervisorsConfig; +import org.apache.druid.server.coordinator.CompactionSupervisorConfig; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; import java.util.Map; /** - * Compaction scheduler that runs on the Overlord if {@link CompactionSupervisorsConfig} + * Compaction scheduler that runs on the Overlord if {@link CompactionSupervisorConfig} * is enabled. *

* Usage: 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 f4102ce4c142..178e597e61f4 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 @@ -43,7 +43,7 @@ import org.apache.druid.server.coordinator.AutoCompactionSnapshot; import org.apache.druid.server.coordinator.ClusterCompactionConfig; import org.apache.druid.server.coordinator.CompactionConfigValidationResult; -import org.apache.druid.server.coordinator.CompactionSupervisorsConfig; +import org.apache.druid.server.coordinator.CompactionSupervisorConfig; import org.apache.druid.server.coordinator.CoordinatorOverlordServiceConfig; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; import org.apache.druid.server.coordinator.DruidCompactionConfig; @@ -66,7 +66,7 @@ /** * Implementation of {@link CompactionScheduler}. *

- * When {@link CompactionSupervisorsConfig} is enabled, this class performs the + * When {@link CompactionSupervisorConfig} is enabled, this class performs the * following responsibilities on the leader Overlord: *

    *
  • Poll segments from metadata
  • @@ -88,7 +88,7 @@ public class OverlordCompactionScheduler implements CompactionScheduler private final ServiceEmitter emitter; private final TaskMaster taskMaster; - private final CompactionSupervisorsConfig schedulerConfig; + private final CompactionSupervisorConfig supervisorConfig; private final Supplier compactionConfigSupplier; private final ConcurrentHashMap activeDatasourceConfigs; @@ -124,7 +124,7 @@ public OverlordCompactionScheduler( SegmentsMetadataManager segmentManager, Supplier compactionConfigSupplier, CompactionStatusTracker statusTracker, - CompactionSupervisorsConfig schedulerConfig, + CompactionSupervisorConfig supervisorConfig, CoordinatorOverlordServiceConfig coordinatorOverlordServiceConfig, ScheduledExecutorFactory executorFactory, ServiceEmitter emitter, @@ -134,7 +134,7 @@ public OverlordCompactionScheduler( this.segmentManager = segmentManager; this.emitter = emitter; this.taskMaster = taskMaster; - this.schedulerConfig = schedulerConfig; + this.supervisorConfig = supervisorConfig; this.compactionConfigSupplier = compactionConfigSupplier; this.executor = executorFactory.create(1, "CompactionScheduler-%s"); @@ -249,7 +249,7 @@ private synchronized void cleanupState() private boolean isEnabled() { - return schedulerConfig.isEnabled(); + return supervisorConfig.isEnabled(); } private synchronized void scheduledRun() diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordCompactionResource.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordCompactionResource.java index 77fc2e916cc9..9f21f2ec4c19 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordCompactionResource.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordCompactionResource.java @@ -25,6 +25,7 @@ import org.apache.druid.indexing.compact.CompactionScheduler; import org.apache.druid.server.coordinator.AutoCompactionSnapshot; import org.apache.druid.server.coordinator.ClusterCompactionConfig; +import org.apache.druid.server.coordinator.CompactionSupervisorConfig; import org.apache.druid.server.http.security.StateResourceFilter; import javax.ws.rs.Consumes; @@ -46,13 +47,16 @@ public class OverlordCompactionResource { private final CompactionScheduler scheduler; + private final CompactionSupervisorConfig supervisorConfig; @Inject public OverlordCompactionResource( + CompactionSupervisorConfig supervisorConfig, CompactionScheduler scheduler ) { this.scheduler = scheduler; + this.supervisorConfig = supervisorConfig; } @GET @@ -61,7 +65,7 @@ public OverlordCompactionResource( @ResourceFilters(StateResourceFilter.class) public Response isCompactionSupervisorEnabled() { - return Response.ok(scheduler.isRunning()).build(); + return Response.ok(supervisorConfig.isEnabled()).build(); } @GET @@ -72,7 +76,7 @@ public Response getCompactionProgress( @QueryParam("dataSource") String dataSource ) { - if (!scheduler.isRunning()) { + if (!supervisorConfig.isEnabled()) { return buildErrorResponseIfSchedulerDisabled(); } @@ -101,7 +105,7 @@ public Response getCompactionSnapshots( @QueryParam("dataSource") String dataSource ) { - if (!scheduler.isRunning()) { + if (!supervisorConfig.isEnabled()) { return buildErrorResponseIfSchedulerDisabled(); } 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 92682e7b0515..0496049e9e8c 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 @@ -50,7 +50,7 @@ import org.apache.druid.server.coordinator.AutoCompactionSnapshot; import org.apache.druid.server.coordinator.ClusterCompactionConfig; import org.apache.druid.server.coordinator.CompactionConfigValidationResult; -import org.apache.druid.server.coordinator.CompactionSupervisorsConfig; +import org.apache.druid.server.coordinator.CompactionSupervisorConfig; import org.apache.druid.server.coordinator.CoordinatorOverlordServiceConfig; import org.apache.druid.server.coordinator.CreateDataSegments; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; @@ -89,7 +89,7 @@ public class OverlordCompactionSchedulerTest ); } - private CompactionSupervisorsConfig schedulerConfig; + private CompactionSupervisorConfig supervisorConfig; private DruidCompactionConfig compactionConfig; private CoordinatorOverlordServiceConfig coordinatorOverlordServiceConfig; @@ -119,7 +119,7 @@ public void setUp() serviceEmitter = new StubServiceEmitter(); segmentsMetadataManager = new TestSegmentsMetadataManager(); - schedulerConfig = new CompactionSupervisorsConfig(true); + supervisorConfig = new CompactionSupervisorConfig(true); compactionConfig = DruidCompactionConfig.empty(); coordinatorOverlordServiceConfig = new CoordinatorOverlordServiceConfig(false, null); @@ -137,7 +137,7 @@ private void initScheduler() segmentsMetadataManager, () -> compactionConfig, new CompactionStatusTracker(OBJECT_MAPPER), - schedulerConfig, + supervisorConfig, coordinatorOverlordServiceConfig, (nameFormat, numThreads) -> new WrappingScheduledExecutorService("test", executor, false), serviceEmitter, @@ -148,7 +148,7 @@ private void initScheduler() @Test public void testStartStopWhenSchedulerIsEnabled() { - schedulerConfig = new CompactionSupervisorsConfig(true); + supervisorConfig = new CompactionSupervisorConfig(true); Assert.assertFalse(scheduler.isRunning()); scheduler.start(); @@ -167,7 +167,7 @@ public void testStartStopWhenSchedulerIsEnabled() @Test public void testStartStopWhenScheduledIsDisabled() { - schedulerConfig = new CompactionSupervisorsConfig(false); + supervisorConfig = new CompactionSupervisorConfig(false); initScheduler(); Assert.assertFalse(scheduler.isRunning()); @@ -182,7 +182,7 @@ public void testStartStopWhenScheduledIsDisabled() @Test public void testSegmentsAreNotPolledWhenSchedulerIsDisabled() { - schedulerConfig = new CompactionSupervisorsConfig(false); + supervisorConfig = new CompactionSupervisorConfig(false); initScheduler(); verifySegmentPolling(false); diff --git a/server/src/main/java/org/apache/druid/server/coordinator/CompactionSupervisorsConfig.java b/server/src/main/java/org/apache/druid/server/coordinator/CompactionSupervisorConfig.java similarity index 71% rename from server/src/main/java/org/apache/druid/server/coordinator/CompactionSupervisorsConfig.java rename to server/src/main/java/org/apache/druid/server/coordinator/CompactionSupervisorConfig.java index cf4542afd5cb..e738a3e7f0bb 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/CompactionSupervisorsConfig.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/CompactionSupervisorConfig.java @@ -27,28 +27,24 @@ import java.util.Objects; /** - * This config must be bound on the following services: - *
      - *
    • CliOverlord - to run the compaction supervisors on the Overlord
    • - *
    • CliOverlord - to prevent the Coordinator from running auto-compaction duty
    • - *
    • CliRouter - to allow the Router to forward compaction stats requests to the Overlord
    • - *
    + * This config must be bound on CliOverlord to enable running compaction supervisors + * on the Overlord. When compaction supervisors are enabled, the Coordinator + * does not run auto-compact duty. */ -public class CompactionSupervisorsConfig +public class CompactionSupervisorConfig { - public static final String PROPERTY_PREFIX = "druid.compaction.supervisors"; - private static final CompactionSupervisorsConfig DEFAULT = new CompactionSupervisorsConfig(null); + private static final CompactionSupervisorConfig DEFAULT = new CompactionSupervisorConfig(null); @JsonProperty private final boolean enabled; - public static CompactionSupervisorsConfig defaultConfig() + public static CompactionSupervisorConfig defaultConfig() { return DEFAULT; } @JsonCreator - public CompactionSupervisorsConfig( + public CompactionSupervisorConfig( @JsonProperty("enabled") @Nullable Boolean enabled ) { @@ -69,7 +65,7 @@ public boolean equals(Object o) if (o == null || getClass() != o.getClass()) { return false; } - CompactionSupervisorsConfig that = (CompactionSupervisorsConfig) o; + CompactionSupervisorConfig that = (CompactionSupervisorConfig) o; return enabled == that.enabled; } diff --git a/services/src/main/java/org/apache/druid/cli/CliOverlord.java b/services/src/main/java/org/apache/druid/cli/CliOverlord.java index 6e81940926bc..52962ddf4c1e 100644 --- a/services/src/main/java/org/apache/druid/cli/CliOverlord.java +++ b/services/src/main/java/org/apache/druid/cli/CliOverlord.java @@ -117,7 +117,6 @@ import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager; import org.apache.druid.segment.realtime.appenderator.DummyForInjectionAppenderatorsManager; import org.apache.druid.server.compaction.CompactionStatusTracker; -import org.apache.druid.server.coordinator.CompactionSupervisorsConfig; import org.apache.druid.server.coordinator.CoordinatorOverlordServiceConfig; import org.apache.druid.server.coordinator.DruidCompactionConfig; import org.apache.druid.server.http.RedirectFilter; @@ -218,11 +217,6 @@ public void configure(Binder binder) binder.bind(SegmentsMetadataManager.class) .toProvider(SegmentsMetadataManagerProvider.class) .in(ManageLifecycle.class); - JsonConfigProvider.bind( - binder, - CompactionSupervisorsConfig.PROPERTY_PREFIX, - CompactionSupervisorsConfig.class - ); } JsonConfigProvider.bind(binder, "druid.coordinator.asOverlord", CoordinatorOverlordServiceConfig.class); From bca9810d59a573c30d7284cc47e1c3895e162d76 Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Sat, 31 Aug 2024 23:51:57 +0530 Subject: [PATCH 22/26] Minor cleanup and tests --- .../compact/OverlordCompactionScheduler.java | 12 +- .../compaction/CompactionRunSimulator.java | 19 +- .../CompactionSegmentSearchPolicy.java | 11 + .../server/compaction/CompactionStatus.java | 9 + .../compaction/CompactionStatusTracker.java | 56 +- .../DataSourceCompactibleSegmentIterator.java | 74 +- .../PriorityBasedSegmentSearchPolicy.java | 12 +- .../server/compaction/SegmentsToCompact.java | 23 +- .../DataSourceCompactionConfig.java | 9 + .../NewestSegmentFirstPolicyTest.java | 1514 +++++++---------- .../coordinator/CreateDataSegments.java | 34 +- 11 files changed, 749 insertions(+), 1024 deletions(-) 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 178e597e61f4..e7b6440deb65 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 @@ -175,7 +175,7 @@ public void start() if (isEnabled() && started.compareAndSet(false, true)) { log.info("Starting compaction scheduler."); initState(); - scheduleOnExecutor(this::scheduledRun, SCHEDULE_PERIOD_SECONDS); + scheduleOnExecutor(this::scheduledRun); } } @@ -210,7 +210,9 @@ public CompactionConfigValidationResult validateCompactionConfig(DataSourceCompa @Override public void startCompaction(String dataSourceName, DataSourceCompactionConfig config) { - if (isRunning()) { + // Track active datasources even if scheduler has not started yet because + // SupervisorManager is started before the scheduler + if (isEnabled()) { activeDatasourceConfigs.put(dataSourceName, config); } } @@ -261,7 +263,7 @@ private synchronized void scheduledRun() catch (Exception e) { log.error(e, "Error processing compaction queue. Continuing schedule."); } - scheduleOnExecutor(this::scheduledRun, SCHEDULE_PERIOD_SECONDS); + scheduleOnExecutor(this::scheduledRun); } else { cleanupState(); } @@ -337,7 +339,7 @@ private Map getCurrentDatasourceTimelines() .getUsedSegmentsTimelinesPerDataSource(); } - private void scheduleOnExecutor(Runnable runnable, long delaySeconds) + private void scheduleOnExecutor(Runnable runnable) { executor.schedule( () -> { @@ -348,7 +350,7 @@ private void scheduleOnExecutor(Runnable runnable, long delaySeconds) log.error(t, "Error while executing runnable"); } }, - delaySeconds, + SCHEDULE_PERIOD_SECONDS, TimeUnit.SECONDS ); } 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 91cdbfa3f812..563dab5278c8 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 @@ -19,7 +19,6 @@ package org.apache.druid.server.compaction; -import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; import org.apache.druid.client.indexing.ClientCompactionTaskQuery; @@ -94,15 +93,13 @@ public CompactionSimulateResult simulateRunWithConfig( final CompactionStatusTracker simulationStatusTracker = new CompactionStatusTracker(null) { @Override - public CompactionTaskStatus getLatestTaskStatus(SegmentsToCompact candidates) - { - return statusTracker.getLatestTaskStatus(candidates); - } - - @Override - public ObjectMapper getObjectMapper() + public CompactionStatus computeCompactionStatus( + SegmentsToCompact candidate, + DataSourceCompactionConfig config, + CompactionSegmentSearchPolicy searchPolicy + ) { - return statusTracker.getObjectMapper(); + return statusTracker.computeCompactionStatus(candidate, config, searchPolicy); } @Override @@ -111,7 +108,7 @@ public void onCompactionStatusComputed( DataSourceCompactionConfig config ) { - final CompactionStatus status = candidateSegments.getCompactionStatus(); + final CompactionStatus status = candidateSegments.getCurrentStatus(); if (status.getState() == CompactionStatus.State.COMPLETE) { compactedIntervals.addRow( createRow(candidateSegments, null, null) @@ -131,7 +128,7 @@ public void onCompactionStatusComputed( public void onTaskSubmitted(ClientCompactionTaskQuery taskPayload, SegmentsToCompact candidateSegments) { // Add a row for each task in order of submission - final CompactionStatus status = candidateSegments.getCompactionStatus(); + final CompactionStatus status = candidateSegments.getCurrentStatus(); queuedIntervals.addRow( createRow(candidateSegments, taskPayload.getTuningConfig(), status.getReason()) ); diff --git a/server/src/main/java/org/apache/druid/server/compaction/CompactionSegmentSearchPolicy.java b/server/src/main/java/org/apache/druid/server/compaction/CompactionSegmentSearchPolicy.java index 46cacf844df2..51fd688706a1 100644 --- a/server/src/main/java/org/apache/druid/server/compaction/CompactionSegmentSearchPolicy.java +++ b/server/src/main/java/org/apache/druid/server/compaction/CompactionSegmentSearchPolicy.java @@ -48,4 +48,15 @@ CompactionSegmentIterator createIterator( Map> skipIntervals, CompactionStatusTracker statusTracker ); + + /** + * Checks if the given candidate segments are eligible for compaction in + * the current iteration. A policy implementation may implement this method + * to avoid compacting intervals that do not fulfil some required criteria. + */ + boolean isEligibleForCompaction( + SegmentsToCompact candidateSegments, + CompactionStatus currentCompactionStatus, + CompactionTaskStatus latestTaskStatus + ); } 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 bbb5cd831cc8..186f03a5136e 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 @@ -105,6 +105,15 @@ public State getState() return state; } + @Override + public String toString() + { + return "CompactionStatus{" + + "state=" + state + + ", reason=" + reason + + '}'; + } + private static CompactionStatus incomplete(String reasonFormat, Object... args) { return new CompactionStatus(State.PENDING, StringUtils.format(reasonFormat, args)); 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 5d467302bc96..644d89a4db86 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 @@ -61,11 +61,6 @@ public void stop() datasourceStatuses.clear(); } - public ObjectMapper getObjectMapper() - { - return objectMapper; - } - public void removeDatasource(String datasource) { datasourceStatuses.remove(datasource); @@ -88,6 +83,40 @@ public Set getSubmittedTaskIds() return submittedTaskIdToSegments.keySet(); } + public CompactionStatus computeCompactionStatus( + SegmentsToCompact candidate, + DataSourceCompactionConfig config, + CompactionSegmentSearchPolicy searchPolicy + ) + { + final CompactionStatus compactionStatus = CompactionStatus.compute(candidate, config, objectMapper); + if (compactionStatus.isComplete()) { + return compactionStatus; + } + + // Skip intervals that violate max allowed input segment size + final long inputSegmentSize = config.getInputSegmentSizeBytes(); + if (candidate.getTotalBytes() > inputSegmentSize) { + return CompactionStatus.skipped( + "'inputSegmentSize' exceeded: Total segment size[%d] is larger than allowed inputSegmentSize[%d]", + candidate.getTotalBytes(), inputSegmentSize + ); + } + + // Skip intervals that already have a running task + final CompactionTaskStatus lastTaskStatus = getLatestTaskStatus(candidate); + if (lastTaskStatus != null && lastTaskStatus.getState() == TaskState.RUNNING) { + return CompactionStatus.skipped("Task for interval is already running"); + } + + // Skip intervals that have been filtered out by the policy + if (!searchPolicy.isEligibleForCompaction(candidate, compactionStatus, lastTaskStatus)) { + return CompactionStatus.skipped("Rejected by search policy"); + } + + return compactionStatus; + } + public void onCompactionStatusComputed( SegmentsToCompact candidateSegments, DataSourceCompactionConfig config @@ -100,21 +129,21 @@ public void onCompactionConfigUpdated(DruidCompactionConfig compactionConfig) { final Set compactionEnabledDatasources = new HashSet<>(); if (compactionConfig.getCompactionConfigs() != null) { - compactionConfig.getCompactionConfigs().forEach( - config -> compactionEnabledDatasources.add(config.getDataSource()) - ); + compactionConfig.getCompactionConfigs().forEach(config -> { + getOrComputeDatasourceStatus(config.getDataSource()) + .cleanupStaleTaskStatuses(); + + compactionEnabledDatasources.add(config.getDataSource()); + }); } - // Clean up state for datasources where compaction has been freshly disabled + // Clean up state for datasources where compaction has been disabled final Set allDatasources = new HashSet<>(datasourceStatuses.keySet()); allDatasources.forEach(datasource -> { if (!compactionEnabledDatasources.contains(datasource)) { datasourceStatuses.remove(datasource); } }); - - // Clean up stale task statuses - datasourceStatuses.values().forEach(DatasourceStatus::cleanupStaleTaskStatuses); } public void onTaskSubmitted( @@ -149,6 +178,9 @@ private DatasourceStatus getOrComputeDatasourceStatus(String datasource) return datasourceStatuses.computeIfAbsent(datasource, ds -> new DatasourceStatus()); } + /** + * Contains compaction task status of intervals of a datasource. + */ private static class DatasourceStatus { static final DatasourceStatus EMPTY = new DatasourceStatus(); 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 6b2cb8099f23..d5cb5d80fe6a 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 @@ -22,7 +22,6 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; import com.google.common.collect.Lists; -import org.apache.druid.indexer.TaskState; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.JodaUtils; @@ -69,8 +68,6 @@ public class DataSourceCompactibleSegmentIterator implements CompactionSegmentIt private final CompactionStatusTracker statusTracker; private final PriorityBasedSegmentSearchPolicy searchPolicy; - private final Granularity configuredSegmentGranularity; - private final List compactedSegments = new ArrayList<>(); private final List skippedSegments = new ArrayList<>(); @@ -96,12 +93,6 @@ public DataSourceCompactibleSegmentIterator( this.searchPolicy = searchPolicy; this.queue = new PriorityQueue<>(searchPolicy.getSegmentComparator()); - if (config.getGranularitySpec() == null || config.getGranularitySpec().getSegmentGranularity() == null) { - this.configuredSegmentGranularity = null; - } else { - this.configuredSegmentGranularity = config.getGranularitySpec().getSegmentGranularity(); - } - populateQueue(timeline, skipIntervals); } @@ -110,7 +101,7 @@ private void populateQueue(SegmentTimeline timeline, List skipInterval if (timeline != null) { if (!timeline.isEmpty()) { SegmentTimeline originalTimeline = null; - if (configuredSegmentGranularity != null) { + if (config.getSegmentGranularity() != null) { final Set segments = timeline.findNonOvershadowedObjectsInInterval( Intervals.ETERNITY, Partitions.ONLY_COMPLETE @@ -126,7 +117,7 @@ private void populateQueue(SegmentTimeline timeline, List skipInterval } } if (!partialEternitySegments.isEmpty()) { - SegmentsToCompact candidatesWithStatus = SegmentsToCompact.from(partialEternitySegments).withStatus( + SegmentsToCompact candidatesWithStatus = SegmentsToCompact.from(partialEternitySegments).withCurrentStatus( CompactionStatus.skipped("Segments have partial-eternity intervals") ); skippedSegments.add(candidatesWithStatus); @@ -141,7 +132,7 @@ private void populateQueue(SegmentTimeline timeline, List skipInterval final SegmentTimeline timelineWithConfiguredSegmentGranularity = new SegmentTimeline(); final Map> intervalToPartitionMap = new HashMap<>(); for (DataSegment segment : segments) { - for (Interval interval : configuredSegmentGranularity.getIterable(segment.getInterval())) { + for (Interval interval : config.getSegmentGranularity().getIterable(segment.getInterval())) { intervalToPartitionMap.computeIfAbsent(interval, k -> new HashSet<>()) .add(segment); } @@ -215,13 +206,10 @@ public SegmentsToCompact next() } final SegmentsToCompact entry = queue.poll(); - if (entry == null) { + if (entry == null || entry.isEmpty()) { throw new NoSuchElementException(); } - final List resultSegments = entry.getSegments(); - Preconditions.checkState(!resultSegments.isEmpty(), "Queue entry must not be empty"); - return entry; } @@ -326,67 +314,29 @@ private void findAndEnqueueSegmentsToCompact(CompactibleSegmentIterator compacti continue; } - // Do not compact an interval which comprises of a single tombstone + // Do not compact an interval which contains a single tombstone // If there are multiple tombstones in the interval, we may still want to compact them if (segments.size() == 1 && segments.get(0).isTombstone()) { continue; } final SegmentsToCompact candidates = SegmentsToCompact.from(segments); - final CompactionStatus compactionStatus = computeCompactionStatus(candidates); - final SegmentsToCompact candidatesWithStatus = candidates.withStatus(compactionStatus); + final CompactionStatus compactionStatus + = statusTracker.computeCompactionStatus(candidates, config, searchPolicy); + final SegmentsToCompact candidatesWithStatus = candidates.withCurrentStatus(compactionStatus); statusTracker.onCompactionStatusComputed(candidatesWithStatus, config); if (compactionStatus.isComplete()) { compactedSegments.add(candidatesWithStatus); } else if (compactionStatus.isSkipped()) { skippedSegments.add(candidatesWithStatus); - } else { + } else if (!queuedIntervals.contains(candidates.getUmbrellaInterval())) { queue.add(candidatesWithStatus); - if (configuredSegmentGranularity != null) { - queuedIntervals.add(candidates.getUmbrellaInterval()); - } + queuedIntervals.add(candidates.getUmbrellaInterval()); } } } - private CompactionStatus computeCompactionStatus(SegmentsToCompact candidate) - { - final CompactionStatus compactionStatus - = CompactionStatus.compute(candidate, config, statusTracker.getObjectMapper()); - if (compactionStatus.isComplete()) { - return compactionStatus; - } - - // Skip intervals that violate max allowed input segment size - final long inputSegmentSize = config.getInputSegmentSizeBytes(); - if (candidate.getTotalBytes() > inputSegmentSize) { - return CompactionStatus.skipped( - "'inputSegmentSize' exceeded: Total segment size[%d] is larger than allowed inputSegmentSize[%d]", - candidate.getTotalBytes(), inputSegmentSize - ); - } - - // Skip intervals that are already queued - if (configuredSegmentGranularity != null - && queuedIntervals.contains(candidate.getUmbrellaInterval())) { - return CompactionStatus.skipped("Interval is already queued"); - } - - // Skip intervals that already have a running task - final CompactionTaskStatus lastTaskStatus = statusTracker.getLatestTaskStatus(candidate); - if (lastTaskStatus != null && lastTaskStatus.getState() == TaskState.RUNNING) { - return CompactionStatus.skipped("Task for interval is already running"); - } - - // Skip intervals that have been filtered out by the policy - if (searchPolicy.shouldSkipCompaction(candidate, compactionStatus, lastTaskStatus)) { - return CompactionStatus.skipped("Skipped by policy"); - } - - return compactionStatus; - } - /** * Returns the initial searchInterval which is {@code (timeline.first().start, timeline.last().end - skipOffset)}. */ @@ -402,7 +352,7 @@ private List findInitialSearchInterval( final TimelineObjectHolder first = Preconditions.checkNotNull(timeline.first(), "first"); final TimelineObjectHolder last = Preconditions.checkNotNull(timeline.last(), "last"); final Interval latestSkipInterval = computeLatestSkipInterval( - configuredSegmentGranularity, + config.getSegmentGranularity(), last.getInterval().getEnd(), skipOffset ); @@ -424,7 +374,7 @@ private List findInitialSearchInterval( reason = CompactionStatus.skipped("interval locked by another task"); } - final SegmentsToCompact candidatesWithStatus = candidates.withStatus(reason); + final SegmentsToCompact candidatesWithStatus = candidates.withCurrentStatus(reason); skippedSegments.add(candidatesWithStatus); statusTracker.onCompactionStatusComputed(candidatesWithStatus, config); } diff --git a/server/src/main/java/org/apache/druid/server/compaction/PriorityBasedSegmentSearchPolicy.java b/server/src/main/java/org/apache/druid/server/compaction/PriorityBasedSegmentSearchPolicy.java index 1134297d4fb2..93f9b83a265a 100644 --- a/server/src/main/java/org/apache/druid/server/compaction/PriorityBasedSegmentSearchPolicy.java +++ b/server/src/main/java/org/apache/druid/server/compaction/PriorityBasedSegmentSearchPolicy.java @@ -69,20 +69,14 @@ public CompactionSegmentIterator createIterator( ); } - /** - * Checks if compaction of the given candidate segments should be skipped in - * the current iteration. A concrete policy implementation may override this - * method to avoid compacting intervals that do not fulfil some required criteria. - * - * @return false by default - */ - protected boolean shouldSkipCompaction( + @Override + public boolean isEligibleForCompaction( SegmentsToCompact candidateSegments, CompactionStatus currentCompactionStatus, CompactionTaskStatus latestTaskStatus ) { - return false; + return true; } /** diff --git a/server/src/main/java/org/apache/druid/server/compaction/SegmentsToCompact.java b/server/src/main/java/org/apache/druid/server/compaction/SegmentsToCompact.java index d90d8fceb921..8aff403e35c4 100644 --- a/server/src/main/java/org/apache/druid/server/compaction/SegmentsToCompact.java +++ b/server/src/main/java/org/apache/druid/server/compaction/SegmentsToCompact.java @@ -25,6 +25,7 @@ import org.apache.druid.timeline.DataSegment; import org.joda.time.Interval; +import javax.annotation.Nullable; import java.util.List; import java.util.NoSuchElementException; import java.util.stream.Collectors; @@ -36,11 +37,11 @@ public class SegmentsToCompact { private final List segments; private final Interval umbrellaInterval; - private final String datasource; + private final String dataSource; private final long totalBytes; private final int numIntervals; - private final CompactionStatus compactionStatus; + private final CompactionStatus currentStatus; public static SegmentsToCompact from(List segments) { @@ -51,7 +52,7 @@ public static SegmentsToCompact from(List segments) } } - private SegmentsToCompact(List segments, CompactionStatus status) + private SegmentsToCompact(List segments, @Nullable CompactionStatus currentStatus) { this.segments = segments; this.totalBytes = segments.stream().mapToLong(DataSegment::getSize).sum(); @@ -59,8 +60,8 @@ private SegmentsToCompact(List segments, CompactionStatus status) segments.stream().map(DataSegment::getInterval).collect(Collectors.toList()) ); this.numIntervals = (int) segments.stream().map(DataSegment::getInterval).distinct().count(); - this.datasource = segments.get(0).getDataSource(); - this.compactionStatus = status; + this.dataSource = segments.get(0).getDataSource(); + this.currentStatus = currentStatus; } public List getSegments() @@ -99,7 +100,7 @@ public Interval getUmbrellaInterval() public String getDataSource() { - return datasource; + return dataSource; } public CompactionStatistics getStats() @@ -107,12 +108,13 @@ public CompactionStatistics getStats() return CompactionStatistics.create(totalBytes, size(), numIntervals); } - public CompactionStatus getCompactionStatus() + @Nullable + public CompactionStatus getCurrentStatus() { - return compactionStatus; + return currentStatus; } - public SegmentsToCompact withStatus(CompactionStatus status) + public SegmentsToCompact withCurrentStatus(CompactionStatus status) { return new SegmentsToCompact(this.segments, status); } @@ -121,9 +123,10 @@ public SegmentsToCompact withStatus(CompactionStatus status) public String toString() { return "SegmentsToCompact{" + - "datasource=" + datasource + + "datasource=" + dataSource + ", segments=" + SegmentUtils.commaSeparatedIdentifiers(segments) + ", totalSize=" + totalBytes + + ", currentStatus=" + currentStatus + '}'; } } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/DataSourceCompactionConfig.java b/server/src/main/java/org/apache/druid/server/coordinator/DataSourceCompactionConfig.java index 193b53a5d058..b4b2be780328 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/DataSourceCompactionConfig.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/DataSourceCompactionConfig.java @@ -20,9 +20,11 @@ package org.apache.druid.server.coordinator; import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; import org.apache.druid.indexer.CompactionEngine; +import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.query.aggregation.AggregatorFactory; import org.joda.time.Period; @@ -188,6 +190,13 @@ public CompactionEngine getEngine() return engine; } + @Nullable + @JsonIgnore + public Granularity getSegmentGranularity() + { + return granularitySpec == null ? null : granularitySpec.getSegmentGranularity(); + } + @Override public boolean equals(Object o) { 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 4dadccac9dba..5a056e8357b7 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 @@ -22,7 +22,6 @@ import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.InjectableValues; import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; @@ -34,6 +33,7 @@ import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.granularity.Granularities; +import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.common.granularity.PeriodGranularity; import org.apache.druid.java.util.common.guava.Comparators; import org.apache.druid.math.expr.ExprMacroTable; @@ -57,7 +57,6 @@ import org.apache.druid.timeline.Partitions; import org.apache.druid.timeline.SegmentTimeline; import org.apache.druid.timeline.partition.NumberedShardSpec; -import org.apache.druid.timeline.partition.ShardSpec; import org.apache.druid.timeline.partition.TombstoneShardSpec; import org.apache.druid.utils.Streams; import org.joda.time.DateTimeZone; @@ -68,7 +67,6 @@ import org.junit.Test; import java.util.ArrayList; -import java.util.Arrays; import java.util.Collections; import java.util.Comparator; import java.util.HashMap; @@ -97,18 +95,17 @@ public void setup() @Test public void testLargeOffsetAndSmallSegmentInterval() { - final Period segmentPeriod = new Period("PT1H"); - final CompactionSegmentIterator iterator = policy.createIterator( - ImmutableMap.of(DATA_SOURCE, createCompactionConfig(10000, new Period("P2D"), null)), - ImmutableMap.of( - DATA_SOURCE, - createTimeline( - new SegmentGenerateSpec(Intervals.of("2017-11-16T20:00:00/2017-11-17T04:00:00"), segmentPeriod), - new SegmentGenerateSpec(Intervals.of("2017-11-14T00:00:00/2017-11-16T07:00:00"), segmentPeriod) - ) - ), - Collections.emptyMap(), - statusTracker + final Period segmentPeriod = Period.hours(1); + final CompactionSegmentIterator iterator = createIterator( + configBuilder().withSkipOffsetFromLatest(new Period("P2D")).build(), + createTimeline( + createSegments().forIntervals(8, Granularities.HOUR) + .startingAt("2017-11-16T20:00:00Z") + .withNumPartitions(4), + createSegments().forIntervals(55, Granularities.HOUR) + .startingAt("2017-11-14") + .withNumPartitions(4) + ) ); assertCompactSegmentIntervals( @@ -123,18 +120,17 @@ public void testLargeOffsetAndSmallSegmentInterval() @Test public void testSmallOffsetAndLargeSegmentInterval() { - final Period segmentPeriod = new Period("PT1H"); - final CompactionSegmentIterator iterator = policy.createIterator( - ImmutableMap.of(DATA_SOURCE, createCompactionConfig(10000, new Period("PT1M"), null)), - ImmutableMap.of( - DATA_SOURCE, - createTimeline( - new SegmentGenerateSpec(Intervals.of("2017-11-16T20:00:00/2017-11-17T04:00:00"), segmentPeriod), - new SegmentGenerateSpec(Intervals.of("2017-11-14T00:00:00/2017-11-16T07:00:00"), segmentPeriod) - ) - ), - Collections.emptyMap(), - statusTracker + final Period segmentPeriod = Period.hours(1); + final CompactionSegmentIterator iterator = createIterator( + configBuilder().withSkipOffsetFromLatest(new Period("PT1M")).build(), + createTimeline( + createSegments().forIntervals(8, Granularities.HOUR) + .startingAt("2017-11-16T20:00:00Z") + .withNumPartitions(4), + createSegments().forIntervals(55, Granularities.HOUR) + .startingAt("2017-11-14") + .withNumPartitions(4) + ) ); assertCompactSegmentIntervals( @@ -157,19 +153,17 @@ public void testSmallOffsetAndLargeSegmentInterval() @Test public void testLargeGapInData() { - final Period segmentPeriod = new Period("PT1H"); - final CompactionSegmentIterator iterator = policy.createIterator( - ImmutableMap.of(DATA_SOURCE, createCompactionConfig(10000, new Period("PT1H1M"), null)), - ImmutableMap.of( - DATA_SOURCE, - createTimeline( - new SegmentGenerateSpec(Intervals.of("2017-11-16T20:00:00/2017-11-17T04:00:00"), segmentPeriod), - // larger gap than SegmentCompactionUtil.LOOKUP_PERIOD (1 day) - new SegmentGenerateSpec(Intervals.of("2017-11-14T00:00:00/2017-11-15T07:00:00"), segmentPeriod) - ) - ), - Collections.emptyMap(), - statusTracker + final Period segmentPeriod = Period.hours(1); + final CompactionSegmentIterator iterator = createIterator( + configBuilder().withSkipOffsetFromLatest(new Period("PT1H1M")).build(), + createTimeline( + createSegments().forIntervals(8, Granularities.HOUR) + .startingAt("2017-11-16T20:00:00Z") + .withNumPartitions(4), + createSegments().forIntervals(31, Granularities.HOUR) + .startingAt("2017-11-14") + .withNumPartitions(4) + ) ); assertCompactSegmentIntervals( @@ -192,33 +186,22 @@ public void testLargeGapInData() @Test public void testHugeShard() { - final CompactionSegmentIterator iterator = policy.createIterator( - ImmutableMap.of(DATA_SOURCE, createCompactionConfig(10000, new Period("P1D"), null)), - ImmutableMap.of( - DATA_SOURCE, - createTimeline( - new SegmentGenerateSpec( - Intervals.of("2017-11-17T00:00:00/2017-11-18T03:00:00"), - new Period("PT1H"), - 200, - DEFAULT_NUM_SEGMENTS_PER_SHARD - ), - new SegmentGenerateSpec( - Intervals.of("2017-11-09T00:00:00/2017-11-17T00:00:00"), - new Period("P2D"), - 13000, // larger than target compact segment size - 1 - ), - new SegmentGenerateSpec( - Intervals.of("2017-11-05T00:00:00/2017-11-09T00:00:00"), - new Period("PT1H"), - 200, - DEFAULT_NUM_SEGMENTS_PER_SHARD - ) - ) - ), - Collections.emptyMap(), - statusTracker + final CompactionSegmentIterator iterator = createIterator( + configBuilder().withSkipOffsetFromLatest(Period.days(1)).build(), + createTimeline( + createSegments() + .forIntervals(27, Granularities.HOUR) + .startingAt("2017-11-17") + .withNumPartitions(4), + createSegments() + .forIntervals(4, new PeriodGranularity(Period.days(2), null, null)) + .startingAt("2017-11-09") + .withNumPartitions(1), + createSegments() + .forIntervals(96, Granularities.HOUR) + .startingAt("2017-11-05") + .withNumPartitions(4) + ) ); Interval lastInterval = null; @@ -243,39 +226,22 @@ public void testHugeShard() @Test public void testManySegmentsPerShard() { - final CompactionSegmentIterator iterator = policy.createIterator( - ImmutableMap.of(DATA_SOURCE, createCompactionConfig(800000, new Period("P1D"), null)), - ImmutableMap.of( - DATA_SOURCE, - createTimeline( - new SegmentGenerateSpec( - Intervals.of("2017-12-04T01:00:00/2017-12-05T03:00:00"), - new Period("PT1H"), - 375, - 80 - ), - new SegmentGenerateSpec( - Intervals.of("2017-12-04T00:00:00/2017-12-04T01:00:00"), - new Period("PT1H"), - 200, - 150 - ), - new SegmentGenerateSpec( - Intervals.of("2017-12-03T18:00:00/2017-12-04T00:00:00"), - new Period("PT6H"), - 200000, - 1 - ), - new SegmentGenerateSpec( - Intervals.of("2017-12-03T11:00:00/2017-12-03T18:00:00"), - new Period("PT1H"), - 375, - 80 - ) - ) - ), - Collections.emptyMap(), - statusTracker + final CompactionSegmentIterator iterator = createIterator( + configBuilder().withSkipOffsetFromLatest(Period.days(1)).build(), + createTimeline( + createSegments().forIntervals(26, Granularities.HOUR) + .startingAt("2017-12-04T01:00:00Z") + .withNumPartitions(80), + createSegments().forIntervals(1, Granularities.HOUR) + .startingAt("2017-12-04") + .withNumPartitions(150), + createSegments().forIntervals(1, Granularities.SIX_HOUR) + .startingAt("2017-12-03T18:00:00Z") + .withNumPartitions(1), + createSegments().forIntervals(7, Granularities.HOUR) + .startingAt("2017-12-03T11:00:00Z") + .withNumPartitions(80) + ) ); Interval lastInterval = null; @@ -301,19 +267,23 @@ public void testManySegmentsPerShard() public void testSkipUnknownDataSource() { final String unknownDataSource = "unknown"; - final Period segmentPeriod = new Period("PT1H"); + final Period segmentPeriod = Period.hours(1); final CompactionSegmentIterator iterator = policy.createIterator( ImmutableMap.of( unknownDataSource, - createCompactionConfig(10000, new Period("P2D"), null), + configBuilder().forDataSource(unknownDataSource).withSkipOffsetFromLatest(Period.days(2)).build(), DATA_SOURCE, - createCompactionConfig(10000, new Period("P2D"), null) + configBuilder().forDataSource(DATA_SOURCE).withSkipOffsetFromLatest(Period.days(2)).build() ), ImmutableMap.of( DATA_SOURCE, createTimeline( - new SegmentGenerateSpec(Intervals.of("2017-11-16T20:00:00/2017-11-17T04:00:00"), segmentPeriod), - new SegmentGenerateSpec(Intervals.of("2017-11-14T00:00:00/2017-11-16T07:00:00"), segmentPeriod) + createSegments().forIntervals(8, Granularities.HOUR) + .startingAt("2017-11-16T20:00:00Z") + .withNumPartitions(4), + createSegments().forIntervals(55, Granularities.HOUR) + .startingAt("2017-11-14") + .withNumPartitions(4) ) ), Collections.emptyMap(), @@ -332,32 +302,32 @@ public void testSkipUnknownDataSource() @Test public void testClearSegmentsToCompactWhenSkippingSegments() { - final long inputSegmentSizeBytes = 800000; - final SegmentTimeline timeline = createTimeline( - new SegmentGenerateSpec( - Intervals.of("2017-12-03T00:00:00/2017-12-04T00:00:00"), - new Period("P1D"), - inputSegmentSizeBytes / 2 + 10, - 1 - ), - new SegmentGenerateSpec( - Intervals.of("2017-12-02T00:00:00/2017-12-03T00:00:00"), - new Period("P1D"), - inputSegmentSizeBytes + 10, // large segment - 1 - ), - new SegmentGenerateSpec( - Intervals.of("2017-12-01T00:00:00/2017-12-02T00:00:00"), - new Period("P1D"), - inputSegmentSizeBytes / 3 + 10, - 2 - ) - ); - final CompactionSegmentIterator iterator = policy.createIterator( - ImmutableMap.of(DATA_SOURCE, createCompactionConfig(inputSegmentSizeBytes, new Period("P0D"), null)), - ImmutableMap.of(DATA_SOURCE, timeline), - Collections.emptyMap(), - statusTracker + final long inputSegmentSizeBytes = 800_000; + final List segments = new ArrayList<>( + createSegments() + .forIntervals(1, Granularities.DAY) + .startingAt("2017-12-03") + .withNumPartitions(1) + .eachOfSize(inputSegmentSizeBytes / 2 + 10) + ); + segments.addAll( + createSegments() + .forIntervals(1, Granularities.DAY) + .startingAt("2017-12-02") + .withNumPartitions(1) + .eachOfSize(inputSegmentSizeBytes + 10) // large segment + ); + segments.addAll( + createSegments() + .forIntervals(1, Granularities.DAY) + .startingAt("2017-12-01") + .withNumPartitions(2) + .eachOfSize(inputSegmentSizeBytes / 3 + 10) + ); + final SegmentTimeline timeline = SegmentTimeline.forSegments(segments); + final CompactionSegmentIterator iterator = createIterator( + configBuilder().withInputSegmentSizeBytes(inputSegmentSizeBytes).build(), + timeline ); final List expectedSegmentsToCompact = new ArrayList<>( @@ -374,8 +344,8 @@ public void testClearSegmentsToCompactWhenSkippingSegments() .map(SegmentsToCompact::getSegments) .collect(Collectors.toSet()); Assert.assertEquals( - observedSegments, - ImmutableSet.of(expectedSegmentsToCompact, expectedSegmentsToCompact2) + ImmutableSet.of(expectedSegmentsToCompact, expectedSegmentsToCompact2), + observedSegments ); } @@ -383,19 +353,15 @@ public void testClearSegmentsToCompactWhenSkippingSegments() public void testIfFirstSegmentIsInSkipOffset() { final SegmentTimeline timeline = createTimeline( - new SegmentGenerateSpec( - Intervals.of("2017-12-02T14:00:00/2017-12-03T00:00:00"), - new Period("PT5H"), - 40000, - 1 - ) + createSegments() + .forIntervals(2, new PeriodGranularity(Period.hours(5), null, null)) + .startingAt("2017-12-02T14:00:00Z") + .withNumPartitions(1) ); - final CompactionSegmentIterator iterator = policy.createIterator( - ImmutableMap.of(DATA_SOURCE, createCompactionConfig(40000, new Period("P1D"), null)), - ImmutableMap.of(DATA_SOURCE, timeline), - Collections.emptyMap(), - statusTracker + final CompactionSegmentIterator iterator = createIterator( + configBuilder().withSkipOffsetFromLatest(Period.days(1)).build(), + timeline ); Assert.assertFalse(iterator.hasNext()); @@ -405,19 +371,15 @@ public void testIfFirstSegmentIsInSkipOffset() public void testIfFirstSegmentOverlapsSkipOffset() { final SegmentTimeline timeline = createTimeline( - new SegmentGenerateSpec( - Intervals.of("2017-12-01T23:00:00/2017-12-03T00:00:00"), - new Period("PT5H"), - 40000, - 1 - ) + createSegments() + .forIntervals(5, new PeriodGranularity(Period.hours(5), null, null)) + .startingAt("2017-12-01T23:00:00Z") + .withNumPartitions(1) ); - final CompactionSegmentIterator iterator = policy.createIterator( - ImmutableMap.of(DATA_SOURCE, createCompactionConfig(40000, new Period("P1D"), null)), - ImmutableMap.of(DATA_SOURCE, timeline), - Collections.emptyMap(), - statusTracker + final CompactionSegmentIterator iterator = createIterator( + configBuilder().withSkipOffsetFromLatest(Period.days(1)).build(), + timeline ); Assert.assertFalse(iterator.hasNext()); @@ -427,20 +389,25 @@ public void testIfFirstSegmentOverlapsSkipOffset() public void testIfSegmentsSkipOffsetWithConfiguredSegmentGranularityEqual() { final SegmentTimeline timeline = createTimeline( - new SegmentGenerateSpec(Intervals.of("2017-11-30T23:00:00/2017-12-03T00:00:00"), new Period("P1D")), - new SegmentGenerateSpec(Intervals.of("2017-10-14T00:00:00/2017-10-15T00:00:00"), new Period("P1D")) + createSegments().forIntervals(2, Granularities.DAY) + .startingAt("2017-12-01") + .withNumPartitions(4), + createSegments().forIntervals(1, Granularities.DAY) + .startingAt("2017-10-14") + .withNumPartitions(4) ); - final CompactionSegmentIterator iterator = policy.createIterator( - ImmutableMap.of(DATA_SOURCE, createCompactionConfig(40000, new Period("P1D"), new UserCompactionTaskGranularityConfig(Granularities.DAY, null, null))), - ImmutableMap.of(DATA_SOURCE, timeline), - Collections.emptyMap(), - statusTracker + final CompactionSegmentIterator iterator = createIterator( + configBuilder() + .withSkipOffsetFromLatest(Period.days(1)) + .withGranularitySpec(new UserCompactionTaskGranularityConfig(Granularities.DAY, null, null)) + .build(), + timeline ); // We should only get segments in Oct final List expectedSegmentsToCompact = new ArrayList<>( - timeline.findNonOvershadowedObjectsInInterval(Intervals.of("2017-10-14T00:00:00/2017-12-02T00:00:00"), Partitions.ONLY_COMPLETE) + timeline.findNonOvershadowedObjectsInInterval(Intervals.of("2017-10-14/2017-12-02"), Partitions.ONLY_COMPLETE) ); Assert.assertTrue(iterator.hasNext()); @@ -461,20 +428,25 @@ public void testIfSegmentsSkipOffsetWithConfiguredSegmentGranularityLarger() // - Cross between month boundary of latest month (starts in Nov and ends in Dec). This should be skipped // - Fully in latest month (starts in Dec and ends in Dec). This should be skipped // - Does not overlap latest month (starts in Oct and ends in Oct). This should not be skipped - new SegmentGenerateSpec(Intervals.of("2017-11-30T23:00:00/2017-12-03T00:00:00"), new Period("PT5H")), - new SegmentGenerateSpec(Intervals.of("2017-10-14T00:00:00/2017-10-15T00:00:00"), new Period("PT5H")) + createSegments().forIntervals(5, new PeriodGranularity(Period.hours(5), null, null)) + .startingAt("2017-11-30T23:00:00Z") + .withNumPartitions(4), + createSegments().forIntervals(4, new PeriodGranularity(Period.hours(5), null, null)) + .startingAt("2017-10-14T04:00:00Z") + .withNumPartitions(4) ); - final CompactionSegmentIterator iterator = policy.createIterator( - ImmutableMap.of(DATA_SOURCE, createCompactionConfig(40000, new Period("P1D"), new UserCompactionTaskGranularityConfig(Granularities.MONTH, null, null))), - ImmutableMap.of(DATA_SOURCE, timeline), - Collections.emptyMap(), - statusTracker + final CompactionSegmentIterator iterator = createIterator( + configBuilder() + .withSkipOffsetFromLatest(Period.days(1)) + .withGranularitySpec(new UserCompactionTaskGranularityConfig(Granularities.MONTH, null, null)) + .build(), + timeline ); // We should only get segments in Oct final List expectedSegmentsToCompact = new ArrayList<>( - timeline.findNonOvershadowedObjectsInInterval(Intervals.of("2017-10-14T00:00:00/2017-10-15T00:00:00"), Partitions.ONLY_COMPLETE) + timeline.findNonOvershadowedObjectsInInterval(Intervals.of("2017-10-14/P1D"), Partitions.ONLY_COMPLETE) ); Assert.assertTrue(iterator.hasNext()); @@ -488,15 +460,20 @@ public void testIfSegmentsSkipOffsetWithConfiguredSegmentGranularityLarger() public void testIfSegmentsSkipOffsetWithConfiguredSegmentGranularitySmaller() { final SegmentTimeline timeline = createTimeline( - new SegmentGenerateSpec(Intervals.of("2017-12-01T23:00:00/2017-12-03T00:00:00"), new Period("PT5H")), - new SegmentGenerateSpec(Intervals.of("2017-10-14T00:00:00/2017-10-15T00:00:00"), new Period("PT5H")) + createSegments().forIntervals(5, new PeriodGranularity(Period.hours(5), null, null)) + .startingAt("2017-12-01T23:00:00Z") + .withNumPartitions(4), + createSegments().forIntervals(4, new PeriodGranularity(Period.hours(5), null, null)) + .startingAt("2017-10-14") + .withNumPartitions(4) ); - final CompactionSegmentIterator iterator = policy.createIterator( - ImmutableMap.of(DATA_SOURCE, createCompactionConfig(40000, new Period("P1D"), new UserCompactionTaskGranularityConfig(Granularities.MINUTE, null, null))), - ImmutableMap.of(DATA_SOURCE, timeline), - Collections.emptyMap(), - statusTracker + final CompactionSegmentIterator iterator = createIterator( + configBuilder() + .withSkipOffsetFromLatest(Period.days(1)) + .withGranularitySpec(new UserCompactionTaskGranularityConfig(Granularities.MINUTE, null, null)) + .build(), + timeline ); // We should only get segments in Oct @@ -517,14 +494,18 @@ public void testIfSegmentsSkipOffsetWithConfiguredSegmentGranularitySmaller() @Test public void testWithSkipIntervals() { - final Period segmentPeriod = new Period("PT1H"); + final Period segmentPeriod = Period.hours(1); final CompactionSegmentIterator iterator = policy.createIterator( - ImmutableMap.of(DATA_SOURCE, createCompactionConfig(10000, new Period("P1D"), null)), + ImmutableMap.of(DATA_SOURCE, configBuilder().withSkipOffsetFromLatest(Period.days(1)).build()), ImmutableMap.of( DATA_SOURCE, createTimeline( - new SegmentGenerateSpec(Intervals.of("2017-11-16T20:00:00/2017-11-17T04:00:00"), segmentPeriod), - new SegmentGenerateSpec(Intervals.of("2017-11-14T00:00:00/2017-11-16T07:00:00"), segmentPeriod) + createSegments().forIntervals(8, Granularities.HOUR) + .startingAt("2017-11-16T20:00:00Z") + .withNumPartitions(4), + createSegments().forIntervals(55, Granularities.HOUR) + .startingAt("2017-11-14") + .withNumPartitions(4) ) ), ImmutableMap.of( @@ -558,13 +539,13 @@ public void testWithSkipIntervals() @Test public void testHoleInSearchInterval() { - final Period segmentPeriod = new Period("PT1H"); + final Period segmentPeriod = Period.hours(1); final CompactionSegmentIterator iterator = policy.createIterator( - ImmutableMap.of(DATA_SOURCE, createCompactionConfig(10000, new Period("PT1H"), null)), + ImmutableMap.of(DATA_SOURCE, configBuilder().withSkipOffsetFromLatest(Period.hours(1)).build()), ImmutableMap.of( DATA_SOURCE, createTimeline( - new SegmentGenerateSpec(Intervals.of("2017-11-16T00:00:00/2017-11-17T00:00:00"), segmentPeriod) + createSegments().forIntervals(1, Granularities.HOUR).startingAt("2017-11-16").withNumPartitions(4) ) ), ImmutableMap.of( @@ -606,15 +587,15 @@ public void testHoleInSearchInterval() public void testIteratorReturnsSegmentsInConfiguredSegmentGranularity() { final SegmentTimeline timeline = createTimeline( - // Segments with day interval from Oct to Dec - new SegmentGenerateSpec(Intervals.of("2017-10-01T00:00:00/2017-12-31T00:00:00"), new Period("P1D")) + createSegments() + .forIntervals(3, Granularities.MONTH) + .startingAt("2017-10-01") + .withNumPartitions(4) ); - final CompactionSegmentIterator iterator = policy.createIterator( - ImmutableMap.of(DATA_SOURCE, createCompactionConfig(130000, new Period("P0D"), new UserCompactionTaskGranularityConfig(Granularities.MONTH, null, null))), - ImmutableMap.of(DATA_SOURCE, timeline), - Collections.emptyMap(), - statusTracker + final CompactionSegmentIterator iterator = createIterator( + createConfigWithSegmentGranularity(Granularities.MONTH), + timeline ); // We should get all segments in timeline back since skip offset is P0D. @@ -655,16 +636,14 @@ public void testIteratorReturnsSegmentsInConfiguredSegmentGranularity() public void testIteratorReturnsSegmentsInMultipleIntervalIfConfiguredSegmentGranularityCrossBoundary() { final SegmentTimeline timeline = createTimeline( - new SegmentGenerateSpec(Intervals.of("2020-01-01/2020-01-08"), new Period("P7D")), - new SegmentGenerateSpec(Intervals.of("2020-01-28/2020-02-03"), new Period("P7D")), - new SegmentGenerateSpec(Intervals.of("2020-02-08/2020-02-15"), new Period("P7D")) + createSegments().forIntervals(1, Granularities.WEEK).startingAt("2020-01-01").withNumPartitions(4), + createSegments().forIntervals(1, Granularities.WEEK).startingAt("2020-01-28").withNumPartitions(4), + createSegments().forIntervals(1, Granularities.WEEK).startingAt("2020-02-08").withNumPartitions(4) ); - final CompactionSegmentIterator iterator = policy.createIterator( - ImmutableMap.of(DATA_SOURCE, createCompactionConfig(130000, new Period("P0D"), new UserCompactionTaskGranularityConfig(Granularities.MONTH, null, null))), - ImmutableMap.of(DATA_SOURCE, timeline), - Collections.emptyMap(), - statusTracker + final CompactionSegmentIterator iterator = createIterator( + createConfigWithSegmentGranularity(Granularities.MONTH), + timeline ); // We should get the segment of "2020-01-28/2020-02-03" back twice when the iterator returns for Jan and when the // iterator returns for Feb. @@ -693,14 +672,12 @@ public void testIteratorReturnsSegmentsInMultipleIntervalIfConfiguredSegmentGran public void testIteratorDoesNotReturnCompactedInterval() { final SegmentTimeline timeline = createTimeline( - new SegmentGenerateSpec(Intervals.of("2017-12-01T00:00:00/2017-12-02T00:00:00"), new Period("P1D")) + createSegments().forIntervals(1, Granularities.DAY).startingAt("2017-12-01").withNumPartitions(4) ); - final CompactionSegmentIterator iterator = policy.createIterator( - ImmutableMap.of(DATA_SOURCE, createCompactionConfig(40000, new Period("P0D"), new UserCompactionTaskGranularityConfig(Granularities.MINUTE, null, null))), - ImmutableMap.of(DATA_SOURCE, timeline), - Collections.emptyMap(), - statusTracker + final CompactionSegmentIterator iterator = createIterator( + createConfigWithSegmentGranularity(Granularities.MINUTE), + timeline ); final List expectedSegmentsToCompact = new ArrayList<>( @@ -719,15 +696,15 @@ public void testIteratorDoesNotReturnCompactedInterval() public void testIteratorReturnsAllMixedVersionSegmentsInConfiguredSegmentGranularity() { final SegmentTimeline timeline = createTimeline( - new SegmentGenerateSpec(Intervals.of("2017-10-01T00:00:00/2017-10-02T00:00:00"), new Period("P1D"), "1994-04-29T00:00:00.000Z", null), - new SegmentGenerateSpec(Intervals.of("2017-10-01T01:00:00/2017-10-01T02:00:00"), new Period("PT1H"), "1994-04-30T00:00:00.000Z", null) + createSegments().forIntervals(1, Granularities.DAY).startingAt("2017-10-01") + .withVersion("v1").withNumPartitions(4), + createSegments().forIntervals(1, Granularities.HOUR).startingAt("2017-10-01") + .withVersion("v2").withNumPartitions(4) ); - final CompactionSegmentIterator iterator = policy.createIterator( - ImmutableMap.of(DATA_SOURCE, createCompactionConfig(130000, new Period("P0D"), new UserCompactionTaskGranularityConfig(Granularities.MONTH, null, null))), - ImmutableMap.of(DATA_SOURCE, timeline), - Collections.emptyMap(), - statusTracker + final CompactionSegmentIterator iterator = createIterator( + createConfigWithSegmentGranularity(Granularities.MONTH), + timeline ); // We should get all segments in timeline back since skip offset is P0D. @@ -753,26 +730,20 @@ public void testIteratorReturnsNothingAsSegmentsWasCompactedAndHaveSameSegmentGr // Create segments that were compacted (CompactionState != null) and have segmentGranularity=DAY final SegmentTimeline timeline = createTimeline( - new SegmentGenerateSpec( - Intervals.of("2017-10-01T00:00:00/2017-10-02T00:00:00"), - new Period("P1D"), - null, - new CompactionState(partitionsSpec, null, null, null, indexSpec, null) - ), - new SegmentGenerateSpec( - Intervals.of("2017-10-02T00:00:00/2017-10-03T00:00:00"), - new Period("P1D"), - null, - new CompactionState(partitionsSpec, null, null, null, indexSpec, null) - ) + createSegments() + .startingAt("2017-10-01") + .withCompactionState(new CompactionState(partitionsSpec, null, null, null, indexSpec, null)) + .withNumPartitions(4), + createSegments() + .startingAt("2017-10-02") + .withCompactionState(new CompactionState(partitionsSpec, null, null, null, indexSpec, null)) + .withNumPartitions(4) ); // Auto compaction config sets segmentGranularity=DAY - final CompactionSegmentIterator iterator = policy.createIterator( - ImmutableMap.of(DATA_SOURCE, createCompactionConfig(130000, new Period("P0D"), new UserCompactionTaskGranularityConfig(Granularities.DAY, null, null))), - ImmutableMap.of(DATA_SOURCE, timeline), - Collections.emptyMap(), - statusTracker + final CompactionSegmentIterator iterator = createIterator( + createConfigWithSegmentGranularity(Granularities.DAY), + timeline ); Assert.assertFalse(iterator.hasNext()); } @@ -786,27 +757,20 @@ public void testIteratorReturnsNothingAsSegmentsWasCompactedAndHaveSameSegmentGr PartitionsSpec partitionsSpec = CompactionStatus.findPartitionsSpecFromConfig(ClientCompactionTaskQueryTuningConfig.from(null)); // Create segments that were compacted (CompactionState != null) and have segmentGranularity=DAY + final CompactionState compactionState + = new CompactionState(partitionsSpec, null, null, null, indexSpec, ImmutableMap.of("segmentGranularity", "day")); final SegmentTimeline timeline = createTimeline( - new SegmentGenerateSpec( - Intervals.of("2017-10-01T00:00:00/2017-10-02T00:00:00"), - new Period("P1D"), - null, - new CompactionState(partitionsSpec, null, null, null, indexSpec, ImmutableMap.of("segmentGranularity", "day")) - ), - new SegmentGenerateSpec( - Intervals.of("2017-10-02T00:00:00/2017-10-03T00:00:00"), - new Period("P1D"), - null, - new CompactionState(partitionsSpec, null, null, null, indexSpec, ImmutableMap.of("segmentGranularity", "day")) - ) + createSegments() + .forIntervals(2, Granularities.DAY) + .startingAt("2017-10-01") + .withNumPartitions(4) + .withCompactionState(compactionState) ); // Auto compaction config sets segmentGranularity=DAY - final CompactionSegmentIterator iterator = policy.createIterator( - ImmutableMap.of(DATA_SOURCE, createCompactionConfig(130000, new Period("P0D"), new UserCompactionTaskGranularityConfig(Granularities.DAY, null, null))), - ImmutableMap.of(DATA_SOURCE, timeline), - Collections.emptyMap(), - statusTracker + final CompactionSegmentIterator iterator = createIterator( + createConfigWithSegmentGranularity(Granularities.DAY), + timeline ); Assert.assertFalse(iterator.hasNext()); } @@ -820,27 +784,19 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentSeg PartitionsSpec partitionsSpec = CompactionStatus.findPartitionsSpecFromConfig(ClientCompactionTaskQueryTuningConfig.from(null)); // Create segments that were compacted (CompactionState != null) and have segmentGranularity=DAY + final CompactionState compactionState = new CompactionState(partitionsSpec, null, null, null, indexSpec, null); final SegmentTimeline timeline = createTimeline( - new SegmentGenerateSpec( - Intervals.of("2017-10-01T00:00:00/2017-10-02T00:00:00"), - new Period("P1D"), - null, - new CompactionState(partitionsSpec, null, null, null, indexSpec, null) - ), - new SegmentGenerateSpec( - Intervals.of("2017-10-02T00:00:00/2017-10-03T00:00:00"), - new Period("P1D"), - null, - new CompactionState(partitionsSpec, null, null, null, indexSpec, null) - ) + createSegments() + .forIntervals(2, Granularities.DAY) + .startingAt("2017-10-01") + .withNumPartitions(4) + .withCompactionState(compactionState) ); // Auto compaction config sets segmentGranularity=YEAR - final CompactionSegmentIterator iterator = policy.createIterator( - ImmutableMap.of(DATA_SOURCE, createCompactionConfig(130000, new Period("P0D"), new UserCompactionTaskGranularityConfig(Granularities.YEAR, null, null))), - ImmutableMap.of(DATA_SOURCE, timeline), - Collections.emptyMap(), - statusTracker + final CompactionSegmentIterator iterator = createIterator( + createConfigWithSegmentGranularity(Granularities.YEAR), + timeline ); // We should get all segments in timeline back since skip offset is P0D. Assert.assertTrue(iterator.hasNext()); @@ -864,27 +820,20 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentSeg PartitionsSpec partitionsSpec = CompactionStatus.findPartitionsSpecFromConfig(ClientCompactionTaskQueryTuningConfig.from(null)); // Create segments that were compacted (CompactionState != null) and have segmentGranularity=DAY + final CompactionState compactionState + = new CompactionState(partitionsSpec, null, null, null, indexSpec, ImmutableMap.of("segmentGranularity", "day")); final SegmentTimeline timeline = createTimeline( - new SegmentGenerateSpec( - Intervals.of("2017-10-01T00:00:00/2017-10-02T00:00:00"), - new Period("P1D"), - null, - new CompactionState(partitionsSpec, null, null, null, indexSpec, ImmutableMap.of("segmentGranularity", "day")) - ), - new SegmentGenerateSpec( - Intervals.of("2017-10-02T00:00:00/2017-10-03T00:00:00"), - new Period("P1D"), - null, - new CompactionState(partitionsSpec, null, null, null, indexSpec, ImmutableMap.of("segmentGranularity", "day")) - ) + createSegments() + .forIntervals(2, Granularities.DAY) + .startingAt("2017-10-01") + .withNumPartitions(4) + .withCompactionState(compactionState) ); // Auto compaction config sets segmentGranularity=YEAR - final CompactionSegmentIterator iterator = policy.createIterator( - ImmutableMap.of(DATA_SOURCE, createCompactionConfig(130000, new Period("P0D"), new UserCompactionTaskGranularityConfig(Granularities.YEAR, null, null))), - ImmutableMap.of(DATA_SOURCE, timeline), - Collections.emptyMap(), - statusTracker + final CompactionSegmentIterator iterator = createIterator( + createConfigWithSegmentGranularity(Granularities.YEAR), + timeline ); // We should get all segments in timeline back since skip offset is P0D. Assert.assertTrue(iterator.hasNext()); @@ -909,35 +858,28 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentTim // Create segments that were compacted (CompactionState != null) and have segmentGranularity=DAY final SegmentTimeline timeline = createTimeline( - new SegmentGenerateSpec( - Intervals.of("2017-10-02T00:00:00/2017-10-03T00:00:00"), - new Period("P1D"), - null, - new CompactionState(partitionsSpec, null, null, null, indexSpec, null) - ) + createSegments() + .forIntervals(1, Granularities.DAY) + .startingAt("2017-10-02") + .withNumPartitions(4) + .withCompactionState(new CompactionState(partitionsSpec, null, null, null, indexSpec, null)) ); // Duration of new segmentGranularity is the same as before (P1D), // but we changed the timezone from UTC to Bangkok in the auto compaction spec - final CompactionSegmentIterator iterator = policy.createIterator( - ImmutableMap.of(DATA_SOURCE, - createCompactionConfig( - 130000, - new Period("P0D"), - new UserCompactionTaskGranularityConfig( - new PeriodGranularity( - new Period("P1D"), - null, - DateTimeZone.forTimeZone(TimeZone.getTimeZone("Asia/Bangkok")) - ), - null, - null - ) - ) - ), - ImmutableMap.of(DATA_SOURCE, timeline), - Collections.emptyMap(), - statusTracker + final CompactionSegmentIterator iterator = createIterator( + configBuilder().withGranularitySpec( + new UserCompactionTaskGranularityConfig( + new PeriodGranularity( + Period.days(1), + null, + DateTimeZone.forTimeZone(TimeZone.getTimeZone("Asia/Bangkok")) + ), + null, + null + ) + ).build(), + timeline ); // We should get all segments in timeline back since skip offset is P0D. Assert.assertTrue(iterator.hasNext()); @@ -962,34 +904,27 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentOri // Create segments that were compacted (CompactionState != null) and have segmentGranularity=DAY final SegmentTimeline timeline = createTimeline( - new SegmentGenerateSpec( - Intervals.of("2017-10-02T00:00:00/2017-10-03T00:00:00"), - new Period("P1D"), - null, - new CompactionState(partitionsSpec, null, null, null, indexSpec, null) - ) + createSegments() + .forIntervals(1, Granularities.DAY) + .startingAt("2017-10-02") + .withNumPartitions(4) + .withCompactionState(new CompactionState(partitionsSpec, null, null, null, indexSpec, null)) ); // Duration of new segmentGranularity is the same as before (P1D), but we changed the origin in the autocompaction spec - final CompactionSegmentIterator iterator = policy.createIterator( - ImmutableMap.of(DATA_SOURCE, - createCompactionConfig( - 130000, - new Period("P0D"), - new UserCompactionTaskGranularityConfig( - new PeriodGranularity( - new Period("P1D"), - DateTimes.of("2012-01-02T00:05:00.000Z"), - DateTimeZone.UTC - ), - null, - null - ) - ) - ), - ImmutableMap.of(DATA_SOURCE, timeline), - Collections.emptyMap(), - statusTracker + final CompactionSegmentIterator iterator = createIterator( + configBuilder().withGranularitySpec( + new UserCompactionTaskGranularityConfig( + new PeriodGranularity( + Period.days(1), + DateTimes.of("2012-01-02T00:05:00.000Z"), + DateTimeZone.UTC + ), + null, + null + ) + ).build(), + timeline ); // We should get all segments in timeline back since skip offset is P0D. Assert.assertTrue(iterator.hasNext()); @@ -1017,32 +952,29 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentRol // rollup=true for interval 2017-10-02T00:00:00/2017-10-03T00:00:00, // and rollup=null for interval 2017-10-03T00:00:00/2017-10-04T00:00:00 (queryGranularity was not set during last compaction) final SegmentTimeline timeline = createTimeline( - new SegmentGenerateSpec( - Intervals.of("2017-10-01T00:00:00/2017-10-02T00:00:00"), - new Period("P1D"), - null, - new CompactionState(partitionsSpec, null, null, null, indexSpec, ImmutableMap.of("rollup", "false")) - ), - new SegmentGenerateSpec( - Intervals.of("2017-10-02T00:00:00/2017-10-03T00:00:00"), - new Period("P1D"), - null, - new CompactionState(partitionsSpec, null, null, null, indexSpec, ImmutableMap.of("rollup", "true")) - ), - new SegmentGenerateSpec( - Intervals.of("2017-10-03T00:00:00/2017-10-04T00:00:00"), - new Period("P1D"), - null, - new CompactionState(partitionsSpec, null, null, null, indexSpec, ImmutableMap.of()) - ) + createSegments() + .forIntervals(1, Granularities.DAY) + .startingAt("2017-10-01") + .withNumPartitions(4) + .withCompactionState(new CompactionState(partitionsSpec, null, null, null, indexSpec, ImmutableMap.of("rollup", "false"))), + createSegments() + .forIntervals(1, Granularities.DAY) + .startingAt("2017-10-02") + .withNumPartitions(4) + .withCompactionState(new CompactionState(partitionsSpec, null, null, null, indexSpec, ImmutableMap.of("rollup", "true"))), + createSegments() + .forIntervals(1, Granularities.DAY) + .startingAt("2017-10-03") + .withNumPartitions(4) + .withCompactionState(new CompactionState(partitionsSpec, null, null, null, indexSpec, ImmutableMap.of())) ); // Auto compaction config sets rollup=true - final CompactionSegmentIterator iterator = policy.createIterator( - ImmutableMap.of(DATA_SOURCE, createCompactionConfig(130000, new Period("P0D"), new UserCompactionTaskGranularityConfig(null, null, true))), - ImmutableMap.of(DATA_SOURCE, timeline), - Collections.emptyMap(), - statusTracker + final CompactionSegmentIterator iterator = createIterator( + configBuilder().withGranularitySpec( + new UserCompactionTaskGranularityConfig(null, null, true) + ).build(), + timeline ); // We should get interval 2017-10-01T00:00:00/2017-10-02T00:00:00 and interval 2017-10-03T00:00:00/2017-10-04T00:00:00. Assert.assertTrue(iterator.hasNext()); @@ -1078,32 +1010,29 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentQue // queryGranularity=MINUTE for interval 2017-10-02T00:00:00/2017-10-03T00:00:00, // and queryGranularity=null for interval 2017-10-03T00:00:00/2017-10-04T00:00:00 (queryGranularity was not set during last compaction) final SegmentTimeline timeline = createTimeline( - new SegmentGenerateSpec( - Intervals.of("2017-10-01T00:00:00/2017-10-02T00:00:00"), - new Period("P1D"), - null, - new CompactionState(partitionsSpec, null, null, null, indexSpec, ImmutableMap.of("queryGranularity", "day")) - ), - new SegmentGenerateSpec( - Intervals.of("2017-10-02T00:00:00/2017-10-03T00:00:00"), - new Period("P1D"), - null, - new CompactionState(partitionsSpec, null, null, null, indexSpec, ImmutableMap.of("queryGranularity", "minute")) - ), - new SegmentGenerateSpec( - Intervals.of("2017-10-03T00:00:00/2017-10-04T00:00:00"), - new Period("P1D"), - null, - new CompactionState(partitionsSpec, null, null, null, indexSpec, ImmutableMap.of()) - ) + createSegments() + .forIntervals(1, Granularities.DAY) + .startingAt("2017-10-01") + .withNumPartitions(4) + .withCompactionState(new CompactionState(partitionsSpec, null, null, null, indexSpec, ImmutableMap.of("queryGranularity", "day"))), + createSegments() + .forIntervals(1, Granularities.DAY) + .startingAt("2017-10-02") + .withNumPartitions(4) + .withCompactionState(new CompactionState(partitionsSpec, null, null, null, indexSpec, ImmutableMap.of("queryGranularity", "minute"))), + createSegments() + .forIntervals(1, Granularities.DAY) + .startingAt("2017-10-03") + .withNumPartitions(4) + .withCompactionState(new CompactionState(partitionsSpec, null, null, null, indexSpec, ImmutableMap.of())) ); // Auto compaction config sets queryGranularity=MINUTE - final CompactionSegmentIterator iterator = policy.createIterator( - ImmutableMap.of(DATA_SOURCE, createCompactionConfig(130000, new Period("P0D"), new UserCompactionTaskGranularityConfig(null, Granularities.MINUTE, null))), - ImmutableMap.of(DATA_SOURCE, timeline), - Collections.emptyMap(), - statusTracker + final CompactionSegmentIterator iterator = createIterator( + configBuilder().withGranularitySpec( + new UserCompactionTaskGranularityConfig(null, Granularities.MINUTE, null) + ).build(), + timeline ); // We should get interval 2017-10-01T00:00:00/2017-10-02T00:00:00 and interval 2017-10-03T00:00:00/2017-10-04T00:00:00. Assert.assertTrue(iterator.hasNext()); @@ -1140,45 +1069,34 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentDim // Dimensions=null for interval 2017-10-03T00:00:00/2017-10-04T00:00:00 (dimensions was not set during last compaction) // and dimensionsSpec=null for interval 2017-10-04T00:00:00/2017-10-05T00:00:00 (dimensionsSpec was not set during last compaction) final SegmentTimeline timeline = createTimeline( - new SegmentGenerateSpec( - Intervals.of("2017-10-01T00:00:00/2017-10-02T00:00:00"), - new Period("P1D"), - null, - new CompactionState(partitionsSpec, new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("bar", "foo"))), null, null, indexSpec, null) - ), - new SegmentGenerateSpec( - Intervals.of("2017-10-02T00:00:00/2017-10-03T00:00:00"), - new Period("P1D"), - null, - new CompactionState(partitionsSpec, new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("foo"))), null, null, indexSpec, null) - ), - new SegmentGenerateSpec( - Intervals.of("2017-10-03T00:00:00/2017-10-04T00:00:00"), - new Period("P1D"), - null, - new CompactionState(partitionsSpec, DimensionsSpec.EMPTY, null, null, indexSpec, null) - ), - new SegmentGenerateSpec( - Intervals.of("2017-10-04T00:00:00/2017-10-05T00:00:00"), - new Period("P1D"), - null, - new CompactionState(partitionsSpec, null, null, null, indexSpec, null) - ) + createSegments() + .startingAt("2017-10-01") + .withNumPartitions(4) + .withCompactionState( + new CompactionState(partitionsSpec, new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("bar", "foo"))), null, null, indexSpec, null) + ), + createSegments() + .startingAt("2017-10-02") + .withNumPartitions(4) + .withCompactionState( + new CompactionState(partitionsSpec, new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("foo"))), null, null, indexSpec, null) + ), + createSegments() + .startingAt("2017-10-03") + .withNumPartitions(4) + .withCompactionState(new CompactionState(partitionsSpec, DimensionsSpec.EMPTY, null, null, indexSpec, null)), + createSegments() + .startingAt("2017-10-04") + .withNumPartitions(4) + .withCompactionState(new CompactionState(partitionsSpec, null, null, null, indexSpec, null)) ); // Auto compaction config sets Dimensions=["foo"] - CompactionSegmentIterator iterator = policy.createIterator( - ImmutableMap.of(DATA_SOURCE, createCompactionConfig( - 130000, - new Period("P0D"), - null, - new UserCompactionTaskDimensionsConfig(DimensionsSpec.getDefaultSchemas(ImmutableList.of("foo"))), - null, - null - )), - ImmutableMap.of(DATA_SOURCE, timeline), - Collections.emptyMap(), - statusTracker + CompactionSegmentIterator iterator = createIterator( + configBuilder().withDimensionsSpec( + new UserCompactionTaskDimensionsConfig(DimensionsSpec.getDefaultSchemas(ImmutableList.of("foo"))) + ).build(), + timeline ); // We should get interval 2017-10-01T00:00:00/2017-10-02T00:00:00, interval 2017-10-04T00:00:00/2017-10-05T00:00:00, and interval 2017-10-03T00:00:00/2017-10-04T00:00:00. Assert.assertTrue(iterator.hasNext()); @@ -1209,18 +1127,11 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentDim Assert.assertFalse(iterator.hasNext()); // Auto compaction config sets Dimensions=null - iterator = policy.createIterator( - ImmutableMap.of(DATA_SOURCE, createCompactionConfig( - 130000, - new Period("P0D"), - null, - new UserCompactionTaskDimensionsConfig(null), - null, - null - )), - ImmutableMap.of(DATA_SOURCE, timeline), - Collections.emptyMap(), - statusTracker + iterator = createIterator( + configBuilder().withDimensionsSpec( + new UserCompactionTaskDimensionsConfig(null) + ).build(), + timeline ); // No more Assert.assertFalse(iterator.hasNext()); @@ -1241,66 +1152,57 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentFil // filter=null for interval 2017-10-03T00:00:00/2017-10-04T00:00:00 (filter was not set during last compaction) // and transformSpec=null for interval 2017-10-04T00:00:00/2017-10-05T00:00:00 (transformSpec was not set during last compaction) final SegmentTimeline timeline = createTimeline( - new SegmentGenerateSpec( - Intervals.of("2017-10-01T00:00:00/2017-10-02T00:00:00"), - new Period("P1D"), - null, - new CompactionState( - partitionsSpec, - null, - null, - mapper.readValue(mapper.writeValueAsString(new TransformSpec(new SelectorDimFilter("dim1", "foo", null), null)), new TypeReference>() {}), - indexSpec, - null - ) - ), - new SegmentGenerateSpec( - Intervals.of("2017-10-02T00:00:00/2017-10-03T00:00:00"), - new Period("P1D"), - null, - new CompactionState( - partitionsSpec, - null, - null, - mapper.readValue(mapper.writeValueAsString(new TransformSpec(new SelectorDimFilter("dim1", "bar", null), null)), new TypeReference>() {}), - indexSpec, - null - ) - ), - new SegmentGenerateSpec( - Intervals.of("2017-10-03T00:00:00/2017-10-04T00:00:00"), - new Period("P1D"), - null, - new CompactionState( - partitionsSpec, - null, - null, - mapper.readValue(mapper.writeValueAsString(new TransformSpec(null, null)), new TypeReference>() {}), - indexSpec, - null - ) - ), - new SegmentGenerateSpec( - Intervals.of("2017-10-04T00:00:00/2017-10-05T00:00:00"), - new Period("P1D"), - null, - new CompactionState(partitionsSpec, null, null, null, indexSpec, null) - ) + createSegments() + .startingAt("2017-10-01") + .withNumPartitions(4) + .withCompactionState( + new CompactionState( + partitionsSpec, + null, + null, + mapper.readValue(mapper.writeValueAsString(new TransformSpec(new SelectorDimFilter("dim1", "foo", null), null)), new TypeReference>() {}), + indexSpec, + null + ) + ), + createSegments() + .startingAt("2017-10-02") + .withNumPartitions(4) + .withCompactionState( + new CompactionState( + partitionsSpec, + null, + null, + mapper.readValue(mapper.writeValueAsString(new TransformSpec(new SelectorDimFilter("dim1", "bar", null), null)), new TypeReference>() {}), + indexSpec, + null + ) + ), + createSegments() + .startingAt("2017-10-03") + .withNumPartitions(4) + .withCompactionState( + new CompactionState( + partitionsSpec, + null, + null, + mapper.readValue(mapper.writeValueAsString(new TransformSpec(null, null)), new TypeReference>() {}), + indexSpec, + null + ) + ), + createSegments() + .startingAt("2017-10-04") + .withNumPartitions(4) + .withCompactionState(new CompactionState(partitionsSpec, null, null, null, indexSpec, null)) ); // Auto compaction config sets filter=SelectorDimFilter("dim1", "bar", null) - CompactionSegmentIterator iterator = policy.createIterator( - ImmutableMap.of(DATA_SOURCE, createCompactionConfig( - 130000, - new Period("P0D"), - null, - null, - new UserCompactionTaskTransformConfig(new SelectorDimFilter("dim1", "bar", null)), - null - )), - ImmutableMap.of(DATA_SOURCE, timeline), - Collections.emptyMap(), - statusTracker + CompactionSegmentIterator iterator = createIterator( + configBuilder().withTransformSpec( + new UserCompactionTaskTransformConfig(new SelectorDimFilter("dim1", "bar", null)) + ).build(), + timeline ); // We should get interval 2017-10-01T00:00:00/2017-10-02T00:00:00, interval 2017-10-04T00:00:00/2017-10-05T00:00:00, and interval 2017-10-03T00:00:00/2017-10-04T00:00:00. Assert.assertTrue(iterator.hasNext()); @@ -1331,18 +1233,11 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentFil Assert.assertFalse(iterator.hasNext()); // Auto compaction config sets filter=null - iterator = policy.createIterator( - ImmutableMap.of(DATA_SOURCE, createCompactionConfig( - 130000, - new Period("P0D"), - null, - null, - new UserCompactionTaskTransformConfig(null), - null - )), - ImmutableMap.of(DATA_SOURCE, timeline), - Collections.emptyMap(), - statusTracker + iterator = createIterator( + configBuilder().withTransformSpec( + new UserCompactionTaskTransformConfig(null) + ).build(), + timeline ); // No more Assert.assertFalse(iterator.hasNext()); @@ -1367,66 +1262,57 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentMet // metricsSpec=[] for interval 2017-10-03T00:00:00/2017-10-04T00:00:00 (filter was not set during last compaction) // and metricsSpec=null for interval 2017-10-04T00:00:00/2017-10-05T00:00:00 (transformSpec was not set during last compaction) final SegmentTimeline timeline = createTimeline( - new SegmentGenerateSpec( - Intervals.of("2017-10-01T00:00:00/2017-10-02T00:00:00"), - new Period("P1D"), - null, - new CompactionState( - partitionsSpec, - null, - mapper.convertValue(new AggregatorFactory[] {new CountAggregatorFactory("cnt")}, new TypeReference>() {}), - null, - indexSpec, - null - ) - ), - new SegmentGenerateSpec( - Intervals.of("2017-10-02T00:00:00/2017-10-03T00:00:00"), - new Period("P1D"), - null, - new CompactionState( - partitionsSpec, - null, - mapper.convertValue(new AggregatorFactory[] {new CountAggregatorFactory("cnt"), new LongSumAggregatorFactory("val", "val")}, new TypeReference>() {}), - null, - indexSpec, - null - ) - ), - new SegmentGenerateSpec( - Intervals.of("2017-10-03T00:00:00/2017-10-04T00:00:00"), - new Period("P1D"), - null, - new CompactionState( - partitionsSpec, - null, - mapper.convertValue(new AggregatorFactory[] {}, new TypeReference>() {}), - null, - indexSpec, - null - ) - ), - new SegmentGenerateSpec( - Intervals.of("2017-10-04T00:00:00/2017-10-05T00:00:00"), - new Period("P1D"), - null, - new CompactionState(partitionsSpec, null, null, null, indexSpec, null) - ) + createSegments() + .startingAt("2017-10-01") + .withNumPartitions(4) + .withCompactionState( + new CompactionState( + partitionsSpec, + null, + mapper.convertValue(new AggregatorFactory[] {new CountAggregatorFactory("cnt")}, new TypeReference>() {}), + null, + indexSpec, + null + ) + ), + createSegments() + .startingAt("2017-10-02") + .withNumPartitions(4) + .withCompactionState( + new CompactionState( + partitionsSpec, + null, + mapper.convertValue(new AggregatorFactory[] {new CountAggregatorFactory("cnt"), new LongSumAggregatorFactory("val", "val")}, new TypeReference>() {}), + null, + indexSpec, + null + ) + ), + createSegments() + .startingAt("2017-10-03") + .withNumPartitions(4) + .withCompactionState( + new CompactionState( + partitionsSpec, + null, + mapper.convertValue(new AggregatorFactory[] {}, new TypeReference>() {}), + null, + indexSpec, + null + ) + ), + createSegments() + .startingAt("2017-10-04") + .withNumPartitions(4) + .withCompactionState(new CompactionState(partitionsSpec, null, null, null, indexSpec, null)) ); // Auto compaction config sets metricsSpec={CountAggregatorFactory("cnt"), LongSumAggregatorFactory("val", "val")} - CompactionSegmentIterator iterator = policy.createIterator( - ImmutableMap.of(DATA_SOURCE, createCompactionConfig( - 130000, - new Period("P0D"), - null, - null, - null, - new AggregatorFactory[] {new CountAggregatorFactory("cnt"), new LongSumAggregatorFactory("val", "val")} - )), - ImmutableMap.of(DATA_SOURCE, timeline), - Collections.emptyMap(), - statusTracker + CompactionSegmentIterator iterator = createIterator( + configBuilder().withMetricsSpec( + new AggregatorFactory[]{new CountAggregatorFactory("cnt"), new LongSumAggregatorFactory("val", "val")} + ).build(), + timeline ); // We should get interval 2017-10-01T00:00:00/2017-10-02T00:00:00, interval 2017-10-04T00:00:00/2017-10-05T00:00:00, and interval 2017-10-03T00:00:00/2017-10-04T00:00:00. Assert.assertTrue(iterator.hasNext()); @@ -1457,18 +1343,9 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentMet Assert.assertFalse(iterator.hasNext()); // Auto compaction config sets metricsSpec=null - iterator = policy.createIterator( - ImmutableMap.of(DATA_SOURCE, createCompactionConfig( - 130000, - new Period("P0D"), - null, - null, - null, - null - )), - ImmutableMap.of(DATA_SOURCE, timeline), - Collections.emptyMap(), - statusTracker + iterator = createIterator( + configBuilder().build(), + timeline ); // No more Assert.assertFalse(iterator.hasNext()); @@ -1478,15 +1355,15 @@ public void testIteratorReturnsSegmentsAsSegmentsWasCompactedAndHaveDifferentMet public void testIteratorReturnsSegmentsSmallerSegmentGranularityCoveringMultipleSegmentsInTimeline() { final SegmentTimeline timeline = createTimeline( - new SegmentGenerateSpec(Intervals.of("2017-10-01T00:00:00/2017-10-02T00:00:00"), new Period("P1D"), "1994-04-29T00:00:00.000Z", null), - new SegmentGenerateSpec(Intervals.of("2017-10-01T01:00:00/2017-10-01T02:00:00"), new Period("PT1H"), "1994-04-30T00:00:00.000Z", null) + createSegments().forIntervals(1, Granularities.DAY).startingAt("2017-10-01") + .withNumPartitions(4).withVersion("v1"), + createSegments().forIntervals(1, Granularities.HOUR).startingAt("2017-10-01") + .withNumPartitions(4).withVersion("v2") ); - final CompactionSegmentIterator iterator = policy.createIterator( - ImmutableMap.of(DATA_SOURCE, createCompactionConfig(130000, new Period("P0D"), new UserCompactionTaskGranularityConfig(Granularities.HOUR, null, null))), - ImmutableMap.of(DATA_SOURCE, timeline), - Collections.emptyMap(), - statusTracker + final CompactionSegmentIterator iterator = createIterator( + createConfigWithSegmentGranularity(Granularities.HOUR), + timeline ); // We should get all segments in timeline back since skip offset is P0D. @@ -1515,34 +1392,27 @@ public void testIteratorReturnsSegmentsAsCompactionStateChangedWithCompactedStat // Create segments that were compacted (CompactionState != null) and have segmentGranularity=DAY final SegmentTimeline timeline = createTimeline( - new SegmentGenerateSpec( - Intervals.of("2017-10-02T00:00:00/2017-10-03T00:00:00"), - new Period("P1D"), - null, - new CompactionState(partitionsSpec, null, null, null, newIndexSpecMap, null) - ) + createSegments() + .forIntervals(1, Granularities.DAY) + .startingAt("2017-10-02") + .withNumPartitions(4) + .withCompactionState(new CompactionState(partitionsSpec, null, null, null, newIndexSpecMap, null)) ); // Duration of new segmentGranularity is the same as before (P1D) - final CompactionSegmentIterator iterator = policy.createIterator( - ImmutableMap.of(DATA_SOURCE, - createCompactionConfig( - 130000, - new Period("P0D"), - new UserCompactionTaskGranularityConfig( - new PeriodGranularity( - new Period("P1D"), - null, - DateTimeZone.UTC - ), - null, - null - ) - ) - ), - ImmutableMap.of(DATA_SOURCE, timeline), - Collections.emptyMap(), - statusTracker + final CompactionSegmentIterator iterator = createIterator( + configBuilder().withGranularitySpec( + new UserCompactionTaskGranularityConfig( + new PeriodGranularity( + Period.days(1), + null, + DateTimeZone.UTC + ), + null, + null + ) + ).build(), + timeline ); // We should get all segments in timeline back since indexSpec changed Assert.assertTrue(iterator.hasNext()); @@ -1563,28 +1433,17 @@ public void testIteratorDoesNotReturnSegmentWithChangingAppendableIndexSpec() NullHandling.initializeForTests(); PartitionsSpec partitionsSpec = CompactionStatus.findPartitionsSpecFromConfig(ClientCompactionTaskQueryTuningConfig.from(null)); final SegmentTimeline timeline = createTimeline( - new SegmentGenerateSpec( - Intervals.of("2017-10-01T00:00:00/2017-10-02T00:00:00"), - new Period("P1D"), - null, - new CompactionState( - partitionsSpec, - null, - null, - null, - IndexSpec.DEFAULT.asMap(mapper), - null + createSegments() + .forIntervals(1, Granularities.DAY) + .startingAt("2017-10-01") + .withNumPartitions(4) + .withCompactionState( + new CompactionState(partitionsSpec, null, null, null, IndexSpec.DEFAULT.asMap(mapper), null) ) - ) ); - CompactionSegmentIterator iterator = policy.createIterator( - ImmutableMap.of(DATA_SOURCE, createCompactionConfig( - 130000, - new Period("P0D"), - null, - null, - null, + CompactionSegmentIterator iterator = createIterator( + configBuilder().withTuningConfig( new UserCompactionTaskQueryTuningConfig( null, new OnheapIncrementalIndex.Spec(true), @@ -1605,22 +1464,14 @@ public void testIteratorDoesNotReturnSegmentWithChangingAppendableIndexSpec() null, null, null - ), - null - )), - ImmutableMap.of(DATA_SOURCE, timeline), - Collections.emptyMap(), - statusTracker + ) + ).build(), + timeline ); Assert.assertFalse(iterator.hasNext()); - iterator = policy.createIterator( - ImmutableMap.of(DATA_SOURCE, createCompactionConfig( - 130000, - new Period("P0D"), - null, - null, - null, + iterator = createIterator( + configBuilder().withTuningConfig( new UserCompactionTaskQueryTuningConfig( null, new OnheapIncrementalIndex.Spec(false), @@ -1641,12 +1492,9 @@ public void testIteratorDoesNotReturnSegmentWithChangingAppendableIndexSpec() null, null, null - ), - null - )), - ImmutableMap.of(DATA_SOURCE, timeline), - Collections.emptyMap(), - statusTracker + ) + ).build(), + timeline ); Assert.assertFalse(iterator.hasNext()); } @@ -1654,31 +1502,22 @@ public void testIteratorDoesNotReturnSegmentWithChangingAppendableIndexSpec() @Test public void testSkipAllGranularityToDefault() { - CompactionSegmentIterator iterator = policy.createIterator( - ImmutableMap.of(DATA_SOURCE, - createCompactionConfig(10000, - new Period("P0D"), - null - ) - ), - ImmutableMap.of( - DATA_SOURCE, - SegmentTimeline.forSegments(ImmutableSet.of( - new DataSegment( - DATA_SOURCE, - Intervals.ETERNITY, - "0", - new HashMap<>(), - new ArrayList<>(), - new ArrayList<>(), - new NumberedShardSpec(0, 0), - 0, - 100) + CompactionSegmentIterator iterator = createIterator( + configBuilder().build(), + SegmentTimeline.forSegments(ImmutableSet.of( + new DataSegment( + DATA_SOURCE, + Intervals.ETERNITY, + "0", + new HashMap<>(), + new ArrayList<>(), + new ArrayList<>(), + new NumberedShardSpec(0, 0), + 0, + 100 ) - ) - ), - Collections.emptyMap(), - statusTracker + ) + ) ); Assert.assertFalse(iterator.hasNext()); @@ -1687,31 +1526,22 @@ public void testSkipAllGranularityToDefault() @Test public void testSkipFirstHalfEternityToDefault() { - CompactionSegmentIterator iterator = policy.createIterator( - ImmutableMap.of(DATA_SOURCE, - createCompactionConfig(10000, - new Period("P0D"), - null - ) - ), - ImmutableMap.of( - DATA_SOURCE, - SegmentTimeline.forSegments(ImmutableSet.of( - new DataSegment( - DATA_SOURCE, - new Interval(DateTimes.MIN, DateTimes.of("2024-01-01")), - "0", - new HashMap<>(), - new ArrayList<>(), - new ArrayList<>(), - new NumberedShardSpec(0, 0), - 0, - 100) + CompactionSegmentIterator iterator = createIterator( + configBuilder().build(), + SegmentTimeline.forSegments(ImmutableSet.of( + new DataSegment( + DATA_SOURCE, + new Interval(DateTimes.MIN, DateTimes.of("2024-01-01")), + "0", + new HashMap<>(), + new ArrayList<>(), + new ArrayList<>(), + new NumberedShardSpec(0, 0), + 0, + 100 ) - ) - ), - Collections.emptyMap(), - statusTracker + ) + ) ); Assert.assertFalse(iterator.hasNext()); @@ -1720,31 +1550,22 @@ public void testSkipFirstHalfEternityToDefault() @Test public void testSkipSecondHalfOfEternityToDefault() { - CompactionSegmentIterator iterator = policy.createIterator( - ImmutableMap.of(DATA_SOURCE, - createCompactionConfig(10000, - new Period("P0D"), - null - ) - ), - ImmutableMap.of( - DATA_SOURCE, - SegmentTimeline.forSegments(ImmutableSet.of( - new DataSegment( - DATA_SOURCE, - new Interval(DateTimes.of("2024-01-01"), DateTimes.MAX), - "0", - new HashMap<>(), - new ArrayList<>(), - new ArrayList<>(), - new NumberedShardSpec(0, 0), - 0, - 100) + CompactionSegmentIterator iterator = createIterator( + configBuilder().build(), + SegmentTimeline.forSegments(ImmutableSet.of( + new DataSegment( + DATA_SOURCE, + new Interval(DateTimes.of("2024-01-01"), DateTimes.MAX), + "0", + new HashMap<>(), + new ArrayList<>(), + new ArrayList<>(), + new NumberedShardSpec(0, 0), + 0, + 100 ) - ) - ), - Collections.emptyMap(), - statusTracker + ) + ) ); Assert.assertFalse(iterator.hasNext()); @@ -1753,31 +1574,24 @@ public void testSkipSecondHalfOfEternityToDefault() @Test public void testSkipAllToAllGranularity() { - CompactionSegmentIterator iterator = policy.createIterator( - ImmutableMap.of(DATA_SOURCE, - createCompactionConfig(10000, - new Period("P0D"), - new UserCompactionTaskGranularityConfig(Granularities.ALL, null, null) - ) - ), - ImmutableMap.of( - DATA_SOURCE, - SegmentTimeline.forSegments(ImmutableSet.of( - new DataSegment( - DATA_SOURCE, - Intervals.ETERNITY, - "0", - new HashMap<>(), - new ArrayList<>(), - new ArrayList<>(), - new NumberedShardSpec(0, 0), - 0, - 100) + CompactionSegmentIterator iterator = createIterator( + configBuilder().withGranularitySpec( + new UserCompactionTaskGranularityConfig(Granularities.ALL, null, null) + ).build(), + SegmentTimeline.forSegments(ImmutableSet.of( + new DataSegment( + DATA_SOURCE, + Intervals.ETERNITY, + "0", + new HashMap<>(), + new ArrayList<>(), + new ArrayList<>(), + new NumberedShardSpec(0, 0), + 0, + 100 ) - ) - ), - Collections.emptyMap(), - statusTracker + ) + ) ); Assert.assertFalse(iterator.hasNext()); @@ -1786,31 +1600,24 @@ public void testSkipAllToAllGranularity() @Test public void testSkipAllToFinerGranularity() { - CompactionSegmentIterator iterator = policy.createIterator( - ImmutableMap.of(DATA_SOURCE, - createCompactionConfig(10000, - new Period("P0D"), - new UserCompactionTaskGranularityConfig(Granularities.DAY, null, null) - ) - ), - ImmutableMap.of( - DATA_SOURCE, - SegmentTimeline.forSegments(ImmutableSet.of( - new DataSegment( - DATA_SOURCE, - Intervals.ETERNITY, - "0", - new HashMap<>(), - new ArrayList<>(), - new ArrayList<>(), - new NumberedShardSpec(0, 0), - 0, - 100) + CompactionSegmentIterator iterator = createIterator( + configBuilder().withGranularitySpec( + new UserCompactionTaskGranularityConfig(Granularities.DAY, null, null) + ).build(), + SegmentTimeline.forSegments(ImmutableSet.of( + new DataSegment( + DATA_SOURCE, + Intervals.ETERNITY, + "0", + new HashMap<>(), + new ArrayList<>(), + new ArrayList<>(), + new NumberedShardSpec(0, 0), + 0, + 100 ) - ) - ), - Collections.emptyMap(), - statusTracker + ) + ) ); Assert.assertFalse(iterator.hasNext()); @@ -1850,19 +1657,11 @@ public void testSkipCompactionForIntervalsContainingSingleTombstone() 0, 1); - CompactionSegmentIterator iterator = policy.createIterator( - ImmutableMap.of(DATA_SOURCE, - createCompactionConfig(10000, - new Period("P0D"), - new UserCompactionTaskGranularityConfig(Granularities.YEAR, null, null) - ) - ), - ImmutableMap.of( - DATA_SOURCE, - SegmentTimeline.forSegments(ImmutableSet.of(tombstone2023, dataSegment2023, tombstone2024)) - ), - Collections.emptyMap(), - statusTracker + CompactionSegmentIterator iterator = createIterator( + configBuilder().withGranularitySpec( + new UserCompactionTaskGranularityConfig(Granularities.YEAR, null, null) + ).build(), + SegmentTimeline.forSegments(ImmutableSet.of(tombstone2023, dataSegment2023, tombstone2024)) ); // Skips 2024/2025 since it has a single tombstone and no data. @@ -1902,26 +1701,18 @@ public void testSkipCompactionForIntervalsContainingSingleTombstone() TombstoneShardSpec.INSTANCE, 0, 1); - iterator = policy.createIterator( - ImmutableMap.of(DATA_SOURCE, - createCompactionConfig(10000, - new Period("P0D"), - new UserCompactionTaskGranularityConfig(Granularities.YEAR, null, null) - ) - ), - ImmutableMap.of( - DATA_SOURCE, - SegmentTimeline.forSegments(ImmutableSet.of( - tombstone2023, - dataSegment2023, - tombstone2024, - tombstone2025Jan, - tombstone2025Feb, - tombstone2025Mar - )) - ), - Collections.emptyMap(), - statusTracker + iterator = createIterator( + configBuilder().withGranularitySpec( + new UserCompactionTaskGranularityConfig(Granularities.YEAR, null, null) + ).build(), + SegmentTimeline.forSegments(ImmutableSet.of( + tombstone2023, + dataSegment2023, + tombstone2024, + tombstone2025Jan, + tombstone2025Feb, + tombstone2025Mar + )) ); // Does not skip the tombstones in 2025 since there are multiple of them which could potentially be condensed to one Assert.assertEquals( @@ -1950,8 +1741,8 @@ public void testPriorityDatasource() final NewestSegmentFirstPolicy policy = new NewestSegmentFirstPolicy(DATA_SOURCE); CompactionSegmentIterator iterator = policy.createIterator( ImmutableMap.of( - DATA_SOURCE, createCompactionConfig(Long.MAX_VALUE, Period.seconds(0), null), - DATASOURCE_KOALA, createCompactionConfig(Long.MAX_VALUE, Period.seconds(0), null) + DATA_SOURCE, configBuilder().forDataSource(DATA_SOURCE).build(), + DATASOURCE_KOALA, configBuilder().forDataSource(DATASOURCE_KOALA).build() ), ImmutableMap.of( DATA_SOURCE, SegmentTimeline.forSegments(wikiSegments), @@ -2020,145 +1811,46 @@ private static void assertCompactSegmentIntervals( } } - private static SegmentTimeline createTimeline(SegmentGenerateSpec... specs) + private static CreateDataSegments createSegments() { - List segments = new ArrayList<>(); - final String version = DateTimes.nowUtc().toString(); - - final List orderedSpecs = Arrays.asList(specs); - orderedSpecs.sort(Comparator.comparing(s -> s.totalInterval, Comparators.intervalsByStartThenEnd().reversed())); - - for (SegmentGenerateSpec spec : orderedSpecs) { - Interval remainingInterval = spec.totalInterval; - - while (!Intervals.isEmpty(remainingInterval)) { - final Interval segmentInterval; - if (remainingInterval.toDuration().isLongerThan(spec.segmentPeriod.toStandardDuration())) { - segmentInterval = new Interval(spec.segmentPeriod, remainingInterval.getEnd()); - } else { - segmentInterval = remainingInterval; - } - - for (int i = 0; i < spec.numSegmentsPerShard; i++) { - final ShardSpec shardSpec = new NumberedShardSpec(i, spec.numSegmentsPerShard); - final DataSegment segment = new DataSegment( - DATA_SOURCE, - segmentInterval, - spec.version == null ? version : spec.version, - null, - ImmutableList.of(), - ImmutableList.of(), - shardSpec, - spec.lastCompactionState, - 0, - spec.segmentSize - ); - segments.add(segment); - } - - remainingInterval = removeIntervalFromEnd(remainingInterval, segmentInterval); - } - } - - return SegmentTimeline.forSegments(segments); + return CreateDataSegments.ofDatasource(DATA_SOURCE).withNumPartitions(DEFAULT_NUM_SEGMENTS_PER_SHARD); } - /** - * Returns an interval [largeInterval.start - smallInterval.start) given that - * the end of both intervals is the same. - */ - private static Interval removeIntervalFromEnd(Interval largeInterval, Interval smallInterval) + private CompactionSegmentIterator createIterator(DataSourceCompactionConfig config, SegmentTimeline timeline) { - Preconditions.checkArgument( - largeInterval.getEnd().equals(smallInterval.getEnd()), - "end should be same. largeInterval[%s] smallInterval[%s]", - largeInterval, - smallInterval + return policy.createIterator( + Collections.singletonMap(DATA_SOURCE, config), + Collections.singletonMap(DATA_SOURCE, timeline), + Collections.emptyMap(), + statusTracker ); - return new Interval(largeInterval.getStart(), smallInterval.getStart()); } - private DataSourceCompactionConfig createCompactionConfig( - long inputSegmentSizeBytes, - Period skipOffsetFromLatest, - UserCompactionTaskGranularityConfig granularitySpec + private static SegmentTimeline createTimeline( + CreateDataSegments... segmentBuilders ) { - return createCompactionConfig(inputSegmentSizeBytes, skipOffsetFromLatest, granularitySpec, null, null, null, null); - } + final SegmentTimeline timeline = new SegmentTimeline(); + for (CreateDataSegments builder : segmentBuilders) { + timeline.addSegments(builder.eachOfSizeInMb(100).iterator()); + } - private DataSourceCompactionConfig createCompactionConfig( - long inputSegmentSizeBytes, - Period skipOffsetFromLatest, - UserCompactionTaskGranularityConfig granularitySpec, - UserCompactionTaskDimensionsConfig dimensionsSpec, - UserCompactionTaskTransformConfig transformSpec, - AggregatorFactory[] metricsSpec - ) - { - return createCompactionConfig(inputSegmentSizeBytes, skipOffsetFromLatest, granularitySpec, dimensionsSpec, transformSpec, null, metricsSpec); + return timeline; } - private DataSourceCompactionConfig createCompactionConfig( - long inputSegmentSizeBytes, - Period skipOffsetFromLatest, - UserCompactionTaskGranularityConfig granularitySpec, - UserCompactionTaskDimensionsConfig dimensionsSpec, - UserCompactionTaskTransformConfig transformSpec, - UserCompactionTaskQueryTuningConfig tuningConfig, - AggregatorFactory[] metricsSpec + private static DataSourceCompactionConfig createConfigWithSegmentGranularity( + Granularity segmentGranularity ) { - return new DataSourceCompactionConfig( - DATA_SOURCE, - 0, - inputSegmentSizeBytes, - null, - skipOffsetFromLatest, - tuningConfig, - granularitySpec, - dimensionsSpec, - metricsSpec, - transformSpec, - null, - null, - null - ); + return configBuilder().withGranularitySpec( + new UserCompactionTaskGranularityConfig(segmentGranularity, null, null) + ).build(); } - private static class SegmentGenerateSpec + private static DataSourceCompactionConfig.Builder configBuilder() { - private final Interval totalInterval; - private final Period segmentPeriod; - private final long segmentSize; - private final int numSegmentsPerShard; - private final String version; - private final CompactionState lastCompactionState; - - SegmentGenerateSpec(Interval totalInterval, Period segmentPeriod) - { - this(totalInterval, segmentPeriod, null, null); - } - - SegmentGenerateSpec(Interval totalInterval, Period segmentPeriod, String version, CompactionState lastCompactionState) - { - this(totalInterval, segmentPeriod, DEFAULT_SEGMENT_SIZE, DEFAULT_NUM_SEGMENTS_PER_SHARD, version, lastCompactionState); - } - - SegmentGenerateSpec(Interval totalInterval, Period segmentPeriod, long segmentSize, int numSegmentsPerShard) - { - this(totalInterval, segmentPeriod, segmentSize, numSegmentsPerShard, null, null); - } - - SegmentGenerateSpec(Interval totalInterval, Period segmentPeriod, long segmentSize, int numSegmentsPerShard, String version, CompactionState lastCompactionState) - { - Preconditions.checkArgument(numSegmentsPerShard >= 1); - this.totalInterval = totalInterval; - this.segmentPeriod = segmentPeriod; - this.segmentSize = segmentSize; - this.numSegmentsPerShard = numSegmentsPerShard; - this.version = version; - this.lastCompactionState = lastCompactionState; - } + return DataSourceCompactionConfig.builder() + .forDataSource(DATA_SOURCE) + .withSkipOffsetFromLatest(Period.seconds(0)); } } diff --git a/server/src/test/java/org/apache/druid/server/coordinator/CreateDataSegments.java b/server/src/test/java/org/apache/druid/server/coordinator/CreateDataSegments.java index 7afa880927b0..b5e6879bedf0 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/CreateDataSegments.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/CreateDataSegments.java @@ -24,6 +24,7 @@ import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.segment.IndexIO; +import org.apache.druid.timeline.CompactionState; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.partition.NumberedShardSpec; import org.joda.time.DateTime; @@ -50,6 +51,9 @@ public class CreateDataSegments private int numPartitions = 1; private int numIntervals = 1; + private String version = "1"; + private CompactionState compactionState = null; + public static CreateDataSegments ofDatasource(String datasource) { return new CreateDataSegments(datasource); @@ -85,7 +89,24 @@ public CreateDataSegments withNumPartitions(int numPartitions) return this; } + public CreateDataSegments withCompactionState(CompactionState compactionState) + { + this.compactionState = compactionState; + return this; + } + + public CreateDataSegments withVersion(String version) + { + this.version = version; + return this; + } + public List eachOfSizeInMb(long sizeMb) + { + return eachOfSize(sizeMb * 1_000_000); + } + + public List eachOfSize(long sizeInBytes) { boolean isEternityInterval = Objects.equals(granularity, Granularities.ALL); if (isEternityInterval) { @@ -105,9 +126,11 @@ public List eachOfSizeInMb(long sizeMb) new NumberedDataSegment( datasource, nextInterval, + version, new NumberedShardSpec(numPartition, numPartitions), ++uniqueIdInInterval, - sizeMb * 1_000_000 + compactionState, + sizeInBytes ) ); } @@ -128,23 +151,26 @@ private static class NumberedDataSegment extends DataSegment private NumberedDataSegment( String datasource, Interval interval, + String version, NumberedShardSpec shardSpec, - int uinqueId, + int uniqueId, + CompactionState compactionState, long size ) { super( datasource, interval, - "1", + version, Collections.emptyMap(), Collections.emptyList(), Collections.emptyList(), shardSpec, + compactionState, IndexIO.CURRENT_VERSION_ID, size ); - this.uniqueId = uinqueId; + this.uniqueId = uniqueId; } @Override From b43993510b964089ffaa7e49214e2541841d5ec5 Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Sun, 1 Sep 2024 10:18:22 +0530 Subject: [PATCH 23/26] Remove unused methods, cleanup tests --- .../client/ControllerChatHandlerTest.java | 6 +- ...ionBasedPublishedSegmentRetrieverTest.java | 15 +- .../actions/SegmentAllocationQueueTest.java | 54 ++--- .../ConcurrentReplaceAndAppendTest.java | 11 +- ...ncurrentReplaceAndStreamingAppendTest.java | 17 +- .../compact/CompactionSupervisorSpecTest.java | 1 + .../OverlordCompactionSchedulerTest.java | 10 +- .../druid/java/util/common/Intervals.java | 5 - .../apache/druid/segment}/TestDataSource.java | 5 +- .../druid/timeline/SegmentTimelineTest.java | 3 +- ...exerSQLMetadataStorageCoordinatorTest.java | 57 ++--- ...SqlMetadataStorageCoordinatorTestBase.java | 19 +- .../metadata/SQLMetadataRuleManagerTest.java | 43 ++-- .../SqlSegmentsMetadataManagerTest.java | 225 +++++++++--------- .../compaction/CompactionStatusTest.java | 18 +- .../CompactionStatusTrackerTest.java | 23 +- .../NewestSegmentFirstPolicyTest.java | 99 ++++---- ...aSourceCompactionConfigAuditEntryTest.java | 12 +- ...DataSourceCompactionConfigHistoryTest.java | 15 +- .../DruidCompactionConfigTest.java | 23 +- .../balancer/CostBalancerStrategyTest.java | 27 +-- .../balancer/ReservoirSegmentSamplerTest.java | 3 +- .../balancer/SegmentToMoveCalculatorTest.java | 5 +- .../duty/KillStalePendingSegmentsTest.java | 73 +++--- .../loading/RoundRobinServerSelectorTest.java | 3 +- .../rules/BroadcastDistributionRuleTest.java | 26 +- .../coordinator/rules/LoadRuleTest.java | 53 ++--- .../CoordinatorSimulationBaseTest.java | 19 +- .../simulate/RoundRobinAssignmentTest.java | 11 +- ...rdinatorCompactionConfigsResourceTest.java | 40 ++-- 30 files changed, 443 insertions(+), 478 deletions(-) rename {indexing-service/src/test/java/org/apache/druid/indexing/compact => processing/src/test/java/org/apache/druid/segment}/TestDataSource.java (92%) diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/client/ControllerChatHandlerTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/client/ControllerChatHandlerTest.java index dd04814d879e..1701d43c1cf5 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/client/ControllerChatHandlerTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/client/ControllerChatHandlerTest.java @@ -22,6 +22,7 @@ import org.apache.druid.indexer.report.KillTaskReport; import org.apache.druid.indexer.report.TaskReport; import org.apache.druid.msq.exec.Controller; +import org.apache.druid.segment.TestDataSource; import org.apache.druid.server.security.AuthorizerMapper; import org.junit.Assert; import org.junit.Test; @@ -33,8 +34,6 @@ public class ControllerChatHandlerTest { - private static final String DATASOURCE = "wiki"; - @Test public void testHttpGetLiveReports() { @@ -47,7 +46,8 @@ public void testHttpGetLiveReports() .thenReturn(reportMap); final AuthorizerMapper authorizerMapper = new AuthorizerMapper(null); - ControllerChatHandler chatHandler = new ControllerChatHandler(controller, DATASOURCE, authorizerMapper); + ControllerChatHandler chatHandler + = new ControllerChatHandler(controller, TestDataSource.WIKI, authorizerMapper); HttpServletRequest httpRequest = Mockito.mock(HttpServletRequest.class); Mockito.when(httpRequest.getAttribute(ArgumentMatchers.anyString())) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/appenderator/ActionBasedPublishedSegmentRetrieverTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/appenderator/ActionBasedPublishedSegmentRetrieverTest.java index dd1cd5c59caa..744d357f4e80 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/appenderator/ActionBasedPublishedSegmentRetrieverTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/appenderator/ActionBasedPublishedSegmentRetrieverTest.java @@ -28,6 +28,7 @@ import org.apache.druid.indexing.overlord.Segments; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.granularity.Granularities; +import org.apache.druid.segment.TestDataSource; import org.apache.druid.server.coordinator.CreateDataSegments; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.SegmentId; @@ -59,7 +60,7 @@ public void setup() public void testRetrieveSegmentsById() throws IOException { final List segments = - CreateDataSegments.ofDatasource("wiki") + CreateDataSegments.ofDatasource(TestDataSource.WIKI) .forIntervals(3, Granularities.DAY) .startingAt("2013-01-01") .eachOfSizeInMb(400); @@ -67,7 +68,7 @@ public void testRetrieveSegmentsById() throws IOException EasyMock.expect( taskActionClient.submit( new RetrieveSegmentsByIdAction( - "wiki", + TestDataSource.WIKI, segments.stream().map(segment -> segment.getId().toString()).collect(Collectors.toSet()) ) ) @@ -89,20 +90,20 @@ public void testRetrieveSegmentsById() throws IOException public void testRetrieveUsedSegmentsIfNotFoundById() throws IOException { final List segments = - CreateDataSegments.ofDatasource("wiki") + CreateDataSegments.ofDatasource(TestDataSource.WIKI) .forIntervals(3, Granularities.DAY) .startingAt("2013-01-01") .eachOfSizeInMb(400); EasyMock.expect( taskActionClient.submit( - new RetrieveSegmentsByIdAction("wiki", EasyMock.anyObject()) + new RetrieveSegmentsByIdAction(TestDataSource.WIKI, EasyMock.anyObject()) ) ).andThrow(InvalidInput.exception("task action not supported yet")).once(); EasyMock.expect( taskActionClient.submit( new RetrieveUsedSegmentsAction( - "wiki", + TestDataSource.WIKI, Collections.singletonList(Intervals.of("2013-01-01/P3D")), Segments.INCLUDING_OVERSHADOWED ) @@ -128,8 +129,8 @@ public void testSegmentsForMultipleDatasourcesThrowsException() DruidException.class, () -> segmentRetriever.findPublishedSegments( ImmutableSet.of( - SegmentId.of("wiki", Intervals.ETERNITY, "v1", 0), - SegmentId.of("koala", Intervals.ETERNITY, "v1", 0) + SegmentId.of(TestDataSource.WIKI, Intervals.ETERNITY, "v1", 0), + SegmentId.of(TestDataSource.KOALA, Intervals.ETERNITY, "v1", 0) ) ) ); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentAllocationQueueTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentAllocationQueueTest.java index a71cacca13a4..6a5c84082f2c 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentAllocationQueueTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentAllocationQueueTest.java @@ -27,6 +27,7 @@ import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.metrics.StubServiceEmitter; +import org.apache.druid.segment.TestDataSource; import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; import org.apache.druid.server.coordinator.simulate.BlockingExecutorService; import org.apache.druid.server.coordinator.simulate.WrappingScheduledExecutorService; @@ -48,9 +49,6 @@ public class SegmentAllocationQueueTest @Rule public TaskActionTestKit taskActionTestKit = new TaskActionTestKit(); - private static final String DS_WIKI = "wiki"; - private static final String DS_KOALA = "koala"; - private SegmentAllocationQueue allocationQueue; private StubServiceEmitter emitter; @@ -105,14 +103,14 @@ public void tearDown() public void testBatchWithMultipleTimestamps() { verifyAllocationWithBatching( - allocateRequest().forTask(createTask(DS_WIKI, "group_1")) + allocateRequest().forTask(createTask(TestDataSource.WIKI, "group_1")) .forTimestamp("2022-01-01T01:00:00") .withSegmentGranularity(Granularities.DAY) .withQueryGranularity(Granularities.SECOND) .withLockGranularity(LockGranularity.TIME_CHUNK) .withSequenceName("seq_1") .build(), - allocateRequest().forTask(createTask(DS_WIKI, "group_1")) + allocateRequest().forTask(createTask(TestDataSource.WIKI, "group_1")) .forTimestamp("2022-01-01T02:00:00") .withSegmentGranularity(Granularities.DAY) .withQueryGranularity(Granularities.SECOND) @@ -127,9 +125,9 @@ public void testBatchWithMultipleTimestamps() public void testBatchWithExclusiveLocks() { verifyAllocationWithBatching( - allocateRequest().forTask(createTask(DS_WIKI, "group_1")) + allocateRequest().forTask(createTask(TestDataSource.WIKI, "group_1")) .withTaskLockType(TaskLockType.EXCLUSIVE).build(), - allocateRequest().forTask(createTask(DS_WIKI, "group_1")) + allocateRequest().forTask(createTask(TestDataSource.WIKI, "group_1")) .withTaskLockType(TaskLockType.EXCLUSIVE).build(), true ); @@ -139,9 +137,9 @@ public void testBatchWithExclusiveLocks() public void testBatchWithSharedLocks() { verifyAllocationWithBatching( - allocateRequest().forTask(createTask(DS_WIKI, "group_1")) + allocateRequest().forTask(createTask(TestDataSource.WIKI, "group_1")) .withTaskLockType(TaskLockType.SHARED).build(), - allocateRequest().forTask(createTask(DS_WIKI, "group_1")) + allocateRequest().forTask(createTask(TestDataSource.WIKI, "group_1")) .withTaskLockType(TaskLockType.SHARED).build(), true ); @@ -151,9 +149,9 @@ public void testBatchWithSharedLocks() public void testBatchWithMultipleQueryGranularities() { verifyAllocationWithBatching( - allocateRequest().forTask(createTask(DS_WIKI, "group_1")) + allocateRequest().forTask(createTask(TestDataSource.WIKI, "group_1")) .withQueryGranularity(Granularities.SECOND).build(), - allocateRequest().forTask(createTask(DS_WIKI, "group_1")) + allocateRequest().forTask(createTask(TestDataSource.WIKI, "group_1")) .withQueryGranularity(Granularities.MINUTE).build(), true ); @@ -163,8 +161,8 @@ public void testBatchWithMultipleQueryGranularities() public void testMultipleDatasourcesCannotBatch() { verifyAllocationWithBatching( - allocateRequest().forTask(createTask(DS_WIKI, "group_1")).build(), - allocateRequest().forTask(createTask(DS_KOALA, "group_1")).build(), + allocateRequest().forTask(createTask(TestDataSource.WIKI, "group_1")).build(), + allocateRequest().forTask(createTask(TestDataSource.KOALA, "group_1")).build(), false ); } @@ -173,8 +171,8 @@ public void testMultipleDatasourcesCannotBatch() public void testMultipleGroupIdsCannotBatch() { verifyAllocationWithBatching( - allocateRequest().forTask(createTask(DS_WIKI, "group_2")).build(), - allocateRequest().forTask(createTask(DS_WIKI, "group_3")).build(), + allocateRequest().forTask(createTask(TestDataSource.WIKI, "group_2")).build(), + allocateRequest().forTask(createTask(TestDataSource.WIKI, "group_3")).build(), false ); } @@ -183,9 +181,9 @@ public void testMultipleGroupIdsCannotBatch() public void testMultipleLockGranularitiesCannotBatch() { verifyAllocationWithBatching( - allocateRequest().forTask(createTask(DS_WIKI, "group_1")) + allocateRequest().forTask(createTask(TestDataSource.WIKI, "group_1")) .withLockGranularity(LockGranularity.TIME_CHUNK).build(), - allocateRequest().forTask(createTask(DS_WIKI, "group_1")) + allocateRequest().forTask(createTask(TestDataSource.WIKI, "group_1")) .withLockGranularity(LockGranularity.SEGMENT).build(), false ); @@ -195,10 +193,10 @@ public void testMultipleLockGranularitiesCannotBatch() public void testMultipleAllocateIntervalsCannotBatch() { verifyAllocationWithBatching( - allocateRequest().forTask(createTask(DS_WIKI, "group_1")) + allocateRequest().forTask(createTask(TestDataSource.WIKI, "group_1")) .forTimestamp("2022-01-01") .withSegmentGranularity(Granularities.DAY).build(), - allocateRequest().forTask(createTask(DS_WIKI, "group_1")) + allocateRequest().forTask(createTask(TestDataSource.WIKI, "group_1")) .forTimestamp("2022-01-02") .withSegmentGranularity(Granularities.DAY).build(), false @@ -209,13 +207,13 @@ public void testMultipleAllocateIntervalsCannotBatch() public void testConflictingPendingSegment() { SegmentAllocateRequest hourSegmentRequest = - allocateRequest().forTask(createTask(DS_WIKI, "group_1")) + allocateRequest().forTask(createTask(TestDataSource.WIKI, "group_1")) .withSegmentGranularity(Granularities.HOUR) .build(); Future hourSegmentFuture = allocationQueue.add(hourSegmentRequest); SegmentAllocateRequest halfHourSegmentRequest = - allocateRequest().forTask(createTask(DS_WIKI, "group_1")) + allocateRequest().forTask(createTask(TestDataSource.WIKI, "group_1")) .withSegmentGranularity(Granularities.THIRTY_MINUTE) .build(); Future halfHourSegmentFuture = allocationQueue.add(halfHourSegmentRequest); @@ -231,12 +229,12 @@ public void testFullAllocationQueue() { for (int i = 0; i < 2000; ++i) { SegmentAllocateRequest request = - allocateRequest().forTask(createTask(DS_WIKI, "group_" + i)).build(); + allocateRequest().forTask(createTask(TestDataSource.WIKI, "group_" + i)).build(); allocationQueue.add(request); } SegmentAllocateRequest request = - allocateRequest().forTask(createTask(DS_WIKI, "next_group")).build(); + allocateRequest().forTask(createTask(TestDataSource.WIKI, "next_group")).build(); Future future = allocationQueue.add(request); // Verify that the future is already complete and segment allocation has failed @@ -253,14 +251,14 @@ public void testMaxBatchSize() { for (int i = 0; i < 500; ++i) { SegmentAllocateRequest request = - allocateRequest().forTask(createTask(DS_WIKI, "group_1")).build(); + allocateRequest().forTask(createTask(TestDataSource.WIKI, "group_1")).build(); allocationQueue.add(request); } // Verify that next request is added to a new batch Assert.assertEquals(1, allocationQueue.size()); SegmentAllocateRequest request = - allocateRequest().forTask(createTask(DS_WIKI, "group_1")).build(); + allocateRequest().forTask(createTask(TestDataSource.WIKI, "group_1")).build(); allocationQueue.add(request); Assert.assertEquals(2, allocationQueue.size()); } @@ -271,7 +269,7 @@ public void testMultipleRequestsForSameSegment() final List> segmentFutures = new ArrayList<>(); for (int i = 0; i < 10; ++i) { SegmentAllocateRequest request = - allocateRequest().forTask(createTask(DS_WIKI, "group_" + i)) + allocateRequest().forTask(createTask(TestDataSource.WIKI, "group_" + i)) .withSequenceName("sequence_1") .withPreviousSegmentId("segment_1") .build(); @@ -299,7 +297,7 @@ public void testRequestsFailOnLeaderChange() final List> segmentFutures = new ArrayList<>(); for (int i = 0; i < 10; ++i) { SegmentAllocateRequest request = - allocateRequest().forTask(createTask(DS_WIKI, "group_" + i)).build(); + allocateRequest().forTask(createTask(TestDataSource.WIKI, "group_" + i)).build(); segmentFutures.add(allocationQueue.add(request)); } @@ -348,7 +346,7 @@ private SegmentIdWithShardSpec getSegmentId(Future futur private SegmentAllocateActionBuilder allocateRequest() { return new SegmentAllocateActionBuilder() - .forDatasource(DS_WIKI) + .forDatasource(TestDataSource.WIKI) .forTimestamp("2022-01-01") .withLockGranularity(LockGranularity.TIME_CHUNK) .withTaskLockType(TaskLockType.SHARED) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/concurrent/ConcurrentReplaceAndAppendTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/concurrent/ConcurrentReplaceAndAppendTest.java index f179e4707eca..96c72e7130d3 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/concurrent/ConcurrentReplaceAndAppendTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/concurrent/ConcurrentReplaceAndAppendTest.java @@ -53,6 +53,7 @@ import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.segment.IndexIO; +import org.apache.druid.segment.TestDataSource; import org.apache.druid.segment.column.ColumnConfig; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; @@ -105,8 +106,6 @@ public class ConcurrentReplaceAndAppendTest extends IngestionTestBase private static final Interval OCT_NOV_DEC_23 = Intervals.of("2023-10-01/2024-01-01"); private static final Interval FIRST_OF_JAN_23 = Intervals.of("2023-01-01/2023-01-02"); - private static final String WIKI = "wiki"; - private TaskQueue taskQueue; private TaskActionClientFactory taskActionClientFactory; private TaskActionClient dummyTaskActionClient; @@ -983,7 +982,7 @@ private void verifySegments(Interval interval, Segments visibility, DataSegment. Collection allUsedSegments = dummyTaskActionClient.submit( new RetrieveUsedSegmentsAction( - WIKI, + TestDataSource.WIKI, ImmutableList.of(interval), visibility ) @@ -1001,7 +1000,7 @@ private void verifyInputSegments(Task task, Interval interval, DataSegment... ex final TaskActionClient taskActionClient = taskActionClientFactory.create(task); Collection allUsedSegments = taskActionClient.submit( new RetrieveUsedSegmentsAction( - WIKI, + TestDataSource.WIKI, Collections.singletonList(interval) ) ); @@ -1037,7 +1036,7 @@ public TaskToolbox build(TaskConfig config, Task task) private DataSegment createSegment(Interval interval, String version) { return DataSegment.builder() - .dataSource(WIKI) + .dataSource(TestDataSource.WIKI) .interval(interval) .version(version) .size(100) @@ -1046,7 +1045,7 @@ private DataSegment createSegment(Interval interval, String version) private ActionsTestTask createAndStartTask() { - ActionsTestTask task = new ActionsTestTask(WIKI, "test_" + groupId.incrementAndGet(), taskActionClientFactory); + ActionsTestTask task = new ActionsTestTask(TestDataSource.WIKI, "test_" + groupId.incrementAndGet(), taskActionClientFactory); taskQueue.add(task); runningTasks.add(task); return task; diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/concurrent/ConcurrentReplaceAndStreamingAppendTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/concurrent/ConcurrentReplaceAndStreamingAppendTest.java index a8ce15538d37..1b2fc1bbde0b 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/concurrent/ConcurrentReplaceAndStreamingAppendTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/concurrent/ConcurrentReplaceAndStreamingAppendTest.java @@ -57,6 +57,7 @@ import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.metadata.PendingSegmentRecord; import org.apache.druid.segment.IndexIO; +import org.apache.druid.segment.TestDataSource; import org.apache.druid.segment.column.ColumnConfig; import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; import org.apache.druid.server.DruidNode; @@ -108,8 +109,6 @@ public class ConcurrentReplaceAndStreamingAppendTest extends IngestionTestBase private static final Interval JAN_23 = Intervals.of("2023-01/2023-02"); private static final Interval FIRST_OF_JAN_23 = Intervals.of("2023-01-01/2023-01-02"); - private static final String WIKI = "wiki"; - private TaskQueue taskQueue; private TaskActionClientFactory taskActionClientFactory; private TaskActionClient dummyTaskActionClient; @@ -130,8 +129,8 @@ public class ConcurrentReplaceAndStreamingAppendTest extends IngestionTestBase public void setUpIngestionTestBase() throws IOException { EasyMock.reset(supervisorManager); - EasyMock.expect(supervisorManager.getActiveSupervisorIdForDatasourceWithAppendLock(WIKI)) - .andReturn(Optional.of(WIKI)).anyTimes(); + EasyMock.expect(supervisorManager.getActiveSupervisorIdForDatasourceWithAppendLock(TestDataSource.WIKI)) + .andReturn(Optional.of(TestDataSource.WIKI)).anyTimes(); super.setUpIngestionTestBase(); final TaskConfig taskConfig = new TaskConfigBuilder().build(); taskActionClientFactory = createActionClientFactory(); @@ -708,7 +707,7 @@ private void verifySegments(Interval interval, Segments visibility, DataSegment. try { Collection allUsedSegments = dummyTaskActionClient.submit( new RetrieveUsedSegmentsAction( - WIKI, + TestDataSource.WIKI, ImmutableList.of(interval), visibility ) @@ -741,9 +740,9 @@ public TaskToolbox build(TaskConfig config, Task task) private DataSegment createSegment(Interval interval, String version) { - SegmentId id = SegmentId.of(WIKI, interval, version, null); + SegmentId id = SegmentId.of(TestDataSource.WIKI, interval, version, null); return DataSegment.builder() - .dataSource(WIKI) + .dataSource(TestDataSource.WIKI) .interval(interval) .version(version) .loadSpec(Collections.singletonMap(id.toString(), id.toString())) @@ -753,7 +752,7 @@ private DataSegment createSegment(Interval interval, String version) private ActionsTestTask createAndStartTask() { - ActionsTestTask task = new ActionsTestTask(WIKI, "test_" + groupId.incrementAndGet(), taskActionClientFactory); + ActionsTestTask task = new ActionsTestTask(TestDataSource.WIKI, "test_" + groupId.incrementAndGet(), taskActionClientFactory); taskQueue.add(task); runningTasks.add(task); return task; @@ -827,7 +826,7 @@ private Collection getAllUsedSegments() try { return dummyTaskActionClient.submit( new RetrieveUsedSegmentsAction( - WIKI, + TestDataSource.WIKI, ImmutableList.of(Intervals.ETERNITY), Segments.INCLUDING_OVERSHADOWED ) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/compact/CompactionSupervisorSpecTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/compact/CompactionSupervisorSpecTest.java index 1b2c1826c06f..2e6a1cf8cc37 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/compact/CompactionSupervisorSpecTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/compact/CompactionSupervisorSpecTest.java @@ -25,6 +25,7 @@ import org.apache.druid.guice.SupervisorModule; import org.apache.druid.indexing.overlord.supervisor.SupervisorSpec; import org.apache.druid.jackson.DefaultObjectMapper; +import org.apache.druid.segment.TestDataSource; import org.apache.druid.server.coordinator.CompactionConfigValidationResult; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; import org.junit.Assert; 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 0496049e9e8c..3133e2b9466a 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 @@ -41,6 +41,7 @@ import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.metrics.StubServiceEmitter; +import org.apache.druid.segment.TestDataSource; import org.apache.druid.segment.TestIndex; import org.apache.druid.server.compaction.CompactionSimulateResult; import org.apache.druid.server.compaction.CompactionStatistics; @@ -346,7 +347,14 @@ public void testRunSimulation() ); Assert.assertEquals( Collections.singletonList( - Arrays.asList("wiki", Intervals.of("2013-01-01/P1D"), 10, 1_000_000_000L, 1, "not compacted yet") + Arrays.asList( + TestDataSource.WIKI, + Intervals.of("2013-01-01/P1D"), + 10, + 1_000_000_000L, + 1, + "not compacted yet" + ) ), pendingCompactionTable.getRows() ); diff --git a/processing/src/main/java/org/apache/druid/java/util/common/Intervals.java b/processing/src/main/java/org/apache/druid/java/util/common/Intervals.java index 4dfe09144e6c..623f546349f9 100644 --- a/processing/src/main/java/org/apache/druid/java/util/common/Intervals.java +++ b/processing/src/main/java/org/apache/druid/java/util/common/Intervals.java @@ -53,11 +53,6 @@ public static Interval of(String format, Object... formatArgs) return of(StringUtils.format(format, formatArgs)); } - public static boolean isEmpty(Interval interval) - { - return interval.getStart().equals(interval.getEnd()); - } - /** * Returns true if the provided interval has endpoints that can be compared against other DateTimes using their * string representations. diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/compact/TestDataSource.java b/processing/src/test/java/org/apache/druid/segment/TestDataSource.java similarity index 92% rename from indexing-service/src/test/java/org/apache/druid/indexing/compact/TestDataSource.java rename to processing/src/test/java/org/apache/druid/segment/TestDataSource.java index f111a6bbbccf..8789d5f3f9a1 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/compact/TestDataSource.java +++ b/processing/src/test/java/org/apache/druid/segment/TestDataSource.java @@ -17,8 +17,11 @@ * under the License. */ -package org.apache.druid.indexing.compact; +package org.apache.druid.segment; +/** + * DataSource names used in tests. + */ public final class TestDataSource { public static final String WIKI = "wiki"; diff --git a/processing/src/test/java/org/apache/druid/timeline/SegmentTimelineTest.java b/processing/src/test/java/org/apache/druid/timeline/SegmentTimelineTest.java index 0d765f245013..64f0ee776d26 100644 --- a/processing/src/test/java/org/apache/druid/timeline/SegmentTimelineTest.java +++ b/processing/src/test/java/org/apache/druid/timeline/SegmentTimelineTest.java @@ -20,6 +20,7 @@ package org.apache.druid.timeline; import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.segment.TestDataSource; import org.apache.druid.timeline.partition.NumberedShardSpec; import org.junit.Assert; import org.junit.Test; @@ -67,7 +68,7 @@ public void testIsOvershadowed() private DataSegment createSegment(String interval, String version, int partitionNum, int totalNumPartitions) { return new DataSegment( - "wiki", + TestDataSource.WIKI, Intervals.of(interval), version, Collections.emptyMap(), diff --git a/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java b/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java index 6eccbccaa849..f82cfbf2a043 100644 --- a/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java +++ b/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java @@ -35,6 +35,7 @@ import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.segment.SegmentSchemaMapping; +import org.apache.druid.segment.TestDataSource; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; import org.apache.druid.segment.metadata.FingerprintGenerator; import org.apache.druid.segment.metadata.SegmentSchemaManager; @@ -174,7 +175,7 @@ public void testCommitAppendSegments() pendingSegmentsForTask.add( new PendingSegmentRecord( new SegmentIdWithShardSpec( - DS.WIKI, + TestDataSource.WIKI, Intervals.of("2023-01-01/2023-02-01"), alreadyUpgradedVersion, new NumberedShardSpec(i, 0) @@ -209,7 +210,7 @@ public void testCommitAppendSegments() pendingSegmentsForTask.add( new PendingSegmentRecord( new SegmentIdWithShardSpec( - DS.WIKI, + TestDataSource.WIKI, Intervals.of("2023-01-01/2023-02-01"), alreadyUpgradedVersion, new NumberedShardSpec(10 + i, 0) @@ -243,7 +244,7 @@ public void testCommitAppendSegments() pendingSegmentsForTask.add( new PendingSegmentRecord( new SegmentIdWithShardSpec( - DS.WIKI, + TestDataSource.WIKI, Intervals.of("2023-01-01/2023-02-01"), alreadyUpgradedVersion, new NumberedShardSpec(20 + i, 0) @@ -257,7 +258,7 @@ public void testCommitAppendSegments() } derbyConnector.retryWithHandle( - handle -> coordinator.insertPendingSegmentsIntoMetastore(handle, pendingSegmentsForTask, DS.WIKI, false) + handle -> coordinator.insertPendingSegmentsIntoMetastore(handle, pendingSegmentsForTask, TestDataSource.WIKI, false) ); final Map segmentToReplaceLock @@ -272,7 +273,7 @@ public void testCommitAppendSegments() Set allCommittedSegments = new HashSet<>(retrieveUsedSegments(derbyConnectorRule.metadataTablesConfigSupplier().get())); Map upgradedFromSegmentIdMap = coordinator.retrieveUpgradedFromSegmentIds( - DS.WIKI, + TestDataSource.WIKI, allCommittedSegments.stream().map(DataSegment::getId).map(SegmentId::toString).collect(Collectors.toSet()) ); // Verify the segments present in the metadata store @@ -901,7 +902,7 @@ public void testRetrieveUsedSegmentsUsingMultipleIntervals() final List intervals = segments.stream().map(DataSegment::getInterval).collect(Collectors.toList()); final Collection actualUsedSegments = coordinator.retrieveUsedSegmentsForIntervals( - DS.WIKI, + TestDataSource.WIKI, intervals, Segments.ONLY_VISIBLE ); @@ -920,7 +921,7 @@ public void testRetrieveAllUsedSegmentsUsingIntervalsOutOfRange() .anyMatch(segment -> !segment.getInterval().overlaps(outOfRangeInterval))); final Collection actualUsedSegments = coordinator.retrieveUsedSegmentsForIntervals( - DS.WIKI, + TestDataSource.WIKI, ImmutableList.of(outOfRangeInterval), Segments.ONLY_VISIBLE ); @@ -934,7 +935,7 @@ public void testRetrieveAllUsedSegmentsUsingNoIntervals() final List segments = createAndGetUsedYearSegments(1900, 2133); final Collection actualUsedSegments = coordinator.retrieveAllUsedSegments( - DS.WIKI, + TestDataSource.WIKI, Segments.ONLY_VISIBLE ); @@ -949,7 +950,7 @@ public void testRetrieveUnusedSegmentsUsingSingleIntervalAndNoLimit() markAllSegmentsUnused(new HashSet<>(segments), DateTimes.nowUtc()); final List actualUnusedSegments = coordinator.retrieveUnusedSegmentsForInterval( - DS.WIKI, + TestDataSource.WIKI, Intervals.of("1900/3000"), null, null @@ -967,7 +968,7 @@ public void testRetrieveUnusedSegmentsUsingSingleIntervalAndLimitAtRange() final int requestedLimit = segments.size(); final List actualUnusedSegments = coordinator.retrieveUnusedSegmentsForInterval( - DS.WIKI, + TestDataSource.WIKI, Intervals.of("1900/3000"), requestedLimit, null @@ -985,7 +986,7 @@ public void testRetrieveUnusedSegmentsUsingSingleIntervalAndLimitInRange() final int requestedLimit = segments.size() - 1; final List actualUnusedSegments = coordinator.retrieveUnusedSegmentsForInterval( - DS.WIKI, + TestDataSource.WIKI, Intervals.of("1900/3000"), requestedLimit, null @@ -1003,7 +1004,7 @@ public void testRetrieveUnusedSegmentsUsingSingleIntervalAndLimitOutOfRange() final int limit = segments.size() + 1; final List actualUnusedSegments = coordinator.retrieveUnusedSegmentsForInterval( - DS.WIKI, + TestDataSource.WIKI, Intervals.of("1900/3000"), limit, null @@ -1024,7 +1025,7 @@ public void testRetrieveUnusedSegmentsUsingSingleIntervalOutOfRange() final int limit = segments.size() + 1; final List actualUnusedSegments = coordinator.retrieveUnusedSegmentsForInterval( - DS.WIKI, + TestDataSource.WIKI, outOfRangeInterval, limit, null @@ -1540,7 +1541,7 @@ public void testRetrieveUnusedSegmentsWithVersions() for (DataSegment unusedSegment : unusedSegments) { Assertions.assertThat( coordinator.retrieveUnusedSegmentsForInterval( - DS.WIKI, + TestDataSource.WIKI, Intervals.of("2023-01-01/2023-01-04"), ImmutableList.of(unusedSegment.getVersion()), null, @@ -1551,7 +1552,7 @@ public void testRetrieveUnusedSegmentsWithVersions() Assertions.assertThat( coordinator.retrieveUnusedSegmentsForInterval( - DS.WIKI, + TestDataSource.WIKI, Intervals.of("2023-01-01/2023-01-04"), ImmutableList.of(v1, v2), null, @@ -1561,7 +1562,7 @@ public void testRetrieveUnusedSegmentsWithVersions() Assertions.assertThat( coordinator.retrieveUnusedSegmentsForInterval( - DS.WIKI, + TestDataSource.WIKI, Intervals.of("2023-01-01/2023-01-04"), null, null, @@ -1571,7 +1572,7 @@ public void testRetrieveUnusedSegmentsWithVersions() Assertions.assertThat( coordinator.retrieveUnusedSegmentsForInterval( - DS.WIKI, + TestDataSource.WIKI, Intervals.of("2023-01-01/2023-01-04"), ImmutableList.of("some-non-existent-version"), null, @@ -3235,7 +3236,7 @@ public void testTimelineVisibilityWith0CorePartitionTombstone() // Allocate and commit a data segment by appending to the same interval final SegmentIdWithShardSpec identifier = coordinator.allocatePendingSegment( - DS.WIKI, + TestDataSource.WIKI, "seq", tombstoneSegment.getVersion(), interval, @@ -3260,7 +3261,7 @@ public void testTimelineVisibilityWith0CorePartitionTombstone() markAllSegmentsUnused(tombstones, DateTimes.nowUtc()); final Collection allUsedSegments = coordinator.retrieveAllUsedSegments( - DS.WIKI, + TestDataSource.WIKI, Segments.ONLY_VISIBLE ); @@ -3290,7 +3291,7 @@ public void testTimelineWith1CorePartitionTombstone() // Allocate and commit a data segment by appending to the same interval final SegmentIdWithShardSpec identifier = coordinator.allocatePendingSegment( - DS.WIKI, + TestDataSource.WIKI, "seq", tombstoneSegment.getVersion(), interval, @@ -3315,7 +3316,7 @@ public void testTimelineWith1CorePartitionTombstone() markAllSegmentsUnused(tombstones, DateTimes.nowUtc()); final Collection allUsedSegments = coordinator.retrieveAllUsedSegments( - DS.WIKI, + TestDataSource.WIKI, Segments.ONLY_VISIBLE ); @@ -3329,7 +3330,7 @@ public void testTimelineWith1CorePartitionTombstone() public void testSegmentIdShouldNotBeReallocated() { final SegmentIdWithShardSpec idWithNullTaskAllocator = coordinator.allocatePendingSegment( - DS.WIKI, + TestDataSource.WIKI, "seq", "0", Intervals.ETERNITY, @@ -3345,7 +3346,7 @@ public void testSegmentIdShouldNotBeReallocated() ); final SegmentIdWithShardSpec idWithValidTaskAllocator = coordinator.allocatePendingSegment( - DS.WIKI, + TestDataSource.WIKI, "seq", "1", Intervals.ETERNITY, @@ -3363,12 +3364,12 @@ public void testSegmentIdShouldNotBeReallocated() // Insert pending segments coordinator.commitSegments(ImmutableSet.of(dataSegment0, dataSegment1), null); // Clean up pending segments corresponding to the valid task allocator id - coordinator.deletePendingSegmentsForTaskAllocatorId(DS.WIKI, "taskAllocatorId"); + coordinator.deletePendingSegmentsForTaskAllocatorId(TestDataSource.WIKI, "taskAllocatorId"); // Mark all segments as unused - coordinator.markSegmentsAsUnusedWithinInterval(DS.WIKI, Intervals.ETERNITY); + coordinator.markSegmentsAsUnusedWithinInterval(TestDataSource.WIKI, Intervals.ETERNITY); final SegmentIdWithShardSpec theId = coordinator.allocatePendingSegment( - DS.WIKI, + TestDataSource.WIKI, "seq", "2", Intervals.ETERNITY, @@ -3406,7 +3407,7 @@ public void testRetrieveUnusedSegmentsForExactIntervalAndVersion() ), null ); - coordinator.markSegmentsAsUnusedWithinInterval(DS.WIKI, Intervals.ETERNITY); + coordinator.markSegmentsAsUnusedWithinInterval(TestDataSource.WIKI, Intervals.ETERNITY); DataSegment usedSegmentForExactIntervalAndVersion = createSegment( Intervals.of("2024/2025"), @@ -3417,7 +3418,7 @@ public void testRetrieveUnusedSegmentsForExactIntervalAndVersion() List unusedSegmentIdsForIntervalAndVersion = - coordinator.retrieveUnusedSegmentIdsForExactIntervalAndVersion(DS.WIKI, Intervals.of("2024/2025"), "v1"); + coordinator.retrieveUnusedSegmentIdsForExactIntervalAndVersion(TestDataSource.WIKI, Intervals.of("2024/2025"), "v1"); Assert.assertEquals(1, unusedSegmentIdsForIntervalAndVersion.size()); Assert.assertEquals( unusedSegmentForExactIntervalAndVersion.getId().toString(), diff --git a/server/src/test/java/org/apache/druid/metadata/IndexerSqlMetadataStorageCoordinatorTestBase.java b/server/src/test/java/org/apache/druid/metadata/IndexerSqlMetadataStorageCoordinatorTestBase.java index 2076e5ffa461..cb0bea36c209 100644 --- a/server/src/test/java/org/apache/druid/metadata/IndexerSqlMetadataStorageCoordinatorTestBase.java +++ b/server/src/test/java/org/apache/druid/metadata/IndexerSqlMetadataStorageCoordinatorTestBase.java @@ -32,6 +32,7 @@ import org.apache.druid.java.util.common.jackson.JacksonUtils; import org.apache.druid.java.util.common.parsers.CloseableIterator; import org.apache.druid.segment.SegmentSchemaMapping; +import org.apache.druid.segment.TestDataSource; import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; import org.apache.druid.segment.metadata.FingerprintGenerator; @@ -310,15 +311,10 @@ public class IndexerSqlMetadataStorageCoordinatorTestBase protected FingerprintGenerator fingerprintGenerator; protected SegmentSchemaTestUtils segmentSchemaTestUtils; - protected static class DS - { - static final String WIKI = "wiki"; - } - protected DataSegment createSegment(Interval interval, String version, ShardSpec shardSpec) { return DataSegment.builder() - .dataSource(DS.WIKI) + .dataSource(TestDataSource.WIKI) .interval(interval) .version(version) .shardSpec(shardSpec) @@ -365,7 +361,7 @@ protected ImmutableList retrieveUnusedSegments( tablesConfig, mapper ) - .retrieveUnusedSegments(DS.WIKI, intervals, null, limit, lastSegmentId, sortOrder, maxUsedStatusLastUpdatedTime)) { + .retrieveUnusedSegments(TestDataSource.WIKI, intervals, null, limit, lastSegmentId, sortOrder, maxUsedStatusLastUpdatedTime)) { return ImmutableList.copyOf(iterator); } } @@ -384,13 +380,8 @@ protected ImmutableList retrieveUnusedSegmentsPlus( return derbyConnector.inReadOnlyTransaction( (handle, status) -> { try (final CloseableIterator iterator = - SqlSegmentsMetadataQuery.forHandle( - handle, - derbyConnector, - tablesConfig, - mapper - ) - .retrieveUnusedSegmentsPlus(DS.WIKI, intervals, null, limit, lastSegmentId, sortOrder, maxUsedStatusLastUpdatedTime)) { + SqlSegmentsMetadataQuery.forHandle(handle, derbyConnector, tablesConfig, mapper) + .retrieveUnusedSegmentsPlus(TestDataSource.WIKI, intervals, null, limit, lastSegmentId, sortOrder, maxUsedStatusLastUpdatedTime)) { return ImmutableList.copyOf(iterator); } } diff --git a/server/src/test/java/org/apache/druid/metadata/SQLMetadataRuleManagerTest.java b/server/src/test/java/org/apache/druid/metadata/SQLMetadataRuleManagerTest.java index efeadbdc04ea..82a87dd15a98 100644 --- a/server/src/test/java/org/apache/druid/metadata/SQLMetadataRuleManagerTest.java +++ b/server/src/test/java/org/apache/druid/metadata/SQLMetadataRuleManagerTest.java @@ -33,6 +33,7 @@ import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.segment.TestDataSource; import org.apache.druid.server.audit.AuditSerdeHelper; import org.apache.druid.server.audit.SQLAuditManager; import org.apache.druid.server.audit.SQLAuditManagerConfig; @@ -52,8 +53,6 @@ public class SQLMetadataRuleManagerTest { - private static final String DATASOURCE = "wiki"; - @org.junit.Rule public final TestDerbyConnector.DerbyConnectorRule derbyConnectorRule = new TestDerbyConnector.DerbyConnectorRule(); @@ -108,12 +107,12 @@ public void testRuleInsert() null ) ); - ruleManager.overrideRule(DATASOURCE, rules, createAuditInfo("override rule")); + ruleManager.overrideRule(TestDataSource.WIKI, rules, createAuditInfo("override rule")); // New rule should be be reflected in the in memory rules map immediately after being set by user Map> allRules = ruleManager.getAllRules(); Assert.assertEquals(1, allRules.size()); - Assert.assertEquals(1, allRules.get(DATASOURCE).size()); - Assert.assertEquals(rules.get(0), allRules.get(DATASOURCE).get(0)); + Assert.assertEquals(1, allRules.get(TestDataSource.WIKI).size()); + Assert.assertEquals(rules.get(0), allRules.get(TestDataSource.WIKI).get(0)); } @Test @@ -122,7 +121,7 @@ public void testOverrideRuleWithNull() // Datasource level rules cannot be null IAE exception = Assert.assertThrows( IAE.class, - () -> ruleManager.overrideRule(DATASOURCE, null, createAuditInfo("null rule")) + () -> ruleManager.overrideRule(TestDataSource.WIKI, null, createAuditInfo("null rule")) ); Assert.assertEquals("Rules cannot be null.", exception.getMessage()); @@ -155,7 +154,7 @@ public void testOverrideRuleWithEmpty() // Datasource level rules can be empty Assert.assertTrue( ruleManager.overrideRule( - DATASOURCE, + TestDataSource.WIKI, Collections.emptyList(), createAuditInfo("empty rule") ) @@ -173,14 +172,14 @@ public void testAuditEntryCreated() throws Exception ) ); final AuditInfo auditInfo = createAuditInfo("create audit entry"); - ruleManager.overrideRule(DATASOURCE, rules, auditInfo); + ruleManager.overrideRule(TestDataSource.WIKI, rules, auditInfo); // fetch rules from metadata storage ruleManager.poll(); - Assert.assertEquals(rules, ruleManager.getRules(DATASOURCE)); + Assert.assertEquals(rules, ruleManager.getRules(TestDataSource.WIKI)); // verify audit entry is created - List auditEntries = auditManager.fetchAuditHistory(DATASOURCE, "rules", null); + List auditEntries = auditManager.fetchAuditHistory(TestDataSource.WIKI, "rules", null); Assert.assertEquals(1, auditEntries.size()); AuditEntry entry = auditEntries.get(0); @@ -189,7 +188,7 @@ public void testAuditEntryCreated() throws Exception mapper.readValue(entry.getPayload().serialized(), new TypeReference>() {}) ); Assert.assertEquals(auditInfo, entry.getAuditInfo()); - Assert.assertEquals(DATASOURCE, entry.getKey()); + Assert.assertEquals(TestDataSource.WIKI, entry.getKey()); } @Test @@ -205,12 +204,12 @@ public void testFetchAuditEntriesForAllDataSources() throws Exception ) ); final AuditInfo auditInfo = createAuditInfo("test_comment"); - ruleManager.overrideRule(DATASOURCE, rules, auditInfo); + ruleManager.overrideRule(TestDataSource.WIKI, rules, auditInfo); ruleManager.overrideRule("test_dataSource2", rules, auditInfo); // fetch rules from metadata storage ruleManager.poll(); - Assert.assertEquals(rules, ruleManager.getRules(DATASOURCE)); + Assert.assertEquals(rules, ruleManager.getRules(TestDataSource.WIKI)); Assert.assertEquals(rules, ruleManager.getRules("test_dataSource2")); // test fetch audit entries @@ -235,13 +234,13 @@ public void testRemoveRulesOlderThanWithNonExistenceDatasourceAndOlderThanTimest null ) ); - ruleManager.overrideRule(DATASOURCE, rules, createAuditInfo("test")); + ruleManager.overrideRule(TestDataSource.WIKI, rules, createAuditInfo("test")); // Verify that the rule was added ruleManager.poll(); Map> allRules = ruleManager.getAllRules(); Assert.assertEquals(1, allRules.size()); - Assert.assertEquals(1, allRules.get(DATASOURCE).size()); + Assert.assertEquals(1, allRules.get(TestDataSource.WIKI).size()); // Now delete rules ruleManager.removeRulesForEmptyDatasourcesOlderThan(System.currentTimeMillis()); @@ -262,13 +261,13 @@ public void testRemoveRulesOlderThanWithNonExistenceDatasourceAndNewerThanTimest null ) ); - ruleManager.overrideRule(DATASOURCE, rules, createAuditInfo("update rules")); + ruleManager.overrideRule(TestDataSource.WIKI, rules, createAuditInfo("update rules")); // Verify that rule was added ruleManager.poll(); Map> allRules = ruleManager.getAllRules(); Assert.assertEquals(1, allRules.size()); - Assert.assertEquals(1, allRules.get(DATASOURCE).size()); + Assert.assertEquals(1, allRules.get(TestDataSource.WIKI).size()); // This will not delete the rule as the rule was created just now so it will have the created timestamp later than // the timestamp 2012-01-01T00:00:00Z @@ -278,7 +277,7 @@ public void testRemoveRulesOlderThanWithNonExistenceDatasourceAndNewerThanTimest ruleManager.poll(); allRules = ruleManager.getAllRules(); Assert.assertEquals(1, allRules.size()); - Assert.assertEquals(1, allRules.get(DATASOURCE).size()); + Assert.assertEquals(1, allRules.get(TestDataSource.WIKI).size()); } @Test @@ -291,17 +290,17 @@ public void testRemoveRulesOlderThanWithActiveDatasourceShouldNotDelete() throws null ) ); - ruleManager.overrideRule(DATASOURCE, rules, createAuditInfo("update rules")); + ruleManager.overrideRule(TestDataSource.WIKI, rules, createAuditInfo("update rules")); // Verify that rule was added ruleManager.poll(); Map> allRules = ruleManager.getAllRules(); Assert.assertEquals(1, allRules.size()); - Assert.assertEquals(1, allRules.get(DATASOURCE).size()); + Assert.assertEquals(1, allRules.get(TestDataSource.WIKI).size()); // Add segment metadata to segment table so that the datasource is considered active DataSegment dataSegment = new DataSegment( - DATASOURCE, + TestDataSource.WIKI, Intervals.of("2015-01-01/2015-02-01"), "1", ImmutableMap.of( @@ -324,7 +323,7 @@ public void testRemoveRulesOlderThanWithActiveDatasourceShouldNotDelete() throws ruleManager.poll(); allRules = ruleManager.getAllRules(); Assert.assertEquals(1, allRules.size()); - Assert.assertEquals(1, allRules.get(DATASOURCE).size()); + Assert.assertEquals(1, allRules.get(TestDataSource.WIKI).size()); } @Test diff --git a/server/src/test/java/org/apache/druid/metadata/SqlSegmentsMetadataManagerTest.java b/server/src/test/java/org/apache/druid/metadata/SqlSegmentsMetadataManagerTest.java index d101a7a74b47..16a0747acaaa 100644 --- a/server/src/test/java/org/apache/druid/metadata/SqlSegmentsMetadataManagerTest.java +++ b/server/src/test/java/org/apache/druid/metadata/SqlSegmentsMetadataManagerTest.java @@ -33,6 +33,7 @@ import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.emitter.EmittingLogger; +import org.apache.druid.segment.TestDataSource; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; import org.apache.druid.segment.metadata.SegmentSchemaCache; import org.apache.druid.segment.metadata.SegmentSchemaManager; @@ -58,12 +59,6 @@ public class SqlSegmentsMetadataManagerTest extends SqlSegmentsMetadataManagerTestBase { - private static class DS - { - static final String WIKI = "wikipedia"; - static final String KOALA = "koala"; - } - private static DataSegment createSegment( String dataSource, String interval, @@ -88,9 +83,9 @@ private static DataSegment createSegment( = new TestDerbyConnector.DerbyConnectorRule(); private final DataSegment wikiSegment1 = - CreateDataSegments.ofDatasource(DS.WIKI).startingAt("2012-03-15").eachOfSizeInMb(500).get(0); + CreateDataSegments.ofDatasource(TestDataSource.WIKI).startingAt("2012-03-15").eachOfSizeInMb(500).get(0); private final DataSegment wikiSegment2 = - CreateDataSegments.ofDatasource(DS.WIKI).startingAt("2012-01-05").eachOfSizeInMb(500).get(0); + CreateDataSegments.ofDatasource(TestDataSource.WIKI).startingAt("2012-01-05").eachOfSizeInMb(500).get(0); private void publishUnusedSegments(DataSegment... segments) throws IOException { @@ -165,7 +160,7 @@ public void testPollEmpty() .stream() .map(ImmutableDruidDataSource::getName).count() ); - Assert.assertNull(sqlSegmentsMetadataManager.getImmutableDataSourceWithUsedSegments(DS.WIKI)); + Assert.assertNull(sqlSegmentsMetadataManager.getImmutableDataSourceWithUsedSegments(TestDataSource.WIKI)); Assert.assertTrue( ImmutableSet.copyOf(sqlSegmentsMetadataManager.iterateAllUsedSegments()).isEmpty() ); @@ -184,11 +179,11 @@ public void testPollPeriodically() Assert.assertTrue(sqlSegmentsMetadataManager.getLatestDatabasePoll() instanceof SqlSegmentsMetadataManager.PeriodicDatabasePoll); dataSourcesSnapshot = sqlSegmentsMetadataManager.getDataSourcesSnapshot(); Assert.assertEquals( - ImmutableSet.of(DS.WIKI), + ImmutableSet.of(TestDataSource.WIKI), sqlSegmentsMetadataManager.retrieveAllDataSourceNames() ); Assert.assertEquals( - ImmutableList.of(DS.WIKI), + ImmutableList.of(TestDataSource.WIKI), dataSourcesSnapshot.getDataSourcesWithAllUsedSegments() .stream() .map(ImmutableDruidDataSource::getName) @@ -196,7 +191,7 @@ public void testPollPeriodically() ); Assert.assertEquals( ImmutableSet.of(wikiSegment1, wikiSegment2), - ImmutableSet.copyOf(dataSourcesSnapshot.getDataSource(DS.WIKI).getSegments()) + ImmutableSet.copyOf(dataSourcesSnapshot.getDataSource(TestDataSource.WIKI).getSegments()) ); Assert.assertEquals( ImmutableSet.of(wikiSegment1, wikiSegment2), @@ -219,11 +214,11 @@ public void testPollOnDemand() Assert.assertTrue(sqlSegmentsMetadataManager.getLatestDatabasePoll() instanceof SqlSegmentsMetadataManager.OnDemandDatabasePoll); dataSourcesSnapshot = sqlSegmentsMetadataManager.getDataSourcesSnapshot(); Assert.assertEquals( - ImmutableSet.of(DS.WIKI), + ImmutableSet.of(TestDataSource.WIKI), sqlSegmentsMetadataManager.retrieveAllDataSourceNames() ); Assert.assertEquals( - ImmutableList.of(DS.WIKI), + ImmutableList.of(TestDataSource.WIKI), dataSourcesSnapshot.getDataSourcesWithAllUsedSegments() .stream() .map(ImmutableDruidDataSource::getName) @@ -231,7 +226,7 @@ public void testPollOnDemand() ); Assert.assertEquals( ImmutableSet.of(wikiSegment1, wikiSegment2), - ImmutableSet.copyOf(dataSourcesSnapshot.getDataSource(DS.WIKI).getSegments()) + ImmutableSet.copyOf(dataSourcesSnapshot.getDataSource(TestDataSource.WIKI).getSegments()) ); Assert.assertEquals( ImmutableSet.of(wikiSegment1, wikiSegment2), @@ -252,13 +247,13 @@ public void testPollPeriodicallyAndOnDemandInterleave() throws Exception Assert.assertTrue(sqlSegmentsMetadataManager.getLatestDatabasePoll() instanceof SqlSegmentsMetadataManager.PeriodicDatabasePoll); dataSourcesSnapshot = sqlSegmentsMetadataManager.getDataSourcesSnapshot(); Assert.assertEquals( - ImmutableList.of(DS.WIKI), + ImmutableList.of(TestDataSource.WIKI), dataSourcesSnapshot.getDataSourcesWithAllUsedSegments() .stream() .map(ImmutableDruidDataSource::getName) .collect(Collectors.toList()) ); - publishSegment(createNewSegment1(DS.KOALA)); + publishSegment(createNewSegment1(TestDataSource.KOALA)); // This call will force on demand poll sqlSegmentsMetadataManager.forceOrWaitOngoingDatabasePoll(); @@ -267,7 +262,7 @@ public void testPollPeriodicallyAndOnDemandInterleave() throws Exception // New datasource should now be in the snapshot since we just force on demand poll. dataSourcesSnapshot = sqlSegmentsMetadataManager.getDataSourcesSnapshot(); Assert.assertEquals( - ImmutableList.of(DS.KOALA, DS.WIKI), + ImmutableList.of(TestDataSource.KOALA, TestDataSource.WIKI), dataSourcesSnapshot.getDataSourcesWithAllUsedSegments() .stream() .map(ImmutableDruidDataSource::getName) @@ -285,7 +280,7 @@ public void testPollPeriodicallyAndOnDemandInterleave() throws Exception Assert.assertTrue(sqlSegmentsMetadataManager.getLatestDatabasePoll() instanceof SqlSegmentsMetadataManager.PeriodicDatabasePoll); dataSourcesSnapshot = sqlSegmentsMetadataManager.getDataSourcesSnapshot(); Assert.assertEquals( - ImmutableSet.of(DS.KOALA, "wikipedia3", DS.WIKI), + ImmutableSet.of(TestDataSource.KOALA, "wikipedia3", TestDataSource.WIKI), dataSourcesSnapshot.getDataSourcesWithAllUsedSegments() .stream() .map(ImmutableDruidDataSource::getName) @@ -300,7 +295,7 @@ public void testPrepareImmutableDataSourceWithUsedSegmentsAwaitsPollOnRestart() DataSegment koalaSegment = pollThenStopThenPublishKoalaSegment(); Assert.assertEquals( ImmutableSet.of(koalaSegment), - ImmutableSet.copyOf(sqlSegmentsMetadataManager.getImmutableDataSourceWithUsedSegments(DS.KOALA).getSegments()) + ImmutableSet.copyOf(sqlSegmentsMetadataManager.getImmutableDataSourceWithUsedSegments(TestDataSource.KOALA).getSegments()) ); } @@ -311,7 +306,7 @@ public void testGetDataSourceWithUsedSegmentsAwaitsPollOnRestart() throws IOExce DataSegment koalaSegment = pollThenStopThenPublishKoalaSegment(); Assert.assertEquals( ImmutableSet.of(koalaSegment), - ImmutableSet.copyOf(sqlSegmentsMetadataManager.getImmutableDataSourceWithUsedSegments(DS.KOALA).getSegments()) + ImmutableSet.copyOf(sqlSegmentsMetadataManager.getImmutableDataSourceWithUsedSegments(TestDataSource.KOALA).getSegments()) ); } @@ -350,10 +345,10 @@ private DataSegment pollThenStopThenPublishKoalaSegment() throws IOException sqlSegmentsMetadataManager.stopPollingDatabasePeriodically(); Assert.assertFalse(sqlSegmentsMetadataManager.isPollingDatabasePeriodically()); Assert.assertEquals( - ImmutableSet.of(DS.WIKI), + ImmutableSet.of(TestDataSource.WIKI), sqlSegmentsMetadataManager.retrieveAllDataSourceNames() ); - final DataSegment koalaSegment = createNewSegment1(DS.KOALA); + final DataSegment koalaSegment = createNewSegment1(TestDataSource.KOALA); publishSegment(koalaSegment); sqlSegmentsMetadataManager.startPollingDatabasePeriodically(); return koalaSegment; @@ -378,7 +373,7 @@ public void testPollWithCorruptedSegment() throws IOException Assert.assertTrue(sqlSegmentsMetadataManager.isPollingDatabasePeriodically()); Assert.assertEquals( - DS.WIKI, + TestDataSource.WIKI, Iterables.getOnlyElement(sqlSegmentsMetadataManager.getImmutableDataSourcesWithAllUsedSegments()).getName() ); } @@ -394,12 +389,12 @@ public void testGetUnusedSegmentIntervals() throws IOException allowUsedFlagLastUpdatedToBeNullable(); Assert.assertTrue(sqlSegmentsMetadataManager.isPollingDatabasePeriodically()); - int numChangedSegments = sqlSegmentsMetadataManager.markAsUnusedAllSegmentsInDataSource(DS.WIKI); + int numChangedSegments = sqlSegmentsMetadataManager.markAsUnusedAllSegmentsInDataSource(TestDataSource.WIKI); Assert.assertEquals(2, numChangedSegments); // Publish an unused segment with used_status_last_updated 2 hours ago final DataSegment koalaSegment1 = createSegment( - DS.KOALA, + TestDataSource.KOALA, "2017-10-15T00:00:00.000/2017-10-16T00:00:00.000", "2017-10-15T20:19:12.565Z" ); @@ -411,7 +406,7 @@ public void testGetUnusedSegmentIntervals() throws IOException // Publish an unused segment with used_status_last_updated 2 days ago final DataSegment koalaSegment2 = createSegment( - DS.KOALA, + TestDataSource.KOALA, "2017-10-16T00:00:00.000/2017-10-17T00:00:00.000", "2017-10-15T20:19:12.565Z" ); @@ -423,7 +418,7 @@ public void testGetUnusedSegmentIntervals() throws IOException // Publish an unused segment and set used_status_last_updated to null final DataSegment koalaSegment3 = createSegment( - DS.KOALA, + TestDataSource.KOALA, "2017-10-17T00:00:00.000/2017-10-18T00:00:00.000", "2017-10-15T20:19:12.565Z" ); @@ -433,7 +428,7 @@ public void testGetUnusedSegmentIntervals() throws IOException Assert.assertEquals( ImmutableList.of(wikiSegment2.getInterval()), sqlSegmentsMetadataManager.getUnusedSegmentIntervals( - DS.WIKI, + TestDataSource.WIKI, null, DateTimes.of("3000"), 1, @@ -445,7 +440,7 @@ public void testGetUnusedSegmentIntervals() throws IOException Assert.assertEquals( ImmutableList.of(wikiSegment2.getInterval()), sqlSegmentsMetadataManager.getUnusedSegmentIntervals( - DS.WIKI, + TestDataSource.WIKI, null, DateTimes.of(2012, 1, 7, 0, 0), 1, @@ -455,7 +450,7 @@ public void testGetUnusedSegmentIntervals() throws IOException Assert.assertEquals( ImmutableList.of(wikiSegment1.getInterval()), sqlSegmentsMetadataManager.getUnusedSegmentIntervals( - DS.WIKI, + TestDataSource.WIKI, DateTimes.of(2012, 1, 7, 0, 0), DateTimes.of(2012, 4, 7, 0, 0), 1, @@ -465,7 +460,7 @@ public void testGetUnusedSegmentIntervals() throws IOException Assert.assertEquals( ImmutableList.of(), sqlSegmentsMetadataManager.getUnusedSegmentIntervals( - DS.WIKI, + TestDataSource.WIKI, DateTimes.of(2012, 1, 7, 0, 0), DateTimes.of(2012, 1, 7, 0, 0), 1, @@ -476,7 +471,7 @@ public void testGetUnusedSegmentIntervals() throws IOException Assert.assertEquals( ImmutableList.of(wikiSegment2.getInterval(), wikiSegment1.getInterval()), sqlSegmentsMetadataManager.getUnusedSegmentIntervals( - DS.WIKI, + TestDataSource.WIKI, null, DateTimes.of("3000"), 5, @@ -490,7 +485,7 @@ public void testGetUnusedSegmentIntervals() throws IOException Assert.assertEquals( ImmutableList.of(), sqlSegmentsMetadataManager.getUnusedSegmentIntervals( - DS.WIKI, + TestDataSource.WIKI, DateTimes.COMPARE_DATE_AS_STRING_MIN, DateTimes.of("3000"), 5, @@ -504,7 +499,7 @@ public void testGetUnusedSegmentIntervals() throws IOException Assert.assertEquals( ImmutableList.of(koalaSegment2.getInterval()), sqlSegmentsMetadataManager.getUnusedSegmentIntervals( - DS.KOALA, + TestDataSource.KOALA, DateTimes.COMPARE_DATE_AS_STRING_MIN, DateTimes.of("3000"), 5, @@ -520,13 +515,13 @@ public void testMarkAsUnusedAllSegmentsInDataSource() throws IOException, Interr sqlSegmentsMetadataManager.poll(); Assert.assertTrue(sqlSegmentsMetadataManager.isPollingDatabasePeriodically()); - publishSegment(createNewSegment1(DS.KOALA)); + publishSegment(createNewSegment1(TestDataSource.KOALA)); - awaitDataSourceAppeared(DS.KOALA); - int numChangedSegments = sqlSegmentsMetadataManager.markAsUnusedAllSegmentsInDataSource(DS.KOALA); + awaitDataSourceAppeared(TestDataSource.KOALA); + int numChangedSegments = sqlSegmentsMetadataManager.markAsUnusedAllSegmentsInDataSource(TestDataSource.KOALA); Assert.assertEquals(1, numChangedSegments); - awaitDataSourceDisappeared(DS.KOALA); - Assert.assertNull(sqlSegmentsMetadataManager.getImmutableDataSourceWithUsedSegments(DS.KOALA)); + awaitDataSourceDisappeared(TestDataSource.KOALA); + Assert.assertNull(sqlSegmentsMetadataManager.getImmutableDataSourceWithUsedSegments(TestDataSource.KOALA)); } private static DataSegment createNewSegment1(String datasource) @@ -555,18 +550,18 @@ public void testMarkSegmentAsUnused() throws IOException, InterruptedException Assert.assertTrue(sqlSegmentsMetadataManager.isPollingDatabasePeriodically()); final DataSegment koalaSegment = createSegment( - DS.KOALA, + TestDataSource.KOALA, "2017-10-15T00:00:00.000/2017-10-16T00:00:00.000", "2017-10-15T20:19:12.565Z" ); publishSegment(koalaSegment); - awaitDataSourceAppeared(DS.KOALA); - Assert.assertNotNull(sqlSegmentsMetadataManager.getImmutableDataSourceWithUsedSegments(DS.KOALA)); + awaitDataSourceAppeared(TestDataSource.KOALA); + Assert.assertNotNull(sqlSegmentsMetadataManager.getImmutableDataSourceWithUsedSegments(TestDataSource.KOALA)); Assert.assertTrue(sqlSegmentsMetadataManager.markSegmentAsUnused(koalaSegment.getId())); - awaitDataSourceDisappeared(DS.KOALA); - Assert.assertNull(sqlSegmentsMetadataManager.getImmutableDataSourceWithUsedSegments(DS.KOALA)); + awaitDataSourceDisappeared(TestDataSource.KOALA); + Assert.assertNull(sqlSegmentsMetadataManager.getImmutableDataSourceWithUsedSegments(TestDataSource.KOALA)); } private void awaitDataSourceAppeared(String datasource) throws InterruptedException @@ -592,20 +587,20 @@ public void testMarkAsUsedNonOvershadowedSegments() throws Exception Assert.assertTrue(sqlSegmentsMetadataManager.isPollingDatabasePeriodically()); final DataSegment koalaSegment1 = createSegment( - DS.KOALA, + TestDataSource.KOALA, "2017-10-15T00:00:00.000/2017-10-17T00:00:00.000", "2017-10-15T20:19:12.565Z" ); final DataSegment koalaSegment2 = createSegment( - DS.KOALA, + TestDataSource.KOALA, "2017-10-17T00:00:00.000/2017-10-18T00:00:00.000", "2017-10-16T20:19:12.565Z" ); // Overshadowed by koalaSegment2 final DataSegment koalaSegment3 = createSegment( - DS.KOALA, + TestDataSource.KOALA, "2017-10-17T00:00:00.000/2017-10-18T00:00:00.000", "2017-10-15T20:19:12.565Z" ); @@ -622,7 +617,7 @@ public void testMarkAsUsedNonOvershadowedSegments() throws Exception ImmutableSet.of(wikiSegment1, wikiSegment2), ImmutableSet.copyOf(sqlSegmentsMetadataManager.iterateAllUsedSegments()) ); - Assert.assertEquals(2, sqlSegmentsMetadataManager.markAsUsedNonOvershadowedSegments(DS.KOALA, segmentIds)); + Assert.assertEquals(2, sqlSegmentsMetadataManager.markAsUsedNonOvershadowedSegments(TestDataSource.KOALA, segmentIds)); sqlSegmentsMetadataManager.poll(); Assert.assertEquals( ImmutableSet.of(wikiSegment1, wikiSegment2, koalaSegment1, koalaSegment2), @@ -639,20 +634,20 @@ public void testMarkAsUsedNonOvershadowedSegmentsInEternityIntervalWithVersions( Assert.assertTrue(sqlSegmentsMetadataManager.isPollingDatabasePeriodically()); final DataSegment koalaSegment1 = createSegment( - DS.KOALA, + TestDataSource.KOALA, "2017-10-15T00:00:00.000/2017-10-17T00:00:00.000", "2017-10-15T20:19:12.565Z" ); final DataSegment koalaSegment2 = createSegment( - DS.KOALA, + TestDataSource.KOALA, "2017-10-17T00:00:00.000/2017-10-18T00:00:00.000", "2017-10-16T20:19:12.565Z" ); // Overshadowed by koalaSegment2 final DataSegment koalaSegment3 = createSegment( - DS.KOALA, + TestDataSource.KOALA, "2017-10-17T00:00:00.000/2017-10-18T00:00:00.000", "2017-10-15T20:19:12.565Z" ); @@ -667,7 +662,7 @@ public void testMarkAsUsedNonOvershadowedSegmentsInEternityIntervalWithVersions( Assert.assertEquals( 2, sqlSegmentsMetadataManager.markAsUsedNonOvershadowedSegmentsInInterval( - DS.KOALA, + TestDataSource.KOALA, Intervals.ETERNITY, ImmutableList.of("2017-10-15T20:19:12.565Z", "2017-10-16T20:19:12.565Z") ) @@ -689,20 +684,20 @@ public void testMarkAsUsedNonOvershadowedSegmentsInIntervalWithEmptyVersions() t Assert.assertTrue(sqlSegmentsMetadataManager.isPollingDatabasePeriodically()); final DataSegment koalaSegment1 = createSegment( - DS.KOALA, + TestDataSource.KOALA, "2017-10-15T00:00:00.000/2017-10-17T00:00:00.000", "2017-10-15T20:19:12.565Z" ); final DataSegment koalaSegment2 = createSegment( - DS.KOALA, + TestDataSource.KOALA, "2017-10-17T00:00:00.000/2017-10-18T00:00:00.000", "2017-10-16T20:19:12.565Z" ); // Overshadowed by koalaSegment2 final DataSegment koalaSegment3 = createSegment( - DS.KOALA, + TestDataSource.KOALA, "2017-10-17T00:00:00.000/2017-10-18T00:00:00.000", "2017-10-15T20:19:12.565Z" ); @@ -717,7 +712,7 @@ public void testMarkAsUsedNonOvershadowedSegmentsInIntervalWithEmptyVersions() t Assert.assertEquals( 0, sqlSegmentsMetadataManager.markAsUsedNonOvershadowedSegmentsInInterval( - DS.KOALA, + TestDataSource.KOALA, Intervals.of("2017/2018"), ImmutableList.of() ) @@ -739,20 +734,20 @@ public void testMarkAsUsedNonOvershadowedSegmentsInEternityIntervalWithEmptyVers Assert.assertTrue(sqlSegmentsMetadataManager.isPollingDatabasePeriodically()); final DataSegment koalaSegment1 = createSegment( - DS.KOALA, + TestDataSource.KOALA, "2017-10-15T00:00:00.000/2017-10-17T00:00:00.000", "2017-10-15T20:19:12.565Z" ); final DataSegment koalaSegment2 = createSegment( - DS.KOALA, + TestDataSource.KOALA, "2017-10-17T00:00:00.000/2017-10-18T00:00:00.000", "2017-10-16T20:19:12.565Z" ); // Overshadowed by koalaSegment2 final DataSegment koalaSegment3 = createSegment( - DS.KOALA, + TestDataSource.KOALA, "2017-10-17T00:00:00.000/2017-10-18T00:00:00.000", "2017-10-15T20:19:12.565Z" ); @@ -767,7 +762,7 @@ public void testMarkAsUsedNonOvershadowedSegmentsInEternityIntervalWithEmptyVers Assert.assertEquals( 0, sqlSegmentsMetadataManager.markAsUsedNonOvershadowedSegmentsInInterval( - DS.KOALA, + TestDataSource.KOALA, Intervals.ETERNITY, ImmutableList.of() ) @@ -789,20 +784,20 @@ public void testMarkAsUsedNonOvershadowedSegmentsInFiniteIntervalWithVersions() Assert.assertTrue(sqlSegmentsMetadataManager.isPollingDatabasePeriodically()); final DataSegment koalaSegment1 = createSegment( - DS.KOALA, + TestDataSource.KOALA, "2017-10-15T00:00:00.000/2017-10-17T00:00:00.000", "2017-10-15T20:19:12.565Z" ); final DataSegment koalaSegment2 = createSegment( - DS.KOALA, + TestDataSource.KOALA, "2017-10-17T00:00:00.000/2017-10-18T00:00:00.000", "2017-10-16T20:19:12.565Z" ); // Overshadowed by koalaSegment2 final DataSegment koalaSegment3 = createSegment( - DS.KOALA, + TestDataSource.KOALA, "2017-10-17T00:00:00.000/2017-10-18T00:00:00.000", "2017-10-15T20:19:12.565Z" ); @@ -817,7 +812,7 @@ public void testMarkAsUsedNonOvershadowedSegmentsInFiniteIntervalWithVersions() Assert.assertEquals( 2, sqlSegmentsMetadataManager.markAsUsedNonOvershadowedSegmentsInInterval( - DS.KOALA, + TestDataSource.KOALA, Intervals.of("2017-10-15/2017-10-18"), ImmutableList.of("2017-10-15T20:19:12.565Z", "2017-10-16T20:19:12.565Z") ) @@ -839,20 +834,20 @@ public void testMarkAsUsedNonOvershadowedSegmentsWithNonExistentVersions() throw Assert.assertTrue(sqlSegmentsMetadataManager.isPollingDatabasePeriodically()); final DataSegment koalaSegment1 = createSegment( - DS.KOALA, + TestDataSource.KOALA, "2017-10-15T00:00:00.000/2017-10-17T00:00:00.000", "2017-10-15T20:19:12.565Z" ); final DataSegment koalaSegment2 = createSegment( - DS.KOALA, + TestDataSource.KOALA, "2017-10-17T00:00:00.000/2017-10-18T00:00:00.000", "2017-10-16T20:19:12.565Z" ); // Overshadowed by koalaSegment2 final DataSegment koalaSegment3 = createSegment( - DS.KOALA, + TestDataSource.KOALA, "2017-10-17T00:00:00.000/2017-10-18T00:00:00.000", "2017-10-15T20:19:12.565Z" ); @@ -867,7 +862,7 @@ public void testMarkAsUsedNonOvershadowedSegmentsWithNonExistentVersions() throw Assert.assertEquals( 0, sqlSegmentsMetadataManager.markAsUsedNonOvershadowedSegmentsInInterval( - DS.KOALA, + TestDataSource.KOALA, Intervals.ETERNITY, ImmutableList.of("foo", "bar") ) @@ -888,8 +883,8 @@ public void testMarkAsUsedNonOvershadowedSegmentsInvalidDataSource() throws Exce sqlSegmentsMetadataManager.poll(); Assert.assertTrue(sqlSegmentsMetadataManager.isPollingDatabasePeriodically()); - final DataSegment koalaSegment1 = createNewSegment1(DS.KOALA); - final DataSegment koalaSegment2 = createNewSegment1(DS.KOALA); + final DataSegment koalaSegment1 = createNewSegment1(TestDataSource.KOALA); + final DataSegment koalaSegment2 = createNewSegment1(TestDataSource.KOALA); publishUnusedSegments(koalaSegment1, koalaSegment2); final ImmutableSet segmentIds = @@ -919,8 +914,8 @@ public void testMarkAsUsedNonOvershadowedSegmentsWithInvalidSegmentIds() sqlSegmentsMetadataManager.poll(); Assert.assertTrue(sqlSegmentsMetadataManager.isPollingDatabasePeriodically()); - final DataSegment koalaSegment1 = createNewSegment1(DS.KOALA); - final DataSegment koalaSegment2 = createNewSegment1(DS.KOALA); + final DataSegment koalaSegment1 = createNewSegment1(TestDataSource.KOALA); + final DataSegment koalaSegment2 = createNewSegment1(TestDataSource.KOALA); final ImmutableSet segmentIds = ImmutableSet.of(koalaSegment1.getId().toString(), koalaSegment2.getId().toString()); @@ -933,7 +928,7 @@ public void testMarkAsUsedNonOvershadowedSegmentsWithInvalidSegmentIds() MatcherAssert.assertThat( Assert.assertThrows( DruidException.class, - () -> sqlSegmentsMetadataManager.markAsUsedNonOvershadowedSegments(DS.KOALA, segmentIds) + () -> sqlSegmentsMetadataManager.markAsUsedNonOvershadowedSegments(TestDataSource.KOALA, segmentIds) ), DruidExceptionMatcher .invalidInput() @@ -949,20 +944,20 @@ public void testMarkAsUsedNonOvershadowedSegmentsInInterval() throws IOException sqlSegmentsMetadataManager.poll(); Assert.assertTrue(sqlSegmentsMetadataManager.isPollingDatabasePeriodically()); - final DataSegment koalaSegment1 = createNewSegment1(DS.KOALA); + final DataSegment koalaSegment1 = createNewSegment1(TestDataSource.KOALA); final DataSegment koalaSegment2 = createSegment( - DS.KOALA, + TestDataSource.KOALA, "2017-10-17T00:00:00.000/2017-10-18T00:00:00.000", "2017-10-16T20:19:12.565Z" ); final DataSegment koalaSegment3 = createSegment( - DS.KOALA, + TestDataSource.KOALA, "2017-10-19T00:00:00.000/2017-10-20T00:00:00.000", "2017-10-15T20:19:12.565Z" ); // Overshadowed by koalaSegment2 - final DataSegment koalaSegment4 = createNewSegment2(DS.KOALA); + final DataSegment koalaSegment4 = createNewSegment2(TestDataSource.KOALA); publishUnusedSegments(koalaSegment1, koalaSegment2, koalaSegment3, koalaSegment4); final Interval theInterval = Intervals.of("2017-10-15T00:00:00.000/2017-10-18T00:00:00.000"); @@ -974,7 +969,7 @@ public void testMarkAsUsedNonOvershadowedSegmentsInInterval() throws IOException ); // 2 out of 3 segments match the interval - Assert.assertEquals(2, sqlSegmentsMetadataManager.markAsUsedNonOvershadowedSegmentsInInterval(DS.KOALA, theInterval, null)); + Assert.assertEquals(2, sqlSegmentsMetadataManager.markAsUsedNonOvershadowedSegmentsInInterval(TestDataSource.KOALA, theInterval, null)); sqlSegmentsMetadataManager.poll(); Assert.assertEquals( @@ -992,25 +987,25 @@ public void testMarkAsUsedNonOvershadowedSegmentsInIntervalWithOverlappingInterv Assert.assertTrue(sqlSegmentsMetadataManager.isPollingDatabasePeriodically()); final DataSegment koalaSegment1 = createSegment( - DS.KOALA, + TestDataSource.KOALA, "2017-10-15T00:00:00.000/2017-10-17T00:00:00.000", "2017-10-15T20:19:12.565Z" ); final DataSegment koalaSegment2 = createSegment( - DS.KOALA, + TestDataSource.KOALA, "2017-10-17T00:00:00.000/2017-10-18T00:00:00.000", "2017-10-16T20:19:12.565Z" ); final DataSegment koalaSegment3 = createSegment( - DS.KOALA, + TestDataSource.KOALA, "2017-10-19T00:00:00.000/2017-10-22T00:00:00.000", "2017-10-15T20:19:12.565Z" ); // Overshadowed by koalaSegment2 - final DataSegment koalaSegment4 = createNewSegment2(DS.KOALA); + final DataSegment koalaSegment4 = createNewSegment2(TestDataSource.KOALA); publishUnusedSegments(koalaSegment1, koalaSegment2, koalaSegment3, koalaSegment4); final Interval theInterval = Intervals.of("2017-10-16T00:00:00.000/2017-10-20T00:00:00.000"); @@ -1022,7 +1017,7 @@ public void testMarkAsUsedNonOvershadowedSegmentsInIntervalWithOverlappingInterv ); // 1 out of 3 segments match the interval, other 2 overlap, only the segment fully contained will be marked unused - Assert.assertEquals(1, sqlSegmentsMetadataManager.markAsUsedNonOvershadowedSegmentsInInterval(DS.KOALA, theInterval, null)); + Assert.assertEquals(1, sqlSegmentsMetadataManager.markAsUsedNonOvershadowedSegmentsInInterval(TestDataSource.KOALA, theInterval, null)); sqlSegmentsMetadataManager.poll(); Assert.assertEquals( @@ -1039,8 +1034,8 @@ public void testMarkSegmentsAsUnused() throws IOException sqlSegmentsMetadataManager.poll(); Assert.assertTrue(sqlSegmentsMetadataManager.isPollingDatabasePeriodically()); - final DataSegment koalaSegment1 = createNewSegment1(DS.KOALA); - final DataSegment koalaSegment2 = createNewSegment1(DS.KOALA); + final DataSegment koalaSegment1 = createNewSegment1(TestDataSource.KOALA); + final DataSegment koalaSegment2 = createNewSegment1(TestDataSource.KOALA); publishSegment(koalaSegment1); publishSegment(koalaSegment2); @@ -1064,10 +1059,10 @@ public void testMarkAsUnusedSegmentsInInterval() throws IOException sqlSegmentsMetadataManager.poll(); Assert.assertTrue(sqlSegmentsMetadataManager.isPollingDatabasePeriodically()); - final DataSegment koalaSegment1 = createNewSegment1(DS.KOALA); - final DataSegment koalaSegment2 = createNewSegment2(DS.KOALA); + final DataSegment koalaSegment1 = createNewSegment1(TestDataSource.KOALA); + final DataSegment koalaSegment2 = createNewSegment2(TestDataSource.KOALA); final DataSegment koalaSegment3 = createSegment( - DS.KOALA, + TestDataSource.KOALA, "2017-10-19T00:00:00.000/2017-10-20T00:00:00.000", "2017-10-15T20:19:12.565Z" ); @@ -1078,7 +1073,7 @@ public void testMarkAsUnusedSegmentsInInterval() throws IOException final Interval theInterval = Intervals.of("2017-10-15T00:00:00.000/2017-10-18T00:00:00.000"); // 2 out of 3 segments match the interval - Assert.assertEquals(2, sqlSegmentsMetadataManager.markAsUnusedSegmentsInInterval(DS.KOALA, theInterval, null)); + Assert.assertEquals(2, sqlSegmentsMetadataManager.markAsUnusedSegmentsInInterval(TestDataSource.KOALA, theInterval, null)); sqlSegmentsMetadataManager.poll(); Assert.assertEquals( @@ -1100,17 +1095,17 @@ public void testMarkAsUnusedSegmentsInIntervalAndVersions() throws IOException final String v2 = now.plus(Duration.standardDays(1)).toString(); final DataSegment koalaSegment1 = createSegment( - DS.KOALA, + TestDataSource.KOALA, "2017-10-15T00:00:00.000/2017-10-16T00:00:00.000", v1 ); final DataSegment koalaSegment2 = createSegment( - DS.KOALA, + TestDataSource.KOALA, "2017-10-17T00:00:00.000/2017-10-18T00:00:00.000", v2 ); final DataSegment koalaSegment3 = createSegment( - DS.KOALA, + TestDataSource.KOALA, "2017-10-19T00:00:00.000/2017-10-20T00:00:00.000", v2 ); @@ -1123,7 +1118,7 @@ public void testMarkAsUnusedSegmentsInIntervalAndVersions() throws IOException Assert.assertEquals( 2, sqlSegmentsMetadataManager.markAsUnusedSegmentsInInterval( - DS.KOALA, + TestDataSource.KOALA, theInterval, ImmutableList.of(v1, v2) ) @@ -1149,17 +1144,17 @@ public void testMarkAsUnusedSegmentsInIntervalAndNonExistentVersions() throws IO final String v2 = now.plus(Duration.standardDays(1)).toString(); final DataSegment koalaSegment1 = createSegment( - DS.KOALA, + TestDataSource.KOALA, "2017-10-15T00:00:00.000/2017-10-16T00:00:00.000", v1 ); final DataSegment koalaSegment2 = createSegment( - DS.KOALA, + TestDataSource.KOALA, "2017-10-17T00:00:00.000/2017-10-18T00:00:00.000", v2 ); final DataSegment koalaSegment3 = createSegment( - DS.KOALA, + TestDataSource.KOALA, "2017-10-19T00:00:00.000/2017-10-20T00:00:00.000", v2 ); @@ -1172,7 +1167,7 @@ public void testMarkAsUnusedSegmentsInIntervalAndNonExistentVersions() throws IO Assert.assertEquals( 0, sqlSegmentsMetadataManager.markAsUnusedSegmentsInInterval( - DS.KOALA, + TestDataSource.KOALA, theInterval, ImmutableList.of("foo", "bar", "baz") ) @@ -1198,17 +1193,17 @@ public void testMarkAsUnusedSegmentsInIntervalWithEmptyVersions() throws IOExcep final String v2 = now.plus(Duration.standardDays(1)).toString(); final DataSegment koalaSegment1 = createSegment( - DS.KOALA, + TestDataSource.KOALA, "2017-10-15T00:00:00.000/2017-10-16T00:00:00.000", v1 ); final DataSegment koalaSegment2 = createSegment( - DS.KOALA, + TestDataSource.KOALA, "2017-10-17T00:00:00.000/2017-10-18T00:00:00.000", v2 ); final DataSegment koalaSegment3 = createSegment( - DS.KOALA, + TestDataSource.KOALA, "2017-10-19T00:00:00.000/2017-10-20T00:00:00.000", v2 ); @@ -1221,7 +1216,7 @@ public void testMarkAsUnusedSegmentsInIntervalWithEmptyVersions() throws IOExcep Assert.assertEquals( 0, sqlSegmentsMetadataManager.markAsUnusedSegmentsInInterval( - DS.KOALA, + TestDataSource.KOALA, theInterval, ImmutableList.of() ) @@ -1247,17 +1242,17 @@ public void testMarkAsUnusedSegmentsInEternityIntervalWithEmptyVersions() throws final String v2 = now.plus(Duration.standardDays(1)).toString(); final DataSegment koalaSegment1 = createSegment( - DS.KOALA, + TestDataSource.KOALA, "2017-10-15T00:00:00.000/2017-10-16T00:00:00.000", v1 ); final DataSegment koalaSegment2 = createSegment( - DS.KOALA, + TestDataSource.KOALA, "2017-10-17T00:00:00.000/2017-10-18T00:00:00.000", v2 ); final DataSegment koalaSegment3 = createSegment( - DS.KOALA, + TestDataSource.KOALA, "2017-10-19T00:00:00.000/2017-10-20T00:00:00.000", v2 ); @@ -1270,7 +1265,7 @@ public void testMarkAsUnusedSegmentsInEternityIntervalWithEmptyVersions() throws Assert.assertEquals( 0, sqlSegmentsMetadataManager.markAsUnusedSegmentsInInterval( - DS.KOALA, + TestDataSource.KOALA, theInterval, ImmutableList.of() ) @@ -1292,13 +1287,13 @@ public void testMarkAsUnusedSegmentsInIntervalWithOverlappingInterval() throws I Assert.assertTrue(sqlSegmentsMetadataManager.isPollingDatabasePeriodically()); final DataSegment koalaSegment1 = createSegment( - DS.KOALA, + TestDataSource.KOALA, "2017-10-15T00:00:00.000/2017-10-17T00:00:00.000", "2017-10-15T20:19:12.565Z" ); - final DataSegment koalaSegment2 = createNewSegment2(DS.KOALA); + final DataSegment koalaSegment2 = createNewSegment2(TestDataSource.KOALA); final DataSegment koalaSegment3 = createSegment( - DS.KOALA, + TestDataSource.KOALA, "2017-10-19T00:00:00.000/2017-10-22T00:00:00.000", "2017-10-15T20:19:12.565Z" ); @@ -1309,7 +1304,7 @@ public void testMarkAsUnusedSegmentsInIntervalWithOverlappingInterval() throws I final Interval theInterval = Intervals.of("2017-10-16T00:00:00.000/2017-10-20T00:00:00.000"); // 1 out of 3 segments match the interval, other 2 overlap, only the segment fully contained will be marked unused - Assert.assertEquals(1, sqlSegmentsMetadataManager.markAsUnusedSegmentsInInterval(DS.KOALA, theInterval, null)); + Assert.assertEquals(1, sqlSegmentsMetadataManager.markAsUnusedSegmentsInInterval(TestDataSource.KOALA, theInterval, null)); sqlSegmentsMetadataManager.poll(); Assert.assertEquals( @@ -1348,14 +1343,14 @@ public void testIterateAllUsedNonOvershadowedSegmentsForDatasourceInterval() thr sqlSegmentsMetadataManager.start(); Optional> segments = sqlSegmentsMetadataManager - .iterateAllUsedNonOvershadowedSegmentsForDatasourceInterval(DS.WIKI, theInterval, true); + .iterateAllUsedNonOvershadowedSegmentsForDatasourceInterval(TestDataSource.WIKI, theInterval, true); Assert.assertTrue(segments.isPresent()); Set dataSegmentSet = ImmutableSet.copyOf(segments.get()); Assert.assertEquals(1, dataSegmentSet.size()); Assert.assertTrue(dataSegmentSet.contains(wikiSegment1)); final DataSegment wikiSegment3 = createSegment( - DS.WIKI, + TestDataSource.WIKI, "2012-03-16T00:00:00.000/2012-03-17T00:00:00.000", "2017-10-15T20:19:12.565Z" ); @@ -1363,7 +1358,7 @@ public void testIterateAllUsedNonOvershadowedSegmentsForDatasourceInterval() thr // New segment is not returned since we call without force poll segments = sqlSegmentsMetadataManager - .iterateAllUsedNonOvershadowedSegmentsForDatasourceInterval(DS.WIKI, theInterval, false); + .iterateAllUsedNonOvershadowedSegmentsForDatasourceInterval(TestDataSource.WIKI, theInterval, false); Assert.assertTrue(segments.isPresent()); dataSegmentSet = ImmutableSet.copyOf(segments.get()); Assert.assertEquals(1, dataSegmentSet.size()); @@ -1371,7 +1366,7 @@ public void testIterateAllUsedNonOvershadowedSegmentsForDatasourceInterval() thr // New segment is returned since we call with force poll segments = sqlSegmentsMetadataManager - .iterateAllUsedNonOvershadowedSegmentsForDatasourceInterval(DS.WIKI, theInterval, true); + .iterateAllUsedNonOvershadowedSegmentsForDatasourceInterval(TestDataSource.WIKI, theInterval, true); Assert.assertTrue(segments.isPresent()); dataSegmentSet = ImmutableSet.copyOf(segments.get()); Assert.assertEquals(2, dataSegmentSet.size()); @@ -1384,7 +1379,7 @@ public void testPopulateUsedFlagLastUpdated() throws IOException { allowUsedFlagLastUpdatedToBeNullable(); final DataSegment koalaSegment = createSegment( - DS.KOALA, + TestDataSource.KOALA, "2017-10-17T00:00:00.000/2017-10-18T00:00:00.000", "2017-10-15T20:19:12.565Z" ); 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 841f49f70ecf..ac7bf28a4b08 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 @@ -29,6 +29,7 @@ import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.segment.IndexSpec; +import org.apache.druid.segment.TestDataSource; import org.apache.druid.segment.data.CompressionStrategy; import org.apache.druid.segment.indexing.granularity.GranularitySpec; import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; @@ -45,11 +46,10 @@ public class CompactionStatusTest { private static final ObjectMapper OBJECT_MAPPER = new DefaultObjectMapper(); - private static final String DS_WIKI = "wiki"; private static final DataSegment WIKI_SEGMENT = DataSegment.builder() - .dataSource(DS_WIKI) + .dataSource(TestDataSource.WIKI) .interval(Intervals.of("2013-01-01/PT1H")) .size(100_000_000L) .version("v1") @@ -179,7 +179,7 @@ public void testStatusWhenLastCompactionStateIsNull() { verifyCompactionStatusIsPendingBecause( null, - DataSourceCompactionConfig.builder().forDataSource(DS_WIKI).build(), + DataSourceCompactionConfig.builder().forDataSource(TestDataSource.WIKI).build(), "not compacted yet" ); } @@ -189,7 +189,7 @@ public void testStatusWhenLastCompactionStateIsEmpty() { verifyCompactionStatusIsPendingBecause( new CompactionState(null, null, null, null, null, null), - DataSourceCompactionConfig.builder().forDataSource(DS_WIKI).build(), + DataSourceCompactionConfig.builder().forDataSource(TestDataSource.WIKI).build(), "'partitionsSpec' mismatch: required['dynamic' with 5,000,000 rows], current[null]" ); } @@ -202,7 +202,7 @@ public void testStatusOnPartitionsSpecMismatch() final CompactionState lastCompactionState = new CompactionState(currentPartitionsSpec, null, null, null, null, null); final DataSourceCompactionConfig compactionConfig - = DataSourceCompactionConfig.builder().forDataSource(DS_WIKI).build(); + = DataSourceCompactionConfig.builder().forDataSource(TestDataSource.WIKI).build(); verifyCompactionStatusIsPendingBecause( lastCompactionState, @@ -229,7 +229,7 @@ public void testStatusOnIndexSpecMismatch() ); final DataSourceCompactionConfig compactionConfig = DataSourceCompactionConfig .builder() - .forDataSource(DS_WIKI) + .forDataSource(TestDataSource.WIKI) .withTuningConfig(createTuningConfig(currentPartitionsSpec, null)) .build(); @@ -267,7 +267,7 @@ public void testStatusOnSegmentGranularityMismatch() ); final DataSourceCompactionConfig compactionConfig = DataSourceCompactionConfig .builder() - .forDataSource(DS_WIKI) + .forDataSource(TestDataSource.WIKI) .withTuningConfig(createTuningConfig(currentPartitionsSpec, currentIndexSpec)) .withGranularitySpec(new UserCompactionTaskGranularityConfig(Granularities.DAY, null, null)) .build(); @@ -297,7 +297,7 @@ public void testStatusWhenLastCompactionStateSameAsRequired() ); final DataSourceCompactionConfig compactionConfig = DataSourceCompactionConfig .builder() - .forDataSource(DS_WIKI) + .forDataSource(TestDataSource.WIKI) .withTuningConfig(createTuningConfig(currentPartitionsSpec, currentIndexSpec)) .withGranularitySpec(new UserCompactionTaskGranularityConfig(Granularities.HOUR, null, null)) .build(); @@ -336,7 +336,7 @@ private static DataSourceCompactionConfig createCompactionConfig( ) { return DataSourceCompactionConfig.builder() - .forDataSource(DS_WIKI) + .forDataSource(TestDataSource.WIKI) .withTuningConfig(createTuningConfig(partitionsSpec, null)) .build(); } 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 aa9d1d67c12e..9677b4890dcf 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 @@ -24,6 +24,7 @@ import org.apache.druid.indexer.TaskState; import org.apache.druid.indexer.TaskStatus; import org.apache.druid.jackson.DefaultObjectMapper; +import org.apache.druid.segment.TestDataSource; import org.apache.druid.server.coordinator.CreateDataSegments; import org.apache.druid.timeline.DataSegment; import org.junit.Assert; @@ -36,7 +37,7 @@ public class CompactionStatusTrackerTest { private static final ObjectMapper MAPPER = new DefaultObjectMapper(); private static final DataSegment WIKI_SEGMENT - = CreateDataSegments.ofDatasource(DS.WIKI).eachOfSizeInMb(100).get(0); + = CreateDataSegments.ofDatasource(TestDataSource.WIKI).eachOfSizeInMb(100).get(0); private CompactionStatusTracker statusTracker; @@ -51,7 +52,7 @@ public void testGetLatestTaskStatusForSubmittedTask() { final SegmentsToCompact candidateSegments = SegmentsToCompact.from(Collections.singletonList(WIKI_SEGMENT)); - statusTracker.onTaskSubmitted(createCompactionTask("task1", DS.WIKI), candidateSegments); + statusTracker.onTaskSubmitted(createCompactionTask("task1"), candidateSegments); CompactionTaskStatus status = statusTracker.getLatestTaskStatus(candidateSegments); Assert.assertEquals(TaskState.RUNNING, status.getState()); @@ -62,7 +63,7 @@ public void testGetLatestTaskStatusForSuccessfulTask() { final SegmentsToCompact candidateSegments = SegmentsToCompact.from(Collections.singletonList(WIKI_SEGMENT)); - statusTracker.onTaskSubmitted(createCompactionTask("task1", DS.WIKI), candidateSegments); + statusTracker.onTaskSubmitted(createCompactionTask("task1"), candidateSegments); statusTracker.onTaskFinished("task1", TaskStatus.success("task1")); CompactionTaskStatus status = statusTracker.getLatestTaskStatus(candidateSegments); @@ -74,7 +75,7 @@ public void testGetLatestTaskStatusForFailedTask() { final SegmentsToCompact candidateSegments = SegmentsToCompact.from(Collections.singletonList(WIKI_SEGMENT)); - statusTracker.onTaskSubmitted(createCompactionTask("task1", DS.WIKI), candidateSegments); + statusTracker.onTaskSubmitted(createCompactionTask("task1"), candidateSegments); statusTracker.onTaskFinished("task1", TaskStatus.failure("task1", "some failure")); CompactionTaskStatus status = statusTracker.getLatestTaskStatus(candidateSegments); @@ -88,10 +89,10 @@ public void testGetLatestTaskStatusForRepeatedlyFailingTask() final SegmentsToCompact candidateSegments = SegmentsToCompact.from(Collections.singletonList(WIKI_SEGMENT)); - statusTracker.onTaskSubmitted(createCompactionTask("task1", DS.WIKI), candidateSegments); + statusTracker.onTaskSubmitted(createCompactionTask("task1"), candidateSegments); statusTracker.onTaskFinished("task1", TaskStatus.failure("task1", "some failure")); - statusTracker.onTaskSubmitted(createCompactionTask("task2", DS.WIKI), candidateSegments); + statusTracker.onTaskSubmitted(createCompactionTask("task2"), candidateSegments); CompactionTaskStatus status = statusTracker.getLatestTaskStatus(candidateSegments); Assert.assertEquals(TaskState.RUNNING, status.getState()); Assert.assertEquals(1, status.getNumConsecutiveFailures()); @@ -104,13 +105,12 @@ public void testGetLatestTaskStatusForRepeatedlyFailingTask() } private ClientCompactionTaskQuery createCompactionTask( - String taskId, - String datasource + String taskId ) { return new ClientCompactionTaskQuery( taskId, - datasource, + TestDataSource.WIKI, null, null, null, @@ -121,9 +121,4 @@ private ClientCompactionTaskQuery createCompactionTask( null ); } - - private static class DS - { - static final String WIKI = "wiki"; - } } 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 5a056e8357b7..c4d694b8a935 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 @@ -43,6 +43,7 @@ import org.apache.druid.query.expression.TestExprMacroTable; import org.apache.druid.query.filter.SelectorDimFilter; import org.apache.druid.segment.IndexSpec; +import org.apache.druid.segment.TestDataSource; import org.apache.druid.segment.data.ConciseBitmapSerdeFactory; import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.segment.transform.TransformSpec; @@ -78,9 +79,6 @@ public class NewestSegmentFirstPolicyTest { - private static final String DATA_SOURCE = "wikipedia"; - private static final String DATASOURCE_KOALA = "koala"; - private static final long DEFAULT_SEGMENT_SIZE = 1000; private static final int DEFAULT_NUM_SEGMENTS_PER_SHARD = 4; private final ObjectMapper mapper = new DefaultObjectMapper(); private final NewestSegmentFirstPolicy policy = new NewestSegmentFirstPolicy(null); @@ -155,7 +153,7 @@ public void testLargeGapInData() { final Period segmentPeriod = Period.hours(1); final CompactionSegmentIterator iterator = createIterator( - configBuilder().withSkipOffsetFromLatest(new Period("PT1H1M")).build(), + configBuilder().withSkipOffsetFromLatest(Period.minutes(61)).build(), createTimeline( createSegments().forIntervals(8, Granularities.HOUR) .startingAt("2017-11-16T20:00:00Z") @@ -264,19 +262,18 @@ public void testManySegmentsPerShard() } @Test - public void testSkipUnknownDataSource() + public void testSkipDataSourceWithNoSegments() { - final String unknownDataSource = "unknown"; final Period segmentPeriod = Period.hours(1); final CompactionSegmentIterator iterator = policy.createIterator( ImmutableMap.of( - unknownDataSource, - configBuilder().forDataSource(unknownDataSource).withSkipOffsetFromLatest(Period.days(2)).build(), - DATA_SOURCE, - configBuilder().forDataSource(DATA_SOURCE).withSkipOffsetFromLatest(Period.days(2)).build() + TestDataSource.KOALA, + configBuilder().forDataSource(TestDataSource.KOALA).build(), + TestDataSource.WIKI, + configBuilder().forDataSource(TestDataSource.WIKI).withSkipOffsetFromLatest(Period.days(2)).build() ), ImmutableMap.of( - DATA_SOURCE, + TestDataSource.WIKI, createTimeline( createSegments().forIntervals(8, Granularities.HOUR) .startingAt("2017-11-16T20:00:00Z") @@ -496,9 +493,9 @@ public void testWithSkipIntervals() { final Period segmentPeriod = Period.hours(1); final CompactionSegmentIterator iterator = policy.createIterator( - ImmutableMap.of(DATA_SOURCE, configBuilder().withSkipOffsetFromLatest(Period.days(1)).build()), + ImmutableMap.of(TestDataSource.WIKI, configBuilder().withSkipOffsetFromLatest(Period.days(1)).build()), ImmutableMap.of( - DATA_SOURCE, + TestDataSource.WIKI, createTimeline( createSegments().forIntervals(8, Granularities.HOUR) .startingAt("2017-11-16T20:00:00Z") @@ -509,7 +506,7 @@ public void testWithSkipIntervals() ) ), ImmutableMap.of( - DATA_SOURCE, + TestDataSource.WIKI, ImmutableList.of( Intervals.of("2017-11-16T00:00:00/2017-11-17T00:00:00"), Intervals.of("2017-11-15T00:00:00/2017-11-15T20:00:00"), @@ -541,15 +538,15 @@ public void testHoleInSearchInterval() { final Period segmentPeriod = Period.hours(1); final CompactionSegmentIterator iterator = policy.createIterator( - ImmutableMap.of(DATA_SOURCE, configBuilder().withSkipOffsetFromLatest(Period.hours(1)).build()), + ImmutableMap.of(TestDataSource.WIKI, configBuilder().withSkipOffsetFromLatest(Period.hours(1)).build()), ImmutableMap.of( - DATA_SOURCE, + TestDataSource.WIKI, createTimeline( createSegments().forIntervals(1, Granularities.HOUR).startingAt("2017-11-16").withNumPartitions(4) ) ), ImmutableMap.of( - DATA_SOURCE, + TestDataSource.WIKI, ImmutableList.of( Intervals.of("2017-11-16T04:00:00/2017-11-16T10:00:00"), Intervals.of("2017-11-16T14:00:00/2017-11-16T20:00:00") @@ -1506,7 +1503,7 @@ public void testSkipAllGranularityToDefault() configBuilder().build(), SegmentTimeline.forSegments(ImmutableSet.of( new DataSegment( - DATA_SOURCE, + TestDataSource.WIKI, Intervals.ETERNITY, "0", new HashMap<>(), @@ -1530,7 +1527,7 @@ public void testSkipFirstHalfEternityToDefault() configBuilder().build(), SegmentTimeline.forSegments(ImmutableSet.of( new DataSegment( - DATA_SOURCE, + TestDataSource.WIKI, new Interval(DateTimes.MIN, DateTimes.of("2024-01-01")), "0", new HashMap<>(), @@ -1554,7 +1551,7 @@ public void testSkipSecondHalfOfEternityToDefault() configBuilder().build(), SegmentTimeline.forSegments(ImmutableSet.of( new DataSegment( - DATA_SOURCE, + TestDataSource.WIKI, new Interval(DateTimes.of("2024-01-01"), DateTimes.MAX), "0", new HashMap<>(), @@ -1580,7 +1577,7 @@ public void testSkipAllToAllGranularity() ).build(), SegmentTimeline.forSegments(ImmutableSet.of( new DataSegment( - DATA_SOURCE, + TestDataSource.WIKI, Intervals.ETERNITY, "0", new HashMap<>(), @@ -1606,7 +1603,7 @@ public void testSkipAllToFinerGranularity() ).build(), SegmentTimeline.forSegments(ImmutableSet.of( new DataSegment( - DATA_SOURCE, + TestDataSource.WIKI, Intervals.ETERNITY, "0", new HashMap<>(), @@ -1627,7 +1624,7 @@ public void testSkipAllToFinerGranularity() public void testSkipCompactionForIntervalsContainingSingleTombstone() { final DataSegment tombstone2023 = new DataSegment( - DATA_SOURCE, + TestDataSource.WIKI, Intervals.of("2023/2024"), "0", new HashMap<>(), @@ -1637,7 +1634,7 @@ public void testSkipCompactionForIntervalsContainingSingleTombstone() 0, 1); final DataSegment dataSegment2023 = new DataSegment( - DATA_SOURCE, + TestDataSource.WIKI, Intervals.of("2023/2024"), "0", new HashMap<>(), @@ -1647,7 +1644,7 @@ public void testSkipCompactionForIntervalsContainingSingleTombstone() 0, 100); final DataSegment tombstone2024 = new DataSegment( - DATA_SOURCE, + TestDataSource.WIKI, Intervals.of("2024/2025"), "0", new HashMap<>(), @@ -1672,7 +1669,7 @@ public void testSkipCompactionForIntervalsContainingSingleTombstone() ); final DataSegment tombstone2025Jan = new DataSegment( - DATA_SOURCE, + TestDataSource.WIKI, Intervals.of("2025-01-01/2025-02-01"), "0", new HashMap<>(), @@ -1682,7 +1679,7 @@ public void testSkipCompactionForIntervalsContainingSingleTombstone() 0, 1); final DataSegment tombstone2025Feb = new DataSegment( - DATA_SOURCE, + TestDataSource.WIKI, Intervals.of("2025-02-01/2025-03-01"), "0", new HashMap<>(), @@ -1692,7 +1689,7 @@ public void testSkipCompactionForIntervalsContainingSingleTombstone() 0, 1); final DataSegment tombstone2025Mar = new DataSegment( - DATA_SOURCE, + TestDataSource.WIKI, Intervals.of("2025-03-01/2025-04-01"), "0", new HashMap<>(), @@ -1725,47 +1722,57 @@ public void testSkipCompactionForIntervalsContainingSingleTombstone() public void testPriorityDatasource() { final List wikiSegments - = CreateDataSegments.ofDatasource(DATA_SOURCE) + = CreateDataSegments.ofDatasource(TestDataSource.WIKI) .forIntervals(1, Granularities.DAY) .startingAt("2012-01-01") .withNumPartitions(10) .eachOfSizeInMb(100); final List koalaSegments - = CreateDataSegments.ofDatasource(DATASOURCE_KOALA) + = CreateDataSegments.ofDatasource(TestDataSource.KOALA) .forIntervals(1, Granularities.DAY) .startingAt("2013-01-01") .withNumPartitions(10) .eachOfSizeInMb(100); - // Setup policy and iterator with priorityDatasource = wikipedia - final NewestSegmentFirstPolicy policy = new NewestSegmentFirstPolicy(DATA_SOURCE); + // Setup policy and iterator with priorityDatasource = WIKI + final NewestSegmentFirstPolicy policy = new NewestSegmentFirstPolicy(TestDataSource.WIKI); CompactionSegmentIterator iterator = policy.createIterator( ImmutableMap.of( - DATA_SOURCE, configBuilder().forDataSource(DATA_SOURCE).build(), - DATASOURCE_KOALA, configBuilder().forDataSource(DATASOURCE_KOALA).build() + TestDataSource.WIKI, configBuilder().forDataSource(TestDataSource.WIKI).build(), + TestDataSource.KOALA, configBuilder().forDataSource(TestDataSource.KOALA).build() ), ImmutableMap.of( - DATA_SOURCE, SegmentTimeline.forSegments(wikiSegments), - DATASOURCE_KOALA, SegmentTimeline.forSegments(koalaSegments) + TestDataSource.WIKI, SegmentTimeline.forSegments(wikiSegments), + TestDataSource.KOALA, SegmentTimeline.forSegments(koalaSegments) ), Collections.emptyMap(), statusTracker ); - // Verify that the segments of "wikipedia" are preferred even though they are older + // Verify that the segments of WIKI are preferred even though they are older Assert.assertTrue(iterator.hasNext()); SegmentsToCompact next = iterator.next(); Assert.assertFalse(next.isEmpty()); - Assert.assertEquals(DATA_SOURCE, next.getDataSource()); + Assert.assertEquals(TestDataSource.WIKI, next.getDataSource()); Assert.assertEquals(Intervals.of("2012-01-01/P1D"), next.getUmbrellaInterval()); Assert.assertTrue(iterator.hasNext()); next = iterator.next(); Assert.assertFalse(next.isEmpty()); - Assert.assertEquals(DATASOURCE_KOALA, next.getDataSource()); + Assert.assertEquals(TestDataSource.KOALA, next.getDataSource()); Assert.assertEquals(Intervals.of("2013-01-01/P1D"), next.getUmbrellaInterval()); } + private CompactionSegmentIterator createIterator(DataSourceCompactionConfig config, SegmentTimeline timeline) + { + return policy.createIterator( + Collections.singletonMap(TestDataSource.WIKI, config), + Collections.singletonMap(TestDataSource.WIKI, timeline), + Collections.emptyMap(), + statusTracker + ); + } + private static void assertCompactSegmentIntervals( CompactionSegmentIterator iterator, Period segmentPeriod, @@ -1813,17 +1820,7 @@ private static void assertCompactSegmentIntervals( private static CreateDataSegments createSegments() { - return CreateDataSegments.ofDatasource(DATA_SOURCE).withNumPartitions(DEFAULT_NUM_SEGMENTS_PER_SHARD); - } - - private CompactionSegmentIterator createIterator(DataSourceCompactionConfig config, SegmentTimeline timeline) - { - return policy.createIterator( - Collections.singletonMap(DATA_SOURCE, config), - Collections.singletonMap(DATA_SOURCE, timeline), - Collections.emptyMap(), - statusTracker - ); + return CreateDataSegments.ofDatasource(TestDataSource.WIKI); } private static SegmentTimeline createTimeline( @@ -1850,7 +1847,7 @@ private static DataSourceCompactionConfig createConfigWithSegmentGranularity( private static DataSourceCompactionConfig.Builder configBuilder() { return DataSourceCompactionConfig.builder() - .forDataSource(DATA_SOURCE) + .forDataSource(TestDataSource.WIKI) .withSkipOffsetFromLatest(Period.seconds(0)); } } diff --git a/server/src/test/java/org/apache/druid/server/coordinator/DataSourceCompactionConfigAuditEntryTest.java b/server/src/test/java/org/apache/druid/server/coordinator/DataSourceCompactionConfigAuditEntryTest.java index ddb495572ab4..bdd028469211 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/DataSourceCompactionConfigAuditEntryTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/DataSourceCompactionConfigAuditEntryTest.java @@ -22,6 +22,7 @@ import org.apache.druid.audit.AuditInfo; import org.apache.druid.indexer.CompactionEngine; import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.segment.TestDataSource; import org.junit.Assert; import org.junit.Test; import org.junit.runner.RunWith; @@ -30,12 +31,11 @@ @RunWith(MockitoJUnitRunner.class) public class DataSourceCompactionConfigAuditEntryTest { - private static final String DS_WIKI = "wiki"; private final AuditInfo auditInfo = new AuditInfo("author", "identity", "comment", "ip"); private final DataSourceCompactionConfigAuditEntry firstEntry = new DataSourceCompactionConfigAuditEntry( new ClusterCompactionConfig(0.1, 9, true, CompactionEngine.MSQ, null), - DataSourceCompactionConfig.builder().forDataSource(DS_WIKI).build(), + DataSourceCompactionConfig.builder().forDataSource(TestDataSource.WIKI).build(), auditInfo, DateTimes.nowUtc() ); @@ -45,7 +45,7 @@ public void testhasSameConfigWithSameBaseConfigIsTrue() { final DataSourceCompactionConfigAuditEntry secondEntry = new DataSourceCompactionConfigAuditEntry( new ClusterCompactionConfig(0.1, 9, true, CompactionEngine.MSQ, null), - DataSourceCompactionConfig.builder().forDataSource(DS_WIKI).build(), + DataSourceCompactionConfig.builder().forDataSource(TestDataSource.WIKI).build(), auditInfo, DateTimes.nowUtc() ); @@ -58,7 +58,7 @@ public void testhasSameConfigWithDifferentClusterConfigIsFalse() { DataSourceCompactionConfigAuditEntry secondEntry = new DataSourceCompactionConfigAuditEntry( new ClusterCompactionConfig(0.1, 9, false, CompactionEngine.MSQ, null), - DataSourceCompactionConfig.builder().forDataSource(DS_WIKI).build(), + DataSourceCompactionConfig.builder().forDataSource(TestDataSource.WIKI).build(), auditInfo, DateTimes.nowUtc() ); @@ -67,7 +67,7 @@ public void testhasSameConfigWithDifferentClusterConfigIsFalse() secondEntry = new DataSourceCompactionConfigAuditEntry( new ClusterCompactionConfig(0.1, 9, true, CompactionEngine.NATIVE, null), - DataSourceCompactionConfig.builder().forDataSource(DS_WIKI).build(), + DataSourceCompactionConfig.builder().forDataSource(TestDataSource.WIKI).build(), auditInfo, DateTimes.nowUtc() ); @@ -80,7 +80,7 @@ public void testhasSameConfigWithDifferentDatasourceConfigIsFalse() { DataSourceCompactionConfigAuditEntry secondEntry = new DataSourceCompactionConfigAuditEntry( new ClusterCompactionConfig(0.1, 9, true, CompactionEngine.NATIVE, null), - DataSourceCompactionConfig.builder().forDataSource(DS_WIKI).build(), + DataSourceCompactionConfig.builder().forDataSource(TestDataSource.WIKI).build(), auditInfo, DateTimes.nowUtc() ); diff --git a/server/src/test/java/org/apache/druid/server/coordinator/DataSourceCompactionConfigHistoryTest.java b/server/src/test/java/org/apache/druid/server/coordinator/DataSourceCompactionConfigHistoryTest.java index 1809a5bc54b1..4426d58b258e 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/DataSourceCompactionConfigHistoryTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/DataSourceCompactionConfigHistoryTest.java @@ -22,6 +22,7 @@ import org.apache.druid.audit.AuditInfo; import org.apache.druid.indexer.CompactionEngine; import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.segment.TestDataSource; import org.joda.time.DateTime; import org.joda.time.Period; import org.junit.Assert; @@ -37,14 +38,14 @@ public class DataSourceCompactionConfigHistoryTest { private final AuditInfo auditInfo = new AuditInfo("author", "identity", "comment", "ip"); private final DataSourceCompactionConfig wikiCompactionConfig - = DataSourceCompactionConfig.builder().forDataSource(DS.WIKI).build(); + = DataSourceCompactionConfig.builder().forDataSource(TestDataSource.WIKI).build(); private DataSourceCompactionConfigHistory wikiAuditHistory; @Before public void setup() { - wikiAuditHistory = new DataSourceCompactionConfigHistory(DS.WIKI); + wikiAuditHistory = new DataSourceCompactionConfigHistory(TestDataSource.WIKI); } @Test @@ -94,7 +95,7 @@ public void testAddDeleteDatasourceConfigShouldAddBothToHistory() public void testAddDeleteAnotherDatasourceConfigShouldNotAddToHistory() { final DataSourceCompactionConfig koalaCompactionConfig - = DataSourceCompactionConfig.builder().forDataSource(DS.KOALA).build(); + = DataSourceCompactionConfig.builder().forDataSource(TestDataSource.KOALA).build(); wikiAuditHistory.add( DruidCompactionConfig.empty().withDatasourceConfig(koalaCompactionConfig), @@ -147,7 +148,7 @@ public void testAddModifyDatasourceConfigShouldAddBothToHistory() final DataSourceCompactionConfig updatedWikiConfig = DataSourceCompactionConfig.builder() - .forDataSource(DS.WIKI) + .forDataSource(TestDataSource.WIKI) .withSkipOffsetFromLatest(Period.hours(5)) .build(); wikiAuditHistory.add( @@ -192,10 +193,4 @@ public void testAddAndModifyClusterConfigShouldAddTwice() Assert.assertEquals(updatedConfig.clusterConfig(), secondEntry.getGlobalConfig()); Assert.assertFalse(firstEntry.hasSameConfig(secondEntry)); } - - private static class DS - { - static final String KOALA = "koala"; - static final String WIKI = "wiki"; - } } diff --git a/server/src/test/java/org/apache/druid/server/coordinator/DruidCompactionConfigTest.java b/server/src/test/java/org/apache/druid/server/coordinator/DruidCompactionConfigTest.java index 5eadead4e684..c8ed8d9ba530 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/DruidCompactionConfigTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/DruidCompactionConfigTest.java @@ -22,6 +22,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import org.apache.druid.indexer.CompactionEngine; import org.apache.druid.jackson.DefaultObjectMapper; +import org.apache.druid.segment.TestDataSource; import org.apache.druid.server.compaction.NewestSegmentFirstPolicy; import org.joda.time.Period; import org.junit.Assert; @@ -51,12 +52,12 @@ public void testSerdeWithDatasourceConfigs() throws Exception Arrays.asList( DataSourceCompactionConfig .builder() - .forDataSource(DS.WIKI) + .forDataSource(TestDataSource.WIKI) .withSkipOffsetFromLatest(Period.hours(1)) .build(), DataSourceCompactionConfig .builder() - .forDataSource(DS.KOALA) + .forDataSource(TestDataSource.KOALA) .withSkipOffsetFromLatest(Period.hours(2)) .build() ), @@ -96,17 +97,15 @@ public void testCopyWithDatasourceConfigs() final DruidCompactionConfig config = DruidCompactionConfig.empty(); Assert.assertTrue(config.getCompactionConfigs().isEmpty()); - final DataSourceCompactionConfig dataSourceConfig - = DataSourceCompactionConfig.builder().forDataSource(DS.WIKI).withEngine(CompactionEngine.NATIVE).build(); - final DruidCompactionConfig copy = config.withDatasourceConfigs(Collections.singletonList(dataSourceConfig)); + final DataSourceCompactionConfig dataSourceConfig = DataSourceCompactionConfig + .builder() + .forDataSource(TestDataSource.WIKI) + .withEngine(CompactionEngine.NATIVE) + .build(); + final DruidCompactionConfig copy + = config.withDatasourceConfigs(Collections.singletonList(dataSourceConfig)); Assert.assertEquals(1, copy.getCompactionConfigs().size()); - Assert.assertEquals(dataSourceConfig, copy.findConfigForDatasource(DS.WIKI).orNull()); - } - - private static class DS - { - static final String WIKI = "wiki"; - static final String KOALA = "koala"; + Assert.assertEquals(dataSourceConfig, copy.findConfigForDatasource(TestDataSource.WIKI).orNull()); } } diff --git a/server/src/test/java/org/apache/druid/server/coordinator/balancer/CostBalancerStrategyTest.java b/server/src/test/java/org/apache/druid/server/coordinator/balancer/CostBalancerStrategyTest.java index 576dfbb03d7a..5113f91e65b8 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/balancer/CostBalancerStrategyTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/balancer/CostBalancerStrategyTest.java @@ -26,6 +26,7 @@ import org.apache.druid.java.util.common.granularity.GranularityType; import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.java.util.metrics.StubServiceEmitter; +import org.apache.druid.segment.TestDataSource; import org.apache.druid.server.coordination.ServerType; import org.apache.druid.server.coordinator.CreateDataSegments; import org.apache.druid.server.coordinator.ServerHolder; @@ -51,9 +52,7 @@ public class CostBalancerStrategyTest { private static final double DELTA = 1e-6; - private static final String DS_WIKI = "wiki"; - private StubServiceEmitter serviceEmitter; private ExecutorService balancerExecutor; private CostBalancerStrategy strategy; private int uniqueServerId; @@ -64,7 +63,7 @@ public void setup() balancerExecutor = Execs.singleThreaded("test-balance-exec-%d"); strategy = new CostBalancerStrategy(MoreExecutors.listeningDecorator(balancerExecutor)); - serviceEmitter = new StubServiceEmitter("test-service", "host"); + StubServiceEmitter serviceEmitter = new StubServiceEmitter("test-service", "host"); EmittingLogger.registerEmitter(serviceEmitter); } @@ -154,11 +153,11 @@ public void testJointSegmentsCost() @Test public void testJointSegmentsCostSymmetry() { - final DataSegment segmentA = CreateDataSegments.ofDatasource(DS_WIKI) + final DataSegment segmentA = CreateDataSegments.ofDatasource(TestDataSource.WIKI) .forIntervals(1, Granularities.DAY) .startingAt("2010-01-01") .eachOfSizeInMb(100).get(0); - final DataSegment segmentB = CreateDataSegments.ofDatasource(DS_WIKI) + final DataSegment segmentB = CreateDataSegments.ofDatasource(TestDataSource.WIKI) .forIntervals(1, Granularities.MONTH) .startingAt("2010-01-01") .eachOfSizeInMb(100).get(0); @@ -173,7 +172,7 @@ public void testJointSegmentsCostSymmetry() @Test public void testJointSegmentsCostMultipleDatasources() { - final DataSegment wikiSegment = CreateDataSegments.ofDatasource(DS_WIKI) + final DataSegment wikiSegment = CreateDataSegments.ofDatasource(TestDataSource.WIKI) .forIntervals(1, Granularities.DAY) .startingAt("2010-01-01") .eachOfSizeInMb(100).get(0); @@ -245,21 +244,21 @@ public void testComputePlacementCost() { // Create segments for different granularities final List daySegments = - CreateDataSegments.ofDatasource(DS_WIKI) + CreateDataSegments.ofDatasource(TestDataSource.WIKI) .forIntervals(10, Granularities.DAY) .startingAt("2022-01-01") .withNumPartitions(10) .eachOfSizeInMb(100); final List monthSegments = - CreateDataSegments.ofDatasource(DS_WIKI) + CreateDataSegments.ofDatasource(TestDataSource.WIKI) .forIntervals(10, Granularities.MONTH) .startingAt("2022-03-01") .withNumPartitions(10) .eachOfSizeInMb(100); final List yearSegments = - CreateDataSegments.ofDatasource(DS_WIKI) + CreateDataSegments.ofDatasource(TestDataSource.WIKI) .forIntervals(1, Granularities.YEAR) .startingAt("2023-01-01") .withNumPartitions(30) @@ -306,7 +305,7 @@ public void testComputePlacementCost() // Verify costs for an ALL granularity segment final DataSegment allGranularitySegment = - CreateDataSegments.ofDatasource(DS_WIKI) + CreateDataSegments.ofDatasource(TestDataSource.WIKI) .forIntervals(1, Granularities.ALL) .eachOfSizeInMb(100).get(0); verifyPlacementCost(allGranularitySegment, serverA, 1.1534173737329768e7); @@ -326,7 +325,7 @@ public void testGetStats() new TestLoadQueuePeon() ); - final DataSegment segment = CreateDataSegments.ofDatasource(DS_WIKI).eachOfSizeInMb(100).get(0); + final DataSegment segment = CreateDataSegments.ofDatasource(TestDataSource.WIKI).eachOfSizeInMb(100).get(0); // Verify that computation stats have been tracked strategy.findServersToLoadSegment(segment, Arrays.asList(serverA, serverB)); @@ -342,7 +341,7 @@ public void testGetStats() @Test public void testFindServerAfterExecutorShutdownThrowsException() { - DataSegment segment = CreateDataSegments.ofDatasource(DS_WIKI) + DataSegment segment = CreateDataSegments.ofDatasource(TestDataSource.WIKI) .forIntervals(1, Granularities.DAY) .startingAt("2012-10-24") .eachOfSizeInMb(100).get(0); @@ -386,14 +385,14 @@ private void verifyJointSegmentsCost( ) { final DataSegment segmentX = - CreateDataSegments.ofDatasource(DS_WIKI) + CreateDataSegments.ofDatasource(TestDataSource.WIKI) .forIntervals(1, granularityX.getDefaultGranularity()) .startingAt("2012-10-24") .eachOfSizeInMb(100).get(0); long startTimeY = segmentX.getInterval().getStartMillis() + startGapMillis; final DataSegment segmentY = - CreateDataSegments.ofDatasource(DS_WIKI) + CreateDataSegments.ofDatasource(TestDataSource.WIKI) .forIntervals(1, granularityY.getDefaultGranularity()) .startingAt(startTimeY) .eachOfSizeInMb(100).get(0); diff --git a/server/src/test/java/org/apache/druid/server/coordinator/balancer/ReservoirSegmentSamplerTest.java b/server/src/test/java/org/apache/druid/server/coordinator/balancer/ReservoirSegmentSamplerTest.java index c438219e1b36..cc522fcb546a 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/balancer/ReservoirSegmentSamplerTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/balancer/ReservoirSegmentSamplerTest.java @@ -22,6 +22,7 @@ import com.google.common.collect.Lists; import org.apache.druid.client.DruidServer; import org.apache.druid.java.util.common.granularity.Granularities; +import org.apache.druid.segment.TestDataSource; import org.apache.druid.server.coordination.ServerType; import org.apache.druid.server.coordinator.CreateDataSegments; import org.apache.druid.server.coordinator.ServerHolder; @@ -50,7 +51,7 @@ public class ReservoirSegmentSamplerTest * num segments = 10 x 100 days */ private final List segments = - CreateDataSegments.ofDatasource("wiki") + CreateDataSegments.ofDatasource(TestDataSource.WIKI) .forIntervals(100, Granularities.DAY) .startingAt("2022-01-01") .withNumPartitions(10) diff --git a/server/src/test/java/org/apache/druid/server/coordinator/balancer/SegmentToMoveCalculatorTest.java b/server/src/test/java/org/apache/druid/server/coordinator/balancer/SegmentToMoveCalculatorTest.java index 56fb316aebf5..b5b06afa709f 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/balancer/SegmentToMoveCalculatorTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/balancer/SegmentToMoveCalculatorTest.java @@ -21,6 +21,7 @@ import org.apache.druid.client.DruidServer; import org.apache.druid.java.util.common.granularity.Granularities; +import org.apache.druid.segment.TestDataSource; import org.apache.druid.server.coordination.ServerType; import org.apache.druid.server.coordinator.CreateDataSegments; import org.apache.druid.server.coordinator.ServerHolder; @@ -44,7 +45,7 @@ public class SegmentToMoveCalculatorTest * 100 days x 100 partitions = 10,000 segments. */ private static final List WIKI_SEGMENTS - = CreateDataSegments.ofDatasource("wiki") + = CreateDataSegments.ofDatasource(TestDataSource.WIKI) .forIntervals(100, Granularities.DAY) .withNumPartitions(100) .eachOfSizeInMb(500); @@ -53,7 +54,7 @@ public class SegmentToMoveCalculatorTest * 10 days * 1 partitions = 10 segments. */ private static final List KOALA_SEGMENTS - = CreateDataSegments.ofDatasource("koala") + = CreateDataSegments.ofDatasource(TestDataSource.KOALA) .forIntervals(10, Granularities.DAY) .eachOfSizeInMb(500); diff --git a/server/src/test/java/org/apache/druid/server/coordinator/duty/KillStalePendingSegmentsTest.java b/server/src/test/java/org/apache/druid/server/coordinator/duty/KillStalePendingSegmentsTest.java index 11ea5bd4b57b..b7a481f49cf3 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/duty/KillStalePendingSegmentsTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/duty/KillStalePendingSegmentsTest.java @@ -30,6 +30,7 @@ import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.parsers.CloseableIterator; +import org.apache.druid.segment.TestDataSource; import org.apache.druid.server.coordinator.CoordinatorDynamicConfig; import org.apache.druid.server.coordinator.DruidCoordinatorRuntimeParams; import org.apache.druid.server.coordinator.stats.CoordinatorRunStats; @@ -68,10 +69,10 @@ public void setup() @Test public void testRetentionStarts1DayBeforeNowWhenNoKnownTask() { - DruidCoordinatorRuntimeParams params = createParamsWithDatasources(DS.WIKI).build(); + DruidCoordinatorRuntimeParams params = createParamsWithDatasources(TestDataSource.WIKI).build(); killDuty.run(params); - final Interval observedKillInterval = overlordClient.observedKillIntervals.get(DS.WIKI); + final Interval observedKillInterval = overlordClient.observedKillIntervals.get(TestDataSource.WIKI); Assert.assertEquals(DateTimes.MIN, observedKillInterval.getStart()); // Verify that the cutoff time is no later than 1 day ago from now @@ -85,15 +86,15 @@ public void testRetentionStarts1DayBeforeNowWhenNoKnownTask() public void testRetentionStarts1DayBeforeEarliestActiveTask() { final DateTime startOfEarliestActiveTask = DateTimes.of("2023-01-01"); - overlordClient.addTaskAndSegment(DS.WIKI, startOfEarliestActiveTask, TaskState.RUNNING); - overlordClient.addTaskAndSegment(DS.WIKI, startOfEarliestActiveTask.plusHours(2), TaskState.RUNNING); - overlordClient.addTaskAndSegment(DS.WIKI, startOfEarliestActiveTask.plusDays(1), TaskState.RUNNING); - overlordClient.addTaskAndSegment(DS.WIKI, startOfEarliestActiveTask.plusHours(3), TaskState.RUNNING); + overlordClient.addTaskAndSegment(TestDataSource.WIKI, startOfEarliestActiveTask, TaskState.RUNNING); + overlordClient.addTaskAndSegment(TestDataSource.WIKI, startOfEarliestActiveTask.plusHours(2), TaskState.RUNNING); + overlordClient.addTaskAndSegment(TestDataSource.WIKI, startOfEarliestActiveTask.plusDays(1), TaskState.RUNNING); + overlordClient.addTaskAndSegment(TestDataSource.WIKI, startOfEarliestActiveTask.plusHours(3), TaskState.RUNNING); - DruidCoordinatorRuntimeParams params = createParamsWithDatasources(DS.WIKI).build(); + DruidCoordinatorRuntimeParams params = createParamsWithDatasources(TestDataSource.WIKI).build(); killDuty.run(params); - final Interval observedKillInterval = overlordClient.observedKillIntervals.get(DS.WIKI); + final Interval observedKillInterval = overlordClient.observedKillIntervals.get(TestDataSource.WIKI); Assert.assertEquals(DateTimes.MIN, observedKillInterval.getStart()); Assert.assertEquals(startOfEarliestActiveTask.minusDays(1), observedKillInterval.getEnd()); } @@ -102,40 +103,40 @@ public void testRetentionStarts1DayBeforeEarliestActiveTask() public void testRetentionStarts1DayBeforeLatestCompletedTask() { final DateTime startOfLatestCompletedTask = DateTimes.of("2023-01-01"); - overlordClient.addTaskAndSegment(DS.WIKI, startOfLatestCompletedTask, TaskState.FAILED); - overlordClient.addTaskAndSegment(DS.WIKI, startOfLatestCompletedTask.minusHours(2), TaskState.SUCCESS); - overlordClient.addTaskAndSegment(DS.WIKI, startOfLatestCompletedTask.minusDays(2), TaskState.FAILED); - overlordClient.addTaskAndSegment(DS.WIKI, startOfLatestCompletedTask.minusDays(3), TaskState.SUCCESS); + overlordClient.addTaskAndSegment(TestDataSource.WIKI, startOfLatestCompletedTask, TaskState.FAILED); + overlordClient.addTaskAndSegment(TestDataSource.WIKI, startOfLatestCompletedTask.minusHours(2), TaskState.SUCCESS); + overlordClient.addTaskAndSegment(TestDataSource.WIKI, startOfLatestCompletedTask.minusDays(2), TaskState.FAILED); + overlordClient.addTaskAndSegment(TestDataSource.WIKI, startOfLatestCompletedTask.minusDays(3), TaskState.SUCCESS); - DruidCoordinatorRuntimeParams params = createParamsWithDatasources(DS.WIKI).build(); + DruidCoordinatorRuntimeParams params = createParamsWithDatasources(TestDataSource.WIKI).build(); killDuty.run(params); - final Interval observedKillInterval = overlordClient.observedKillIntervals.get(DS.WIKI); + final Interval observedKillInterval = overlordClient.observedKillIntervals.get(TestDataSource.WIKI); Assert.assertEquals(DateTimes.MIN, observedKillInterval.getStart()); Assert.assertEquals(startOfLatestCompletedTask.minusDays(1), observedKillInterval.getEnd()); final CoordinatorRunStats stats = params.getCoordinatorStats(); - Assert.assertEquals(2, stats.get(Stats.Kill.PENDING_SEGMENTS, RowKey.of(Dimension.DATASOURCE, DS.WIKI))); + Assert.assertEquals(2, stats.get(Stats.Kill.PENDING_SEGMENTS, RowKey.of(Dimension.DATASOURCE, TestDataSource.WIKI))); } @Test public void testRetentionStarts1DayBeforeLatestCompletedOrEarliestActiveTask() { final DateTime startOfLatestCompletedTask = DateTimes.of("2023-02-01"); - overlordClient.addTaskAndSegment(DS.WIKI, startOfLatestCompletedTask, TaskState.FAILED); + overlordClient.addTaskAndSegment(TestDataSource.WIKI, startOfLatestCompletedTask, TaskState.FAILED); final DateTime startOfEarliestActiveTask = DateTimes.of("2023-01-01"); - overlordClient.addTaskAndSegment(DS.KOALA, startOfEarliestActiveTask, TaskState.RUNNING); + overlordClient.addTaskAndSegment(TestDataSource.KOALA, startOfEarliestActiveTask, TaskState.RUNNING); - DruidCoordinatorRuntimeParams params = createParamsWithDatasources(DS.WIKI, DS.KOALA).build(); + DruidCoordinatorRuntimeParams params = createParamsWithDatasources(TestDataSource.WIKI, TestDataSource.KOALA).build(); killDuty.run(params); DateTime earliestEligibleTask = DateTimes.earlierOf(startOfEarliestActiveTask, startOfLatestCompletedTask); - final Interval wikiKillInterval = overlordClient.observedKillIntervals.get(DS.WIKI); + final Interval wikiKillInterval = overlordClient.observedKillIntervals.get(TestDataSource.WIKI); Assert.assertEquals(DateTimes.MIN, wikiKillInterval.getStart()); Assert.assertEquals(earliestEligibleTask.minusDays(1), wikiKillInterval.getEnd()); - final Interval koalaKillInterval = overlordClient.observedKillIntervals.get(DS.KOALA); + final Interval koalaKillInterval = overlordClient.observedKillIntervals.get(TestDataSource.KOALA); Assert.assertEquals(DateTimes.MIN, koalaKillInterval.getStart()); Assert.assertEquals(earliestEligibleTask.minusDays(1), wikiKillInterval.getEnd()); } @@ -144,32 +145,32 @@ public void testRetentionStarts1DayBeforeLatestCompletedOrEarliestActiveTask() public void testPendingSegmentOfDisallowedDatasourceIsNotDeleted() { DruidCoordinatorRuntimeParams params = - createParamsWithDatasources(DS.WIKI, DS.KOALA).withDynamicConfigs( + createParamsWithDatasources(TestDataSource.WIKI, TestDataSource.KOALA).withDynamicConfigs( CoordinatorDynamicConfig .builder() .withDatasourcesToNotKillPendingSegmentsIn( - Collections.singleton(DS.KOALA) + Collections.singleton(TestDataSource.KOALA) ) .build() ).build(); DateTime startOfLatestCompletedTask = DateTimes.of("2023-01-01"); - overlordClient.addTaskAndSegment(DS.WIKI, startOfLatestCompletedTask, TaskState.SUCCESS); - overlordClient.addTaskAndSegment(DS.WIKI, startOfLatestCompletedTask.minusDays(3), TaskState.SUCCESS); - overlordClient.addTaskAndSegment(DS.WIKI, startOfLatestCompletedTask.minusDays(5), TaskState.SUCCESS); - overlordClient.addTaskAndSegment(DS.KOALA, startOfLatestCompletedTask, TaskState.SUCCESS); - overlordClient.addTaskAndSegment(DS.KOALA, startOfLatestCompletedTask.minusDays(3), TaskState.SUCCESS); - overlordClient.addTaskAndSegment(DS.KOALA, startOfLatestCompletedTask.minusDays(5), TaskState.SUCCESS); + overlordClient.addTaskAndSegment(TestDataSource.WIKI, startOfLatestCompletedTask, TaskState.SUCCESS); + overlordClient.addTaskAndSegment(TestDataSource.WIKI, startOfLatestCompletedTask.minusDays(3), TaskState.SUCCESS); + overlordClient.addTaskAndSegment(TestDataSource.WIKI, startOfLatestCompletedTask.minusDays(5), TaskState.SUCCESS); + overlordClient.addTaskAndSegment(TestDataSource.KOALA, startOfLatestCompletedTask, TaskState.SUCCESS); + overlordClient.addTaskAndSegment(TestDataSource.KOALA, startOfLatestCompletedTask.minusDays(3), TaskState.SUCCESS); + overlordClient.addTaskAndSegment(TestDataSource.KOALA, startOfLatestCompletedTask.minusDays(5), TaskState.SUCCESS); killDuty.run(params); // Verify that stale pending segments are killed in "wiki" but not in "koala" final CoordinatorRunStats stats = params.getCoordinatorStats(); - Assert.assertTrue(overlordClient.observedKillIntervals.containsKey(DS.WIKI)); - Assert.assertEquals(2, stats.get(Stats.Kill.PENDING_SEGMENTS, RowKey.of(Dimension.DATASOURCE, DS.WIKI))); + Assert.assertTrue(overlordClient.observedKillIntervals.containsKey(TestDataSource.WIKI)); + Assert.assertEquals(2, stats.get(Stats.Kill.PENDING_SEGMENTS, RowKey.of(Dimension.DATASOURCE, TestDataSource.WIKI))); - Assert.assertFalse(overlordClient.observedKillIntervals.containsKey(DS.KOALA)); - Assert.assertEquals(0, stats.get(Stats.Kill.PENDING_SEGMENTS, RowKey.of(Dimension.DATASOURCE, DS.KOALA))); + Assert.assertFalse(overlordClient.observedKillIntervals.containsKey(TestDataSource.KOALA)); + Assert.assertEquals(0, stats.get(Stats.Kill.PENDING_SEGMENTS, RowKey.of(Dimension.DATASOURCE, TestDataSource.KOALA))); } private DruidCoordinatorRuntimeParams.Builder createParamsWithDatasources(String... datasources) @@ -188,12 +189,6 @@ private DruidCoordinatorRuntimeParams.Builder createParamsWithDatasources(String return builder.withUsedSegments(usedSegments); } - private static class DS - { - static final String WIKI = "wiki"; - static final String KOALA = "koala"; - } - /** * Simulates an Overlord with a configurable list of tasks and pending segments. */ @@ -254,7 +249,7 @@ public ListenableFuture killPendingSegments(String dataSource, Interval } } - if (remainingPendingSegments.size() > 0) { + if (!remainingPendingSegments.isEmpty()) { datasourceToPendingSegments.put(dataSource, remainingPendingSegments); } diff --git a/server/src/test/java/org/apache/druid/server/coordinator/loading/RoundRobinServerSelectorTest.java b/server/src/test/java/org/apache/druid/server/coordinator/loading/RoundRobinServerSelectorTest.java index a3c769ef96e2..cc9aa92fd9ac 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/loading/RoundRobinServerSelectorTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/loading/RoundRobinServerSelectorTest.java @@ -22,6 +22,7 @@ import org.apache.druid.client.DruidServer; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.segment.IndexIO; +import org.apache.druid.segment.TestDataSource; import org.apache.druid.server.coordination.ServerType; import org.apache.druid.server.coordinator.DruidCluster; import org.apache.druid.server.coordinator.ServerHolder; @@ -38,7 +39,7 @@ public class RoundRobinServerSelectorTest private static final String TIER = "normal"; private final DataSegment segment = new DataSegment( - "wiki", + TestDataSource.WIKI, Intervals.of("2022-01-01/2022-01-02"), "1", Collections.emptyMap(), diff --git a/server/src/test/java/org/apache/druid/server/coordinator/rules/BroadcastDistributionRuleTest.java b/server/src/test/java/org/apache/druid/server/coordinator/rules/BroadcastDistributionRuleTest.java index 46e62fe14321..d430c8646418 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/rules/BroadcastDistributionRuleTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/rules/BroadcastDistributionRuleTest.java @@ -22,6 +22,7 @@ import org.apache.druid.client.DruidServer; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.granularity.Granularities; +import org.apache.druid.segment.TestDataSource; import org.apache.druid.server.coordination.ServerType; import org.apache.druid.server.coordinator.CreateDataSegments; import org.apache.druid.server.coordinator.DruidCluster; @@ -47,12 +48,11 @@ public class BroadcastDistributionRuleTest { private int serverId = 0; - private static final String DS_WIKI = "wiki"; private static final String TIER_1 = "tier1"; private static final String TIER_2 = "tier2"; private final DataSegment wikiSegment - = CreateDataSegments.ofDatasource(DS_WIKI).eachOfSizeInMb(100).get(0); + = CreateDataSegments.ofDatasource(TestDataSource.WIKI).eachOfSizeInMb(100).get(0); @Before public void setUp() @@ -73,10 +73,10 @@ public void testSegmentIsBroadcastToAllTiers() CoordinatorRunStats stats = runRuleOnSegment(rule, wikiSegment, params); // Verify that segment is assigned to servers of all tiers - Assert.assertEquals(1L, stats.getSegmentStat(Stats.Segments.ASSIGNED, TIER_1, DS_WIKI)); + Assert.assertEquals(1L, stats.getSegmentStat(Stats.Segments.ASSIGNED, TIER_1, TestDataSource.WIKI)); Assert.assertTrue(serverT11.isLoadingSegment(wikiSegment)); - Assert.assertEquals(1L, stats.getSegmentStat(Stats.Segments.ASSIGNED, TIER_2, DS_WIKI)); + Assert.assertEquals(1L, stats.getSegmentStat(Stats.Segments.ASSIGNED, TIER_2, TestDataSource.WIKI)); Assert.assertTrue(serverT21.isLoadingSegment(wikiSegment)); } @@ -93,7 +93,7 @@ public void testSegmentIsNotBroadcastToServerIfAlreadyLoaded() CoordinatorRunStats stats = runRuleOnSegment(rule, wikiSegment, params); // Verify that serverT11 is already serving and serverT12 is loading segment - Assert.assertEquals(1L, stats.getSegmentStat(Stats.Segments.ASSIGNED, TIER_1, DS_WIKI)); + Assert.assertEquals(1L, stats.getSegmentStat(Stats.Segments.ASSIGNED, TIER_1, TestDataSource.WIKI)); Assert.assertFalse(serverT11.isLoadingSegment(wikiSegment)); Assert.assertTrue(serverT11.isServingSegment(wikiSegment)); Assert.assertTrue(serverT12.isLoadingSegment(wikiSegment)); @@ -112,7 +112,7 @@ public void testSegmentIsNotBroadcastToDecommissioningServer() ForeverBroadcastDistributionRule rule = new ForeverBroadcastDistributionRule(); CoordinatorRunStats stats = runRuleOnSegment(rule, wikiSegment, params); - Assert.assertEquals(1L, stats.getSegmentStat(Stats.Segments.ASSIGNED, TIER_1, DS_WIKI)); + Assert.assertEquals(1L, stats.getSegmentStat(Stats.Segments.ASSIGNED, TIER_1, TestDataSource.WIKI)); Assert.assertTrue(activeServer.isLoadingSegment(wikiSegment)); Assert.assertTrue(decommissioningServer.getLoadingSegments().isEmpty()); } @@ -133,7 +133,7 @@ public void testBroadcastSegmentIsDroppedFromDecommissioningServer() CoordinatorRunStats stats = runRuleOnSegment(rule, wikiSegment, params); // Verify that segment is dropped only from the decommissioning server - Assert.assertEquals(1L, stats.getSegmentStat(Stats.Segments.DROPPED, TIER_1, DS_WIKI)); + Assert.assertEquals(1L, stats.getSegmentStat(Stats.Segments.DROPPED, TIER_1, TestDataSource.WIKI)); Assert.assertTrue(activeServer.getPeon().getSegmentsToDrop().isEmpty()); Assert.assertTrue(decommissioningServer.getPeon().getSegmentsToDrop().contains(wikiSegment)); } @@ -160,9 +160,9 @@ public void testSegmentIsBroadcastToAllServerTypes() final CoordinatorRunStats stats = runRuleOnSegment(rule, wikiSegment, params); // Verify that segment is assigned to historical, broker as well as indexer - Assert.assertEquals(1L, stats.getSegmentStat(Stats.Segments.ASSIGNED, TIER_1, DS_WIKI)); - Assert.assertEquals(1L, stats.getSegmentStat(Stats.Segments.ASSIGNED, TIER_2, DS_WIKI)); - Assert.assertEquals(1L, stats.getSegmentStat(Stats.Segments.ASSIGNED, broker.getServer().getTier(), DS_WIKI)); + Assert.assertEquals(1L, stats.getSegmentStat(Stats.Segments.ASSIGNED, TIER_1, TestDataSource.WIKI)); + Assert.assertEquals(1L, stats.getSegmentStat(Stats.Segments.ASSIGNED, TIER_2, TestDataSource.WIKI)); + Assert.assertEquals(1L, stats.getSegmentStat(Stats.Segments.ASSIGNED, broker.getServer().getTier(), TestDataSource.WIKI)); Assert.assertTrue(historical.isLoadingSegment(wikiSegment)); Assert.assertTrue(indexer.isLoadingSegment(wikiSegment)); @@ -205,13 +205,13 @@ public void testReasonForBroadcastFailure() final CoordinatorRunStats stats = runRuleOnSegment(rule, wikiSegment, params); // Verify that the segment is broadcast only to the eligible server - Assert.assertEquals(1L, stats.getSegmentStat(Stats.Segments.ASSIGNED, TIER_1, DS_WIKI)); - RowKey metricKey = RowKey.with(Dimension.DATASOURCE, DS_WIKI) + Assert.assertEquals(1L, stats.getSegmentStat(Stats.Segments.ASSIGNED, TIER_1, TestDataSource.WIKI)); + RowKey metricKey = RowKey.with(Dimension.DATASOURCE, TestDataSource.WIKI) .with(Dimension.TIER, TIER_1) .and(Dimension.DESCRIPTION, "Not enough disk space"); Assert.assertEquals(1L, stats.get(Stats.Segments.ASSIGN_SKIPPED, metricKey)); - metricKey = RowKey.with(Dimension.DATASOURCE, DS_WIKI) + metricKey = RowKey.with(Dimension.DATASOURCE, TestDataSource.WIKI) .with(Dimension.TIER, TIER_1) .and(Dimension.DESCRIPTION, "Load queue is full"); Assert.assertEquals(1L, stats.get(Stats.Segments.ASSIGN_SKIPPED, metricKey)); diff --git a/server/src/test/java/org/apache/druid/server/coordinator/rules/LoadRuleTest.java b/server/src/test/java/org/apache/druid/server/coordinator/rules/LoadRuleTest.java index 1e43d89bdda4..5face5b85080 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/rules/LoadRuleTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/rules/LoadRuleTest.java @@ -28,6 +28,7 @@ import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.common.granularity.Granularities; +import org.apache.druid.segment.TestDataSource; import org.apache.druid.server.coordination.ServerType; import org.apache.druid.server.coordinator.CoordinatorDynamicConfig; import org.apache.druid.server.coordinator.CreateDataSegments; @@ -65,8 +66,6 @@ @RunWith(Parameterized.class) public class LoadRuleTest { - private static final String DS_WIKI = "wiki"; - private ListeningExecutorService exec; private BalancerStrategy balancerStrategy; @@ -112,13 +111,13 @@ public void testLoadRuleAssignsSegments() .addTier(Tier.T2, server2) .build(); - final DataSegment segment = createDataSegment(DS_WIKI); + final DataSegment segment = createDataSegment(TestDataSource.WIKI); LoadRule rule = loadForever(ImmutableMap.of(Tier.T1, 1, Tier.T2, 2)); Assert.assertTrue(rule.shouldMatchingSegmentBeLoaded()); CoordinatorRunStats stats = runRuleAndGetStats(rule, segment, druidCluster); - Assert.assertEquals(1L, stats.getSegmentStat(Stats.Segments.ASSIGNED, Tier.T1, DS_WIKI)); - Assert.assertEquals(1L, stats.getSegmentStat(Stats.Segments.ASSIGNED, Tier.T2, DS_WIKI)); + Assert.assertEquals(1L, stats.getSegmentStat(Stats.Segments.ASSIGNED, Tier.T1, TestDataSource.WIKI)); + Assert.assertEquals(1L, stats.getSegmentStat(Stats.Segments.ASSIGNED, Tier.T2, TestDataSource.WIKI)); } private CoordinatorRunStats runRuleAndGetStats(LoadRule rule, DataSegment segment, DruidCluster cluster) @@ -168,7 +167,7 @@ public void testLoadPrimaryAssignDoesNotOverAssign() .build(); final LoadRule rule = loadForever(ImmutableMap.of(Tier.T1, 1)); - final DataSegment segment = createDataSegment(DS_WIKI); + final DataSegment segment = createDataSegment(TestDataSource.WIKI); CoordinatorRunStats firstRunStats = runRuleAndGetStats(rule, segment, druidCluster); Assert.assertEquals(1L, firstRunStats.getSegmentStat(Stats.Segments.ASSIGNED, Tier.T1, segment.getDataSource())); Assert.assertEquals(1, server1.getLoadingSegments().size() + server2.getLoadingSegments().size()); @@ -191,7 +190,7 @@ public void testOverAssignForTimedOutSegments() .build(); final LoadRule rule = loadForever(ImmutableMap.of(Tier.T1, 1)); - final DataSegment segment = createDataSegment(DS_WIKI); + final DataSegment segment = createDataSegment(TestDataSource.WIKI); CoordinatorRunStats stats = runRuleAndGetStats(rule, segment, druidCluster); // Ensure that the segment is assigned to one of the historicals @@ -199,7 +198,7 @@ public void testOverAssignForTimedOutSegments() // Ensure that the primary segment is assigned again in case the peon timed out on loading the segment CoordinatorRunStats statsAfterLoadPrimary = runRuleAndGetStats(rule, segment, druidCluster); - Assert.assertEquals(1L, statsAfterLoadPrimary.getSegmentStat(Stats.Segments.ASSIGNED, Tier.T1, DS_WIKI)); + Assert.assertEquals(1L, statsAfterLoadPrimary.getSegmentStat(Stats.Segments.ASSIGNED, Tier.T1, TestDataSource.WIKI)); } @Test @@ -213,7 +212,7 @@ public void testSkipReplicationForTimedOutSegments() .build(); final LoadRule rule = loadForever(ImmutableMap.of(Tier.T1, 1)); - final DataSegment segment = createDataSegment(DS_WIKI); + final DataSegment segment = createDataSegment(TestDataSource.WIKI); CoordinatorRunStats stats = runRuleAndGetStats(rule, segment, druidCluster); // Ensure that the segment is assigned to one of the historicals @@ -230,7 +229,7 @@ public void testSkipReplicationForTimedOutSegments() public void testLoadUsedSegmentsForAllSegmentGranularityAndCachingCostBalancerStrategy() { final List segments = - CreateDataSegments.ofDatasource(DS_WIKI) + CreateDataSegments.ofDatasource(TestDataSource.WIKI) .forIntervals(1, Granularities.ALL) .withNumPartitions(2) .eachOfSizeInMb(100); @@ -249,13 +248,13 @@ public void testLoadUsedSegmentsForAllSegmentGranularityAndCachingCostBalancerSt segments.get(1), makeCoordinatorRuntimeParams(druidCluster, segments.toArray(new DataSegment[0])) ); - Assert.assertEquals(1L, stats.getSegmentStat(Stats.Segments.ASSIGNED, Tier.T1, DS_WIKI)); + Assert.assertEquals(1L, stats.getSegmentStat(Stats.Segments.ASSIGNED, Tier.T1, TestDataSource.WIKI)); } @Test public void testSegmentsAreDroppedIfLoadRuleHasZeroReplicas() { - final DataSegment segment = createDataSegment(DS_WIKI); + final DataSegment segment = createDataSegment(TestDataSource.WIKI); final ServerHolder serverT11 = createServer(Tier.T1, segment); final ServerHolder serverT12 = createServer(Tier.T2, segment); @@ -271,8 +270,8 @@ public void testSegmentsAreDroppedIfLoadRuleHasZeroReplicas() Assert.assertFalse(rule.shouldMatchingSegmentBeLoaded()); CoordinatorRunStats stats = runRuleAndGetStats(rule, segment, druidCluster); - Assert.assertEquals(1L, stats.getSegmentStat(Stats.Segments.DROPPED, Tier.T1, DS_WIKI)); - Assert.assertEquals(2L, stats.getSegmentStat(Stats.Segments.DROPPED, Tier.T2, DS_WIKI)); + Assert.assertEquals(1L, stats.getSegmentStat(Stats.Segments.DROPPED, Tier.T1, TestDataSource.WIKI)); + Assert.assertEquals(2L, stats.getSegmentStat(Stats.Segments.DROPPED, Tier.T2, TestDataSource.WIKI)); } @Test @@ -284,18 +283,18 @@ public void testLoadIgnoresInvalidTiers() .addTier(Tier.T1, server) .build(); - final DataSegment segment = createDataSegment(DS_WIKI); + final DataSegment segment = createDataSegment(TestDataSource.WIKI); LoadRule rule = loadForever(ImmutableMap.of("invalidTier", 1, Tier.T1, 1)); Assert.assertTrue(rule.shouldMatchingSegmentBeLoaded()); CoordinatorRunStats stats = runRuleAndGetStats(rule, segment, druidCluster); - Assert.assertEquals(1L, stats.getSegmentStat(Stats.Segments.ASSIGNED, Tier.T1, DS_WIKI)); - Assert.assertEquals(0L, stats.getSegmentStat(Stats.Segments.ASSIGNED, "invalidTier", DS_WIKI)); + Assert.assertEquals(1L, stats.getSegmentStat(Stats.Segments.ASSIGNED, Tier.T1, TestDataSource.WIKI)); + Assert.assertEquals(0L, stats.getSegmentStat(Stats.Segments.ASSIGNED, "invalidTier", TestDataSource.WIKI)); } @Test public void testDropIgnoresInvalidTiers() { - final DataSegment segment = createDataSegment(DS_WIKI); + final DataSegment segment = createDataSegment(TestDataSource.WIKI); // Cluster has 1 tier with 2 servers ServerHolder server1 = createServer(Tier.T1, segment); @@ -308,8 +307,8 @@ public void testDropIgnoresInvalidTiers() LoadRule rule = loadForever(ImmutableMap.of("invalidTier", 1, Tier.T1, 1)); CoordinatorRunStats stats = runRuleAndGetStats(rule, segment, druidCluster); - Assert.assertEquals(1L, stats.getSegmentStat(Stats.Segments.DROPPED, Tier.T1, DS_WIKI)); - Assert.assertEquals(0L, stats.getSegmentStat(Stats.Segments.DROPPED, "invalidTier", DS_WIKI)); + Assert.assertEquals(1L, stats.getSegmentStat(Stats.Segments.DROPPED, Tier.T1, TestDataSource.WIKI)); + Assert.assertEquals(0L, stats.getSegmentStat(Stats.Segments.DROPPED, "invalidTier", TestDataSource.WIKI)); } @Test @@ -374,11 +373,11 @@ public void testSegmentIsAssignedOnlyToActiveServer() // Load rule requires 1 replica on each tier LoadRule rule = loadForever(ImmutableMap.of(Tier.T1, 1, Tier.T2, 1)); Assert.assertTrue(rule.shouldMatchingSegmentBeLoaded()); - DataSegment segment = createDataSegment(DS_WIKI); + DataSegment segment = createDataSegment(TestDataSource.WIKI); CoordinatorRunStats stats = runRuleAndGetStats(rule, segment, druidCluster); // Verify that segment is not loaded on decommissioning server - Assert.assertEquals(1L, stats.getSegmentStat(Stats.Segments.ASSIGNED, Tier.T2, DS_WIKI)); + Assert.assertEquals(1L, stats.getSegmentStat(Stats.Segments.ASSIGNED, Tier.T2, TestDataSource.WIKI)); Assert.assertEquals(0, decommServerT1.getLoadingSegments().size()); Assert.assertTrue(serverT2.getLoadingSegments().contains(segment)); } @@ -392,7 +391,7 @@ public void testSegmentIsAssignedOnlyToActiveServers() ServerHolder serverT21 = createServer(Tier.T2); ServerHolder serverT22 = createServer(Tier.T2); - final DataSegment segment = createDataSegment(DS_WIKI); + final DataSegment segment = createDataSegment(TestDataSource.WIKI); DruidCluster druidCluster = DruidCluster .builder() .addTier(Tier.T1, decommServerT11, serverT12) @@ -404,11 +403,11 @@ public void testSegmentIsAssignedOnlyToActiveServers() CoordinatorRunStats stats = runRuleAndGetStats(rule, segment, druidCluster); // Verify that no replica is assigned to decommissioning server - Assert.assertEquals(1L, stats.getSegmentStat(Stats.Segments.ASSIGNED, Tier.T1, DS_WIKI)); + Assert.assertEquals(1L, stats.getSegmentStat(Stats.Segments.ASSIGNED, Tier.T1, TestDataSource.WIKI)); Assert.assertTrue(decommServerT11.getLoadingSegments().isEmpty()); Assert.assertEquals(0, decommServerT11.getLoadingSegments().size()); - Assert.assertEquals(2L, stats.getSegmentStat(Stats.Segments.ASSIGNED, Tier.T2, DS_WIKI)); + Assert.assertEquals(2L, stats.getSegmentStat(Stats.Segments.ASSIGNED, Tier.T2, TestDataSource.WIKI)); } /** @@ -444,7 +443,7 @@ public void testDropDuringDecommissioning() @Test public void testExtraReplicasAreDroppedFromDecommissioningServer() { - final DataSegment segment1 = createDataSegment(DS_WIKI); + final DataSegment segment1 = createDataSegment(TestDataSource.WIKI); // 3 servers, each serving the same segment final ServerHolder server1 = createServer(Tier.T1, segment1); @@ -465,7 +464,7 @@ public void testExtraReplicasAreDroppedFromDecommissioningServer() ); // Verify that the extra replica is dropped from the decommissioning server - Assert.assertEquals(1L, stats.getSegmentStat(Stats.Segments.DROPPED, Tier.T1, DS_WIKI)); + Assert.assertEquals(1L, stats.getSegmentStat(Stats.Segments.DROPPED, Tier.T1, TestDataSource.WIKI)); Assert.assertEquals(0, server1.getPeon().getSegmentsToDrop().size()); Assert.assertEquals(1, server2.getPeon().getSegmentsToDrop().size()); Assert.assertEquals(0, server3.getPeon().getSegmentsToDrop().size()); diff --git a/server/src/test/java/org/apache/druid/server/coordinator/simulate/CoordinatorSimulationBaseTest.java b/server/src/test/java/org/apache/druid/server/coordinator/simulate/CoordinatorSimulationBaseTest.java index 6fd23bd8f469..1fd3d54dc456 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/simulate/CoordinatorSimulationBaseTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/simulate/CoordinatorSimulationBaseTest.java @@ -22,6 +22,7 @@ import org.apache.druid.client.DruidServer; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.metrics.MetricsVerifier; +import org.apache.druid.segment.TestDataSource; import org.apache.druid.server.coordination.ServerType; import org.apache.druid.server.coordinator.CoordinatorDynamicConfig; import org.apache.druid.server.coordinator.CreateDataSegments; @@ -201,12 +202,6 @@ static DruidServer createHistorical(int uniqueIdInTier, String tier, long server // Utility and constant holder classes - static class DS - { - static final String WIKI = "wiki"; - static final String KOALA = "koala"; - } - static class Tier { static final String T1 = "tier_t1"; @@ -230,33 +225,33 @@ static class Metric static class Segments { /** - * Segments of datasource {@link DS#WIKI}, size 500 MB each, + * Segments of datasource {@link TestDataSource#WIKI}, size 500 MB each, * spanning 1 day containing 10 partitions each. */ static final List WIKI_10X1D = - CreateDataSegments.ofDatasource(DS.WIKI) + CreateDataSegments.ofDatasource(TestDataSource.WIKI) .forIntervals(1, Granularities.DAY) .startingAt("2022-01-01") .withNumPartitions(10) .eachOfSizeInMb(500); /** - * Segments of datasource {@link DS#WIKI}, size 500 MB each, + * Segments of datasource {@link TestDataSource#WIKI}, size 500 MB each, * spanning 100 days containing 10 partitions each. */ static final List WIKI_10X100D = - CreateDataSegments.ofDatasource(DS.WIKI) + CreateDataSegments.ofDatasource(TestDataSource.WIKI) .forIntervals(100, Granularities.DAY) .startingAt("2022-01-01") .withNumPartitions(10) .eachOfSizeInMb(500); /** - * Segments of datasource {@link DS#KOALA}, size 500 MB each, + * Segments of datasource {@link TestDataSource#KOALA}, size 500 MB each, * spanning 100 days containing 100 partitions each. */ static final List KOALA_100X100D = - CreateDataSegments.ofDatasource(DS.KOALA) + CreateDataSegments.ofDatasource(TestDataSource.KOALA) .forIntervals(100, Granularities.DAY) .startingAt("2022-01-01") .withNumPartitions(100) diff --git a/server/src/test/java/org/apache/druid/server/coordinator/simulate/RoundRobinAssignmentTest.java b/server/src/test/java/org/apache/druid/server/coordinator/simulate/RoundRobinAssignmentTest.java index 0fe884f58ebb..ee78ad17ad4a 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/simulate/RoundRobinAssignmentTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/simulate/RoundRobinAssignmentTest.java @@ -20,6 +20,7 @@ package org.apache.druid.server.coordinator.simulate; import org.apache.druid.client.DruidServer; +import org.apache.druid.segment.TestDataSource; import org.apache.druid.server.coordinator.CoordinatorDynamicConfig; import org.apache.druid.timeline.DataSegment; import org.junit.Assert; @@ -58,7 +59,7 @@ public void testSegmentsAreAssignedUniformly() CoordinatorSimulation.builder() .withDynamicConfig(config) .withBalancer("random") - .withRules(DS.WIKI, Load.on(Tier.T1, 2).forever()) + .withRules(TestDataSource.WIKI, Load.on(Tier.T1, 2).forever()) .withServers(historicals) .withSegments(Segments.WIKI_10X100D) .build(); @@ -92,8 +93,8 @@ public void testMultipleDatasourceSegmentsAreAssignedUniformly() CoordinatorSimulation.builder() .withDynamicConfig(config) .withBalancer("random") - .withRules(DS.WIKI, Load.on(Tier.T1, 3).forever()) - .withRules(DS.KOALA, Load.on(Tier.T1, 1).forever()) + .withRules(TestDataSource.WIKI, Load.on(Tier.T1, 3).forever()) + .withRules(TestDataSource.KOALA, Load.on(Tier.T1, 1).forever()) .withServers(historicals) .withSegments(segments) .build(); @@ -102,8 +103,8 @@ public void testMultipleDatasourceSegmentsAreAssignedUniformly() // Run 1: all segments are assigned and loaded runCoordinatorCycle(); loadQueuedSegments(); - verifyValue(Metric.ASSIGNED_COUNT, filterByDatasource(DS.KOALA), 10000L); - verifyValue(Metric.ASSIGNED_COUNT, filterByDatasource(DS.WIKI), 3000L); + verifyValue(Metric.ASSIGNED_COUNT, filterByDatasource(TestDataSource.KOALA), 10000L); + verifyValue(Metric.ASSIGNED_COUNT, filterByDatasource(TestDataSource.WIKI), 3000L); for (DruidServer historical : historicals) { Assert.assertEquals(1300, historical.getTotalSegments()); diff --git a/server/src/test/java/org/apache/druid/server/http/CoordinatorCompactionConfigsResourceTest.java b/server/src/test/java/org/apache/druid/server/http/CoordinatorCompactionConfigsResourceTest.java index e8ae3c93b66c..24641c81a156 100644 --- a/server/src/test/java/org/apache/druid/server/http/CoordinatorCompactionConfigsResourceTest.java +++ b/server/src/test/java/org/apache/druid/server/http/CoordinatorCompactionConfigsResourceTest.java @@ -37,6 +37,7 @@ import org.apache.druid.metadata.MetadataStorageTablesConfig; import org.apache.druid.metadata.TestMetadataStorageConnector; import org.apache.druid.metadata.TestMetadataStorageTablesConfig; +import org.apache.druid.segment.TestDataSource; import org.apache.druid.server.coordinator.ClusterCompactionConfig; import org.apache.druid.server.coordinator.CoordinatorConfigManager; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; @@ -154,7 +155,7 @@ public void testSetCompactionTaskLimit() @Test public void testGetUnknownDatasourceConfigThrowsNotFound() { - Response response = resource.getDatasourceCompactionConfig(DS.WIKI); + Response response = resource.getDatasourceCompactionConfig(TestDataSource.WIKI); verifyStatus(Response.Status.NOT_FOUND, response); } @@ -162,12 +163,12 @@ public void testGetUnknownDatasourceConfigThrowsNotFound() public void testAddDatasourceConfig() { final DataSourceCompactionConfig newDatasourceConfig - = DataSourceCompactionConfig.builder().forDataSource(DS.WIKI).build(); + = DataSourceCompactionConfig.builder().forDataSource(TestDataSource.WIKI).build(); Response response = resource.addOrUpdateDatasourceCompactionConfig(newDatasourceConfig, mockHttpServletRequest); verifyStatus(Response.Status.OK, response); final DataSourceCompactionConfig fetchedDatasourceConfig - = verifyAndGetPayload(resource.getDatasourceCompactionConfig(DS.WIKI), DataSourceCompactionConfig.class); + = verifyAndGetPayload(resource.getDatasourceCompactionConfig(TestDataSource.WIKI), DataSourceCompactionConfig.class); Assert.assertEquals(newDatasourceConfig, fetchedDatasourceConfig); final DruidCompactionConfig fullCompactionConfig @@ -181,7 +182,7 @@ public void testUpdateDatasourceConfig() { final DataSourceCompactionConfig originalDatasourceConfig = DataSourceCompactionConfig .builder() - .forDataSource(DS.WIKI) + .forDataSource(TestDataSource.WIKI) .withInputSegmentSizeBytes(500L) .withSkipOffsetFromLatest(Period.hours(1)) .withGranularitySpec( @@ -198,7 +199,7 @@ public void testUpdateDatasourceConfig() final DataSourceCompactionConfig updatedDatasourceConfig = DataSourceCompactionConfig .builder() - .forDataSource(DS.WIKI) + .forDataSource(TestDataSource.WIKI) .withInputSegmentSizeBytes(1000L) .withSkipOffsetFromLatest(Period.hours(3)) .withGranularitySpec( @@ -211,7 +212,7 @@ public void testUpdateDatasourceConfig() verifyStatus(Response.Status.OK, response); final DataSourceCompactionConfig latestDatasourceConfig - = verifyAndGetPayload(resource.getDatasourceCompactionConfig(DS.WIKI), DataSourceCompactionConfig.class); + = verifyAndGetPayload(resource.getDatasourceCompactionConfig(TestDataSource.WIKI), DataSourceCompactionConfig.class); Assert.assertEquals(updatedDatasourceConfig, latestDatasourceConfig); final DruidCompactionConfig fullCompactionConfig @@ -224,21 +225,21 @@ public void testUpdateDatasourceConfig() public void testDeleteDatasourceConfig() { final DataSourceCompactionConfig datasourceConfig - = DataSourceCompactionConfig.builder().forDataSource(DS.WIKI).build(); + = DataSourceCompactionConfig.builder().forDataSource(TestDataSource.WIKI).build(); Response response = resource.addOrUpdateDatasourceCompactionConfig(datasourceConfig, mockHttpServletRequest); verifyStatus(Response.Status.OK, response); - response = resource.deleteCompactionConfig(DS.WIKI, mockHttpServletRequest); + response = resource.deleteCompactionConfig(TestDataSource.WIKI, mockHttpServletRequest); verifyStatus(Response.Status.OK, response); - response = resource.getDatasourceCompactionConfig(DS.WIKI); + response = resource.getDatasourceCompactionConfig(TestDataSource.WIKI); verifyStatus(Response.Status.NOT_FOUND, response); } @Test public void testDeleteUnknownDatasourceConfigThrowsNotFound() { - Response response = resource.deleteCompactionConfig(DS.WIKI, mockHttpServletRequest); + Response response = resource.deleteCompactionConfig(TestDataSource.WIKI, mockHttpServletRequest); verifyStatus(Response.Status.NOT_FOUND, response); } @@ -248,7 +249,7 @@ public void testUpdateIsRetriedIfFailureIsRetryable() configManager.configUpdateResult = ConfigManager.SetResult.retryableFailure(new Exception("retryable")); resource.addOrUpdateDatasourceCompactionConfig( - DataSourceCompactionConfig.builder().forDataSource(DS.WIKI).build(), + DataSourceCompactionConfig.builder().forDataSource(TestDataSource.WIKI).build(), mockHttpServletRequest ); @@ -264,7 +265,7 @@ public void testUpdateIsNotRetriedIfFailureIsNotRetryable() configManager.configUpdateResult = ConfigManager.SetResult.failure(new Exception("not retryable")); resource.addOrUpdateDatasourceCompactionConfig( - DataSourceCompactionConfig.builder().forDataSource(DS.WIKI).build(), + DataSourceCompactionConfig.builder().forDataSource(TestDataSource.WIKI).build(), mockHttpServletRequest ); @@ -275,7 +276,7 @@ public void testUpdateIsNotRetriedIfFailureIsNotRetryable() public void testGetDatasourceConfigHistory() { final DataSourceCompactionConfig.Builder builder - = DataSourceCompactionConfig.builder().forDataSource(DS.WIKI); + = DataSourceCompactionConfig.builder().forDataSource(TestDataSource.WIKI); final DataSourceCompactionConfig configV1 = builder.build(); resource.addOrUpdateDatasourceCompactionConfig(configV1, mockHttpServletRequest); @@ -289,7 +290,7 @@ public void testGetDatasourceConfigHistory() .build(); resource.addOrUpdateDatasourceCompactionConfig(configV3, mockHttpServletRequest); - Response response = resource.getCompactionConfigHistory(DS.WIKI, null, null); + Response response = resource.getCompactionConfigHistory(TestDataSource.WIKI, null, null); verifyStatus(Response.Status.OK, response); final List history @@ -303,7 +304,7 @@ public void testGetDatasourceConfigHistory() @Test public void testGetHistoryOfUnknownDatasourceReturnsEmpty() { - Response response = resource.getCompactionConfigHistory(DS.WIKI, null, null); + Response response = resource.getCompactionConfigHistory(TestDataSource.WIKI, null, null); verifyStatus(Response.Status.OK, response); Assert.assertTrue(((List) response.getEntity()).isEmpty()); } @@ -313,7 +314,7 @@ public void testAddInvalidDatasourceConfigThrowsBadRequest() { final DataSourceCompactionConfig datasourceConfig = DataSourceCompactionConfig .builder() - .forDataSource(DS.WIKI) + .forDataSource(TestDataSource.WIKI) .withTaskContext(Collections.singletonMap(ClientMSQContext.CTX_MAX_NUM_TASKS, 1)) .withEngine(CompactionEngine.MSQ) .build(); @@ -333,7 +334,7 @@ public void testUpdateEngineToMSQWithInvalidDatasourceConfigThrowsBadRequest() { final DataSourceCompactionConfig datasourceConfig = DataSourceCompactionConfig .builder() - .forDataSource(DS.WIKI) + .forDataSource(TestDataSource.WIKI) .withTaskContext(Collections.singletonMap(ClientMSQContext.CTX_MAX_NUM_TASKS, 1)) .build(); Response response = resource.addOrUpdateDatasourceCompactionConfig(datasourceConfig, mockHttpServletRequest); @@ -540,9 +541,4 @@ public boolean compareAndSwap(List updates) return true; } } - - private static class DS - { - static final String WIKI = "wiki"; - } } From 517f9ad39c0de25db2ab482663f4374e8f6be1d6 Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Sun, 1 Sep 2024 12:40:18 +0530 Subject: [PATCH 24/26] Rename SegmentsToCompact to CompactionCandidate --- .../NewestSegmentFirstPolicyBenchmark.java | 4 +- ...cy.java => BaseCandidateSearchPolicy.java} | 54 +++++++++---------- ...oCompact.java => CompactionCandidate.java} | 47 ++++++++-------- ...a => CompactionCandidateSearchPolicy.java} | 32 +++++------ .../compaction/CompactionRunSimulator.java | 12 ++--- .../compaction/CompactionSegmentIterator.java | 6 +-- .../server/compaction/CompactionStatus.java | 11 ++-- .../compaction/CompactionStatusTracker.java | 16 +++--- .../DataSourceCompactibleSegmentIterator.java | 39 ++++++-------- .../compaction/NewestSegmentFirstPolicy.java | 6 +-- ...riorityBasedCompactionSegmentIterator.java | 38 +++++-------- .../SmallestSegmentFirstPolicy.java | 49 ----------------- .../coordinator/AutoCompactionSnapshot.java | 18 ------- .../coordinator/ClusterCompactionConfig.java | 8 +-- .../coordinator/DruidCompactionConfig.java | 10 ++-- .../coordinator/duty/CompactSegments.java | 35 ++++++------ .../compaction/CompactionStatusTest.java | 4 +- .../CompactionStatusTrackerTest.java | 16 +++--- .../NewestSegmentFirstPolicyTest.java | 21 ++++---- .../simulate/BalancingStrategiesTest.java | 7 +-- .../simulate/MarkSegmentsAsUnusedTest.java | 3 +- .../simulate/SegmentBalancingTest.java | 5 +- .../simulate/SegmentLoadingTest.java | 7 +-- 23 files changed, 177 insertions(+), 271 deletions(-) rename server/src/main/java/org/apache/druid/server/compaction/{PriorityBasedSegmentSearchPolicy.java => BaseCandidateSearchPolicy.java} (54%) rename server/src/main/java/org/apache/druid/server/compaction/{SegmentsToCompact.java => CompactionCandidate.java} (72%) rename server/src/main/java/org/apache/druid/server/compaction/{CompactionSegmentSearchPolicy.java => CompactionCandidateSearchPolicy.java} (56%) delete mode 100644 server/src/main/java/org/apache/druid/server/compaction/SmallestSegmentFirstPolicy.java diff --git a/benchmarks/src/test/java/org/apache/druid/server/coordinator/NewestSegmentFirstPolicyBenchmark.java b/benchmarks/src/test/java/org/apache/druid/server/coordinator/NewestSegmentFirstPolicyBenchmark.java index a8fae1c97474..3f109f2d0d6d 100644 --- a/benchmarks/src/test/java/org/apache/druid/server/coordinator/NewestSegmentFirstPolicyBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/server/coordinator/NewestSegmentFirstPolicyBenchmark.java @@ -25,7 +25,7 @@ import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.server.compaction.CompactionSegmentIterator; -import org.apache.druid.server.compaction.CompactionSegmentSearchPolicy; +import org.apache.druid.server.compaction.CompactionCandidateSearchPolicy; import org.apache.druid.server.compaction.CompactionStatusTracker; import org.apache.druid.server.compaction.NewestSegmentFirstPolicy; import org.apache.druid.timeline.DataSegment; @@ -62,7 +62,7 @@ public class NewestSegmentFirstPolicyBenchmark { private static final String DATA_SOURCE_PREFIX = "dataSource_"; - private final CompactionSegmentSearchPolicy policy = new NewestSegmentFirstPolicy(null); + private final CompactionCandidateSearchPolicy policy = new NewestSegmentFirstPolicy(null); @Param("100") private int numDataSources; diff --git a/server/src/main/java/org/apache/druid/server/compaction/PriorityBasedSegmentSearchPolicy.java b/server/src/main/java/org/apache/druid/server/compaction/BaseCandidateSearchPolicy.java similarity index 54% rename from server/src/main/java/org/apache/druid/server/compaction/PriorityBasedSegmentSearchPolicy.java rename to server/src/main/java/org/apache/druid/server/compaction/BaseCandidateSearchPolicy.java index 93f9b83a265a..21f6cafbfaf4 100644 --- a/server/src/main/java/org/apache/druid/server/compaction/PriorityBasedSegmentSearchPolicy.java +++ b/server/src/main/java/org/apache/druid/server/compaction/BaseCandidateSearchPolicy.java @@ -20,31 +20,37 @@ package org.apache.druid.server.compaction; import com.fasterxml.jackson.annotation.JsonProperty; -import org.apache.druid.server.coordinator.DataSourceCompactionConfig; -import org.apache.druid.timeline.SegmentTimeline; -import org.joda.time.Interval; +import org.apache.druid.java.util.common.guava.Comparators; import javax.annotation.Nullable; import java.util.Comparator; -import java.util.List; -import java.util.Map; import java.util.Objects; /** - * {@link CompactionSegmentSearchPolicy} that selects segments in order of a - * given priority. + * Base implementation of {@link CompactionCandidateSearchPolicy} that can have + * a {@code priorityDatasource}. */ -public abstract class PriorityBasedSegmentSearchPolicy implements CompactionSegmentSearchPolicy +public abstract class BaseCandidateSearchPolicy + implements CompactionCandidateSearchPolicy, Comparator { private final String priorityDatasource; + private final Comparator comparator; - protected PriorityBasedSegmentSearchPolicy( - @Nullable String priorityDatasource - ) + protected BaseCandidateSearchPolicy(@Nullable String priorityDatasource) { this.priorityDatasource = priorityDatasource; + if (priorityDatasource == null || priorityDatasource.isEmpty()) { + this.comparator = getSegmentComparator(); + } else { + this.comparator = Comparators.alwaysFirst(priorityDatasource) + .onResultOf(CompactionCandidate::getDataSource) + .thenComparing(getSegmentComparator()); + } } + /** + * The candidates of this datasource are prioritized over all others. + */ @Nullable @JsonProperty public final String getPriorityDatasource() @@ -53,25 +59,14 @@ public final String getPriorityDatasource() } @Override - public CompactionSegmentIterator createIterator( - Map compactionConfigs, - Map dataSources, - Map> skipIntervals, - CompactionStatusTracker statusTracker - ) + public final int compare(CompactionCandidate o1, CompactionCandidate o2) { - return new PriorityBasedCompactionSegmentIterator( - compactionConfigs, - dataSources, - skipIntervals, - this, - statusTracker - ); + return comparator.compare(o1, o2); } @Override public boolean isEligibleForCompaction( - SegmentsToCompact candidateSegments, + CompactionCandidate candidate, CompactionStatus currentCompactionStatus, CompactionTaskStatus latestTaskStatus ) @@ -80,9 +75,10 @@ public boolean isEligibleForCompaction( } /** - * Comparator used to prioritize between compactible segments. + * 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) @@ -93,8 +89,8 @@ public boolean equals(Object o) if (o == null || getClass() != o.getClass()) { return false; } - PriorityBasedSegmentSearchPolicy that = (PriorityBasedSegmentSearchPolicy) o; - return Objects.equals(priorityDatasource, that.priorityDatasource); + BaseCandidateSearchPolicy that = (BaseCandidateSearchPolicy) o; + return Objects.equals(this.priorityDatasource, that.priorityDatasource); } @Override diff --git a/server/src/main/java/org/apache/druid/server/compaction/SegmentsToCompact.java b/server/src/main/java/org/apache/druid/server/compaction/CompactionCandidate.java similarity index 72% rename from server/src/main/java/org/apache/druid/server/compaction/SegmentsToCompact.java rename to server/src/main/java/org/apache/druid/server/compaction/CompactionCandidate.java index 8aff403e35c4..4cd9b22df812 100644 --- a/server/src/main/java/org/apache/druid/server/compaction/SegmentsToCompact.java +++ b/server/src/main/java/org/apache/druid/server/compaction/CompactionCandidate.java @@ -27,13 +27,13 @@ import javax.annotation.Nullable; import java.util.List; -import java.util.NoSuchElementException; import java.util.stream.Collectors; /** - * List of segments to compact. + * Non-empty list of segments of a datasource being considered for compaction. + * A candidate typically contains all the segments of a single time chunk. */ -public class SegmentsToCompact +public class CompactionCandidate { private final List segments; private final Interval umbrellaInterval; @@ -43,16 +43,16 @@ public class SegmentsToCompact private final CompactionStatus currentStatus; - public static SegmentsToCompact from(List segments) + public static CompactionCandidate from(List segments) { if (segments == null || segments.isEmpty()) { throw InvalidInput.exception("Segments to compact must be non-empty"); } else { - return new SegmentsToCompact(segments, null); + return new CompactionCandidate(segments, null); } } - private SegmentsToCompact(List segments, @Nullable CompactionStatus currentStatus) + private CompactionCandidate(List segments, @Nullable CompactionStatus currentStatus) { this.segments = segments; this.totalBytes = segments.stream().mapToLong(DataSegment::getSize).sum(); @@ -64,35 +64,28 @@ private SegmentsToCompact(List segments, @Nullable CompactionStatus this.currentStatus = currentStatus; } + /** + * @return Non-empty list of segments that make up this candidate. + */ public List getSegments() { return segments; } - public DataSegment getFirst() - { - if (segments.isEmpty()) { - throw new NoSuchElementException("No segment to compact"); - } else { - return segments.get(0); - } - } - - public boolean isEmpty() - { - return segments.isEmpty(); - } - public long getTotalBytes() { return totalBytes; } - public int size() + public int numSegments() { return segments.size(); } + /** + * Umbrella interval of all the segments in this candidate. This typically + * corresponds to a single time chunk in the segment timeline. + */ public Interval getUmbrellaInterval() { return umbrellaInterval; @@ -105,18 +98,24 @@ public String getDataSource() public CompactionStatistics getStats() { - return CompactionStatistics.create(totalBytes, size(), numIntervals); + return CompactionStatistics.create(totalBytes, numSegments(), numIntervals); } + /** + * Current compaction status of the time chunk corresponding to this candidate. + */ @Nullable public CompactionStatus getCurrentStatus() { return currentStatus; } - public SegmentsToCompact withCurrentStatus(CompactionStatus status) + /** + * Creates a copy of this CompactionCandidate object with the given status. + */ + public CompactionCandidate withCurrentStatus(CompactionStatus status) { - return new SegmentsToCompact(this.segments, status); + return new CompactionCandidate(this.segments, status); } @Override diff --git a/server/src/main/java/org/apache/druid/server/compaction/CompactionSegmentSearchPolicy.java b/server/src/main/java/org/apache/druid/server/compaction/CompactionCandidateSearchPolicy.java similarity index 56% rename from server/src/main/java/org/apache/druid/server/compaction/CompactionSegmentSearchPolicy.java rename to server/src/main/java/org/apache/druid/server/compaction/CompactionCandidateSearchPolicy.java index 51fd688706a1..cf0e016f0540 100644 --- a/server/src/main/java/org/apache/druid/server/compaction/CompactionSegmentSearchPolicy.java +++ b/server/src/main/java/org/apache/druid/server/compaction/CompactionCandidateSearchPolicy.java @@ -21,41 +21,33 @@ import com.fasterxml.jackson.annotation.JsonSubTypes; import com.fasterxml.jackson.annotation.JsonTypeInfo; -import org.apache.druid.server.coordinator.DataSourceCompactionConfig; import org.apache.druid.server.coordinator.duty.CompactSegments; -import org.apache.druid.timeline.SegmentTimeline; -import org.joda.time.Interval; -import java.util.List; -import java.util.Map; +import java.util.Comparator; /** - * Segment searching policy used by {@link CompactSegments}. + * Policy used by {@link CompactSegments} duty to pick segments for compaction. */ @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type") @JsonSubTypes(value = { - @JsonSubTypes.Type(name = "newestSegmentFirst", value = NewestSegmentFirstPolicy.class), - @JsonSubTypes.Type(name = "smallestSegmentFirst", value = SmallestSegmentFirstPolicy.class) + @JsonSubTypes.Type(name = "newestSegmentFirst", value = NewestSegmentFirstPolicy.class) }) -public interface CompactionSegmentSearchPolicy +public interface CompactionCandidateSearchPolicy extends Comparator { /** - * Creates an iterator that returns compactible segments. + * Compares between two compaction candidates. Used to determine the + * order in which segments and intervals should be picked for compaction. */ - CompactionSegmentIterator createIterator( - Map compactionConfigs, - Map dataSources, - Map> skipIntervals, - CompactionStatusTracker statusTracker - ); + @Override + int compare(CompactionCandidate o1, CompactionCandidate o2); /** - * Checks if the given candidate segments are eligible for compaction in - * the current iteration. A policy implementation may implement this method - * to avoid compacting intervals that do not fulfil some required criteria. + * Checks if the given {@link CompactionCandidate} 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. */ boolean isEligibleForCompaction( - SegmentsToCompact candidateSegments, + CompactionCandidate candidate, CompactionStatus currentCompactionStatus, CompactionTaskStatus latestTaskStatus ); 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 563dab5278c8..6e8ec6b4b116 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 @@ -94,9 +94,9 @@ public CompactionSimulateResult simulateRunWithConfig( { @Override public CompactionStatus computeCompactionStatus( - SegmentsToCompact candidate, + CompactionCandidate candidate, DataSourceCompactionConfig config, - CompactionSegmentSearchPolicy searchPolicy + CompactionCandidateSearchPolicy searchPolicy ) { return statusTracker.computeCompactionStatus(candidate, config, searchPolicy); @@ -104,7 +104,7 @@ public CompactionStatus computeCompactionStatus( @Override public void onCompactionStatusComputed( - SegmentsToCompact candidateSegments, + CompactionCandidate candidateSegments, DataSourceCompactionConfig config ) { @@ -125,7 +125,7 @@ public void onCompactionStatusComputed( } @Override - public void onTaskSubmitted(ClientCompactionTaskQuery taskPayload, SegmentsToCompact candidateSegments) + public void onTaskSubmitted(ClientCompactionTaskQuery taskPayload, CompactionCandidate candidateSegments) { // Add a row for each task in order of submission final CompactionStatus status = candidateSegments.getCurrentStatus(); @@ -165,7 +165,7 @@ public void onTaskSubmitted(ClientCompactionTaskQuery taskPayload, SegmentsToCom } private Object[] createRow( - SegmentsToCompact candidate, + CompactionCandidate candidate, ClientCompactionTaskQueryTuningConfig tuningConfig, String reason ) @@ -173,7 +173,7 @@ private Object[] createRow( final List row = new ArrayList<>(); row.add(candidate.getDataSource()); row.add(candidate.getUmbrellaInterval()); - row.add(candidate.size()); + row.add(candidate.numSegments()); row.add(candidate.getTotalBytes()); if (tuningConfig != null) { row.add(CompactSegments.findMaxNumTaskSlotsUsedByOneNativeCompactionTask(tuningConfig)); 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 30b5390bff63..295aa2881adf 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,19 +25,19 @@ /** * 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 * compacted again. None of these segments are returned by {@link #next()}. */ - List getCompactedSegments(); + List getCompactedSegments(); /** * List of candidate segments that have been skipped for compaction as they * 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/CompactionStatus.java b/server/src/main/java/org/apache/druid/server/compaction/CompactionStatus.java index 186f03a5136e..592331a0d4ea 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 @@ -20,7 +20,6 @@ package org.apache.druid.server.compaction; import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.base.Preconditions; import org.apache.commons.lang.ArrayUtils; import org.apache.druid.client.indexing.ClientCompactionTaskGranularitySpec; import org.apache.druid.client.indexing.ClientCompactionTaskQueryTuningConfig; @@ -202,7 +201,7 @@ static CompactionStatus running(String reasonForCompaction) * further checks are not performed and the incomplete status is returned. */ static CompactionStatus compute( - SegmentsToCompact candidateSegments, + CompactionCandidate candidateSegments, DataSourceCompactionConfig config, ObjectMapper objectMapper ) @@ -236,23 +235,21 @@ private static class Evaluator { private final ObjectMapper objectMapper; private final DataSourceCompactionConfig compactionConfig; - private final SegmentsToCompact candidateSegments; + private final CompactionCandidate candidateSegments; private final CompactionState lastCompactionState; private final ClientCompactionTaskQueryTuningConfig tuningConfig; private final ClientCompactionTaskGranularitySpec existingGranularitySpec; private final UserCompactionTaskGranularityConfig configuredGranularitySpec; private Evaluator( - SegmentsToCompact candidateSegments, + CompactionCandidate candidateSegments, DataSourceCompactionConfig compactionConfig, ObjectMapper objectMapper ) { - Preconditions.checkArgument(!candidateSegments.isEmpty(), "Empty candidates"); - this.candidateSegments = candidateSegments; this.objectMapper = objectMapper; - this.lastCompactionState = candidateSegments.getFirst().getLastCompactionState(); + this.lastCompactionState = candidateSegments.getSegments().get(0).getLastCompactionState(); this.compactionConfig = compactionConfig; this.tuningConfig = ClientCompactionTaskQueryTuningConfig.from(compactionConfig); this.configuredGranularitySpec = compactionConfig.getGranularitySpec(); 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 644d89a4db86..ab7ddbbb91a7 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 @@ -47,7 +47,7 @@ public class CompactionStatusTracker private final ObjectMapper objectMapper; private final ConcurrentHashMap datasourceStatuses = new ConcurrentHashMap<>(); - private final ConcurrentHashMap submittedTaskIdToSegments + private final ConcurrentHashMap submittedTaskIdToSegments = new ConcurrentHashMap<>(); @Inject @@ -66,7 +66,7 @@ public void removeDatasource(String datasource) datasourceStatuses.remove(datasource); } - public CompactionTaskStatus getLatestTaskStatus(SegmentsToCompact candidates) + public CompactionTaskStatus getLatestTaskStatus(CompactionCandidate candidates) { return datasourceStatuses .getOrDefault(candidates.getDataSource(), DatasourceStatus.EMPTY) @@ -84,9 +84,9 @@ public Set getSubmittedTaskIds() } public CompactionStatus computeCompactionStatus( - SegmentsToCompact candidate, + CompactionCandidate candidate, DataSourceCompactionConfig config, - CompactionSegmentSearchPolicy searchPolicy + CompactionCandidateSearchPolicy searchPolicy ) { final CompactionStatus compactionStatus = CompactionStatus.compute(candidate, config, objectMapper); @@ -118,7 +118,7 @@ public CompactionStatus computeCompactionStatus( } public void onCompactionStatusComputed( - SegmentsToCompact candidateSegments, + CompactionCandidate candidateSegments, DataSourceCompactionConfig config ) { @@ -148,7 +148,7 @@ public void onCompactionConfigUpdated(DruidCompactionConfig compactionConfig) public void onTaskSubmitted( ClientCompactionTaskQuery taskPayload, - SegmentsToCompact candidateSegments + CompactionCandidate candidateSegments ) { submittedTaskIdToSegments.put(taskPayload.getId(), candidateSegments); @@ -162,7 +162,7 @@ public void onTaskFinished(String taskId, TaskStatus taskStatus) return; } - final SegmentsToCompact candidateSegments = submittedTaskIdToSegments.remove(taskId); + final CompactionCandidate candidateSegments = submittedTaskIdToSegments.remove(taskId); if (candidateSegments == null) { // Nothing to do since we don't know the corresponding datasource or interval return; @@ -209,7 +209,7 @@ void handleCompletedTask(Interval compactionInterval, TaskStatus taskStatus) intervalToTaskStatus.put(compactionInterval, updatedStatus); } - void handleSubmittedTask(SegmentsToCompact candidateSegments) + void handleSubmittedTask(CompactionCandidate candidateSegments) { final Interval interval = candidateSegments.getUmbrellaInterval(); final CompactionTaskStatus lastStatus = intervalToTaskStatus.get(interval); 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 d5cb5d80fe6a..0abbd00e9755 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 @@ -66,10 +66,10 @@ public class DataSourceCompactibleSegmentIterator implements CompactionSegmentIt private final String dataSource; private final DataSourceCompactionConfig config; private final CompactionStatusTracker statusTracker; - private final PriorityBasedSegmentSearchPolicy searchPolicy; + private final CompactionCandidateSearchPolicy searchPolicy; - private final List compactedSegments = new ArrayList<>(); - private final List skippedSegments = new ArrayList<>(); + private final List compactedSegments = 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 @@ -77,13 +77,13 @@ 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 PriorityQueue queue; public DataSourceCompactibleSegmentIterator( DataSourceCompactionConfig config, SegmentTimeline timeline, List skipIntervals, - PriorityBasedSegmentSearchPolicy searchPolicy, + CompactionCandidateSearchPolicy searchPolicy, CompactionStatusTracker statusTracker ) { @@ -91,7 +91,7 @@ public DataSourceCompactibleSegmentIterator( this.config = config; this.dataSource = config.getDataSource(); this.searchPolicy = searchPolicy; - this.queue = new PriorityQueue<>(searchPolicy.getSegmentComparator()); + this.queue = new PriorityQueue<>(searchPolicy); populateQueue(timeline, skipIntervals); } @@ -117,7 +117,7 @@ private void populateQueue(SegmentTimeline timeline, List skipInterval } } if (!partialEternitySegments.isEmpty()) { - SegmentsToCompact candidatesWithStatus = SegmentsToCompact.from(partialEternitySegments).withCurrentStatus( + CompactionCandidate candidatesWithStatus = CompactionCandidate.from(partialEternitySegments).withCurrentStatus( CompactionStatus.skipped("Segments have partial-eternity intervals") ); skippedSegments.add(candidatesWithStatus); @@ -181,13 +181,13 @@ private void populateQueue(SegmentTimeline timeline, List skipInterval } @Override - public List getCompactedSegments() + public List getCompactedSegments() { return compactedSegments; } @Override - public List getSkippedSegments() + public List getSkippedSegments() { return skippedSegments; } @@ -199,18 +199,13 @@ public boolean hasNext() } @Override - public SegmentsToCompact next() + public CompactionCandidate next() { - if (!hasNext()) { - throw new NoSuchElementException(); - } - - final SegmentsToCompact entry = queue.poll(); - if (entry == null || entry.isEmpty()) { + if (hasNext()) { + return queue.poll(); + } else { throw new NoSuchElementException(); } - - return entry; } /** @@ -320,10 +315,10 @@ private void findAndEnqueueSegmentsToCompact(CompactibleSegmentIterator compacti continue; } - final SegmentsToCompact candidates = SegmentsToCompact.from(segments); + final CompactionCandidate candidates = CompactionCandidate.from(segments); final CompactionStatus compactionStatus = statusTracker.computeCompactionStatus(candidates, config, searchPolicy); - final SegmentsToCompact candidatesWithStatus = candidates.withCurrentStatus(compactionStatus); + final CompactionCandidate candidatesWithStatus = candidates.withCurrentStatus(compactionStatus); statusTracker.onCompactionStatusComputed(candidatesWithStatus, config); if (compactionStatus.isComplete()) { @@ -365,7 +360,7 @@ private List findInitialSearchInterval( timeline.findNonOvershadowedObjectsInInterval(skipInterval, Partitions.ONLY_COMPLETE) ); if (!CollectionUtils.isNullOrEmpty(segments)) { - final SegmentsToCompact candidates = SegmentsToCompact.from(segments); + final CompactionCandidate candidates = CompactionCandidate.from(segments); final CompactionStatus reason; if (candidates.getUmbrellaInterval().overlaps(latestSkipInterval)) { @@ -374,7 +369,7 @@ private List findInitialSearchInterval( reason = CompactionStatus.skipped("interval locked by another task"); } - final SegmentsToCompact candidatesWithStatus = candidates.withCurrentStatus(reason); + final CompactionCandidate candidatesWithStatus = candidates.withCurrentStatus(reason); skippedSegments.add(candidatesWithStatus); statusTracker.onCompactionStatusComputed(candidatesWithStatus, config); } diff --git a/server/src/main/java/org/apache/druid/server/compaction/NewestSegmentFirstPolicy.java b/server/src/main/java/org/apache/druid/server/compaction/NewestSegmentFirstPolicy.java index 32b41dcde924..7c440900ba74 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 @@ -27,10 +27,10 @@ import java.util.Comparator; /** - * Implementation of {@link CompactionSegmentSearchPolicy} that prioritizes + * Implementation of {@link CompactionCandidateSearchPolicy} that prioritizes * intervals which have the latest data. */ -public class NewestSegmentFirstPolicy extends PriorityBasedSegmentSearchPolicy +public class NewestSegmentFirstPolicy extends BaseCandidateSearchPolicy { @JsonCreator public NewestSegmentFirstPolicy( @@ -41,7 +41,7 @@ public NewestSegmentFirstPolicy( } @Override - protected Comparator getSegmentComparator() + protected Comparator getSegmentComparator() { return (o1, o2) -> Comparators.intervalsByStartThenEnd() .compare(o2.getUmbrellaInterval(), o1.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 999b0938e165..33a480e19724 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 @@ -21,14 +21,12 @@ import com.google.common.collect.Maps; import org.apache.druid.error.DruidException; -import org.apache.druid.java.util.common.guava.Comparators; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; import org.apache.druid.timeline.SegmentTimeline; import org.joda.time.Interval; import java.util.Collections; -import java.util.Comparator; import java.util.List; import java.util.Map; import java.util.NoSuchElementException; @@ -36,35 +34,25 @@ import java.util.stream.Collectors; /** - * Implementation of {@link CompactionSegmentIterator} that returns segments in - * order of their priority. + * Implementation of {@link CompactionSegmentIterator} that returns candidate + * segments in order of their priority. */ public class PriorityBasedCompactionSegmentIterator implements CompactionSegmentIterator { private static final Logger log = new Logger(PriorityBasedCompactionSegmentIterator.class); - private final PriorityQueue queue; + private final PriorityQueue queue; private final Map datasourceIterators; public PriorityBasedCompactionSegmentIterator( Map compactionConfigs, Map datasourceToTimeline, Map> skipIntervals, - PriorityBasedSegmentSearchPolicy searchPolicy, + CompactionCandidateSearchPolicy searchPolicy, CompactionStatusTracker statusTracker ) { - final Comparator comparator; - final String priorityDatasource = searchPolicy.getPriorityDatasource(); - if (priorityDatasource == null || priorityDatasource.isEmpty()) { - comparator = searchPolicy.getSegmentComparator(); - } else { - comparator = Comparators.alwaysFirst(priorityDatasource) - .onResultOf(SegmentsToCompact::getDataSource) - .thenComparing(searchPolicy.getSegmentComparator()); - } - this.queue = new PriorityQueue<>(comparator); - + this.queue = new PriorityQueue<>(searchPolicy); this.datasourceIterators = Maps.newHashMapWithExpectedSize(datasourceToTimeline.size()); compactionConfigs.forEach((datasource, config) -> { if (config == null) { @@ -91,7 +79,7 @@ public PriorityBasedCompactionSegmentIterator( } @Override - public List getCompactedSegments() + public List getCompactedSegments() { return datasourceIterators.values().stream().flatMap( iterator -> iterator.getCompactedSegments().stream() @@ -99,7 +87,7 @@ public List getCompactedSegments() } @Override - public List getSkippedSegments() + public List getSkippedSegments() { return datasourceIterators.values().stream().flatMap( iterator -> iterator.getSkippedSegments().stream() @@ -113,14 +101,14 @@ public boolean hasNext() } @Override - public SegmentsToCompact next() + public CompactionCandidate next() { if (!hasNext()) { throw new NoSuchElementException(); } - final SegmentsToCompact entry = queue.poll(); - if (entry == null || entry.isEmpty()) { + final CompactionCandidate entry = queue.poll(); + if (entry == null) { throw new NoSuchElementException(); } @@ -132,9 +120,9 @@ private void addNextItemForDatasourceToQueue(String dataSourceName) { final DataSourceCompactibleSegmentIterator iterator = datasourceIterators.get(dataSourceName); if (iterator.hasNext()) { - final SegmentsToCompact segmentsToCompact = iterator.next(); - if (!segmentsToCompact.isEmpty()) { - queue.add(segmentsToCompact); + final CompactionCandidate compactionCandidate = iterator.next(); + if (compactionCandidate != null) { + queue.add(compactionCandidate); } } } diff --git a/server/src/main/java/org/apache/druid/server/compaction/SmallestSegmentFirstPolicy.java b/server/src/main/java/org/apache/druid/server/compaction/SmallestSegmentFirstPolicy.java deleted file mode 100644 index 73931ccddc8e..000000000000 --- a/server/src/main/java/org/apache/druid/server/compaction/SmallestSegmentFirstPolicy.java +++ /dev/null @@ -1,49 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.druid.server.compaction; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; -import com.google.common.collect.Ordering; - -import javax.annotation.Nullable; -import java.util.Comparator; - -/** - * Implementation of {@link CompactionSegmentSearchPolicy} that prioritizes - * intervals which have the smallest average size of segments. - */ -public class SmallestSegmentFirstPolicy extends PriorityBasedSegmentSearchPolicy -{ - @JsonCreator - public SmallestSegmentFirstPolicy( - @JsonProperty("priorityDatasource") @Nullable String priorityDatasource - ) - { - super(priorityDatasource); - } - - @Override - protected Comparator getSegmentComparator() - { - return Ordering.natural() - .onResultOf(entry -> entry.getTotalBytes() / entry.size()); - } -} diff --git a/server/src/main/java/org/apache/druid/server/coordinator/AutoCompactionSnapshot.java b/server/src/main/java/org/apache/druid/server/coordinator/AutoCompactionSnapshot.java index d45db24475d2..d6fa4835b48e 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/AutoCompactionSnapshot.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/AutoCompactionSnapshot.java @@ -190,24 +190,6 @@ public int hashCode() ); } - @Override - public String toString() - { - return "AutoCompactionSnapshot{" + - "dataSource='" + dataSource + '\'' + - ", scheduleStatus=" + scheduleStatus + - ", bytesAwaitingCompaction=" + bytesAwaitingCompaction + - ", bytesCompacted=" + bytesCompacted + - ", bytesSkipped=" + bytesSkipped + - ", segmentCountAwaitingCompaction=" + segmentCountAwaitingCompaction + - ", segmentCountCompacted=" + segmentCountCompacted + - ", segmentCountSkipped=" + segmentCountSkipped + - ", intervalCountAwaitingCompaction=" + intervalCountAwaitingCompaction + - ", intervalCountCompacted=" + intervalCountCompacted + - ", intervalCountSkipped=" + intervalCountSkipped + - '}'; - } - public static class Builder { private final String dataSource; diff --git a/server/src/main/java/org/apache/druid/server/coordinator/ClusterCompactionConfig.java b/server/src/main/java/org/apache/druid/server/coordinator/ClusterCompactionConfig.java index 36c54214e209..e2b98a32a92c 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/ClusterCompactionConfig.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/ClusterCompactionConfig.java @@ -22,7 +22,7 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import org.apache.druid.indexer.CompactionEngine; -import org.apache.druid.server.compaction.CompactionSegmentSearchPolicy; +import org.apache.druid.server.compaction.CompactionCandidateSearchPolicy; import javax.annotation.Nullable; import java.util.Objects; @@ -38,7 +38,7 @@ public class ClusterCompactionConfig private final Integer maxCompactionTaskSlots; private final Boolean useAutoScaleSlots; private final CompactionEngine engine; - private final CompactionSegmentSearchPolicy compactionPolicy; + private final CompactionCandidateSearchPolicy compactionPolicy; @JsonCreator public ClusterCompactionConfig( @@ -46,7 +46,7 @@ public ClusterCompactionConfig( @JsonProperty("maxCompactionTaskSlots") @Nullable Integer maxCompactionTaskSlots, @JsonProperty("useAutoScaleSlots") @Nullable Boolean useAutoScaleSlots, @JsonProperty("engine") @Nullable CompactionEngine engine, - @JsonProperty("compactionPolicy") @Nullable CompactionSegmentSearchPolicy compactionPolicy + @JsonProperty("compactionPolicy") @Nullable CompactionCandidateSearchPolicy compactionPolicy ) { this.compactionTaskSlotRatio = compactionTaskSlotRatio; @@ -86,7 +86,7 @@ public CompactionEngine getEngine() @Nullable @JsonProperty - public CompactionSegmentSearchPolicy getCompactionPolicy() + public CompactionCandidateSearchPolicy getCompactionPolicy() { return compactionPolicy; } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/DruidCompactionConfig.java b/server/src/main/java/org/apache/druid/server/coordinator/DruidCompactionConfig.java index 7e69ed2ab6dc..b35ba7d29389 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/DruidCompactionConfig.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/DruidCompactionConfig.java @@ -24,7 +24,7 @@ import com.google.common.base.Optional; import org.apache.druid.common.config.Configs; import org.apache.druid.indexer.CompactionEngine; -import org.apache.druid.server.compaction.CompactionSegmentSearchPolicy; +import org.apache.druid.server.compaction.CompactionCandidateSearchPolicy; import org.apache.druid.server.compaction.NewestSegmentFirstPolicy; import javax.annotation.Nullable; @@ -40,7 +40,7 @@ public class DruidCompactionConfig { public static final String CONFIG_KEY = "coordinator.compaction.config"; - private static final CompactionSegmentSearchPolicy DEFAULT_COMPACTION_POLICY + private static final CompactionCandidateSearchPolicy DEFAULT_COMPACTION_POLICY = new NewestSegmentFirstPolicy(null); private static final DruidCompactionConfig EMPTY_INSTANCE = new DruidCompactionConfig(Collections.emptyList(), null, null, null, null, null); @@ -50,7 +50,7 @@ public class DruidCompactionConfig private final int maxCompactionTaskSlots; private final boolean useAutoScaleSlots; private final CompactionEngine engine; - private final CompactionSegmentSearchPolicy compactionPolicy; + private final CompactionCandidateSearchPolicy compactionPolicy; public DruidCompactionConfig withDatasourceConfigs( List compactionConfigs @@ -99,7 +99,7 @@ public DruidCompactionConfig( @JsonProperty("maxCompactionTaskSlots") @Nullable Integer maxCompactionTaskSlots, @JsonProperty("useAutoScaleSlots") @Nullable Boolean useAutoScaleSlots, @JsonProperty("engine") @Nullable CompactionEngine compactionEngine, - @JsonProperty("compactionPolicy") @Nullable CompactionSegmentSearchPolicy compactionPolicy + @JsonProperty("compactionPolicy") @Nullable CompactionCandidateSearchPolicy compactionPolicy ) { this.compactionConfigs = Configs.valueOrDefault(compactionConfigs, Collections.emptyList()); @@ -171,7 +171,7 @@ public Optional findConfigForDatasource(String dataS } @JsonProperty - public CompactionSegmentSearchPolicy getCompactionPolicy() + public CompactionCandidateSearchPolicy getCompactionPolicy() { return compactionPolicy; } 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 de7529f8d659..98edab7b2f6d 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 @@ -48,9 +48,10 @@ import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.rpc.indexing.OverlordClient; import org.apache.druid.server.compaction.CompactionSegmentIterator; -import org.apache.druid.server.compaction.CompactionSegmentSearchPolicy; +import org.apache.druid.server.compaction.CompactionCandidateSearchPolicy; import org.apache.druid.server.compaction.CompactionStatusTracker; -import org.apache.druid.server.compaction.SegmentsToCompact; +import org.apache.druid.server.compaction.PriorityBasedCompactionSegmentIterator; +import org.apache.druid.server.compaction.CompactionCandidate; import org.apache.druid.server.coordinator.AutoCompactionSnapshot; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; import org.apache.druid.server.coordinator.DruidCompactionConfig; @@ -214,9 +215,14 @@ public void run( ); // Get iterator over segments to compact and submit compaction tasks - final CompactionSegmentSearchPolicy policy = dynamicConfig.getCompactionPolicy(); - final CompactionSegmentIterator iterator = - policy.createIterator(compactionConfigs, dataSources, intervalsToSkipCompaction, statusTracker); + final CompactionCandidateSearchPolicy policy = dynamicConfig.getCompactionPolicy(); + final CompactionSegmentIterator iterator = new PriorityBasedCompactionSegmentIterator( + compactionConfigs, + dataSources, + intervalsToSkipCompaction, + policy, + statusTracker + ); final int compactionTaskCapacity = getCompactionTaskCapacity(dynamicConfig); final int availableCompactionTaskSlots @@ -433,11 +439,7 @@ private int submitCompactionTasks( int totalTaskSlotsAssigned = 0; while (iterator.hasNext() && totalTaskSlotsAssigned < numAvailableCompactionTaskSlots) { - final SegmentsToCompact entry = iterator.next(); - if (entry.isEmpty()) { - throw new ISE("segmentsToCompact is empty?"); - } - + final CompactionCandidate entry = iterator.next(); final String dataSourceName = entry.getDataSource(); // As these segments will be compacted, we will aggregate the statistic to the Compacted statistics @@ -592,13 +594,10 @@ private void updateCompactionSnapshotStats( { // Mark all the segments remaining in the iterator as "awaiting compaction" while (iterator.hasNext()) { - final SegmentsToCompact entry = iterator.next(); - if (!entry.isEmpty()) { - final String dataSourceName = entry.getDataSource(); - currentRunAutoCompactionSnapshotBuilders - .computeIfAbsent(dataSourceName, AutoCompactionSnapshot::builder) - .incrementWaitingStats(entry.getStats()); - } + final CompactionCandidate entry = iterator.next(); + currentRunAutoCompactionSnapshotBuilders + .computeIfAbsent(entry.getDataSource(), AutoCompactionSnapshot::builder) + .incrementWaitingStats(entry.getStats()); } // Statistics of all segments considered compacted after this run @@ -656,7 +655,7 @@ public Map getAutoCompactionSnapshot() } private String compactSegments( - SegmentsToCompact entry, + CompactionCandidate entry, int compactionTaskPriority, ClientCompactionTaskQueryTuningConfig tuningConfig, ClientCompactionTaskGranularitySpec granularitySpec, 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 ac7bf28a4b08..c4177f186c72 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 @@ -304,7 +304,7 @@ public void testStatusWhenLastCompactionStateSameAsRequired() final DataSegment segment = DataSegment.builder(WIKI_SEGMENT).lastCompactionState(lastCompactionState).build(); final CompactionStatus status = CompactionStatus.compute( - SegmentsToCompact.from(Collections.singletonList(segment)), + CompactionCandidate.from(Collections.singletonList(segment)), compactionConfig, OBJECT_MAPPER ); @@ -322,7 +322,7 @@ private void verifyCompactionStatusIsPendingBecause( .lastCompactionState(lastCompactionState) .build(); final CompactionStatus status = CompactionStatus.compute( - SegmentsToCompact.from(Collections.singletonList(segment)), + CompactionCandidate.from(Collections.singletonList(segment)), compactionConfig, OBJECT_MAPPER ); 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 9677b4890dcf..f64c0865d6db 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 @@ -50,8 +50,8 @@ public void setup() @Test public void testGetLatestTaskStatusForSubmittedTask() { - final SegmentsToCompact candidateSegments - = SegmentsToCompact.from(Collections.singletonList(WIKI_SEGMENT)); + final CompactionCandidate candidateSegments + = CompactionCandidate.from(Collections.singletonList(WIKI_SEGMENT)); statusTracker.onTaskSubmitted(createCompactionTask("task1"), candidateSegments); CompactionTaskStatus status = statusTracker.getLatestTaskStatus(candidateSegments); @@ -61,8 +61,8 @@ public void testGetLatestTaskStatusForSubmittedTask() @Test public void testGetLatestTaskStatusForSuccessfulTask() { - final SegmentsToCompact candidateSegments - = SegmentsToCompact.from(Collections.singletonList(WIKI_SEGMENT)); + final CompactionCandidate candidateSegments + = CompactionCandidate.from(Collections.singletonList(WIKI_SEGMENT)); statusTracker.onTaskSubmitted(createCompactionTask("task1"), candidateSegments); statusTracker.onTaskFinished("task1", TaskStatus.success("task1")); @@ -73,8 +73,8 @@ public void testGetLatestTaskStatusForSuccessfulTask() @Test public void testGetLatestTaskStatusForFailedTask() { - final SegmentsToCompact candidateSegments - = SegmentsToCompact.from(Collections.singletonList(WIKI_SEGMENT)); + final CompactionCandidate candidateSegments + = CompactionCandidate.from(Collections.singletonList(WIKI_SEGMENT)); statusTracker.onTaskSubmitted(createCompactionTask("task1"), candidateSegments); statusTracker.onTaskFinished("task1", TaskStatus.failure("task1", "some failure")); @@ -86,8 +86,8 @@ public void testGetLatestTaskStatusForFailedTask() @Test public void testGetLatestTaskStatusForRepeatedlyFailingTask() { - final SegmentsToCompact candidateSegments - = SegmentsToCompact.from(Collections.singletonList(WIKI_SEGMENT)); + final CompactionCandidate candidateSegments + = CompactionCandidate.from(Collections.singletonList(WIKI_SEGMENT)); statusTracker.onTaskSubmitted(createCompactionTask("task1"), candidateSegments); statusTracker.onTaskFinished("task1", TaskStatus.failure("task1", "some failure")); 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 c4d694b8a935..12ad7d3dd92f 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 @@ -265,7 +265,7 @@ public void testManySegmentsPerShard() public void testSkipDataSourceWithNoSegments() { final Period segmentPeriod = Period.hours(1); - final CompactionSegmentIterator iterator = policy.createIterator( + final CompactionSegmentIterator iterator = new PriorityBasedCompactionSegmentIterator( ImmutableMap.of( TestDataSource.KOALA, configBuilder().forDataSource(TestDataSource.KOALA).build(), @@ -284,6 +284,7 @@ public void testSkipDataSourceWithNoSegments() ) ), Collections.emptyMap(), + policy, statusTracker ); @@ -338,7 +339,7 @@ public void testClearSegmentsToCompactWhenSkippingSegments() expectedSegmentsToCompact2.sort(Comparator.naturalOrder()); Set> observedSegments = Streams.sequentialStreamFrom(iterator) - .map(SegmentsToCompact::getSegments) + .map(CompactionCandidate::getSegments) .collect(Collectors.toSet()); Assert.assertEquals( ImmutableSet.of(expectedSegmentsToCompact, expectedSegmentsToCompact2), @@ -492,7 +493,7 @@ public void testIfSegmentsSkipOffsetWithConfiguredSegmentGranularitySmaller() public void testWithSkipIntervals() { final Period segmentPeriod = Period.hours(1); - final CompactionSegmentIterator iterator = policy.createIterator( + final CompactionSegmentIterator iterator = new PriorityBasedCompactionSegmentIterator( ImmutableMap.of(TestDataSource.WIKI, configBuilder().withSkipOffsetFromLatest(Period.days(1)).build()), ImmutableMap.of( TestDataSource.WIKI, @@ -513,6 +514,7 @@ public void testWithSkipIntervals() Intervals.of("2017-11-13T00:00:00/2017-11-14T01:00:00") ) ), + policy, statusTracker ); @@ -537,7 +539,7 @@ public void testWithSkipIntervals() public void testHoleInSearchInterval() { final Period segmentPeriod = Period.hours(1); - final CompactionSegmentIterator iterator = policy.createIterator( + final CompactionSegmentIterator iterator = new PriorityBasedCompactionSegmentIterator( ImmutableMap.of(TestDataSource.WIKI, configBuilder().withSkipOffsetFromLatest(Period.hours(1)).build()), ImmutableMap.of( TestDataSource.WIKI, @@ -552,6 +554,7 @@ public void testHoleInSearchInterval() Intervals.of("2017-11-16T14:00:00/2017-11-16T20:00:00") ) ), + policy, statusTracker ); @@ -1736,7 +1739,7 @@ public void testPriorityDatasource() // Setup policy and iterator with priorityDatasource = WIKI final NewestSegmentFirstPolicy policy = new NewestSegmentFirstPolicy(TestDataSource.WIKI); - CompactionSegmentIterator iterator = policy.createIterator( + CompactionSegmentIterator iterator = new PriorityBasedCompactionSegmentIterator( ImmutableMap.of( TestDataSource.WIKI, configBuilder().forDataSource(TestDataSource.WIKI).build(), TestDataSource.KOALA, configBuilder().forDataSource(TestDataSource.KOALA).build() @@ -1746,29 +1749,29 @@ TestDataSource.KOALA, configBuilder().forDataSource(TestDataSource.KOALA).build( TestDataSource.KOALA, SegmentTimeline.forSegments(koalaSegments) ), Collections.emptyMap(), + policy, statusTracker ); // Verify that the segments of WIKI are preferred even though they are older Assert.assertTrue(iterator.hasNext()); - SegmentsToCompact next = iterator.next(); - Assert.assertFalse(next.isEmpty()); + CompactionCandidate next = iterator.next(); Assert.assertEquals(TestDataSource.WIKI, next.getDataSource()); Assert.assertEquals(Intervals.of("2012-01-01/P1D"), next.getUmbrellaInterval()); Assert.assertTrue(iterator.hasNext()); next = iterator.next(); - Assert.assertFalse(next.isEmpty()); Assert.assertEquals(TestDataSource.KOALA, next.getDataSource()); Assert.assertEquals(Intervals.of("2013-01-01/P1D"), next.getUmbrellaInterval()); } private CompactionSegmentIterator createIterator(DataSourceCompactionConfig config, SegmentTimeline timeline) { - return policy.createIterator( + return new PriorityBasedCompactionSegmentIterator( Collections.singletonMap(TestDataSource.WIKI, config), Collections.singletonMap(TestDataSource.WIKI, timeline), Collections.emptyMap(), + policy, statusTracker ); } diff --git a/server/src/test/java/org/apache/druid/server/coordinator/simulate/BalancingStrategiesTest.java b/server/src/test/java/org/apache/druid/server/coordinator/simulate/BalancingStrategiesTest.java index 89bff979a0c4..ceaeb777c8c1 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/simulate/BalancingStrategiesTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/simulate/BalancingStrategiesTest.java @@ -20,6 +20,7 @@ package org.apache.druid.server.coordinator.simulate; import org.apache.druid.client.DruidServer; +import org.apache.druid.segment.TestDataSource; import org.apache.druid.timeline.DataSegment; import org.junit.Assert; import org.junit.Test; @@ -65,7 +66,7 @@ public void testNewClusterGetsBalanced() CoordinatorSimulation sim = CoordinatorSimulation.builder() .withBalancer(strategy) - .withRules(DS.WIKI, Load.on(Tier.T1, 1).forever()) + .withRules(TestDataSource.WIKI, Load.on(Tier.T1, 1).forever()) .withServers(historicals) .withSegments(segments) .build(); @@ -99,7 +100,7 @@ public void testClusterGetsBalancedWhenServerIsAdded() CoordinatorSimulation sim = CoordinatorSimulation.builder() .withBalancer(strategy) - .withRules(DS.WIKI, Load.on(Tier.T1, 1).forever()) + .withRules(TestDataSource.WIKI, Load.on(Tier.T1, 1).forever()) .withServers(historicals) .withSegments(segments) .build(); @@ -145,7 +146,7 @@ public void testClusterGetsBalancedWhenServerIsRemoved() CoordinatorSimulation sim = CoordinatorSimulation.builder() .withBalancer(strategy) - .withRules(DS.WIKI, Load.on(Tier.T1, 1).forever()) + .withRules(TestDataSource.WIKI, Load.on(Tier.T1, 1).forever()) .withServers(historicals) .withSegments(segments) .build(); diff --git a/server/src/test/java/org/apache/druid/server/coordinator/simulate/MarkSegmentsAsUnusedTest.java b/server/src/test/java/org/apache/druid/server/coordinator/simulate/MarkSegmentsAsUnusedTest.java index cd9ddcca1ae2..f3c1a3f4fbec 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/simulate/MarkSegmentsAsUnusedTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/simulate/MarkSegmentsAsUnusedTest.java @@ -20,6 +20,7 @@ package org.apache.druid.server.coordinator.simulate; import org.apache.druid.client.DruidServer; +import org.apache.druid.segment.TestDataSource; import org.apache.druid.server.coordinator.CoordinatorDynamicConfig; import org.apache.druid.timeline.DataSegment; import org.junit.Test; @@ -50,7 +51,7 @@ public void testSegmentsOvershadowedByZeroReplicaSegmentsAreMarkedAsUnused() = CoordinatorDynamicConfig.builder().withMarkSegmentAsUnusedDelayMillis(0).build(); final CoordinatorSimulation sim = CoordinatorSimulation.builder() - .withRules(DS.WIKI, Load.on(Tier.T1, 0).forever()) + .withRules(TestDataSource.WIKI, Load.on(Tier.T1, 0).forever()) .withServers(servers) .withSegments(segmentsV0) .withDynamicConfig(dynamicConfig) diff --git a/server/src/test/java/org/apache/druid/server/coordinator/simulate/SegmentBalancingTest.java b/server/src/test/java/org/apache/druid/server/coordinator/simulate/SegmentBalancingTest.java index 141c6ba7a11b..dabf1f2ee81a 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/simulate/SegmentBalancingTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/simulate/SegmentBalancingTest.java @@ -20,6 +20,7 @@ package org.apache.druid.server.coordinator.simulate; import org.apache.druid.client.DruidServer; +import org.apache.druid.segment.TestDataSource; import org.apache.druid.server.coordinator.CoordinatorDynamicConfig; import org.apache.druid.timeline.DataSegment; import org.junit.Assert; @@ -37,7 +38,7 @@ public class SegmentBalancingTest extends CoordinatorSimulationBaseTest private DruidServer historicalT11; private DruidServer historicalT12; - private final String datasource = DS.WIKI; + private final String datasource = TestDataSource.WIKI; private final List segments = Segments.WIKI_10X1D; @Override @@ -245,7 +246,7 @@ public void testMaxSegmentsAreMovedWhenClusterIsSkewed() CoordinatorSimulation.builder() .withSegments(Segments.KOALA_100X100D) .withServers(historicals) - .withRules(DS.KOALA, Load.on(Tier.T1, 1).forever()) + .withRules(TestDataSource.KOALA, Load.on(Tier.T1, 1).forever()) .build(); startSimulation(sim); diff --git a/server/src/test/java/org/apache/druid/server/coordinator/simulate/SegmentLoadingTest.java b/server/src/test/java/org/apache/druid/server/coordinator/simulate/SegmentLoadingTest.java index 4b7965e9596e..7343e439c9e4 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/simulate/SegmentLoadingTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/simulate/SegmentLoadingTest.java @@ -20,6 +20,7 @@ package org.apache.druid.server.coordinator.simulate; import org.apache.druid.client.DruidServer; +import org.apache.druid.segment.TestDataSource; import org.apache.druid.server.coordinator.CoordinatorDynamicConfig; import org.apache.druid.timeline.DataSegment; import org.junit.Assert; @@ -39,7 +40,7 @@ public class SegmentLoadingTest extends CoordinatorSimulationBaseTest private DruidServer historicalT21; private DruidServer historicalT22; - private final String datasource = DS.WIKI; + private final String datasource = TestDataSource.WIKI; private final List segments = Segments.WIKI_10X1D; @Override @@ -409,7 +410,7 @@ public void testBroadcastReplicasAreNotThrottled() // Verify that all the segments are broadcast to all historicals // irrespective of throttle limit - verifyValue(Metric.ASSIGNED_COUNT, filterByDatasource(DS.WIKI), 30L); + verifyValue(Metric.ASSIGNED_COUNT, filterByDatasource(datasource), 30L); verifyNotEmitted(Metric.DROPPED_COUNT); } @@ -517,7 +518,7 @@ public void testSegmentsAreDroppedFromFullServersFirst() .withServers(historicalT11, historicalT12) .withDynamicConfig(withReplicationThrottleLimit(100)) .withRules(datasource, Load.on(Tier.T1, 1).forever()) - .withRules(DS.KOALA, Load.on(Tier.T1, 1).forever()) + .withRules(TestDataSource.KOALA, Load.on(Tier.T1, 1).forever()) .build(); startSimulation(sim); From eceb7679d981ccd733bdd7e9a3d5bab112495b21 Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Sun, 1 Sep 2024 12:46:11 +0530 Subject: [PATCH 25/26] Fix tests --- .../coordinator/NewestSegmentFirstPolicyBenchmark.java | 4 +++- .../druid/server/compaction/CompactionStatus.java | 2 +- .../PriorityBasedCompactionSegmentIterator.java | 2 +- .../druid/server/coordinator/duty/CompactSegments.java | 2 +- .../compaction/NewestSegmentFirstPolicyTest.java | 10 +++++----- 5 files changed, 11 insertions(+), 9 deletions(-) diff --git a/benchmarks/src/test/java/org/apache/druid/server/coordinator/NewestSegmentFirstPolicyBenchmark.java b/benchmarks/src/test/java/org/apache/druid/server/coordinator/NewestSegmentFirstPolicyBenchmark.java index 3f109f2d0d6d..d5af3ff09989 100644 --- a/benchmarks/src/test/java/org/apache/druid/server/coordinator/NewestSegmentFirstPolicyBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/server/coordinator/NewestSegmentFirstPolicyBenchmark.java @@ -28,6 +28,7 @@ import org.apache.druid.server.compaction.CompactionCandidateSearchPolicy; import org.apache.druid.server.compaction.CompactionStatusTracker; import org.apache.druid.server.compaction.NewestSegmentFirstPolicy; +import org.apache.druid.server.compaction.PriorityBasedCompactionSegmentIterator; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.SegmentTimeline; import org.apache.druid.timeline.partition.NumberedShardSpec; @@ -133,7 +134,8 @@ public void setup() @Benchmark public void measureNewestSegmentFirstPolicy(Blackhole blackhole) { - final CompactionSegmentIterator iterator = policy.createIterator( + final CompactionSegmentIterator iterator = new PriorityBasedCompactionSegmentIterator( + policy, compactionConfigs, dataSources, Collections.emptyMap(), 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 592331a0d4ea..9e9a199917ac 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 @@ -46,7 +46,7 @@ import java.util.function.Function; /** - * Represents the status of compaction for a given list of candidate segments. + * Represents the status of compaction for a given {@link CompactionCandidate}. */ public class CompactionStatus { 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 33a480e19724..a64c3b1e6410 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 @@ -45,10 +45,10 @@ public class PriorityBasedCompactionSegmentIterator implements CompactionSegment private final Map datasourceIterators; public PriorityBasedCompactionSegmentIterator( + CompactionCandidateSearchPolicy searchPolicy, Map compactionConfigs, Map datasourceToTimeline, Map> skipIntervals, - CompactionCandidateSearchPolicy searchPolicy, CompactionStatusTracker statusTracker ) { 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 98edab7b2f6d..0b5457e202dd 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 @@ -217,10 +217,10 @@ public void run( // Get iterator over segments to compact and submit compaction tasks final CompactionCandidateSearchPolicy policy = dynamicConfig.getCompactionPolicy(); final CompactionSegmentIterator iterator = new PriorityBasedCompactionSegmentIterator( + policy, compactionConfigs, dataSources, intervalsToSkipCompaction, - policy, statusTracker ); 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 12ad7d3dd92f..7580582685b0 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 @@ -266,6 +266,7 @@ public void testSkipDataSourceWithNoSegments() { final Period segmentPeriod = Period.hours(1); final CompactionSegmentIterator iterator = new PriorityBasedCompactionSegmentIterator( + policy, ImmutableMap.of( TestDataSource.KOALA, configBuilder().forDataSource(TestDataSource.KOALA).build(), @@ -284,7 +285,6 @@ public void testSkipDataSourceWithNoSegments() ) ), Collections.emptyMap(), - policy, statusTracker ); @@ -494,6 +494,7 @@ public void testWithSkipIntervals() { final Period segmentPeriod = Period.hours(1); final CompactionSegmentIterator iterator = new PriorityBasedCompactionSegmentIterator( + policy, ImmutableMap.of(TestDataSource.WIKI, configBuilder().withSkipOffsetFromLatest(Period.days(1)).build()), ImmutableMap.of( TestDataSource.WIKI, @@ -514,7 +515,6 @@ public void testWithSkipIntervals() Intervals.of("2017-11-13T00:00:00/2017-11-14T01:00:00") ) ), - policy, statusTracker ); @@ -540,6 +540,7 @@ public void testHoleInSearchInterval() { final Period segmentPeriod = Period.hours(1); final CompactionSegmentIterator iterator = new PriorityBasedCompactionSegmentIterator( + policy, ImmutableMap.of(TestDataSource.WIKI, configBuilder().withSkipOffsetFromLatest(Period.hours(1)).build()), ImmutableMap.of( TestDataSource.WIKI, @@ -554,7 +555,6 @@ public void testHoleInSearchInterval() Intervals.of("2017-11-16T14:00:00/2017-11-16T20:00:00") ) ), - policy, statusTracker ); @@ -1740,6 +1740,7 @@ public void testPriorityDatasource() // Setup policy and iterator with priorityDatasource = WIKI final NewestSegmentFirstPolicy policy = new NewestSegmentFirstPolicy(TestDataSource.WIKI); CompactionSegmentIterator iterator = new PriorityBasedCompactionSegmentIterator( + policy, ImmutableMap.of( TestDataSource.WIKI, configBuilder().forDataSource(TestDataSource.WIKI).build(), TestDataSource.KOALA, configBuilder().forDataSource(TestDataSource.KOALA).build() @@ -1749,7 +1750,6 @@ TestDataSource.KOALA, configBuilder().forDataSource(TestDataSource.KOALA).build( TestDataSource.KOALA, SegmentTimeline.forSegments(koalaSegments) ), Collections.emptyMap(), - policy, statusTracker ); @@ -1768,10 +1768,10 @@ TestDataSource.KOALA, configBuilder().forDataSource(TestDataSource.KOALA).build( private CompactionSegmentIterator createIterator(DataSourceCompactionConfig config, SegmentTimeline timeline) { return new PriorityBasedCompactionSegmentIterator( + policy, Collections.singletonMap(TestDataSource.WIKI, config), Collections.singletonMap(TestDataSource.WIKI, timeline), Collections.emptyMap(), - policy, statusTracker ); } From 9d04d9b8070ef4599daddab65d1eaa5fe69e3940 Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Sun, 1 Sep 2024 14:05:51 +0530 Subject: [PATCH 26/26] Fix checkstyle --- .../server/coordinator/NewestSegmentFirstPolicyBenchmark.java | 2 +- .../apache/druid/server/coordinator/duty/CompactSegments.java | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/benchmarks/src/test/java/org/apache/druid/server/coordinator/NewestSegmentFirstPolicyBenchmark.java b/benchmarks/src/test/java/org/apache/druid/server/coordinator/NewestSegmentFirstPolicyBenchmark.java index d5af3ff09989..fb938dca4cc2 100644 --- a/benchmarks/src/test/java/org/apache/druid/server/coordinator/NewestSegmentFirstPolicyBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/server/coordinator/NewestSegmentFirstPolicyBenchmark.java @@ -24,8 +24,8 @@ import org.apache.druid.client.DataSourcesSnapshot; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.DateTimes; -import org.apache.druid.server.compaction.CompactionSegmentIterator; import org.apache.druid.server.compaction.CompactionCandidateSearchPolicy; +import org.apache.druid.server.compaction.CompactionSegmentIterator; import org.apache.druid.server.compaction.CompactionStatusTracker; import org.apache.druid.server.compaction.NewestSegmentFirstPolicy; import org.apache.druid.server.compaction.PriorityBasedCompactionSegmentIterator; 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 0b5457e202dd..a2f97f298afc 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 @@ -47,11 +47,11 @@ import org.apache.druid.metadata.LockFilterPolicy; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.rpc.indexing.OverlordClient; -import org.apache.druid.server.compaction.CompactionSegmentIterator; +import org.apache.druid.server.compaction.CompactionCandidate; import org.apache.druid.server.compaction.CompactionCandidateSearchPolicy; +import org.apache.druid.server.compaction.CompactionSegmentIterator; import org.apache.druid.server.compaction.CompactionStatusTracker; import org.apache.druid.server.compaction.PriorityBasedCompactionSegmentIterator; -import org.apache.druid.server.compaction.CompactionCandidate; import org.apache.druid.server.coordinator.AutoCompactionSnapshot; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; import org.apache.druid.server.coordinator.DruidCompactionConfig;