-
Notifications
You must be signed in to change notification settings - Fork 4.8k
[DSIP-55][Master] Separate the waiting dispatched task into different queue by worker group #17037
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Merged
+808
−175
Merged
Changes from all commits
Commits
Show all changes
46 commits
Select commit
Hold shift + click to select a range
e76f57b
fix #16260
b03e038
fix code style
7bf262c
fix spotless
cdc3749
fix unit test
c771b2c
fix spotless
e20ba6d
WorkerGroupTaskDispatchManager manages threads and queues.
6e232c9
WorkerGroupTaskDispatchManager manages threads and queues.
89f173a
rollback master logback.xml
c3a45a9
fix style
098e971
close workerGroupTaskDispatchManager
ffea08e
close workerGroupTaskDispatchManager
0686ae1
fix spotless
45e5e69
add ThreadCreatingAndDestroyingWorkerGroupListener
eb29608
add unit test
9ed7d43
add unit test
6096beb
add unit test
b9f8434
optimized code
63f51b0
fix lazy
02b4a2c
add todo
61deeb8
fix spotless:
9accebc
change WorkerGroupTaskDispatchManager -> WorkerGroupTaskDispatcherMan…
6038d23
fix sonar
f75fc95
optimized code
8a60ce4
optimized code
19c6128
global queue is delayQueue
1159a1e
PriorityDelayQueue move to WorkerGroupTaskDispatcher,Simplify the Wor…
bd6d9ee
Remove unnecessary resource cleanup
63ffb7c
less unit test time
fcdec92
fix jdk8 unit test
ee3fae8
optimized code
1e86477
fix sonar
c751272
override equals
85fa740
Optimize concurrency issues
af6be6a
\cannot find the workergroup, task will kill.
ca78f4f
fix unit test
48ebb78
fix unit test
9340610
spotless
7c9f842
retry e2e
057cac9
jdk 11 unit test
18cf5c0
Maintain the original logic: if the worker group cannot be found, pla…
d86b44a
optimized code
b4abd0b
Remove the close logic for WorkerGroupTaskDispatcher. Since this scen…
0fb49ae
fix unit test
f5f0bfe
Change the logic to dynamically create the workerGroupTaskDispatcher …
c2487a1
fix unit test
a8b6650
optimization unit test
File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
125 changes: 125 additions & 0 deletions
125
...main/java/org/apache/dolphinscheduler/server/master/runner/WorkerGroupTaskDispatcher.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,125 @@ | ||
/* | ||
* 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.dolphinscheduler.server.master.runner; | ||
|
||
import org.apache.dolphinscheduler.common.thread.BaseDaemonThread; | ||
import org.apache.dolphinscheduler.dao.entity.TaskInstance; | ||
import org.apache.dolphinscheduler.plugin.task.api.enums.TaskExecutionStatus; | ||
import org.apache.dolphinscheduler.server.master.engine.task.client.ITaskExecutorClient; | ||
import org.apache.dolphinscheduler.server.master.engine.task.runnable.ITaskExecutionRunnable; | ||
import org.apache.dolphinscheduler.server.master.runner.queue.PriorityAndDelayBasedTaskEntry; | ||
import org.apache.dolphinscheduler.server.master.runner.queue.PriorityDelayQueue; | ||
|
||
import java.util.concurrent.atomic.AtomicBoolean; | ||
|
||
import lombok.extern.slf4j.Slf4j; | ||
|
||
/** | ||
* WorkerGroupTaskDispatcher is responsible for dispatching tasks from the task queue. | ||
* The main responsibilities include: | ||
* 1. Continuously fetching tasks from the {@link PriorityDelayQueue} for dispatch. | ||
* 2. Re-queuing tasks that fail to dispatch according to retry logic. | ||
* 3. Ensuring thread safety and correct state transitions during task processing. | ||
*/ | ||
@Slf4j | ||
public class WorkerGroupTaskDispatcher extends BaseDaemonThread { | ||
|
||
private final ITaskExecutorClient taskExecutorClient; | ||
|
||
// TODO The current queue is flawed. When a high-priority task fails, | ||
// it will be delayed and will not return to the first or second position. | ||
// Tasks with the same priority will preempt its position. | ||
// If it needs to be placed at the front of the queue, the queue needs to be re-implemented. | ||
private final PriorityDelayQueue<PriorityAndDelayBasedTaskEntry<ITaskExecutionRunnable>> workerGroupQueue; | ||
|
||
private final AtomicBoolean runningFlag = new AtomicBoolean(false); | ||
|
||
public WorkerGroupTaskDispatcher(String workerGroupName, ITaskExecutorClient taskExecutorClient) { | ||
super("WorkerGroupTaskDispatcher-" + workerGroupName); | ||
this.taskExecutorClient = taskExecutorClient; | ||
this.workerGroupQueue = new PriorityDelayQueue<>(); | ||
} | ||
|
||
/** | ||
* Adds a task to the worker group queue. | ||
* This method wraps the given task execution object into a priority and delay-based task entry and adds it to the worker group queue. | ||
* The task is only added if the current dispatcher status is either STARTED or INIT. If the dispatcher is in any other state, | ||
* the task addition will fail, and a warning message will be logged. | ||
* | ||
* @param taskExecutionRunnable The task execution object to add to the queue, which implements the {@link ITaskExecutionRunnable} interface. | ||
* @param delayTimeMills The delay time in milliseconds before the task should be executed. | ||
*/ | ||
public void addTaskToWorkerGroupQueue(ITaskExecutionRunnable taskExecutionRunnable, | ||
long delayTimeMills) { | ||
workerGroupQueue.add(new PriorityAndDelayBasedTaskEntry<>(delayTimeMills, taskExecutionRunnable)); | ||
} | ||
|
||
@Override | ||
public synchronized void start() { | ||
if (runningFlag.compareAndSet(false, true)) { | ||
log.info("The {} starting...", this.getName()); | ||
super.start(); | ||
log.info("The {} started", this.getName()); | ||
} else { | ||
log.error("The {} status is {}, will not start again", this.getName(), runningFlag.get()); | ||
} | ||
} | ||
|
||
public synchronized void close() { | ||
log.info("The {} closed called but not implemented", this.getName()); | ||
// todo WorkerGroupTaskDispatcher thread needs to be shut down after the WorkerGroup is deleted. | ||
} | ||
|
||
@Override | ||
public void run() { | ||
while (runningFlag.get()) { | ||
dispatch(); | ||
} | ||
} | ||
|
||
private void dispatch() { | ||
PriorityAndDelayBasedTaskEntry<ITaskExecutionRunnable> taskEntry = workerGroupQueue.take(); | ||
ITaskExecutionRunnable taskExecutionRunnable = taskEntry.getData(); | ||
final TaskInstance taskInstance = taskExecutionRunnable.getTaskInstance(); | ||
try { | ||
final TaskExecutionStatus taskStatus = taskInstance.getState(); | ||
if (taskStatus != TaskExecutionStatus.SUBMITTED_SUCCESS | ||
&& taskStatus != TaskExecutionStatus.DELAY_EXECUTION) { | ||
log.warn("The TaskInstance {} state is : {}, will not dispatch", taskInstance.getName(), taskStatus); | ||
return; | ||
} | ||
taskExecutorClient.dispatch(taskExecutionRunnable); | ||
} catch (Exception e) { | ||
// If dispatch failed, will put the task back to the queue | ||
// The task will be dispatched after waiting time. | ||
// the waiting time will increase multiple of times, but will not exceed 60 seconds | ||
long waitingTimeMills = Math.min( | ||
taskExecutionRunnable.getTaskExecutionContext().increaseDispatchFailTimes() * 1_000L, 60_000L); | ||
workerGroupQueue.add(new PriorityAndDelayBasedTaskEntry<>(waitingTimeMills, taskExecutionRunnable)); | ||
log.error("Dispatch Task: {} failed will retry after: {}/ms", taskInstance.getName(), waitingTimeMills, e); | ||
} | ||
} | ||
|
||
/** | ||
* ony use unit test | ||
* @return size | ||
*/ | ||
protected int queueSize() { | ||
return this.workerGroupQueue.size(); | ||
} | ||
} |
83 changes: 83 additions & 0 deletions
83
...va/org/apache/dolphinscheduler/server/master/runner/WorkerGroupTaskDispatcherManager.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,83 @@ | ||
/* | ||
* 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.dolphinscheduler.server.master.runner; | ||
|
||
import org.apache.dolphinscheduler.server.master.engine.task.client.ITaskExecutorClient; | ||
import org.apache.dolphinscheduler.server.master.engine.task.runnable.ITaskExecutionRunnable; | ||
|
||
import java.util.Map; | ||
import java.util.concurrent.ConcurrentHashMap; | ||
|
||
import lombok.Getter; | ||
import lombok.extern.slf4j.Slf4j; | ||
|
||
import org.springframework.beans.factory.annotation.Autowired; | ||
import org.springframework.stereotype.Component; | ||
|
||
/** | ||
* WorkerGroupTaskDispatcherManager is responsible for managing the task dispatching for worker groups. | ||
* It maintains a mapping of worker groups to their task dispatchers and priority delay queues, | ||
* and supports adding tasks, starting and stopping worker groups, as well as cleaning up resources upon shutdown. | ||
*/ | ||
@Component | ||
@Slf4j | ||
public class WorkerGroupTaskDispatcherManager implements AutoCloseable { | ||
|
||
@Autowired | ||
private ITaskExecutorClient taskExecutorClient; | ||
|
||
@Getter | ||
private final ConcurrentHashMap<String, WorkerGroupTaskDispatcher> dispatchWorkerMap; | ||
|
||
public WorkerGroupTaskDispatcherManager() { | ||
dispatchWorkerMap = new ConcurrentHashMap<>(); | ||
} | ||
|
||
/** | ||
* Adds a task to the specified worker group queue and starts or wakes up the corresponding processing loop. | ||
* | ||
* @param workerGroup the identifier for the worker group, used to distinguish different task queues | ||
* @param taskExecutionRunnable an instance of ITaskExecutionRunnable representing the task to be executed | ||
* @param delayTimeMills the delay time before the task is executed, in milliseconds | ||
*/ | ||
public synchronized void addTaskToWorkerGroup(String workerGroup, ITaskExecutionRunnable taskExecutionRunnable, | ||
long delayTimeMills) { | ||
WorkerGroupTaskDispatcher workerGroupTaskDispatcher = dispatchWorkerMap.computeIfAbsent( | ||
workerGroup, key -> new WorkerGroupTaskDispatcher(workerGroup, taskExecutorClient)); | ||
if (!workerGroupTaskDispatcher.isAlive()) { | ||
workerGroupTaskDispatcher.start(); | ||
} | ||
workerGroupTaskDispatcher.addTaskToWorkerGroupQueue(taskExecutionRunnable, delayTimeMills); | ||
} | ||
|
||
/** | ||
* Stop all workerGroupTaskDispatchWaitingQueueLooper | ||
*/ | ||
@Override | ||
public void close() throws Exception { | ||
log.info("WorkerGroupTaskDispatcherManager start close"); | ||
for (Map.Entry<String, WorkerGroupTaskDispatcher> entry : dispatchWorkerMap.entrySet()) { | ||
try { | ||
entry.getValue().close(); | ||
} catch (Exception e) { | ||
log.error("close worker group error", e); | ||
} | ||
} | ||
log.info("WorkerGroupTaskDispatcherManager closed"); | ||
} | ||
} |
Oops, something went wrong.
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Should before
log.info("GlobalTaskDispatchWaitingQueueLooper stopped...");
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
My mistake led to the meaningless log. I will make the changes as suggested.