You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@dolphinscheduler.apache.org by we...@apache.org on 2022/06/21 06:27:12 UTC

[dolphinscheduler] branch dev updated: [Improvement][Metrics] Apply micrometer naming convention to metrics (#10477)

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

wenjun 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 cc06eaaf54 [Improvement][Metrics] Apply micrometer naming convention to metrics (#10477)
cc06eaaf54 is described below

commit cc06eaaf54de04a1d763e7a355467fa80e8a3052
Author: Eric Gao <er...@gmail.com>
AuthorDate: Tue Jun 21 14:27:06 2022 +0800

    [Improvement][Metrics] Apply micrometer naming convention to metrics (#10477)
    
    * Apply micrometer naming convention to worker metrics
    * Apply micrometer naming convention all current metrics
    * Fix remaining metrics names, update English docs and add Chinese docs
    * Fix metrics names in grafana-demo dashboards
---
 docs/docs/en/guide/metrics/metrics.md              | 292 +++++++++++----------
 docs/docs/zh/guide/metrics/metrics.md              | 169 ++++++++++++
 .../server/master/metrics/MasterServerMetrics.java |   4 +-
 .../master/metrics/ProcessInstanceMetrics.java     |  16 +-
 .../server/master/metrics/TaskMetrics.java         |  28 +-
 .../master/runner/WorkflowExecuteRunnable.java     |   2 +-
 .../server/master/service/FailoverService.java     |   4 +-
 .../resources/grafana/DolphinSchedulerMaster.json  |  22 +-
 .../resources/grafana/DolphinSchedulerWorker.json  |  36 +--
 .../scheduler/quartz/ProcessScheduleTask.java      |   4 +-
 .../service/process/ProcessServiceImpl.java        |   2 +-
 .../server/worker/metrics/TaskMetrics.java         |   8 +-
 .../server/worker/metrics/WorkerServerMetrics.java |  12 +-
 .../worker/processor/TaskExecuteProcessor.java     |   4 +-
 14 files changed, 393 insertions(+), 210 deletions(-)

diff --git a/docs/docs/en/guide/metrics/metrics.md b/docs/docs/en/guide/metrics/metrics.md
index 0f75db0ef4..d4c6d2a239 100644
--- a/docs/docs/en/guide/metrics/metrics.md
+++ b/docs/docs/en/guide/metrics/metrics.md
@@ -1,154 +1,168 @@
 # 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.
+Apache DolphinScheduler exports metrics for system observability. We use [Micrometer](https://micrometer.io/) as application metrics facade.
+Currently, we only support `Prometheus Exporter` but more are coming soon.
 
-## Quick Start
+## 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:
+- We enable Apache DolphinScheduler export metrics in `standalone` mode to help users get hands dirty easily. 
+- After triggering tasks in `standalone` mode, you could access metrics list by visiting url `http://localhost:12345/dolphinscheduler/actuator/metrics`.
+- After triggering tasks in `standalone` mode, you could access `prometheus-format` metrics by visiting url `http://localhost:12345/dolphinscheduler/actuator/prometheus`.
+- For a better experience with `Prometheus` and `Grafana`, we have prepared the out-of-the-box `Grafana` configuration for you, you could find the `Grafana` dashboard
+at `dolphinscheduler-meter/resources/grafana` and directly import these dashboards to your `Grafana` instance.
+- If you want to try with `docker`, you can use the following command to start the out-of-the-box `Prometheus` and `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`
+then access the `Grafana` by the url: `http://localhost/3001` for dashboards.    
 
 ![image.png](../../../../img/metrics/metrics-master.png)
 ![image.png](../../../../img/metrics/metrics-worker.png)
 ![image.png](../../../../img/metrics/metrics-datasource.png)
+      
+- If you prefer to have some experiments in `cluster` mode, please refer to the [Configuration](#configuration) section below:
+
+## Configuration
+
+- Please add the following config in master/worker/alert/api's yaml file to enable the metrics exporter.
+
+```yaml
+metrics:
+  enabled: true
+```
+
+- Once the metrics exporter enabled, you could 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`.
 
-## 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
+For example, you can get the master metrics by `curl http://localhost:5679/actuator/prometheus`.
+
+## Naming Convention & Mapping
+
+- Apache DolphinScheduler metrics naming follows the officially-recommended approach by [Micrometer](https://github.com/micrometer-metrics/micrometer-docs/blob/main/src/docs/concepts/naming.adoc#:~:text=Micrometer%20employs%20a%20naming%20convention,between%20one%20system%20and%20another.)
+- `Micrometer` automatically maps the metrics name to suit the external metrics system you configured. Currently, we only support `Prometheus Exporter` but more are coming soon.
+
+### Prometheus
+
+- all dots mapped to underscores
+- metric name starting with number added with prefix `m_` 
+- COUNTER: add `_total` suffix if not ending with it
+- LONG_TASK_TIMER: `_timer_seconds` suffix added if not ending with them
+- GAUGE: `_baseUnit` suffix added if not ending with it
+
+## Dolphin Scheduler Metrics Cheatsheet
+
+- We categorize metrics by dolphin scheduler components such as `master server`, `worker server`, `api server` and `alert server`.
+- Although task / workflow related metrics exported by `master server` and `worker server`, we categorize them separately for users to query them more conveniently.  
+
+### Task Related Metrics
+
+- ds.task.timeout.count: (counter) the number of timeout tasks
+- ds.task.finish.count: (counter) the number of finished tasks, both succeeded and failed included
+- ds.task.success.count: (counter) the number of successful tasks
+- ds.task.failure.count: (counter) the number of failed tasks
+- ds.task.stop.count: (counter) the number of stopped tasks
+- ds.task.retry.count: (counter) the number of retried tasks 
+- ds.task.submit.count: (counter) the number of submitted tasks
+- ds.task.failover.count: (counter) the number of task fail-overs
+- ds.task.dispatch.count: (counter) the number of tasks dispatched to worker
+- ds.task.dispatch.failure.count: (counter) the number of tasks failed to dispatch, retry failure included
+- ds.task.dispatch.error.count: (counter) the number of task dispatch errors
+- ds.task.execution.count.by.type: (counter) the number of task executions grouped by tag `task_type`
+- ds.task.running: (gauge) the number of running tasks 
+- ds.task.prepared: (gauge) the number of tasks prepared for task queue 
+- ds.task.execution.count: (histogram) the number of executed tasks  
+- ds.task.execution.duration: (histogram) duration of task executions
+
+
+### Workflow Related Metrics
+
+- ds.workflow.create.command.count: (counter) the number of commands created and inserted by workflows
+- ds.workflow.instance.submit.count: (counter) the number of submitted workflow instances
+- ds.workflow.instance.running: (gauge) the number of running workflow instances
+- ds.workflow.instance.timeout.count: (counter) the number of timeout workflow instances
+- ds.workflow.instance.finish.count: (counter) indicates the number of finished workflow instances, both successes and failures included
+- ds.workflow.instance.success.count: (counter) the number of successful workflow instances
+- ds.workflow.instance.failure.count: (counter) the number of failed workflow instances 
+- ds.workflow.instance.stop.count: (counter) the number of stopped workflow instances 
+- ds.workflow.instance.failover.count: (counter) the number of workflow instance fail-overs
+
+### Master Server Metrics
+
+- ds.master.overload.count: (counter) the number of times the master overloaded
+- ds.master.consume.command.count: (counter) the number of commands consumed by master 
+- ds.master.scheduler.failover.check.count: (counter) the number of scheduler (master) fail-over checks
+- ds.master.scheduler.failover.check.time: (histogram) the total time cost of scheduler (master) fail-over checks
+- ds.master.quartz.job.executed: the total number of quartz jobs executed
+- ds.master.quartz.job.execution.time: the total execution time of quartz jobs
+
+### Worker Server Metrics
+
+- ds.worker.overload.count: (counter) the number of times the worker overloaded
+- ds.worker.full.submit.queue.count: (counter) the number of times the worker's submit queue being full
+
+
+### Api Server Metrics
+
+### Alert Server Related
+
+In each server, there are some default system-level metrics related to `database connection`, `JVM`, etc. We list them below for your reference:
+
+### Database Related Metrics (Default)
+
+- hikaricp.connections: the total number of connections
+- hikaricp.connections.creation: connection creation time (max, count, sum included)
+- hikaricp.connections.acquire: connection acquirement time (max, count, sum included) 
+- hikaricp.connections.usage: connection usage time (max, count, sum included)
+- hikaricp.connections.max: the max number of connections
+- hikaricp.connections.min: the min number of connections
+- hikaricp.connections.active: the number of active connections
+- hikaricp.connections.idle: the number of idle connections
+- hikaricp.connections.pending: the number of pending connections
+- hikaricp.connections.timeout: the total number of timeout connections
+- jdbc.connections.max: the max number of active connections that can be allocated at the same time
+- jdbc.connections.min: the min number of idle connections in the pool
+- jdbc.connections.idle: the number of established but idle connections
+- jdbc.connections.active: the current number of active connections allocated from the data source
+
+### JVM Related Metrics (Default)
+
+- jvm.buffer.total.capacity: an estimate of the total capacity of the buffers in the pool
+- jvm.buffer.count: an estimate of the number of buffers in the pool
+- jvm.buffer.memory.used: an estimate of the memory that the JVM is using for this buffer pool
+- jvm.memory.committed: the amount of memory in bytes committed for the JVM to use
+- jvm.memory.max: the maximum amount of memory in bytes that can be used for memory management
+- jvm.memory.used: the amount of used memory
+- jvm.threads.peak: the peak live thread count since the JVM started or peak reset
+- jvm.threads.states: the current number of threads having NEW state
+- jvm.gc.memory.allocated: 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: max size of long-lived heap memory pool
+- jvm.gc.pause: time spent in GC pause (count, sum, max included)
+- jvm.gc.live.data.size: the size of long-lived heap memory pool after reclamation
+- jvm.gc.memory.promoted: the count of positive increases in the size of the old generation memory pool before GC to after GC.
+- jvm.classes.loaded: the number of classes currently loaded in the JVM
+- jvm.threads.live: the current number of live threads including both daemon and non-daemon threads
+- jvm.threads.daemon: the current number of live daemon threads
+- jvm.classes.unloaded: the total number of classes unloaded since the JVM started
+- process.cpu.usage: the `recent cpu usage` for the JVM process
+- process.start.time: start time of the process since unix epoch
+- process.uptime: the uptime of the JVM
+
+### Others (Default)
+
+- jetty.threads.config.max: the max number of threads in the pool
+- jetty.threads.config.min: the min 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 queued jobs waiting for a thread
+- process.files.max: the max number of file descriptors
+- process.files.open: the number of open file descriptors
+- system.cpu.usage: the recent cpu usage for the whole system
+- system.cpu.count: the number of processors available to the JVM
+- system.load.average.1m: the total number of runnable entities queued to available processors and runnable entities running on the available processors averaged over a period
+- logback.events: the number of events that made it to the logs grouped by the tag `level`
+- http.server.requests: total number of http requests
diff --git a/docs/docs/zh/guide/metrics/metrics.md b/docs/docs/zh/guide/metrics/metrics.md
new file mode 100644
index 0000000000..7fd266d978
--- /dev/null
+++ b/docs/docs/zh/guide/metrics/metrics.md
@@ -0,0 +1,169 @@
+# 指标(Metrics)使用指南
+
+Apache DolphinScheduler通过向外透出指标来提高系统的监控告警能力。 我们使用[Micrometer](https://micrometer.io/)作为指标采集和透出框架。
+目前,我们只支持`Prometheus Exporter`,但是多样化的Exporter将会持续贡献给用户。
+
+## 快速上手
+
+- 我们提供Apache DolphinScheduler `standalone` 模式下采集并透出指标的能力,提供用户轻松快速的体验。 
+- 当您在`standalone`模式下触发任务后,您可通过链接 `http://localhost:12345/dolphinscheduler/actuator/metrics` 访问生成的metrics列表。
+- 当您在`standalone`模式下触发任务后,您可通过链接 `http://localhost:12345/dolphinscheduler/actuator/prometheus` 访问`prometheus格式`指标。
+- 为了给您提供一个一站式的`Prometheus` + `Grafana`体验, 我们已经为您准备好了开箱即用的 `Grafana` 配置。您可在`dolphinscheduler-meter/resources/grafana`找到`Grafana`面板配置。
+您可直接将这些配置导入您的`Grafana`实例中。
+- 如果您想通过`docker`方式体验,可使用如下命令启动我们为您准备好的开箱即用的`Prometheus`和`Grafana`:
+
+```shell
+cd dolphinscheduler-meter/src/main/resources/grafana-demo
+docker compose up
+```
+
+然后,您即可通过http://localhost/3001`链接访问`Grafana`面板。    
+
+![image.png](../../../../img/metrics/metrics-master.png)
+![image.png](../../../../img/metrics/metrics-worker.png)
+![image.png](../../../../img/metrics/metrics-datasource.png)
+      
+- 如果您想在`集群`模式下体验指标,请参照下面的[配置](#配置)一栏:
+
+## 配置
+
+- 请按照如下配置在您的 master/worker/alert/api's yaml 文件里启用metrics exporter:
+
+```yaml
+metrics:
+  enabled: true
+```
+
+- 当您启用metrics exporter后,您可通过链接`http://ip:port/actuator/prometheus`获取metrics。
+
+metrics exporter端口`server.port`是在application.yaml里定义的: master: `server.port: 5679`, worker: `server.port: 1235`, alert: `server.port: 50053`, api: `server.port: 12345`.
+
+举例来说,您可通过访问链接获取`curl http://localhost:5679/actuator/prometheus`master metrics。
+
+## 命名规则 & 命名映射
+
+- Apache DolphinScheduler指标命名遵循[Micrometer](https://github.com/micrometer-metrics/micrometer-docs/blob/main/src/docs/concepts/naming.adoc#:~:text=Micrometer%20employs%20a%20naming%20convention,between%20one%20system%20and%20another)
+官方推荐的命名方式。
+- `Micrometer` 会根据您配置的外部指标系统自动将指标名称转化成适合您指标系统的格式。目前,我们只支持`Prometheus Exporter`,但是多样化的指标格式将会持续贡献给用户。
+
+### Prometheus
+
+- 指标名中的点会被映射为下划线
+- 以数字开头的指标名会被加上`m_`前缀 
+- COUNTER: 如果没有以`_total`结尾,会自动加上此后缀
+- LONG_TASK_TIMER: 如果没有以`_timer_seconds`结尾,会自动加上此后缀
+- GAUGE: 如果没有以`_baseUnit`结尾,会自动加上此后缀
+
+## Dolphin Scheduler指标清单
+
+- Dolphin Scheduler按照组成部分进行指标分类,如:`master server`, `worker server`, `api server` and `alert server`。
+- 尽管任务 / 工作流相关指标是由 `master server` 和 `worker server` 透出的,我们将这两块指标单独罗列出来,以方便您对任务 / 工作流的监控。  
+
+### 任务相关指标
+
+- ds.task.timeout.count: (counter) 超时的任务数量
+- ds.task.finish.count: (counter) 完成的任务数量,成功和失败的任务都算在内
+- ds.task.success.count: (counter) 成功完成的任务数量
+- ds.task.failure.count: (counter) 失败的任务数量
+- ds.task.stop.count: (counter) 暂停的任务数量
+- ds.task.retry.count: (counter) 重试的任务数量 
+- ds.task.submit.count: (counter) 已提交的任务数量
+- ds.task.failover.count: (counter) 容错的任务数量
+- ds.task.dispatch.count: (counter) 分发到worker上的任务数量
+- ds.task.dispatch.failure.count: (counter) 分发失败的任务数量,重试也包含在内
+- ds.task.dispatch.error.count: (counter) 分发任务的错误数量
+- ds.task.execution.count.by.type: (counter) 任务执行数量,按标签`task_type`聚类
+- ds.task.running: (gauge) 正在运行的任务数量 
+- ds.task.prepared: (gauge) 准备好且待提交的任务数量 
+- ds.task.execution.count: (counter) 已执行的任务数量  
+- ds.task.execution.duration: (histogram) 任务执行时长
+
+
+### 工作流相关指标
+
+- ds.workflow.create.command.count: (counter) 工作量创建并插入的命令数量
+- ds.workflow.instance.submit.count: (counter) 已提交的工作量实例数量
+- ds.workflow.instance.running: (gauge) 正在运行的工作流实例数量
+- ds.workflow.instance.timeout.count: (counter) 运行超时的工作流实例数量
+- ds.workflow.instance.finish.count: (counter) 已完成的工作流实例数量,包含成功和失败
+- ds.workflow.instance.success.count: (counter) 运行成功的工作流实例数量
+- ds.workflow.instance.failure.count: (counter) 运行失败的工作流实例数量 
+- ds.workflow.instance.stop.count: (counter) 停止的工作流实例数量 
+- ds.workflow.instance.failover.count: (counter) 容错的工作流实例数量
+
+### Master Server指标
+
+- ds.master.overload.count: (counter) master过载次数
+- ds.master.consume.command.count: (counter) master消耗指令数量 
+- ds.master.scheduler.failover.check.count: (counter) scheduler (master) 容错检查次数
+- ds.master.scheduler.failover.check.time: (histogram) scheduler (master) 容错检查耗时
+- ds.master.quartz.job.executed: 已执行quartz任务数量
+- ds.master.quartz.job.execution.time: 已执行quartz任务总耗时
+
+### Worker Server指标
+
+- ds.worker.overload.count: (counter) worker过载次数
+- ds.worker.full.submit.queue.count: (counter) worker提交队列全满次数
+
+
+### Api Server指标
+
+### Alert Server指标
+
+在每个server中都有一些系统层面(如数据库链接、JVM)的默认指标,为了您的检阅方便,我们也将它们列在了这里:
+
+### 数据库相关指标(默认)
+
+- hikaricp.connections: 连接综述
+- hikaricp.connections.creation: 连接创建时间 (包含最长时间,创建数量和时间总和)
+- hikaricp.connections.acquire: 连接获取时间 (包含最长时间,创建数量和时间总和) 
+- hikaricp.connections.usage: 连接使用时长 (包含最长时间,创建数量和时间总和)
+- hikaricp.connections.max: 最大连接数量
+- hikaricp.connections.min: 最小连接数量
+- hikaricp.connections.active: 活跃的连接数量
+- hikaricp.connections.idle: 闲置的连接数量
+- hikaricp.connections.pending: 等待中的连接数量
+- hikaricp.connections.timeout: 超时连接数量
+- jdbc.connections.max: 可同时分配的最大活跃连接数量
+- jdbc.connections.min: 连接池中最小的闲置连接数量
+- jdbc.connections.idle: 已创建但闲置的连接总数
+- jdbc.connections.active: 当前数据源分配的活跃连接数量
+
+### JVM相关指标(默认)
+
+- jvm.buffer.total.capacity: 资源池中buffer总容量估计
+- jvm.buffer.count: 资源池中buffer总数估计
+- jvm.buffer.memory.used: JVM因buffer资源池使用的内存估计
+- jvm.memory.committed: 以bytes为单位的提供JVM使用的内存
+- jvm.memory.max: 以bytes为单位的可用于内存管理的最大内存
+- jvm.memory.used: 已占用的内存量
+- jvm.threads.peak: JVM启动以来的峰值线程数
+- jvm.threads.states: 当前拥有新状态的线程数
+- jvm.gc.memory.allocated: GC时,年轻代分配的内存空间
+- jvm.gc.max.data.size: GC时,老年代的最大内存空间
+- jvm.gc.pause: GC耗时 (包含次数, 总时长, 最大时长)
+- jvm.gc.live.data.size: FullGC时,老年代的内存空间
+- jvm.gc.memory.promoted: GC时,老年代分配的内存空间
+- jvm.classes.loaded: JVM当前加载的类的数量
+- jvm.threads.live: JVM当前活跃线程数(包含守护和非守护线程)
+- jvm.threads.daemon: JVM守护线程数
+- jvm.classes.unloaded: 未加载的classes数
+- process.cpu.usage: 当前JVM进程CPU使用率
+- process.start.time: 应用启动时间
+- process.uptime: 应用已运行时间
+
+### 其他指标(默认)
+
+- jetty.threads.config.max: 资源池中最大线程数
+- jetty.threads.config.min: 资源池中最小线程数
+- jetty.threads.current: 资源池中线程总数
+- jetty.threads.idle: 资源池中闲置线程数
+- jetty.threads.busy: 资源池中繁忙线程数
+- jetty.threads.jobs: 队列中等待执行线程的任务数
+- process.files.max: 文件描述符最大数量
+- process.files.open: 打开状态的文件描述符数量
+- system.cpu.usage: 整个系统当前的CPU使用率
+- system.cpu.count: JVM可用处理器数量
+- system.load.average.1m: 系统的平均负荷(1分钟)
+- logback.events: 日志时间数量,以标签`level`聚类
+- http.server.requests: http请求总数
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
index e5c86d9641..2a2ba20dc9 100644
--- 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
@@ -30,7 +30,7 @@ public final class MasterServerMetrics {
      * Used to measure the master server is overload.
      */
     private static final Counter MASTER_OVERLOAD_COUNTER =
-            Counter.builder("dolphinscheduler_master_overload_count")
+            Counter.builder("ds.master.overload.count")
                     .description("Master server overload count")
                     .register(Metrics.globalRegistry);
 
@@ -38,7 +38,7 @@ public final class MasterServerMetrics {
      * 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")
+            Counter.builder("ds.master.consume.command.count")
                     .description("Master server consume command count")
                     .register(Metrics.globalRegistry);
 
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
index 9e9b11c95d..1693972dac 100644
--- 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
@@ -30,42 +30,42 @@ public final class ProcessInstanceMetrics {
     }
 
     private static final Counter PROCESS_INSTANCE_SUBMIT_COUNTER =
-            Counter.builder("dolphinscheduler_process_instance_submit_count")
+            Counter.builder("ds.workflow.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")
+            Counter.builder("ds.workflow.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")
+            Counter.builder("ds.workflow.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")
+            Counter.builder("ds.workflow.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")
+            Counter.builder("ds.workflow.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")
+            Counter.builder("ds.workflow.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")
+            Counter.builder("ds.workflow.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)
+        Gauge.builder("ds.workflow.instance.running", function)
                 .description("The current running process instance count")
                 .register(Metrics.globalRegistry);
     }
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
index c4516d0644..3f2ed1544f 100644
--- 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
@@ -30,57 +30,57 @@ public final class TaskMetrics {
     }
 
     private static final Counter TASK_SUBMIT_COUNTER =
-            Counter.builder("dolphinscheduler_task_submit_count")
+            Counter.builder("ds.task.submit.count")
                     .description("Task submit total count")
                     .register(Metrics.globalRegistry);
 
     private static final Counter TASK_FINISH_COUNTER =
-            Counter.builder("dolphinscheduler_task_finish_count")
+            Counter.builder("ds.task.finish.count")
                     .description("Task finish total count")
                     .register(Metrics.globalRegistry);
 
     private static final Counter TASK_SUCCESS_COUNTER =
-            Counter.builder("dolphinscheduler_task_success_count")
+            Counter.builder("ds.task.success.count")
                     .description("Task success total count")
                     .register(Metrics.globalRegistry);
 
     private static final Counter TASK_FAILURE_COUNTER =
-            Counter.builder("dolphinscheduler_task_failure_count")
+            Counter.builder("ds.task.failure.count")
                     .description("Task failure total count")
                     .register(Metrics.globalRegistry);
 
     private static final Counter TASK_TIMEOUT_COUNTER =
-            Counter.builder("dolphinscheduler_task_timeout_count")
+            Counter.builder("ds.task.timeout.count")
                     .description("Task timeout total count")
                     .register(Metrics.globalRegistry);
 
     private static final Counter TASK_RETRY_COUNTER =
-            Counter.builder("dolphinscheduler_task_retry_count")
+            Counter.builder("ds.task.retry.count")
                     .description("Task retry total count")
                     .register(Metrics.globalRegistry);
 
     private static final Counter TASK_STOP_COUNTER =
-            Counter.builder("dolphinscheduler_task_stop_count")
+            Counter.builder("ds.task.stop.count")
                     .description("Task stop total count")
                     .register(Metrics.globalRegistry);
 
     private static final Counter TASK_FAILOVER_COUNTER =
-            Counter.builder("dolphinscheduler_task_failover_count")
+            Counter.builder("ds.task.failover.count")
                     .description("Task failover total count")
                     .register(Metrics.globalRegistry);
 
     private static final Counter TASK_DISPATCH_COUNTER =
-            Counter.builder("dolphinscheduler_task_dispatch_count")
+            Counter.builder("ds.task.dispatch.count")
                     .description("Task dispatch count")
                     .register(Metrics.globalRegistry);
 
     private static final Counter TASK_DISPATCHER_FAILED =
-            Counter.builder("dolphinscheduler_task_dispatch_failed_count")
+            Counter.builder("ds.task.dispatch.failure.count")
                     .description("Task dispatch failed count")
                     .register(Metrics.globalRegistry);
 
     private static final Counter TASK_DISPATCH_ERROR =
-            Counter.builder("dolphinscheduler_task_dispatch_error_count")
+            Counter.builder("ds.task.dispatch.error.count")
                     .description("Task dispatch error")
                     .register(Metrics.globalRegistry);
 
@@ -88,9 +88,9 @@ public final class TaskMetrics {
         TASK_SUBMIT_COUNTER.increment();
     }
 
-    public synchronized static void registerTaskRunning(Supplier<Number> consumer) {
-        Gauge.builder("dolphinscheduler_task_running_gauge", consumer)
-                .description("Task running count")
+    public synchronized static void registerTaskPrepared(Supplier<Number> consumer) {
+        Gauge.builder("ds.task.prepared", consumer)
+                .description("Task prepared count")
                 .register(Metrics.globalRegistry);
     }
 
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 dfaa1810be..801b482be2 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
@@ -254,7 +254,7 @@ public class WorkflowExecuteRunnable implements Runnable {
         this.nettyExecutorManager = nettyExecutorManager;
         this.processAlertManager = processAlertManager;
         this.stateWheelExecuteThread = stateWheelExecuteThread;
-        TaskMetrics.registerTaskRunning(readyToSubmitTaskQueue::size);
+        TaskMetrics.registerTaskPrepared(readyToSubmitTaskQueue::size);
     }
 
     /**
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 3cc92fb905..bc2589c585 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
@@ -77,8 +77,8 @@ public class FailoverService {
     /**
      * check master failover
      */
-    @Counted(value = "failover_scheduler_check_task_count")
-    @Timed(value = "failover_scheduler_check_task_time", percentiles = {0.5, 0.75, 0.95, 0.99}, histogram = true)
+    @Counted(value = "ds.master.scheduler.failover.check.count")
+    @Timed(value = "ds.master.scheduler.failover.check.time", percentiles = {0.5, 0.75, 0.95, 0.99}, histogram = true)
     public void checkMasterFailover() {
         List<String> hosts = getNeedFailoverMasterServers();
         if (CollectionUtils.isEmpty(hosts)) {
diff --git a/dolphinscheduler-meter/src/main/resources/grafana/DolphinSchedulerMaster.json b/dolphinscheduler-meter/src/main/resources/grafana/DolphinSchedulerMaster.json
index 90ba34c5fc..4d7a1ae524 100644
--- a/dolphinscheduler-meter/src/main/resources/grafana/DolphinSchedulerMaster.json
+++ b/dolphinscheduler-meter/src/main/resources/grafana/DolphinSchedulerMaster.json
@@ -1585,7 +1585,7 @@
             "type": "prometheus",
             "uid": "PBFA97CFB590B2093"
           },
-          "expr": "sum(increase(dolphinscheduler_task_dispatch_count_total{}[1m]))",
+          "expr": "sum(increase(ds_task_dispatch_count_total{}[1m]))",
           "refId": "A"
         },
         {
@@ -1593,7 +1593,7 @@
             "type": "prometheus",
             "uid": "PBFA97CFB590B2093"
           },
-          "expr": "sum(increase(dolphinscheduler_task_dispatch_failed_count_total{}[1m]))",
+          "expr": "sum(increase(ds_task_dispatch_failure_count_total{}[1m]))",
           "hide": false,
           "refId": "B"
         },
@@ -1602,7 +1602,7 @@
             "type": "prometheus",
             "uid": "PBFA97CFB590B2093"
           },
-          "expr": "sum(increase(dolphinscheduler_task_dispatch_error_count_total{}[1m]))",
+          "expr": "sum(increase(ds_task_dispatch_error_count_total{}[1m]))",
           "hide": false,
           "refId": "C"
         }
@@ -1689,7 +1689,7 @@
             "type": "prometheus",
             "uid": "PBFA97CFB590B2093"
           },
-          "expr": "sum(increase(dolphinscheduler_task_submit_count_total{}[1m]))",
+          "expr": "sum(increase(ds_task_submit_count_total{}[1m]))",
           "refId": "A"
         }
       ],
@@ -1775,7 +1775,7 @@
             "type": "prometheus",
             "uid": "PBFA97CFB590B2093"
           },
-          "expr": "sum(increase(dolphinscheduler_task_finish_count_total{}[1m]))",
+          "expr": "sum(increase(ds_task_finish_count_total{}[1m]))",
           "refId": "A"
         }
       ],
@@ -1861,7 +1861,7 @@
             "type": "prometheus",
             "uid": "PBFA97CFB590B2093"
           },
-          "expr": "sum(increase(dolphinscheduler_task_success_count_total{}[1m]))",
+          "expr": "sum(increase(ds_task_success_count_total{}[1m]))",
           "refId": "A"
         }
       ],
@@ -1947,7 +1947,7 @@
             "type": "prometheus",
             "uid": "PBFA97CFB590B2093"
           },
-          "expr": "sum(increase(dolphinscheduler_task_failure_count_total{}[1m]))",
+          "expr": "sum(increase(ds_task_failure_count_total{}[1m]))",
           "refId": "A"
         }
       ],
@@ -2033,7 +2033,7 @@
             "type": "prometheus",
             "uid": "PBFA97CFB590B2093"
           },
-          "expr": "sum(increase(dolphinscheduler_task_timeout_count_total{}[1m]))",
+          "expr": "sum(increase(ds_task_timeout_count_total{}[1m]))",
           "refId": "A"
         }
       ],
@@ -2119,7 +2119,7 @@
             "type": "prometheus",
             "uid": "PBFA97CFB590B2093"
           },
-          "expr": "sum(increase(dolphinscheduler_task_retry_count_total{}[1m]))",
+          "expr": "sum(increase(ds_task_retry_count_total{}[1m]))",
           "refId": "A"
         }
       ],
@@ -2205,7 +2205,7 @@
             "type": "prometheus",
             "uid": "PBFA97CFB590B2093"
           },
-          "expr": "sum(increase(dolphinscheduler_task_stop_count_total{}[1m]))",
+          "expr": "sum(increase(ds_task_stop_count_total{}[1m]))",
           "refId": "A"
         }
       ],
@@ -2291,7 +2291,7 @@
             "type": "prometheus",
             "uid": "PBFA97CFB590B2093"
           },
-          "expr": "sum(increase(dolphinscheduler_task_failover_count_total{}[1m]))",
+          "expr": "sum(increase(ds_task_failover_count_total{}[1m]))",
           "refId": "A"
         }
       ],
diff --git a/dolphinscheduler-meter/src/main/resources/grafana/DolphinSchedulerWorker.json b/dolphinscheduler-meter/src/main/resources/grafana/DolphinSchedulerWorker.json
index 765f40bf8f..01cd09baad 100644
--- a/dolphinscheduler-meter/src/main/resources/grafana/DolphinSchedulerWorker.json
+++ b/dolphinscheduler-meter/src/main/resources/grafana/DolphinSchedulerWorker.json
@@ -265,7 +265,7 @@
             "type": "prometheus",
             "uid": "PBFA97CFB590B2093"
           },
-          "expr": "increase(dolphinscheduler_worker_submit_queue_is_full_count_total{}[1m])",
+          "expr": "increase(ds_worker_full_submit_queue_count_total{}[1m])",
           "refId": "A"
         }
       ],
@@ -351,7 +351,7 @@
             "type": "prometheus",
             "uid": "PBFA97CFB590B2093"
           },
-          "expr": "increase(dolphinscheduler_worker_overload_count_total{}[1m])",
+          "expr": "increase(ds_worker_overload_count_total{}[1m])",
           "refId": "A"
         }
       ],
@@ -437,7 +437,7 @@
             "type": "prometheus",
             "uid": "PBFA97CFB590B2093"
           },
-          "expr": "dolphinscheduler_worker_running_task_gauge{}",
+          "expr": "ds_task_running{}",
           "refId": "A"
         }
       ],
@@ -495,7 +495,7 @@
             "type": "prometheus",
             "uid": "PBFA97CFB590B2093"
           },
-          "expr": "sum(dolphinscheduler_task_execution_count_total)",
+          "expr": "sum(ds_task_execution_count_total)",
           "refId": "A"
         }
       ],
@@ -551,7 +551,7 @@
             "type": "prometheus",
             "uid": "PBFA97CFB590B2093"
           },
-          "expr": "sum(dolphinscheduler_task_execution_count_total{result=\"success\"}) / sum(dolphinscheduler_task_execution_count_total) * 100",
+          "expr": "sum(ds_task_execution_count_total{result=\"success\"}) / sum(ds_task_execution_count_total) * 100",
           "refId": "A"
         }
       ],
@@ -614,7 +614,7 @@
             "uid": "PBFA97CFB590B2093"
           },
           "editorMode": "code",
-          "expr": "rate(dolphinscheduler_task_execution_timer_seconds_sum[1m])/rate(dolphinscheduler_task_execution_timer_seconds_count[1m])",
+          "expr": "rate(ds_task_execution_duration_seconds_sum[1m])/rate(ds_task_execution_duration_seconds_count[1m])",
           "legendFormat": "avg ({{exception}})",
           "range": true,
           "refId": "A"
@@ -625,7 +625,7 @@
             "uid": "PBFA97CFB590B2093"
           },
           "editorMode": "code",
-          "expr": "dolphinscheduler_task_execution_timer_seconds_max",
+          "expr": "ds_task_execution_duration_seconds_max",
           "hide": false,
           "legendFormat": "max ({{exception}})",
           "range": true,
@@ -715,7 +715,7 @@
           },
           "editorMode": "code",
           "exemplar": true,
-          "expr": "sum(increase(dolphinscheduler_task_execution_count_total[1m]))",
+          "expr": "sum(increase(ds_task_execution_count_total[1m]))",
           "legendFormat": "Total",
           "range": true,
           "refId": "A"
@@ -727,7 +727,7 @@
           },
           "editorMode": "code",
           "exemplar": true,
-          "expr": "increase(dolphinscheduler_task_execution_count_total{result=\"success\"}[1m])",
+          "expr": "increase(ds_task_execution_count_total{result=\"success\"}[1m])",
           "hide": false,
           "legendFormat": "Successful",
           "range": true,
@@ -857,7 +857,7 @@
             "uid": "PBFA97CFB590B2093"
           },
           "editorMode": "code",
-          "expr": "sum(increase(dolphinscheduler_task_execute_count_total{task_type=\"SHELL\"}[1d]))",
+          "expr": "sum(increase(ds_task_execution_count_by_type_total{task_type=\"SHELL\"}[1d]))",
           "legendFormat": "SHELL",
           "range": true,
           "refId": "A"
@@ -946,7 +946,7 @@
             "uid": "PBFA97CFB590B2093"
           },
           "editorMode": "code",
-          "expr": "sum(increase(dolphinscheduler_task_execute_count_total{task_type=\"unknown\"}[1d]))",
+          "expr": "sum(increase(ds_task_execution_count_by_type_total{task_type=\"unknown\"}[1d]))",
           "legendFormat": "unknown",
           "range": true,
           "refId": "A"
@@ -1035,7 +1035,7 @@
             "uid": "PBFA97CFB590B2093"
           },
           "editorMode": "code",
-          "expr": "sum(increase(dolphinscheduler_task_execute_count_total{task_type=\"CONDITIONS\"}[1d]))",
+          "expr": "sum(increase(ds_task_execution_count_by_type_total{task_type=\"CONDITIONS\"}[1d]))",
           "legendFormat": "CONDITIONS",
           "range": true,
           "refId": "A"
@@ -1124,7 +1124,7 @@
             "uid": "PBFA97CFB590B2093"
           },
           "editorMode": "code",
-          "expr": "sum(increase(dolphinscheduler_task_execute_count_total{task_type=\"BLOCKING\"}[1d]))",
+          "expr": "sum(increase(ds_task_execution_count_by_type_total{task_type=\"BLOCKING\"}[1d]))",
           "legendFormat": "BLOCKING",
           "range": true,
           "refId": "A"
@@ -1213,7 +1213,7 @@
             "uid": "PBFA97CFB590B2093"
           },
           "editorMode": "code",
-          "expr": "sum(increase(dolphinscheduler_task_execute_count_total{task_type=\"DATAX\"}[1d]))",
+          "expr": "sum(increase(ds_task_execution_count_by_type_total{task_type=\"DATAX\"}[1d]))",
           "legendFormat": "DATAX",
           "range": true,
           "refId": "A"
@@ -1302,7 +1302,7 @@
             "uid": "PBFA97CFB590B2093"
           },
           "editorMode": "code",
-          "expr": "sum(increase(dolphinscheduler_task_execute_count_total{task_type=\"DATA_QUALITY\"}[1d]))",
+          "expr": "sum(increase(ds_task_execution_count_by_type_total{task_type=\"DATA_QUALITY\"}[1d]))",
           "legendFormat": "DATA_QUALITY",
           "range": true,
           "refId": "A"
@@ -1391,7 +1391,7 @@
             "uid": "PBFA97CFB590B2093"
           },
           "editorMode": "code",
-          "expr": "sum(increase(dolphinscheduler_task_execute_count_total{task_type=\"DATA_QUALITY\"}[1d]))",
+          "expr": "sum(increase(ds_task_execution_count_by_type_total{task_type=\"DATA_QUALITY\"}[1d]))",
           "legendFormat": "DATA_QUALITY",
           "range": true,
           "refId": "A"
@@ -1480,7 +1480,7 @@
             "uid": "PBFA97CFB590B2093"
           },
           "editorMode": "code",
-          "expr": "sum(increase(dolphinscheduler_task_execute_count_total{task_type=\"DEPENDENT\"}[1d]))",
+          "expr": "sum(increase(ds_task_execution_count_by_type_total{task_type=\"DEPENDENT\"}[1d]))",
           "legendFormat": "DEPENDENT",
           "range": true,
           "refId": "A"
@@ -1569,7 +1569,7 @@
             "uid": "PBFA97CFB590B2093"
           },
           "editorMode": "code",
-          "expr": "sum(increase(dolphinscheduler_task_execute_count_total{task_type=\"EMR\"}[1d]))",
+          "expr": "sum(increase(ds_task_execution_count_by_type_total{task_type=\"EMR\"}[1d]))",
           "legendFormat": "EMR",
           "range": true,
           "refId": "A"
diff --git a/dolphinscheduler-scheduler-plugin/dolphinscheduler-scheduler-quartz/src/main/java/org/apache/dolphinscheduler/scheduler/quartz/ProcessScheduleTask.java b/dolphinscheduler-scheduler-plugin/dolphinscheduler-scheduler-quartz/src/main/java/org/apache/dolphinscheduler/scheduler/quartz/ProcessScheduleTask.java
index d65fae32b8..2b546940ec 100644
--- a/dolphinscheduler-scheduler-plugin/dolphinscheduler-scheduler-quartz/src/main/java/org/apache/dolphinscheduler/scheduler/quartz/ProcessScheduleTask.java
+++ b/dolphinscheduler-scheduler-plugin/dolphinscheduler-scheduler-quartz/src/main/java/org/apache/dolphinscheduler/scheduler/quartz/ProcessScheduleTask.java
@@ -48,8 +48,8 @@ public class ProcessScheduleTask extends QuartzJobBean {
     @Autowired
     private ProcessService processService;
 
-    @Counted(value = "quartz_job_executed")
-    @Timed(value = "quartz_job_execution", percentiles = {0.5, 0.75, 0.95, 0.99}, histogram = true)
+    @Counted(value = "ds.master.quartz.job.executed")
+    @Timed(value = "ds.master.quartz.job.execution.time", percentiles = {0.5, 0.75, 0.95, 0.99}, histogram = true)
     @Override
     protected void executeInternal(JobExecutionContext context) {
         JobDataMap dataMap = context.getJobDetail().getJobDataMap();
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 b42e033b20..b66fb915bb 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
@@ -408,7 +408,7 @@ public class ProcessServiceImpl implements ProcessService {
      * @return create result
      */
     @Override
-    @Counted("dolphinscheduler_create_command_count")
+    @Counted("ds.workflow.create.command.count")
     public int createCommand(Command command) {
         int result = 0;
         if (command != null) {
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
index c45275a8a3..6ec6a3a3cf 100644
--- 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
@@ -34,18 +34,18 @@ public final class TaskMetrics {
 
     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")
+            Counter.builder("ds.task.execution.count.by.type")
                     .tag("task_type", "unknown")
-                    .description("task execute counter")
+                    .description("task execution counter by type")
                     .register(Metrics.globalRegistry);
 
     static {
         for (TaskChannelFactory taskChannelFactory : ServiceLoader.load(TaskChannelFactory.class)) {
             TASK_TYPE_EXECUTE_COUNTER.put(
                     taskChannelFactory.getName(),
-                    Counter.builder("dolphinscheduler_task_execute_count")
+                    Counter.builder("ds.task.execution.count.by.type")
                             .tag("task_type", taskChannelFactory.getName())
-                            .description("task execute counter")
+                            .description("task execution counter by type")
                             .register(Metrics.globalRegistry)
             );
         }
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
index 9613211f43..1e69d873ab 100644
--- 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
@@ -30,13 +30,13 @@ public final class WorkerServerMetrics {
     }
 
     private static final Counter WORKER_OVERLOAD_COUNTER =
-            Counter.builder("dolphinscheduler_worker_overload_count")
-                    .description("worker load count")
+            Counter.builder("ds.worker.overload.count")
+                    .description("overloaded workers 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")
+            Counter.builder("ds.worker.full.submit.queue.count")
+                    .description("full worker submit queues count")
                     .register(Metrics.globalRegistry);
 
     public static void incWorkerOverloadCount() {
@@ -48,8 +48,8 @@ public final class WorkerServerMetrics {
     }
 
     public static void registerWorkerRunningTaskGauge(Supplier<Number> supplier) {
-        Gauge.builder("dolphinscheduler_worker_running_task_gauge", supplier)
-                .description("worker running task gauge")
+        Gauge.builder("ds.task.running", supplier)
+                .description("number of running tasks on workers")
                 .register(Metrics.globalRegistry);
 
     }
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 6de69c30c6..91f463522a 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
@@ -90,8 +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)
+    @Counted(value = "ds.task.execution.count", description = "task execute total count")
+    @Timed(value = "ds.task.execution.duration", 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(),