Skip to content

Commit 0e15ce3

Browse files
authored
[Improvement][Metrics] Switch to use tags to indicate task / workflow execution status for metrics (#11128)
* [Improvement][Metrics] Switch to use tags to indicate task / workflow execution status for metrics (#10867) * Update docs and grafana demo dashboards
1 parent d76d6da commit 0e15ce3

14 files changed

Lines changed: 1443 additions & 1525 deletions

File tree

‎docs/docs/en/guide/metrics/metrics.md‎

Lines changed: 16 additions & 14 deletions
Original file line numberDiff line numberDiff line change
@@ -60,14 +60,15 @@ For example, you can get the master metrics by `curl http://localhost:5679/actua
6060

6161
### Task Related Metrics
6262

63-
- ds.task.timeout.count: (counter) the number of timeout tasks
64-
- ds.task.finish.count: (counter) the number of finished tasks, both succeeded and failed included
65-
- ds.task.success.count: (counter) the number of successful tasks
66-
- ds.task.failure.count: (counter) the number of failed tasks
67-
- ds.task.stop.count: (counter) the number of stopped tasks
68-
- ds.task.retry.count: (counter) the number of retried tasks
69-
- ds.task.submit.count: (counter) the number of submitted tasks
70-
- ds.task.failover.count: (counter) the number of task fail-overs
63+
- ds.task.instance.count: (counter) the number of task instances, sliced by the tag `state`:
64+
- timeout: the number of timeout tasks
65+
- finish: the number of finished tasks, both succeeded and failed included
66+
- success: the number of successful tasks
67+
- fail: the number of failed tasks
68+
- stop: the number of stopped tasks
69+
- retry: the number of retried tasks
70+
- submit: the number of submitted tasks
71+
- failover: the number of task fail-overs
7172
- ds.task.dispatch.count: (counter) the number of tasks dispatched to worker
7273
- ds.task.dispatch.failure.count: (counter) the number of tasks failed to dispatch, retry failure included
7374
- ds.task.dispatch.error.count: (counter) the number of task dispatch errors
@@ -83,12 +84,13 @@ For example, you can get the master metrics by `curl http://localhost:5679/actua
8384
- ds.workflow.create.command.count: (counter) the number of commands created and inserted by workflows
8485
- ds.workflow.instance.submit.count: (counter) the number of submitted workflow instances
8586
- ds.workflow.instance.running: (gauge) the number of running workflow instances
86-
- ds.workflow.instance.timeout.count: (counter) the number of timeout workflow instances
87-
- ds.workflow.instance.finish.count: (counter) indicates the number of finished workflow instances, both successes and failures included
88-
- ds.workflow.instance.success.count: (counter) the number of successful workflow instances
89-
- ds.workflow.instance.failure.count: (counter) the number of failed workflow instances
90-
- ds.workflow.instance.stop.count: (counter) the number of stopped workflow instances
91-
- ds.workflow.instance.failover.count: (counter) the number of workflow instance fail-overs
87+
- ds.workflow.instance.count: (counter) the number of workflow instances, sliced by the tag `state`:
88+
- timeout: the number of timeout workflow instances
89+
- finish: the number of finished workflow instances, both successes and failures included
90+
- success: the number of successful workflow instances
91+
- fail: the number of failed workflow instances
92+
- stop: the number of stopped workflow instances
93+
- failover: the number of workflow instance fail-overs
9294

9395
### Master Server Metrics
9496

‎docs/docs/zh/guide/metrics/metrics.md‎

Lines changed: 17 additions & 15 deletions
Original file line numberDiff line numberDiff line change
@@ -61,14 +61,15 @@ metrics exporter端口`server.port`是在application.yaml里定义的: master: `
6161

6262
### 任务相关指标
6363

64-
- ds.task.timeout.count: (counter) 超时的任务数量
65-
- ds.task.finish.count: (counter) 完成的任务数量,成功和失败的任务都算在内
66-
- ds.task.success.count: (counter) 成功完成的任务数量
67-
- ds.task.failure.count: (counter) 失败的任务数量
68-
- ds.task.stop.count: (counter) 暂停的任务数量
69-
- ds.task.retry.count: (counter) 重试的任务数量
70-
- ds.task.submit.count: (counter) 已提交的任务数量
71-
- ds.task.failover.count: (counter) 容错的任务数量
64+
- ds.task.instance.count: (counter) 任务实例数量,由tag `state`按状态切分:
65+
- timeout:超时的任务数量
66+
- finish:完成的任务数量,成功和失败的任务都算在内
67+
- success:成功完成的任务数量
68+
- fail:失败的任务数量
69+
- stop:暂停的任务数量
70+
- retry:重试的任务数量
71+
- submit:已提交的任务数量
72+
- failover:容错的任务数量
7273
- ds.task.dispatch.count: (counter) 分发到worker上的任务数量
7374
- ds.task.dispatch.failure.count: (counter) 分发失败的任务数量,重试也包含在内
7475
- ds.task.dispatch.error.count: (counter) 分发任务的错误数量
@@ -82,14 +83,15 @@ metrics exporter端口`server.port`是在application.yaml里定义的: master: `
8283
### 工作流相关指标
8384

8485
- ds.workflow.create.command.count: (counter) 工作量创建并插入的命令数量
85-
- ds.workflow.instance.submit.count: (counter) 已提交的工作量实例数量
8686
- ds.workflow.instance.running: (gauge) 正在运行的工作流实例数量
87-
- ds.workflow.instance.timeout.count: (counter) 运行超时的工作流实例数量
88-
- ds.workflow.instance.finish.count: (counter) 已完成的工作流实例数量,包含成功和失败
89-
- ds.workflow.instance.success.count: (counter) 运行成功的工作流实例数量
90-
- ds.workflow.instance.failure.count: (counter) 运行失败的工作流实例数量
91-
- ds.workflow.instance.stop.count: (counter) 停止的工作流实例数量
92-
- ds.workflow.instance.failover.count: (counter) 容错的工作流实例数量
87+
- ds.workflow.instance.count: (counter) 工作流实例数量,由tag `state`按状态切分:
88+
- submit:已提交的工作量实例数量
89+
- timeout:运行超时的工作流实例数量
90+
- finish:已完成的工作流实例数量,包含成功和失败
91+
- success:运行成功的工作流实例数量
92+
- fail:运行失败的工作流实例数量
93+
- stop:停止的工作流实例数量
94+
- failover:容错的工作流实例数量
9395

9496
### Master Server指标
9597

‎dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/TaskRetryStateEventHandler.java‎

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -31,7 +31,7 @@ public class TaskRetryStateEventHandler implements StateEventHandler {
3131
@Override
3232
public boolean handleStateEvent(WorkflowExecuteRunnable workflowExecuteRunnable, StateEvent stateEvent)
3333
throws StateEventHandleException {
34-
TaskMetrics.incTaskRetry();
34+
TaskMetrics.incTaskInstanceByState("retry");
3535
Map<Long, TaskInstance> waitToRetryTaskInstanceMap = workflowExecuteRunnable.getWaitToRetryTaskInstanceMap();
3636
TaskInstance taskInstance = waitToRetryTaskInstanceMap.get(stateEvent.getTaskCode());
3737
workflowExecuteRunnable.addTaskToStandByList(taskInstance);

‎dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/TaskStateEventHandler.java‎

Lines changed: 4 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -96,17 +96,17 @@ private void measureTaskState(StateEvent taskStateEvent) {
9696
return;
9797
}
9898
if (taskStateEvent.getExecutionStatus().typeIsFinished()) {
99-
TaskMetrics.incTaskFinish();
99+
TaskMetrics.incTaskInstanceByState("finish");
100100
}
101101
switch (taskStateEvent.getExecutionStatus()) {
102102
case STOP:
103-
TaskMetrics.incTaskStop();
103+
TaskMetrics.incTaskInstanceByState("stop");
104104
break;
105105
case SUCCESS:
106-
TaskMetrics.incTaskSuccess();
106+
TaskMetrics.incTaskInstanceByState("success");
107107
break;
108108
case FAILURE:
109-
TaskMetrics.incTaskFailure();
109+
TaskMetrics.incTaskInstanceByState("fail");
110110
break;
111111
default:
112112
break;

‎dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/TaskTimeoutStateEventHandler.java‎

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -35,7 +35,7 @@ public class TaskTimeoutStateEventHandler implements StateEventHandler {
3535
@Override
3636
public boolean handleStateEvent(WorkflowExecuteRunnable workflowExecuteRunnable, StateEvent stateEvent)
3737
throws StateEventHandleError {
38-
TaskMetrics.incTaskTimeout();
38+
TaskMetrics.incTaskInstanceByState("timeout");
3939
workflowExecuteRunnable.checkTaskInstanceByStateEvent(stateEvent);
4040

4141
TaskInstance taskInstance = workflowExecuteRunnable.getTaskInstance(stateEvent.getTaskInstanceId()).get();

‎dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/WorkflowStartEventHandler.java‎

Lines changed: 1 addition & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -59,8 +59,7 @@ public void handleWorkflowEvent(WorkflowEvent workflowEvent) throws WorkflowEven
5959
"The workflow start event is invalid, cannot find the workflow instance from cache");
6060
}
6161
ProcessInstance processInstance = workflowExecuteRunnable.getProcessInstance();
62-
63-
ProcessInstanceMetrics.incProcessInstanceSubmit();
62+
ProcessInstanceMetrics.incProcessInstanceByState("submit");
6463
CompletableFuture<WorkflowSubmitStatue> workflowSubmitFuture =
6564
CompletableFuture.supplyAsync(workflowExecuteRunnable::call, workflowExecuteThreadPool);
6665
workflowSubmitFuture.thenAccept(workflowSubmitStatue -> {

‎dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/WorkflowStateEventHandler.java‎

Lines changed: 4 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -75,17 +75,17 @@ public StateEventType getEventType() {
7575

7676
private void measureProcessState(StateEvent processStateEvent) {
7777
if (processStateEvent.getExecutionStatus().typeIsFinished()) {
78-
ProcessInstanceMetrics.incProcessInstanceFinish();
78+
ProcessInstanceMetrics.incProcessInstanceByState("finish");
7979
}
8080
switch (processStateEvent.getExecutionStatus()) {
8181
case STOP:
82-
ProcessInstanceMetrics.incProcessInstanceStop();
82+
ProcessInstanceMetrics.incProcessInstanceByState("stop");
8383
break;
8484
case SUCCESS:
85-
ProcessInstanceMetrics.incProcessInstanceSuccess();
85+
ProcessInstanceMetrics.incProcessInstanceByState("success");
8686
break;
8787
case FAILURE:
88-
ProcessInstanceMetrics.incProcessInstanceFailure();
88+
ProcessInstanceMetrics.incProcessInstanceByState("fail");
8989
break;
9090
default:
9191
break;

‎dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/WorkflowTimeoutStateEventHandler.java‎

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -27,7 +27,7 @@
2727
public class WorkflowTimeoutStateEventHandler implements StateEventHandler {
2828
@Override
2929
public boolean handleStateEvent(WorkflowExecuteRunnable workflowExecuteRunnable, StateEvent stateEvent) {
30-
ProcessInstanceMetrics.incProcessInstanceTimeout();
30+
ProcessInstanceMetrics.incProcessInstanceByState("timeout");
3131
workflowExecuteRunnable.processTimeout();
3232
return true;
3333
}

‎dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/metrics/ProcessInstanceMetrics.java‎

Lines changed: 25 additions & 60 deletions
Original file line numberDiff line numberDiff line change
@@ -17,9 +17,14 @@
1717

1818
package org.apache.dolphinscheduler.server.master.metrics;
1919

20+
import java.util.HashMap;
21+
import java.util.Map;
22+
import java.util.Set;
2023
import java.util.concurrent.TimeUnit;
2124
import java.util.function.Supplier;
2225

26+
import com.google.common.collect.ImmutableSet;
27+
2328
import io.micrometer.core.instrument.Counter;
2429
import io.micrometer.core.instrument.Gauge;
2530
import io.micrometer.core.instrument.Metrics;
@@ -31,6 +36,24 @@ private ProcessInstanceMetrics() {
3136
throw new UnsupportedOperationException("Utility class");
3237
}
3338

39+
private static Map<String, Counter> PROCESS_INSTANCE_COUNTERS = new HashMap<>();
40+
41+
private static final Set<String> PROCESS_INSTANCE_STATES = ImmutableSet.of(
42+
"submit", "timeout", "finish", "failover", "success", "fail", "stop");
43+
44+
static {
45+
for (final String state : PROCESS_INSTANCE_STATES) {
46+
PROCESS_INSTANCE_COUNTERS.put(
47+
state,
48+
Counter.builder("ds.workflow.instance.count")
49+
.tag("state", state)
50+
.description(String.format("Process instance %s total count", state))
51+
.register(Metrics.globalRegistry)
52+
);
53+
}
54+
55+
}
56+
3457
private static final Timer COMMAND_QUERY_TIMETER =
3558
Timer.builder("ds.workflow.command.query.duration")
3659
.description("Command query duration")
@@ -41,41 +64,6 @@ private ProcessInstanceMetrics() {
4164
.description("Process instance generated duration")
4265
.register(Metrics.globalRegistry);
4366

44-
private static final Counter PROCESS_INSTANCE_SUBMIT_COUNTER =
45-
Counter.builder("ds.workflow.instance.submit.count")
46-
.description("Process instance submit total count")
47-
.register(Metrics.globalRegistry);
48-
49-
private static final Counter PROCESS_INSTANCE_TIMEOUT_COUNTER =
50-
Counter.builder("ds.workflow.instance.timeout.count")
51-
.description("Process instance timeout total count")
52-
.register(Metrics.globalRegistry);
53-
54-
private static final Counter PROCESS_INSTANCE_FINISH_COUNTER =
55-
Counter.builder("ds.workflow.instance.finish.count")
56-
.description("Process instance finish total count")
57-
.register(Metrics.globalRegistry);
58-
59-
private static final Counter PROCESS_INSTANCE_SUCCESS_COUNTER =
60-
Counter.builder("ds.workflow.instance.success.count")
61-
.description("Process instance success total count")
62-
.register(Metrics.globalRegistry);
63-
64-
private static final Counter PROCESS_INSTANCE_FAILURE_COUNTER =
65-
Counter.builder("ds.workflow.instance.failure.count")
66-
.description("Process instance failure total count")
67-
.register(Metrics.globalRegistry);
68-
69-
private static final Counter PROCESS_INSTANCE_STOP_COUNTER =
70-
Counter.builder("ds.workflow.instance.stop.count")
71-
.description("Process instance stop total count")
72-
.register(Metrics.globalRegistry);
73-
74-
private static final Counter PROCESS_INSTANCE_FAILOVER_COUNTER =
75-
Counter.builder("ds.workflow.instance.failover.count")
76-
.description("Process instance failover total count")
77-
.register(Metrics.globalRegistry);
78-
7967
public static void recordCommandQueryTime(long milliseconds) {
8068
COMMAND_QUERY_TIMETER.record(milliseconds, TimeUnit.MILLISECONDS);
8169
}
@@ -96,31 +84,8 @@ public static synchronized void registerProcessInstanceResubmitGauge(Supplier<Nu
9684
.register(Metrics.globalRegistry);
9785
}
9886

99-
public static void incProcessInstanceSubmit() {
100-
PROCESS_INSTANCE_SUBMIT_COUNTER.increment();
101-
}
102-
103-
public static void incProcessInstanceTimeout() {
104-
PROCESS_INSTANCE_TIMEOUT_COUNTER.increment();
105-
}
106-
107-
public static void incProcessInstanceFinish() {
108-
PROCESS_INSTANCE_FINISH_COUNTER.increment();
87+
public static void incProcessInstanceByState(final String state) {
88+
PROCESS_INSTANCE_COUNTERS.get(state).increment();
10989
}
11090

111-
public static void incProcessInstanceSuccess() {
112-
PROCESS_INSTANCE_SUCCESS_COUNTER.increment();
113-
}
114-
115-
public static void incProcessInstanceFailure() {
116-
PROCESS_INSTANCE_FAILURE_COUNTER.increment();
117-
}
118-
119-
public static void incProcessInstanceStop() {
120-
PROCESS_INSTANCE_STOP_COUNTER.increment();
121-
}
122-
123-
public static void incProcessInstanceFailover() {
124-
PROCESS_INSTANCE_FAILOVER_COUNTER.increment();
125-
}
12691
}

0 commit comments

Comments
 (0)