ruanwenjun commented on code in PR #17037: URL: https://github.com/apache/dolphinscheduler/pull/17037#discussion_r1986540682
########## dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/queue/ComparableEntry.java: ########## @@ -0,0 +1,68 @@ +/* + * 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.queue; + +import static com.google.common.base.Preconditions.checkNotNull; + +import org.apache.dolphinscheduler.server.master.engine.task.runnable.ITaskExecutionRunnable; + +import java.util.Objects; + +import lombok.Getter; + +import org.jetbrains.annotations.NotNull; + +@Getter +public class ComparableEntry implements Comparable<ComparableEntry> { + + // Pass it to the workerGroup queue without participating in the comparison Review Comment: ```suggestion // Pass it to the workerGroup queue without participating in the comparison ``` ########## dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/queue/ComparableEntry.java: ########## @@ -0,0 +1,68 @@ +/* + * 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.queue; + +import static com.google.common.base.Preconditions.checkNotNull; + +import org.apache.dolphinscheduler.server.master.engine.task.runnable.ITaskExecutionRunnable; + +import java.util.Objects; + +import lombok.Getter; + +import org.jetbrains.annotations.NotNull; + +@Getter +public class ComparableEntry implements Comparable<ComparableEntry> { Review Comment: ```suggestion public class TimeBasedTaskExecutionRunnableComparableEntry implements Comparable<ComparableEntry> { ``` ########## dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/queue/ComparableEntry.java: ########## @@ -0,0 +1,68 @@ +/* + * 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.queue; + +import static com.google.common.base.Preconditions.checkNotNull; + +import org.apache.dolphinscheduler.server.master.engine.task.runnable.ITaskExecutionRunnable; + +import java.util.Objects; + +import lombok.Getter; + +import org.jetbrains.annotations.NotNull; + +@Getter +public class ComparableEntry implements Comparable<ComparableEntry> { + + // Pass it to the workerGroup queue without participating in the comparison + private final long delayTimeMills; + + private final ITaskExecutionRunnable data; + + public ComparableEntry(long delayTimeMills, ITaskExecutionRunnable data) { + this.delayTimeMills = delayTimeMills; + this.data = checkNotNull(data, "data is null"); + } + + @Override + public int compareTo(@NotNull ComparableEntry other) { + int priortyCompareResult = data.compareTo(other.data); + if (priortyCompareResult != 0) { + return priortyCompareResult; + } + return Long.compare(delayTimeMills, other.delayTimeMills); + } Review Comment: ```suggestion public int compareTo(@NotNull ComparableEntry other) { int delayTimeCompareResult = Long.compare(delayTimeMills, other.delayTimeMills); if (delayTimeCompareResult != 0) { return delayTimeCompareResult; } return data.compareTo(other.data); } ``` As discussed at https://github.com/apache/dolphinscheduler/issues/16260#issuecomment-2647236911, this comparation will only compare the delay time. ########## dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/cluster/ClusterManager.java: ########## @@ -48,6 +49,10 @@ public class ClusterManager { @Autowired private RegistryClient registryClient; + @Autowired + @Lazy Review Comment: Please remove `@Lazy`, this is not a good practice, we should avoid this. ########## dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/cluster/ThreadCreatingAndDestroyingWorkerGroupListener.java: ########## @@ -0,0 +1,72 @@ +/* + * 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.cluster; + +import org.apache.dolphinscheduler.dao.entity.WorkerGroup; +import org.apache.dolphinscheduler.dao.utils.WorkerGroupUtils; +import org.apache.dolphinscheduler.server.master.runner.WorkerGroupTaskDispatchManager; + +import java.util.List; +import java.util.stream.Collectors; + +import javax.annotation.PostConstruct; + +import lombok.extern.slf4j.Slf4j; + +import org.springframework.beans.factory.annotation.Autowired; +import org.springframework.stereotype.Component; + +@Component +@Slf4j +public class ThreadCreatingAndDestroyingWorkerGroupListener implements WorkerGroupChangeNotifier.WorkerGroupListener { Review Comment: Remove this class, `WorkerGroupTaskDispatchManager` should implements with `WorkerGroupChangeNotifier.WorkerGroupListener`; ########## dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/WorkerGroupTaskDispatchManager.java: ########## @@ -0,0 +1,117 @@ +/* + * 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 org.apache.dolphinscheduler.server.master.runner.queue.DelayEntry; +import org.apache.dolphinscheduler.server.master.runner.queue.PriorityDelayQueue; + +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; + +@Component +@Slf4j +public class WorkerGroupTaskDispatchManager implements AutoCloseable { + + @Autowired + private ITaskExecutorClient taskExecutorClient; + + @Getter + private final ConcurrentHashMap<String, WorkerGroupTaskDispatchWaitingQueueLooper> workerGroupTaskDispatchWaitingQueueLooperMap; + @Getter + private final ConcurrentHashMap<String, PriorityDelayQueue<DelayEntry<ITaskExecutionRunnable>>> workerGroupPriorityDelayQueueMap; + + public WorkerGroupTaskDispatchManager() { + workerGroupTaskDispatchWaitingQueueLooperMap = new ConcurrentHashMap<>(); + workerGroupPriorityDelayQueueMap = new ConcurrentHashMap<>(); + } + + /** + * Adds a task to the specified worker group queue and starts or wakes up the corresponding processing loop. Review Comment: ```suggestion * Adds a task to the specified worker group queue and starts or wakes up the corresponding processing loop. ``` There is no wake up logic. ########## dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/GlobalTaskDispatchWaitingQueue.java: ########## @@ -57,19 +59,22 @@ public synchronized void dispatchTaskExecuteRunnable(ITaskExecutionRunnable ITas public synchronized void dispatchTaskExecuteRunnableWithDelay(ITaskExecutionRunnable taskExecutionRunnable, long delayTimeMills) { waitingTaskInstanceIds.add(taskExecutionRunnable.getTaskInstance().getId()); - priorityDelayQueue.add(new DelayEntry<>(delayTimeMills, taskExecutionRunnable)); + priorityQueue.add(new ComparableEntry(delayTimeMills, taskExecutionRunnable)); } /** - * Consume {@link ITaskExecutionRunnable} from the {@link PriorityDelayQueue}, only the delay time <= 0 can be consumed. + * Consume {@link ITaskExecutionRunnable} from the {@link DelayQueue}, only the delay time <= 0 can be consumed. */ @SneakyThrows - public ITaskExecutionRunnable takeTaskExecuteRunnable() { - ITaskExecutionRunnable taskExecutionRunnable = priorityDelayQueue.take().getData(); + public ComparableEntry takeTaskExecuteRunnable() { Review Comment: ```suggestion public ITaskExecutionRunnable takeTaskExecuteRunnable() { ``` ########## dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/WorkerGroupTaskDispatchWaitingQueueLooper.java: ########## @@ -0,0 +1,78 @@ +/* + * 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.server.master.engine.task.client.ITaskExecutorClient; +import org.apache.dolphinscheduler.server.master.engine.task.runnable.ITaskExecutionRunnable; +import org.apache.dolphinscheduler.server.master.runner.queue.DelayEntry; +import org.apache.dolphinscheduler.server.master.runner.queue.PriorityDelayQueue; + +import java.util.concurrent.atomic.AtomicBoolean; + +import lombok.Getter; +import lombok.extern.slf4j.Slf4j; + +@Slf4j +public class WorkerGroupTaskDispatchWaitingQueueLooper extends BaseDaemonThread implements AutoCloseable { Review Comment: Remove this class, directly use DispatchWorker ########## dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/WorkerGroupTaskDispatchManager.java: ########## @@ -0,0 +1,117 @@ +/* + * 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 org.apache.dolphinscheduler.server.master.runner.queue.DelayEntry; +import org.apache.dolphinscheduler.server.master.runner.queue.PriorityDelayQueue; + +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; + +@Component +@Slf4j +public class WorkerGroupTaskDispatchManager implements AutoCloseable { + + @Autowired + private ITaskExecutorClient taskExecutorClient; + + @Getter + private final ConcurrentHashMap<String, WorkerGroupTaskDispatchWaitingQueueLooper> workerGroupTaskDispatchWaitingQueueLooperMap; + @Getter + private final ConcurrentHashMap<String, PriorityDelayQueue<DelayEntry<ITaskExecutionRunnable>>> workerGroupPriorityDelayQueueMap; + + public WorkerGroupTaskDispatchManager() { + workerGroupTaskDispatchWaitingQueueLooperMap = new ConcurrentHashMap<>(); + workerGroupPriorityDelayQueueMap = 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 void add(String workerGroup, ITaskExecutionRunnable taskExecutionRunnable, long delayTimeMills) { + PriorityDelayQueue<DelayEntry<ITaskExecutionRunnable>> workerGroupQueue = + workerGroupPriorityDelayQueueMap.get(workerGroup); + if (workerGroupQueue != null) { + workerGroupQueue.add(new DelayEntry<>(delayTimeMills, taskExecutionRunnable)); + log.info("queue size {}", workerGroupQueue.size()); + } else { + log.error("workerGroup {} not found", workerGroup); + } + } + + /** + * Stops a specific worker group's task dispatch waiting queue looper. + * + * @param workerGroup the identifier for the worker group + */ + public synchronized void stopWorkerGroup(String workerGroup) throws Exception { + WorkerGroupTaskDispatchWaitingQueueLooper looper = + workerGroupTaskDispatchWaitingQueueLooperMap.remove(workerGroup); + PriorityDelayQueue<DelayEntry<ITaskExecutionRunnable>> workerGroupQueue = + workerGroupPriorityDelayQueueMap.get(workerGroup); + if (workerGroupQueue != null) { + workerGroupQueue.clear(); + } + if (looper != null) { + looper.close(); + } + } Review Comment: There will exist a problem, you cannot clear the task, the running task which put into the queue should continue running. ########## dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/WorkerGroupTaskDispatchManager.java: ########## @@ -0,0 +1,117 @@ +/* + * 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 org.apache.dolphinscheduler.server.master.runner.queue.DelayEntry; +import org.apache.dolphinscheduler.server.master.runner.queue.PriorityDelayQueue; + +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; + +@Component +@Slf4j +public class WorkerGroupTaskDispatchManager implements AutoCloseable { Review Comment: ```suggestion public class WorkerGroupTaskDispatcherManager implements AutoCloseable { ``` ########## dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/DispatchWorker.java: ########## @@ -0,0 +1,63 @@ +/* + * 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.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.DelayEntry; +import org.apache.dolphinscheduler.server.master.runner.queue.PriorityDelayQueue; + +import lombok.extern.slf4j.Slf4j; + +@Slf4j +public class DispatchWorker { + + private final ITaskExecutorClient taskExecutorClient; + + private final PriorityDelayQueue<DelayEntry<ITaskExecutionRunnable>> workerGroupQueue; Review Comment: Please add a `PriorityBasedTaskExecutionRunnableDelayEntry`, this entry should only compare priority rather than delay time. ########## dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/WorkerGroupTaskDispatchManager.java: ########## @@ -0,0 +1,117 @@ +/* + * 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 org.apache.dolphinscheduler.server.master.runner.queue.DelayEntry; +import org.apache.dolphinscheduler.server.master.runner.queue.PriorityDelayQueue; + +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; + +@Component +@Slf4j +public class WorkerGroupTaskDispatchManager implements AutoCloseable { + + @Autowired + private ITaskExecutorClient taskExecutorClient; + + @Getter + private final ConcurrentHashMap<String, WorkerGroupTaskDispatchWaitingQueueLooper> workerGroupTaskDispatchWaitingQueueLooperMap; Review Comment: ```suggestion private final ConcurrentHashMap<String, WorkerGroupTaskDispatcher> workerGroupTaskDispatchWaitingQueueLooperMap; ``` ########## dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/WorkerGroupTaskDispatchManager.java: ########## @@ -0,0 +1,117 @@ +/* + * 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 org.apache.dolphinscheduler.server.master.runner.queue.DelayEntry; +import org.apache.dolphinscheduler.server.master.runner.queue.PriorityDelayQueue; + +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; + +@Component +@Slf4j +public class WorkerGroupTaskDispatchManager implements AutoCloseable { + + @Autowired + private ITaskExecutorClient taskExecutorClient; + + @Getter + private final ConcurrentHashMap<String, WorkerGroupTaskDispatchWaitingQueueLooper> workerGroupTaskDispatchWaitingQueueLooperMap; + @Getter + private final ConcurrentHashMap<String, PriorityDelayQueue<DelayEntry<ITaskExecutionRunnable>>> workerGroupPriorityDelayQueueMap; + Review Comment: ```suggestion @Getter private final ConcurrentHashMap<String, PriorityDelayQueue<DelayEntry<ITaskExecutionRunnable>>> workerGroupPriorityDelayQueueMap; ``` -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
