From de9047b50056c11b18625604b128558e1ad0add7 Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Tue, 22 Oct 2024 20:02:37 +0530 Subject: [PATCH 01/14] Keep TaskLockbox at datasource level for higher concurrency --- .../actions/SegmentAllocationQueue.java | 6 +- .../common/actions/TaskActionToolbox.java | 10 +- .../indexing/common/actions/TaskLocks.java | 12 +- .../indexing/overlord/CriticalAction.java | 8 +- .../indexing/overlord/DruidOverlord.java | 2 +- .../indexing/overlord/GlobalTaskLockbox.java | 401 ++++++++++++++++++ .../druid/indexing/overlord/LockResult.java | 2 +- .../druid/indexing/overlord/TaskLockbox.java | 147 +++---- .../overlord/TaskLockboxSyncResult.java | 9 +- .../indexing/overlord/TaskQueryTool.java | 6 +- .../druid/indexing/overlord/TaskQueue.java | 4 +- .../actions/SegmentAllocateActionTest.java | 4 +- .../common/actions/TaskActionTestKit.java | 8 +- .../common/actions/TaskLocksTest.java | 6 +- .../common/task/IngestionTestBase.java | 8 +- .../OverlordCompactionSchedulerTest.java | 4 +- ...oxTest.java => GlobalTaskLockboxTest.java} | 51 ++- .../indexing/overlord/TaskLifecycleTest.java | 4 +- .../overlord/TaskLockBoxConcurrencyTest.java | 4 +- .../indexing/overlord/TaskLockConfigTest.java | 2 +- .../indexing/overlord/TaskQueueScaleTest.java | 2 +- .../indexing/overlord/TaskQueueTest.java | 2 +- .../overlord/http/OverlordResourceTest.java | 6 +- .../indexing/overlord/http/OverlordTest.java | 6 +- .../SeekableStreamIndexTaskTestBase.java | 6 +- .../org/apache/druid/cli/CliOverlord.java | 4 +- 26 files changed, 548 insertions(+), 176 deletions(-) create mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/overlord/GlobalTaskLockbox.java rename indexing-service/src/test/java/org/apache/druid/indexing/overlord/{TaskLockboxTest.java => GlobalTaskLockboxTest.java} (97%) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocationQueue.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocationQueue.java index 98ab50cff788..6061679ddded 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocationQueue.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocationQueue.java @@ -26,7 +26,7 @@ import org.apache.druid.indexing.common.task.Task; import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator; import org.apache.druid.indexing.overlord.Segments; -import org.apache.druid.indexing.overlord.TaskLockbox; +import org.apache.druid.indexing.overlord.GlobalTaskLockbox; import org.apache.druid.indexing.overlord.config.TaskLockConfig; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.Intervals; @@ -74,7 +74,7 @@ public class SegmentAllocationQueue private final long maxWaitTimeMillis; - private final TaskLockbox taskLockbox; + private final GlobalTaskLockbox taskLockbox; private final IndexerMetadataStorageCoordinator metadataStorage; private final AtomicBoolean isLeader = new AtomicBoolean(false); private final ServiceEmitter emitter; @@ -89,7 +89,7 @@ public class SegmentAllocationQueue @Inject public SegmentAllocationQueue( - TaskLockbox taskLockbox, + GlobalTaskLockbox taskLockbox, TaskLockConfig taskLockConfig, IndexerMetadataStorageCoordinator metadataStorage, ServiceEmitter emitter, diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TaskActionToolbox.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TaskActionToolbox.java index 7d001ecbcb9a..43f9474c8c23 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TaskActionToolbox.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TaskActionToolbox.java @@ -24,7 +24,7 @@ import com.google.inject.Inject; import org.apache.druid.guice.annotations.Json; import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator; -import org.apache.druid.indexing.overlord.TaskLockbox; +import org.apache.druid.indexing.overlord.GlobalTaskLockbox; import org.apache.druid.indexing.overlord.TaskRunner; import org.apache.druid.indexing.overlord.TaskRunnerFactory; import org.apache.druid.indexing.overlord.TaskStorage; @@ -33,7 +33,7 @@ public class TaskActionToolbox { - private final TaskLockbox taskLockbox; + private final GlobalTaskLockbox taskLockbox; private final TaskStorage taskStorage; private final SegmentAllocationQueue segmentAllocationQueue; private final IndexerMetadataStorageCoordinator indexerMetadataStorageCoordinator; @@ -44,7 +44,7 @@ public class TaskActionToolbox @Inject public TaskActionToolbox( - TaskLockbox taskLockbox, + GlobalTaskLockbox taskLockbox, TaskStorage taskStorage, IndexerMetadataStorageCoordinator indexerMetadataStorageCoordinator, SegmentAllocationQueue segmentAllocationQueue, @@ -63,7 +63,7 @@ public TaskActionToolbox( } public TaskActionToolbox( - TaskLockbox taskLockbox, + GlobalTaskLockbox taskLockbox, TaskStorage taskStorage, IndexerMetadataStorageCoordinator indexerMetadataStorageCoordinator, ServiceEmitter emitter, @@ -82,7 +82,7 @@ public TaskActionToolbox( ); } - public TaskLockbox getTaskLockbox() + public GlobalTaskLockbox getTaskLockbox() { return taskLockbox; } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TaskLocks.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TaskLocks.java index aeecba0e289d..b7c5d27f7052 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TaskLocks.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TaskLocks.java @@ -28,7 +28,7 @@ import org.apache.druid.indexing.common.task.AbstractBatchIndexTask; import org.apache.druid.indexing.common.task.Task; import org.apache.druid.indexing.common.task.Tasks; -import org.apache.druid.indexing.overlord.TaskLockbox; +import org.apache.druid.indexing.overlord.GlobalTaskLockbox; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.ISE; import org.apache.druid.metadata.ReplaceTaskLock; @@ -52,7 +52,7 @@ public class TaskLocks { static void checkLockCoversSegments( final Task task, - final TaskLockbox taskLockbox, + final GlobalTaskLockbox taskLockbox, final Collection segments ) { @@ -69,7 +69,7 @@ static void checkLockCoversSegments( @VisibleForTesting static boolean isLockCoversSegments( final Task task, - final TaskLockbox taskLockbox, + final GlobalTaskLockbox taskLockbox, final Collection segments ) { @@ -176,7 +176,7 @@ public static TaskLockType determineLockTypeForAppend( */ public static Map findReplaceLocksCoveringSegments( final String datasource, - final TaskLockbox taskLockbox, + final GlobalTaskLockbox taskLockbox, final Set segments ) { @@ -206,7 +206,7 @@ public static Map findReplaceLocksCoveringSegments public static List findLocksForSegments( final Task task, - final TaskLockbox taskLockbox, + final GlobalTaskLockbox taskLockbox, final Collection segments ) { @@ -245,7 +245,7 @@ public static List findLocksForSegments( return found; } - private static NavigableMap> getTaskLockMap(TaskLockbox taskLockbox, Task task) + private static NavigableMap> getTaskLockMap(GlobalTaskLockbox taskLockbox, Task task) { final List taskLocks = taskLockbox.findLocksForTask(task); final NavigableMap> taskLockMap = new TreeMap<>(); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/CriticalAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/CriticalAction.java index 9a860ea681e0..70aeb5abb918 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/CriticalAction.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/CriticalAction.java @@ -27,13 +27,13 @@ * This class represents a critical action must be done while the task's lock is guaranteed to not be revoked in the * middle of the action. * - * Implementations must not change the lock state by calling {@link TaskLockbox#lock)}, {@link TaskLockbox#tryLock)}, - * or {@link TaskLockbox#unlock(Task, Interval)}. + * Implementations must not change the lock state by calling {@link GlobalTaskLockbox#lock)}, {@link GlobalTaskLockbox#tryLock)}, + * or {@link GlobalTaskLockbox#unlock(Task, Interval)}. * - * Also, implementations should be finished as soon as possible because all methods in {@link TaskLockbox} are blocked + * Also, implementations should be finished as soon as possible because all methods in {@link GlobalTaskLockbox} are blocked * until this action is finished. * - * @see TaskLockbox#doInCriticalSection + * @see GlobalTaskLockbox#doInCriticalSection */ public class CriticalAction { 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..0c70e1208220 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 @@ -76,7 +76,7 @@ public DruidOverlord( final TaskLockConfig taskLockConfig, final TaskQueueConfig taskQueueConfig, final DefaultTaskConfig defaultTaskConfig, - final TaskLockbox taskLockbox, + final GlobalTaskLockbox taskLockbox, final TaskStorage taskStorage, final TaskActionClientFactory taskActionClientFactory, @Self final DruidNode selfNode, diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/GlobalTaskLockbox.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/GlobalTaskLockbox.java new file mode 100644 index 000000000000..13dbd9ca9438 --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/GlobalTaskLockbox.java @@ -0,0 +1,401 @@ +/* + * 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; + +import com.google.common.annotations.VisibleForTesting; +import com.google.inject.Inject; +import org.apache.druid.indexing.common.LockGranularity; +import org.apache.druid.indexing.common.TaskLock; +import org.apache.druid.indexing.common.actions.SegmentAllocateAction; +import org.apache.druid.indexing.common.actions.SegmentAllocateRequest; +import org.apache.druid.indexing.common.actions.SegmentAllocateResult; +import org.apache.druid.indexing.common.task.Task; +import org.apache.druid.java.util.common.Pair; +import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.metadata.LockFilterPolicy; +import org.apache.druid.metadata.ReplaceTaskLock; +import org.joda.time.DateTime; +import org.joda.time.Interval; + +import javax.annotation.Nullable; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.NavigableMap; +import java.util.Optional; +import java.util.Set; +import java.util.SortedMap; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.locks.ReentrantLock; +import java.util.concurrent.locks.ReentrantReadWriteLock; + +/** + * Remembers which activeTasks have locked which intervals or which segments. Tasks are permitted to lock an interval + * or a segment if no other task outside their group has locked an overlapping interval for the same datasource or + * the same segments. Note that TaskLockbox is also responsible for allocating segmentIds when a task requests to lock + * a new segment. Task lock might involve version assignment. + * + * - When a task locks an interval or a new segment, it is assigned a new version string that it can use to publish + * segments. + * - When a task locks a existing segment, it doesn't need to be assigned a new version. + * + * Note that tasks of higher priorities can revoke locks of tasks of lower priorities. + */ +public class GlobalTaskLockbox +{ + private static final Logger log = new Logger(GlobalTaskLockbox.class); + + private final TaskStorage taskStorage; + private final IndexerMetadataStorageCoordinator metadataStorageCoordinator; + private final ReentrantReadWriteLock globalLock = new ReentrantReadWriteLock(true); + private final ConcurrentHashMap datasourceLocks = new ConcurrentHashMap<>(); + + @Inject + public GlobalTaskLockbox( + TaskStorage taskStorage, + IndexerMetadataStorageCoordinator metadataStorageCoordinator + ) + { + this.taskStorage = taskStorage; + this.metadataStorageCoordinator = metadataStorageCoordinator; + } + + private TaskLockbox getDatasourceLockbox(Task task) + { + return getDatasourceLockbox(task.getDataSource()); + } + + private TaskLockbox getDatasourceLockbox(String datasource) + { + return datasourceLocks.computeIfAbsent( + datasource, + ds -> new TaskLockbox(ds, new DatasourceLock(), taskStorage, metadataStorageCoordinator) + ); + } + + private class DatasourceLock extends ReentrantLock + { + @Override + public void lock() + { + globalLock.readLock().lock(); + super.lock(); + } + + @Override + public void lockInterruptibly() throws InterruptedException + { + globalLock.readLock().lock(); + super.lockInterruptibly(); + } + + @Override + public void unlock() + { + super.unlock(); + globalLock.readLock().unlock(); + } + } + + /** + * Wipe out our current in-memory state and resync it from our bundled {@link TaskStorage}. + * + * @return SyncResult which needs to be processed by the caller + */ + public TaskLockboxSyncResult syncFromStorage() + { + globalLock.writeLock().lock(); + + try { + // Load stuff from taskStorage first. If this fails, we don't want to lose all our locks. + final Set storedActiveTasks = new HashSet<>(); + final List> storedLocks = new ArrayList<>(); + for (final Task task : taskStorage.getActiveTasks()) { + storedActiveTasks.add(task); + for (final TaskLock taskLock : taskStorage.getLocks(task.getId())) { + storedLocks.add(Pair.of(task, taskLock)); + } + } + + // Set of task groups in which at least one task failed to re-acquire a lock + final Set tasksToFail = new HashSet<>(); + int taskLockCount = 0; + for (TaskLockbox datasourceLockbox : datasourceLocks.values()) { + TaskLockboxSyncResult result = datasourceLockbox.resetState(storedActiveTasks, storedLocks); + tasksToFail.addAll(result.getTasksToFail()); + taskLockCount += result.getTaskLockCount(); + } + + log.info( + "Synced [%,d] locks for [%,d] activeTasks from storage ([%,d] locks ignored).", + taskLockCount, + storedActiveTasks.size(), + storedLocks.size() - taskLockCount + ); + + return new TaskLockboxSyncResult(tasksToFail, taskLockCount); + } + finally { + globalLock.writeLock().unlock(); + } + } + + /** + * This method is called only in {@link #syncFromStorage()} and verifies the given task and the taskLock have the same + * groupId, dataSource, and priority. + */ + @VisibleForTesting + @Nullable + TaskLockbox.TaskLockPosse verifyAndCreateOrFindLockPosse(Task task, TaskLock taskLock) + { + return getDatasourceLockbox(task).verifyAndCreateOrFindLockPosse(task, taskLock); + } + + /** + * Acquires a lock on behalf of a task. Blocks until the lock is acquired. + * + * @return {@link LockResult} containing a new or an existing lock if succeeded. Otherwise, {@link LockResult} with a + * {@link LockResult#revoked} flag. + * + * @throws InterruptedException if the current thread is interrupted + */ + public LockResult lock(final Task task, final LockRequest request) throws InterruptedException + { + return getDatasourceLockbox(task).lock(task, request); + } + + /** + * Acquires a lock on behalf of a task, waiting up to the specified wait time if necessary. + * + * @return {@link LockResult} containing a new or an existing lock if succeeded. Otherwise, {@link LockResult} with a + * {@link LockResult#revoked} flag. + * + * @throws InterruptedException if the current thread is interrupted + */ + public LockResult lock(final Task task, final LockRequest request, long timeoutMs) throws InterruptedException + { + return getDatasourceLockbox(task).lock(task, request, timeoutMs); + } + + /** + * Attempt to acquire a lock for a task, without removing it from the queue. Can safely be called multiple times on + * the same task until the lock is preempted. + * + * @return {@link LockResult} containing a new or an existing lock if succeeded. Otherwise, {@link LockResult} with a + * {@link LockResult#revoked} flag. + * + * @throws IllegalStateException if the task is not a valid active task + */ + public LockResult tryLock(final Task task, final LockRequest request) + { + return getDatasourceLockbox(task).tryLock(task, request); + } + + /** + * Attempts to allocate segments for the given requests. Each request contains + * a {@link Task} and a {@link SegmentAllocateAction}. This method tries to + * acquire the task locks on the required intervals/segments and then performs + * a batch allocation of segments. It is possible that some requests succeed + * successfully and others failed. In that case, only the failed ones should be + * retried. + * + * @param requests List of allocation requests + * @param dataSource Datasource for which segment is to be allocated. + * @param interval Interval for which segment is to be allocated. + * @param skipSegmentLineageCheck Whether lineage check is to be skipped + * (this is true for streaming ingestion) + * @param lockGranularity Granularity of task lock + * @return List of allocation results in the same order as the requests. + */ + public List allocateSegments( + List requests, + String dataSource, + Interval interval, + boolean skipSegmentLineageCheck, + LockGranularity lockGranularity + ) + { + return getDatasourceLockbox(dataSource).allocateSegments( + requests, + dataSource, + interval, + skipSegmentLineageCheck, + lockGranularity + ); + } + + /** + * Perform the given action with a guarantee that the locks of the task are not revoked in the middle of action. This + * method first checks that all locks for the given task and intervals are valid and perform the right action. + *

+ * The given action should be finished as soon as possible because all other methods in this class are blocked until + * this method is finished. + * + * @param task task performing a critical action + * @param intervals intervals + * @param action action to be performed inside of the critical section + */ + public T doInCriticalSection(Task task, Set intervals, CriticalAction action) throws Exception + { + return getDatasourceLockbox(task).doInCriticalSection(task, intervals, action); + } + + /** + * Mark the lock as revoked. Note that revoked locks are NOT removed. Instead, they are kept in memory + * and {@link #taskStorage} as the normal locks do. This is to check locks are revoked when they are requested to be + * acquired and notify to the callers if revoked. Revoked locks are removed by calling + * {@link #unlock(Task, Interval)}. + * + * @param taskId an id of the task holding the lock + * @param lock lock to be revoked + */ + @VisibleForTesting + public void revokeLock(String taskId, TaskLock lock) + { + getDatasourceLockbox(lock.getDataSource()).revokeLock(taskId, lock); + } + + /** + * Return the currently-active locks for some task. + * + * @param task task for which to locate locks + * @return currently-active locks for the given task + */ + public List findLocksForTask(final Task task) + { + return getDatasourceLockbox(task).findLocksForTask(task); + } + + /** + * Finds the active non-revoked REPLACE locks held by the given task. + */ + public Set findReplaceLocksForTask(Task task) + { + return getDatasourceLockbox(task).findReplaceLocksForTask(task); + } + + /** + * Finds all the active non-revoked REPLACE locks for the given datasource. + */ + public Set getAllReplaceLocksForDatasource(String datasource) + { + return getDatasourceLockbox(datasource).getAllReplaceLocksForDatasource(datasource); + } + + /** + * @param lockFilterPolicies Lock filters for the given datasources + * @return Map from datasource to intervals locked by tasks satisfying the lock filter condititions + */ + public Map> getLockedIntervals(List lockFilterPolicies) + { + final Map> datasourceToFilterPolicies = new HashMap<>(); + for (LockFilterPolicy policy : lockFilterPolicies) { + datasourceToFilterPolicies.computeIfAbsent(policy.getDatasource(), ds -> new ArrayList<>()) + .add(policy); + } + + final Map> datasourceToLockedIntervals = new HashMap<>(); + datasourceToFilterPolicies.forEach( + (datasource, policies) -> datasourceToLockedIntervals.put( + datasource, + getDatasourceLockbox(datasource).getLockedIntervals(policies) + ) + ); + + return datasourceToLockedIntervals; + } + + /** + * @param lockFilterPolicies Lock filters for the given datasources + * @return Map from datasource to list of non-revoked locks with at least as much priority and an overlapping interval + */ + public Map> getActiveLocks(List lockFilterPolicies) + { + final Map> datasourceToFilterPolicies = new HashMap<>(); + for (LockFilterPolicy policy : lockFilterPolicies) { + datasourceToFilterPolicies.computeIfAbsent(policy.getDatasource(), ds -> new ArrayList<>()) + .add(policy); + } + + final Map> datasourceToActiveLocks = new HashMap<>(); + datasourceToFilterPolicies.forEach( + (datasource, policies) -> datasourceToActiveLocks.put( + datasource, + getDatasourceLockbox(datasource).getActiveLocks(policies) + ) + ); + + return datasourceToActiveLocks; + } + + public void unlock(final Task task, final Interval interval) + { + getDatasourceLockbox(task).unlock(task, interval); + } + + public void unlockAll(Task task) + { + getDatasourceLockbox(task).unlockAll(task); + } + + public void add(Task task) + { + getDatasourceLockbox(task).add(task); + } + + /** + * Release all locks for a task and remove task from set of active tasks. Does nothing if the task is not currently locked or not an active task. + * + * @param task task to unlock + */ + public void remove(final Task task) + { + getDatasourceLockbox(task).remove(task); + } + + @VisibleForTesting + Optional getOnlyTaskLockPosseContainingInterval(Task task, Interval interval) + { + return getDatasourceLockbox(task).getOnlyTaskLockPosseContainingInterval(task, interval); + } + + @VisibleForTesting + Set getActiveTasks() + { + final Set allActiveTasks = new HashSet<>(); + datasourceLocks.values().forEach(lockbox -> allActiveTasks.addAll(lockbox.getActiveTasks())); + + return allActiveTasks; + } + + @VisibleForTesting + Map>>> getAllLocks() + { + Map>>> allLocks + = new HashMap<>(); + + datasourceLocks.forEach((datasource, lockbox) -> allLocks.put(datasource, lockbox.getAllLocks())); + + return allLocks; + } + +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/LockResult.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/LockResult.java index fa18cbb87f24..400d66d39a2c 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/LockResult.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/LockResult.java @@ -27,7 +27,7 @@ import javax.annotation.Nullable; /** - * This class represents the result of {@link TaskLockbox#tryLock}. If the lock + * This class represents the result of {@link GlobalTaskLockbox#tryLock}. If the lock * acquisition fails, the callers can tell that it was failed because it was preempted by other locks of higher * priorities or not by checking the {@link #revoked} flag. * diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskLockbox.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskLockbox.java index bebb52157d6f..aee7ce6c36f0 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskLockbox.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskLockbox.java @@ -24,11 +24,9 @@ import com.google.common.base.Objects; import com.google.common.base.Preconditions; import com.google.common.collect.ComparisonChain; -import com.google.common.collect.ImmutableSet; import com.google.common.collect.Lists; import com.google.common.collect.Ordering; import com.google.errorprone.annotations.concurrent.GuardedBy; -import com.google.inject.Inject; import org.apache.druid.error.DruidException; import org.apache.druid.indexing.common.LockGranularity; import org.apache.druid.indexing.common.SegmentLock; @@ -74,31 +72,34 @@ import java.util.stream.Collectors; /** + * Maintains the state of task locks for a single datasource. + *

* Remembers which activeTasks have locked which intervals or which segments. Tasks are permitted to lock an interval * or a segment if no other task outside their group has locked an overlapping interval for the same datasource or * the same segments. Note that TaskLockbox is also responsible for allocating segmentIds when a task requests to lock * a new segment. Task lock might involve version assignment. - * + *

* - When a task locks an interval or a new segment, it is assigned a new version string that it can use to publish * segments. * - When a task locks a existing segment, it doesn't need to be assigned a new version. - * + *

* Note that tasks of higher priorities can revoke locks of tasks of lower priorities. */ public class TaskLockbox { - // Datasource -> startTime -> Interval -> list of (Tasks + TaskLock) + // startTime -> Interval -> list of (Tasks + TaskLock) // Multiple shared locks can be acquired for the same dataSource and interval. // Note that revoked locks are also maintained in this map to notify that those locks are revoked to the callers when // they acquire the same locks again. // Also, the key of the second inner map is the start time to find all intervals properly starting with the same // startTime. - private final Map>>> running = new HashMap<>(); + private final NavigableMap>> running = new TreeMap<>(); + private final String dataSource; private final TaskStorage taskStorage; private final IndexerMetadataStorageCoordinator metadataStorageCoordinator; - private final ReentrantLock giant = new ReentrantLock(true); - private final Condition lockReleaseCondition = giant.newCondition(); + private final ReentrantLock giant; + private final Condition lockReleaseCondition; private static final EmittingLogger log = new EmittingLogger(TaskLockbox.class); @@ -113,38 +114,37 @@ public class TaskLockbox * Used to clean up pending segments for a taskAllocatorId as soon as the set * of corresponding active taskIds becomes empty. */ - @GuardedBy("giant") private final Map> activeAllocatorIdToTaskIds = new HashMap<>(); - @Inject public TaskLockbox( + String dataSource, + ReentrantLock giant, TaskStorage taskStorage, IndexerMetadataStorageCoordinator metadataStorageCoordinator ) { + this.giant = giant; + this.lockReleaseCondition = giant.newCondition(); + this.dataSource = dataSource; this.taskStorage = taskStorage; this.metadataStorageCoordinator = metadataStorageCoordinator; } /** * Wipe out our current in-memory state and resync it from our bundled {@link TaskStorage}. + * This method must be called only by the GlobalTaskLockbox after acquiring + * a WRITE globalLock. * * @return SyncResult which needs to be processed by the caller */ - public TaskLockboxSyncResult syncFromStorage() + TaskLockboxSyncResult resetState( + final Set storedActiveTasks, + final List> storedLocks + ) { - giant.lock(); - + // No locks need to be acquired here since the GlobalTaskLockbox has already + // acquired the required locks try { - // Load stuff from taskStorage first. If this fails, we don't want to lose all our locks. - final Set storedActiveTasks = new HashSet<>(); - final List> storedLocks = new ArrayList<>(); - for (final Task task : taskStorage.getActiveTasks()) { - storedActiveTasks.add(task); - for (final TaskLock taskLock : taskStorage.getLocks(task.getId())) { - storedLocks.add(Pair.of(task, taskLock)); - } - } // Sort locks by version, so we add them back in the order they were acquired. final Ordering> byVersionOrdering = new Ordering>() { @@ -233,27 +233,20 @@ public int compare(Pair left, Pair right) } } - log.info( - "Synced %,d locks for %,d activeTasks from storage (%,d locks ignored).", - taskLockCount, - activeTasks.size(), - storedLocks.size() - taskLockCount - ); - if (!failedToReacquireLockTaskGroups.isEmpty()) { log.warn("Marking all tasks from task groups[%s] to be failed " + "as they failed to reacquire at least one lock.", failedToReacquireLockTaskGroups); } - return new TaskLockboxSyncResult(tasksToFail); - } - finally { - giant.unlock(); + return new TaskLockboxSyncResult(tasksToFail, taskLockCount); + } catch (Throwable t) { + log.noStackTrace().error(t, "Error while resetting state of datasource[%s]", dataSource); + throw t; } } /** - * This method is called only in {@link #syncFromStorage()} and verifies the given task and the taskLock have the same + * This method is called only in {@link #resetState} and verifies the given task and the taskLock have the same * groupId, dataSource, and priority. */ @VisibleForTesting @@ -548,7 +541,6 @@ private TaskLockPosse createOrFindLockPosse(LockRequest request, Task task, bool try { final TaskLockPosse posseToUse; final List foundPosses = findLockPossesOverlapsInterval( - request.getDataSource(), request.getInterval() ); @@ -682,8 +674,7 @@ private TaskLockPosse createNewTaskLockPosse(LockRequest request) giant.lock(); try { final TaskLockPosse posseToUse = new TaskLockPosse(request.toLock()); - running.computeIfAbsent(request.getDataSource(), k -> new TreeMap<>()) - .computeIfAbsent( + running.computeIfAbsent( request.getInterval().getStart(), k -> new TreeMap<>(Comparators.intervalsByStartThenEnd()) ) @@ -823,8 +814,7 @@ public void revokeLock(String taskId, TaskLock lock) final TaskLock revokedLock = lock.revokedCopy(); taskStorage.replaceLock(taskId, lock, revokedLock); - final List possesHolder = running.get(task.getDataSource()) - .get(lock.getInterval().getStart()) + final List possesHolder = running.get(lock.getInterval().getStart()) .get(lock.getInterval()); final TaskLockPosse foundPosse = possesHolder.stream() .filter(posse -> posse.getTaskLock().equals(lock)) @@ -881,13 +871,8 @@ public Set getAllReplaceLocksForDatasource(String datasource) { giant.lock(); try { - final NavigableMap>> activeLocks = running.get(datasource); - if (activeLocks == null) { - return ImmutableSet.of(); - } - List lockPosses - = activeLocks.values() + = running.values() .stream() .flatMap(map -> map.values().stream()) .flatMap(Collection::stream) @@ -926,12 +911,12 @@ private Set getNonRevokedReplaceLocks(List posse } /** - * @param lockFilterPolicies Lock filters for the given datasources - * @return Map from datasource to intervals locked by tasks satisfying the lock filter condititions + * @param lockFilterPolicies Lock filters for this datasource + * @return List of intervals locked by tasks satisfying the lock filter condititions. */ - public Map> getLockedIntervals(List lockFilterPolicies) + public List getLockedIntervals(List lockFilterPolicies) { - final Map> datasourceToIntervals = new HashMap<>(); + final Set lockedIntervals = new HashSet<>(); // Take a lock and populate the maps giant.lock(); @@ -939,8 +924,7 @@ public Map> getLockedIntervals(List loc try { lockFilterPolicies.forEach( lockFilter -> { - final String datasource = lockFilter.getDatasource(); - if (!running.containsKey(datasource)) { + if (!dataSource.equals(lockFilter.getDatasource())) { return; } @@ -959,7 +943,7 @@ public Map> getLockedIntervals(List loc ); final boolean ignoreAppendLocks = isUsingConcurrentLocks || isReplaceLock; - running.get(datasource).forEach( + running.forEach( (startTime, startTimeLocks) -> startTimeLocks.forEach( (interval, taskLockPosses) -> taskLockPosses.forEach( taskLockPosse -> { @@ -972,8 +956,7 @@ public Map> getLockedIntervals(List loc || taskLockPosse.getTaskLock().getPriority() < priority) { // do nothing } else { - datasourceToIntervals.computeIfAbsent(datasource, k -> new HashSet<>()) - .add(interval); + lockedIntervals.add(interval); } } ) @@ -986,20 +969,16 @@ public Map> getLockedIntervals(List loc giant.unlock(); } - return datasourceToIntervals.entrySet().stream() - .collect(Collectors.toMap( - Map.Entry::getKey, - entry -> new ArrayList<>(entry.getValue()) - )); + return new ArrayList<>(lockedIntervals); } /** * @param lockFilterPolicies Lock filters for the given datasources - * @return Map from datasource to list of non-revoked locks with at least as much priority and an overlapping interval + * @return List of non-revoked locks with at least as much priority and an overlapping interval. */ - public Map> getActiveLocks(List lockFilterPolicies) + public List getActiveLocks(List lockFilterPolicies) { - final Map> datasourceToLocks = new HashMap<>(); + final List activeLocks = new ArrayList<>(); // Take a lock and populate the maps giant.lock(); @@ -1007,8 +986,7 @@ public Map> getActiveLocks(List lockFil try { lockFilterPolicies.forEach( lockFilter -> { - final String datasource = lockFilter.getDatasource(); - if (!running.containsKey(datasource)) { + if (!dataSource.equals(lockFilter.getDatasource())) { return; } @@ -1041,7 +1019,7 @@ public Map> getActiveLocks(List lockFil ignoreAppendLocks = useConcurrentLocks; } - running.get(datasource).forEach( + running.forEach( (startTime, startTimeLocks) -> startTimeLocks.forEach( (interval, taskLockPosses) -> taskLockPosses.forEach( taskLockPosse -> { @@ -1056,8 +1034,7 @@ public Map> getActiveLocks(List lockFil } else { for (Interval filterInterval : intervals) { if (interval.overlaps(filterInterval)) { - datasourceToLocks.computeIfAbsent(datasource, ds -> new ArrayList<>()) - .add(taskLockPosse.getTaskLock()); + activeLocks.add(taskLockPosse.getTaskLock()); break; } } @@ -1073,7 +1050,7 @@ public Map> getActiveLocks(List lockFil giant.unlock(); } - return datasourceToLocks; + return activeLocks; } public void unlock(final Task task, final Interval interval) @@ -1093,10 +1070,9 @@ private void unlock(final Task task, final Interval interval, @Nullable Integer giant.lock(); try { - final String dataSource = task.getDataSource(); final NavigableMap>> locksForDatasource - = running.get(task.getDataSource()); - if (locksForDatasource == null || locksForDatasource.isEmpty()) { + = running; + if (locksForDatasource.isEmpty()) { return; } @@ -1138,9 +1114,6 @@ private void unlock(final Task task, final Interval interval, @Nullable Integer if (intervalToPosses.isEmpty()) { locksForDatasource.remove(interval.getStart()); } - if (running.get(dataSource).isEmpty()) { - running.remove(dataSource); - } // Wake up blocking-lock waiters lockReleaseCondition.signalAll(); @@ -1204,7 +1177,6 @@ public void add(Task task) } } - @GuardedBy("giant") private void trackAppendingTask(Task task) { if (task instanceof PendingSegmentAllocatingTask) { @@ -1283,25 +1255,19 @@ private void cleanupUpgradeAndPendingSegments(Task task) private List findLockPossesForTask(final Task task) { // Scan through all locks for this datasource - final NavigableMap>> locksForDatasource - = running.get(task.getDataSource()); - if (locksForDatasource == null) { - return Collections.emptyList(); - } - - return locksForDatasource.values().stream() + return running.values().stream() .flatMap(map -> map.values().stream()) .flatMap(Collection::stream) .filter(taskLockPosse -> taskLockPosse.containsTask(task)) .collect(Collectors.toList()); } - private List findLockPossesContainingInterval(final String dataSource, final Interval interval) + private List findLockPossesContainingInterval(final Interval interval) { giant.lock(); try { - final List intervalOverlapsPosses = findLockPossesOverlapsInterval(dataSource, interval); + final List intervalOverlapsPosses = findLockPossesOverlapsInterval(interval); return intervalOverlapsPosses.stream() .filter(taskLockPosse -> taskLockPosse.taskLock.getInterval().contains(interval)) .collect(Collectors.toList()); @@ -1314,19 +1280,18 @@ private List findLockPossesContainingInterval(final String dataSo /** * Return all locks that overlap some search interval. */ - private List findLockPossesOverlapsInterval(final String dataSource, final Interval interval) + private List findLockPossesOverlapsInterval(final Interval interval) { giant.lock(); try { - final NavigableMap>> dsRunning = running.get(dataSource); - if (dsRunning == null) { + if (running.isEmpty()) { // No locks at all return Collections.emptyList(); } else { - return dsRunning.navigableKeySet().stream() + return running.navigableKeySet().stream() .filter(java.util.Objects::nonNull) - .map(dsRunning::get) + .map(running::get) .filter(java.util.Objects::nonNull) .flatMap(sortedMap -> sortedMap.entrySet().stream()) .filter(entry -> entry.getKey().overlaps(interval)) @@ -1344,7 +1309,7 @@ Optional getOnlyTaskLockPosseContainingInterval(Task task, Interv { giant.lock(); try { - final List filteredPosses = findLockPossesContainingInterval(task.getDataSource(), interval) + final List filteredPosses = findLockPossesContainingInterval(interval) .stream() .filter(lockPosse -> lockPosse.containsTask(task)) .collect(Collectors.toList()); @@ -1378,7 +1343,7 @@ Set getActiveTasks() } @VisibleForTesting - Map>>> getAllLocks() + NavigableMap>> getAllLocks() { return running; } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskLockboxSyncResult.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskLockboxSyncResult.java index b7273b6bdef1..88d5f7c9d299 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskLockboxSyncResult.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskLockboxSyncResult.java @@ -29,10 +29,12 @@ */ class TaskLockboxSyncResult { + private final int taskLockCount; private final Set tasksToFail; - TaskLockboxSyncResult(Set tasksToFail) + TaskLockboxSyncResult(Set tasksToFail, int taskLockCount) { + this.taskLockCount = taskLockCount; this.tasksToFail = tasksToFail; } @@ -43,4 +45,9 @@ Set getTasksToFail() { return tasksToFail; } + + int getTaskLockCount() + { + return taskLockCount; + } } 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 3613b6fa08d2..8e4830611bee 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 @@ -57,7 +57,7 @@ /** * Provides read-only methods to fetch information related to tasks. * This class may serve information that is cached in memory in {@link TaskQueue} - * or {@link TaskLockbox}. If not present in memory, then the underlying + * or {@link GlobalTaskLockbox}. If not present in memory, then the underlying * {@link TaskStorage} is queried. */ public class TaskQueryTool @@ -65,7 +65,7 @@ public class TaskQueryTool private static final Logger log = new Logger(TaskQueryTool.class); private final TaskStorage storage; - private final TaskLockbox taskLockbox; + private final GlobalTaskLockbox taskLockbox; private final TaskMaster taskMaster; private final Supplier workerBehaviorConfigSupplier; private final ProvisioningStrategy provisioningStrategy; @@ -73,7 +73,7 @@ public class TaskQueryTool @Inject public TaskQueryTool( TaskStorage storage, - TaskLockbox taskLockbox, + GlobalTaskLockbox taskLockbox, TaskMaster taskMaster, ProvisioningStrategy provisioningStrategy, Supplier workerBehaviorConfigSupplier diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskQueue.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskQueue.java index ea8f1a56ed85..40ec978a78fd 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskQueue.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskQueue.java @@ -123,7 +123,7 @@ public class TaskQueue private final TaskStorage taskStorage; private final TaskRunner taskRunner; private final TaskActionClientFactory taskActionClientFactory; - private final TaskLockbox taskLockbox; + private final GlobalTaskLockbox taskLockbox; private final ServiceEmitter emitter; private final ObjectMapper passwordRedactingMapper; private final TaskContextEnricher taskContextEnricher; @@ -169,7 +169,7 @@ public TaskQueue( TaskStorage taskStorage, TaskRunner taskRunner, TaskActionClientFactory taskActionClientFactory, - TaskLockbox taskLockbox, + GlobalTaskLockbox taskLockbox, ServiceEmitter emitter, ObjectMapper mapper, TaskContextEnricher taskContextEnricher diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentAllocateActionTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentAllocateActionTest.java index fdb7fcd5595b..b9813950f010 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentAllocateActionTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentAllocateActionTest.java @@ -31,7 +31,7 @@ import org.apache.druid.indexing.common.task.NoopTask; import org.apache.druid.indexing.common.task.Task; import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator; -import org.apache.druid.indexing.overlord.TaskLockbox; +import org.apache.druid.indexing.overlord.GlobalTaskLockbox; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.Intervals; @@ -1133,7 +1133,7 @@ public void testAllocateDayWhenMonthNotPossible() public void testSegmentIdMustNotBeReused() { final IndexerMetadataStorageCoordinator coordinator = taskActionTestKit.getMetadataStorageCoordinator(); - final TaskLockbox lockbox = taskActionTestKit.getTaskLockbox(); + final GlobalTaskLockbox lockbox = taskActionTestKit.getTaskLockbox(); final Task task0 = NoopTask.ofPriority(25); lockbox.add(task0); final NoopTask task1 = NoopTask.ofPriority(50); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/TaskActionTestKit.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/TaskActionTestKit.java index fcbf37c956da..cde347c87c65 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/TaskActionTestKit.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/TaskActionTestKit.java @@ -25,7 +25,7 @@ import org.apache.druid.indexing.common.config.TaskStorageConfig; import org.apache.druid.indexing.overlord.HeapMemoryTaskStorage; import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator; -import org.apache.druid.indexing.overlord.TaskLockbox; +import org.apache.druid.indexing.overlord.GlobalTaskLockbox; import org.apache.druid.indexing.overlord.TaskStorage; import org.apache.druid.indexing.overlord.config.TaskLockConfig; import org.apache.druid.indexing.overlord.supervisor.SupervisorManager; @@ -50,7 +50,7 @@ public class TaskActionTestKit extends ExternalResource private final MetadataStorageTablesConfig metadataStorageTablesConfig = MetadataStorageTablesConfig.fromBase("druid"); private TaskStorage taskStorage; - private TaskLockbox taskLockbox; + private GlobalTaskLockbox taskLockbox; private TestDerbyConnector testDerbyConnector; private IndexerMetadataStorageCoordinator metadataStorageCoordinator; private SegmentsMetadataManager segmentsMetadataManager; @@ -58,7 +58,7 @@ public class TaskActionTestKit extends ExternalResource private SegmentSchemaManager segmentSchemaManager; private SegmentSchemaCache segmentSchemaCache; - public TaskLockbox getTaskLockbox() + public GlobalTaskLockbox getTaskLockbox() { return taskLockbox; } @@ -102,7 +102,7 @@ public int getSqlMetadataMaxRetry() return 2; } }; - taskLockbox = new TaskLockbox(taskStorage, metadataStorageCoordinator); + taskLockbox = new GlobalTaskLockbox(taskStorage, metadataStorageCoordinator); segmentSchemaCache = new SegmentSchemaCache(NoopServiceEmitter.instance()); segmentsMetadataManager = new SqlSegmentsMetadataManager( objectMapper, diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/TaskLocksTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/TaskLocksTest.java index f2749d15cdb1..3700cbeb2392 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/TaskLocksTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/TaskLocksTest.java @@ -34,7 +34,7 @@ import org.apache.druid.indexing.overlord.HeapMemoryTaskStorage; import org.apache.druid.indexing.overlord.LockResult; import org.apache.druid.indexing.overlord.SpecificSegmentLockRequest; -import org.apache.druid.indexing.overlord.TaskLockbox; +import org.apache.druid.indexing.overlord.GlobalTaskLockbox; import org.apache.druid.indexing.overlord.TaskStorage; import org.apache.druid.indexing.overlord.TimeChunkLockRequest; import org.apache.druid.indexing.test.TestIndexerMetadataStorageCoordinator; @@ -58,14 +58,14 @@ public class TaskLocksTest { - private TaskLockbox lockbox; + private GlobalTaskLockbox lockbox; private Task task; @Before public void setup() { final TaskStorage taskStorage = new HeapMemoryTaskStorage(new TaskStorageConfig(null)); - lockbox = new TaskLockbox( + lockbox = new GlobalTaskLockbox( taskStorage, new TestIndexerMetadataStorageCoordinator() ); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IngestionTestBase.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IngestionTestBase.java index 7f32c67c2384..4a7793df2bd2 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IngestionTestBase.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IngestionTestBase.java @@ -51,7 +51,7 @@ import org.apache.druid.indexing.common.config.TaskStorageConfig; import org.apache.druid.indexing.overlord.HeapMemoryTaskStorage; import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator; -import org.apache.druid.indexing.overlord.TaskLockbox; +import org.apache.druid.indexing.overlord.GlobalTaskLockbox; import org.apache.druid.indexing.overlord.TaskRunner; import org.apache.druid.indexing.overlord.TaskRunnerListener; import org.apache.druid.indexing.overlord.TaskRunnerWorkItem; @@ -122,7 +122,7 @@ public abstract class IngestionTestBase extends InitializedNullHandlingTest private TaskStorage taskStorage; private IndexerSQLMetadataStorageCoordinator storageCoordinator; private SegmentsMetadataManager segmentsMetadataManager; - private TaskLockbox lockbox; + private GlobalTaskLockbox lockbox; private File baseDir; private SegmentSchemaManager segmentSchemaManager; private SegmentSchemaCache segmentSchemaCache; @@ -165,7 +165,7 @@ public void setUpIngestionTestBase() throws IOException CentralizedDatasourceSchemaConfig.create(), NoopServiceEmitter.instance() ); - lockbox = new TaskLockbox(taskStorage, storageCoordinator); + lockbox = new GlobalTaskLockbox(taskStorage, storageCoordinator); segmentCacheManagerFactory = new SegmentCacheManagerFactory(TestIndex.INDEX_IO, getObjectMapper()); reportsFile = temporaryFolder.newFile(); dataSegmentKiller = new TestDataSegmentKiller(); @@ -228,7 +228,7 @@ public SegmentsMetadataManager getSegmentsMetadataManager() return segmentsMetadataManager; } - public TaskLockbox getLockbox() + public GlobalTaskLockbox getLockbox() { return lockbox; } 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 f48c1d87a2b2..21c305ca0b35 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 @@ -29,7 +29,7 @@ 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.GlobalTaskLockbox; import org.apache.druid.indexing.overlord.TaskMaster; import org.apache.druid.indexing.overlord.TaskQueryTool; import org.apache.druid.indexing.overlord.TaskQueue; @@ -129,7 +129,7 @@ public void setUp() private void initScheduler() { - TaskLockbox taskLockbox = new TaskLockbox(taskStorage, new TestIndexerMetadataStorageCoordinator()); + GlobalTaskLockbox taskLockbox = new GlobalTaskLockbox(taskStorage, new TestIndexerMetadataStorageCoordinator()); WorkerBehaviorConfig defaultWorkerConfig = new DefaultWorkerBehaviorConfig(WorkerBehaviorConfig.DEFAULT_STRATEGY, null); scheduler = new OverlordCompactionScheduler( diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLockboxTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/GlobalTaskLockboxTest.java similarity index 97% rename from indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLockboxTest.java rename to indexing-service/src/test/java/org/apache/druid/indexing/overlord/GlobalTaskLockboxTest.java index a8c4b5117b1b..57bbb4c853e1 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLockboxTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/GlobalTaskLockboxTest.java @@ -43,7 +43,6 @@ import org.apache.druid.indexing.common.task.NoopTask; import org.apache.druid.indexing.common.task.Task; import org.apache.druid.indexing.common.task.Tasks; -import org.apache.druid.indexing.overlord.TaskLockbox.TaskLockPosse; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.ISE; @@ -85,7 +84,7 @@ import java.util.Set; import java.util.stream.Collectors; -public class TaskLockboxTest +public class GlobalTaskLockboxTest { @Rule public final TestDerbyConnector.DerbyConnectorRule derby = new TestDerbyConnector.DerbyConnectorRule(); @@ -93,7 +92,7 @@ public class TaskLockboxTest private ObjectMapper objectMapper; private TaskStorage taskStorage; private IndexerMetadataStorageCoordinator metadataStorageCoordinator; - private TaskLockbox lockbox; + private GlobalTaskLockbox lockbox; private TaskLockboxValidator validator; private SegmentSchemaManager segmentSchemaManager; @@ -136,7 +135,7 @@ public void setup() CentralizedDatasourceSchemaConfig.create() ); - lockbox = new TaskLockbox(taskStorage, metadataStorageCoordinator); + lockbox = new GlobalTaskLockbox(taskStorage, metadataStorageCoordinator); validator = new TaskLockboxValidator(lockbox, taskStorage); } @@ -332,7 +331,7 @@ public void testTimeoutForLock() throws InterruptedException @Test public void testSyncFromStorage() { - final TaskLockbox originalBox = new TaskLockbox(taskStorage, metadataStorageCoordinator); + final GlobalTaskLockbox originalBox = new GlobalTaskLockbox(taskStorage, metadataStorageCoordinator); for (int i = 0; i < 5; i++) { final Task task = NoopTask.create(); taskStorage.insert(task, TaskStatus.running(task.getId())); @@ -354,7 +353,7 @@ public void testSyncFromStorage() .flatMap(task -> taskStorage.getLocks(task.getId()).stream()) .collect(Collectors.toList()); - final TaskLockbox newBox = new TaskLockbox(taskStorage, metadataStorageCoordinator); + final GlobalTaskLockbox newBox = new GlobalTaskLockbox(taskStorage, metadataStorageCoordinator); newBox.syncFromStorage(); Assert.assertEquals(originalBox.getAllLocks(), newBox.getAllLocks()); @@ -381,7 +380,7 @@ public void testSyncFromStorageWithMissingTaskLockPriority() .flatMap(t -> taskStorage.getLocks(t.getId()).stream()) .collect(Collectors.toList()); - final TaskLockbox lockbox = new TaskLockbox(taskStorage, metadataStorageCoordinator); + final GlobalTaskLockbox lockbox = new GlobalTaskLockbox(taskStorage, metadataStorageCoordinator); lockbox.syncFromStorage(); final List afterLocksInStorage = taskStorage.getActiveTasks().stream() @@ -412,7 +411,7 @@ public void testSyncFromStorageWithMissingTaskPriority() .flatMap(t -> taskStorage.getLocks(t.getId()).stream()) .collect(Collectors.toList()); - final TaskLockbox lockbox = new TaskLockbox(taskStorage, metadataStorageCoordinator); + final GlobalTaskLockbox lockbox = new GlobalTaskLockbox(taskStorage, metadataStorageCoordinator); lockbox.syncFromStorage(); final List afterLocksInStorage = taskStorage.getActiveTasks().stream() @@ -439,7 +438,7 @@ public void testSyncFromStorageWithInvalidPriority() ) ); - final TaskLockbox lockbox = new TaskLockbox(taskStorage, metadataStorageCoordinator); + final GlobalTaskLockbox lockbox = new GlobalTaskLockbox(taskStorage, metadataStorageCoordinator); TaskLockboxSyncResult result = lockbox.syncFromStorage(); Assert.assertEquals(1, result.getTasksToFail().size()); Assert.assertTrue(result.getTasksToFail().contains(task)); @@ -470,8 +469,8 @@ public void testSyncWithUnknownTaskTypesFromModuleNotLoaded() CentralizedDatasourceSchemaConfig.create() ); - TaskLockbox theBox = new TaskLockbox(taskStorage, metadataStorageCoordinator); - TaskLockbox loadedBox = new TaskLockbox(loadedTaskStorage, loadedMetadataStorageCoordinator); + GlobalTaskLockbox theBox = new GlobalTaskLockbox(taskStorage, metadataStorageCoordinator); + GlobalTaskLockbox loadedBox = new GlobalTaskLockbox(loadedTaskStorage, loadedMetadataStorageCoordinator); Task aTask = NoopTask.create(); taskStorage.insert(aTask, TaskStatus.running(aTask.getId())); @@ -496,7 +495,7 @@ public void testSyncWithUnknownTaskTypesFromModuleNotLoaded() @Test public void testRevokedLockSyncFromStorage() { - final TaskLockbox originalBox = new TaskLockbox(taskStorage, metadataStorageCoordinator); + final GlobalTaskLockbox originalBox = new GlobalTaskLockbox(taskStorage, metadataStorageCoordinator); final Task task1 = NoopTask.ofPriority(10); taskStorage.insert(task1, TaskStatus.running(task1.getId())); @@ -522,7 +521,7 @@ public void testRevokedLockSyncFromStorage() Assert.assertEquals(1, task2Locks.size()); Assert.assertTrue(task2Locks.get(0).isRevoked()); - final TaskLockbox newBox = new TaskLockbox(taskStorage, metadataStorageCoordinator); + final GlobalTaskLockbox newBox = new GlobalTaskLockbox(taskStorage, metadataStorageCoordinator); newBox.syncFromStorage(); final Set afterLocksInStorage = taskStorage.getActiveTasks().stream() @@ -757,7 +756,7 @@ public void testFindLockPosseAfterRevokeWithDifferentLockIntervals() ).isOk() ); - final Optional highLockPosse = lockbox.getOnlyTaskLockPosseContainingInterval( + final Optional highLockPosse = lockbox.getOnlyTaskLockPosseContainingInterval( highPriorityTask, Intervals.of("2018-12-16T09:00:00/2018-12-16T09:30:00") ); @@ -766,7 +765,7 @@ public void testFindLockPosseAfterRevokeWithDifferentLockIntervals() Assert.assertTrue(highLockPosse.get().containsTask(highPriorityTask)); Assert.assertFalse(highLockPosse.get().getTaskLock().isRevoked()); - final Optional lowLockPosse = lockbox.getOnlyTaskLockPosseContainingInterval( + final Optional lowLockPosse = lockbox.getOnlyTaskLockPosseContainingInterval( lowPriorityTask, Intervals.of("2018-12-16T09:00:00/2018-12-16T10:00:00") ); @@ -1095,9 +1094,9 @@ public void testLockPosseEquals() task2.getPriority() ); - TaskLockPosse taskLockPosse1 = new TaskLockPosse(taskLock1); - TaskLockPosse taskLockPosse2 = new TaskLockPosse(taskLock2); - TaskLockPosse taskLockPosse3 = new TaskLockPosse(taskLock1); + TaskLockbox.TaskLockPosse taskLockPosse1 = new TaskLockbox.TaskLockPosse(taskLock1); + TaskLockbox.TaskLockPosse taskLockPosse2 = new TaskLockbox.TaskLockPosse(taskLock2); + TaskLockbox.TaskLockPosse taskLockPosse3 = new TaskLockbox.TaskLockPosse(taskLock1); Assert.assertNotEquals(taskLockPosse1, null); Assert.assertNotEquals(null, taskLockPosse1); @@ -1128,7 +1127,7 @@ public void testGetTimeChunkAndSegmentLockForSameGroup() ).isOk() ); - final List posses = lockbox + final List posses = lockbox .getAllLocks() .get(task1.getDataSource()) .get(DateTimes.of("2017")) @@ -1857,7 +1856,7 @@ public void testFailedToReacquireTaskLock() taskStorage.insert(taskWithFailingLockAcquisition1, TaskStatus.running(taskWithFailingLockAcquisition1.getId())); taskStorage.insert(taskWithSuccessfulLockAcquisition, TaskStatus.running(taskWithSuccessfulLockAcquisition.getId())); - TaskLockbox testLockbox = new NullLockPosseTaskLockbox(taskStorage, metadataStorageCoordinator); + GlobalTaskLockbox testLockbox = new NullLockPosseGlobalTaskLockbox(taskStorage, metadataStorageCoordinator); testLockbox.add(taskWithFailingLockAcquisition0); testLockbox.add(taskWithFailingLockAcquisition1); testLockbox.add(taskWithSuccessfulLockAcquisition); @@ -2012,7 +2011,7 @@ public void testCleanupOnUnlock() .andReturn(0).once(); EasyMock.replay(coordinator); - final TaskLockbox taskLockbox = new TaskLockbox(taskStorage, coordinator); + final GlobalTaskLockbox taskLockbox = new GlobalTaskLockbox(taskStorage, coordinator); taskLockbox.add(replaceTask); taskLockbox.tryLock( @@ -2036,11 +2035,11 @@ private class TaskLockboxValidator { private final Set tasks; - private final TaskLockbox lockbox; + private final GlobalTaskLockbox lockbox; private final TaskStorage taskStorage; private final Map lockToTaskIdMap; - TaskLockboxValidator(TaskLockbox lockbox, TaskStorage taskStorage) + TaskLockboxValidator(GlobalTaskLockbox lockbox, TaskStorage taskStorage) { lockToTaskIdMap = new HashMap<>(); tasks = new HashSet<>(); @@ -2259,9 +2258,9 @@ public TaskStatus runTask(TaskToolbox toolbox) /** * Extends TaskLockbox to return a null TaskLockPosse when the task's group name contains "FailingLockAcquisition". */ - private static class NullLockPosseTaskLockbox extends TaskLockbox + private static class NullLockPosseGlobalTaskLockbox extends GlobalTaskLockbox { - public NullLockPosseTaskLockbox( + public NullLockPosseGlobalTaskLockbox( TaskStorage taskStorage, IndexerMetadataStorageCoordinator metadataStorageCoordinator ) @@ -2270,7 +2269,7 @@ public NullLockPosseTaskLockbox( } @Override - protected TaskLockPosse verifyAndCreateOrFindLockPosse(Task task, TaskLock taskLock) + protected TaskLockbox.TaskLockPosse verifyAndCreateOrFindLockPosse(Task task, TaskLock taskLock) { return task.getGroupId() .contains("FailingLockAcquisition") ? null : super.verifyAndCreateOrFindLockPosse(task, taskLock); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java index abab38bf0c9b..088856b27878 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java @@ -232,7 +232,7 @@ public int compare(DataSegment dataSegment, DataSegment dataSegment2) private ObjectMapper mapper; private TaskQueryTool tsqa = null; private TaskStorage taskStorage = null; - private TaskLockbox taskLockbox = null; + private GlobalTaskLockbox taskLockbox = null; private TaskQueue taskQueue = null; private TaskRunner taskRunner = null; private TestIndexerMetadataStorageCoordinator mdc = null; @@ -547,7 +547,7 @@ private TaskToolboxFactory setUpTaskToolboxFactory( Preconditions.checkNotNull(taskStorage); Preconditions.checkNotNull(emitter); - taskLockbox = new TaskLockbox(taskStorage, mdc); + taskLockbox = new GlobalTaskLockbox(taskStorage, mdc); tac = new LocalTaskActionClientFactory( new TaskActionToolbox( taskLockbox, diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLockBoxConcurrencyTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLockBoxConcurrencyTest.java index 4dc0416cd1f2..189f86591709 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLockBoxConcurrencyTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLockBoxConcurrencyTest.java @@ -58,7 +58,7 @@ public class TaskLockBoxConcurrencyTest private final ObjectMapper objectMapper = new DefaultObjectMapper(); private ExecutorService service; private TaskStorage taskStorage; - private TaskLockbox lockbox; + private GlobalTaskLockbox lockbox; private SegmentSchemaManager segmentSchemaManager; @Before @@ -77,7 +77,7 @@ public void setup() ); segmentSchemaManager = new SegmentSchemaManager(derby.metadataTablesConfigSupplier().get(), objectMapper, derbyConnector); - lockbox = new TaskLockbox( + lockbox = new GlobalTaskLockbox( taskStorage, new IndexerSQLMetadataStorageCoordinator( objectMapper, diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLockConfigTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLockConfigTest.java index 69c2e1fe8cab..f257691fdd87 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLockConfigTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLockConfigTest.java @@ -112,7 +112,7 @@ public boolean isForceTimeChunkLock() final TaskQueueConfig queueConfig = new TaskQueueConfig(null, null, null, null, null, null); final TaskRunner taskRunner = EasyMock.createNiceMock(RemoteTaskRunner.class); final TaskActionClientFactory actionClientFactory = EasyMock.createNiceMock(LocalTaskActionClientFactory.class); - final TaskLockbox lockbox = new TaskLockbox(taskStorage, new TestIndexerMetadataStorageCoordinator()); + final GlobalTaskLockbox lockbox = new GlobalTaskLockbox(taskStorage, new TestIndexerMetadataStorageCoordinator()); final ServiceEmitter emitter = new NoopServiceEmitter(); return new TaskQueue( lockConfig, diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskQueueScaleTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskQueueScaleTest.java index f67e9fc28614..7158b08edcba 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskQueueScaleTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskQueueScaleTest.java @@ -127,7 +127,7 @@ public RetType submit(TaskAction taskAction) taskStorage, taskRunner, unsupportedTaskActionFactory, // Not used for anything serious - new TaskLockbox(taskStorage, storageCoordinator), + new GlobalTaskLockbox(taskStorage, storageCoordinator), new NoopServiceEmitter(), jsonMapper, new NoopTaskContextEnricher() diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskQueueTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskQueueTest.java index 8f1393f2c675..70fa459a9443 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskQueueTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskQueueTest.java @@ -543,7 +543,7 @@ public void testGetActiveTaskRedactsPassword() throws JsonProcessingException taskStorage, EasyMock.createMock(HttpRemoteTaskRunner.class), createActionClientFactory(), - new TaskLockbox(taskStorage, new TestIndexerMetadataStorageCoordinator()), + new GlobalTaskLockbox(taskStorage, new TestIndexerMetadataStorageCoordinator()), new StubServiceEmitter("druid/overlord", "testHost"), mapper, new NoopTaskContextEnricher() 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 732d586002f8..bfe82597c816 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 @@ -44,7 +44,7 @@ import org.apache.druid.indexing.common.task.Task; import org.apache.druid.indexing.overlord.DruidOverlord; import org.apache.druid.indexing.overlord.IndexerMetadataStorageAdapter; -import org.apache.druid.indexing.overlord.TaskLockbox; +import org.apache.druid.indexing.overlord.GlobalTaskLockbox; import org.apache.druid.indexing.overlord.TaskMaster; import org.apache.druid.indexing.overlord.TaskQueryTool; import org.apache.druid.indexing.overlord.TaskQueue; @@ -107,7 +107,7 @@ public class OverlordResourceTest private DruidOverlord overlord; private TaskMaster taskMaster; private TaskStorage taskStorage; - private TaskLockbox taskLockbox; + private GlobalTaskLockbox taskLockbox; private JacksonConfigManager configManager; private ProvisioningStrategy provisioningStrategy; private AuthConfig authConfig; @@ -133,7 +133,7 @@ public void setUp() overlord = EasyMock.createStrictMock(DruidOverlord.class); taskMaster = EasyMock.createStrictMock(TaskMaster.class); taskStorage = EasyMock.createStrictMock(TaskStorage.class); - taskLockbox = EasyMock.createStrictMock(TaskLockbox.class); + taskLockbox = EasyMock.createStrictMock(GlobalTaskLockbox.class); taskQueryTool = new TaskQueryTool( taskStorage, taskLockbox, 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..02f5a463ed8d 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 @@ -52,7 +52,7 @@ import org.apache.druid.indexing.overlord.HeapMemoryTaskStorage; import org.apache.druid.indexing.overlord.IndexerMetadataStorageAdapter; import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator; -import org.apache.druid.indexing.overlord.TaskLockbox; +import org.apache.druid.indexing.overlord.GlobalTaskLockbox; import org.apache.druid.indexing.overlord.TaskMaster; import org.apache.druid.indexing.overlord.TaskQueryTool; import org.apache.druid.indexing.overlord.TaskRunner; @@ -110,7 +110,7 @@ public class OverlordTest private CuratorFramework curator; private DruidOverlord overlord; private TaskMaster taskMaster; - private TaskLockbox taskLockbox; + private GlobalTaskLockbox taskLockbox; private TaskStorage taskStorage; private TaskActionClientFactory taskActionClientFactory; private CountDownLatch announcementLatch; @@ -179,7 +179,7 @@ public void setUp() throws Exception IndexerMetadataStorageCoordinator mdc = new TestIndexerMetadataStorageCoordinator(); - taskLockbox = new TaskLockbox(taskStorage, mdc); + taskLockbox = new GlobalTaskLockbox(taskStorage, mdc); task0 = NoopTask.create(); taskId0 = task0.getId(); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskTestBase.java b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskTestBase.java index 7a4fd7dadb3a..4063f89a0932 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskTestBase.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskTestBase.java @@ -72,7 +72,7 @@ import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator; import org.apache.druid.indexing.overlord.MetadataTaskStorage; import org.apache.druid.indexing.overlord.Segments; -import org.apache.druid.indexing.overlord.TaskLockbox; +import org.apache.druid.indexing.overlord.GlobalTaskLockbox; import org.apache.druid.indexing.overlord.TaskStorage; import org.apache.druid.indexing.overlord.supervisor.SupervisorManager; import org.apache.druid.indexing.test.TestDataSegmentAnnouncer; @@ -200,7 +200,7 @@ public abstract class SeekableStreamIndexTaskTestBase extends EasyMockSupport protected File reportsFile; protected TaskToolboxFactory toolboxFactory; protected TaskStorage taskStorage; - protected TaskLockbox taskLockbox; + protected GlobalTaskLockbox taskLockbox; protected IndexerMetadataStorageCoordinator metadataStorageCoordinator; protected final Set checkpointRequestsHash = new HashSet<>(); protected SegmentSchemaManager segmentSchemaManager; @@ -598,7 +598,7 @@ protected void makeToolboxFactory(TestUtils testUtils, ServiceEmitter emitter, b segmentSchemaManager, CentralizedDatasourceSchemaConfig.create() ); - taskLockbox = new TaskLockbox(taskStorage, metadataStorageCoordinator); + taskLockbox = new GlobalTaskLockbox(taskStorage, metadataStorageCoordinator); final TaskActionToolbox taskActionToolbox = new TaskActionToolbox( taskLockbox, taskStorage, 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 52962ddf4c1e..8928ab7772d7 100644 --- a/services/src/main/java/org/apache/druid/cli/CliOverlord.java +++ b/services/src/main/java/org/apache/druid/cli/CliOverlord.java @@ -75,7 +75,7 @@ import org.apache.druid.indexing.overlord.IndexerMetadataStorageAdapter; import org.apache.druid.indexing.overlord.MetadataTaskStorage; import org.apache.druid.indexing.overlord.RemoteTaskRunnerFactory; -import org.apache.druid.indexing.overlord.TaskLockbox; +import org.apache.druid.indexing.overlord.GlobalTaskLockbox; import org.apache.druid.indexing.overlord.TaskMaster; import org.apache.druid.indexing.overlord.TaskQueryTool; import org.apache.druid.indexing.overlord.TaskRunnerFactory; @@ -245,7 +245,7 @@ public void configure(Binder binder) binder.bind(TaskActionClientFactory.class).to(LocalTaskActionClientFactory.class).in(LazySingleton.class); binder.bind(TaskActionToolbox.class).in(LazySingleton.class); - binder.bind(TaskLockbox.class).in(LazySingleton.class); + binder.bind(GlobalTaskLockbox.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); From 5fd4198ccc24aa404ce42d20cda12b4012dcd429 Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Tue, 22 Oct 2024 23:03:25 +0530 Subject: [PATCH 02/14] Fix tests --- .../indexing/overlord/GlobalTaskLockbox.java | 68 ++++++++++++------- 1 file changed, 44 insertions(+), 24 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/GlobalTaskLockbox.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/GlobalTaskLockbox.java index 13dbd9ca9438..4497f855bbc9 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/GlobalTaskLockbox.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/GlobalTaskLockbox.java @@ -116,6 +116,12 @@ public void unlock() } } + private static class DatasourceSync + { + final Set storedActiveTasks = new HashSet<>(); + final List> storedLocks = new ArrayList<>(); + } + /** * Wipe out our current in-memory state and resync it from our bundled {@link TaskStorage}. * @@ -127,20 +133,28 @@ public TaskLockboxSyncResult syncFromStorage() try { // Load stuff from taskStorage first. If this fails, we don't want to lose all our locks. - final Set storedActiveTasks = new HashSet<>(); - final List> storedLocks = new ArrayList<>(); + final Map datasourceSyncs = new HashMap<>(); + int activeTaskCount = 0; + int totalLockCount = 0; for (final Task task : taskStorage.getActiveTasks()) { - storedActiveTasks.add(task); + ++activeTaskCount; + final DatasourceSync sync = datasourceSyncs.computeIfAbsent(task.getDataSource(), ds -> new DatasourceSync()); + sync.storedActiveTasks.add(task); for (final TaskLock taskLock : taskStorage.getLocks(task.getId())) { - storedLocks.add(Pair.of(task, taskLock)); + ++totalLockCount; + sync.storedLocks.add(Pair.of(task, taskLock)); } } // Set of task groups in which at least one task failed to re-acquire a lock final Set tasksToFail = new HashSet<>(); int taskLockCount = 0; - for (TaskLockbox datasourceLockbox : datasourceLocks.values()) { - TaskLockboxSyncResult result = datasourceLockbox.resetState(storedActiveTasks, storedLocks); + + datasourceLocks.clear(); + for (String dataSource : datasourceSyncs.keySet()) { + final DatasourceSync sync = datasourceSyncs.get(dataSource); + final TaskLockboxSyncResult result = getDatasourceLockbox(dataSource) + .resetState(sync.storedActiveTasks, sync.storedLocks); tasksToFail.addAll(result.getTasksToFail()); taskLockCount += result.getTaskLockCount(); } @@ -148,8 +162,8 @@ public TaskLockboxSyncResult syncFromStorage() log.info( "Synced [%,d] locks for [%,d] activeTasks from storage ([%,d] locks ignored).", taskLockCount, - storedActiveTasks.size(), - storedLocks.size() - taskLockCount + activeTaskCount, + totalLockCount - taskLockCount ); return new TaskLockboxSyncResult(tasksToFail, taskLockCount); @@ -314,12 +328,12 @@ public Map> getLockedIntervals(List loc } final Map> datasourceToLockedIntervals = new HashMap<>(); - datasourceToFilterPolicies.forEach( - (datasource, policies) -> datasourceToLockedIntervals.put( - datasource, - getDatasourceLockbox(datasource).getLockedIntervals(policies) - ) - ); + datasourceToFilterPolicies.forEach((datasource, policies) -> { + final List lockedIntervals = getDatasourceLockbox(datasource).getLockedIntervals(policies); + if (!lockedIntervals.isEmpty()) { + datasourceToLockedIntervals.put(datasource, lockedIntervals); + } + }); return datasourceToLockedIntervals; } @@ -337,12 +351,12 @@ public Map> getActiveLocks(List lockFil } final Map> datasourceToActiveLocks = new HashMap<>(); - datasourceToFilterPolicies.forEach( - (datasource, policies) -> datasourceToActiveLocks.put( - datasource, - getDatasourceLockbox(datasource).getActiveLocks(policies) - ) - ); + datasourceToFilterPolicies.forEach((datasource, policies) -> { + final List datasourceLocks = getDatasourceLockbox(datasource).getActiveLocks(policies); + if (!datasourceLocks.isEmpty()) { + datasourceToActiveLocks.put(datasource, datasourceLocks); + } + }); return datasourceToActiveLocks; } @@ -390,10 +404,16 @@ Set getActiveTasks() @VisibleForTesting Map>>> getAllLocks() { - Map>>> allLocks - = new HashMap<>(); - - datasourceLocks.forEach((datasource, lockbox) -> allLocks.put(datasource, lockbox.getAllLocks())); + final Map>>> + allLocks = new HashMap<>(); + + datasourceLocks.forEach((datasource, lockbox) -> { + final NavigableMap>> + locks = lockbox.getAllLocks(); + if (!locks.isEmpty()) { + allLocks.put(datasource, locks); + } + }); return allLocks; } From 4a38391b4caabb7da328cd43f1429ed3c1cba05e Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Tue, 22 Oct 2024 23:28:29 +0530 Subject: [PATCH 03/14] Acquire lock when updating list of datasources --- .../druid/indexing/overlord/GlobalTaskLockbox.java | 13 +++++++++---- 1 file changed, 9 insertions(+), 4 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/GlobalTaskLockbox.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/GlobalTaskLockbox.java index 4497f855bbc9..989b478aa260 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/GlobalTaskLockbox.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/GlobalTaskLockbox.java @@ -86,10 +86,15 @@ private TaskLockbox getDatasourceLockbox(Task task) private TaskLockbox getDatasourceLockbox(String datasource) { - return datasourceLocks.computeIfAbsent( - datasource, - ds -> new TaskLockbox(ds, new DatasourceLock(), taskStorage, metadataStorageCoordinator) - ); + globalLock.readLock().lock(); + try { + return datasourceLocks.computeIfAbsent( + datasource, + ds -> new TaskLockbox(ds, new DatasourceLock(), taskStorage, metadataStorageCoordinator) + ); + } finally { + globalLock.readLock().unlock(); + } } private class DatasourceLock extends ReentrantLock From 07c52213f25483e1a1489e33415327055c60b166 Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Wed, 23 Oct 2024 00:13:35 +0530 Subject: [PATCH 04/14] Checkstyle and stuff --- .../indexing/common/actions/SegmentAllocationQueue.java | 2 +- .../druid/indexing/common/actions/TaskActionToolbox.java | 2 +- .../druid/indexing/overlord/GlobalTaskLockbox.java | 3 ++- .../org/apache/druid/indexing/overlord/TaskLockbox.java | 9 +++++---- .../common/actions/SegmentAllocateActionTest.java | 2 +- .../druid/indexing/common/actions/TaskActionTestKit.java | 2 +- .../druid/indexing/common/actions/TaskLocksTest.java | 2 +- .../druid/indexing/common/task/IngestionTestBase.java | 2 +- .../compact/OverlordCompactionSchedulerTest.java | 2 +- .../indexing/overlord/http/OverlordResourceTest.java | 2 +- .../druid/indexing/overlord/http/OverlordTest.java | 2 +- .../seekablestream/SeekableStreamIndexTaskTestBase.java | 2 +- .../src/main/java/org/apache/druid/cli/CliOverlord.java | 2 +- 13 files changed, 18 insertions(+), 16 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocationQueue.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocationQueue.java index 6061679ddded..783d6d34ca01 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocationQueue.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocationQueue.java @@ -24,9 +24,9 @@ import org.apache.druid.indexing.common.LockGranularity; import org.apache.druid.indexing.common.task.IndexTaskUtils; import org.apache.druid.indexing.common.task.Task; +import org.apache.druid.indexing.overlord.GlobalTaskLockbox; import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator; import org.apache.druid.indexing.overlord.Segments; -import org.apache.druid.indexing.overlord.GlobalTaskLockbox; import org.apache.druid.indexing.overlord.config.TaskLockConfig; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.Intervals; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TaskActionToolbox.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TaskActionToolbox.java index 43f9474c8c23..d430e3f5a9f2 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TaskActionToolbox.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TaskActionToolbox.java @@ -23,8 +23,8 @@ import com.google.common.base.Optional; import com.google.inject.Inject; import org.apache.druid.guice.annotations.Json; -import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator; import org.apache.druid.indexing.overlord.GlobalTaskLockbox; +import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator; import org.apache.druid.indexing.overlord.TaskRunner; import org.apache.druid.indexing.overlord.TaskRunnerFactory; import org.apache.druid.indexing.overlord.TaskStorage; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/GlobalTaskLockbox.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/GlobalTaskLockbox.java index 989b478aa260..e10faf0b8b3f 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/GlobalTaskLockbox.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/GlobalTaskLockbox.java @@ -92,7 +92,8 @@ private TaskLockbox getDatasourceLockbox(String datasource) datasource, ds -> new TaskLockbox(ds, new DatasourceLock(), taskStorage, metadataStorageCoordinator) ); - } finally { + } + finally { globalLock.readLock().unlock(); } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskLockbox.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskLockbox.java index aee7ce6c36f0..d8698329ac1b 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskLockbox.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskLockbox.java @@ -239,7 +239,8 @@ public int compare(Pair left, Pair right) } return new TaskLockboxSyncResult(tasksToFail, taskLockCount); - } catch (Throwable t) { + } + catch (Throwable t) { log.noStackTrace().error(t, "Error while resetting state of datasource[%s]", dataSource); throw t; } @@ -675,9 +676,9 @@ private TaskLockPosse createNewTaskLockPosse(LockRequest request) try { final TaskLockPosse posseToUse = new TaskLockPosse(request.toLock()); running.computeIfAbsent( - request.getInterval().getStart(), - k -> new TreeMap<>(Comparators.intervalsByStartThenEnd()) - ) + request.getInterval().getStart(), + k -> new TreeMap<>(Comparators.intervalsByStartThenEnd()) + ) .computeIfAbsent(request.getInterval(), k -> new ArrayList<>()) .add(posseToUse); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentAllocateActionTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentAllocateActionTest.java index b9813950f010..05fe54bd4cac 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentAllocateActionTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentAllocateActionTest.java @@ -30,8 +30,8 @@ import org.apache.druid.indexing.common.TaskLockType; import org.apache.druid.indexing.common.task.NoopTask; import org.apache.druid.indexing.common.task.Task; -import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator; import org.apache.druid.indexing.overlord.GlobalTaskLockbox; +import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.Intervals; diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/TaskActionTestKit.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/TaskActionTestKit.java index cde347c87c65..61b94db1183b 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/TaskActionTestKit.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/TaskActionTestKit.java @@ -23,9 +23,9 @@ import com.google.common.base.Suppliers; import org.apache.druid.indexing.common.TestUtils; import org.apache.druid.indexing.common.config.TaskStorageConfig; +import org.apache.druid.indexing.overlord.GlobalTaskLockbox; import org.apache.druid.indexing.overlord.HeapMemoryTaskStorage; import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator; -import org.apache.druid.indexing.overlord.GlobalTaskLockbox; import org.apache.druid.indexing.overlord.TaskStorage; import org.apache.druid.indexing.overlord.config.TaskLockConfig; import org.apache.druid.indexing.overlord.supervisor.SupervisorManager; diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/TaskLocksTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/TaskLocksTest.java index 3700cbeb2392..e9147f9d3311 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/TaskLocksTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/TaskLocksTest.java @@ -31,10 +31,10 @@ import org.apache.druid.indexing.common.task.NoopTask; import org.apache.druid.indexing.common.task.Task; import org.apache.druid.indexing.common.task.Tasks; +import org.apache.druid.indexing.overlord.GlobalTaskLockbox; import org.apache.druid.indexing.overlord.HeapMemoryTaskStorage; import org.apache.druid.indexing.overlord.LockResult; import org.apache.druid.indexing.overlord.SpecificSegmentLockRequest; -import org.apache.druid.indexing.overlord.GlobalTaskLockbox; import org.apache.druid.indexing.overlord.TaskStorage; import org.apache.druid.indexing.overlord.TimeChunkLockRequest; import org.apache.druid.indexing.test.TestIndexerMetadataStorageCoordinator; diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IngestionTestBase.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IngestionTestBase.java index 4a7793df2bd2..81c9de53a360 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IngestionTestBase.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IngestionTestBase.java @@ -49,9 +49,9 @@ import org.apache.druid.indexing.common.config.TaskConfig; import org.apache.druid.indexing.common.config.TaskConfigBuilder; import org.apache.druid.indexing.common.config.TaskStorageConfig; +import org.apache.druid.indexing.overlord.GlobalTaskLockbox; import org.apache.druid.indexing.overlord.HeapMemoryTaskStorage; import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator; -import org.apache.druid.indexing.overlord.GlobalTaskLockbox; import org.apache.druid.indexing.overlord.TaskRunner; import org.apache.druid.indexing.overlord.TaskRunnerListener; import org.apache.druid.indexing.overlord.TaskRunnerWorkItem; 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 21c305ca0b35..22d0b40e6403 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 @@ -28,8 +28,8 @@ 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.GlobalTaskLockbox; +import org.apache.druid.indexing.overlord.HeapMemoryTaskStorage; import org.apache.druid.indexing.overlord.TaskMaster; import org.apache.druid.indexing.overlord.TaskQueryTool; import org.apache.druid.indexing.overlord.TaskQueue; 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 bfe82597c816..a95dd9a26f39 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 @@ -43,8 +43,8 @@ import org.apache.druid.indexing.common.task.NoopTask; import org.apache.druid.indexing.common.task.Task; import org.apache.druid.indexing.overlord.DruidOverlord; -import org.apache.druid.indexing.overlord.IndexerMetadataStorageAdapter; import org.apache.druid.indexing.overlord.GlobalTaskLockbox; +import org.apache.druid.indexing.overlord.IndexerMetadataStorageAdapter; import org.apache.druid.indexing.overlord.TaskMaster; import org.apache.druid.indexing.overlord.TaskQueryTool; import org.apache.druid.indexing.overlord.TaskQueue; 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 02f5a463ed8d..b6fe6699196f 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 @@ -49,10 +49,10 @@ 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.GlobalTaskLockbox; import org.apache.druid.indexing.overlord.HeapMemoryTaskStorage; import org.apache.druid.indexing.overlord.IndexerMetadataStorageAdapter; import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator; -import org.apache.druid.indexing.overlord.GlobalTaskLockbox; import org.apache.druid.indexing.overlord.TaskMaster; import org.apache.druid.indexing.overlord.TaskQueryTool; import org.apache.druid.indexing.overlord.TaskRunner; diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskTestBase.java b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskTestBase.java index 4063f89a0932..6d288e57e7da 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskTestBase.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskTestBase.java @@ -69,10 +69,10 @@ import org.apache.druid.indexing.common.task.Tasks; import org.apache.druid.indexing.common.task.TestAppenderatorsManager; import org.apache.druid.indexing.overlord.DataSourceMetadata; +import org.apache.druid.indexing.overlord.GlobalTaskLockbox; import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator; import org.apache.druid.indexing.overlord.MetadataTaskStorage; import org.apache.druid.indexing.overlord.Segments; -import org.apache.druid.indexing.overlord.GlobalTaskLockbox; import org.apache.druid.indexing.overlord.TaskStorage; import org.apache.druid.indexing.overlord.supervisor.SupervisorManager; import org.apache.druid.indexing.test.TestDataSegmentAnnouncer; 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 8928ab7772d7..af42ca2424fc 100644 --- a/services/src/main/java/org/apache/druid/cli/CliOverlord.java +++ b/services/src/main/java/org/apache/druid/cli/CliOverlord.java @@ -71,11 +71,11 @@ 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.GlobalTaskLockbox; import org.apache.druid.indexing.overlord.HeapMemoryTaskStorage; import org.apache.druid.indexing.overlord.IndexerMetadataStorageAdapter; import org.apache.druid.indexing.overlord.MetadataTaskStorage; import org.apache.druid.indexing.overlord.RemoteTaskRunnerFactory; -import org.apache.druid.indexing.overlord.GlobalTaskLockbox; import org.apache.druid.indexing.overlord.TaskMaster; import org.apache.druid.indexing.overlord.TaskQueryTool; import org.apache.druid.indexing.overlord.TaskRunnerFactory; From ed98bb8ce4f1e64643289ea7a248e0cd799fe47f Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Thu, 20 Mar 2025 10:08:58 +0530 Subject: [PATCH 05/14] wip: temp changes --- .../indexing/overlord/GlobalTaskLockbox.java | 152 ++++++------------ .../druid/indexing/overlord/TaskLockbox.java | 28 ++-- .../overlord/TaskLockboxSyncResult.java | 2 +- .../overlord/GlobalTaskLockboxTest.java | 7 - 4 files changed, 68 insertions(+), 121 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/GlobalTaskLockbox.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/GlobalTaskLockbox.java index 041f5f1a8d08..21e1a7780f4a 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/GlobalTaskLockbox.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/GlobalTaskLockbox.java @@ -27,6 +27,7 @@ import org.apache.druid.indexing.common.actions.SegmentAllocateRequest; import org.apache.druid.indexing.common.actions.SegmentAllocateResult; import org.apache.druid.indexing.common.task.Task; +import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.metadata.LockFilterPolicy; @@ -34,7 +35,6 @@ import org.joda.time.DateTime; import org.joda.time.Interval; -import javax.annotation.Nullable; import java.util.ArrayList; import java.util.HashMap; import java.util.HashSet; @@ -45,20 +45,10 @@ import java.util.Set; import java.util.SortedMap; import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.locks.ReentrantLock; -import java.util.concurrent.locks.ReentrantReadWriteLock; +import java.util.concurrent.atomic.AtomicBoolean; /** - * Remembers which activeTasks have locked which intervals or which segments. Tasks are permitted to lock an interval - * or a segment if no other task outside their group has locked an overlapping interval for the same datasource or - * the same segments. Note that TaskLockbox is also responsible for allocating segmentIds when a task requests to lock - * a new segment. Task lock might involve version assignment. - * - * - When a task locks an interval or a new segment, it is assigned a new version string that it can use to publish - * segments. - * - When a task locks a existing segment, it doesn't need to be assigned a new version. - * - * Note that tasks of higher priorities can revoke locks of tasks of lower priorities. + * Maintains a {@link TaskLockbox} for each datasource. */ public class GlobalTaskLockbox { @@ -66,9 +56,10 @@ public class GlobalTaskLockbox private final TaskStorage taskStorage; private final IndexerMetadataStorageCoordinator metadataStorageCoordinator; - private final ReentrantReadWriteLock globalLock = new ReentrantReadWriteLock(true); private final ConcurrentHashMap datasourceLocks = new ConcurrentHashMap<>(); + private final AtomicBoolean syncComplete = new AtomicBoolean(false); + @Inject public GlobalTaskLockbox( TaskStorage taskStorage, @@ -86,42 +77,20 @@ private TaskLockbox getDatasourceLockbox(Task task) private TaskLockbox getDatasourceLockbox(String datasource) { - globalLock.readLock().lock(); - try { - return datasourceLocks.computeIfAbsent( - datasource, - ds -> new TaskLockbox(ds, new DatasourceLock(), taskStorage, metadataStorageCoordinator) - ); - } - finally { - globalLock.readLock().unlock(); - } - } - - private class DatasourceLock extends ReentrantLock - { - @Override - public void lock() - { - globalLock.readLock().lock(); - super.lock(); + // Verify that sync is complete + if (!syncComplete.get()) { + throw new ISE("Cannot get TaskLockbox for datasource[%s]. Sync with storage is not complete yet", datasource); } - @Override - public void lockInterruptibly() throws InterruptedException - { - globalLock.readLock().lock(); - super.lockInterruptibly(); - } - - @Override - public void unlock() - { - super.unlock(); - globalLock.readLock().unlock(); - } + return datasourceLocks.computeIfAbsent( + datasource, + ds -> new TaskLockbox(ds, taskStorage, metadataStorageCoordinator) + ); } + /** + * Result of metadata store sync for a single datasource. + */ private static class DatasourceSync { final Set storedActiveTasks = new HashSet<>(); @@ -129,65 +98,51 @@ private static class DatasourceSync } /** - * Wipe out our current in-memory state and resync it from our bundled {@link TaskStorage}. + * Syncs the current in-memory state with the {@link TaskStorage}. + * This method should be called only once when the {@link TaskQueue#start()} + * is invoked. * * @return SyncResult which needs to be processed by the caller */ public TaskLockboxSyncResult syncFromStorage() { - globalLock.writeLock().lock(); - - try { - // Load stuff from taskStorage first. If this fails, we don't want to lose all our locks. - final Map datasourceSyncs = new HashMap<>(); - int activeTaskCount = 0; - int totalLockCount = 0; - for (final Task task : taskStorage.getActiveTasks()) { - ++activeTaskCount; - final DatasourceSync sync = datasourceSyncs.computeIfAbsent(task.getDataSource(), ds -> new DatasourceSync()); - sync.storedActiveTasks.add(task); - for (final TaskLock taskLock : taskStorage.getLocks(task.getId())) { - ++totalLockCount; - sync.storedLocks.add(Pair.of(task, taskLock)); - } + // Load stuff from taskStorage first. If this fails, we don't want to lose all our locks. + final Map datasourceSyncs = new HashMap<>(); + int activeTaskCount = 0; + int totalLockCount = 0; + for (final Task task : taskStorage.getActiveTasks()) { + ++activeTaskCount; + final DatasourceSync sync = datasourceSyncs + .computeIfAbsent(task.getDataSource(), ds -> new DatasourceSync()); + sync.storedActiveTasks.add(task); + for (final TaskLock taskLock : taskStorage.getLocks(task.getId())) { + ++totalLockCount; + sync.storedLocks.add(Pair.of(task, taskLock)); } - - // Set of task groups in which at least one task failed to re-acquire a lock - final Set tasksToFail = new HashSet<>(); - int taskLockCount = 0; - - datasourceLocks.clear(); - for (String dataSource : datasourceSyncs.keySet()) { - final DatasourceSync sync = datasourceSyncs.get(dataSource); - final TaskLockboxSyncResult result = getDatasourceLockbox(dataSource) - .resetState(sync.storedActiveTasks, sync.storedLocks); - tasksToFail.addAll(result.getTasksToFail()); - taskLockCount += result.getTaskLockCount(); - } - - log.info( - "Synced [%,d] locks for [%,d] activeTasks from storage ([%,d] locks ignored).", - taskLockCount, - activeTaskCount, - totalLockCount - taskLockCount - ); - - return new TaskLockboxSyncResult(tasksToFail, taskLockCount); } - finally { - globalLock.writeLock().unlock(); + + // Set of task groups in which at least one task failed to re-acquire a lock + final Set tasksToFail = new HashSet<>(); + int taskLockCount = 0; + + datasourceLocks.clear(); + for (String dataSource : datasourceSyncs.keySet()) { + final DatasourceSync sync = datasourceSyncs.get(dataSource); + final TaskLockboxSyncResult result = getDatasourceLockbox(dataSource) + .resetState(sync.storedActiveTasks, sync.storedLocks); + tasksToFail.addAll(result.getTasksToFail()); + taskLockCount += result.getTaskLockCount(); } - } - /** - * This method is called only in {@link #syncFromStorage()} and verifies the given task and the taskLock have the same - * groupId, dataSource, and priority. - */ - @VisibleForTesting - @Nullable - TaskLockbox.TaskLockPosse verifyAndCreateOrFindLockPosse(Task task, TaskLock taskLock) - { - return getDatasourceLockbox(task).verifyAndCreateOrFindLockPosse(task, taskLock); + log.info( + "Synced [%,d] locks for [%,d] activeTasks from storage ([%,d] locks ignored).", + taskLockCount, + activeTaskCount, + totalLockCount - taskLockCount + ); + + syncComplete.set(true); + return new TaskLockboxSyncResult(tasksToFail, taskLockCount); } /** @@ -195,7 +150,6 @@ TaskLockbox.TaskLockPosse verifyAndCreateOrFindLockPosse(Task task, TaskLock tas * * @return {@link LockResult} containing a new or an existing lock if succeeded. Otherwise, {@link LockResult} with a * {@link LockResult#revoked} flag. - * * @throws InterruptedException if the current thread is interrupted */ public LockResult lock(final Task task, final LockRequest request) throws InterruptedException @@ -208,7 +162,6 @@ public LockResult lock(final Task task, final LockRequest request) throws Interr * * @return {@link LockResult} containing a new or an existing lock if succeeded. Otherwise, {@link LockResult} with a * {@link LockResult#revoked} flag. - * * @throws InterruptedException if the current thread is interrupted */ public LockResult lock(final Task task, final LockRequest request, long timeoutMs) throws InterruptedException @@ -222,7 +175,6 @@ public LockResult lock(final Task task, final LockRequest request, long timeoutM * * @return {@link LockResult} containing a new or an existing lock if succeeded. Otherwise, {@link LockResult} with a * {@link LockResult#revoked} flag. - * * @throws IllegalStateException if the task is not a valid active task */ public LockResult tryLock(final Task task, final LockRequest request) @@ -274,7 +226,7 @@ public List allocateSegments( * * @param task task performing a critical action * @param intervals intervals - * @param action action to be performed inside of the critical section + * @param action action to be performed inside the critical section */ public T doInCriticalSection(Task task, Set intervals, CriticalAction action) throws Exception { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskLockbox.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskLockbox.java index 6ecdd7457641..c77e55dba201 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskLockbox.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskLockbox.java @@ -98,8 +98,8 @@ public class TaskLockbox private final String dataSource; private final TaskStorage taskStorage; private final IndexerMetadataStorageCoordinator metadataStorageCoordinator; - private final ReentrantLock giant; - private final Condition lockReleaseCondition; + private final ReentrantLock giant = new ReentrantLock(); + private final Condition lockReleaseCondition = giant.newCondition(); private static final EmittingLogger log = new EmittingLogger(TaskLockbox.class); @@ -114,17 +114,15 @@ public class TaskLockbox * Used to clean up pending segments for a taskAllocatorId as soon as the set * of corresponding active taskIds becomes empty. */ + @GuardedBy("giant") private final Map> activeAllocatorIdToTaskIds = new HashMap<>(); public TaskLockbox( String dataSource, - ReentrantLock giant, TaskStorage taskStorage, IndexerMetadataStorageCoordinator metadataStorageCoordinator ) { - this.giant = giant; - this.lockReleaseCondition = giant.newCondition(); this.dataSource = dataSource; this.taskStorage = taskStorage; this.metadataStorageCoordinator = metadataStorageCoordinator; @@ -142,8 +140,8 @@ TaskLockboxSyncResult resetState( final List> storedLocks ) { - // No locks need to be acquired here since the GlobalTaskLockbox has already - // acquired the required locks + giant.lock(); + try { // Sort locks by version, so we add them back in the order they were acquired. final Ordering> byVersionOrdering = new Ordering<>() @@ -240,6 +238,9 @@ public int compare(Pair left, Pair right) log.noStackTrace().error(t, "Error while resetting state of datasource[%s]", dataSource); throw t; } + finally { + giant.unlock(); + } } /** @@ -247,8 +248,8 @@ public int compare(Pair left, Pair right) * * @return null if the lock could not be reacquired. */ - @VisibleForTesting @Nullable + @VisibleForTesting protected TaskLockPosse reacquireLockOnStartup(Task task, TaskLock taskLock) { if (!taskMatchesLock(task, taskLock)) { @@ -308,10 +309,10 @@ protected TaskLockPosse reacquireLockOnStartup(Task task, TaskLock taskLock) } /** - * Returns true if the datasource, groupId and priority of the given Task + * @return true if the datasource, groupId and priority of the given Task * match that of the TaskLock. */ - private boolean taskMatchesLock(Task task, TaskLock taskLock) + private static boolean taskMatchesLock(Task task, TaskLock taskLock) { return task.getGroupId().equals(taskLock.getGroupId()) && task.getDataSource().equals(taskLock.getDataSource()) @@ -1185,6 +1186,7 @@ public void add(Task task) } } + @GuardedBy("giant") private void trackAppendingTask(Task task) { if (task instanceof PendingSegmentAllocatingTask) { @@ -1411,8 +1413,8 @@ private boolean canAppendLockCoexist(List conflictPosses, LockReq } /** - * Check if a REPLACE lock can coexist with a given set of conflicting posses. - * A REPLACE lock can coexist with any number of other APPEND locks and revoked locks + * Check if a REPLACE-lock can coexist with a given set of conflicting posses. + * A REPLACE-lock can coexist with any number of other APPEND locks and revoked locks * @param conflictPosses conflicting lock posses * @param replaceLock replace lock request * @return true iff replace lock can coexist with all its conflicting locks @@ -1546,7 +1548,7 @@ private boolean revokeAllIncompatibleActiveLocksIfPossible( /** * Task locks for tasks of the same groupId */ - static class TaskLockPosse + protected static class TaskLockPosse { private final TaskLock taskLock; private final Set taskIds; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskLockboxSyncResult.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskLockboxSyncResult.java index 88d5f7c9d299..20b78e44650d 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskLockboxSyncResult.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskLockboxSyncResult.java @@ -27,7 +27,7 @@ * Result of TaskLockbox#syncFromStorage() * Contains tasks which need to be forcefully failed to let the overlord become the leader */ -class TaskLockboxSyncResult +public class TaskLockboxSyncResult { private final int taskLockCount; private final Set tasksToFail; diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/GlobalTaskLockboxTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/GlobalTaskLockboxTest.java index 1518cc060def..1d33bab437bc 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/GlobalTaskLockboxTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/GlobalTaskLockboxTest.java @@ -2283,12 +2283,5 @@ public NullLockPosseGlobalTaskLockbox( { super(taskStorage, metadataStorageCoordinator); } - - @Override - protected TaskLockbox.TaskLockPosse verifyAndCreateOrFindLockPosse(Task task, TaskLock taskLock) - { - return task.getGroupId().contains("FailingLockAcquisition") - ? null : super.verifyAndCreateOrFindLockPosse(task, taskLock); - } } } From 1206322ecc022c3442bd9abe2e569e20c6f8d756 Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Sat, 17 May 2025 14:20:14 +0530 Subject: [PATCH 06/14] Remove extra changes --- .../indexing/common/actions/TaskActionTestKit.java | 14 -------------- 1 file changed, 14 deletions(-) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/TaskActionTestKit.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/TaskActionTestKit.java index 9d48f647a010..fd3854cfddd0 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/TaskActionTestKit.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/TaskActionTestKit.java @@ -37,12 +37,9 @@ import org.apache.druid.metadata.SegmentsMetadataManagerConfig; import org.apache.druid.metadata.TestDerbyConnector; import org.apache.druid.metadata.segment.SqlSegmentMetadataTransactionFactory; -import org.apache.druid.metadata.segment.SqlSegmentsMetadataManagerV2; import org.apache.druid.metadata.segment.cache.HeapMemorySegmentMetadataCache; -import org.apache.druid.metadata.segment.cache.NoopSegmentMetadataCache; import org.apache.druid.metadata.segment.cache.SegmentMetadataCache; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; -import org.apache.druid.segment.metadata.SegmentSchemaCache; import org.apache.druid.segment.metadata.SegmentSchemaManager; import org.apache.druid.server.coordinator.simulate.BlockingExecutorService; import org.apache.druid.server.coordinator.simulate.TestDruidLeaderSelector; @@ -124,17 +121,6 @@ public void before() CentralizedDatasourceSchemaConfig.create() ); taskLockbox = new GlobalTaskLockbox(taskStorage, metadataStorageCoordinator); - segmentSchemaCache = new SegmentSchemaCache(); - segmentsMetadataManager = new SqlSegmentsMetadataManagerV2( - new NoopSegmentMetadataCache(), - new SegmentSchemaCache(), - testDerbyConnector, - Suppliers.ofInstance(new SegmentsMetadataManagerConfig(null, null)), - Suppliers.ofInstance(metadataStorageTablesConfig), - CentralizedDatasourceSchemaConfig.create(), - NoopServiceEmitter.instance(), - objectMapper - ); final TaskLockConfig taskLockConfig = new TaskLockConfig() { @Override From 60bd77743610acdf95e81ddeb319044aba95a0fc Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Tue, 3 Jun 2025 23:48:04 +0530 Subject: [PATCH 07/14] wip: temp changes --- .../indexing/overlord/GlobalTaskLockbox.java | 29 ++++++++++--------- 1 file changed, 15 insertions(+), 14 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/GlobalTaskLockbox.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/GlobalTaskLockbox.java index 21e1a7780f4a..91697bf66dfe 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/GlobalTaskLockbox.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/GlobalTaskLockbox.java @@ -56,7 +56,7 @@ public class GlobalTaskLockbox private final TaskStorage taskStorage; private final IndexerMetadataStorageCoordinator metadataStorageCoordinator; - private final ConcurrentHashMap datasourceLocks = new ConcurrentHashMap<>(); + private final ConcurrentHashMap datasourceToLockbox = new ConcurrentHashMap<>(); private final AtomicBoolean syncComplete = new AtomicBoolean(false); @@ -79,10 +79,10 @@ private TaskLockbox getDatasourceLockbox(String datasource) { // Verify that sync is complete if (!syncComplete.get()) { - throw new ISE("Cannot get TaskLockbox for datasource[%s]. Sync with storage is not complete yet", datasource); + throw new ISE("Cannot get TaskLockbox for datasource[%s]. Sync with storage is not complete yet.", datasource); } - return datasourceLocks.computeIfAbsent( + return datasourceToLockbox.computeIfAbsent( datasource, ds -> new TaskLockbox(ds, taskStorage, metadataStorageCoordinator) ); @@ -110,12 +110,14 @@ public TaskLockboxSyncResult syncFromStorage() final Map datasourceSyncs = new HashMap<>(); int activeTaskCount = 0; int totalLockCount = 0; - for (final Task task : taskStorage.getActiveTasks()) { + for (Task task : taskStorage.getActiveTasks()) { ++activeTaskCount; - final DatasourceSync sync = datasourceSyncs - .computeIfAbsent(task.getDataSource(), ds -> new DatasourceSync()); + final DatasourceSync sync = datasourceSyncs.computeIfAbsent( + task.getDataSource(), + ds -> new DatasourceSync() + ); sync.storedActiveTasks.add(task); - for (final TaskLock taskLock : taskStorage.getLocks(task.getId())) { + for (TaskLock taskLock : taskStorage.getLocks(task.getId())) { ++totalLockCount; sync.storedLocks.add(Pair.of(task, taskLock)); } @@ -125,7 +127,7 @@ public TaskLockboxSyncResult syncFromStorage() final Set tasksToFail = new HashSet<>(); int taskLockCount = 0; - datasourceLocks.clear(); + datasourceToLockbox.clear(); for (String dataSource : datasourceSyncs.keySet()) { final DatasourceSync sync = datasourceSyncs.get(dataSource); final TaskLockboxSyncResult result = getDatasourceLockbox(dataSource) @@ -136,9 +138,7 @@ public TaskLockboxSyncResult syncFromStorage() log.info( "Synced [%,d] locks for [%,d] activeTasks from storage ([%,d] locks ignored).", - taskLockCount, - activeTaskCount, - totalLockCount - taskLockCount + taskLockCount, activeTaskCount, totalLockCount - taskLockCount ); syncComplete.set(true); @@ -337,7 +337,8 @@ public void add(Task task) } /** - * Release all locks for a task and remove task from set of active tasks. Does nothing if the task is not currently locked or not an active task. + * Release all locks for a task and remove task from set of active tasks. + * Does nothing if the task is not currently locked or not an active task. * * @param task task to unlock */ @@ -356,7 +357,7 @@ Optional getOnlyTaskLockPosseContainingInterval(Task Set getActiveTasks() { final Set allActiveTasks = new HashSet<>(); - datasourceLocks.values().forEach(lockbox -> allActiveTasks.addAll(lockbox.getActiveTasks())); + datasourceToLockbox.values().forEach(lockbox -> allActiveTasks.addAll(lockbox.getActiveTasks())); return allActiveTasks; } @@ -367,7 +368,7 @@ Map>>> allLocks = new HashMap<>(); - datasourceLocks.forEach((datasource, lockbox) -> { + datasourceToLockbox.forEach((datasource, lockbox) -> { final NavigableMap>> locks = lockbox.getAllLocks(); if (!locks.isEmpty()) { From f2ee06aafb6a2ecc239f4322f15cf63baec80020 Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Wed, 4 Jun 2025 20:17:53 +0530 Subject: [PATCH 08/14] Fix for upstream changes --- .../org/apache/druid/guice/ClusterTestingModule.java | 4 ++-- .../testing/cluster/overlord/FaultyTaskLockbox.java | 3 ++- .../org/apache/druid/guice/ClusterTestingModuleTest.java | 4 ++-- .../druid/indexing/overlord/GlobalTaskLockbox.java | 9 +++++++++ 4 files changed, 15 insertions(+), 5 deletions(-) diff --git a/extensions-core/testing-tools/src/main/java/org/apache/druid/guice/ClusterTestingModule.java b/extensions-core/testing-tools/src/main/java/org/apache/druid/guice/ClusterTestingModule.java index c1fa71a5f319..4c32caf1f6bb 100644 --- a/extensions-core/testing-tools/src/main/java/org/apache/druid/guice/ClusterTestingModule.java +++ b/extensions-core/testing-tools/src/main/java/org/apache/druid/guice/ClusterTestingModule.java @@ -31,7 +31,7 @@ import org.apache.druid.guice.annotations.Self; import org.apache.druid.indexing.common.actions.RemoteTaskActionClientFactory; import org.apache.druid.indexing.common.task.Task; -import org.apache.druid.indexing.overlord.TaskLockbox; +import org.apache.druid.indexing.overlord.GlobalTaskLockbox; import org.apache.druid.initialization.DruidModule; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.rpc.indexing.OverlordClient; @@ -106,7 +106,7 @@ private void bindDependenciesForClusterTestingMode(Binder binder) } else if (roles.contains(NodeRole.OVERLORD)) { // If this is the Overlord, bind a faulty storage coordinator log.warn("Running Overlord in cluster testing mode."); - binder.bind(TaskLockbox.class) + binder.bind(GlobalTaskLockbox.class) .to(FaultyTaskLockbox.class) .in(LazySingleton.class); } diff --git a/extensions-core/testing-tools/src/main/java/org/apache/druid/testing/cluster/overlord/FaultyTaskLockbox.java b/extensions-core/testing-tools/src/main/java/org/apache/druid/testing/cluster/overlord/FaultyTaskLockbox.java index d2639dcfbbab..67ce9e829ae6 100644 --- a/extensions-core/testing-tools/src/main/java/org/apache/druid/testing/cluster/overlord/FaultyTaskLockbox.java +++ b/extensions-core/testing-tools/src/main/java/org/apache/druid/testing/cluster/overlord/FaultyTaskLockbox.java @@ -22,6 +22,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.inject.Inject; import org.apache.druid.indexing.common.task.Task; +import org.apache.druid.indexing.overlord.GlobalTaskLockbox; import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator; import org.apache.druid.indexing.overlord.TaskLockbox; import org.apache.druid.indexing.overlord.TaskStorage; @@ -34,7 +35,7 @@ *

  • Skip cleanup of pending segments
  • * */ -public class FaultyTaskLockbox extends TaskLockbox +public class FaultyTaskLockbox extends GlobalTaskLockbox { private static final Logger log = new Logger(FaultyTaskLockbox.class); diff --git a/extensions-core/testing-tools/src/test/java/org/apache/druid/guice/ClusterTestingModuleTest.java b/extensions-core/testing-tools/src/test/java/org/apache/druid/guice/ClusterTestingModuleTest.java index 39ff06b60350..569ad33e3ce3 100644 --- a/extensions-core/testing-tools/src/test/java/org/apache/druid/guice/ClusterTestingModuleTest.java +++ b/extensions-core/testing-tools/src/test/java/org/apache/druid/guice/ClusterTestingModuleTest.java @@ -40,7 +40,7 @@ import org.apache.druid.indexing.common.task.batch.parallel.ParallelIndexSupervisorTask; import org.apache.druid.indexing.common.task.batch.parallel.ParallelIndexTuningConfig; import org.apache.druid.indexing.input.DruidInputSource; -import org.apache.druid.indexing.overlord.TaskLockbox; +import org.apache.druid.indexing.overlord.GlobalTaskLockbox; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.query.filter.TrueDimFilter; import org.apache.druid.rpc.indexing.OverlordClient; @@ -244,7 +244,7 @@ public void test_overlordService_hasFaultyStorageCoordinator_ifTestingIsEnabled( final Injector overlordInjector = overlord.makeInjector(Set.of(NodeRole.OVERLORD)); - TaskLockbox taskLockbox = overlordInjector.getInstance(TaskLockbox.class); + GlobalTaskLockbox taskLockbox = overlordInjector.getInstance(GlobalTaskLockbox.class); Assert.assertTrue(taskLockbox instanceof FaultyTaskLockbox); } finally { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/GlobalTaskLockbox.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/GlobalTaskLockbox.java index 91697bf66dfe..93121c748cb1 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/GlobalTaskLockbox.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/GlobalTaskLockbox.java @@ -248,6 +248,15 @@ public void revokeLock(String taskId, TaskLock lock) getDatasourceLockbox(lock.getDataSource()).revokeLock(taskId, lock); } + /** + * Cleans up pending segments associated with the given task, if any. + */ + @VisibleForTesting + protected void cleanupPendingSegments(Task task) + { + getDatasourceLockbox(task.getDataSource()).cleanupPendingSegments(task); + } + /** * Return the currently-active locks for some task. * From a26b06f00cc495cdf2f0f92bcb159d4b9d5e8405 Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Thu, 5 Jun 2025 10:37:17 +0530 Subject: [PATCH 09/14] Some improvements --- .../indexing/overlord/GlobalTaskLockbox.java | 279 ++++++++++++++---- .../druid/indexing/overlord/TaskLockbox.java | 3 +- .../common/actions/TaskActionTestKit.java | 1 + .../common/task/IngestionTestBase.java | 1 + .../overlord/GlobalTaskLockboxTest.java | 10 + 5 files changed, 235 insertions(+), 59 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/GlobalTaskLockbox.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/GlobalTaskLockbox.java index 93121c748cb1..c02aa3230981 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/GlobalTaskLockbox.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/GlobalTaskLockbox.java @@ -41,14 +41,19 @@ import java.util.List; import java.util.Map; import java.util.NavigableMap; +import java.util.Objects; import java.util.Optional; import java.util.Set; import java.util.SortedMap; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; /** * Maintains a {@link TaskLockbox} for each datasource. + * A {@link TaskLockbox} is used to maintain locks over intervals of a datasource + * to ensure data consistency while various types of ingestion tasks append, + * overwrite or delete data. */ public class GlobalTaskLockbox { @@ -56,7 +61,7 @@ public class GlobalTaskLockbox private final TaskStorage taskStorage; private final IndexerMetadataStorageCoordinator metadataStorageCoordinator; - private final ConcurrentHashMap datasourceToLockbox = new ConcurrentHashMap<>(); + private final ConcurrentHashMap datasourceToLockbox = new ConcurrentHashMap<>(); private final AtomicBoolean syncComplete = new AtomicBoolean(false); @@ -70,79 +75,176 @@ public GlobalTaskLockbox( this.metadataStorageCoordinator = metadataStorageCoordinator; } - private TaskLockbox getDatasourceLockbox(Task task) + /** + * Gets the {@link DatasourceLockboxResource} for the given datasource and + * increments the number of references currently in use. + * This resource must be closed once it is not needed anymore. + */ + private DatasourceLockboxResource getLockboxResource(String datasource) { - return getDatasourceLockbox(task.getDataSource()); + return datasourceToLockbox.compute( + datasource, + (ds, existingResource) -> { + final DatasourceLockboxResource resource = Objects.requireNonNullElseGet( + existingResource, + () -> new DatasourceLockboxResource( + new TaskLockbox(ds, taskStorage, metadataStorageCoordinator) + ) + ); + resource.acquireReference(); + return resource; + } + ); } - private TaskLockbox getDatasourceLockbox(String datasource) + /** + * Performs a computation using the {@link TaskLockbox} corresponding to the + * given datasource and returns the result. + */ + private R computeForDatasource( + String datasource, + LockComputation computation + ) throws T { // Verify that sync is complete if (!syncComplete.get()) { - throw new ISE("Cannot get TaskLockbox for datasource[%s]. Sync with storage is not complete yet.", datasource); + throw new ISE( + "Cannot get TaskLockbox for datasource[%s] as sync with storage has not happened.", + datasource + ); } - return datasourceToLockbox.computeIfAbsent( - datasource, - ds -> new TaskLockbox(ds, taskStorage, metadataStorageCoordinator) + try (final DatasourceLockboxResource lockbox = getLockboxResource(datasource)) { + return computation.perform(lockbox.delegate); + } + } + + /** + * Performs a computation using the {@link TaskLockbox} corresponding to the + * given task and returns the result. + */ + private R computeForTask( + Task task, + LockComputation computation + ) throws T + { + return computeForDatasource(task.getDataSource(), computation); + } + + /** + * Executes an operation on the {@link TaskLockbox} corresponding to the given + * task. + */ + private void executeForTask( + Task task, + LockOperation operation + ) throws T + { + computeForDatasource( + task.getDataSource(), + lockbox -> { + operation.perform(lockbox); + return 0; + } ); } + @FunctionalInterface + private interface LockComputation + { + R perform(TaskLockbox lockbox) throws T; + } + + @FunctionalInterface + private interface LockOperation + { + void perform(TaskLockbox lockbox) throws T; + } + /** * Result of metadata store sync for a single datasource. */ - private static class DatasourceSync + private static class DatasourceSyncResult { final Set storedActiveTasks = new HashSet<>(); final List> storedLocks = new ArrayList<>(); } + private static class DatasourceLockboxResource implements AutoCloseable + { + final AtomicInteger references; + final TaskLockbox delegate; + + DatasourceLockboxResource(TaskLockbox delegate) + { + this.delegate = delegate; + this.references = new AtomicInteger(0); + } + + void acquireReference() + { + references.incrementAndGet(); + } + + @Override + public void close() + { + references.decrementAndGet(); + } + } + /** * Syncs the current in-memory state with the {@link TaskStorage}. - * This method should be called only once when the {@link TaskQueue#start()} - * is invoked. + * This method should be called only from {@link TaskQueue#start()}. + * If the sync fails, no other operation can be performed on this lockbox. * * @return SyncResult which needs to be processed by the caller */ public TaskLockboxSyncResult syncFromStorage() { - // Load stuff from taskStorage first. If this fails, we don't want to lose all our locks. - final Map datasourceSyncs = new HashMap<>(); + // TODO: While sync from storage is in progress, should anything else be allowed + // Can sync ever be called in parallel with anything else? + + // Retrieve all active tasks and locks associated with them + final Map datasourceToSyncResult = new HashMap<>(); int activeTaskCount = 0; int totalLockCount = 0; for (Task task : taskStorage.getActiveTasks()) { ++activeTaskCount; - final DatasourceSync sync = datasourceSyncs.computeIfAbsent( + final DatasourceSyncResult result = datasourceToSyncResult.computeIfAbsent( task.getDataSource(), - ds -> new DatasourceSync() + ds -> new DatasourceSyncResult() ); - sync.storedActiveTasks.add(task); + result.storedActiveTasks.add(task); for (TaskLock taskLock : taskStorage.getLocks(task.getId())) { ++totalLockCount; - sync.storedLocks.add(Pair.of(task, taskLock)); + result.storedLocks.add(Pair.of(task, taskLock)); } } - // Set of task groups in which at least one task failed to re-acquire a lock + // Identify task groups in which at least one task failed to re-acquire a lock final Set tasksToFail = new HashSet<>(); - int taskLockCount = 0; + final AtomicInteger taskLockCount = new AtomicInteger(0); datasourceToLockbox.clear(); - for (String dataSource : datasourceSyncs.keySet()) { - final DatasourceSync sync = datasourceSyncs.get(dataSource); - final TaskLockboxSyncResult result = getDatasourceLockbox(dataSource) - .resetState(sync.storedActiveTasks, sync.storedLocks); - tasksToFail.addAll(result.getTasksToFail()); - taskLockCount += result.getTaskLockCount(); - } + datasourceToSyncResult.forEach((dataSource, syncResult) -> { + try (final DatasourceLockboxResource lockboxResource = getLockboxResource(dataSource)) { + final TaskLockboxSyncResult lockboxSyncResult = lockboxResource.delegate.resetState( + syncResult.storedActiveTasks, + syncResult.storedLocks + ); + tasksToFail.addAll(lockboxSyncResult.getTasksToFail()); + taskLockCount.addAndGet(lockboxSyncResult.getTaskLockCount()); + } + }); log.info( - "Synced [%,d] locks for [%,d] activeTasks from storage ([%,d] locks ignored).", - taskLockCount, activeTaskCount, totalLockCount - taskLockCount + "Synced [%,d] locks for [%,d] active tasks from storage ([%,d] locks ignored).", + taskLockCount.get(), activeTaskCount, totalLockCount - taskLockCount.get() ); syncComplete.set(true); - return new TaskLockboxSyncResult(tasksToFail, taskLockCount); + return new TaskLockboxSyncResult(tasksToFail, taskLockCount.get()); } /** @@ -154,7 +256,10 @@ public TaskLockboxSyncResult syncFromStorage() */ public LockResult lock(final Task task, final LockRequest request) throws InterruptedException { - return getDatasourceLockbox(task).lock(task, request); + return computeForTask( + task, + lockbox -> lockbox.lock(task, request) + ); } /** @@ -166,7 +271,10 @@ public LockResult lock(final Task task, final LockRequest request) throws Interr */ public LockResult lock(final Task task, final LockRequest request, long timeoutMs) throws InterruptedException { - return getDatasourceLockbox(task).lock(task, request, timeoutMs); + return computeForTask( + task, + lockbox -> lockbox.lock(task, request, timeoutMs) + ); } /** @@ -179,7 +287,10 @@ public LockResult lock(final Task task, final LockRequest request, long timeoutM */ public LockResult tryLock(final Task task, final LockRequest request) { - return getDatasourceLockbox(task).tryLock(task, request); + return computeForTask( + task, + lockbox -> lockbox.tryLock(task, request) + ); } /** @@ -207,13 +318,16 @@ public List allocateSegments( boolean reduceMetadataIO ) { - return getDatasourceLockbox(dataSource).allocateSegments( - requests, + return computeForDatasource( dataSource, - interval, - skipSegmentLineageCheck, - lockGranularity, - reduceMetadataIO + lockbox -> lockbox.allocateSegments( + requests, + dataSource, + interval, + skipSegmentLineageCheck, + lockGranularity, + reduceMetadataIO + ) ); } @@ -230,7 +344,10 @@ public List allocateSegments( */ public T doInCriticalSection(Task task, Set intervals, CriticalAction action) throws Exception { - return getDatasourceLockbox(task).doInCriticalSection(task, intervals, action); + return computeForTask( + task, + lockbox -> lockbox.doInCriticalSection(task, intervals, action) + ); } /** @@ -245,7 +362,13 @@ public T doInCriticalSection(Task task, Set intervals, CriticalAct @VisibleForTesting public void revokeLock(String taskId, TaskLock lock) { - getDatasourceLockbox(lock.getDataSource()).revokeLock(taskId, lock); + computeForDatasource( + lock.getDataSource(), + lockbox -> { + lockbox.revokeLock(taskId, lock); + return 0; + } + ); } /** @@ -254,7 +377,10 @@ public void revokeLock(String taskId, TaskLock lock) @VisibleForTesting protected void cleanupPendingSegments(Task task) { - getDatasourceLockbox(task.getDataSource()).cleanupPendingSegments(task); + executeForTask( + task, + lockbox -> lockbox.cleanupPendingSegments(task) + ); } /** @@ -265,7 +391,10 @@ protected void cleanupPendingSegments(Task task) */ public List findLocksForTask(final Task task) { - return getDatasourceLockbox(task).findLocksForTask(task); + return computeForTask( + task, + lockbox -> lockbox.findLocksForTask(task) + ); } /** @@ -273,7 +402,10 @@ public List findLocksForTask(final Task task) */ public Set findReplaceLocksForTask(Task task) { - return getDatasourceLockbox(task).findReplaceLocksForTask(task); + return computeForTask( + task, + lockbox -> lockbox.findReplaceLocksForTask(task) + ); } /** @@ -281,7 +413,10 @@ public Set findReplaceLocksForTask(Task task) */ public Set getAllReplaceLocksForDatasource(String datasource) { - return getDatasourceLockbox(datasource).getAllReplaceLocksForDatasource(datasource); + return computeForDatasource( + datasource, + lockbox -> lockbox.getAllReplaceLocksForDatasource(datasource) + ); } /** @@ -298,7 +433,10 @@ public Map> getLockedIntervals(List loc final Map> datasourceToLockedIntervals = new HashMap<>(); datasourceToFilterPolicies.forEach((datasource, policies) -> { - final List lockedIntervals = getDatasourceLockbox(datasource).getLockedIntervals(policies); + final List lockedIntervals = computeForDatasource( + datasource, + lockbox -> lockbox.getLockedIntervals(policies) + ); if (!lockedIntervals.isEmpty()) { datasourceToLockedIntervals.put(datasource, lockedIntervals); } @@ -321,7 +459,10 @@ public Map> getActiveLocks(List lockFil final Map> datasourceToActiveLocks = new HashMap<>(); datasourceToFilterPolicies.forEach((datasource, policies) -> { - final List datasourceLocks = getDatasourceLockbox(datasource).getActiveLocks(policies); + final List datasourceLocks = computeForDatasource( + datasource, + lockbox -> lockbox.getActiveLocks(policies) + ); if (!datasourceLocks.isEmpty()) { datasourceToActiveLocks.put(datasource, datasourceLocks); } @@ -332,17 +473,26 @@ public Map> getActiveLocks(List lockFil public void unlock(final Task task, final Interval interval) { - getDatasourceLockbox(task).unlock(task, interval); + executeForTask( + task, + lockbox -> lockbox.unlock(task, interval) + ); } public void unlockAll(Task task) { - getDatasourceLockbox(task).unlockAll(task); + executeForTask( + task, + lockbox -> lockbox.unlockAll(task) + ); } public void add(Task task) { - getDatasourceLockbox(task).add(task); + executeForTask( + task, + lockbox -> lockbox.add(task) + ); } /** @@ -353,20 +503,34 @@ public void add(Task task) */ public void remove(final Task task) { - getDatasourceLockbox(task).remove(task); + executeForTask( + task, + lockbox -> lockbox.remove(task) + ); + + // TODO: Clean up the datasource lockbox if it is now empty } @VisibleForTesting Optional getOnlyTaskLockPosseContainingInterval(Task task, Interval interval) { - return getDatasourceLockbox(task).getOnlyTaskLockPosseContainingInterval(task, interval); + return computeForTask( + task, + lockbox -> lockbox.getOnlyTaskLockPosseContainingInterval(task, interval) + ); } @VisibleForTesting Set getActiveTasks() { final Set allActiveTasks = new HashSet<>(); - datasourceToLockbox.values().forEach(lockbox -> allActiveTasks.addAll(lockbox.getActiveTasks())); + + final Set datasourceNames = Set.copyOf(datasourceToLockbox.keySet()); + for (String datasource : datasourceNames) { + allActiveTasks.addAll( + computeForDatasource(datasource, TaskLockbox::getActiveTasks) + ); + } return allActiveTasks; } @@ -377,13 +541,14 @@ Map>>> allLocks = new HashMap<>(); - datasourceToLockbox.forEach((datasource, lockbox) -> { + final Set datasourceNames = Set.copyOf(datasourceToLockbox.keySet()); + for (String datasource : datasourceNames) { final NavigableMap>> - locks = lockbox.getAllLocks(); - if (!locks.isEmpty()) { - allLocks.put(datasource, locks); + datasourceLocks = computeForDatasource(datasource, TaskLockbox::getAllLocks); + if (!datasourceLocks.isEmpty()) { + allLocks.put(datasource, datasourceLocks); } - }); + } return allLocks; } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskLockbox.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskLockbox.java index 44fd998cb929..962a54317e08 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskLockbox.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskLockbox.java @@ -130,8 +130,7 @@ public TaskLockbox( /** * Wipe out our current in-memory state and resync it from our bundled {@link TaskStorage}. - * This method must be called only by the GlobalTaskLockbox after acquiring - * a WRITE globalLock. + * This method must be called only from {@link GlobalTaskLockbox#syncFromStorage()}. * * @return SyncResult which needs to be processed by the caller */ diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/TaskActionTestKit.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/TaskActionTestKit.java index 3bff73925800..76e207521a72 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/TaskActionTestKit.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/TaskActionTestKit.java @@ -137,6 +137,7 @@ public void before() CentralizedDatasourceSchemaConfig.create() ); taskLockbox = new GlobalTaskLockbox(taskStorage, metadataStorageCoordinator); + taskLockbox.syncFromStorage(); final TaskLockConfig taskLockConfig = new TaskLockConfig() { @Override diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IngestionTestBase.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IngestionTestBase.java index 3615f94239e8..e51a7017a67b 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IngestionTestBase.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IngestionTestBase.java @@ -188,6 +188,7 @@ public void setUpIngestionTestBase() throws IOException objectMapper ); lockbox = new GlobalTaskLockbox(taskStorage, storageCoordinator); + lockbox.syncFromStorage(); segmentCacheManagerFactory = new SegmentCacheManagerFactory(TestIndex.INDEX_IO, getObjectMapper()); reportsFile = temporaryFolder.newFile(); dataSegmentKiller = new TestDataSegmentKiller(); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/GlobalTaskLockboxTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/GlobalTaskLockboxTest.java index 7d96992614f0..1401a8a3bbdb 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/GlobalTaskLockboxTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/GlobalTaskLockboxTest.java @@ -147,6 +147,8 @@ public void setup() ); lockbox = new GlobalTaskLockbox(taskStorage, metadataStorageCoordinator); + lockbox.syncFromStorage(); + validator = new TaskLockboxValidator(lockbox, taskStorage); } @@ -343,6 +345,8 @@ public void testTimeoutForLock() throws InterruptedException public void testSyncFromStorage() { final GlobalTaskLockbox originalBox = new GlobalTaskLockbox(taskStorage, metadataStorageCoordinator); + originalBox.syncFromStorage(); + for (int i = 0; i < 5; i++) { final Task task = NoopTask.create(); taskStorage.insert(task, TaskStatus.running(task.getId())); @@ -489,7 +493,9 @@ public void testSyncWithUnknownTaskTypesFromModuleNotLoaded() ); GlobalTaskLockbox theBox = new GlobalTaskLockbox(taskStorage, metadataStorageCoordinator); + theBox.syncFromStorage(); GlobalTaskLockbox loadedBox = new GlobalTaskLockbox(loadedTaskStorage, loadedMetadataStorageCoordinator); + loadedBox.syncFromStorage(); Task aTask = NoopTask.create(); taskStorage.insert(aTask, TaskStatus.running(aTask.getId())); @@ -515,6 +521,7 @@ public void testSyncWithUnknownTaskTypesFromModuleNotLoaded() public void testRevokedLockSyncFromStorage() { final GlobalTaskLockbox originalBox = new GlobalTaskLockbox(taskStorage, metadataStorageCoordinator); + originalBox.syncFromStorage(); final Task task1 = NoopTask.ofPriority(10); taskStorage.insert(task1, TaskStatus.running(task1.getId())); @@ -1876,6 +1883,8 @@ public void testFailedToReacquireTaskLock() taskStorage.insert(taskWithSuccessfulLockAcquisition, TaskStatus.running(taskWithSuccessfulLockAcquisition.getId())); GlobalTaskLockbox testLockbox = new NullLockPosseGlobalTaskLockbox(taskStorage, metadataStorageCoordinator); + testLockbox.syncFromStorage(); + testLockbox.add(taskWithFailingLockAcquisition0); testLockbox.add(taskWithFailingLockAcquisition1); testLockbox.add(taskWithSuccessfulLockAcquisition); @@ -2031,6 +2040,7 @@ public void testCleanupOnUnlock() EasyMock.replay(coordinator); final GlobalTaskLockbox taskLockbox = new GlobalTaskLockbox(taskStorage, coordinator); + taskLockbox.syncFromStorage(); taskLockbox.add(replaceTask); taskLockbox.tryLock( From 794066ada7551665eeee85662cd4c5d3588597d9 Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Thu, 5 Jun 2025 20:38:07 +0530 Subject: [PATCH 10/14] Fix up tests --- .../indexing/overlord/GlobalTaskLockbox.java | 302 +++++++++++------- .../druid/indexing/overlord/TaskLockbox.java | 24 ++ .../druid/indexing/overlord/TaskQueue.java | 5 +- .../common/actions/TaskLocksTest.java | 1 + .../OverlordCompactionSchedulerTest.java | 1 + .../overlord/TaskLockBoxConcurrencyTest.java | 4 + 6 files changed, 211 insertions(+), 126 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/GlobalTaskLockbox.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/GlobalTaskLockbox.java index c02aa3230981..445caca6dcc4 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/GlobalTaskLockbox.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/GlobalTaskLockbox.java @@ -48,6 +48,7 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.Predicate; /** * Maintains a {@link TaskLockbox} for each datasource. @@ -75,124 +76,6 @@ public GlobalTaskLockbox( this.metadataStorageCoordinator = metadataStorageCoordinator; } - /** - * Gets the {@link DatasourceLockboxResource} for the given datasource and - * increments the number of references currently in use. - * This resource must be closed once it is not needed anymore. - */ - private DatasourceLockboxResource getLockboxResource(String datasource) - { - return datasourceToLockbox.compute( - datasource, - (ds, existingResource) -> { - final DatasourceLockboxResource resource = Objects.requireNonNullElseGet( - existingResource, - () -> new DatasourceLockboxResource( - new TaskLockbox(ds, taskStorage, metadataStorageCoordinator) - ) - ); - resource.acquireReference(); - return resource; - } - ); - } - - /** - * Performs a computation using the {@link TaskLockbox} corresponding to the - * given datasource and returns the result. - */ - private R computeForDatasource( - String datasource, - LockComputation computation - ) throws T - { - // Verify that sync is complete - if (!syncComplete.get()) { - throw new ISE( - "Cannot get TaskLockbox for datasource[%s] as sync with storage has not happened.", - datasource - ); - } - - try (final DatasourceLockboxResource lockbox = getLockboxResource(datasource)) { - return computation.perform(lockbox.delegate); - } - } - - /** - * Performs a computation using the {@link TaskLockbox} corresponding to the - * given task and returns the result. - */ - private R computeForTask( - Task task, - LockComputation computation - ) throws T - { - return computeForDatasource(task.getDataSource(), computation); - } - - /** - * Executes an operation on the {@link TaskLockbox} corresponding to the given - * task. - */ - private void executeForTask( - Task task, - LockOperation operation - ) throws T - { - computeForDatasource( - task.getDataSource(), - lockbox -> { - operation.perform(lockbox); - return 0; - } - ); - } - - @FunctionalInterface - private interface LockComputation - { - R perform(TaskLockbox lockbox) throws T; - } - - @FunctionalInterface - private interface LockOperation - { - void perform(TaskLockbox lockbox) throws T; - } - - /** - * Result of metadata store sync for a single datasource. - */ - private static class DatasourceSyncResult - { - final Set storedActiveTasks = new HashSet<>(); - final List> storedLocks = new ArrayList<>(); - } - - private static class DatasourceLockboxResource implements AutoCloseable - { - final AtomicInteger references; - final TaskLockbox delegate; - - DatasourceLockboxResource(TaskLockbox delegate) - { - this.delegate = delegate; - this.references = new AtomicInteger(0); - } - - void acquireReference() - { - references.incrementAndGet(); - } - - @Override - public void close() - { - references.decrementAndGet(); - } - } - /** * Syncs the current in-memory state with the {@link TaskStorage}. * This method should be called only from {@link TaskQueue#start()}. @@ -202,8 +85,9 @@ public void close() */ public TaskLockboxSyncResult syncFromStorage() { - // TODO: While sync from storage is in progress, should anything else be allowed - // Can sync ever be called in parallel with anything else? + // Shutdown to reset the state and ensure that no other operation is + // in progress during the sync + shutdown(); // Retrieve all active tasks and locks associated with them final Map datasourceToSyncResult = new HashMap<>(); @@ -226,7 +110,6 @@ public TaskLockboxSyncResult syncFromStorage() final Set tasksToFail = new HashSet<>(); final AtomicInteger taskLockCount = new AtomicInteger(0); - datasourceToLockbox.clear(); datasourceToSyncResult.forEach((dataSource, syncResult) -> { try (final DatasourceLockboxResource lockboxResource = getLockboxResource(dataSource)) { final TaskLockboxSyncResult lockboxSyncResult = lockboxResource.delegate.resetState( @@ -247,6 +130,26 @@ public TaskLockboxSyncResult syncFromStorage() return new TaskLockboxSyncResult(tasksToFail, taskLockCount.get()); } + /** + * Clears up the state of the lockbox. Should be called when leadership is lost. + */ + public void shutdown() + { + // Mark sync as incomplete so that no more lockboxes are created + syncComplete.set(false); + + // Clean up all existing lockboxes + final Set datasourceNames = Set.copyOf(datasourceToLockbox.keySet()); + for (String dataSource : datasourceNames) { + cleanupLockboxResourceIf(dataSource, resource -> true); + } + + log.info( + "Removed lockboxes for [%d] datasources, [%d] remaining.", + datasourceNames.size(), datasourceToLockbox.size() + ); + } + /** * Acquires a lock on behalf of a task. Blocks until the lock is acquired. * @@ -503,12 +406,17 @@ public void add(Task task) */ public void remove(final Task task) { - executeForTask( + final boolean isEmpty = computeForTask( task, - lockbox -> lockbox.remove(task) + lockbox -> { + lockbox.remove(task); + return lockbox.isEmpty(); + } ); - // TODO: Clean up the datasource lockbox if it is now empty + if (isEmpty) { + cleanupLockboxResourceIf(task.getDataSource(), resource -> resource.references.get() <= 0); + } } @VisibleForTesting @@ -553,4 +461,150 @@ Map { + final DatasourceLockboxResource resource = Objects.requireNonNullElseGet( + existingResource, + () -> new DatasourceLockboxResource( + new TaskLockbox(ds, taskStorage, metadataStorageCoordinator) + ) + ); + resource.acquireReference(); + return resource; + } + ); + } + + /** + * Cleans up the lockbox for the given datasource if the {@link DatasourceLockboxResource} + * meets the given criteria. + */ + private void cleanupLockboxResourceIf( + String dataSource, + Predicate resourcePredicate + ) + { + datasourceToLockbox.compute( + dataSource, + (ds, resource) -> { + if (resource != null && resourcePredicate.test(resource)) { + // TODO: what if a bad runaway operation is holding the TaskLockbox.giant? + resource.delegate.clear(); + return null; + } else { + return resource; + } + } + ); + } + + /** + * Performs a computation using the {@link TaskLockbox} corresponding to the + * given datasource and returns the result. + */ + private R computeForDatasource( + String datasource, + LockComputation computation + ) throws T + { + // Verify that sync is complete + if (!syncComplete.get()) { + throw new ISE( + "Cannot get TaskLockbox for datasource[%s] as sync with storage has not happened yet.", + datasource + ); + } + + try (final DatasourceLockboxResource lockbox = getLockboxResource(datasource)) { + return computation.perform(lockbox.delegate); + } + } + + /** + * Performs a computation using the {@link TaskLockbox} corresponding to the + * given task and returns the result. + */ + private R computeForTask( + Task task, + LockComputation computation + ) throws T + { + return computeForDatasource(task.getDataSource(), computation); + } + + /** + * Executes an operation on the {@link TaskLockbox} corresponding to the given + * task. + */ + private void executeForTask( + Task task, + LockOperation operation + ) throws T + { + computeForDatasource( + task.getDataSource(), + lockbox -> { + operation.perform(lockbox); + return 0; + } + ); + } + + @FunctionalInterface + private interface LockComputation + { + R perform(TaskLockbox lockbox) throws T; + } + + @FunctionalInterface + private interface LockOperation + { + void perform(TaskLockbox lockbox) throws T; + } + + /** + * Result of metadata store sync for a single datasource. + */ + private static class DatasourceSyncResult + { + final Set storedActiveTasks = new HashSet<>(); + final List> storedLocks = new ArrayList<>(); + } + + /** + * Wrapper around a {@link TaskLockbox} for a specific datasource which keeps + * track of the number of active references of this resource that are currently + * in use. + */ + private static class DatasourceLockboxResource implements AutoCloseable + { + final AtomicInteger references; + final TaskLockbox delegate; + + DatasourceLockboxResource(TaskLockbox delegate) + { + this.delegate = delegate; + this.references = new AtomicInteger(0); + } + + void acquireReference() + { + references.incrementAndGet(); + } + + @Override + public void close() + { + references.decrementAndGet(); + } + } + } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskLockbox.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskLockbox.java index 962a54317e08..fd17cf0e8870 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskLockbox.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskLockbox.java @@ -128,6 +128,30 @@ public TaskLockbox( this.metadataStorageCoordinator = metadataStorageCoordinator; } + void clear() + { + giant.lock(); + try { + running.clear(); + activeTasks.clear(); + activeAllocatorIdToTaskIds.clear(); + } + finally { + giant.unlock(); + } + } + + boolean isEmpty() + { + giant.lock(); + try { + return activeTasks.isEmpty() && running.isEmpty() && activeAllocatorIdToTaskIds.isEmpty(); + } + finally { + giant.unlock(); + } + } + /** * Wipe out our current in-memory state and resync it from our bundled {@link TaskStorage}. * This method must be called only from {@link GlobalTaskLockbox#syncFromStorage()}. diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskQueue.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskQueue.java index ae0f0bc87d6d..c3e8e56bcd6c 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskQueue.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskQueue.java @@ -218,10 +218,10 @@ public void start() try { Preconditions.checkState(!active, "queue must be stopped"); setActive(true); - syncFromStorage(); // Mark these tasks as failed as they could not reacuire the lock // Clean up needs to happen after tasks have been synced from storage Set tasksToFail = taskLockbox.syncFromStorage().getTasksToFail(); + syncFromStorage(); for (Task task : tasksToFail) { shutdown(task.getId(), "Shutting down forcefully as task failed to reacquire lock while becoming leader"); @@ -295,6 +295,7 @@ public void stop() try { setActive(false); activeTasks.clear(); + taskLockbox.shutdown(); managerExec.shutdownNow(); storageSyncExec.shutdownNow(); requestManagement(); @@ -565,7 +566,7 @@ private void addTaskInternal(final Task task, final DateTime updateTime) if (added.get()) { taskLockbox.add(task); } else if (!entry.task.equals(task)) { - throw new ISE("Cannot add task ID [%s] with same ID as task that has already been added", task.getId()); + throw new ISE("Cannot add task[%s] as a different task for the same ID has already been added.", task.getId()); } } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/TaskLocksTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/TaskLocksTest.java index e9147f9d3311..3fd3cd9cbd9e 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/TaskLocksTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/TaskLocksTest.java @@ -69,6 +69,7 @@ public void setup() taskStorage, new TestIndexerMetadataStorageCoordinator() ); + lockbox.syncFromStorage(); task = NoopTask.create(); taskStorage.insert(task, TaskStatus.running(task.getId())); lockbox.add(task); 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 3c64c9f23824..9353953fc2af 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 @@ -137,6 +137,7 @@ public void setUp() private void initScheduler() { GlobalTaskLockbox taskLockbox = new GlobalTaskLockbox(taskStorage, new TestIndexerMetadataStorageCoordinator()); + taskLockbox.syncFromStorage(); WorkerBehaviorConfig defaultWorkerConfig = new DefaultWorkerBehaviorConfig(WorkerBehaviorConfig.DEFAULT_STRATEGY, null); scheduler = new OverlordCompactionScheduler( diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLockBoxConcurrencyTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLockBoxConcurrencyTest.java index ff8da618a51a..b4aecc1e6d0c 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLockBoxConcurrencyTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLockBoxConcurrencyTest.java @@ -45,6 +45,8 @@ import org.junit.Before; import org.junit.Rule; import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.util.Collections; import java.util.HashSet; @@ -56,6 +58,7 @@ public class TaskLockBoxConcurrencyTest { + private static final Logger log = LoggerFactory.getLogger(TaskLockBoxConcurrencyTest.class); @Rule public final TestDerbyConnector.DerbyConnectorRule derby = new TestDerbyConnector.DerbyConnectorRule(); @@ -99,6 +102,7 @@ public void setup() CentralizedDatasourceSchemaConfig.create() ) ); + lockbox.syncFromStorage(); service = Execs.multiThreaded(2, "TaskLockBoxConcurrencyTest-%d"); } From 1a4a5fd8e38c211247f51157eca0c23e96de467c Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Thu, 5 Jun 2025 21:38:20 +0530 Subject: [PATCH 11/14] Remove unused logger --- .../druid/indexing/overlord/TaskLockBoxConcurrencyTest.java | 3 --- 1 file changed, 3 deletions(-) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLockBoxConcurrencyTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLockBoxConcurrencyTest.java index b4aecc1e6d0c..fffa90bf732d 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLockBoxConcurrencyTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLockBoxConcurrencyTest.java @@ -45,8 +45,6 @@ import org.junit.Before; import org.junit.Rule; import org.junit.Test; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import java.util.Collections; import java.util.HashSet; @@ -58,7 +56,6 @@ public class TaskLockBoxConcurrencyTest { - private static final Logger log = LoggerFactory.getLogger(TaskLockBoxConcurrencyTest.class); @Rule public final TestDerbyConnector.DerbyConnectorRule derby = new TestDerbyConnector.DerbyConnectorRule(); From 19c72ca444cbfb85325237993f41c1dd1e886e3d Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Fri, 6 Jun 2025 15:10:49 +0530 Subject: [PATCH 12/14] Fix up some tests --- .../overlord/GlobalTaskLockboxTest.java | 52 +++++-------------- 1 file changed, 14 insertions(+), 38 deletions(-) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/GlobalTaskLockboxTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/GlobalTaskLockboxTest.java index 1401a8a3bbdb..414e3aaca36a 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/GlobalTaskLockboxTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/GlobalTaskLockboxTest.java @@ -1877,38 +1877,28 @@ public void testFailedToReacquireTaskLock() // Please refer to NullLockPosseTaskLockbox final Task taskWithFailingLockAcquisition0 = new NoopTask(null, "FailingLockAcquisition", null, 0, 0, null); final Task taskWithFailingLockAcquisition1 = new NoopTask(null, "FailingLockAcquisition", null, 0, 0, null); - final Task taskWithSuccessfulLockAcquisition = NoopTask.create(); + final Task taskWithSuccessfulLockAcquisition = new NoopTask(null, "successGroup", "foo", 0L, 0L, Map.of()); taskStorage.insert(taskWithFailingLockAcquisition0, TaskStatus.running(taskWithFailingLockAcquisition0.getId())); taskStorage.insert(taskWithFailingLockAcquisition1, TaskStatus.running(taskWithFailingLockAcquisition1.getId())); taskStorage.insert(taskWithSuccessfulLockAcquisition, TaskStatus.running(taskWithSuccessfulLockAcquisition.getId())); + Assert.assertEquals(3, taskStorage.getActiveTasks().size()); - GlobalTaskLockbox testLockbox = new NullLockPosseGlobalTaskLockbox(taskStorage, metadataStorageCoordinator); - testLockbox.syncFromStorage(); - - testLockbox.add(taskWithFailingLockAcquisition0); - testLockbox.add(taskWithFailingLockAcquisition1); - testLockbox.add(taskWithSuccessfulLockAcquisition); - - testLockbox.tryLock(taskWithFailingLockAcquisition0, - new TimeChunkLockRequest(TaskLockType.EXCLUSIVE, - taskWithFailingLockAcquisition0, - Intervals.of("2017-07-01/2017-08-01"), - null - ) - ); - - testLockbox.tryLock(taskWithSuccessfulLockAcquisition, - new TimeChunkLockRequest(TaskLockType.EXCLUSIVE, - taskWithSuccessfulLockAcquisition, - Intervals.of("2017-07-01/2017-08-01"), - null - ) + final TaskLock lock = new TimeChunkLock( + null, + taskWithSuccessfulLockAcquisition.getGroupId(), + taskWithSuccessfulLockAcquisition.getDataSource(), + Intervals.of("2025-01-01/P1D"), + "v1", + taskWithSuccessfulLockAcquisition.getPriority() ); + taskStorage.addLock(taskWithSuccessfulLockAcquisition.getId(), lock); + taskStorage.addLock(taskWithFailingLockAcquisition0.getId(), lock); + taskStorage.addLock(taskWithFailingLockAcquisition1.getId(), lock); - Assert.assertEquals(3, taskStorage.getActiveTasks().size()); + GlobalTaskLockbox testLockbox = new GlobalTaskLockbox(taskStorage, metadataStorageCoordinator); + TaskLockboxSyncResult result = testLockbox.syncFromStorage(); // The tasks must be marked for failure - TaskLockboxSyncResult result = testLockbox.syncFromStorage(); Assert.assertEquals(ImmutableSet.of(taskWithFailingLockAcquisition0, taskWithFailingLockAcquisition1), result.getTasksToFail()); } @@ -2283,18 +2273,4 @@ public TaskStatus runTask(TaskToolbox toolbox) return TaskStatus.failure("how?", "Dummy task status err msg"); } } - - /** - * Extends TaskLockbox to return a null TaskLockPosse when the task's group name contains "FailingLockAcquisition". - */ - private static class NullLockPosseGlobalTaskLockbox extends GlobalTaskLockbox - { - public NullLockPosseGlobalTaskLockbox( - TaskStorage taskStorage, - IndexerMetadataStorageCoordinator metadataStorageCoordinator - ) - { - super(taskStorage, metadataStorageCoordinator); - } - } } From 55459045ee3ee8919af66ea750196fd2a1adb0a9 Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Mon, 9 Jun 2025 21:11:28 +0530 Subject: [PATCH 13/14] Javadocs cleanup --- .../indexing/overlord/GlobalTaskLockbox.java | 43 ++++++++++--------- .../druid/indexing/overlord/TaskLockbox.java | 4 +- 2 files changed, 24 insertions(+), 23 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/GlobalTaskLockbox.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/GlobalTaskLockbox.java index 445caca6dcc4..94f5452d1cee 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/GlobalTaskLockbox.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/GlobalTaskLockbox.java @@ -153,8 +153,8 @@ public void shutdown() /** * Acquires a lock on behalf of a task. Blocks until the lock is acquired. * - * @return {@link LockResult} containing a new or an existing lock if succeeded. Otherwise, {@link LockResult} with a - * {@link LockResult#revoked} flag. + * @return {@link LockResult} containing a new or an existing lock if succeeded. + * Otherwise, {@link LockResult} with {@link LockResult#isRevoked()} true. * @throws InterruptedException if the current thread is interrupted */ public LockResult lock(final Task task, final LockRequest request) throws InterruptedException @@ -166,10 +166,11 @@ public LockResult lock(final Task task, final LockRequest request) throws Interr } /** - * Acquires a lock on behalf of a task, waiting up to the specified wait time if necessary. + * Acquires a lock on behalf of a task, waiting up to the specified wait time + * if necessary. * - * @return {@link LockResult} containing a new or an existing lock if succeeded. Otherwise, {@link LockResult} with a - * {@link LockResult#revoked} flag. + * @return {@link LockResult} containing a new or an existing lock if succeeded. + * Otherwise, a {@link LockResult} with {@link LockResult#isRevoked()} true. * @throws InterruptedException if the current thread is interrupted */ public LockResult lock(final Task task, final LockRequest request, long timeoutMs) throws InterruptedException @@ -184,8 +185,8 @@ public LockResult lock(final Task task, final LockRequest request, long timeoutM * Attempt to acquire a lock for a task, without removing it from the queue. Can safely be called multiple times on * the same task until the lock is preempted. * - * @return {@link LockResult} containing a new or an existing lock if succeeded. Otherwise, {@link LockResult} with a - * {@link LockResult#revoked} flag. + * @return {@link LockResult} containing a new or an existing lock if succeeded. + * Otherwise, {@link LockResult} with {@link LockResult#isRevoked()} true. * @throws IllegalStateException if the task is not a valid active task */ public LockResult tryLock(final Task task, final LockRequest request) @@ -201,8 +202,7 @@ public LockResult tryLock(final Task task, final LockRequest request) * a {@link Task} and a {@link SegmentAllocateAction}. This method tries to * acquire the task locks on the required intervals/segments and then performs * a batch allocation of segments. It is possible that some requests succeed - * successfully and others failed. In that case, only the failed ones should be - * retried. + * and others fail. In that case, only the failed ones should be retried. * * @param requests List of allocation requests * @param dataSource Datasource for which segment is to be allocated. @@ -277,7 +277,6 @@ public void revokeLock(String taskId, TaskLock lock) /** * Cleans up pending segments associated with the given task, if any. */ - @VisibleForTesting protected void cleanupPendingSegments(Task task) { executeForTask( @@ -287,10 +286,7 @@ protected void cleanupPendingSegments(Task task) } /** - * Return the currently-active locks for some task. - * - * @param task task for which to locate locks - * @return currently-active locks for the given task + * Returns all the active locks currently held by the given task. */ public List findLocksForTask(final Task task) { @@ -301,7 +297,7 @@ public List findLocksForTask(final Task task) } /** - * Finds the active non-revoked REPLACE locks held by the given task. + * Returns all the active non-revoked REPLACE locks held by the given task. */ public Set findReplaceLocksForTask(Task task) { @@ -312,7 +308,7 @@ public Set findReplaceLocksForTask(Task task) } /** - * Finds all the active non-revoked REPLACE locks for the given datasource. + * Returns all the active non-revoked REPLACE locks for the given datasource. */ public Set getAllReplaceLocksForDatasource(String datasource) { @@ -374,6 +370,9 @@ public Map> getActiveLocks(List lockFil return datasourceToActiveLocks; } + /** + * Releases the lock held by a task over the given interval. + */ public void unlock(final Task task, final Interval interval) { executeForTask( @@ -382,6 +381,9 @@ public void unlock(final Task task, final Interval interval) ); } + /** + * Releases all locks currently held by the given task. + */ public void unlockAll(Task task) { executeForTask( @@ -390,6 +392,9 @@ public void unlockAll(Task task) ); } + /** + * Adds the given task to the set of active tasks. + */ public void add(Task task) { executeForTask( @@ -399,10 +404,8 @@ public void add(Task task) } /** - * Release all locks for a task and remove task from set of active tasks. - * Does nothing if the task is not currently locked or not an active task. - * - * @param task task to unlock + * Removes a task from the set of active tasks and releases all locks held by it. + * Does nothing if the task is not active or doesn't hold any locks. */ public void remove(final Task task) { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskLockbox.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskLockbox.java index fd17cf0e8870..c7161b75463b 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskLockbox.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskLockbox.java @@ -272,8 +272,7 @@ public int compare(Pair left, Pair right) * @return null if the lock could not be reacquired. */ @Nullable - @VisibleForTesting - protected TaskLockPosse reacquireLockOnStartup(Task task, TaskLock taskLock) + private TaskLockPosse reacquireLockOnStartup(Task task, TaskLock taskLock) { if (!taskMatchesLock(task, taskLock)) { log.warn( @@ -823,7 +822,6 @@ private void revokeLock(TaskLockPosse lockPosse) * @param taskId an id of the task holding the lock * @param lock lock to be revoked */ - @VisibleForTesting public void revokeLock(String taskId, TaskLock lock) { giant.lock(); From aa0fafb1b69e06fbf9c6f9872ee472c0d305ba67 Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Wed, 11 Jun 2025 06:05:02 +0530 Subject: [PATCH 14/14] Verify sync complete before acquiring lock resource --- .../indexing/overlord/GlobalTaskLockbox.java | 32 ++++++++++++------- .../overlord/GlobalTaskLockboxTest.java | 17 ++++++++++ 2 files changed, 37 insertions(+), 12 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/GlobalTaskLockbox.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/GlobalTaskLockbox.java index 94f5452d1cee..a464af86868e 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/GlobalTaskLockbox.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/GlobalTaskLockbox.java @@ -111,7 +111,7 @@ public TaskLockboxSyncResult syncFromStorage() final AtomicInteger taskLockCount = new AtomicInteger(0); datasourceToSyncResult.forEach((dataSource, syncResult) -> { - try (final DatasourceLockboxResource lockboxResource = getLockboxResource(dataSource)) { + try (final DatasourceLockboxResource lockboxResource = getLockboxResource(dataSource, false)) { final TaskLockboxSyncResult lockboxSyncResult = lockboxResource.delegate.resetState( syncResult.storedActiveTasks, syncResult.storedLocks @@ -141,6 +141,8 @@ public void shutdown() // Clean up all existing lockboxes final Set datasourceNames = Set.copyOf(datasourceToLockbox.keySet()); for (String dataSource : datasourceNames) { + // This can block if a bad runaway operation is still working on the underlying TaskLockbox. + // There is currently no clear way to interrupt ongoing operations. cleanupLockboxResourceIf(dataSource, resource -> true); } @@ -468,8 +470,14 @@ Map { if (resource != null && resourcePredicate.test(resource)) { - // TODO: what if a bad runaway operation is holding the TaskLockbox.giant? resource.delegate.clear(); return null; } else { @@ -518,15 +534,7 @@ private R computeForDatasource( LockComputation computation ) throws T { - // Verify that sync is complete - if (!syncComplete.get()) { - throw new ISE( - "Cannot get TaskLockbox for datasource[%s] as sync with storage has not happened yet.", - datasource - ); - } - - try (final DatasourceLockboxResource lockbox = getLockboxResource(datasource)) { + try (final DatasourceLockboxResource lockbox = getLockboxResource(datasource, true)) { return computation.perform(lockbox.delegate); } } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/GlobalTaskLockboxTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/GlobalTaskLockboxTest.java index 414e3aaca36a..2c5390e568b9 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/GlobalTaskLockboxTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/GlobalTaskLockboxTest.java @@ -29,6 +29,7 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Iterables; +import org.apache.druid.error.ExceptionMatcher; import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexing.common.LockGranularity; import org.apache.druid.indexing.common.SegmentLock; @@ -71,6 +72,7 @@ import org.apache.druid.timeline.partition.PartialShardSpec; import org.apache.druid.timeline.partition.PartitionIds; import org.easymock.EasyMock; +import org.hamcrest.MatcherAssert; import org.joda.time.Interval; import org.junit.Assert; import org.junit.Before; @@ -2050,6 +2052,21 @@ public void testCleanupOnUnlock() EasyMock.verify(coordinator); } + @Test + public void test_add_throwsException_ifSyncIsNotComplete() + { + lockbox = new GlobalTaskLockbox(taskStorage, metadataStorageCoordinator); + MatcherAssert.assertThat( + Assert.assertThrows( + ISE.class, + () -> lockbox.add(NoopTask.create()) + ), + ExceptionMatcher.of(ISE.class).expectMessageIs( + "Cannot get TaskLockbox for datasource[none] as sync with storage has not happened yet." + ) + ); + } + private class TaskLockboxValidator {