You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@dolphinscheduler.apache.org by ca...@apache.org on 2022/06/09 02:55:48 UTC

[dolphinscheduler] branch dev updated: [Feature][metrics] Add master, worker metrics (#10326)

This is an automated email from the ASF dual-hosted git repository.

caishunfeng pushed a commit to branch dev
in repository https://gitbox.apache.org/repos/asf/dolphinscheduler.git


The following commit(s) were added to refs/heads/dev by this push:
     new e21d7b1551 [Feature][metrics] Add master, worker metrics (#10326)
e21d7b1551 is described below

commit e21d7b1551c891caa5aeb8d3d61407e8cced7107
Author: Wenjun Ruan <we...@apache.org>
AuthorDate: Thu Jun 9 10:55:39 2022 +0800

    [Feature][metrics] Add master, worker metrics (#10326)
    
    * Add mater metrics
    
    * fix UT
    
    * Add url to mysql profile
    
    * Add worker metrics
    
    * Update grafana config
    
    * Add system metrics doc
    
    * Add process failover counter
    
    * Add metrics image
    
    * Change jpg to png
    
    * Add command insert metrics
    
    * Fix UT
    
    * Revert UT
---
 docs/docs/en/guide/metrics/metrics.md              |  154 ++
 docs/img/metrics/metrics-datasource.png            |  Bin 0 -> 336581 bytes
 docs/img/metrics/metrics-master.png                |  Bin 0 -> 467778 bytes
 docs/img/metrics/metrics-worker.png                |  Bin 0 -> 403432 bytes
 .../src/main/resources/banner.txt                  |    2 +-
 .../src/main/resources/application.yaml            |    3 +
 dolphinscheduler-api/src/main/resources/banner.txt |    2 +-
 .../apache/dolphinscheduler/common/Constants.java  |    8 +-
 .../dolphinscheduler/common/model/TaskNode.java    |    2 +-
 .../impl/ProcessInstanceExecCacheManagerImpl.java  |    8 +
 .../master/consumer/TaskPriorityQueueConsumer.java |   13 +-
 .../server/master/metrics/MasterServerMetrics.java |   53 +
 .../master/metrics/ProcessInstanceMetrics.java     |  101 +
 .../server/master/metrics/TaskMetrics.java         |  137 +
 .../master/runner/FailoverExecuteThread.java       |    2 +-
 .../master/runner/MasterSchedulerService.java      |    4 +-
 .../master/runner/WorkflowExecuteRunnable.java     |   91 +-
 .../master/runner/WorkflowExecuteThreadPool.java   |   18 +-
 .../master/runner/task/CommonTaskProcessor.java    |    6 +-
 .../master/runner/task/TaskProcessorFactory.java   |   15 +-
 .../server/master/service/FailoverService.java     |    5 +-
 .../src/main/resources/application.yaml            |    3 +
 .../src/main/resources/banner.txt                  |    2 +-
 .../server/master/WorkflowExecuteTaskTest.java     |    4 -
 .../master/registry/ServerNodeManagerTest.java     |   60 -
 .../dolphinscheduler/meter/MeterConfiguration.java |   16 +
 .../resources/grafana-demo/docker-compose.yaml     |    5 +-
 .../resources/grafana/DolphinSchedulerMaster.json  | 2607 ++++++++++++++++----
 .../resources/grafana/DolphinSchedulerWorker.json  | 1598 ++++++++++++
 .../processor/StateEventCallbackService.java       |    2 +-
 .../service/process/ProcessServiceImpl.java        |    2 +
 .../src/main/resources/banner.txt                  |    2 +-
 .../server/worker/metrics/TaskMetrics.java         |   58 +
 .../server/worker/metrics/WorkerServerMetrics.java |   56 +
 .../worker/processor/TaskCallbackService.java      |    2 +-
 .../worker/processor/TaskExecuteProcessor.java     |    9 +-
 .../server/worker/runner/WorkerExecService.java    |    3 +
 .../server/worker/runner/WorkerManagerThread.java  |    5 +-
 .../src/main/resources/application.yaml            |    3 +
 .../src/main/resources/banner.txt                  |    2 +-
 40 files changed, 4496 insertions(+), 567 deletions(-)

diff --git a/docs/docs/en/guide/metrics/metrics.md b/docs/docs/en/guide/metrics/metrics.md
new file mode 100644
index 0000000000..0f75db0ef4
--- /dev/null
+++ b/docs/docs/en/guide/metrics/metrics.md
@@ -0,0 +1,154 @@
+# Introduction
+
+Apache DolphinScheduler has export some metrics to monitor the system. We use micrometer for the exporter facade, and
+the default exporter is prometheus, more exporter is coming soon.
+
+## Quick Start
+
+You can add the following config in master/worker/alert/api's yaml file to open the metrics exporter.
+
+```yaml
+metrics:
+  enabled: true
+```
+
+Once you open the metrics exporter, you can access the metrics by the url: `http://ip:port/actuator/prometheus`
+
+The exporter port is the `server.port` defined in application.yaml, e.g: master: `server.port: 5679`, worker: `server.port: 1235`, alert: `server.port: 50053`, api: `server.port: 12345`.
+
+For example, you can get the master metrics by `curl http://localhost:5679/actuator/prometheus`
+
+We have prepared the out-of-the-box Grafana configuration for you, you can find the Grafana dashboard
+at `dolphinscheduler-meter/resources/grafana`, you can directly import these dashboards to grafana.
+
+If you want to try at docker, you can use the following command to start the prometheus with grafana:
+
+```shell
+cd dolphinscheduler-meter/src/main/resources/grafana-demo
+docker compose up
+```
+
+Then you can access the grafana by the url: `http://localhost/3001`
+
+![image.png](../../../../img/metrics/metrics-master.png)
+![image.png](../../../../img/metrics/metrics-worker.png)
+![image.png](../../../../img/metrics/metrics-datasource.png)
+
+## Master Metrics
+
+Master metrics are exported by the DolphinScheduler master server.
+
+### System Metrics
+
+* dolphinscheduler_master_overload_count: Indicates the number of times the master has been overloaded.
+* dolphinscheduler_master_consume_command_count: Indicates the number of commands has consumed.
+
+### Process Metrics
+
+* dolphinscheduler_create_command_count: Indicates the number of command has been inserted.
+* dolphinscheduler_process_instance_submit_count: Indicates the number of process has been submitted.
+* dolphinscheduler_process_instance_running_gauge: Indicates the number of process are running now.
+* dolphinscheduler_process_instance_timeout_count: Indicates the number of process has been timeout.
+* dolphinscheduler_process_instance_finish_count: Indicates the number of process has been finished, include success or
+  failure.
+* dolphinscheduler_process_instance_success_count: Indicates the number of process has been successful.
+* dolphinscheduler_process_instance_stop_count: Indicates the number of process has been stopped.
+* dolphinscheduler_process_instance_failover_count: Indicates the number of process has been failed over.
+
+### Task Metrics
+
+* dolphinscheduler_task_timeout_count: Indicates the number of tasks has been timeout.
+* dolphinscheduler_task_finish_count: Indicates the number of tasks has been finished, include success or failure.
+* dolphinscheduler_task_success_count: Indicates the number of tasks has been successful.
+* dolphinscheduler_task_timeout_count: Indicates the number of tasks has been timeout.
+* dolphinscheduler_task_retry_count: Indicates the number of tasks has been retry.
+* dolphinscheduler_task_failover_count: Indicates the number of tasks has been failover.
+* dolphinscheduler_task_dispatch_count: Indicates the number of tasks has been dispatched to worker.
+* dolphinscheduler_task_dispatch_failed_count: Indicates the number of tasks dispatched failed, if dispatched failed
+  will retry.
+* dolphinscheduler_task_dispatch_error_count: Indicates the number of tasks dispatched error, if dispatched error, means
+  there are exception occur.
+
+## Worker Metrics
+
+Worker metrics are exported by the DolphinScheduler worker server.
+
+### System Metrics
+
+* dolphinscheduler_worker_overload_count: Indicates the number of times the worker has been overloaded.
+* dolphinscheduler_worker_submit_queue_is_full_count: Indicates the number of times the worker's submit queue has been
+  full.
+
+### Task Metrics
+
+* dolphinscheduler_task_execute_count: Indicates the number of times a task has been executed, it contains a tag -
+  `task_type`.
+* dolphinscheduler_task_execution_count: Indicates the total number of task has been executed.
+* dolphinscheduler_task_execution_timer: Indicates the time spent executing tasks.
+
+## Default System Metrics
+
+In each server, there are some default metrics related to the system instance.
+
+### Database Metrics
+
+* hikaricp_connections_creation_seconds_max: Connection creation time max.
+* hikaricp_connections_creation_seconds_count: Connection creation time count.
+* hikaricp_connections_creation_seconds_sum: Connection creation time sum.
+* hikaricp_connections_acquire_seconds_max: Connection acquire time max.
+* hikaricp_connections_acquire_seconds_count: Connection acquire time count.
+* hikaricp_connections_acquire_seconds_sum: Connection acquire time sum.
+* hikaricp_connections_usage_seconds_max: Connection usage max.
+* hikaricp_connections_usage_seconds_count: Connection usage time count.
+* hikaricp_connections_usage_seconds_sum: Connection usage time sum.
+* hikaricp_connections_max: Max connections.
+* hikaricp_connections_min Min connections
+* hikaricp_connections_active: Active connections.
+* hikaricp_connections_idle: Idle connections.
+* hikaricp_connections_pending: Pending connections.
+* hikaricp_connections_timeout_total: Timeout connections.
+* hikaricp_connections: Total connections
+* jdbc_connections_max: Maximum number of active connections that can be allocated at the same time.
+* jdbc_connections_min: Minimum number of idle connections in the pool.
+* jdbc_connections_idle: Number of established but idle connections.
+* jdbc_connections_active: Current number of active connections that have been allocated from the data source.
+
+### JVM Metrics
+
+* jvm_buffer_total_capacity_bytes: An estimate of the total capacity of the buffers in this pool.
+* jvm_buffer_count_buffers: An estimate of the number of buffers in the pool.
+* jvm_buffer_memory_used_bytes: An estimate of the memory that the Java virtual machine is using for this buffer pool.
+* jvm_memory_committed_bytes: The amount of memory in bytes that is committed for the Java virtual machine to use.
+* jvm_memory_max_bytes: The maximum amount of memory in bytes that can be used for memory management.
+* jvm_memory_used_bytes: The amount of used memory.
+* jvm_threads_peak_threads: The peak live thread count since the Java virtual machine started or peak was reset.
+* jvm_threads_states_threads: The current number of threads having NEW state.
+* jvm_gc_memory_allocated_bytes_total: Incremented for an increase in the size of the (young) heap memory pool after one GC to before the next.
+* jvm_gc_max_data_size_bytes: Max size of long-lived heap memory pool.
+* jvm_gc_pause_seconds_count: Time spent count in GC pause.
+* jvm_gc_pause_seconds_sum: Time spent sum in GC pause.
+* jvm_gc_pause_seconds_max: Time spent max in GC pause.
+* jvm_gc_live_data_size_bytes: Size of long-lived heap memory pool after reclamation.
+* jvm_gc_memory_promoted_bytes_total: Count of positive increases in the size of the old generation memory pool before GC to after GC.
+* jvm_classes_loaded_classes: The number of classes that are currently loaded in the Java virtual machine.
+* jvm_threads_live_threads: The current number of live threads including both daemon and non-daemon threads.
+* jvm_threads_daemon_threads: The current number of live daemon threads.
+* jvm_classes_unloaded_classes_total: The total number of classes unloaded since the Java virtual machine has started execution.
+* process_cpu_usage: The "recent cpu usage" for the Java Virtual Machine process.
+* process_start_time_seconds: Start time of the process since unix epoch.
+* process_uptime_seconds: The uptime of the Java virtual machine.
+
+
+## Other Metrics
+* jetty_threads_config_max: The maximum number of threads in the pool.
+* jetty_threads_config_min: The minimum number of threads in the pool.
+* jetty_threads_current: The total number of threads in the pool.
+* jetty_threads_idle: The number of idle threads in the pool.
+* jetty_threads_busy: The number of busy threads in the pool.
+* jetty_threads_jobs: Number of jobs queued waiting for a thread.
+* process_files_max_files: The maximum file descriptor count.
+* process_files_open_files: The open file descriptor count.
+* system_cpu_usage: The "recent cpu usage" for the whole system.
+* system_cpu_count: The number of processors available to the Java virtual machine.
+* system_load_average_1m: The sum of the number of runnable entities queued to available processors and the number of runnable entities running on the available processors averaged over a period of time.
+* logback_events_total: Number of level events that made it to the logs
\ No newline at end of file
diff --git a/docs/img/metrics/metrics-datasource.png b/docs/img/metrics/metrics-datasource.png
new file mode 100644
index 0000000000..7bc96ddd4e
Binary files /dev/null and b/docs/img/metrics/metrics-datasource.png differ
diff --git a/docs/img/metrics/metrics-master.png b/docs/img/metrics/metrics-master.png
new file mode 100644
index 0000000000..af8c44947e
Binary files /dev/null and b/docs/img/metrics/metrics-master.png differ
diff --git a/docs/img/metrics/metrics-worker.png b/docs/img/metrics/metrics-worker.png
new file mode 100644
index 0000000000..a45fb93f1a
Binary files /dev/null and b/docs/img/metrics/metrics-worker.png differ
diff --git a/dolphinscheduler-alert/dolphinscheduler-alert-server/src/main/resources/banner.txt b/dolphinscheduler-alert/dolphinscheduler-alert-server/src/main/resources/banner.txt
index 1b0e621288..8353e2c083 100644
--- a/dolphinscheduler-alert/dolphinscheduler-alert-server/src/main/resources/banner.txt
+++ b/dolphinscheduler-alert/dolphinscheduler-alert-server/src/main/resources/banner.txt
@@ -8,5 +8,5 @@ ${AnsiColor.BLUE}${AnsiStyle.BOLD}
               |_|
 ================================================================================
 ${AnsiColor.BLUE}${AnsiStyle.BOLD}
-::  Dolphinscheduler alert server ::  ${application.formatted-version}
+::  DolphinScheduler alert server ::  ${application.formatted-version}
 ${AnsiStyle.NORMAL}
diff --git a/dolphinscheduler-api/src/main/resources/application.yaml b/dolphinscheduler-api/src/main/resources/application.yaml
index 59e28b5064..6ef23be3ad 100644
--- a/dolphinscheduler-api/src/main/resources/application.yaml
+++ b/dolphinscheduler-api/src/main/resources/application.yaml
@@ -137,6 +137,9 @@ spring:
       on-profile: mysql
   datasource:
     driver-class-name: com.mysql.cj.jdbc.Driver
+    url: jdbc:mysql://127.0.0.1:3306/dolphinscheduler
+    username: root
+    password: root
   quartz:
     properties:
       org.quartz.jobStore.driverDelegateClass: org.quartz.impl.jdbcjobstore.StdJDBCDelegate
diff --git a/dolphinscheduler-api/src/main/resources/banner.txt b/dolphinscheduler-api/src/main/resources/banner.txt
index 28ece5e91f..49aa2a889e 100644
--- a/dolphinscheduler-api/src/main/resources/banner.txt
+++ b/dolphinscheduler-api/src/main/resources/banner.txt
@@ -8,5 +8,5 @@ ${AnsiColor.BLUE}${AnsiStyle.BOLD}
               |_|
 ================================================================================
 ${AnsiColor.BLUE}${AnsiStyle.BOLD}
-::  Dolphinscheduler api server ::  ${application.formatted-version}
+::  DolphinScheduler api server ::  ${application.formatted-version}
 ${AnsiStyle.NORMAL}
diff --git a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/Constants.java b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/Constants.java
index 9c16ce697e..b908cd233d 100644
--- a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/Constants.java
+++ b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/Constants.java
@@ -369,22 +369,22 @@ public final class Constants {
     /**
      * sleep 1000ms
      */
-    public static final int SLEEP_TIME_MILLIS = 1000;
+    public static final long SLEEP_TIME_MILLIS = 1_000L;
 
     /**
      * short sleep 100ms
      */
-    public static final int SLEEP_TIME_MILLIS_SHORT = 100;
+    public static final long SLEEP_TIME_MILLIS_SHORT = 100L;
 
     /**
      * one second mils
      */
-    public static final int SECOND_TIME_MILLIS = 1000;
+    public static final long SECOND_TIME_MILLIS = 1_000L;
 
     /**
      * master task instance cache-database refresh interval
      */
-    public static final int CACHE_REFRESH_TIME_MILLIS = 20 * 1000;
+    public static final long CACHE_REFRESH_TIME_MILLIS = 20 * 1_000L;
 
     /**
      * heartbeat for zk info length
diff --git a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/model/TaskNode.java b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/model/TaskNode.java
index 6999ac3baf..b5f975b4d2 100644
--- a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/model/TaskNode.java
+++ b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/model/TaskNode.java
@@ -272,7 +272,7 @@ public class TaskNode {
         this.runFlag = runFlag;
     }
 
-    public Boolean isForbidden() {
+    public boolean isForbidden() {
         return (!StringUtils.isEmpty(this.runFlag)
                 && this.runFlag.equals(Constants.FLOWNODE_RUN_FLAG_FORBIDDEN));
     }
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/cache/impl/ProcessInstanceExecCacheManagerImpl.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/cache/impl/ProcessInstanceExecCacheManagerImpl.java
index 137bd4ecc5..dc562d37bd 100644
--- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/cache/impl/ProcessInstanceExecCacheManagerImpl.java
+++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/cache/impl/ProcessInstanceExecCacheManagerImpl.java
@@ -18,11 +18,14 @@
 package org.apache.dolphinscheduler.server.master.cache.impl;
 
 import org.apache.dolphinscheduler.server.master.cache.ProcessInstanceExecCacheManager;
+import org.apache.dolphinscheduler.server.master.metrics.ProcessInstanceMetrics;
 import org.apache.dolphinscheduler.server.master.runner.WorkflowExecuteRunnable;
 
 import java.util.Collection;
 import java.util.concurrent.ConcurrentHashMap;
 
+import javax.annotation.PostConstruct;
+
 import org.springframework.stereotype.Component;
 
 import com.google.common.collect.ImmutableList;
@@ -35,6 +38,11 @@ public class ProcessInstanceExecCacheManagerImpl implements ProcessInstanceExecC
 
     private final ConcurrentHashMap<Integer, WorkflowExecuteRunnable> processInstanceExecMaps = new ConcurrentHashMap<>();
 
+    @PostConstruct
+    public void registerMetrics() {
+        ProcessInstanceMetrics.registerProcessInstanceRunningGauge(processInstanceExecMaps::size);
+    }
+
     @Override
     public WorkflowExecuteRunnable getByProcessInstanceId(int processInstanceId) {
         return processInstanceExecMaps.get(processInstanceId);
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/consumer/TaskPriorityQueueConsumer.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/consumer/TaskPriorityQueueConsumer.java
index f692685009..adeaff248f 100644
--- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/consumer/TaskPriorityQueueConsumer.java
+++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/consumer/TaskPriorityQueueConsumer.java
@@ -30,6 +30,7 @@ import org.apache.dolphinscheduler.server.master.dispatch.ExecutorDispatcher;
 import org.apache.dolphinscheduler.server.master.dispatch.context.ExecutionContext;
 import org.apache.dolphinscheduler.server.master.dispatch.enums.ExecutorType;
 import org.apache.dolphinscheduler.server.master.dispatch.exceptions.ExecuteException;
+import org.apache.dolphinscheduler.server.master.metrics.TaskMetrics;
 import org.apache.dolphinscheduler.server.master.processor.queue.TaskEvent;
 import org.apache.dolphinscheduler.server.master.processor.queue.TaskEventService;
 import org.apache.dolphinscheduler.service.exceptions.TaskPriorityQueueException;
@@ -53,6 +54,11 @@ import org.slf4j.LoggerFactory;
 import org.springframework.beans.factory.annotation.Autowired;
 import org.springframework.stereotype.Component;
 
+import io.micrometer.core.annotation.Counted;
+import io.micrometer.core.annotation.Timed;
+
+import org.apache.commons.lang3.time.StopWatch;
+
 /**
  * TaskUpdateQueue consumer
  */
@@ -119,6 +125,7 @@ public class TaskPriorityQueueConsumer extends Thread {
                 List<TaskPriority> failedDispatchTasks = this.batchDispatch(fetchTaskNum);
 
                 if (!failedDispatchTasks.isEmpty()) {
+                    TaskMetrics.incTaskDispatchFailed(failedDispatchTasks.size());
                     for (TaskPriority dispatchFailedTask : failedDispatchTasks) {
                         taskPriorityQueue.put(dispatchFailedTask);
                     }
@@ -129,6 +136,7 @@ public class TaskPriorityQueueConsumer extends Thread {
                     }
                 }
             } catch (Exception e) {
+                TaskMetrics.incTaskDispatchError();
                 logger.error("dispatcher task error", e);
             }
         }
@@ -137,7 +145,7 @@ public class TaskPriorityQueueConsumer extends Thread {
     /**
      * batch dispatch with thread pool
      */
-    private List<TaskPriority> batchDispatch(int fetchTaskNum) throws TaskPriorityQueueException, InterruptedException {
+    public List<TaskPriority> batchDispatch(int fetchTaskNum) throws TaskPriorityQueueException, InterruptedException {
         List<TaskPriority> failedDispatchTasks = Collections.synchronizedList(new ArrayList<>());
         CountDownLatch latch = new CountDownLatch(fetchTaskNum);
 
@@ -169,6 +177,7 @@ public class TaskPriorityQueueConsumer extends Thread {
      * @return result
      */
     protected boolean dispatchTask(TaskPriority taskPriority) {
+        TaskMetrics.incTaskDispatch();
         boolean result = false;
         try {
             TaskExecutionContext context = taskPriority.getTaskExecutionContext();
@@ -215,7 +224,7 @@ public class TaskPriorityQueueConsumer extends Thread {
      * @param taskInstanceId taskInstanceId
      * @return taskInstance is final state
      */
-    public Boolean taskInstanceIsFinalState(int taskInstanceId) {
+    public boolean taskInstanceIsFinalState(int taskInstanceId) {
         TaskInstance taskInstance = processService.findTaskInstanceById(taskInstanceId);
         return taskInstance.getState().typeIsFinished();
     }
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/metrics/MasterServerMetrics.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/metrics/MasterServerMetrics.java
new file mode 100644
index 0000000000..e5c86d9641
--- /dev/null
+++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/metrics/MasterServerMetrics.java
@@ -0,0 +1,53 @@
+/*
+ * 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.metrics;
+
+import io.micrometer.core.instrument.Counter;
+import io.micrometer.core.instrument.Metrics;
+
+public final class MasterServerMetrics {
+
+    private MasterServerMetrics() {
+        throw new UnsupportedOperationException("Utility class");
+    }
+
+    /**
+     * Used to measure the master server is overload.
+     */
+    private static final Counter MASTER_OVERLOAD_COUNTER =
+            Counter.builder("dolphinscheduler_master_overload_count")
+                    .description("Master server overload count")
+                    .register(Metrics.globalRegistry);
+
+    /**
+     * Used to measure the number of process command consumed by master.
+     */
+    private static final Counter MASTER_CONSUME_COMMAND_COUNTER =
+            Counter.builder("dolphinscheduler_master_consume_command_count")
+                    .description("Master server consume command count")
+                    .register(Metrics.globalRegistry);
+
+    public static void incMasterOverload() {
+        MASTER_OVERLOAD_COUNTER.increment();
+    }
+
+    public static void incMasterConsumeCommand(int commandCount) {
+        MASTER_CONSUME_COMMAND_COUNTER.increment(commandCount);
+    }
+
+}
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/metrics/ProcessInstanceMetrics.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/metrics/ProcessInstanceMetrics.java
new file mode 100644
index 0000000000..9e9b11c95d
--- /dev/null
+++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/metrics/ProcessInstanceMetrics.java
@@ -0,0 +1,101 @@
+/*
+ * 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.metrics;
+
+import java.util.function.Supplier;
+
+import io.micrometer.core.instrument.Counter;
+import io.micrometer.core.instrument.Gauge;
+import io.micrometer.core.instrument.Metrics;
+
+public final class ProcessInstanceMetrics {
+
+    private ProcessInstanceMetrics() {
+        throw new UnsupportedOperationException("Utility class");
+    }
+
+    private static final Counter PROCESS_INSTANCE_SUBMIT_COUNTER =
+            Counter.builder("dolphinscheduler_process_instance_submit_count")
+                    .description("Process instance submit total count")
+                    .register(Metrics.globalRegistry);
+
+    private static final Counter PROCESS_INSTANCE_TIMEOUT_COUNTER =
+            Counter.builder("dolphinscheduler_process_instance_timeout_count")
+                    .description("Process instance timeout total count")
+                    .register(Metrics.globalRegistry);
+
+    private static final Counter PROCESS_INSTANCE_FINISH_COUNTER =
+            Counter.builder("dolphinscheduler_process_instance_finish_count")
+                    .description("Process instance finish total count")
+                    .register(Metrics.globalRegistry);
+
+    private static final Counter PROCESS_INSTANCE_SUCCESS_COUNTER =
+            Counter.builder("dolphinscheduler_process_instance_success_count")
+                    .description("Process instance success total count")
+                    .register(Metrics.globalRegistry);
+
+    private static final Counter PROCESS_INSTANCE_FAILURE_COUNTER =
+            Counter.builder("dolphinscheduler_process_instance_failure_count")
+                    .description("Process instance failure total count")
+                    .register(Metrics.globalRegistry);
+
+    private static final Counter PROCESS_INSTANCE_STOP_COUNTER =
+            Counter.builder("dolphinscheduler_process_instance_stop_count")
+                    .description("Process instance stop total count")
+                    .register(Metrics.globalRegistry);
+
+    private static final Counter PROCESS_INSTANCE_FAILOVER_COUNTER =
+            Counter.builder("dolphinscheduler_process_instance_failover_count")
+                    .description("Process instance failover total count")
+                    .register(Metrics.globalRegistry);
+
+    public static synchronized void registerProcessInstanceRunningGauge(Supplier<Number> function) {
+        Gauge.builder("dolphinscheduler_process_instance_running_gauge", function)
+                .description("The current running process instance count")
+                .register(Metrics.globalRegistry);
+    }
+
+    public static void incProcessInstanceSubmit() {
+        PROCESS_INSTANCE_SUBMIT_COUNTER.increment();
+    }
+
+    public static void incProcessInstanceTimeout() {
+        PROCESS_INSTANCE_TIMEOUT_COUNTER.increment();
+    }
+
+    public static void incProcessInstanceFinish() {
+        PROCESS_INSTANCE_FINISH_COUNTER.increment();
+    }
+
+    public static void incProcessInstanceSuccess() {
+        PROCESS_INSTANCE_SUCCESS_COUNTER.increment();
+    }
+
+    public static void incProcessInstanceFailure() {
+        PROCESS_INSTANCE_FAILURE_COUNTER.increment();
+    }
+
+    public static void incProcessInstanceStop() {
+        PROCESS_INSTANCE_STOP_COUNTER.increment();
+    }
+
+    public static void incProcessInstanceFailover() {
+        PROCESS_INSTANCE_FAILOVER_COUNTER.increment();
+    }
+
+}
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/metrics/TaskMetrics.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/metrics/TaskMetrics.java
new file mode 100644
index 0000000000..c4516d0644
--- /dev/null
+++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/metrics/TaskMetrics.java
@@ -0,0 +1,137 @@
+/*
+ * 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.metrics;
+
+import java.util.function.Supplier;
+
+import io.micrometer.core.instrument.Counter;
+import io.micrometer.core.instrument.Gauge;
+import io.micrometer.core.instrument.Metrics;
+
+
+public final class TaskMetrics {
+    private TaskMetrics() {
+        throw new UnsupportedOperationException("Utility class");
+    }
+
+    private static final Counter TASK_SUBMIT_COUNTER =
+            Counter.builder("dolphinscheduler_task_submit_count")
+                    .description("Task submit total count")
+                    .register(Metrics.globalRegistry);
+
+    private static final Counter TASK_FINISH_COUNTER =
+            Counter.builder("dolphinscheduler_task_finish_count")
+                    .description("Task finish total count")
+                    .register(Metrics.globalRegistry);
+
+    private static final Counter TASK_SUCCESS_COUNTER =
+            Counter.builder("dolphinscheduler_task_success_count")
+                    .description("Task success total count")
+                    .register(Metrics.globalRegistry);
+
+    private static final Counter TASK_FAILURE_COUNTER =
+            Counter.builder("dolphinscheduler_task_failure_count")
+                    .description("Task failure total count")
+                    .register(Metrics.globalRegistry);
+
+    private static final Counter TASK_TIMEOUT_COUNTER =
+            Counter.builder("dolphinscheduler_task_timeout_count")
+                    .description("Task timeout total count")
+                    .register(Metrics.globalRegistry);
+
+    private static final Counter TASK_RETRY_COUNTER =
+            Counter.builder("dolphinscheduler_task_retry_count")
+                    .description("Task retry total count")
+                    .register(Metrics.globalRegistry);
+
+    private static final Counter TASK_STOP_COUNTER =
+            Counter.builder("dolphinscheduler_task_stop_count")
+                    .description("Task stop total count")
+                    .register(Metrics.globalRegistry);
+
+    private static final Counter TASK_FAILOVER_COUNTER =
+            Counter.builder("dolphinscheduler_task_failover_count")
+                    .description("Task failover total count")
+                    .register(Metrics.globalRegistry);
+
+    private static final Counter TASK_DISPATCH_COUNTER =
+            Counter.builder("dolphinscheduler_task_dispatch_count")
+                    .description("Task dispatch count")
+                    .register(Metrics.globalRegistry);
+
+    private static final Counter TASK_DISPATCHER_FAILED =
+            Counter.builder("dolphinscheduler_task_dispatch_failed_count")
+                    .description("Task dispatch failed count")
+                    .register(Metrics.globalRegistry);
+
+    private static final Counter TASK_DISPATCH_ERROR =
+            Counter.builder("dolphinscheduler_task_dispatch_error_count")
+                    .description("Task dispatch error")
+                    .register(Metrics.globalRegistry);
+
+    public static void incTaskSubmit() {
+        TASK_SUBMIT_COUNTER.increment();
+    }
+
+    public synchronized static void registerTaskRunning(Supplier<Number> consumer) {
+        Gauge.builder("dolphinscheduler_task_running_gauge", consumer)
+                .description("Task running count")
+                .register(Metrics.globalRegistry);
+    }
+
+    public static void incTaskFinish() {
+        TASK_FINISH_COUNTER.increment();
+    }
+
+    public static void incTaskSuccess() {
+        TASK_SUCCESS_COUNTER.increment();
+    }
+
+    public static void incTaskFailure() {
+        TASK_FAILURE_COUNTER.increment();
+    }
+
+    public static void incTaskTimeout() {
+        TASK_TIMEOUT_COUNTER.increment();
+    }
+
+    public static void incTaskRetry() {
+        TASK_RETRY_COUNTER.increment();
+    }
+
+    public static void incTaskStop() {
+        TASK_STOP_COUNTER.increment();
+    }
+
+    public static void incTaskFailover() {
+        TASK_FAILOVER_COUNTER.increment();
+    }
+
+    public static void incTaskDispatchFailed(int failedCount) {
+        TASK_DISPATCHER_FAILED.increment(failedCount);
+    }
+
+    public static void incTaskDispatchError() {
+        TASK_DISPATCH_ERROR.increment();
+    }
+
+    public static void incTaskDispatch() {
+        TASK_DISPATCH_COUNTER.increment();
+    }
+
+}
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/FailoverExecuteThread.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/FailoverExecuteThread.java
index 5fd812f162..4bac6cdf15 100644
--- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/FailoverExecuteThread.java
+++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/FailoverExecuteThread.java
@@ -62,7 +62,7 @@ public class FailoverExecuteThread extends Thread {
             } catch (Exception e) {
                 logger.error("failover execute error", e);
             } finally {
-                ThreadUtils.sleep((long) Constants.SLEEP_TIME_MILLIS * masterConfig.getFailoverInterval() * 60);
+                ThreadUtils.sleep(Constants.SLEEP_TIME_MILLIS * masterConfig.getFailoverInterval() * 60);
             }
         }
     }
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterSchedulerService.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterSchedulerService.java
index e688863d06..ae9a461d10 100644
--- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterSchedulerService.java
+++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterSchedulerService.java
@@ -21,7 +21,6 @@ import org.apache.dolphinscheduler.common.Constants;
 import org.apache.dolphinscheduler.common.enums.SlotCheckState;
 import org.apache.dolphinscheduler.common.thread.Stopper;
 import org.apache.dolphinscheduler.common.thread.ThreadUtils;
-import org.apache.dolphinscheduler.common.utils.JSONUtils;
 import org.apache.dolphinscheduler.common.utils.NetUtils;
 import org.apache.dolphinscheduler.common.utils.OSUtils;
 import org.apache.dolphinscheduler.dao.entity.Command;
@@ -31,6 +30,7 @@ import org.apache.dolphinscheduler.remote.config.NettyClientConfig;
 import org.apache.dolphinscheduler.server.master.cache.ProcessInstanceExecCacheManager;
 import org.apache.dolphinscheduler.server.master.config.MasterConfig;
 import org.apache.dolphinscheduler.server.master.dispatch.executor.NettyExecutorManager;
+import org.apache.dolphinscheduler.server.master.metrics.MasterServerMetrics;
 import org.apache.dolphinscheduler.server.master.registry.ServerNodeManager;
 import org.apache.dolphinscheduler.service.alert.ProcessAlertManager;
 import org.apache.dolphinscheduler.service.process.ProcessService;
@@ -133,6 +133,7 @@ public class MasterSchedulerService extends Thread {
             try {
                 boolean runCheckFlag = OSUtils.checkResource(masterConfig.getMaxCpuLoadAvg(), masterConfig.getReservedMemory());
                 if (!runCheckFlag) {
+                    MasterServerMetrics.incMasterOverload();
                     Thread.sleep(Constants.SLEEP_TIME_MILLIS);
                     continue;
                 }
@@ -159,6 +160,7 @@ public class MasterSchedulerService extends Thread {
         if (CollectionUtils.isEmpty(processInstances)) {
             return;
         }
+        MasterServerMetrics.incMasterConsumeCommand(commands.size());
 
         for (ProcessInstance processInstance : processInstances) {
             if (processInstance == null) {
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/WorkflowExecuteRunnable.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/WorkflowExecuteRunnable.java
index ca6a7aa4fa..6c162717ce 100644
--- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/WorkflowExecuteRunnable.java
+++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/WorkflowExecuteRunnable.java
@@ -66,6 +66,8 @@ import org.apache.dolphinscheduler.remote.command.HostUpdateCommand;
 import org.apache.dolphinscheduler.remote.utils.Host;
 import org.apache.dolphinscheduler.server.master.config.MasterConfig;
 import org.apache.dolphinscheduler.server.master.dispatch.executor.NettyExecutorManager;
+import org.apache.dolphinscheduler.server.master.metrics.ProcessInstanceMetrics;
+import org.apache.dolphinscheduler.server.master.metrics.TaskMetrics;
 import org.apache.dolphinscheduler.server.master.runner.task.ITaskProcessor;
 import org.apache.dolphinscheduler.server.master.runner.task.TaskAction;
 import org.apache.dolphinscheduler.server.master.runner.task.TaskProcessorFactory;
@@ -83,6 +85,7 @@ import java.util.Arrays;
 import java.util.Collection;
 import java.util.Date;
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
@@ -92,7 +95,6 @@ import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentLinkedQueue;
 import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.stream.Collectors;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -249,6 +251,7 @@ public class WorkflowExecuteRunnable implements Runnable {
         this.nettyExecutorManager = nettyExecutorManager;
         this.processAlertManager = processAlertManager;
         this.stateWheelExecuteThread = stateWheelExecuteThread;
+        TaskMetrics.registerTaskRunning(readyToSubmitTaskQueue::size);
     }
 
     /**
@@ -292,7 +295,7 @@ public class WorkflowExecuteRunnable implements Runnable {
 
     public boolean addStateEvent(StateEvent stateEvent) {
         if (processInstance.getId() != stateEvent.getProcessInstanceId()) {
-            logger.info("state event would be abounded :{}", stateEvent.toString());
+            logger.info("state event would be abounded :{}", stateEvent);
             return false;
         }
         this.stateEvents.add(stateEvent);
@@ -308,7 +311,7 @@ public class WorkflowExecuteRunnable implements Runnable {
     }
 
     private boolean stateEventHandler(StateEvent stateEvent) {
-        logger.info("process event: {}", stateEvent.toString());
+        logger.info("process event: {}", stateEvent);
 
         if (!checkProcessInstance(stateEvent)) {
             return false;
@@ -317,21 +320,26 @@ public class WorkflowExecuteRunnable implements Runnable {
         boolean result = false;
         switch (stateEvent.getType()) {
             case PROCESS_STATE_CHANGE:
+                measureProcessState(stateEvent);
                 result = processStateChangeHandler(stateEvent);
                 break;
             case TASK_STATE_CHANGE:
+                measureTaskState(stateEvent);
                 result = taskStateChangeHandler(stateEvent);
                 break;
             case PROCESS_TIMEOUT:
+                ProcessInstanceMetrics.incProcessInstanceTimeout();
                 result = processTimeout();
                 break;
             case TASK_TIMEOUT:
+                TaskMetrics.incTaskTimeout();
                 result = taskTimeout(stateEvent);
                 break;
             case WAIT_TASK_GROUP:
                 result = checkForceStartAndWakeUp(stateEvent);
                 break;
             case TASK_RETRY:
+                TaskMetrics.incTaskRetry();
                 result = taskRetryEventHandler(stateEvent);
                 break;
             case PROCESS_BLOCKED:
@@ -438,10 +446,10 @@ public class WorkflowExecuteRunnable implements Runnable {
 
     private void taskFinished(TaskInstance taskInstance) {
         logger.info("work flow {} task id:{} code:{} state:{} ",
-            processInstance.getId(),
-            taskInstance.getId(),
-            taskInstance.getTaskCode(),
-            taskInstance.getState());
+                processInstance.getId(),
+                taskInstance.getId(),
+                taskInstance.getTaskCode(),
+                taskInstance.getState());
 
         activeTaskProcessorMaps.remove(taskInstance.getTaskCode());
         stateWheelExecuteThread.removeTask4TimeoutCheck(processInstance, taskInstance);
@@ -733,7 +741,7 @@ public class WorkflowExecuteRunnable implements Runnable {
             scheduleDate = complementListDate.get(0);
         } else if (processInstance.getState().typeIsFinished()) {
             endProcess();
-            if (complementListDate.size() <= 0) {
+            if (complementListDate.isEmpty()) {
                 logger.info("process complement end. process id:{}", processInstance.getId());
                 return true;
             }
@@ -744,9 +752,9 @@ public class WorkflowExecuteRunnable implements Runnable {
                 return true;
             }
             logger.info("process complement continue. process id:{}, schedule time:{} complementListDate:{}",
-                processInstance.getId(),
-                processInstance.getScheduleTime(),
-                complementListDate.toString());
+                    processInstance.getId(),
+                    processInstance.getScheduleTime(),
+                    complementListDate);
             scheduleDate = complementListDate.get(index + 1);
         }
         //the next process complement
@@ -946,7 +954,7 @@ public class WorkflowExecuteRunnable implements Runnable {
             }
         }
 
-        if (processInstance.isComplementData() && complementListDate.size() == 0) {
+        if (processInstance.isComplementData() && complementListDate.isEmpty()) {
             Map<String, String> cmdParam = JSONUtils.toMap(processInstance.getCommandParam());
             if (cmdParam != null && cmdParam.containsKey(CMDPARAM_COMPLEMENT_DATA_START_DATE)) {
                 // reset global params while there are start parameters
@@ -955,17 +963,17 @@ public class WorkflowExecuteRunnable implements Runnable {
                 Date start = DateUtils.stringToDate(cmdParam.get(CMDPARAM_COMPLEMENT_DATA_START_DATE));
                 Date end = DateUtils.stringToDate(cmdParam.get(CMDPARAM_COMPLEMENT_DATA_END_DATE));
                 List<Schedule> schedules = processService.queryReleaseSchedulerListByProcessDefinitionCode(processInstance.getProcessDefinitionCode());
-                if (complementListDate.size() == 0 && needComplementProcess()) {
+                if (complementListDate.isEmpty() && needComplementProcess()) {
                     complementListDate = CronUtils.getSelfFireDateList(start, end, schedules);
                     logger.info(" process definition code:{} complement data: {}",
-                        processInstance.getProcessDefinitionCode(), complementListDate.toString());
+                            processInstance.getProcessDefinitionCode(), complementListDate);
 
-                    if (complementListDate.size() > 0 && Flag.NO == processInstance.getIsSubProcess()) {
+                    if (!complementListDate.isEmpty() && Flag.NO == processInstance.getIsSubProcess()) {
                         processInstance.setScheduleTime(complementListDate.get(0));
                         processInstance.setGlobalParams(ParameterUtils.curingGlobalParams(
-                            processDefinition.getGlobalParamMap(),
-                            processDefinition.getGlobalParamList(),
-                            CommandType.COMPLEMENT_DATA, processInstance.getScheduleTime(), cmdParam.get(Constants.SCHEDULE_TIMEZONE)));
+                                processDefinition.getGlobalParamMap(),
+                                processDefinition.getGlobalParamList(),
+                                CommandType.COMPLEMENT_DATA, processInstance.getScheduleTime(), cmdParam.get(Constants.SCHEDULE_TIMEZONE)));
                         processService.updateProcessInstance(processInstance);
                     }
                 }
@@ -1322,12 +1330,10 @@ public class WorkflowExecuteRunnable implements Runnable {
             TaskInstance endTaskInstance = taskInstanceMap.get(completeTaskMap.get(NumberUtils.toLong(parentNodeCode)));
             String taskInstanceVarPool = endTaskInstance.getVarPool();
             if (StringUtils.isNotEmpty(taskInstanceVarPool)) {
-                Set<Property> taskProperties = JSONUtils.toList(taskInstanceVarPool, Property.class)
-                    .stream().collect(Collectors.toSet());
+                Set<Property> taskProperties = new HashSet<>(JSONUtils.toList(taskInstanceVarPool, Property.class));
                 String processInstanceVarPool = processInstance.getVarPool();
                 if (StringUtils.isNotEmpty(processInstanceVarPool)) {
-                    Set<Property> properties = JSONUtils.toList(processInstanceVarPool, Property.class)
-                        .stream().collect(Collectors.toSet());
+                    Set<Property> properties = new HashSet<>(JSONUtils.toList(processInstanceVarPool, Property.class));
                     properties.addAll(taskProperties);
                     processInstance.setVarPool(JSONUtils.toJsonString(properties));
                 } else {
@@ -1730,7 +1736,8 @@ public class WorkflowExecuteRunnable implements Runnable {
                 return;
             }
             logger.info("add task to stand by list, task name:{}, task id:{}, task code:{}",
-                taskInstance.getName(), taskInstance.getId(), taskInstance.getTaskCode());
+                    taskInstance.getName(), taskInstance.getId(), taskInstance.getTaskCode());
+            TaskMetrics.incTaskSubmit();
             readyToSubmitTaskQueue.put(taskInstance);
         } catch (Exception e) {
             logger.error("add task instance to readyToSubmitTaskQueue, taskName:{}, task id:{}", taskInstance.getName(), taskInstance.getId(), e);
@@ -2025,4 +2032,42 @@ public class WorkflowExecuteRunnable implements Runnable {
             }
         }
     }
+
+    private void measureProcessState(StateEvent processStateEvent) {
+        if (processStateEvent.getExecutionStatus().typeIsFinished()) {
+            ProcessInstanceMetrics.incProcessInstanceFinish();
+        }
+        switch (processStateEvent.getExecutionStatus()) {
+            case STOP:
+                ProcessInstanceMetrics.incProcessInstanceStop();
+                break;
+            case SUCCESS:
+                ProcessInstanceMetrics.incProcessInstanceSuccess();
+                break;
+            case FAILURE:
+                ProcessInstanceMetrics.incProcessInstanceFailure();
+                break;
+            default:
+                break;
+        }
+    }
+
+    private void measureTaskState(StateEvent taskStateEvent) {
+        if (taskStateEvent.getExecutionStatus().typeIsFinished()) {
+            TaskMetrics.incTaskFinish();
+        }
+        switch (taskStateEvent.getExecutionStatus()) {
+            case STOP:
+                TaskMetrics.incTaskStop();
+                break;
+            case SUCCESS:
+                TaskMetrics.incTaskSuccess();
+                break;
+            case FAILURE:
+                TaskMetrics.incTaskFailure();
+                break;
+            default:
+                break;
+        }
+    }
 }
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/WorkflowExecuteThreadPool.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/WorkflowExecuteThreadPool.java
index 642e33b6d6..324e4d9367 100644
--- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/WorkflowExecuteThreadPool.java
+++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/WorkflowExecuteThreadPool.java
@@ -28,13 +28,14 @@ import org.apache.dolphinscheduler.remote.command.StateEventChangeCommand;
 import org.apache.dolphinscheduler.remote.processor.StateEventCallbackService;
 import org.apache.dolphinscheduler.server.master.cache.ProcessInstanceExecCacheManager;
 import org.apache.dolphinscheduler.server.master.config.MasterConfig;
+import org.apache.dolphinscheduler.server.master.metrics.ProcessInstanceMetrics;
 import org.apache.dolphinscheduler.service.process.ProcessService;
 
 import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
 
 import javax.annotation.PostConstruct;
-import com.google.common.base.Strings;
+
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.springframework.beans.factory.annotation.Autowired;
@@ -43,6 +44,8 @@ import org.springframework.stereotype.Component;
 import org.springframework.util.concurrent.ListenableFuture;
 import org.springframework.util.concurrent.ListenableFutureCallback;
 
+import com.google.common.base.Strings;
+
 @Component
 public class WorkflowExecuteThreadPool extends ThreadPoolTaskExecutor {
 
@@ -92,6 +95,7 @@ public class WorkflowExecuteThreadPool extends ThreadPoolTaskExecutor {
      * start workflow
      */
     public void startWorkflow(WorkflowExecuteRunnable workflowExecuteThread) {
+        ProcessInstanceMetrics.incProcessInstanceSubmit();
         submit(workflowExecuteThread);
     }
 
@@ -107,7 +111,7 @@ public class WorkflowExecuteThreadPool extends ThreadPoolTaskExecutor {
         }
         multiThreadFilterMap.put(workflowExecuteThread.getKey(), workflowExecuteThread);
         int processInstanceId = workflowExecuteThread.getProcessInstance().getId();
-        ListenableFuture future = this.submitListenable(workflowExecuteThread::handleEvents);
+        ListenableFuture<?> future = this.submitListenable(workflowExecuteThread::handleEvents);
         future.addCallback(new ListenableFutureCallback() {
             @Override
             public void onFailure(Throwable ex) {
@@ -118,7 +122,7 @@ public class WorkflowExecuteThreadPool extends ThreadPoolTaskExecutor {
             @Override
             public void onSuccess(Object result) {
                 // if an exception occurs, first, the error message cannot be printed in the log;
-                // secondly, the `multiThreadFilterMap` cannot be remove the `workflowExecuteThread`, resulting in the state of process instance cannot be changed and memory leak
+                // secondly, the `multiThreadFilterMap` cannot remove the `workflowExecuteThread`, resulting in the state of process instance cannot be changed and memory leak
                 try {
                     if (workflowExecuteThread.workFlowFinish()) {
                         stateWheelExecuteThread.removeProcess4TimeoutCheck(workflowExecuteThread.getProcessInstance());
@@ -142,12 +146,14 @@ public class WorkflowExecuteThreadPool extends ThreadPoolTaskExecutor {
             return;
         }
         Map<ProcessInstance, TaskInstance> fatherMaps = processService.notifyProcessList(finishProcessInstance.getId());
-        for (ProcessInstance processInstance : fatherMaps.keySet()) {
+        for (Map.Entry<ProcessInstance, TaskInstance> entry : fatherMaps.entrySet()) {
+            ProcessInstance processInstance = entry.getKey();
+            TaskInstance taskInstance = entry.getValue();
             String address = NetUtils.getAddr(masterConfig.getListenPort());
             if (processInstance.getHost().equalsIgnoreCase(address)) {
-                this.notifyMyself(processInstance, fatherMaps.get(processInstance));
+                this.notifyMyself(processInstance, taskInstance);
             } else {
-                this.notifyProcess(finishProcessInstance, processInstance, fatherMaps.get(processInstance));
+                this.notifyProcess(finishProcessInstance, processInstance, taskInstance);
             }
         }
     }
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/CommonTaskProcessor.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/CommonTaskProcessor.java
index 30e9d8d621..fd803ac371 100644
--- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/CommonTaskProcessor.java
+++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/CommonTaskProcessor.java
@@ -43,7 +43,7 @@ import com.google.auto.service.AutoService;
 @AutoService(ITaskProcessor.class)
 public class CommonTaskProcessor extends BaseTaskProcessor {
 
-    private TaskPriorityQueue taskUpdateQueue;
+    private TaskPriorityQueue<TaskPriority> taskUpdateQueue;
 
     private NettyExecutorManager nettyExecutorManager = SpringApplicationContext.getBean(NettyExecutorManager.class);
 
@@ -110,7 +110,7 @@ public class CommonTaskProcessor extends BaseTaskProcessor {
                 this.initQueue();
             }
             if (taskInstance.getState().typeIsFinished()) {
-                logger.info(String.format("submit task , but task [%s] state [%s] is already  finished. ", taskInstance.getName(), taskInstance.getState().toString()));
+                logger.info("submit task , but task [{}] state [{}] is already  finished. ", taskInstance.getName(), taskInstance.getState());
                 return true;
             }
             // task cannot be submitted because its execution state is RUNNING or DELAY.
@@ -134,7 +134,7 @@ public class CommonTaskProcessor extends BaseTaskProcessor {
             taskPriority.setTaskExecutionContext(taskExecutionContext);
 
             taskUpdateQueue.put(taskPriority);
-            logger.info(String.format("master submit success, task : %s", taskInstance.getName()));
+            logger.info("master submit success, task : {}", taskInstance.getName());
             return true;
         } catch (Exception e) {
             logger.error("submit task error", e);
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/TaskProcessorFactory.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/TaskProcessorFactory.java
index 0129338649..542697a92f 100644
--- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/TaskProcessorFactory.java
+++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/TaskProcessorFactory.java
@@ -26,10 +26,15 @@ import java.util.Objects;
 import java.util.ServiceLoader;
 import java.util.concurrent.ConcurrentHashMap;
 
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
 /**
  * the factory to create task processor
  */
-public class TaskProcessorFactory {
+public final class TaskProcessorFactory {
+
+    private static final Logger logger = LoggerFactory.getLogger(TaskProcessorFactory.class);
 
     public static final Map<String, ITaskProcessor> PROCESS_MAP = new ConcurrentHashMap<>();
 
@@ -47,7 +52,8 @@ public class TaskProcessorFactory {
         }
         ITaskProcessor iTaskProcessor = PROCESS_MAP.get(type);
         if (Objects.isNull(iTaskProcessor)) {
-            iTaskProcessor = PROCESS_MAP.get(DEFAULT_PROCESSOR);
+            logger.warn("task processor not found for type: {}", type);
+            return PROCESS_MAP.get(DEFAULT_PROCESSOR);
         }
 
         return iTaskProcessor.getClass().newInstance();
@@ -55,10 +61,15 @@ public class TaskProcessorFactory {
 
     /**
      * if match master processor, then this task type is processed on the master
+     *
      * @param type
      * @return
      */
     public static boolean isMasterTask(String type) {
         return PROCESS_MAP.containsKey(type);
     }
+
+    private TaskProcessorFactory() {
+        throw new UnsupportedOperationException("TaskProcessorFactory cannot be instantiated");
+    }
 }
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/service/FailoverService.java b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/service/FailoverService.java
index dad2dbfe1e..3cc92fb905 100644
--- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/service/FailoverService.java
+++ b/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/service/FailoverService.java
@@ -32,6 +32,8 @@ import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
 import org.apache.dolphinscheduler.plugin.task.api.enums.ExecutionStatus;
 import org.apache.dolphinscheduler.server.builder.TaskExecutionContextBuilder;
 import org.apache.dolphinscheduler.server.master.config.MasterConfig;
+import org.apache.dolphinscheduler.server.master.metrics.ProcessInstanceMetrics;
+import org.apache.dolphinscheduler.server.master.metrics.TaskMetrics;
 import org.apache.dolphinscheduler.server.master.runner.WorkflowExecuteThreadPool;
 import org.apache.dolphinscheduler.server.master.runner.task.TaskProcessorFactory;
 import org.apache.dolphinscheduler.server.utils.ProcessUtils;
@@ -157,6 +159,7 @@ public class FailoverService {
             }
 
             LOGGER.info("failover process instance id: {}", processInstance.getId());
+            ProcessInstanceMetrics.incProcessInstanceFailover();
             //updateProcessInstance host is null and insert into command
             processInstance.setHost(Constants.NULL);
             processService.processNeedFailoverProcessInstances(processInstance);
@@ -227,7 +230,7 @@ public class FailoverService {
         if (!checkTaskInstanceNeedFailover(servers, taskInstance)) {
             return;
         }
-
+        TaskMetrics.incTaskFailover();
         boolean isMasterTask = TaskProcessorFactory.isMasterTask(taskInstance.getTaskType());
 
         taskInstance.setProcessInstance(processInstance);
diff --git a/dolphinscheduler-master/src/main/resources/application.yaml b/dolphinscheduler-master/src/main/resources/application.yaml
index e353bb8dcc..a6c70f9871 100644
--- a/dolphinscheduler-master/src/main/resources/application.yaml
+++ b/dolphinscheduler-master/src/main/resources/application.yaml
@@ -136,6 +136,9 @@ spring:
       on-profile: mysql
   datasource:
     driver-class-name: com.mysql.cj.jdbc.Driver
+    url: jdbc:mysql://127.0.0.1:3306/dolphinscheduler
+    username: root
+    password: root
   quartz:
     properties:
       org.quartz.jobStore.driverDelegateClass: org.quartz.impl.jdbcjobstore.StdJDBCDelegate
diff --git a/dolphinscheduler-master/src/main/resources/banner.txt b/dolphinscheduler-master/src/main/resources/banner.txt
index ce8c3b508d..ef081edfff 100644
--- a/dolphinscheduler-master/src/main/resources/banner.txt
+++ b/dolphinscheduler-master/src/main/resources/banner.txt
@@ -8,5 +8,5 @@ ${AnsiColor.BLUE}${AnsiStyle.BOLD}
               |_|
 ================================================================================
 ${AnsiColor.BLUE}${AnsiStyle.BOLD}
-::  Dolphinscheduler master server ::  ${application.formatted-version}
+::  DolphinScheduler master server ::  ${application.formatted-version}
 ${AnsiStyle.NORMAL}
diff --git a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/WorkflowExecuteTaskTest.java b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/WorkflowExecuteTaskTest.java
index 339c5e80ae..f2d263f699 100644
--- a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/WorkflowExecuteTaskTest.java
+++ b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/WorkflowExecuteTaskTest.java
@@ -84,8 +84,6 @@ public class WorkflowExecuteTaskTest {
 
     private ApplicationContext applicationContext;
 
-    private TaskProcessorFactory taskProcessorFactory;
-
     private StateWheelExecuteThread stateWheelExecuteThread;
 
     @Before
@@ -100,8 +98,6 @@ public class WorkflowExecuteTaskTest {
         processService = mock(ProcessService.class);
         Mockito.when(applicationContext.getBean(ProcessService.class)).thenReturn(processService);
 
-        taskProcessorFactory = mock(TaskProcessorFactory.class);
-
         processInstance = mock(ProcessInstance.class);
         Mockito.when(processInstance.getState()).thenReturn(ExecutionStatus.SUCCESS);
         Mockito.when(processInstance.getHistoryCmd()).thenReturn(CommandType.COMPLEMENT_DATA.toString());
diff --git a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/registry/ServerNodeManagerTest.java b/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/registry/ServerNodeManagerTest.java
deleted file mode 100644
index 42b49888bc..0000000000
--- a/dolphinscheduler-master/src/test/java/org/apache/dolphinscheduler/server/master/registry/ServerNodeManagerTest.java
+++ /dev/null
@@ -1,60 +0,0 @@
-/*
- * 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.registry;
-
-import org.apache.dolphinscheduler.dao.AlertDao;
-import org.apache.dolphinscheduler.dao.mapper.WorkerGroupMapper;
-import org.apache.dolphinscheduler.service.registry.RegistryClient;
-
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.mockito.Mock;
-import org.powermock.api.mockito.PowerMockito;
-import org.powermock.core.classloader.annotations.PowerMockIgnore;
-import org.powermock.core.classloader.annotations.PrepareForTest;
-import org.powermock.modules.junit4.PowerMockRunner;
-
-/**
- * server node manager test
- */
-@RunWith(PowerMockRunner.class)
-@PrepareForTest({ RegistryClient.class })
-@PowerMockIgnore({"javax.management.*"})
-public class ServerNodeManagerTest {
-
-    private ServerNodeManager serverNodeManager;
-
-    @Mock
-    private WorkerGroupMapper workerGroupMapper;
-
-    @Mock
-    private AlertDao alertDao;
-
-    @Before
-    public void before() {
-        PowerMockito.suppress(PowerMockito.constructor(RegistryClient.class));
-        serverNodeManager = PowerMockito.mock(ServerNodeManager.class);
-    }
-
-    @Test
-    public void test(){
-        //serverNodeManager.getWorkerGroupNodes()
-    }
-
-}
diff --git a/dolphinscheduler-meter/src/main/java/org/apache/dolphinscheduler/meter/MeterConfiguration.java b/dolphinscheduler-meter/src/main/java/org/apache/dolphinscheduler/meter/MeterConfiguration.java
index d8b7297853..b7aecf7ee6 100644
--- a/dolphinscheduler-meter/src/main/java/org/apache/dolphinscheduler/meter/MeterConfiguration.java
+++ b/dolphinscheduler-meter/src/main/java/org/apache/dolphinscheduler/meter/MeterConfiguration.java
@@ -20,6 +20,10 @@
 
 package org.apache.dolphinscheduler.meter;
 
+import javax.annotation.PostConstruct;
+
+import org.springframework.beans.factory.annotation.Autowired;
+import org.springframework.beans.factory.annotation.Value;
 import org.springframework.boot.autoconfigure.EnableAutoConfiguration;
 import org.springframework.boot.autoconfigure.condition.ConditionalOnProperty;
 import org.springframework.context.annotation.Bean;
@@ -30,6 +34,18 @@ import io.micrometer.core.aop.CountedAspect;
 import io.micrometer.core.aop.TimedAspect;
 import io.micrometer.core.instrument.MeterRegistry;
 
+/**
+ * This configuration class is used to config the metrics. We use <a href="https://micrometer.io/docs/concepts">micrometer</a> as the metrics fade.
+ *
+ * <p>To open the metrics, you need to set the property "metrics.enabled" to true. Right now, we only support expose the metrics to Prometheus,
+ * after you open metrics expose, you can get the metrics data at: http://host:port/actuator/prometheus.
+ * <p>You can use the below method to get a meter:
+ * <pre>
+ *     {@code
+ *      Counter counter = Metrics.counter("name", "tag1", "tag2");
+ *     }
+ * </pre>
+ */
 @Configuration
 @EnableAspectJAutoProxy
 @EnableAutoConfiguration
diff --git a/dolphinscheduler-meter/src/main/resources/grafana-demo/docker-compose.yaml b/dolphinscheduler-meter/src/main/resources/grafana-demo/docker-compose.yaml
index 6705f30644..185803ad4f 100644
--- a/dolphinscheduler-meter/src/main/resources/grafana-demo/docker-compose.yaml
+++ b/dolphinscheduler-meter/src/main/resources/grafana-demo/docker-compose.yaml
@@ -32,9 +32,10 @@ services:
     image: grafana/grafana
     networks: [ test ]
     ports:
-      - "3000:3000"
+      # due to the DolphinScheduler frontend port is 3000, so we change the grafana default port to 3001.
+      - "3001:3000"
     environment:
-      GF_AUTH_ANONYMOUS_ENABLED: true
+      GF_AUTH_ANONYMOUS_ENABLED: "true"
     volumes:
       - ../grafana:/dashboards:ro
       - ./datasources:/etc/grafana/provisioning/datasources:ro
diff --git a/dolphinscheduler-meter/src/main/resources/grafana/DolphinSchedulerMaster.json b/dolphinscheduler-meter/src/main/resources/grafana/DolphinSchedulerMaster.json
index 42b17494c0..90ba34c5fc 100644
--- a/dolphinscheduler-meter/src/main/resources/grafana/DolphinSchedulerMaster.json
+++ b/dolphinscheduler-meter/src/main/resources/grafana/DolphinSchedulerMaster.json
@@ -3,7 +3,10 @@
     "list": [
       {
         "builtIn": 1,
-        "datasource": "-- Grafana --",
+        "datasource": {
+          "type": "datasource",
+          "uid": "grafana"
+        },
         "enable": true,
         "hide": true,
         "iconColor": "rgba(0, 211, 255, 1)",
@@ -19,7 +22,10 @@
         "type": "dashboard"
       },
       {
-        "datasource": "Prometheus",
+        "datasource": {
+          "type": "prometheus",
+          "uid": "PBFA97CFB590B2093"
+        },
         "enable": true,
         "expr": "resets(process_uptime_seconds{application=\"$application\", instance=\"$instance\"}[1m]) > 0",
         "iconColor": "rgba(255, 96, 96, 1)",
@@ -37,32 +43,1506 @@
   "fiscalYearStartMonth": 0,
   "gnetId": 4701,
   "graphTooltip": 1,
-  "id": 4,
-  "iteration": 1636680427082,
+  "iteration": 1654674717443,
   "links": [],
   "liveNow": false,
   "panels": [
     {
-      "collapsed": false,
-      "datasource": null,
+      "collapsed": true,
       "gridPos": {
         "h": 1,
         "w": 24,
         "x": 0,
         "y": 0
       },
+      "id": 164,
+      "panels": [
+        {
+          "datasource": {
+            "type": "prometheus",
+            "uid": "PBFA97CFB590B2093"
+          },
+          "fieldConfig": {
+            "defaults": {
+              "color": {
+                "mode": "palette-classic"
+              },
+              "custom": {
+                "axisLabel": "",
+                "axisPlacement": "auto",
+                "barAlignment": 0,
+                "drawStyle": "line",
+                "fillOpacity": 0,
+                "gradientMode": "none",
+                "hideFrom": {
+                  "legend": false,
+                  "tooltip": false,
+                  "viz": false
+                },
+                "lineInterpolation": "linear",
+                "lineWidth": 1,
+                "pointSize": 5,
+                "scaleDistribution": {
+                  "type": "linear"
+                },
+                "showPoints": "auto",
+                "spanNulls": false,
+                "stacking": {
+                  "group": "A",
+                  "mode": "none"
+                },
+                "thresholdsStyle": {
+                  "mode": "off"
+                }
+              },
+              "mappings": [],
+              "thresholds": {
+                "mode": "absolute",
+                "steps": [
+                  {
+                    "color": "green",
+                    "value": null
+                  },
+                  {
+                    "color": "red",
+                    "value": 80
+                  }
+                ]
+              }
+            },
+            "overrides": []
+          },
+          "gridPos": {
+            "h": 8,
+            "w": 6,
+            "x": 0,
+            "y": 1
+          },
+          "id": 148,
+          "options": {
+            "legend": {
+              "calcs": [],
+              "displayMode": "list",
+              "placement": "bottom"
+            },
+            "tooltip": {
+              "mode": "single",
+              "sort": "none"
+            }
+          },
+          "targets": [
+            {
+              "datasource": {
+                "type": "prometheus",
+                "uid": "PBFA97CFB590B2093"
+              },
+              "editorMode": "code",
+              "expr": "increase(dolphinscheduler_master_overload_count_total[1m])",
+              "legendFormat": "",
+              "range": true,
+              "refId": "A"
+            }
+          ],
+          "title": "Master Overload/1m",
+          "type": "timeseries"
+        },
+        {
+          "datasource": {
+            "type": "prometheus",
+            "uid": "PBFA97CFB590B2093"
+          },
+          "fieldConfig": {
+            "defaults": {
+              "color": {
+                "mode": "palette-classic"
+              },
+              "custom": {
+                "axisLabel": "",
+                "axisPlacement": "auto",
+                "barAlignment": 0,
+                "drawStyle": "line",
+                "fillOpacity": 0,
+                "gradientMode": "none",
+                "hideFrom": {
+                  "legend": false,
+                  "tooltip": false,
+                  "viz": false
+                },
+                "lineInterpolation": "linear",
+                "lineWidth": 1,
+                "pointSize": 5,
+                "scaleDistribution": {
+                  "type": "linear"
+                },
+                "showPoints": "auto",
+                "spanNulls": false,
+                "stacking": {
+                  "group": "A",
+                  "mode": "none"
+                },
+                "thresholdsStyle": {
+                  "mode": "off"
+                }
+              },
+              "mappings": [],
+              "thresholds": {
+                "mode": "absolute",
+                "steps": [
+                  {
+                    "color": "green",
+                    "value": null
+                  },
+                  {
+                    "color": "red",
+                    "value": 80
+                  }
+                ]
+              }
+            },
+            "overrides": []
+          },
+          "gridPos": {
+            "h": 8,
+            "w": 6,
+            "x": 6,
+            "y": 1
+          },
+          "id": 150,
+          "options": {
+            "legend": {
+              "calcs": [],
+              "displayMode": "list",
+              "placement": "bottom"
+            },
+            "tooltip": {
+              "mode": "single",
+              "sort": "none"
+            }
+          },
+          "targets": [
+            {
+              "datasource": {
+                "type": "prometheus",
+                "uid": "PBFA97CFB590B2093"
+              },
+              "editorMode": "code",
+              "expr": "increase(dolphinscheduler_master_consume_command_count_total{}[1m])",
+              "legendFormat": "master_consume_command",
+              "range": true,
+              "refId": "A"
+            }
+          ],
+          "title": "Master Consume Command/1m",
+          "type": "timeseries"
+        },
+        {
+          "datasource": {
+            "type": "prometheus",
+            "uid": "PBFA97CFB590B2093"
+          },
+          "fieldConfig": {
+            "defaults": {
+              "color": {
+                "mode": "palette-classic"
+              },
+              "custom": {
+                "axisLabel": "",
+                "axisPlacement": "auto",
+                "barAlignment": 0,
+                "drawStyle": "line",
+                "fillOpacity": 0,
+                "gradientMode": "none",
+                "hideFrom": {
+                  "legend": false,
+                  "tooltip": false,
+                  "viz": false
+                },
+                "lineInterpolation": "linear",
+                "lineWidth": 1,
+                "pointSize": 5,
+                "scaleDistribution": {
+                  "type": "linear"
+                },
+                "showPoints": "auto",
+                "spanNulls": false,
+                "stacking": {
+                  "group": "A",
+                  "mode": "none"
+                },
+                "thresholdsStyle": {
+                  "mode": "off"
+                }
+              },
+              "mappings": [],
+              "thresholds": {
+                "mode": "absolute",
+                "steps": [
+                  {
+                    "color": "green",
+                    "value": null
+                  },
+                  {
+                    "color": "red",
+                    "value": 80
+                  }
+                ]
+              }
+            },
+            "overrides": []
+          },
+          "gridPos": {
+            "h": 8,
+            "w": 6,
+            "x": 12,
+            "y": 1
+          },
+          "id": 168,
+          "options": {
+            "legend": {
+              "calcs": [],
+              "displayMode": "list",
+              "placement": "bottom"
+            },
+            "tooltip": {
+              "mode": "single",
+              "sort": "none"
+            }
+          },
+          "targets": [
+            {
+              "datasource": {
+                "type": "prometheus",
+                "uid": "PBFA97CFB590B2093"
+              },
+              "editorMode": "code",
+              "exemplar": false,
+              "expr": "jvm_threads_live_threads{application=\"master-server\"}",
+              "legendFormat": "live_thread",
+              "range": true,
+              "refId": "A"
+            },
+            {
+              "datasource": {
+                "type": "prometheus",
+                "uid": "PBFA97CFB590B2093"
+              },
+              "editorMode": "code",
+              "expr": "jvm_threads_daemon_threads{application=\"master-server\"}",
+              "hide": false,
+              "legendFormat": "daemon_thread",
+              "range": true,
+              "refId": "B"
+            },
+            {
+              "datasource": {
+                "type": "prometheus",
+                "uid": "PBFA97CFB590B2093"
+              },
+              "editorMode": "code",
+              "expr": "jvm_threads_peak_threads{application=\"master-server\"}",
+              "hide": false,
+              "legendFormat": "peak_thread",
+              "range": true,
+              "refId": "C"
+            }
+          ],
+          "title": "JVM Thread",
+          "type": "timeseries"
+        },
+        {
+          "datasource": {
+            "type": "prometheus",
+            "uid": "PBFA97CFB590B2093"
+          },
+          "fieldConfig": {
+            "defaults": {
+              "color": {
+                "mode": "palette-classic"
+              },
+              "custom": {
+                "axisLabel": "",
+                "axisPlacement": "auto",
+                "barAlignment": 0,
+                "drawStyle": "line",
+                "fillOpacity": 0,
+                "gradientMode": "none",
+                "hideFrom": {
+                  "legend": false,
+                  "tooltip": false,
+                  "viz": false
+                },
+                "lineInterpolation": "linear",
+                "lineWidth": 1,
+                "pointSize": 5,
+                "scaleDistribution": {
+                  "type": "linear"
+                },
+                "showPoints": "auto",
+                "spanNulls": false,
+                "stacking": {
+                  "group": "A",
+                  "mode": "none"
+                },
+                "thresholdsStyle": {
+                  "mode": "off"
+                }
+              },
+              "mappings": [],
+              "thresholds": {
+                "mode": "absolute",
+                "steps": [
+                  {
+                    "color": "green",
+                    "value": null
+                  },
+                  {
+                    "color": "red",
+                    "value": 80
+                  }
+                ]
+              }
+            },
+            "overrides": []
+          },
+          "gridPos": {
+            "h": 8,
+            "w": 6,
+            "x": 18,
+            "y": 1
+          },
+          "id": 170,
+          "options": {
+            "legend": {
+              "calcs": [],
+              "displayMode": "list",
+              "placement": "bottom"
+            },
+            "tooltip": {
+              "mode": "single",
+              "sort": "none"
+            }
+          },
+          "targets": [
+            {
+              "datasource": {
+                "type": "prometheus",
+                "uid": "PBFA97CFB590B2093"
+              },
+              "expr": "jvm_threads_states_threads{application=\"master-server\"}",
+              "refId": "A"
+            }
+          ],
+          "title": "Thread Status",
+          "type": "timeseries"
+        }
+      ],
+      "title": "MasterServer",
+      "type": "row"
+    },
+    {
+      "collapsed": true,
+      "datasource": {
+        "type": "datasource",
+        "uid": "grafana"
+      },
+      "gridPos": {
+        "h": 1,
+        "w": 24,
+        "x": 0,
+        "y": 1
+      },
       "id": 126,
-      "panels": [],
+      "panels": [
+        {
+          "datasource": {
+            "type": "prometheus",
+            "uid": "PBFA97CFB590B2093"
+          },
+          "fieldConfig": {
+            "defaults": {
+              "color": {
+                "mode": "thresholds"
+              },
+              "mappings": [],
+              "thresholds": {
+                "mode": "absolute",
+                "steps": [
+                  {
+                    "color": "green",
+                    "value": null
+                  }
+                ]
+              },
+              "unit": "JOBS"
+            },
+            "overrides": []
+          },
+          "gridPos": {
+            "h": 8,
+            "w": 12,
+            "x": 0,
+            "y": 2
+          },
+          "id": 63,
+          "links": [],
+          "maxDataPoints": 100,
+          "options": {
+            "colorMode": "value",
+            "graphMode": "area",
+            "justifyMode": "auto",
+            "orientation": "horizontal",
+            "reduceOptions": {
+              "calcs": [
+                "lastNotNull"
+              ],
+              "fields": "",
+              "values": false
+            },
+            "text": {},
+            "textMode": "auto"
+          },
+          "pluginVersion": "8.5.3",
+          "targets": [
+            {
+              "exemplar": true,
+              "expr": "sum(quartz_job_executed_total)",
+              "format": "time_series",
+              "interval": "",
+              "intervalFactor": 1,
+              "legendFormat": "",
+              "metric": "",
+              "refId": "A",
+              "step": 14400
+            }
+          ],
+          "title": "Job Total Count",
+          "type": "stat"
+        },
+        {
+          "datasource": {
+            "type": "prometheus",
+            "uid": "PBFA97CFB590B2093"
+          },
+          "fieldConfig": {
+            "defaults": {
+              "color": {
+                "mode": "thresholds"
+              },
+              "mappings": [],
+              "thresholds": {
+                "mode": "absolute",
+                "steps": [
+                  {
+                    "color": "red",
+                    "value": null
+                  },
+                  {
+                    "color": "green",
+                    "value": 80
+                  }
+                ]
+              },
+              "unit": "percent"
+            },
+            "overrides": []
+          },
+          "gridPos": {
+            "h": 8,
+            "w": 12,
+            "x": 12,
+            "y": 2
+          },
+          "id": 144,
+          "links": [],
+          "maxDataPoints": 100,
+          "options": {
+            "orientation": "horizontal",
+            "reduceOptions": {
+              "calcs": [
+                "lastNotNull"
+              ],
+              "fields": "",
+              "values": false
+            },
+            "showThresholdLabels": false,
+            "showThresholdMarkers": true,
+            "text": {}
+          },
+          "pluginVersion": "8.5.3",
+          "targets": [
+            {
+              "exemplar": true,
+              "expr": "sum(quartz_job_executed_total{result=\"success\"}) / sum(quartz_job_executed_total) * 100",
+              "format": "time_series",
+              "interval": "",
+              "intervalFactor": 1,
+              "legendFormat": "",
+              "metric": "",
+              "refId": "A",
+              "step": 14400
+            }
+          ],
+          "title": "Job Successful Rate",
+          "type": "gauge"
+        },
+        {
+          "aliasColors": {},
+          "bars": false,
+          "dashLength": 10,
+          "dashes": false,
+          "datasource": {
+            "type": "prometheus",
+            "uid": "PBFA97CFB590B2093"
+          },
+          "description": "",
+          "fill": 1,
+          "fillGradient": 0,
+          "gridPos": {
+            "h": 9,
+            "w": 6,
+            "x": 0,
+            "y": 10
+          },
+          "hiddenSeries": false,
+          "id": 139,
+          "legend": {
+            "alignAsTable": false,
+            "avg": false,
+            "current": true,
+            "max": false,
+            "min": false,
+            "show": true,
+            "total": false,
+            "values": true
+          },
+          "lines": true,
+          "linewidth": 1,
+          "links": [],
+          "nullPointMode": "null",
+          "options": {
+            "alertThreshold": true
+          },
+          "percentage": false,
+          "pluginVersion": "8.5.3",
+          "pointradius": 5,
+          "points": false,
+          "renderer": "flot",
+          "seriesOverrides": [],
+          "spaceLength": 10,
+          "stack": false,
+          "steppedLine": false,
+          "targets": [
+            {
+              "exemplar": true,
+              "expr": "sum(quartz_job_executed_total{})",
+              "hide": false,
+              "interval": "",
+              "legendFormat": "Total",
+              "refId": "A"
+            },
+            {
+              "exemplar": true,
+              "expr": "quartz_job_executed_total{result=\"success\"}",
+              "format": "time_series",
+              "hide": false,
+              "interval": "",
+              "intervalFactor": 2,
+              "legendFormat": "Successful",
+              "refId": "B"
+            },
+            {
+              "exemplar": true,
+              "expr": "quartz_job_executed_total{result=\"failure\"}",
+              "format": "time_series",
+              "hide": false,
+              "interval": "",
+              "intervalFactor": 2,
+              "legendFormat": "Failed ({{exception}})",
+              "refId": "C"
+            }
+          ],
+          "thresholds": [],
+          "timeRegions": [],
+          "title": "Quartz Job Executed Count",
+          "tooltip": {
+            "shared": true,
+            "sort": 0,
+            "value_type": "individual"
+          },
+          "type": "graph",
+          "xaxis": {
+            "mode": "time",
+            "show": true,
+            "values": []
+          },
+          "yaxes": [
+            {
+              "$$hashKey": "object:1516",
+              "format": "short",
+              "logBase": 1,
+              "min": "0",
+              "show": true
+            },
+            {
+              "$$hashKey": "object:1517",
+              "format": "short",
+              "logBase": 1,
+              "show": true
+            }
+          ],
+          "yaxis": {
+            "align": false
+          }
+        },
+        {
+          "aliasColors": {},
+          "bars": false,
+          "dashLength": 10,
+          "dashes": false,
+          "datasource": {
+            "type": "prometheus",
+            "uid": "PBFA97CFB590B2093"
+          },
+          "fill": 1,
+          "fillGradient": 0,
+          "gridPos": {
+            "h": 9,
+            "w": 6,
+            "x": 6,
+            "y": 10
+          },
+          "hiddenSeries": false,
+          "id": 101,
+          "legend": {
+            "avg": false,
+            "current": false,
+            "max": false,
+            "min": false,
+            "show": true,
+            "total": false,
+            "values": false
+          },
+          "lines": true,
+          "linewidth": 1,
+          "links": [],
+          "nullPointMode": "null",
+          "options": {
+            "alertThreshold": true
+          },
+          "percentage": false,
+          "pluginVersion": "8.5.3",
+          "pointradius": 5,
+          "points": false,
+          "renderer": "flot",
+          "seriesOverrides": [],
+          "spaceLength": 10,
+          "stack": false,
+          "steppedLine": false,
+          "targets": [
+            {
+              "exemplar": true,
+              "expr": "rate(quartz_job_execution_seconds_sum[1m])/rate(quartz_job_execution_seconds_count[1m])",
+              "format": "time_series",
+              "hide": false,
+              "instant": false,
+              "interval": "",
+              "intervalFactor": 1,
+              "legendFormat": "avg ({{exception}})",
+              "refId": "A"
+            },
+            {
+              "exemplar": true,
+              "expr": "quartz_job_execution_seconds_max",
+              "format": "time_series",
+              "hide": false,
+              "instant": false,
+              "interval": "",
+              "intervalFactor": 1,
+              "legendFormat": "max ({{exception}})",
+              "refId": "B"
+            }
+          ],
+          "thresholds": [],
+          "timeRegions": [],
+          "title": "Quartz Job Execution Time",
+          "tooltip": {
+            "shared": true,
+            "sort": 0,
+            "value_type": "individual"
+          },
+          "type": "graph",
+          "xaxis": {
+            "mode": "time",
+            "show": true,
+            "values": []
+          },
+          "yaxes": [
+            {
+              "$$hashKey": "object:1671",
+              "format": "s",
+              "logBase": 1,
+              "min": "0",
+              "show": true
+            },
+            {
+              "$$hashKey": "object:1672",
+              "format": "short",
+              "label": "",
+              "logBase": 1,
+              "show": true
+            }
+          ],
+          "yaxis": {
+            "align": false
+          }
+        },
+        {
+          "aliasColors": {},
+          "bars": false,
+          "dashLength": 10,
+          "dashes": false,
+          "datasource": {
+            "type": "prometheus",
+            "uid": "PBFA97CFB590B2093"
+          },
+          "description": "",
+          "fill": 1,
+          "fillGradient": 0,
+          "gridPos": {
+            "h": 9,
+            "w": 6,
+            "x": 12,
+            "y": 10
+          },
+          "hiddenSeries": false,
+          "id": 119,
+          "legend": {
+            "alignAsTable": false,
+            "avg": false,
+            "current": true,
+            "max": false,
+            "min": false,
+            "show": true,
+            "total": false,
+            "values": true
+          },
+          "lines": true,
+          "linewidth": 1,
+          "links": [],
+          "nullPointMode": "null",
+          "options": {
+            "alertThreshold": true
+          },
+          "percentage": false,
+          "pluginVersion": "8.5.3",
+          "pointradius": 5,
+          "points": false,
+          "renderer": "flot",
+          "seriesOverrides": [],
+          "spaceLength": 10,
+          "stack": false,
+          "steppedLine": false,
+          "targets": [
+            {
+              "exemplar": true,
+              "expr": "sum(increase(quartz_job_executed_total[1m]))",
+              "hide": false,
+              "interval": "",
+              "legendFormat": "Total",
+              "refId": "A"
+            },
+            {
+              "exemplar": true,
+              "expr": "increase(quartz_job_executed_total{result=\"success\"}[1m])",
+              "format": "time_series",
+              "hide": false,
+              "interval": "",
+              "intervalFactor": 2,
+              "legendFormat": "Successful",
+              "refId": "B"
+            },
+            {
+              "exemplar": true,
+              "expr": "increase(quartz_job_executed_total{result=\"failure\"}[1m])",
+              "format": "time_series",
+              "hide": false,
+              "interval": "",
+              "intervalFactor": 2,
+              "legendFormat": "Failed ({{exception}})",
+              "refId": "C"
+            }
+          ],
+          "thresholds": [],
+          "timeRegions": [],
+          "title": "Quartz Job Executed Count / Minute",
+          "tooltip": {
+            "shared": true,
+            "sort": 0,
+            "value_type": "individual"
+          },
+          "type": "graph",
+          "xaxis": {
+            "mode": "time",
+            "show": true,
+            "values": []
+          },
+          "yaxes": [
+            {
+              "$$hashKey": "object:1516",
+              "format": "short",
+              "logBase": 1,
+              "min": "0",
+              "show": true
+            },
+            {
+              "$$hashKey": "object:1517",
+              "format": "short",
+              "logBase": 1,
+              "show": true
+            }
+          ],
+          "yaxis": {
+            "align": false
+          }
+        },
+        {
+          "cards": {},
+          "color": {
+            "cardColor": "#F2495C",
+            "colorScale": "sqrt",
+            "colorScheme": "interpolateReds",
+            "exponent": 0.5,
+            "mode": "opacity"
+          },
+          "dataFormat": "timeseries",
+          "datasource": {
+            "type": "prometheus",
+            "uid": "PBFA97CFB590B2093"
+          },
+          "gridPos": {
+            "h": 9,
+            "w": 6,
+            "x": 18,
+            "y": 10
+          },
+          "heatmap": {},
+          "hideZeroBuckets": false,
+          "highlightCards": true,
+          "id": 146,
+          "legend": {
+            "show": true
+          },
+          "pluginVersion": "8.2.3",
+          "reverseYBuckets": false,
+          "targets": [
+            {
+              "exemplar": true,
+              "expr": "histogram_quantile(0.95, sum(rate(quartz_job_execution_seconds_bucket[5m])) by (le))",
+              "interval": "",
+              "legendFormat": "",
+              "refId": "A"
+            }
+          ],
+          "title": "Quartz Job Execution Time Distribution",
+          "tooltip": {
+            "show": true,
+            "showHistogram": false
+          },
+          "type": "heatmap",
+          "xAxis": {
+            "show": true
+          },
+          "yAxis": {
+            "format": "s",
+            "logBase": 1,
+            "show": true
+          },
+          "yBucketBound": "auto"
+        }
+      ],
       "title": "Scheduler",
       "type": "row"
     },
     {
-      "cacheTimeout": null,
-      "datasource": "Prometheus",
+      "collapsed": true,
+      "gridPos": {
+        "h": 1,
+        "w": 24,
+        "x": 0,
+        "y": 2
+      },
+      "id": 166,
+      "panels": [
+        {
+          "datasource": {
+            "type": "prometheus",
+            "uid": "PBFA97CFB590B2093"
+          },
+          "fieldConfig": {
+            "defaults": {
+              "color": {
+                "mode": "palette-classic"
+              },
+              "custom": {
+                "axisLabel": "",
+                "axisPlacement": "auto",
+                "barAlignment": 0,
+                "drawStyle": "line",
+                "fillOpacity": 0,
+                "gradientMode": "none",
+                "hideFrom": {
+                  "legend": false,
+                  "tooltip": false,
+                  "viz": false
+                },
+                "lineInterpolation": "linear",
+                "lineWidth": 1,
+                "pointSize": 5,
+                "scaleDistribution": {
+                  "type": "linear"
+                },
+                "showPoints": "auto",
+                "spanNulls": false,
+                "stacking": {
+                  "group": "A",
+                  "mode": "none"
+                },
+                "thresholdsStyle": {
+                  "mode": "off"
+                }
+              },
+              "mappings": [],
+              "thresholds": {
+                "mode": "absolute",
+                "steps": [
+                  {
+                    "color": "green",
+                    "value": null
+                  },
+                  {
+                    "color": "red",
+                    "value": 80
+                  }
+                ]
+              }
+            },
+            "overrides": []
+          },
+          "gridPos": {
+            "h": 8,
+            "w": 12,
+            "x": 0,
+            "y": 3
+          },
+          "id": 152,
+          "options": {
+            "legend": {
+              "calcs": [],
+              "displayMode": "list",
+              "placement": "bottom"
+            },
+            "tooltip": {
+              "mode": "single",
+              "sort": "none"
+            }
+          },
+          "targets": [
+            {
+              "datasource": {
+                "type": "prometheus",
+                "uid": "PBFA97CFB590B2093"
+              },
+              "expr": "increase(dolphinscheduler_process_instance_submit_count_total{}[1m])",
+              "refId": "A"
+            }
+          ],
+          "title": "Process Instance Submit/1m",
+          "type": "timeseries"
+        },
+        {
+          "datasource": {
+            "type": "prometheus",
+            "uid": "PBFA97CFB590B2093"
+          },
+          "fieldConfig": {
+            "defaults": {
+              "color": {
+                "mode": "palette-classic"
+              },
+              "custom": {
+                "axisLabel": "",
+                "axisPlacement": "auto",
+                "barAlignment": 0,
+                "drawStyle": "line",
+                "fillOpacity": 0,
+                "gradientMode": "none",
+                "hideFrom": {
+                  "legend": false,
+                  "tooltip": false,
+                  "viz": false
+                },
+                "lineInterpolation": "linear",
+                "lineWidth": 1,
+                "pointSize": 5,
+                "scaleDistribution": {
+                  "type": "linear"
+                },
+                "showPoints": "auto",
+                "spanNulls": false,
+                "stacking": {
+                  "group": "A",
+                  "mode": "none"
+                },
+                "thresholdsStyle": {
+                  "mode": "off"
+                }
+              },
+              "mappings": [],
+              "thresholds": {
+                "mode": "absolute",
+                "steps": [
+                  {
+                    "color": "green",
+                    "value": null
+                  },
+                  {
+                    "color": "red",
+                    "value": 80
+                  }
+                ]
+              }
+            },
+            "overrides": []
+          },
+          "gridPos": {
+            "h": 8,
+            "w": 12,
+            "x": 12,
+            "y": 3
+          },
+          "id": 162,
+          "options": {
+            "legend": {
+              "calcs": [],
+              "displayMode": "list",
+              "placement": "bottom"
+            },
+            "tooltip": {
+              "mode": "single",
+              "sort": "none"
+            }
+          },
+          "targets": [
+            {
+              "datasource": {
+                "type": "prometheus",
+                "uid": "PBFA97CFB590B2093"
+              },
+              "expr": "increase(dolphinscheduler_process_instance_finish_count_total{}[1m])",
+              "refId": "A"
+            }
+          ],
+          "title": "Process Instance Finish/1m",
+          "type": "timeseries"
+        },
+        {
+          "datasource": {
+            "type": "prometheus",
+            "uid": "PBFA97CFB590B2093"
+          },
+          "fieldConfig": {
+            "defaults": {
+              "color": {
+                "mode": "palette-classic"
+              },
+              "custom": {
+                "axisLabel": "",
+                "axisPlacement": "auto",
+                "barAlignment": 0,
+                "drawStyle": "line",
+                "fillOpacity": 0,
+                "gradientMode": "none",
+                "hideFrom": {
+                  "legend": false,
+                  "tooltip": false,
+                  "viz": false
+                },
+                "lineInterpolation": "linear",
+                "lineWidth": 1,
+                "pointSize": 5,
+                "scaleDistribution": {
+                  "type": "linear"
+                },
+                "showPoints": "auto",
+                "spanNulls": false,
+                "stacking": {
+                  "group": "A",
+                  "mode": "none"
+                },
+                "thresholdsStyle": {
+                  "mode": "off"
+                }
+              },
+              "mappings": [],
+              "thresholds": {
+                "mode": "absolute",
+                "steps": [
+                  {
+                    "color": "green",
+                    "value": null
+                  },
+                  {
+                    "color": "red",
+                    "value": 80
+                  }
+                ]
+              }
+            },
+            "overrides": []
+          },
+          "gridPos": {
+            "h": 8,
+            "w": 6,
+            "x": 0,
+            "y": 11
+          },
+          "id": 156,
+          "options": {
+            "legend": {
+              "calcs": [],
+              "displayMode": "list",
+              "placement": "bottom"
+            },
+            "tooltip": {
+              "mode": "single",
+              "sort": "none"
+            }
+          },
+          "targets": [
+            {
+              "datasource": {
+                "type": "prometheus",
+                "uid": "PBFA97CFB590B2093"
+              },
+              "expr": "increase(dolphinscheduler_process_instance_success_count_total{}[1m])",
+              "refId": "A"
+            }
+          ],
+          "title": "Process Instance Success /1m",
+          "type": "timeseries"
+        },
+        {
+          "datasource": {
+            "type": "prometheus",
+            "uid": "PBFA97CFB590B2093"
+          },
+          "fieldConfig": {
+            "defaults": {
+              "color": {
+                "mode": "palette-classic"
+              },
+              "custom": {
+                "axisLabel": "",
+                "axisPlacement": "auto",
+                "barAlignment": 0,
+                "drawStyle": "line",
+                "fillOpacity": 0,
+                "gradientMode": "none",
+                "hideFrom": {
+                  "legend": false,
+                  "tooltip": false,
+                  "viz": false
+                },
+                "lineInterpolation": "linear",
+                "lineWidth": 1,
+                "pointSize": 5,
+                "scaleDistribution": {
+                  "type": "linear"
+                },
+                "showPoints": "auto",
+                "spanNulls": false,
+                "stacking": {
+                  "group": "A",
+                  "mode": "none"
+                },
+                "thresholdsStyle": {
+                  "mode": "off"
+                }
+              },
+              "mappings": [],
+              "thresholds": {
+                "mode": "absolute",
+                "steps": [
+                  {
+                    "color": "green",
+                    "value": null
+                  },
+                  {
+                    "color": "red",
+                    "value": 80
+                  }
+                ]
+              }
+            },
+            "overrides": []
+          },
+          "gridPos": {
+            "h": 8,
+            "w": 6,
+            "x": 6,
+            "y": 11
+          },
+          "id": 160,
+          "options": {
+            "legend": {
+              "calcs": [],
+              "displayMode": "list",
+              "placement": "bottom"
+            },
+            "tooltip": {
+              "mode": "single",
+              "sort": "none"
+            }
+          },
+          "targets": [
+            {
+              "datasource": {
+                "type": "prometheus",
+                "uid": "PBFA97CFB590B2093"
+              },
+              "expr": "increase(dolphinscheduler_process_instance_stop_count_total{}[1m])",
+              "refId": "A"
+            }
+          ],
+          "title": "Process Instance Stop/1m",
+          "type": "timeseries"
+        },
+        {
+          "datasource": {
+            "type": "prometheus",
+            "uid": "PBFA97CFB590B2093"
+          },
+          "fieldConfig": {
+            "defaults": {
+              "color": {
+                "mode": "palette-classic"
+              },
+              "custom": {
+                "axisLabel": "",
+                "axisPlacement": "auto",
+                "barAlignment": 0,
+                "drawStyle": "line",
+                "fillOpacity": 0,
+                "gradientMode": "none",
+                "hideFrom": {
+                  "legend": false,
+                  "tooltip": false,
+                  "viz": false
+                },
+                "lineInterpolation": "linear",
+                "lineWidth": 1,
+                "pointSize": 5,
+                "scaleDistribution": {
+                  "type": "linear"
+                },
+                "showPoints": "auto",
+                "spanNulls": false,
+                "stacking": {
+                  "group": "A",
+                  "mode": "none"
+                },
+                "thresholdsStyle": {
+                  "mode": "off"
+                }
+              },
+              "mappings": [],
+              "thresholds": {
+                "mode": "absolute",
+                "steps": [
+                  {
+                    "color": "green",
+                    "value": null
+                  },
+                  {
+                    "color": "red",
+                    "value": 80
+                  }
+                ]
+              }
+            },
+            "overrides": []
+          },
+          "gridPos": {
+            "h": 8,
+            "w": 6,
+            "x": 12,
+            "y": 11
+          },
+          "id": 154,
+          "options": {
+            "legend": {
+              "calcs": [],
+              "displayMode": "list",
+              "placement": "bottom"
+            },
+            "tooltip": {
+              "mode": "single",
+              "sort": "none"
+            }
+          },
+          "targets": [
+            {
+              "datasource": {
+                "type": "prometheus",
+                "uid": "PBFA97CFB590B2093"
+              },
+              "expr": "increase(dolphinscheduler_process_instance_timeout_count_total{}[1m])",
+              "refId": "A"
+            }
+          ],
+          "title": "Process Instance Timeout/1m",
+          "type": "timeseries"
+        },
+        {
+          "datasource": {
+            "type": "prometheus",
+            "uid": "PBFA97CFB590B2093"
+          },
+          "fieldConfig": {
+            "defaults": {
+              "color": {
+                "mode": "palette-classic"
+              },
+              "custom": {
+                "axisLabel": "",
+                "axisPlacement": "auto",
+                "barAlignment": 0,
+                "drawStyle": "line",
+                "fillOpacity": 0,
+                "gradientMode": "none",
+                "hideFrom": {
+                  "legend": false,
+                  "tooltip": false,
+                  "viz": false
+                },
+                "lineInterpolation": "linear",
+                "lineWidth": 1,
+                "pointSize": 5,
+                "scaleDistribution": {
+                  "type": "linear"
+                },
+                "showPoints": "auto",
+                "spanNulls": false,
+                "stacking": {
+                  "group": "A",
+                  "mode": "none"
+                },
+                "thresholdsStyle": {
+                  "mode": "off"
+                }
+              },
+              "mappings": [],
+              "thresholds": {
+                "mode": "absolute",
+                "steps": [
+                  {
+                    "color": "green",
+                    "value": null
+                  },
+                  {
+                    "color": "red",
+                    "value": 80
+                  }
+                ]
+              }
+            },
+            "overrides": []
+          },
+          "gridPos": {
+            "h": 8,
+            "w": 6,
+            "x": 18,
+            "y": 11
+          },
+          "id": 158,
+          "options": {
+            "legend": {
+              "calcs": [],
+              "displayMode": "list",
+              "placement": "bottom"
+            },
+            "tooltip": {
+              "mode": "single",
+              "sort": "none"
+            }
+          },
+          "targets": [
+            {
+              "datasource": {
+                "type": "prometheus",
+                "uid": "PBFA97CFB590B2093"
+              },
+              "expr": "increase(dolphinscheduler_process_instance_failure_count_total{}[1m])",
+              "refId": "A"
+            }
+          ],
+          "title": "Process Instance Failure/1m",
+          "type": "timeseries"
+        }
+      ],
+      "title": "ProcessInstance",
+      "type": "row"
+    },
+    {
+      "collapsed": false,
+      "gridPos": {
+        "h": 1,
+        "w": 24,
+        "x": 0,
+        "y": 3
+      },
+      "id": 172,
+      "panels": [],
+      "title": "Task",
+      "type": "row"
+    },
+    {
+      "datasource": {
+        "type": "prometheus",
+        "uid": "PBFA97CFB590B2093"
+      },
       "fieldConfig": {
         "defaults": {
           "color": {
-            "mode": "thresholds"
+            "mode": "palette-classic"
+          },
+          "custom": {
+            "axisLabel": "",
+            "axisPlacement": "auto",
+            "barAlignment": 0,
+            "drawStyle": "line",
+            "fillOpacity": 0,
+            "gradientMode": "none",
+            "hideFrom": {
+              "legend": false,
+              "tooltip": false,
+              "viz": false
+            },
+            "lineInterpolation": "linear",
+            "lineWidth": 1,
+            "pointSize": 5,
+            "scaleDistribution": {
+              "type": "linear"
+            },
+            "showPoints": "auto",
+            "spanNulls": false,
+            "stacking": {
+              "group": "A",
+              "mode": "none"
+            },
+            "thresholdsStyle": {
+              "mode": "off"
+            }
           },
           "mappings": [],
           "thresholds": {
@@ -71,541 +1551,771 @@
               {
                 "color": "green",
                 "value": null
+              },
+              {
+                "color": "red",
+                "value": 80
               }
             ]
-          },
-          "unit": "JOBS"
+          }
         },
         "overrides": []
       },
       "gridPos": {
         "h": 8,
-        "w": 12,
+        "w": 8,
         "x": 0,
-        "y": 1
+        "y": 4
       },
-      "id": 63,
-      "interval": null,
-      "links": [],
-      "maxDataPoints": 100,
+      "id": 178,
       "options": {
-        "colorMode": "value",
-        "graphMode": "area",
-        "justifyMode": "auto",
-        "orientation": "horizontal",
-        "reduceOptions": {
-          "calcs": [
-            "lastNotNull"
-          ],
-          "fields": "",
-          "values": false
+        "legend": {
+          "calcs": [],
+          "displayMode": "list",
+          "placement": "bottom"
         },
-        "text": {},
-        "textMode": "auto"
+        "tooltip": {
+          "mode": "single",
+          "sort": "none"
+        }
       },
-      "pluginVersion": "8.2.3",
       "targets": [
         {
-          "exemplar": true,
-          "expr": "sum(quartz_job_executed_total)",
-          "format": "time_series",
-          "interval": "",
-          "intervalFactor": 1,
-          "legendFormat": "",
-          "metric": "",
-          "refId": "A",
-          "step": 14400
+          "datasource": {
+            "type": "prometheus",
+            "uid": "PBFA97CFB590B2093"
+          },
+          "expr": "sum(increase(dolphinscheduler_task_dispatch_count_total{}[1m]))",
+          "refId": "A"
+        },
+        {
+          "datasource": {
+            "type": "prometheus",
+            "uid": "PBFA97CFB590B2093"
+          },
+          "expr": "sum(increase(dolphinscheduler_task_dispatch_failed_count_total{}[1m]))",
+          "hide": false,
+          "refId": "B"
+        },
+        {
+          "datasource": {
+            "type": "prometheus",
+            "uid": "PBFA97CFB590B2093"
+          },
+          "expr": "sum(increase(dolphinscheduler_task_dispatch_error_count_total{}[1m]))",
+          "hide": false,
+          "refId": "C"
         }
       ],
-      "title": "Job Total Count",
-      "type": "stat"
+      "title": "Task Dispatch Count/1m",
+      "type": "timeseries"
     },
     {
-      "cacheTimeout": null,
-      "datasource": "Prometheus",
+      "datasource": {
+        "type": "prometheus",
+        "uid": "PBFA97CFB590B2093"
+      },
       "fieldConfig": {
         "defaults": {
           "color": {
-            "mode": "thresholds"
+            "mode": "palette-classic"
+          },
+          "custom": {
+            "axisLabel": "",
+            "axisPlacement": "auto",
+            "barAlignment": 0,
+            "drawStyle": "line",
+            "fillOpacity": 0,
+            "gradientMode": "none",
+            "hideFrom": {
+              "legend": false,
+              "tooltip": false,
+              "viz": false
+            },
+            "lineInterpolation": "linear",
+            "lineWidth": 1,
+            "pointSize": 5,
+            "scaleDistribution": {
+              "type": "linear"
+            },
+            "showPoints": "auto",
+            "spanNulls": false,
+            "stacking": {
+              "group": "A",
+              "mode": "none"
+            },
+            "thresholdsStyle": {
+              "mode": "off"
+            }
           },
           "mappings": [],
           "thresholds": {
             "mode": "absolute",
             "steps": [
               {
-                "color": "red",
+                "color": "green",
                 "value": null
               },
               {
-                "color": "green",
+                "color": "red",
                 "value": 80
               }
             ]
-          },
-          "unit": "percent"
+          }
         },
         "overrides": []
       },
       "gridPos": {
         "h": 8,
-        "w": 12,
-        "x": 12,
-        "y": 1
+        "w": 8,
+        "x": 8,
+        "y": 4
       },
-      "id": 144,
-      "interval": null,
-      "links": [],
-      "maxDataPoints": 100,
+      "id": 180,
       "options": {
-        "orientation": "horizontal",
-        "reduceOptions": {
-          "calcs": [
-            "lastNotNull"
-          ],
-          "fields": "",
-          "values": false
+        "legend": {
+          "calcs": [],
+          "displayMode": "list",
+          "placement": "bottom"
         },
-        "showThresholdLabels": false,
-        "showThresholdMarkers": true,
-        "text": {}
+        "tooltip": {
+          "mode": "single",
+          "sort": "none"
+        }
       },
-      "pluginVersion": "8.2.3",
       "targets": [
         {
-          "exemplar": true,
-          "expr": "sum(quartz_job_executed_total{result=\"success\"}) / sum(quartz_job_executed_total) * 100",
-          "format": "time_series",
-          "interval": "",
-          "intervalFactor": 1,
-          "legendFormat": "",
-          "metric": "",
-          "refId": "A",
-          "step": 14400
+          "datasource": {
+            "type": "prometheus",
+            "uid": "PBFA97CFB590B2093"
+          },
+          "expr": "sum(increase(dolphinscheduler_task_submit_count_total{}[1m]))",
+          "refId": "A"
         }
       ],
-      "title": "Job Successful Rate",
-      "type": "gauge"
+      "title": "Task Submit Count/1m",
+      "type": "timeseries"
     },
     {
-      "aliasColors": {},
-      "bars": false,
-      "dashLength": 10,
-      "dashes": false,
-      "datasource": "Prometheus",
-      "description": "",
-      "fill": 1,
-      "fillGradient": 0,
+      "datasource": {
+        "type": "prometheus",
+        "uid": "PBFA97CFB590B2093"
+      },
+      "fieldConfig": {
+        "defaults": {
+          "color": {
+            "mode": "palette-classic"
+          },
+          "custom": {
+            "axisLabel": "",
+            "axisPlacement": "auto",
+            "barAlignment": 0,
+            "drawStyle": "line",
+            "fillOpacity": 0,
+            "gradientMode": "none",
+            "hideFrom": {
+              "legend": false,
+              "tooltip": false,
+              "viz": false
+            },
+            "lineInterpolation": "linear",
+            "lineWidth": 1,
+            "pointSize": 5,
+            "scaleDistribution": {
+              "type": "linear"
+            },
+            "showPoints": "auto",
+            "spanNulls": false,
+            "stacking": {
+              "group": "A",
+              "mode": "none"
+            },
+            "thresholdsStyle": {
+              "mode": "off"
+            }
+          },
+          "mappings": [],
+          "thresholds": {
+            "mode": "absolute",
+            "steps": [
+              {
+                "color": "green",
+                "value": null
+              },
+              {
+                "color": "red",
+                "value": 80
+              }
+            ]
+          }
+        },
+        "overrides": []
+      },
       "gridPos": {
-        "h": 9,
-        "w": 12,
-        "x": 0,
-        "y": 9
-      },
-      "hiddenSeries": false,
-      "id": 139,
-      "legend": {
-        "alignAsTable": false,
-        "avg": false,
-        "current": true,
-        "max": false,
-        "min": false,
-        "show": true,
-        "total": false,
-        "values": true
-      },
-      "lines": true,
-      "linewidth": 1,
-      "links": [],
-      "nullPointMode": "null",
+        "h": 8,
+        "w": 8,
+        "x": 16,
+        "y": 4
+      },
+      "id": 182,
       "options": {
-        "alertThreshold": true
-      },
-      "percentage": false,
-      "pluginVersion": "8.2.3",
-      "pointradius": 5,
-      "points": false,
-      "renderer": "flot",
-      "seriesOverrides": [],
-      "spaceLength": 10,
-      "stack": false,
-      "steppedLine": false,
+        "legend": {
+          "calcs": [],
+          "displayMode": "list",
+          "placement": "bottom"
+        },
+        "tooltip": {
+          "mode": "single",
+          "sort": "none"
+        }
+      },
       "targets": [
         {
-          "exemplar": true,
-          "expr": "sum(quartz_job_executed_total{})",
-          "hide": false,
-          "interval": "",
-          "legendFormat": "Total",
+          "datasource": {
+            "type": "prometheus",
+            "uid": "PBFA97CFB590B2093"
+          },
+          "expr": "sum(increase(dolphinscheduler_task_finish_count_total{}[1m]))",
           "refId": "A"
-        },
-        {
-          "exemplar": true,
-          "expr": "quartz_job_executed_total{result=\"success\"}",
-          "format": "time_series",
-          "hide": false,
-          "interval": "",
-          "intervalFactor": 2,
-          "legendFormat": "Successful",
-          "refId": "B"
-        },
-        {
-          "exemplar": true,
-          "expr": "quartz_job_executed_total{result=\"failure\"}",
-          "format": "time_series",
-          "hide": false,
-          "interval": "",
-          "intervalFactor": 2,
-          "legendFormat": "Failed ({{exception}})",
-          "refId": "C"
         }
       ],
-      "thresholds": [],
-      "timeFrom": null,
-      "timeRegions": [],
-      "timeShift": null,
-      "title": "Job Executed Count",
-      "tooltip": {
-        "shared": true,
-        "sort": 0,
-        "value_type": "individual"
+      "title": "Task Finish Count/1m",
+      "type": "timeseries"
+    },
+    {
+      "datasource": {
+        "type": "prometheus",
+        "uid": "PBFA97CFB590B2093"
       },
-      "type": "graph",
-      "xaxis": {
-        "buckets": null,
-        "mode": "time",
-        "name": null,
-        "show": true,
-        "values": []
+      "fieldConfig": {
+        "defaults": {
+          "color": {
+            "mode": "palette-classic"
+          },
+          "custom": {
+            "axisLabel": "",
+            "axisPlacement": "auto",
+            "barAlignment": 0,
+            "drawStyle": "line",
+            "fillOpacity": 0,
+            "gradientMode": "none",
+            "hideFrom": {
+              "legend": false,
+              "tooltip": false,
+              "viz": false
+            },
+            "lineInterpolation": "linear",
+            "lineWidth": 1,
+            "pointSize": 5,
+            "scaleDistribution": {
+              "type": "linear"
+            },
+            "showPoints": "auto",
+            "spanNulls": false,
+            "stacking": {
+              "group": "A",
+              "mode": "none"
+            },
+            "thresholdsStyle": {
+              "mode": "off"
+            }
+          },
+          "mappings": [],
+          "thresholds": {
+            "mode": "absolute",
+            "steps": [
+              {
+                "color": "green",
+                "value": null
+              },
+              {
+                "color": "red",
+                "value": 80
+              }
+            ]
+          }
+        },
+        "overrides": []
       },
-      "yaxes": [
-        {
-          "$$hashKey": "object:1516",
-          "format": "short",
-          "label": null,
-          "logBase": 1,
-          "max": null,
-          "min": "0",
-          "show": true
+      "gridPos": {
+        "h": 8,
+        "w": 8,
+        "x": 0,
+        "y": 12
+      },
+      "id": 184,
+      "options": {
+        "legend": {
+          "calcs": [],
+          "displayMode": "list",
+          "placement": "bottom"
         },
+        "tooltip": {
+          "mode": "single",
+          "sort": "none"
+        }
+      },
+      "targets": [
         {
-          "$$hashKey": "object:1517",
-          "format": "short",
-          "label": null,
-          "logBase": 1,
-          "max": null,
-          "min": null,
-          "show": true
+          "datasource": {
+            "type": "prometheus",
+            "uid": "PBFA97CFB590B2093"
+          },
+          "expr": "sum(increase(dolphinscheduler_task_success_count_total{}[1m]))",
+          "refId": "A"
         }
       ],
-      "yaxis": {
-        "align": false,
-        "alignLevel": null
-      }
+      "title": "Task Success Count/1m",
+      "type": "timeseries"
     },
     {
-      "aliasColors": {},
-      "bars": false,
-      "dashLength": 10,
-      "dashes": false,
-      "datasource": "Prometheus",
-      "description": "",
-      "fill": 1,
-      "fillGradient": 0,
+      "datasource": {
+        "type": "prometheus",
+        "uid": "PBFA97CFB590B2093"
+      },
+      "fieldConfig": {
+        "defaults": {
+          "color": {
+            "mode": "palette-classic"
+          },
+          "custom": {
+            "axisLabel": "",
+            "axisPlacement": "auto",
+            "barAlignment": 0,
+            "drawStyle": "line",
+            "fillOpacity": 0,
+            "gradientMode": "none",
+            "hideFrom": {
+              "legend": false,
+              "tooltip": false,
+              "viz": false
+            },
+            "lineInterpolation": "linear",
+            "lineWidth": 1,
+            "pointSize": 5,
+            "scaleDistribution": {
+              "type": "linear"
+            },
+            "showPoints": "auto",
+            "spanNulls": false,
+            "stacking": {
+              "group": "A",
+              "mode": "none"
+            },
+            "thresholdsStyle": {
+              "mode": "off"
+            }
+          },
+          "mappings": [],
+          "thresholds": {
+            "mode": "absolute",
+            "steps": [
+              {
+                "color": "green",
+                "value": null
+              },
+              {
+                "color": "red",
+                "value": 80
+              }
+            ]
+          }
+        },
+        "overrides": []
+      },
       "gridPos": {
-        "h": 9,
-        "w": 12,
-        "x": 12,
-        "y": 9
-      },
-      "hiddenSeries": false,
-      "id": 119,
-      "legend": {
-        "alignAsTable": false,
-        "avg": false,
-        "current": true,
-        "max": false,
-        "min": false,
-        "show": true,
-        "total": false,
-        "values": true
-      },
-      "lines": true,
-      "linewidth": 1,
-      "links": [],
-      "nullPointMode": "null",
+        "h": 8,
+        "w": 8,
+        "x": 8,
+        "y": 12
+      },
+      "id": 186,
       "options": {
-        "alertThreshold": true
-      },
-      "percentage": false,
-      "pluginVersion": "8.2.3",
-      "pointradius": 5,
-      "points": false,
-      "renderer": "flot",
-      "seriesOverrides": [],
-      "spaceLength": 10,
-      "stack": false,
-      "steppedLine": false,
+        "legend": {
+          "calcs": [],
+          "displayMode": "list",
+          "placement": "bottom"
+        },
+        "tooltip": {
+          "mode": "single",
+          "sort": "none"
+        }
+      },
       "targets": [
         {
-          "exemplar": true,
-          "expr": "sum(increase(quartz_job_executed_total[1m]))",
-          "hide": false,
-          "interval": "",
-          "legendFormat": "Total",
+          "datasource": {
+            "type": "prometheus",
+            "uid": "PBFA97CFB590B2093"
+          },
+          "expr": "sum(increase(dolphinscheduler_task_failure_count_total{}[1m]))",
           "refId": "A"
-        },
-        {
-          "exemplar": true,
-          "expr": "increase(quartz_job_executed_total{result=\"success\"}[1m])",
-          "format": "time_series",
-          "hide": false,
-          "interval": "",
-          "intervalFactor": 2,
-          "legendFormat": "Successful",
-          "refId": "B"
-        },
-        {
-          "exemplar": true,
-          "expr": "increase(quartz_job_executed_total{result=\"failure\"}[1m])",
-          "format": "time_series",
-          "hide": false,
-          "interval": "",
-          "intervalFactor": 2,
-          "legendFormat": "Failed ({{exception}})",
-          "refId": "C"
         }
       ],
-      "thresholds": [],
-      "timeFrom": null,
-      "timeRegions": [],
-      "timeShift": null,
-      "title": "Job Executed Count / Minute",
-      "tooltip": {
-        "shared": true,
-        "sort": 0,
-        "value_type": "individual"
+      "title": "Task failure Count/1m",
+      "type": "timeseries"
+    },
+    {
+      "datasource": {
+        "type": "prometheus",
+        "uid": "PBFA97CFB590B2093"
+      },
+      "fieldConfig": {
+        "defaults": {
+          "color": {
+            "mode": "palette-classic"
+          },
+          "custom": {
+            "axisLabel": "",
+            "axisPlacement": "auto",
+            "barAlignment": 0,
+            "drawStyle": "line",
+            "fillOpacity": 0,
+            "gradientMode": "none",
+            "hideFrom": {
+              "legend": false,
+              "tooltip": false,
+              "viz": false
+            },
+            "lineInterpolation": "linear",
+            "lineWidth": 1,
+            "pointSize": 5,
+            "scaleDistribution": {
+              "type": "linear"
+            },
+            "showPoints": "auto",
+            "spanNulls": false,
+            "stacking": {
+              "group": "A",
+              "mode": "none"
+            },
+            "thresholdsStyle": {
+              "mode": "off"
+            }
+          },
+          "mappings": [],
+          "thresholds": {
+            "mode": "absolute",
+            "steps": [
+              {
+                "color": "green",
+                "value": null
+              },
+              {
+                "color": "red",
+                "value": 80
+              }
+            ]
+          }
+        },
+        "overrides": []
       },
-      "type": "graph",
-      "xaxis": {
-        "buckets": null,
-        "mode": "time",
-        "name": null,
-        "show": true,
-        "values": []
+      "gridPos": {
+        "h": 8,
+        "w": 8,
+        "x": 16,
+        "y": 12
       },
-      "yaxes": [
-        {
-          "$$hashKey": "object:1516",
-          "format": "short",
-          "label": null,
-          "logBase": 1,
-          "max": null,
-          "min": "0",
-          "show": true
+      "id": 188,
+      "options": {
+        "legend": {
+          "calcs": [],
+          "displayMode": "list",
+          "placement": "bottom"
         },
+        "tooltip": {
+          "mode": "single",
+          "sort": "none"
+        }
+      },
+      "targets": [
         {
-          "$$hashKey": "object:1517",
-          "format": "short",
-          "label": null,
-          "logBase": 1,
-          "max": null,
-          "min": null,
-          "show": true
+          "datasource": {
+            "type": "prometheus",
+            "uid": "PBFA97CFB590B2093"
+          },
+          "expr": "sum(increase(dolphinscheduler_task_timeout_count_total{}[1m]))",
+          "refId": "A"
         }
       ],
-      "yaxis": {
-        "align": false,
-        "alignLevel": null
-      }
+      "title": "Task Timeout Count/1m",
+      "type": "timeseries"
     },
     {
-      "aliasColors": {},
-      "bars": false,
-      "dashLength": 10,
-      "dashes": false,
-      "datasource": "Prometheus",
-      "fill": 1,
-      "fillGradient": 0,
+      "datasource": {
+        "type": "prometheus",
+        "uid": "PBFA97CFB590B2093"
+      },
+      "fieldConfig": {
+        "defaults": {
+          "color": {
+            "mode": "palette-classic"
+          },
+          "custom": {
+            "axisLabel": "",
+            "axisPlacement": "auto",
+            "barAlignment": 0,
+            "drawStyle": "line",
+            "fillOpacity": 0,
+            "gradientMode": "none",
+            "hideFrom": {
+              "legend": false,
+              "tooltip": false,
+              "viz": false
+            },
+            "lineInterpolation": "linear",
+            "lineWidth": 1,
+            "pointSize": 5,
+            "scaleDistribution": {
+              "type": "linear"
+            },
+            "showPoints": "auto",
+            "spanNulls": false,
+            "stacking": {
+              "group": "A",
+              "mode": "none"
+            },
+            "thresholdsStyle": {
+              "mode": "off"
+            }
+          },
+          "mappings": [],
+          "thresholds": {
+            "mode": "absolute",
+            "steps": [
+              {
+                "color": "green",
+                "value": null
+              },
+              {
+                "color": "red",
+                "value": 80
+              }
+            ]
+          }
+        },
+        "overrides": []
+      },
       "gridPos": {
-        "h": 9,
-        "w": 12,
+        "h": 8,
+        "w": 8,
         "x": 0,
-        "y": 18
-      },
-      "hiddenSeries": false,
-      "id": 101,
-      "legend": {
-        "avg": false,
-        "current": false,
-        "max": false,
-        "min": false,
-        "show": true,
-        "total": false,
-        "values": false
-      },
-      "lines": true,
-      "linewidth": 1,
-      "links": [],
-      "nullPointMode": "null",
+        "y": 20
+      },
+      "id": 190,
       "options": {
-        "alertThreshold": true
-      },
-      "percentage": false,
-      "pluginVersion": "8.2.3",
-      "pointradius": 5,
-      "points": false,
-      "renderer": "flot",
-      "seriesOverrides": [],
-      "spaceLength": 10,
-      "stack": false,
-      "steppedLine": false,
+        "legend": {
+          "calcs": [],
+          "displayMode": "list",
+          "placement": "bottom"
+        },
+        "tooltip": {
+          "mode": "single",
+          "sort": "none"
+        }
+      },
       "targets": [
         {
-          "exemplar": true,
-          "expr": "rate(quartz_job_execution_seconds_sum[1m])/rate(quartz_job_execution_seconds_count[1m])",
-          "format": "time_series",
-          "hide": false,
-          "instant": false,
-          "interval": "",
-          "intervalFactor": 1,
-          "legendFormat": "avg ({{exception}})",
+          "datasource": {
+            "type": "prometheus",
+            "uid": "PBFA97CFB590B2093"
+          },
+          "expr": "sum(increase(dolphinscheduler_task_retry_count_total{}[1m]))",
           "refId": "A"
-        },
-        {
-          "exemplar": true,
-          "expr": "quartz_job_execution_seconds_max",
-          "format": "time_series",
-          "hide": false,
-          "instant": false,
-          "interval": "",
-          "intervalFactor": 1,
-          "legendFormat": "max ({{exception}})",
-          "refId": "B"
         }
       ],
-      "thresholds": [],
-      "timeFrom": null,
-      "timeRegions": [],
-      "timeShift": null,
-      "title": "Job Execution Time",
-      "tooltip": {
-        "shared": true,
-        "sort": 0,
-        "value_type": "individual"
+      "title": "Task Retry Count/1m",
+      "type": "timeseries"
+    },
+    {
+      "datasource": {
+        "type": "prometheus",
+        "uid": "PBFA97CFB590B2093"
       },
-      "type": "graph",
-      "xaxis": {
-        "buckets": null,
-        "mode": "time",
-        "name": null,
-        "show": true,
-        "values": []
+      "fieldConfig": {
+        "defaults": {
+          "color": {
+            "mode": "palette-classic"
+          },
+          "custom": {
+            "axisLabel": "",
+            "axisPlacement": "auto",
+            "barAlignment": 0,
+            "drawStyle": "line",
+            "fillOpacity": 0,
+            "gradientMode": "none",
+            "hideFrom": {
+              "legend": false,
+              "tooltip": false,
+              "viz": false
+            },
+            "lineInterpolation": "linear",
+            "lineWidth": 1,
+            "pointSize": 5,
+            "scaleDistribution": {
+              "type": "linear"
+            },
+            "showPoints": "auto",
+            "spanNulls": false,
+            "stacking": {
+              "group": "A",
+              "mode": "none"
+            },
+            "thresholdsStyle": {
+              "mode": "off"
+            }
+          },
+          "mappings": [],
+          "thresholds": {
+            "mode": "absolute",
+            "steps": [
+              {
+                "color": "green",
+                "value": null
+              },
+              {
+                "color": "red",
+                "value": 80
+              }
+            ]
+          }
+        },
+        "overrides": []
       },
-      "yaxes": [
-        {
-          "$$hashKey": "object:1671",
-          "format": "s",
-          "label": null,
-          "logBase": 1,
-          "max": null,
-          "min": "0",
-          "show": true
+      "gridPos": {
+        "h": 8,
+        "w": 8,
+        "x": 8,
+        "y": 20
+      },
+      "id": 192,
+      "options": {
+        "legend": {
+          "calcs": [],
+          "displayMode": "list",
+          "placement": "bottom"
         },
+        "tooltip": {
+          "mode": "single",
+          "sort": "none"
+        }
+      },
+      "targets": [
         {
-          "$$hashKey": "object:1672",
-          "format": "short",
-          "label": "",
-          "logBase": 1,
-          "max": null,
-          "min": null,
-          "show": true
+          "datasource": {
+            "type": "prometheus",
+            "uid": "PBFA97CFB590B2093"
+          },
+          "expr": "sum(increase(dolphinscheduler_task_stop_count_total{}[1m]))",
+          "refId": "A"
         }
       ],
-      "yaxis": {
-        "align": false,
-        "alignLevel": null
-      }
+      "title": "Task Stop Count/1m",
+      "type": "timeseries"
     },
     {
-      "cards": {
-        "cardPadding": null,
-        "cardRound": null
-      },
-      "color": {
-        "cardColor": "#F2495C",
-        "colorScale": "sqrt",
-        "colorScheme": "interpolateReds",
-        "exponent": 0.5,
-        "mode": "opacity"
-      },
-      "dataFormat": "timeseries",
-      "datasource": "Prometheus",
+      "datasource": {
+        "type": "prometheus",
+        "uid": "PBFA97CFB590B2093"
+      },
+      "fieldConfig": {
+        "defaults": {
+          "color": {
+            "mode": "palette-classic"
+          },
+          "custom": {
+            "axisLabel": "",
+            "axisPlacement": "auto",
+            "barAlignment": 0,
+            "drawStyle": "line",
+            "fillOpacity": 0,
+            "gradientMode": "none",
+            "hideFrom": {
+              "legend": false,
+              "tooltip": false,
+              "viz": false
+            },
+            "lineInterpolation": "linear",
+            "lineWidth": 1,
+            "pointSize": 5,
+            "scaleDistribution": {
+              "type": "linear"
+            },
+            "showPoints": "auto",
+            "spanNulls": false,
+            "stacking": {
+              "group": "A",
+              "mode": "none"
+            },
+            "thresholdsStyle": {
+              "mode": "off"
+            }
+          },
+          "mappings": [],
+          "thresholds": {
+            "mode": "absolute",
+            "steps": [
+              {
+                "color": "green",
+                "value": null
+              },
+              {
+                "color": "red",
+                "value": 80
+              }
+            ]
+          }
+        },
+        "overrides": []
+      },
       "gridPos": {
-        "h": 9,
-        "w": 12,
-        "x": 12,
-        "y": 18
-      },
-      "heatmap": {},
-      "hideZeroBuckets": false,
-      "highlightCards": true,
-      "id": 146,
-      "legend": {
-        "show": true
-      },
-      "pluginVersion": "8.2.3",
-      "reverseYBuckets": false,
+        "h": 8,
+        "w": 8,
+        "x": 16,
+        "y": 20
+      },
+      "id": 194,
+      "options": {
+        "legend": {
+          "calcs": [],
+          "displayMode": "list",
+          "placement": "bottom"
+        },
+        "tooltip": {
+          "mode": "single",
+          "sort": "none"
+        }
+      },
       "targets": [
         {
-          "exemplar": true,
-          "expr": "histogram_quantile(0.95, sum(rate(quartz_job_execution_seconds_bucket[5m])) by (le))",
-          "interval": "",
-          "legendFormat": "",
+          "datasource": {
+            "type": "prometheus",
+            "uid": "PBFA97CFB590B2093"
+          },
+          "expr": "sum(increase(dolphinscheduler_task_failover_count_total{}[1m]))",
           "refId": "A"
         }
       ],
-      "title": "Job Execution Time Distribution",
-      "tooltip": {
-        "show": true,
-        "showHistogram": false
-      },
-      "type": "heatmap",
-      "xAxis": {
-        "show": true
-      },
-      "xBucketNumber": null,
-      "xBucketSize": null,
-      "yAxis": {
-        "decimals": null,
-        "format": "s",
-        "logBase": 1,
-        "max": null,
-        "min": null,
-        "show": true,
-        "splitFactor": null
-      },
-      "yBucketBound": "auto",
-      "yBucketNumber": null,
-      "yBucketSize": null
+      "title": "Task Failover Count/1m",
+      "type": "timeseries"
     }
   ],
   "refresh": "5s",
-  "schemaVersion": 31,
+  "schemaVersion": 36,
   "style": "dark",
   "tags": [],
   "templating": {
     "list": [
       {
-        "allValue": null,
         "current": {
           "selected": false,
-          "text": "master",
-          "value": "master"
+          "text": "master-server",
+          "value": "master-server"
+        },
+        "datasource": {
+          "type": "prometheus",
+          "uid": "PBFA97CFB590B2093"
         },
-        "datasource": "Prometheus",
         "definition": "",
-        "description": null,
-        "error": null,
         "hide": 0,
         "includeAll": false,
         "label": "Application",
@@ -627,16 +2337,16 @@
       },
       {
         "allFormat": "glob",
-        "allValue": null,
         "current": {
           "selected": false,
           "text": "host.docker.internal:5679",
           "value": "host.docker.internal:5679"
         },
-        "datasource": "Prometheus",
+        "datasource": {
+          "type": "prometheus",
+          "uid": "PBFA97CFB590B2093"
+        },
         "definition": "",
-        "description": null,
-        "error": null,
         "hide": 0,
         "includeAll": false,
         "label": "Instance",
@@ -659,16 +2369,16 @@
       },
       {
         "allFormat": "glob",
-        "allValue": null,
         "current": {
           "selected": false,
           "text": "All",
           "value": "$__all"
         },
-        "datasource": "Prometheus",
+        "datasource": {
+          "type": "prometheus",
+          "uid": "PBFA97CFB590B2093"
+        },
         "definition": "",
-        "description": null,
-        "error": null,
         "hide": 0,
         "includeAll": true,
         "label": "JVM Memory Pools Heap",
@@ -691,16 +2401,16 @@
       },
       {
         "allFormat": "glob",
-        "allValue": null,
         "current": {
           "selected": false,
           "text": "All",
           "value": "$__all"
         },
-        "datasource": "Prometheus",
+        "datasource": {
+          "type": "prometheus",
+          "uid": "PBFA97CFB590B2093"
+        },
         "definition": "",
-        "description": null,
-        "error": null,
         "hide": 0,
         "includeAll": true,
         "label": "JVM Memory Pools Non-Heap",
@@ -724,7 +2434,7 @@
     ]
   },
   "time": {
-    "from": "now-30m",
+    "from": "now-5m",
     "to": "now"
   },
   "timepicker": {
@@ -756,5 +2466,6 @@
   "timezone": "browser",
   "title": "Master",
   "uid": "6XgATOcnz",
-  "version": 20
-}
+  "version": 2,
+  "weekStart": ""
+}
\ No newline at end of file
diff --git a/dolphinscheduler-meter/src/main/resources/grafana/DolphinSchedulerWorker.json b/dolphinscheduler-meter/src/main/resources/grafana/DolphinSchedulerWorker.json
new file mode 100644
index 0000000000..765f40bf8f
--- /dev/null
+++ b/dolphinscheduler-meter/src/main/resources/grafana/DolphinSchedulerWorker.json
@@ -0,0 +1,1598 @@
+{
+  "annotations": {
+    "list": [
+      {
+        "builtIn": 1,
+        "datasource": {
+          "type": "grafana",
+          "uid": "-- Grafana --"
+        },
+        "enable": true,
+        "hide": true,
+        "iconColor": "rgba(0, 211, 255, 1)",
+        "name": "Annotations & Alerts",
+        "target": {
+          "limit": 100,
+          "matchAny": false,
+          "tags": [],
+          "type": "dashboard"
+        },
+        "type": "dashboard"
+      }
+    ]
+  },
+  "editable": true,
+  "fiscalYearStartMonth": 0,
+  "graphTooltip": 0,
+  "links": [],
+  "liveNow": false,
+  "panels": [
+    {
+      "collapsed": false,
+      "gridPos": {
+        "h": 1,
+        "w": 24,
+        "x": 0,
+        "y": 0
+      },
+      "id": 24,
+      "panels": [],
+      "title": "Worker",
+      "type": "row"
+    },
+    {
+      "datasource": {
+        "type": "prometheus",
+        "uid": "PBFA97CFB590B2093"
+      },
+      "fieldConfig": {
+        "defaults": {
+          "color": {
+            "mode": "palette-classic"
+          },
+          "custom": {
+            "axisLabel": "",
+            "axisPlacement": "auto",
+            "barAlignment": 0,
+            "drawStyle": "line",
+            "fillOpacity": 0,
+            "gradientMode": "none",
+            "hideFrom": {
+              "legend": false,
+              "tooltip": false,
+              "viz": false
+            },
+            "lineInterpolation": "linear",
+            "lineWidth": 1,
+            "pointSize": 5,
+            "scaleDistribution": {
+              "type": "linear"
+            },
+            "showPoints": "auto",
+            "spanNulls": false,
+            "stacking": {
+              "group": "A",
+              "mode": "none"
+            },
+            "thresholdsStyle": {
+              "mode": "off"
+            }
+          },
+          "mappings": [],
+          "thresholds": {
+            "mode": "absolute",
+            "steps": [
+              {
+                "color": "green",
+                "value": null
+              },
+              {
+                "color": "red",
+                "value": 80
+              }
+            ]
+          },
+          "unit": "percentunit"
+        },
+        "overrides": []
+      },
+      "gridPos": {
+        "h": 8,
+        "w": 12,
+        "x": 0,
+        "y": 1
+      },
+      "id": 28,
+      "options": {
+        "legend": {
+          "calcs": [],
+          "displayMode": "list",
+          "placement": "bottom"
+        },
+        "tooltip": {
+          "mode": "single",
+          "sort": "none"
+        }
+      },
+      "targets": [
+        {
+          "datasource": {
+            "type": "prometheus",
+            "uid": "PBFA97CFB590B2093"
+          },
+          "editorMode": "code",
+          "exemplar": true,
+          "expr": "process_cpu_usage{application=\"worker-server\"}",
+          "range": true,
+          "refId": "A"
+        }
+      ],
+      "title": "Worker CPU Usage",
+      "type": "timeseries"
+    },
+    {
+      "datasource": {
+        "type": "prometheus",
+        "uid": "PBFA97CFB590B2093"
+      },
+      "fieldConfig": {
+        "defaults": {
+          "color": {
+            "mode": "thresholds"
+          },
+          "mappings": [],
+          "thresholds": {
+            "mode": "absolute",
+            "steps": [
+              {
+                "color": "green",
+                "value": null
+              }
+            ]
+          }
+        },
+        "overrides": []
+      },
+      "gridPos": {
+        "h": 8,
+        "w": 12,
+        "x": 12,
+        "y": 1
+      },
+      "id": 30,
+      "options": {
+        "orientation": "auto",
+        "reduceOptions": {
+          "calcs": [
+            "lastNotNull"
+          ],
+          "fields": "/^Time$/",
+          "values": false
+        },
+        "showThresholdLabels": false,
+        "showThresholdMarkers": false
+      },
+      "pluginVersion": "8.5.3",
+      "targets": [
+        {
+          "datasource": {
+            "type": "prometheus",
+            "uid": "PBFA97CFB590B2093"
+          },
+          "expr": "process_start_time_seconds{}",
+          "refId": "A"
+        }
+      ],
+      "title": "Worker Start Time",
+      "type": "gauge"
+    },
+    {
+      "datasource": {
+        "type": "prometheus",
+        "uid": "PBFA97CFB590B2093"
+      },
+      "fieldConfig": {
+        "defaults": {
+          "color": {
+            "mode": "palette-classic"
+          },
+          "custom": {
+            "axisLabel": "",
+            "axisPlacement": "auto",
+            "barAlignment": 0,
+            "drawStyle": "line",
+            "fillOpacity": 0,
+            "gradientMode": "none",
+            "hideFrom": {
+              "legend": false,
+              "tooltip": false,
+              "viz": false
+            },
+            "lineInterpolation": "linear",
+            "lineWidth": 1,
+            "pointSize": 5,
+            "scaleDistribution": {
+              "type": "linear"
+            },
+            "showPoints": "auto",
+            "spanNulls": false,
+            "stacking": {
+              "group": "A",
+              "mode": "none"
+            },
+            "thresholdsStyle": {
+              "mode": "off"
+            }
+          },
+          "mappings": [],
+          "thresholds": {
+            "mode": "absolute",
+            "steps": [
+              {
+                "color": "green",
+                "value": null
+              },
+              {
+                "color": "red",
+                "value": 80
+              }
+            ]
+          }
+        },
+        "overrides": []
+      },
+      "gridPos": {
+        "h": 8,
+        "w": 8,
+        "x": 0,
+        "y": 9
+      },
+      "id": 4,
+      "options": {
+        "legend": {
+          "calcs": [],
+          "displayMode": "list",
+          "placement": "bottom"
+        },
+        "tooltip": {
+          "mode": "single",
+          "sort": "none"
+        }
+      },
+      "targets": [
+        {
+          "datasource": {
+            "type": "prometheus",
+            "uid": "PBFA97CFB590B2093"
+          },
+          "expr": "increase(dolphinscheduler_worker_submit_queue_is_full_count_total{}[1m])",
+          "refId": "A"
+        }
+      ],
+      "title": "Worker Submit Queue Full/1m",
+      "type": "timeseries"
+    },
+    {
+      "datasource": {
+        "type": "prometheus",
+        "uid": "PBFA97CFB590B2093"
+      },
+      "fieldConfig": {
+        "defaults": {
+          "color": {
+            "mode": "palette-classic"
+          },
+          "custom": {
+            "axisLabel": "",
+            "axisPlacement": "auto",
+            "barAlignment": 0,
+            "drawStyle": "line",
+            "fillOpacity": 0,
+            "gradientMode": "none",
+            "hideFrom": {
+              "legend": false,
+              "tooltip": false,
+              "viz": false
+            },
+            "lineInterpolation": "linear",
+            "lineWidth": 1,
+            "pointSize": 5,
+            "scaleDistribution": {
+              "type": "linear"
+            },
+            "showPoints": "auto",
+            "spanNulls": false,
+            "stacking": {
+              "group": "A",
+              "mode": "none"
+            },
+            "thresholdsStyle": {
+              "mode": "off"
+            }
+          },
+          "mappings": [],
+          "thresholds": {
+            "mode": "absolute",
+            "steps": [
+              {
+                "color": "green",
+                "value": null
+              },
+              {
+                "color": "red",
+                "value": 80
+              }
+            ]
+          }
+        },
+        "overrides": []
+      },
+      "gridPos": {
+        "h": 8,
+        "w": 9,
+        "x": 8,
+        "y": 9
+      },
+      "id": 2,
+      "options": {
+        "legend": {
+          "calcs": [],
+          "displayMode": "list",
+          "placement": "bottom"
+        },
+        "tooltip": {
+          "mode": "single",
+          "sort": "none"
+        }
+      },
+      "targets": [
+        {
+          "datasource": {
+            "type": "prometheus",
+            "uid": "PBFA97CFB590B2093"
+          },
+          "expr": "increase(dolphinscheduler_worker_overload_count_total{}[1m])",
+          "refId": "A"
+        }
+      ],
+      "title": "Worker Overload Increase/1m",
+      "type": "timeseries"
+    },
+    {
+      "datasource": {
+        "type": "prometheus",
+        "uid": "PBFA97CFB590B2093"
+      },
+      "fieldConfig": {
+        "defaults": {
+          "color": {
+            "mode": "palette-classic"
+          },
+          "custom": {
+            "axisLabel": "",
+            "axisPlacement": "auto",
+            "barAlignment": 0,
+            "drawStyle": "line",
+            "fillOpacity": 0,
+            "gradientMode": "none",
+            "hideFrom": {
+              "legend": false,
+              "tooltip": false,
+              "viz": false
+            },
+            "lineInterpolation": "linear",
+            "lineWidth": 1,
+            "pointSize": 5,
+            "scaleDistribution": {
+              "type": "linear"
+            },
+            "showPoints": "auto",
+            "spanNulls": false,
+            "stacking": {
+              "group": "A",
+              "mode": "none"
+            },
+            "thresholdsStyle": {
+              "mode": "off"
+            }
+          },
+          "mappings": [],
+          "thresholds": {
+            "mode": "absolute",
+            "steps": [
+              {
+                "color": "green",
+                "value": null
+              },
+              {
+                "color": "red",
+                "value": 80
+              }
+            ]
+          }
+        },
+        "overrides": []
+      },
+      "gridPos": {
+        "h": 8,
+        "w": 7,
+        "x": 17,
+        "y": 9
+      },
+      "id": 6,
+      "options": {
+        "legend": {
+          "calcs": [],
+          "displayMode": "list",
+          "placement": "bottom"
+        },
+        "tooltip": {
+          "mode": "single",
+          "sort": "none"
+        }
+      },
+      "targets": [
+        {
+          "datasource": {
+            "type": "prometheus",
+            "uid": "PBFA97CFB590B2093"
+          },
+          "expr": "dolphinscheduler_worker_running_task_gauge{}",
+          "refId": "A"
+        }
+      ],
+      "title": "Worker Running Task",
+      "type": "timeseries"
+    },
+    {
+      "datasource": {
+        "type": "prometheus",
+        "uid": "PBFA97CFB590B2093"
+      },
+      "fieldConfig": {
+        "defaults": {
+          "color": {
+            "mode": "thresholds"
+          },
+          "mappings": [],
+          "thresholds": {
+            "mode": "absolute",
+            "steps": [
+              {
+                "color": "green",
+                "value": null
+              }
+            ]
+          }
+        },
+        "overrides": []
+      },
+      "gridPos": {
+        "h": 8,
+        "w": 12,
+        "x": 0,
+        "y": 17
+      },
+      "id": 10,
+      "options": {
+        "colorMode": "value",
+        "graphMode": "area",
+        "justifyMode": "auto",
+        "orientation": "auto",
+        "reduceOptions": {
+          "calcs": [
+            "lastNotNull"
+          ],
+          "fields": "",
+          "values": false
+        },
+        "textMode": "auto"
+      },
+      "pluginVersion": "8.5.3",
+      "targets": [
+        {
+          "datasource": {
+            "type": "prometheus",
+            "uid": "PBFA97CFB590B2093"
+          },
+          "expr": "sum(dolphinscheduler_task_execution_count_total)",
+          "refId": "A"
+        }
+      ],
+      "title": "Task Total Count",
+      "type": "stat"
+    },
+    {
+      "datasource": {
+        "type": "prometheus",
+        "uid": "PBFA97CFB590B2093"
+      },
+      "fieldConfig": {
+        "defaults": {
+          "color": {
+            "mode": "thresholds"
+          },
+          "mappings": [],
+          "thresholds": {
+            "mode": "absolute",
+            "steps": [
+              {
+                "color": "green",
+                "value": null
+              }
+            ]
+          }
+        },
+        "overrides": []
+      },
+      "gridPos": {
+        "h": 8,
+        "w": 12,
+        "x": 12,
+        "y": 17
+      },
+      "id": 12,
+      "options": {
+        "orientation": "auto",
+        "reduceOptions": {
+          "calcs": [
+            "lastNotNull"
+          ],
+          "fields": "",
+          "values": false
+        },
+        "showThresholdLabels": false,
+        "showThresholdMarkers": true
+      },
+      "pluginVersion": "8.5.3",
+      "targets": [
+        {
+          "datasource": {
+            "type": "prometheus",
+            "uid": "PBFA97CFB590B2093"
+          },
+          "expr": "sum(dolphinscheduler_task_execution_count_total{result=\"success\"}) / sum(dolphinscheduler_task_execution_count_total) * 100",
+          "refId": "A"
+        }
+      ],
+      "title": "Task Successful Rate",
+      "type": "gauge"
+    },
+    {
+      "aliasColors": {},
+      "bars": false,
+      "dashLength": 10,
+      "dashes": false,
+      "datasource": {
+        "type": "prometheus",
+        "uid": "PBFA97CFB590B2093"
+      },
+      "fieldConfig": {
+        "defaults": {
+          "unit": "millse"
+        },
+        "overrides": []
+      },
+      "fill": 1,
+      "fillGradient": 0,
+      "gridPos": {
+        "h": 8,
+        "w": 12,
+        "x": 0,
+        "y": 25
+      },
+      "hiddenSeries": false,
+      "id": 16,
+      "legend": {
+        "avg": false,
+        "current": false,
+        "max": false,
+        "min": false,
+        "show": true,
+        "total": false,
+        "values": false
+      },
+      "lines": true,
+      "linewidth": 1,
+      "nullPointMode": "null",
+      "options": {
+        "alertThreshold": true
+      },
+      "percentage": false,
+      "pluginVersion": "8.5.3",
+      "pointradius": 2,
+      "points": false,
+      "renderer": "flot",
+      "seriesOverrides": [],
+      "spaceLength": 10,
+      "stack": false,
+      "steppedLine": false,
+      "targets": [
+        {
+          "datasource": {
+            "type": "prometheus",
+            "uid": "PBFA97CFB590B2093"
+          },
+          "editorMode": "code",
+          "expr": "rate(dolphinscheduler_task_execution_timer_seconds_sum[1m])/rate(dolphinscheduler_task_execution_timer_seconds_count[1m])",
+          "legendFormat": "avg ({{exception}})",
+          "range": true,
+          "refId": "A"
+        },
+        {
+          "datasource": {
+            "type": "prometheus",
+            "uid": "PBFA97CFB590B2093"
+          },
+          "editorMode": "code",
+          "expr": "dolphinscheduler_task_execution_timer_seconds_max",
+          "hide": false,
+          "legendFormat": "max ({{exception}})",
+          "range": true,
+          "refId": "B"
+        }
+      ],
+      "thresholds": [],
+      "timeRegions": [],
+      "title": "Task Execution Time",
+      "tooltip": {
+        "shared": true,
+        "sort": 0,
+        "value_type": "individual"
+      },
+      "type": "graph",
+      "xaxis": {
+        "mode": "time",
+        "show": true,
+        "values": []
+      },
+      "yaxes": [
+        {
+          "$$hashKey": "object:236",
+          "format": "millse",
+          "logBase": 1,
+          "show": true
+        },
+        {
+          "$$hashKey": "object:237",
+          "format": "short",
+          "logBase": 1,
+          "show": true
+        }
+      ],
+      "yaxis": {
+        "align": false
+      }
+    },
+    {
+      "aliasColors": {},
+      "bars": false,
+      "dashLength": 10,
+      "dashes": false,
+      "datasource": {
+        "type": "prometheus",
+        "uid": "PBFA97CFB590B2093"
+      },
+      "fill": 1,
+      "fillGradient": 0,
+      "gridPos": {
+        "h": 8,
+        "w": 12,
+        "x": 12,
+        "y": 25
+      },
+      "hiddenSeries": false,
+      "id": 18,
+      "legend": {
+        "avg": false,
+        "current": false,
+        "max": false,
+        "min": false,
+        "show": true,
+        "total": false,
+        "values": false
+      },
+      "lines": true,
+      "linewidth": 1,
+      "nullPointMode": "null",
+      "options": {
+        "alertThreshold": true
+      },
+      "percentage": false,
+      "pluginVersion": "8.5.3",
+      "pointradius": 2,
+      "points": false,
+      "renderer": "flot",
+      "seriesOverrides": [],
+      "spaceLength": 10,
+      "stack": false,
+      "steppedLine": false,
+      "targets": [
+        {
+          "datasource": {
+            "type": "prometheus",
+            "uid": "PBFA97CFB590B2093"
+          },
+          "editorMode": "code",
+          "exemplar": true,
+          "expr": "sum(increase(dolphinscheduler_task_execution_count_total[1m]))",
+          "legendFormat": "Total",
+          "range": true,
+          "refId": "A"
+        },
+        {
+          "datasource": {
+            "type": "prometheus",
+            "uid": "PBFA97CFB590B2093"
+          },
+          "editorMode": "code",
+          "exemplar": true,
+          "expr": "increase(dolphinscheduler_task_execution_count_total{result=\"success\"}[1m])",
+          "hide": false,
+          "legendFormat": "Successful",
+          "range": true,
+          "refId": "B"
+        }
+      ],
+      "thresholds": [],
+      "timeRegions": [],
+      "title": "Task Execution Total/Success",
+      "tooltip": {
+        "shared": true,
+        "sort": 0,
+        "value_type": "individual"
+      },
+      "type": "graph",
+      "xaxis": {
+        "mode": "time",
+        "show": true,
+        "values": []
+      },
+      "yaxes": [
+        {
+          "format": "short",
+          "logBase": 1,
+          "show": true
+        },
+        {
+          "format": "short",
+          "logBase": 1,
+          "show": true
+        }
+      ],
+      "yaxis": {
+        "align": false
+      }
+    },
+    {
+      "collapsed": false,
+      "gridPos": {
+        "h": 1,
+        "w": 24,
+        "x": 0,
+        "y": 33
+      },
+      "id": 26,
+      "panels": [],
+      "title": "Task",
+      "type": "row"
+    },
+    {
+      "datasource": {
+        "type": "prometheus",
+        "uid": "PBFA97CFB590B2093"
+      },
+      "fieldConfig": {
+        "defaults": {
+          "color": {
+            "mode": "palette-classic"
+          },
+          "custom": {
+            "axisLabel": "",
+            "axisPlacement": "auto",
+            "barAlignment": 0,
+            "drawStyle": "line",
+            "fillOpacity": 0,
+            "gradientMode": "none",
+            "hideFrom": {
+              "legend": false,
+              "tooltip": false,
+              "viz": false
+            },
+            "lineInterpolation": "linear",
+            "lineWidth": 1,
+            "pointSize": 5,
+            "scaleDistribution": {
+              "type": "linear"
+            },
+            "showPoints": "auto",
+            "spanNulls": false,
+            "stacking": {
+              "group": "A",
+              "mode": "none"
+            },
+            "thresholdsStyle": {
+              "mode": "off"
+            }
+          },
+          "mappings": [],
+          "thresholds": {
+            "mode": "absolute",
+            "steps": [
+              {
+                "color": "green",
+                "value": null
+              },
+              {
+                "color": "red",
+                "value": 80
+              }
+            ]
+          }
+        },
+        "overrides": []
+      },
+      "gridPos": {
+        "h": 8,
+        "w": 8,
+        "x": 0,
+        "y": 34
+      },
+      "id": 14,
+      "options": {
+        "legend": {
+          "calcs": [],
+          "displayMode": "list",
+          "placement": "bottom"
+        },
+        "tooltip": {
+          "mode": "single",
+          "sort": "none"
+        }
+      },
+      "targets": [
+        {
+          "datasource": {
+            "type": "prometheus",
+            "uid": "PBFA97CFB590B2093"
+          },
+          "editorMode": "code",
+          "expr": "sum(increase(dolphinscheduler_task_execute_count_total{task_type=\"SHELL\"}[1d]))",
+          "legendFormat": "SHELL",
+          "range": true,
+          "refId": "A"
+        }
+      ],
+      "title": "Shell Task Execute/1d",
+      "type": "timeseries"
+    },
+    {
+      "datasource": {
+        "type": "prometheus",
+        "uid": "PBFA97CFB590B2093"
+      },
+      "fieldConfig": {
+        "defaults": {
+          "color": {
+            "mode": "palette-classic"
+          },
+          "custom": {
+            "axisLabel": "",
+            "axisPlacement": "auto",
+            "barAlignment": 0,
+            "drawStyle": "line",
+            "fillOpacity": 0,
+            "gradientMode": "none",
+            "hideFrom": {
+              "legend": false,
+              "tooltip": false,
+              "viz": false
+            },
+            "lineInterpolation": "linear",
+            "lineWidth": 1,
+            "pointSize": 5,
+            "scaleDistribution": {
+              "type": "linear"
+            },
+            "showPoints": "auto",
+            "spanNulls": false,
+            "stacking": {
+              "group": "A",
+              "mode": "none"
+            },
+            "thresholdsStyle": {
+              "mode": "off"
+            }
+          },
+          "mappings": [],
+          "thresholds": {
+            "mode": "absolute",
+            "steps": [
+              {
+                "color": "green",
+                "value": null
+              },
+              {
+                "color": "red",
+                "value": 80
+              }
+            ]
+          }
+        },
+        "overrides": []
+      },
+      "gridPos": {
+        "h": 8,
+        "w": 9,
+        "x": 8,
+        "y": 34
+      },
+      "id": 20,
+      "options": {
+        "legend": {
+          "calcs": [],
+          "displayMode": "list",
+          "placement": "bottom"
+        },
+        "tooltip": {
+          "mode": "single",
+          "sort": "none"
+        }
+      },
+      "targets": [
+        {
+          "datasource": {
+            "type": "prometheus",
+            "uid": "PBFA97CFB590B2093"
+          },
+          "editorMode": "code",
+          "expr": "sum(increase(dolphinscheduler_task_execute_count_total{task_type=\"unknown\"}[1d]))",
+          "legendFormat": "unknown",
+          "range": true,
+          "refId": "A"
+        }
+      ],
+      "title": "Unkown Task Execution Total/1d",
+      "type": "timeseries"
+    },
+    {
+      "datasource": {
+        "type": "prometheus",
+        "uid": "PBFA97CFB590B2093"
+      },
+      "fieldConfig": {
+        "defaults": {
+          "color": {
+            "mode": "palette-classic"
+          },
+          "custom": {
+            "axisLabel": "",
+            "axisPlacement": "auto",
+            "barAlignment": 0,
+            "drawStyle": "line",
+            "fillOpacity": 0,
+            "gradientMode": "none",
+            "hideFrom": {
+              "legend": false,
+              "tooltip": false,
+              "viz": false
+            },
+            "lineInterpolation": "linear",
+            "lineWidth": 1,
+            "pointSize": 5,
+            "scaleDistribution": {
+              "type": "linear"
+            },
+            "showPoints": "auto",
+            "spanNulls": false,
+            "stacking": {
+              "group": "A",
+              "mode": "none"
+            },
+            "thresholdsStyle": {
+              "mode": "off"
+            }
+          },
+          "mappings": [],
+          "thresholds": {
+            "mode": "absolute",
+            "steps": [
+              {
+                "color": "green",
+                "value": null
+              },
+              {
+                "color": "red",
+                "value": 80
+              }
+            ]
+          }
+        },
+        "overrides": []
+      },
+      "gridPos": {
+        "h": 8,
+        "w": 7,
+        "x": 17,
+        "y": 34
+      },
+      "id": 22,
+      "options": {
+        "legend": {
+          "calcs": [],
+          "displayMode": "list",
+          "placement": "bottom"
+        },
+        "tooltip": {
+          "mode": "single",
+          "sort": "none"
+        }
+      },
+      "targets": [
+        {
+          "datasource": {
+            "type": "prometheus",
+            "uid": "PBFA97CFB590B2093"
+          },
+          "editorMode": "code",
+          "expr": "sum(increase(dolphinscheduler_task_execute_count_total{task_type=\"CONDITIONS\"}[1d]))",
+          "legendFormat": "CONDITIONS",
+          "range": true,
+          "refId": "A"
+        }
+      ],
+      "title": "Condition Task Execution Total/1d",
+      "type": "timeseries"
+    },
+    {
+      "datasource": {
+        "type": "prometheus",
+        "uid": "PBFA97CFB590B2093"
+      },
+      "fieldConfig": {
+        "defaults": {
+          "color": {
+            "mode": "palette-classic"
+          },
+          "custom": {
+            "axisLabel": "",
+            "axisPlacement": "auto",
+            "barAlignment": 0,
+            "drawStyle": "line",
+            "fillOpacity": 0,
+            "gradientMode": "none",
+            "hideFrom": {
+              "legend": false,
+              "tooltip": false,
+              "viz": false
+            },
+            "lineInterpolation": "linear",
+            "lineWidth": 1,
+            "pointSize": 5,
+            "scaleDistribution": {
+              "type": "linear"
+            },
+            "showPoints": "auto",
+            "spanNulls": false,
+            "stacking": {
+              "group": "A",
+              "mode": "none"
+            },
+            "thresholdsStyle": {
+              "mode": "off"
+            }
+          },
+          "mappings": [],
+          "thresholds": {
+            "mode": "absolute",
+            "steps": [
+              {
+                "color": "green",
+                "value": null
+              },
+              {
+                "color": "red",
+                "value": 80
+              }
+            ]
+          }
+        },
+        "overrides": []
+      },
+      "gridPos": {
+        "h": 8,
+        "w": 8,
+        "x": 0,
+        "y": 42
+      },
+      "id": 32,
+      "options": {
+        "legend": {
+          "calcs": [],
+          "displayMode": "list",
+          "placement": "bottom"
+        },
+        "tooltip": {
+          "mode": "single",
+          "sort": "none"
+        }
+      },
+      "targets": [
+        {
+          "datasource": {
+            "type": "prometheus",
+            "uid": "PBFA97CFB590B2093"
+          },
+          "editorMode": "code",
+          "expr": "sum(increase(dolphinscheduler_task_execute_count_total{task_type=\"BLOCKING\"}[1d]))",
+          "legendFormat": "BLOCKING",
+          "range": true,
+          "refId": "A"
+        }
+      ],
+      "title": "Blocking Task Execute/1d",
+      "type": "timeseries"
+    },
+    {
+      "datasource": {
+        "type": "prometheus",
+        "uid": "PBFA97CFB590B2093"
+      },
+      "fieldConfig": {
+        "defaults": {
+          "color": {
+            "mode": "palette-classic"
+          },
+          "custom": {
+            "axisLabel": "",
+            "axisPlacement": "auto",
+            "barAlignment": 0,
+            "drawStyle": "line",
+            "fillOpacity": 0,
+            "gradientMode": "none",
+            "hideFrom": {
+              "legend": false,
+              "tooltip": false,
+              "viz": false
+            },
+            "lineInterpolation": "linear",
+            "lineWidth": 1,
+            "pointSize": 5,
+            "scaleDistribution": {
+              "type": "linear"
+            },
+            "showPoints": "auto",
+            "spanNulls": false,
+            "stacking": {
+              "group": "A",
+              "mode": "none"
+            },
+            "thresholdsStyle": {
+              "mode": "off"
+            }
+          },
+          "mappings": [],
+          "thresholds": {
+            "mode": "absolute",
+            "steps": [
+              {
+                "color": "green",
+                "value": null
+              },
+              {
+                "color": "red",
+                "value": 80
+              }
+            ]
+          }
+        },
+        "overrides": []
+      },
+      "gridPos": {
+        "h": 8,
+        "w": 9,
+        "x": 8,
+        "y": 42
+      },
+      "id": 34,
+      "options": {
+        "legend": {
+          "calcs": [],
+          "displayMode": "list",
+          "placement": "bottom"
+        },
+        "tooltip": {
+          "mode": "single",
+          "sort": "none"
+        }
+      },
+      "targets": [
+        {
+          "datasource": {
+            "type": "prometheus",
+            "uid": "PBFA97CFB590B2093"
+          },
+          "editorMode": "code",
+          "expr": "sum(increase(dolphinscheduler_task_execute_count_total{task_type=\"DATAX\"}[1d]))",
+          "legendFormat": "DATAX",
+          "range": true,
+          "refId": "A"
+        }
+      ],
+      "title": "DATAX Task Execute/1d",
+      "type": "timeseries"
+    },
+    {
+      "datasource": {
+        "type": "prometheus",
+        "uid": "PBFA97CFB590B2093"
+      },
+      "fieldConfig": {
+        "defaults": {
+          "color": {
+            "mode": "palette-classic"
+          },
+          "custom": {
+            "axisLabel": "",
+            "axisPlacement": "auto",
+            "barAlignment": 0,
+            "drawStyle": "line",
+            "fillOpacity": 0,
+            "gradientMode": "none",
+            "hideFrom": {
+              "legend": false,
+              "tooltip": false,
+              "viz": false
+            },
+            "lineInterpolation": "linear",
+            "lineWidth": 1,
+            "pointSize": 5,
+            "scaleDistribution": {
+              "type": "linear"
+            },
+            "showPoints": "auto",
+            "spanNulls": false,
+            "stacking": {
+              "group": "A",
+              "mode": "none"
+            },
+            "thresholdsStyle": {
+              "mode": "off"
+            }
+          },
+          "mappings": [],
+          "thresholds": {
+            "mode": "absolute",
+            "steps": [
+              {
+                "color": "green",
+                "value": null
+              },
+              {
+                "color": "red",
+                "value": 80
+              }
+            ]
+          }
+        },
+        "overrides": []
+      },
+      "gridPos": {
+        "h": 8,
+        "w": 7,
+        "x": 17,
+        "y": 42
+      },
+      "id": 36,
+      "options": {
+        "legend": {
+          "calcs": [],
+          "displayMode": "list",
+          "placement": "bottom"
+        },
+        "tooltip": {
+          "mode": "single",
+          "sort": "none"
+        }
+      },
+      "targets": [
+        {
+          "datasource": {
+            "type": "prometheus",
+            "uid": "PBFA97CFB590B2093"
+          },
+          "editorMode": "code",
+          "expr": "sum(increase(dolphinscheduler_task_execute_count_total{task_type=\"DATA_QUALITY\"}[1d]))",
+          "legendFormat": "DATA_QUALITY",
+          "range": true,
+          "refId": "A"
+        }
+      ],
+      "title": "DATA_QUALITY Task Execute/1d",
+      "type": "timeseries"
+    },
+    {
+      "datasource": {
+        "type": "prometheus",
+        "uid": "PBFA97CFB590B2093"
+      },
+      "fieldConfig": {
+        "defaults": {
+          "color": {
+            "mode": "palette-classic"
+          },
+          "custom": {
+            "axisLabel": "",
+            "axisPlacement": "auto",
+            "barAlignment": 0,
+            "drawStyle": "line",
+            "fillOpacity": 0,
+            "gradientMode": "none",
+            "hideFrom": {
+              "legend": false,
+              "tooltip": false,
+              "viz": false
+            },
+            "lineInterpolation": "linear",
+            "lineWidth": 1,
+            "pointSize": 5,
+            "scaleDistribution": {
+              "type": "linear"
+            },
+            "showPoints": "auto",
+            "spanNulls": false,
+            "stacking": {
+              "group": "A",
+              "mode": "none"
+            },
+            "thresholdsStyle": {
+              "mode": "off"
+            }
+          },
+          "mappings": [],
+          "thresholds": {
+            "mode": "absolute",
+            "steps": [
+              {
+                "color": "green",
+                "value": null
+              },
+              {
+                "color": "red",
+                "value": 80
+              }
+            ]
+          }
+        },
+        "overrides": []
+      },
+      "gridPos": {
+        "h": 8,
+        "w": 8,
+        "x": 0,
+        "y": 50
+      },
+      "id": 38,
+      "options": {
+        "legend": {
+          "calcs": [],
+          "displayMode": "list",
+          "placement": "bottom"
+        },
+        "tooltip": {
+          "mode": "single",
+          "sort": "none"
+        }
+      },
+      "targets": [
+        {
+          "datasource": {
+            "type": "prometheus",
+            "uid": "PBFA97CFB590B2093"
+          },
+          "editorMode": "code",
+          "expr": "sum(increase(dolphinscheduler_task_execute_count_total{task_type=\"DATA_QUALITY\"}[1d]))",
+          "legendFormat": "DATA_QUALITY",
+          "range": true,
+          "refId": "A"
+        }
+      ],
+      "title": "DATA_QUALITY Task Execute/1d",
+      "type": "timeseries"
+    },
+    {
+      "datasource": {
+        "type": "prometheus",
+        "uid": "PBFA97CFB590B2093"
+      },
+      "fieldConfig": {
+        "defaults": {
+          "color": {
+            "mode": "palette-classic"
+          },
+          "custom": {
+            "axisLabel": "",
+            "axisPlacement": "auto",
+            "barAlignment": 0,
+            "drawStyle": "line",
+            "fillOpacity": 0,
+            "gradientMode": "none",
+            "hideFrom": {
+              "legend": false,
+              "tooltip": false,
+              "viz": false
+            },
+            "lineInterpolation": "linear",
+            "lineWidth": 1,
+            "pointSize": 5,
+            "scaleDistribution": {
+              "type": "linear"
+            },
+            "showPoints": "auto",
+            "spanNulls": false,
+            "stacking": {
+              "group": "A",
+              "mode": "none"
+            },
+            "thresholdsStyle": {
+              "mode": "off"
+            }
+          },
+          "mappings": [],
+          "thresholds": {
+            "mode": "absolute",
+            "steps": [
+              {
+                "color": "green",
+                "value": null
+              },
+              {
+                "color": "red",
+                "value": 80
+              }
+            ]
+          }
+        },
+        "overrides": []
+      },
+      "gridPos": {
+        "h": 8,
+        "w": 9,
+        "x": 8,
+        "y": 50
+      },
+      "id": 40,
+      "options": {
+        "legend": {
+          "calcs": [],
+          "displayMode": "list",
+          "placement": "bottom"
+        },
+        "tooltip": {
+          "mode": "single",
+          "sort": "none"
+        }
+      },
+      "targets": [
+        {
+          "datasource": {
+            "type": "prometheus",
+            "uid": "PBFA97CFB590B2093"
+          },
+          "editorMode": "code",
+          "expr": "sum(increase(dolphinscheduler_task_execute_count_total{task_type=\"DEPENDENT\"}[1d]))",
+          "legendFormat": "DEPENDENT",
+          "range": true,
+          "refId": "A"
+        }
+      ],
+      "title": "DEPENDENT Task Execute/1d",
+      "type": "timeseries"
+    },
+    {
+      "datasource": {
+        "type": "prometheus",
+        "uid": "PBFA97CFB590B2093"
+      },
+      "fieldConfig": {
+        "defaults": {
+          "color": {
+            "mode": "palette-classic"
+          },
+          "custom": {
+            "axisLabel": "",
+            "axisPlacement": "auto",
+            "barAlignment": 0,
+            "drawStyle": "line",
+            "fillOpacity": 0,
+            "gradientMode": "none",
+            "hideFrom": {
+              "legend": false,
+              "tooltip": false,
+              "viz": false
+            },
+            "lineInterpolation": "linear",
+            "lineWidth": 1,
+            "pointSize": 5,
+            "scaleDistribution": {
+              "type": "linear"
+            },
+            "showPoints": "auto",
+            "spanNulls": false,
+            "stacking": {
+              "group": "A",
+              "mode": "none"
+            },
+            "thresholdsStyle": {
+              "mode": "off"
+            }
+          },
+          "mappings": [],
+          "thresholds": {
+            "mode": "absolute",
+            "steps": [
+              {
+                "color": "green",
+                "value": null
+              },
+              {
+                "color": "red",
+                "value": 80
+              }
+            ]
+          }
+        },
+        "overrides": []
+      },
+      "gridPos": {
+        "h": 8,
+        "w": 7,
+        "x": 17,
+        "y": 50
+      },
+      "id": 42,
+      "options": {
+        "legend": {
+          "calcs": [],
+          "displayMode": "list",
+          "placement": "bottom"
+        },
+        "tooltip": {
+          "mode": "single",
+          "sort": "none"
+        }
+      },
+      "targets": [
+        {
+          "datasource": {
+            "type": "prometheus",
+            "uid": "PBFA97CFB590B2093"
+          },
+          "editorMode": "code",
+          "expr": "sum(increase(dolphinscheduler_task_execute_count_total{task_type=\"EMR\"}[1d]))",
+          "legendFormat": "EMR",
+          "range": true,
+          "refId": "A"
+        }
+      ],
+      "title": "EMR Task Execute/1d",
+      "type": "timeseries"
+    }
+  ],
+  "schemaVersion": 36,
+  "style": "dark",
+  "tags": [],
+  "templating": {
+    "list": []
+  },
+  "time": {
+    "from": "now-30m",
+    "to": "now"
+  },
+  "timepicker": {},
+  "timezone": "",
+  "title": "Worker",
+  "uid": "6wXtd3r7k",
+  "version": 2,
+  "weekStart": ""
+}
\ No newline at end of file
diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/processor/StateEventCallbackService.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/processor/StateEventCallbackService.java
index 82ae175e29..af51831068 100644
--- a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/processor/StateEventCallbackService.java
+++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/processor/StateEventCallbackService.java
@@ -86,7 +86,7 @@ public class StateEventCallbackService {
         return null;
     }
 
-    public int pause(int ntries) {
+    public long pause(int ntries) {
         return SLEEP_TIME_MILLIS * RETRY_BACKOFF[ntries % RETRY_BACKOFF.length];
     }
 
diff --git a/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/process/ProcessServiceImpl.java b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/process/ProcessServiceImpl.java
index f44f9d5b82..c91a0fcdbb 100644
--- a/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/process/ProcessServiceImpl.java
+++ b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/process/ProcessServiceImpl.java
@@ -17,6 +17,7 @@
 
 package org.apache.dolphinscheduler.service.process;
 
+import io.micrometer.core.annotation.Counted;
 import static java.util.stream.Collectors.toSet;
 import static org.apache.dolphinscheduler.common.Constants.CMDPARAM_COMPLEMENT_DATA_END_DATE;
 import static org.apache.dolphinscheduler.common.Constants.CMDPARAM_COMPLEMENT_DATA_START_DATE;
@@ -406,6 +407,7 @@ public class ProcessServiceImpl implements ProcessService {
      * @return create result
      */
     @Override
+    @Counted("dolphinscheduler_create_command_count")
     public int createCommand(Command command) {
         int result = 0;
         if (command != null) {
diff --git a/dolphinscheduler-standalone-server/src/main/resources/banner.txt b/dolphinscheduler-standalone-server/src/main/resources/banner.txt
index 0b1f0023a8..4040c5c00a 100644
--- a/dolphinscheduler-standalone-server/src/main/resources/banner.txt
+++ b/dolphinscheduler-standalone-server/src/main/resources/banner.txt
@@ -8,5 +8,5 @@ ${AnsiColor.BLUE}${AnsiStyle.BOLD}
               |_|
 ================================================================================
 ${AnsiColor.BLUE}${AnsiStyle.BOLD}
-::  Dolphinscheduler standalone server ::  ${application.formatted-version}
+::  DolphinScheduler standalone server ::  ${application.formatted-version}
 ${AnsiStyle.NORMAL}
diff --git a/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/metrics/TaskMetrics.java b/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/metrics/TaskMetrics.java
new file mode 100644
index 0000000000..c45275a8a3
--- /dev/null
+++ b/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/metrics/TaskMetrics.java
@@ -0,0 +1,58 @@
+/*
+ * 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.worker.metrics;
+
+import org.apache.dolphinscheduler.plugin.task.api.TaskChannelFactory;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.ServiceLoader;
+
+import io.micrometer.core.instrument.Counter;
+import io.micrometer.core.instrument.Metrics;
+
+public final class TaskMetrics {
+
+    private TaskMetrics() {
+        throw new UnsupportedOperationException("Utility class");
+    }
+
+    private static Map<String, Counter> TASK_TYPE_EXECUTE_COUNTER = new HashMap<>();
+    private static final Counter UNKNOWN_TASK_EXECUTE_COUNTER =
+            Counter.builder("dolphinscheduler_task_execute_count")
+                    .tag("task_type", "unknown")
+                    .description("task execute counter")
+                    .register(Metrics.globalRegistry);
+
+    static {
+        for (TaskChannelFactory taskChannelFactory : ServiceLoader.load(TaskChannelFactory.class)) {
+            TASK_TYPE_EXECUTE_COUNTER.put(
+                    taskChannelFactory.getName(),
+                    Counter.builder("dolphinscheduler_task_execute_count")
+                            .tag("task_type", taskChannelFactory.getName())
+                            .description("task execute counter")
+                            .register(Metrics.globalRegistry)
+            );
+        }
+    }
+
+    public static void incrTaskTypeExecuteCount(String taskType) {
+        TASK_TYPE_EXECUTE_COUNTER.getOrDefault(taskType, UNKNOWN_TASK_EXECUTE_COUNTER).increment();
+    }
+
+}
diff --git a/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/metrics/WorkerServerMetrics.java b/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/metrics/WorkerServerMetrics.java
new file mode 100644
index 0000000000..9613211f43
--- /dev/null
+++ b/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/metrics/WorkerServerMetrics.java
@@ -0,0 +1,56 @@
+/*
+ * 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.worker.metrics;
+
+import java.util.function.Supplier;
+
+import io.micrometer.core.instrument.Counter;
+import io.micrometer.core.instrument.Gauge;
+import io.micrometer.core.instrument.Metrics;
+
+public final class WorkerServerMetrics {
+
+    public WorkerServerMetrics() {
+        throw new UnsupportedOperationException("Utility class");
+    }
+
+    private static final Counter WORKER_OVERLOAD_COUNTER =
+            Counter.builder("dolphinscheduler_worker_overload_count")
+                    .description("worker load count")
+                    .register(Metrics.globalRegistry);
+
+    private static final Counter WORKER_SUBMIT_QUEUE_IS_FULL_COUNTER =
+            Counter.builder("dolphinscheduler_worker_submit_queue_is_full_count")
+                    .description("worker task submit queue is full count")
+                    .register(Metrics.globalRegistry);
+
+    public static void incWorkerOverloadCount() {
+        WORKER_OVERLOAD_COUNTER.increment();
+    }
+
+    public static void incWorkerSubmitQueueIsFullCount() {
+        WORKER_SUBMIT_QUEUE_IS_FULL_COUNTER.increment();
+    }
+
+    public static void registerWorkerRunningTaskGauge(Supplier<Number> supplier) {
+        Gauge.builder("dolphinscheduler_worker_running_task_gauge", supplier)
+                .description("worker running task gauge")
+                .register(Metrics.globalRegistry);
+
+    }
+}
diff --git a/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/processor/TaskCallbackService.java b/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/processor/TaskCallbackService.java
index 1c3d21501a..448f62c8c2 100644
--- a/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/processor/TaskCallbackService.java
+++ b/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/processor/TaskCallbackService.java
@@ -119,7 +119,7 @@ public class TaskCallbackService {
         return null;
     }
 
-    public int pause(int ntries) {
+    public long pause(int ntries) {
         return SLEEP_TIME_MILLIS * RETRY_BACKOFF[ntries % RETRY_BACKOFF.length];
     }
 
diff --git a/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/processor/TaskExecuteProcessor.java b/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/processor/TaskExecuteProcessor.java
index c7df9e7876..6de69c30c6 100644
--- a/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/processor/TaskExecuteProcessor.java
+++ b/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/processor/TaskExecuteProcessor.java
@@ -17,7 +17,6 @@
 
 package org.apache.dolphinscheduler.server.worker.processor;
 
-import org.apache.commons.lang.SystemUtils;
 import org.apache.dolphinscheduler.common.Constants;
 import org.apache.dolphinscheduler.common.utils.CommonUtils;
 import org.apache.dolphinscheduler.common.utils.DateUtils;
@@ -35,11 +34,14 @@ import org.apache.dolphinscheduler.remote.processor.NettyRemoteChannel;
 import org.apache.dolphinscheduler.remote.processor.NettyRequestProcessor;
 import org.apache.dolphinscheduler.server.utils.LogUtils;
 import org.apache.dolphinscheduler.server.worker.config.WorkerConfig;
+import org.apache.dolphinscheduler.server.worker.metrics.TaskMetrics;
 import org.apache.dolphinscheduler.server.worker.runner.TaskExecuteThread;
 import org.apache.dolphinscheduler.server.worker.runner.WorkerManagerThread;
 import org.apache.dolphinscheduler.service.alert.AlertClientService;
 import org.apache.dolphinscheduler.service.task.TaskPluginManager;
 
+import org.apache.commons.lang.SystemUtils;
+
 import java.util.Date;
 
 import org.slf4j.Logger;
@@ -49,6 +51,8 @@ import org.springframework.stereotype.Component;
 
 import com.google.common.base.Preconditions;
 
+import io.micrometer.core.annotation.Counted;
+import io.micrometer.core.annotation.Timed;
 import io.netty.channel.Channel;
 
 /**
@@ -86,6 +90,8 @@ public class TaskExecuteProcessor implements NettyRequestProcessor {
     @Autowired
     private WorkerManagerThread workerManager;
 
+    @Counted(value = "dolphinscheduler_task_execution_count", description = "task execute total count")
+    @Timed(value = "dolphinscheduler_task_execution_timer", percentiles = {0.5, 0.75, 0.95, 0.99}, histogram = true)
     @Override
     public void process(Channel channel, Command command) {
         Preconditions.checkArgument(CommandType.TASK_EXECUTE_REQUEST == command.getType(),
@@ -107,6 +113,7 @@ public class TaskExecuteProcessor implements NettyRequestProcessor {
             logger.error("task execution context is null");
             return;
         }
+        TaskMetrics.incrTaskTypeExecuteCount(taskExecutionContext.getTaskType());
 
         // set cache, it will be used when kill task
         TaskExecutionContextCacheManager.cacheTaskExecutionContext(taskExecutionContext);
diff --git a/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/runner/WorkerExecService.java b/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/runner/WorkerExecService.java
index ff050370cc..67abf523ed 100644
--- a/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/runner/WorkerExecService.java
+++ b/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/runner/WorkerExecService.java
@@ -17,6 +17,8 @@
 
 package org.apache.dolphinscheduler.server.worker.runner;
 
+import org.apache.dolphinscheduler.server.worker.metrics.WorkerServerMetrics;
+
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.ThreadPoolExecutor;
@@ -52,6 +54,7 @@ public class WorkerExecService {
         this.execService = execService;
         this.listeningExecutorService = MoreExecutors.listeningDecorator(this.execService);
         this.taskExecuteThreadMap = taskExecuteThreadMap;
+        WorkerServerMetrics.registerWorkerRunningTaskGauge(taskExecuteThreadMap::size);
     }
 
     public void submit(TaskExecuteThread taskExecuteThread) {
diff --git a/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/runner/WorkerManagerThread.java b/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/runner/WorkerManagerThread.java
index d9531b58bf..ddef658ec2 100644
--- a/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/runner/WorkerManagerThread.java
+++ b/dolphinscheduler-worker/src/main/java/org/apache/dolphinscheduler/server/worker/runner/WorkerManagerThread.java
@@ -25,6 +25,7 @@ import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
 import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContextCacheManager;
 import org.apache.dolphinscheduler.plugin.task.api.enums.ExecutionStatus;
 import org.apache.dolphinscheduler.server.worker.config.WorkerConfig;
+import org.apache.dolphinscheduler.server.worker.metrics.WorkerServerMetrics;
 import org.apache.dolphinscheduler.server.worker.processor.TaskCallbackService;
 
 import java.util.concurrent.BlockingQueue;
@@ -132,6 +133,7 @@ public class WorkerManagerThread implements Runnable {
      */
     public boolean offer(TaskExecuteThread taskExecuteThread) {
         if (waitSubmitQueue.size() > workerExecThreads) {
+            WorkerServerMetrics.incWorkerSubmitQueueIsFullCount();
             // if waitSubmitQueue is full, it will wait 1s, then try add
             ThreadUtils.sleep(Constants.SLEEP_TIME_MILLIS);
             if (waitSubmitQueue.size() > workerExecThreads) {
@@ -158,8 +160,9 @@ public class WorkerManagerThread implements Runnable {
                     taskExecuteThread.setStorageOperate(storageOperate);
                     workerExecService.submit(taskExecuteThread);
                 } else {
+                    WorkerServerMetrics.incWorkerOverloadCount();
                     logger.info("Exec queue is full, waiting submit queue {}, waiting exec queue size {}",
-                        this.getWaitSubmitQueueSize(), this.getThreadPoolQueueSize());
+                            this.getWaitSubmitQueueSize(), this.getThreadPoolQueueSize());
                     ThreadUtils.sleep(Constants.SLEEP_TIME_MILLIS);
                 }
             } catch (Exception e) {
diff --git a/dolphinscheduler-worker/src/main/resources/application.yaml b/dolphinscheduler-worker/src/main/resources/application.yaml
index 80e650c7ed..93082fd661 100644
--- a/dolphinscheduler-worker/src/main/resources/application.yaml
+++ b/dolphinscheduler-worker/src/main/resources/application.yaml
@@ -101,3 +101,6 @@ spring:
       on-profile: mysql
   datasource:
     driver-class-name: com.mysql.cj.jdbc.Driver
+    url: jdbc:mysql://127.0.0.1:3306/dolphinscheduler
+    username: root
+    password: root
diff --git a/dolphinscheduler-worker/src/main/resources/banner.txt b/dolphinscheduler-worker/src/main/resources/banner.txt
index 7f93ee991b..bf195ee524 100644
--- a/dolphinscheduler-worker/src/main/resources/banner.txt
+++ b/dolphinscheduler-worker/src/main/resources/banner.txt
@@ -8,5 +8,5 @@ ${AnsiColor.BLUE}${AnsiStyle.BOLD}
               |_|
 ================================================================================
 ${AnsiColor.BLUE}${AnsiStyle.BOLD}
-::  Dolphinscheduler work server ::  ${application.formatted-version}
+::  DolphinScheduler work server ::  ${application.formatted-version}
 ${AnsiStyle.NORMAL}