Skip to content
Merged
Show file tree
Hide file tree
Changes from 1 commit
Commits
Show all changes
52 commits
Select commit Hold shift + click to select a range
00b8dc6
Add dispatch timeout checking logic
Dec 15, 2025
bb089f3
add dispatch-timeout-checker switch
Dec 17, 2025
fbd7edf
add dispatch-timeout-checker switch
Dec 17, 2025
ab84035
update ExceptionUtils
Dec 17, 2025
f193af6
add test for dispatch timeout checker
Dec 17, 2025
ab750bb
Merge branch 'apache:dev' into Improvement-17795
njnu-seafish Dec 19, 2025
584ad2d
add TaskFatalLifecycleEvent and handler
Dec 19, 2025
877db06
add TaskFatalLifecycleEvent and handler
Dec 19, 2025
cb44db7
Add dispatch timeout checking logic
Dec 15, 2025
1aa12df
add dispatch-timeout-checker switch
Dec 17, 2025
0c29a82
add dispatch-timeout-checker switch
Dec 17, 2025
6ad77d1
update ExceptionUtils
Dec 17, 2025
b760003
add test for dispatch timeout checker
Dec 17, 2025
39b40c1
add TaskFatalLifecycleEvent and handler
Dec 19, 2025
d7bf4fa
add TaskFatalLifecycleEvent and handler
Dec 19, 2025
0c8c93e
Merge branch 'apache:dev' into Improvement-17795
njnu-seafish Dec 22, 2025
5e1dd52
Merge branch 'Improvement-17795' of github.com:njnu-seafish/dolphinsc…
Dec 22, 2025
5b2722d
update TaskDispatchPolicy
Dec 22, 2025
33c16e8
add it test case
Dec 23, 2025
3c25528
Merge branch 'dev' into Improvement-17795
njnu-seafish Dec 23, 2025
99a5d7c
update test
Dec 23, 2025
9966047
Merge branch 'Improvement-17795' of github.com:njnu-seafish/dolphinsc…
Dec 23, 2025
42a5aa2
Merge branch 'apache:dev' into Improvement-17795
njnu-seafish Jan 8, 2026
879ba5e
update test
Jan 8, 2026
77a5dfa
Merge branch 'dev' into Improvement-17795
njnu-seafish Jan 8, 2026
9a6d664
Merge branch 'dev' into Improvement-17795
ruanwenjun Jan 12, 2026
cca935f
Merge branch 'apache:dev' into Improvement-17795
njnu-seafish Jan 12, 2026
e49549e
update test
Jan 12, 2026
b5a703b
update test
Jan 12, 2026
9f97478
Remove this test case and track the fix in a separate issue
Jan 13, 2026
ddb7eb7
add it test case by using SHELL task
Jan 13, 2026
93e398c
update test comment
Jan 13, 2026
6864a2e
Merge branch 'dev' into Improvement-17795
njnu-seafish Jan 13, 2026
f7c7fe1
update task instance state
Jan 13, 2026
7437249
remove masterContainer.assertAllResourceReleased if workflow is running
Jan 14, 2026
99d2640
update test
Jan 14, 2026
a6302d3
Merge branch 'dev' into Improvement-17795
njnu-seafish Jan 14, 2026
7a01fab
Resolve merge conflicts from dev branch
Jan 14, 2026
8c4700a
update test
Jan 15, 2026
d6f8ccd
update test
Jan 15, 2026
5a47e02
Merge branch 'apache:dev' into Improvement-17795
njnu-seafish Jan 22, 2026
743dcbc
remove redundant tests and comments
Jan 22, 2026
62cfb30
remove redundant comments
Jan 22, 2026
edf818c
Merge branch 'apache:dev' into Improvement-17795
njnu-seafish Jan 23, 2026
f772a55
update comment and test
Jan 23, 2026
5f19c7b
update test
Jan 26, 2026
e12af5d
Merge branch 'dev' into Improvement-17795
njnu-seafish Jan 27, 2026
65e7359
add dispatch timeout config in doc
Jan 29, 2026
6f103b3
Merge branch 'dev' into Improvement-17795
ruanwenjun Jan 29, 2026
099d3df
Merge branch 'dev' into Improvement-17795
ruanwenjun Feb 1, 2026
0c295a7
Merge branch 'dev' into Improvement-17795
njnu-seafish Feb 2, 2026
aed9460
Merge branch 'dev' into Improvement-17795
njnu-seafish Feb 2, 2026
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -74,6 +74,12 @@ public class MasterConfig implements Validator {
*/
private String masterRegistryPath;

/**
* Maximum time allowed for a task to be successfully dispatched.
* Default: 5 minutes.
*/
private Duration dispatchTimeout = Duration.ofMinutes(5);
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We should make it configurable by users in the configuration file and it can be turned off.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We should make it configurable by users in the configuration file and it can be turned off.

good idea!

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Suggested change
private Duration dispatchTimeout = Duration.ofMinutes(5);
private Duration maxTaskDispatchDuration;

dispatchTimeout might confuse with single RPC timeout when dispatch task, and the default value should be null or a large duration, should compatibility with the previous behavior.

And you need to add this in document.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

maxTaskDispatchDuration

ok


@Override
public boolean supports(Class<?> clazz) {
return MasterConfig.class.isAssignableFrom(clazz);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,10 +24,13 @@
import org.apache.dolphinscheduler.extract.base.utils.Host;
import org.apache.dolphinscheduler.extract.worker.IPhysicalTaskExecutorOperator;
import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
import org.apache.dolphinscheduler.server.master.cluster.ClusterManager;
import org.apache.dolphinscheduler.server.master.cluster.loadbalancer.IWorkerLoadBalancer;
import org.apache.dolphinscheduler.server.master.config.MasterConfig;
import org.apache.dolphinscheduler.server.master.engine.task.runnable.ITaskExecutionRunnable;
import org.apache.dolphinscheduler.server.master.exception.dispatch.TaskDispatchException;
import org.apache.dolphinscheduler.server.master.exception.dispatch.WorkerGroupNotFoundException;
import org.apache.dolphinscheduler.server.master.exception.dispatch.WorkerNotFoundException;
import org.apache.dolphinscheduler.task.executor.eventbus.ITaskExecutorLifecycleEventReporter;
import org.apache.dolphinscheduler.task.executor.operations.TaskExecutorDispatchRequest;
import org.apache.dolphinscheduler.task.executor.operations.TaskExecutorDispatchResponse;
Expand Down Expand Up @@ -55,18 +58,29 @@ public class PhysicalTaskExecutorClientDelegator implements ITaskExecutorClientD
@Autowired
private IWorkerLoadBalancer workerLoadBalancer;

@Autowired
private ClusterManager clusterManager;

@Override
public void dispatch(final ITaskExecutionRunnable taskExecutionRunnable) throws TaskDispatchException {
final TaskExecutionContext taskExecutionContext = taskExecutionRunnable.getTaskExecutionContext();
final String taskName = taskExecutionContext.getTaskName();
final String workerGroup = taskExecutionContext.getWorkerGroup();
if (!clusterManager.getWorkerClusters().containsWorkerGroup(workerGroup)) {
throw new WorkerGroupNotFoundException(
String.format("Cannot find worker group to dispatch Task[id=%s, name=%s, workerGroup=%s]",
taskExecutionContext.getTaskInstanceId(), taskName,
workerGroup));
}
final String physicalTaskExecutorAddress = workerLoadBalancer
.select(taskExecutionContext.getWorkerGroup())
.select(workerGroup)
.map(Host::of)
.map(Host::getAddress)
.orElseThrow(() -> new TaskDispatchException(
String.format("Cannot find the host to dispatch Task[id=%s, name=%s, workerGroup=%s]",
.orElseThrow(() -> new WorkerNotFoundException(
String.format(
"Cannot find available worker host to dispatch Task[id=%s, name=%s, workerGroup=%s]",
taskExecutionContext.getTaskInstanceId(), taskName,
taskExecutionContext.getWorkerGroup())));
workerGroup)));

taskExecutionContext.setHost(physicalTaskExecutorAddress);
taskExecutionRunnable.getTaskInstance().setHost(physicalTaskExecutorAddress);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,12 +18,19 @@
package org.apache.dolphinscheduler.server.master.engine.task.dispatcher;

import org.apache.dolphinscheduler.common.thread.BaseDaemonThread;
import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
import org.apache.dolphinscheduler.plugin.task.api.utils.LogUtils;
import org.apache.dolphinscheduler.server.master.engine.task.client.ITaskExecutorClient;
import org.apache.dolphinscheduler.server.master.engine.task.dispatcher.event.TaskDispatchableEvent;
import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.event.TaskFailedLifecycleEvent;
import org.apache.dolphinscheduler.server.master.engine.task.lifecycle.event.TaskFatalLifecycleEvent;
import org.apache.dolphinscheduler.server.master.engine.task.runnable.ITaskExecutionRunnable;
import org.apache.dolphinscheduler.server.master.exception.dispatch.TaskDispatchException;
import org.apache.dolphinscheduler.server.master.utils.ExceptionUtils;
import org.apache.dolphinscheduler.task.executor.log.TaskExecutorMDCUtils;

import java.time.Duration;
import java.util.Date;
import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.atomic.AtomicBoolean;
Expand All @@ -48,11 +55,15 @@ public class WorkerGroupDispatcher extends BaseDaemonThread {

private final AtomicBoolean runningFlag = new AtomicBoolean(false);

public WorkerGroupDispatcher(String workerGroupName, ITaskExecutorClient taskExecutorClient) {
private final Duration dispatchTimeout;

public WorkerGroupDispatcher(String workerGroupName, ITaskExecutorClient taskExecutorClient,
Duration dispatchTimeout) {
super("WorkerGroupTaskDispatcher-" + workerGroupName);
this.taskExecutorClient = taskExecutorClient;
this.workerGroupEventBus = new TaskDispatchableEventBus<>();
this.waitingDispatchTaskIds = ConcurrentHashMap.newKeySet();
this.dispatchTimeout = dispatchTimeout;
log.info("Initialize WorkerGroupDispatcher: {}", this.getName());
}

Expand Down Expand Up @@ -84,23 +95,77 @@ public void run() {
}

private void doDispatchTask(ITaskExecutionRunnable taskExecutionRunnable) {
final int taskId = taskExecutionRunnable.getId();
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Suggested change
final int taskId = taskExecutionRunnable.getId();
final int taskExecutionRunnableId = taskExecutionRunnable.getId();

Avoid confuse with task definition id.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

taskExecutionRunnableId

final int taskInstanceId = taskExecutionRunnable.getId(); is it better?

/**
* Get the task instance id.
*

Need to know the id might change since the task instance might be regenerated.
*/
default int getId() {
return getTaskInstance().getId();
}

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It's better to use taskExecutionRunnableId , since the outer layer is unaware that this is the taskInstanceId.

final TaskExecutionContext taskExecutionContext = taskExecutionRunnable.getTaskExecutionContext();
final long timeoutMs = this.dispatchTimeout.toMillis();
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It's better to store the mills rather than store duration, then you can avoid execute dispatchTimeout.toMillis() here.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It's better to store the mills rather than store duration, then you can avoid execute dispatchTimeout.toMillis() here.

Using a duration format like '5m' is more concise and readable. Most other configurations I’ve seen also avoid millisecond-level granularity.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

You can translate the duration into milliseconds in the constructor.

try {
if (!waitingDispatchTaskIds.remove(taskExecutionRunnable.getId())) {
if (!waitingDispatchTaskIds.remove(taskId)) {
log.info(
"The task: {} doesn't exist in waitingDispatchTaskIds(it might be paused or killed), will skip dispatch",
taskExecutionRunnable.getId());
taskId);
return;
}
taskExecutorClient.dispatch(taskExecutionRunnable);
} catch (Exception e) {
} catch (TaskDispatchException ex) {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Suggested change
} catch (TaskDispatchException ex) {
} catch (Exception ex) {

Currently, at least in this PR, you can't guarantee that other exceptions won't be thrown here, right?

// Checks whether the given task has exceeded its allowed dispatch timeout.
long elapsed = System.currentTimeMillis() - taskExecutionContext.getFirstDispatchEnqueueTimeMs();
if (elapsed > timeoutMs) {
handleDispatchFailure(taskExecutionRunnable, ex, elapsed, timeoutMs);
return;
}

// 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(
long waitingTimeMillis = Math.min(
taskExecutionRunnable.getTaskExecutionContext().increaseDispatchFailTimes() * 1_000L, 60_000L);
dispatchTask(taskExecutionRunnable, waitingTimeMills);
log.error("Dispatch Task: {} failed will retry after: {}/ms", taskExecutionRunnable.getId(),
waitingTimeMills, e);
dispatchTask(taskExecutionRunnable, waitingTimeMillis);
log.warn("Dispatch Task: {} failed will retry after: {}/ms", taskId,
waitingTimeMillis, ex);
}
}

/**
* Marks the specified task as fatally failed due to an unrecoverable dispatch error,such as timeout or persistent client failure.
* Once this method is called, the task is considered permanently failed and will not be retried.
*
* @param taskExecutionRunnable the task to mark as fatally failed; must not be null
* @param exception the dispatch exception that triggered this failure handling; must not be null
* @param elapsed the time (in milliseconds) already spent attempting to dispatch the task
* @param timeoutMs the configured dispatch timeout threshold (in milliseconds)
*/
private void handleDispatchFailure(ITaskExecutionRunnable taskExecutionRunnable, TaskDispatchException exception,
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Please don't print the taskId and workflowId here, all ids should already be added by MDC. We should only need to print the exception here, the exception already contains failure message.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Please don't print the taskId and workflowId here, all ids should already be added by MDC. We should only need to print the exception here, the exception already contains failure message.

ok

long elapsed, long timeoutMs) {
int taskId = taskExecutionRunnable.getId();
int workflowId = taskExecutionRunnable.getWorkflowInstance().getId();

log.warn("[DISPATCH_FAILED] taskId: {}, workflowId: {}, timed out after {} ms (limit: {} ms))", taskId,
workflowId, elapsed, timeoutMs);

if (ExceptionUtils.isWorkerGroupNotFoundException(exception)) {
log.error("[DISPATCH_FAILED] Worker group not found. taskId: {}, workflowId: {}", taskId, workflowId,
exception);
final TaskFatalLifecycleEvent taskFatalEvent = TaskFatalLifecycleEvent.builder()
.taskExecutionRunnable(taskExecutionRunnable)
.endTime(new Date())
.build();
taskExecutionRunnable.getWorkflowEventBus().publish(taskFatalEvent);
} else if (ExceptionUtils.isWorkerNotFoundException(exception)) {
log.error("[DISPATCH_FAILED] No available workers. taskId: {}, workflowId: {}", taskId, workflowId,
exception);
final TaskFailedLifecycleEvent taskFailedEvent = TaskFailedLifecycleEvent.builder()
.taskExecutionRunnable(taskExecutionRunnable)
.endTime(new Date())
.build();
taskExecutionRunnable.getWorkflowEventBus().publish(taskFailedEvent);
} else {
log.error("[DISPATCH_FAILED] Unexpected dispatch error. taskId: {}, workflowId: {}", taskId, workflowId,
exception);
final TaskFailedLifecycleEvent taskFailedEvent = TaskFailedLifecycleEvent.builder()
.taskExecutionRunnable(taskExecutionRunnable)
.endTime(new Date())
.build();
taskExecutionRunnable.getWorkflowEventBus().publish(taskFailedEvent);
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@

package org.apache.dolphinscheduler.server.master.engine.task.dispatcher;

import org.apache.dolphinscheduler.server.master.config.MasterConfig;
import org.apache.dolphinscheduler.server.master.engine.task.client.ITaskExecutorClient;
import org.apache.dolphinscheduler.server.master.engine.task.runnable.ITaskExecutionRunnable;

Expand All @@ -41,8 +42,11 @@ public class WorkerGroupDispatcherCoordinator implements AutoCloseable {

private final ConcurrentHashMap<String, WorkerGroupDispatcher> workerGroupDispatcherMap;

public WorkerGroupDispatcherCoordinator() {
private final MasterConfig masterConfig;

public WorkerGroupDispatcherCoordinator(final MasterConfig masterConfig) {
workerGroupDispatcherMap = new ConcurrentHashMap<>();
this.masterConfig = masterConfig;
}

public void start() {
Expand Down Expand Up @@ -99,7 +103,8 @@ public void close() throws Exception {

private WorkerGroupDispatcher getOrCreateWorkerGroupDispatcher(String workerGroup) {
return workerGroupDispatcherMap.computeIfAbsent(workerGroup, wg -> {
WorkerGroupDispatcher workerGroupDispatcher = new WorkerGroupDispatcher(wg, taskExecutorClient);
WorkerGroupDispatcher workerGroupDispatcher =
new WorkerGroupDispatcher(wg, taskExecutorClient, masterConfig.getDispatchTimeout());
workerGroupDispatcher.start();
return workerGroupDispatcher;
});
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,7 @@

public class WorkerGroupNotFoundException extends TaskDispatchException {

public WorkerGroupNotFoundException(String workerGroup) {
super("Cannot find worker group: " + workerGroup);
public WorkerGroupNotFoundException(String message) {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Why do this change?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Why do this change?

restore, remove the task info

super(message);
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,25 @@
/*
* 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.exception.dispatch;

public class WorkerNotFoundException extends TaskDispatchException {

public WorkerNotFoundException(String message) {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Suggested change
public WorkerNotFoundException(String message) {
public NoAvailableWorkerException(String workerGroup) {
super("Cannot find available worker under worker group: " + workerGroup);
}

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

ok

super(message);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,9 @@

package org.apache.dolphinscheduler.server.master.utils;

import org.apache.dolphinscheduler.server.master.exception.dispatch.WorkerGroupNotFoundException;
import org.apache.dolphinscheduler.server.master.exception.dispatch.WorkerNotFoundException;

import org.springframework.dao.DataAccessResourceFailureException;

public class ExceptionUtils {
Expand All @@ -25,4 +28,11 @@ public static boolean isDatabaseConnectedFailedException(Throwable e) {
return e instanceof DataAccessResourceFailureException;
}

public static boolean isWorkerGroupNotFoundException(Throwable e) {
return e instanceof WorkerGroupNotFoundException;
}

public static boolean isWorkerNotFoundException(Throwable e) {
return e instanceof WorkerNotFoundException;
}
}
3 changes: 3 additions & 0 deletions dolphinscheduler-master/src/main/resources/application.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -111,6 +111,9 @@ master:
# Master max concurrent workflow instances, when the master's workflow instance count exceeds this value, master server will be marked as busy.
max-concurrent-workflow-instances: 2147483647
worker-group-refresh-interval: 5m
# Timeout for task dispatching. If a task cannot be successfully dispatched to any worker within this duration,
# it will be marked as failed. Prevents tasks from being stuck indefinitely in the dispatch queue.
dispatch-timeout: 5m
command-fetch-strategy:
type: ID_SLOT_BASED
config:
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -28,6 +28,7 @@

import org.apache.dolphinscheduler.dao.entity.TaskInstance;
import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
import org.apache.dolphinscheduler.server.master.config.MasterConfig;
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.exception.dispatch.TaskDispatchException;
Expand All @@ -46,7 +47,8 @@ class WorkerGroupDispatcherTest {
@BeforeEach
void setUp() {
taskExecutorClient = mock(ITaskExecutorClient.class);
dispatcher = new WorkerGroupDispatcher("TestGroup", taskExecutorClient);
final MasterConfig masterConfig = new MasterConfig();
dispatcher = new WorkerGroupDispatcher("TestGroup", taskExecutorClient, masterConfig.getDispatchTimeout());
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

You should add a validated test case.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

You should add a validated test case.

ok, add test for dispatch timeout checker

Copy link
Contributor Author

@njnu-seafish njnu-seafish Jan 13, 2026

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

You should add a validated test case.

I've already validated and tested it in a real environment.
The LogicFakeTask type runs on the Master and does not involve WorkerGroup dispatching.
I track the fix in a separate issue: #17872

}

@Test
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -126,6 +126,11 @@ public class TaskExecutionContext implements Serializable {

private boolean failover;

/**
* Timestamp (ms) when the task was first enqueued for dispatch.
*/
private final long firstDispatchEnqueueTimeMs = System.currentTimeMillis();
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Suggested change
/**
* Timestamp (ms) when the task was first enqueued for dispatch.
*/
private final long firstDispatchEnqueueTimeMs = System.currentTimeMillis();
private final long firstDispatchTime = System.currentTimeMillis();

This is more like the creation time of the taskExecutionContext.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This is more like the creation time of the taskExecutionContext.

ok


public int increaseDispatchFailTimes() {
return ++dispatchFailTimes;
}
Expand Down
Loading