Remove giant lock from Overlord TaskQueue#14293
Remove giant lock from Overlord TaskQueue#14293kfaraz wants to merge 10 commits intoapache:masterfrom
giant lock from Overlord TaskQueue#14293Conversation
| Long::sum | ||
| )); | ||
| return taskRunner.getRunningTasks().stream().collect( | ||
| Collectors.toMap(TaskRunnerWorkItem::getDataSource, task -> 1L, Long::sum) |
Check notice
Code scanning / CodeQL
Useless parameter
| Long::sum | ||
| )); | ||
| return taskRunner.getPendingTasks().stream().collect( | ||
| Collectors.toMap(TaskRunnerWorkItem::getDataSource, task -> 1L, Long::sum) |
Check notice
Code scanning / CodeQL
Useless parameter
| Long::sum | ||
| )); | ||
| return taskRunner.getRunningTasks().stream().collect( | ||
| Collectors.toMap(TaskRunnerWorkItem::getDataSource, task -> 1L, Long::sum) |
Check notice
Code scanning / CodeQL
Useless parameter
| Long::sum | ||
| )); | ||
| return taskRunner.getPendingTasks().stream().collect( | ||
| Collectors.toMap(TaskRunnerWorkItem::getDataSource, task -> 1L, Long::sum) |
Check notice
Code scanning / CodeQL
Useless parameter
| managementMayBeNecessary.offer(this); | ||
| synchronized (managementRequested) { | ||
| managementRequested.set(true); | ||
| managementRequested.notify(); |
Check warning
Code scanning / CodeQL
notify instead of notifyAll
jasonk000
left a comment
There was a problem hiding this comment.
@kfaraz , good to see -- performance of this code has been key to stability of the cluster for us.
I think there's some prior work here that's worth reviewing too to make sure we don't reintroduce these bugs, I've tagged at appropriate points.
| return delta; | ||
| } | ||
| Map<String, Long> total = new HashMap<>(datasourceToSuccessfulTaskCount); | ||
| datasourceToSuccessfulTaskCount.clear(); |
There was a problem hiding this comment.
This and following fns have a small race if the CHM was changed between the call to clear. There could be some alternatives, like iterating entries and using replaceAll, or remove() or merge() in a loop, that would allow a more atomic unload of the CHM? Or, maybe make the value in the map an AtomicLong, and CAS modify it to zero during unload?
There was a problem hiding this comment.
Yeah, I didn't like this either. Updated to iterate over keys and atomically remove them one by one.
| } | ||
| requestManagement(); | ||
| // Remove any unacquired locks from storage (shutdown only clears entries for which a TaskLockPosse was acquired) | ||
| // This is called after requesting management as locks need to be cleared after notifyStatus is processed |
There was a problem hiding this comment.
Observation - This is an interesting comment, given that there's no guarantee the management has run by now. I wonder if it's stale, or is there an issue here? ... , since it has same behaviour as previous code.
There was a problem hiding this comment.
Yeah, my thoughts exactly. Calling requestManagement() does not guarantee that management has actually happened. I am not a 100% on what was going on here, so left it as is. I will double check this code and maybe add some tests.
There was a problem hiding this comment.
I checked the code. This comment is outdated/irrelevant now.
Firstly, calling requestManagement() just adds a request to the queue and does not ensure that task management has actually taken place. In fact, on start-up, it typically wouldn't take place until after 1 minute (default value of start delay).
Secondly, even if for some other reason, requestManagement() needs to be called before clearing the dangling locks, it would have already been called in the for loop preceding this (shutdown calls notifyStatus calls requestManagement).
I am removing this comment and moving the contents of the next for loop into the first one. I am also adding some tests to ensure cleanup of such dangling locks.
| catch (Exception e) { | ||
| log.warn(e, "TaskRunner failed to clean up task: %s", taskId); | ||
| } | ||
| final Set<String> knownTaskIds = tasks.keySet(); |
There was a problem hiding this comment.
I'm a little wary of reintroducing this bug: #12901, in the case that tasks was changed. Can you check over it? It seems like if tasks has changed we might have an issue. But - you've covered most of them with synchronized. Can you check in add() and removeTasksInternal, I think these operate on tasks without being synchronized, so it might lead to some race?
There was a problem hiding this comment.
Nb: earlier PR #12099 is maybe relevant here too for background/history.
There was a problem hiding this comment.
Thanks for calling this out, @jasonk000 !
The issue described in #12901 is basically a race condition between notifyStatus and manageInternalCritical (renamed in this patch to runReadyTasks), where a task that is being shutdown might get relaunched. In #12901, this was solved by using this order of events:
- Mark the task as
recentlyCompleted - Update metadata store
- Finish cleanup of in-memory data structures
AllrecentlyCompletedtasks were ignored inmanageInternalCritical.
In the new set of changes, I have retained this complete behaviour. As soon as a task is marked as recentlyCompleted, it will not be touched by runReadyTasks or killUnknownTasks. The in-memory data-structures (including recentlyCompleted itself) are finally cleaned up atomically only when the task shutdown is finished.
I will try to add some tests for this scenario. Please let me know if you think of any other race condition, would be nice to have tests for all of those.
There was a problem hiding this comment.
I have tried to ensure that all the changes made in the patch #12901 are retained.
This patch does the following:
- Fixes the race by adding a recentlyCompletedTasks set that prevents
the main management loop from doing anything with tasks that are
currently being cleaned up.
The set of recentlyCompletedTaskIds is still being used for this purpose.
- Switches the order of the critical sections in notifyStatus, so
metadata store updates happen first. This is useful in case of
server failures: it ensures that if the Overlord fails in the midst
of notifyStatus, then completed-task statuses are still available in
ZK or on MMs for the next Overlord. (Those are cleaned up by
taskRunner.shutdown, which formerly ran first.) This isn't related
to the race described above, but is fixed opportunistically as part
of the same patch.
Order of calls is still the same i.e. metadata store update happens first
- Changes the tasks list to a map. Many operations require retrieval
or removal of individual tasks; those are now O(1) instead of O(N)
in the number of running tasks.
- The
tasksmap is now aConcurrentHashMapwhich still performs O(1) get and remove. - But now we also have an
activeTaskIdQueueon which we do an O(n) remove. But this is fine because it does not block other operations of theTaskQueue. We need a queue(-like) structure here that must be thread-safe, so I ended up using aLinkedBlockingDeque. I am not sure of an alternative that would offer better time complexity.
- Changes various log messages to use task ID instead of full task
payload, to make the logs more readable.
Retained as is.
| } | ||
| tasks.computeIfAbsent( | ||
| task.getId(), | ||
| taskId -> { |
There was a problem hiding this comment.
It seems the implementation here relies on an implicit lock being taken by CHM. Might be worth noting given that it is implementations-specific with CHM, in case implementation is switched in the future.
There was a problem hiding this comment.
Yeah, it is intentional. I will add a comment calling out why it must be a ConcurrentHashMap.
| // Critical section: remove this task from all of our tracking data structures. | ||
| giant.lock(); | ||
| try { | ||
| if (removeTaskInternal(task.getId())) { |
There was a problem hiding this comment.
re: earlier note about possibly re-introducing #12901.
| // Add new tasks and clean up removed tasks | ||
| addedTasks.forEach(this::addTaskInternal); | ||
| removedTasks.forEach(this::removeTaskInternal); | ||
| log.info( |
There was a problem hiding this comment.
This existed before, but it might be wise to remove as much as possible, including logging, from the synchronized block, and tighten it down a bit.
There was a problem hiding this comment.
I don't think the footprint of this call would really make much of a difference compared to the earlier operations, fetching from DB and map manipulation.
But yeah, you never really know with logging. I suppose I could just return the counts from this method and log the values in the caller itself.
There was a problem hiding this comment.
Could use synchronized (this) { .... } ?
There was a problem hiding this comment.
But yes - it does raise a good qn about the remainder of the workflow.
|
Thanks for the feedback, @jasonk000 ! I have responded to your comments and plan to make changes/add more tests wherever necessary. |
| // do not care if the item fits into the queue: | ||
| // if the queue is already full, request has been triggered anyway | ||
| managementMayBeNecessary.offer(this); | ||
| managementRequestQueue.offer(new Object()); |
Check notice
Code scanning / CodeQL
Ignored error status of call
| // do not care if the item fits into the queue: | ||
| // if the queue is already full, request has been triggered anyway | ||
| managementMayBeNecessary.offer(this); | ||
| managementRequestQueue.offer(this); |
Check notice
Code scanning / CodeQL
Ignored error status of call
|
This pull request has been marked as stale due to 60 days of inactivity. |
|
This pull request/issue has been closed due to lack of activity. If you think that |
Description
The
TaskQueuein the Overlord implements concurrency control using agiantlock. A similar technique has been used in other classes such asTaskMasterandTaskLockbox. While thisgiantlock does guarantee thread-safe access of critical sections of code, it can be too restrictive at times and can even lead to the Overlord being completely stuck.A typical scenario is described below.
index_paralleltask fails with aSQLTransientException(say, due to an oversized payload)index_parallelrepeatedly requests the Overlord to insert this sub-taskRetryUtils)TaskQueue.giantlockTaskQueueoperation can proceed without the lock. This includes operations like adding a new task, killing a task, submitting tasks to runner for execution, syncing from metadata, etc.Note: The indefinite retry issue in the above scenario is also being addressed separately in #14271
Current implementation
The
giantlock is a reentrant lock, which is effectively the same as the object monitor associated with any java object. In principle, this lock could be replaced by simply making all the methods ofTaskQueuesynchronized.There are several fields which can be protected with more lenient locking to improve performance.
Proposed implementation
The operations/fields currently protected by the
giantlock are discussed below.(a) Methods:
start(),stop(),syncFromStorage(),manage()Change: Make methods
synchronizedRationale: This effectively remains the same as the current implementation
(b) Field
LinkedHashMap<String, Task> tasks:putIfAbsent(),get(),values(),remove()Change: Use a
ConcurrentHashMapinstead in conjunction with aBlockingDeque<String>Rationale:
ConcurrentHashMap.BlockingDequeis used to maintain the order in which task IDs were submitted to theTaskQueue.compute()andcomputeIfAbsent().(c) Field
HashMap<String, Future> taskFutures:put(),remove()Change: Replace with a
Sets.newConcurrentHashSet()insteadRationale: This field is just used to track task ids already submitted to the task runner.
(d) Field
taskStorage:insert()Change: Move outside critical section
Rationale:
TaskStorageimplementations do not maintain any state and thus don't require any concurrency control(e) Field
taskLockbox:add(),remove()Change: Move outside critical section
Rationale:
TaskLockboxhas its owngiantlock and can thus be safely accessed here.Other changes
Pending items (WIP)
Unit tests for
This PR has: