kfaraz commented on code in PR #18028: URL: https://github.com/apache/druid/pull/18028#discussion_r2113104811
########## indexing-service/src/main/java/org/apache/druid/indexing/overlord/duty/UnusedSegmentsKiller.java: ########## @@ -0,0 +1,472 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.overlord.duty; + +import com.google.common.collect.Ordering; +import com.google.inject.Inject; +import org.apache.druid.client.indexing.IndexingService; +import org.apache.druid.common.utils.IdUtils; +import org.apache.druid.discovery.DruidLeaderSelector; +import org.apache.druid.indexer.TaskStatus; +import org.apache.druid.indexing.common.TaskToolbox; +import org.apache.druid.indexing.common.actions.TaskActionClient; +import org.apache.druid.indexing.common.actions.TaskActionClientFactory; +import org.apache.druid.indexing.common.task.IndexTaskUtils; +import org.apache.druid.indexing.common.task.KillUnusedSegmentsTask; +import org.apache.druid.indexing.common.task.TaskMetrics; +import org.apache.druid.indexing.common.task.Tasks; +import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator; +import org.apache.druid.indexing.overlord.TaskLockbox; +import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.java.util.common.Stopwatch; +import org.apache.druid.java.util.common.concurrent.ScheduledExecutorFactory; +import org.apache.druid.java.util.common.guava.Comparators; +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.SegmentsMetadataManagerConfig; +import org.apache.druid.metadata.UnusedSegmentKillerConfig; +import org.apache.druid.query.DruidMetrics; +import org.apache.druid.segment.loading.DataSegmentKiller; +import org.apache.druid.timeline.DataSegment; +import org.joda.time.DateTime; +import org.joda.time.Duration; +import org.joda.time.Interval; + +import javax.annotation.Nullable; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.Future; +import java.util.concurrent.PriorityBlockingQueue; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.atomic.AtomicReference; + +/** + * {@link OverlordDuty} to delete unused segments from metadata store and the + * deep storage. Launches {@link EmbeddedKillTask}s to clean unused segments + * of a single datasource-interval. + * + * @see SegmentsMetadataManagerConfig to enable the cleanup + */ +public class UnusedSegmentsKiller implements OverlordDuty +{ + private static final Logger log = new Logger(UnusedSegmentsKiller.class); + + private static final String TASK_ID_PREFIX = "overlord-issued"; + + /** + * Period after which the queue is reset even if there are existing jobs in queue. + */ + private static final Duration QUEUE_RESET_PERIOD = Duration.standardDays(1); + + /** + * Duration for which a kill task is allowed to run. + */ + private static final Duration MAX_TASK_DURATION = Duration.standardMinutes(10); + + private final ServiceEmitter emitter; + private final TaskLockbox taskLockbox; + private final DruidLeaderSelector leaderSelector; + private final DataSegmentKiller dataSegmentKiller; + + private final UnusedSegmentKillerConfig killConfig; + private final TaskActionClientFactory taskActionClientFactory; + private final IndexerMetadataStorageCoordinator storageCoordinator; + + /** + * Single-threaded executor to process kill jobs. + */ + private final ScheduledExecutorService exec; + private int previousLeaderTerm; + private final AtomicReference<DateTime> lastResetTime = new AtomicReference<>(null); + + private final AtomicReference<TaskInfo> currentTaskInfo = new AtomicReference<>(null); + + /** + * Queue of kill candidates. Use a PriorityBlockingQueue to ensure thread-safety + * since this queue is accessed by both {@link #run()} and {@link #startNextJobInKillQueue}. + */ + private final PriorityBlockingQueue<KillCandidate> killQueue; + + @Inject + public UnusedSegmentsKiller( + SegmentsMetadataManagerConfig config, + TaskActionClientFactory taskActionClientFactory, + IndexerMetadataStorageCoordinator storageCoordinator, + @IndexingService DruidLeaderSelector leaderSelector, + ScheduledExecutorFactory executorFactory, + DataSegmentKiller dataSegmentKiller, + TaskLockbox taskLockbox, + ServiceEmitter emitter + ) + { + this.emitter = emitter; + this.taskLockbox = taskLockbox; + this.leaderSelector = leaderSelector; + this.dataSegmentKiller = dataSegmentKiller; + this.storageCoordinator = storageCoordinator; + this.taskActionClientFactory = taskActionClientFactory; + + this.killConfig = config.getKillUnused(); + + if (isEnabled()) { + this.exec = executorFactory.create(1, "UnusedSegmentsKiller-%s"); + this.killQueue = new PriorityBlockingQueue<>( + 1000, + Ordering.from(Comparators.intervalsByEndThenStart()) + .onResultOf(candidate -> candidate.interval) + ); + } else { + this.exec = null; + this.killQueue = null; + } + } + + @Override + public boolean isEnabled() + { + return killConfig.isEnabled(); + } + + /** + * Ensures that things are moving along and the kill queue is not stuck. + * Updates the state if leadership changes or if the queue needs to be reset. + */ + @Override + public void run() + { + if (!isEnabled()) { + return; + } + + updateStateIfNewLeader(); + if (shouldResetKillQueue()) { + // Clear the killQueue to stop further processing of already queued jobs + killQueue.clear(); Review Comment: This has two benefits: - We clear the queue proactively thus stopping further processing right here. - There is no interleaving between submissions of `startNextJobInKillQueue` and `resetKillQueue` i.e. at any given point, the `exec` either has a single `startNextJobInKillQueue` or a single `resetKillQueue` in its executor queue. This keeps the semantics simple and easy to debug and reason about. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected] --------------------------------------------------------------------- To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
