You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@dolphinscheduler.apache.org by ch...@apache.org on 2022/08/03 07:42:14 UTC

[dolphinscheduler] branch dev updated: [Improvement][Metrics] Add metrics for alert server (#11240)

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

chufenggao 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 9ca1eb96c4 [Improvement][Metrics] Add metrics for alert server (#11240)
9ca1eb96c4 is described below

commit 9ca1eb96c429f76574bfa98b0ca5eff8d8c89a82
Author: Eric Gao <er...@gmail.com>
AuthorDate: Wed Aug 3 15:42:06 2022 +0800

    [Improvement][Metrics] Add metrics for alert server (#11240)
    
    * [Improvement][Metrics] Add metrics for alert server (#11131)
    
    * Update related docs of metrics
    
    * Add grafana demo dashboards for alert server metrics
    
    * Refactor metric classes with UtilityClass annotation
    
    * Refactor meter names in camelCase for checkstyle
---
 docs/docs/en/guide/metrics/metrics.md              |   5 +-
 docs/docs/zh/guide/metrics/metrics.md              |   5 +-
 .../dolphinscheduler/alert/AlertSenderService.java |   4 +
 .../dolphinscheduler/alert/AlertServerMetrics.java |  47 +--
 .../server/master/metrics/MasterServerMetrics.java |  20 +-
 .../master/metrics/ProcessInstanceMetrics.java     |  36 +-
 .../server/master/metrics/TaskMetrics.java         |  42 +--
 .../resources/grafana/DolphinSchedulerAlert.json   | 416 +++++++++++++++++++++
 .../server/worker/metrics/TaskMetrics.java         |  19 +-
 .../server/worker/metrics/WorkerServerMetrics.java |  38 +-
 10 files changed, 525 insertions(+), 107 deletions(-)

diff --git a/docs/docs/en/guide/metrics/metrics.md b/docs/docs/en/guide/metrics/metrics.md
index 1b5d099baf..6e2730af67 100644
--- a/docs/docs/en/guide/metrics/metrics.md
+++ b/docs/docs/en/guide/metrics/metrics.md
@@ -115,9 +115,10 @@ For example, you can get the master metrics by `curl http://localhost:5679/actua
 
 ### Alert Server Related
 
-- Currently, we have not embedded any metrics in Alert Server.
+- ds.alert.send.count: (counter) the number of sent alerts, sliced by tag `status`
+- ds.alert.pending: (gauge) the number of alerts waiting to get fired
 
-In each server, there are some default system-level metrics related to `database connection`, `JVM`, etc. We list them below for your reference:
+**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)
 
diff --git a/docs/docs/zh/guide/metrics/metrics.md b/docs/docs/zh/guide/metrics/metrics.md
index 9101935cc0..adaffbbbbe 100644
--- a/docs/docs/zh/guide/metrics/metrics.md
+++ b/docs/docs/zh/guide/metrics/metrics.md
@@ -116,9 +116,10 @@ metrics exporter端口`server.port`是在application.yaml里定义的: master: `
 
 ### Alert Server指标
 
-- 目前我们尚未提供任何Alert Server指标
+- ds.alert.send.count: (counter) 已发送的告警数量,可由标签`status`切分
+- ds.alert.pending: (gauge) 等待发送的告警数量
 
-在每个server中都有一些系统层面(如数据库链接、JVM)的默认指标,为了您的检阅方便,我们也将它们列在了这里:
+**在每个server中都有一些系统层面(如数据库链接、JVM)的默认指标,为了您的检阅方便,我们也将它们列在了这里:**
 
 ### 数据库相关指标(默认)
 
diff --git a/dolphinscheduler-alert/dolphinscheduler-alert-server/src/main/java/org/apache/dolphinscheduler/alert/AlertSenderService.java b/dolphinscheduler-alert/dolphinscheduler-alert-server/src/main/java/org/apache/dolphinscheduler/alert/AlertSenderService.java
index 48dfaf3e3d..798c717d59 100644
--- a/dolphinscheduler-alert/dolphinscheduler-alert-server/src/main/java/org/apache/dolphinscheduler/alert/AlertSenderService.java
+++ b/dolphinscheduler-alert/dolphinscheduler-alert-server/src/main/java/org/apache/dolphinscheduler/alert/AlertSenderService.java
@@ -78,6 +78,7 @@ public final class AlertSenderService extends Thread {
         while (Stopper.isRunning()) {
             try {
                 List<Alert> alerts = alertDao.listPendingAlerts();
+                AlertServerMetrics.registerPendingAlertGauge(alerts::size);
                 this.send(alerts);
                 ThreadUtils.sleep(Constants.SLEEP_TIME_MILLIS * 5L);
             } catch (Exception e) {
@@ -117,6 +118,9 @@ public final class AlertSenderService extends Thread {
                     alertDao.addAlertSendStatus(sendStatus, JSONUtils.toJsonString(alertResult), alertId, instance.getId());
                     if (sendStatus.equals(AlertStatus.EXECUTION_SUCCESS)) {
                         sendSuccessCount++;
+                        AlertServerMetrics.incAlertSuccessCount();
+                    } else {
+                        AlertServerMetrics.incAlertFailCount();
                     }
                     alertResults.add(alertResult);
                 }
diff --git a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/metrics/MasterServerMetrics.java b/dolphinscheduler-alert/dolphinscheduler-alert-server/src/main/java/org/apache/dolphinscheduler/alert/AlertServerMetrics.java
similarity index 50%
copy from dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/metrics/MasterServerMetrics.java
copy to dolphinscheduler-alert/dolphinscheduler-alert-server/src/main/java/org/apache/dolphinscheduler/alert/AlertServerMetrics.java
index 2a2ba20dc9..0218e51d2b 100644
--- a/dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/metrics/MasterServerMetrics.java
+++ b/dolphinscheduler-alert/dolphinscheduler-alert-server/src/main/java/org/apache/dolphinscheduler/alert/AlertServerMetrics.java
@@ -15,39 +15,42 @@
  * limitations under the License.
  */
 
-package org.apache.dolphinscheduler.server.master.metrics;
+package org.apache.dolphinscheduler.alert;
+
+import java.util.function.Supplier;
 
 import io.micrometer.core.instrument.Counter;
+import io.micrometer.core.instrument.Gauge;
 import io.micrometer.core.instrument.Metrics;
+import lombok.experimental.UtilityClass;
 
-public final class MasterServerMetrics {
-
-    private MasterServerMetrics() {
-        throw new UnsupportedOperationException("Utility class");
-    }
+@UtilityClass
+public class AlertServerMetrics {
 
-    /**
-     * Used to measure the master server is overload.
-     */
-    private static final Counter MASTER_OVERLOAD_COUNTER =
-            Counter.builder("ds.master.overload.count")
-                    .description("Master server overload count")
+    private final Counter alertSuccessCounter =
+            Counter.builder("ds.alert.send.count")
+                    .tag("status", "success")
+                    .description("Alert success 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("ds.master.consume.command.count")
-                    .description("Master server consume command count")
+    private final Counter alertFailCounter =
+            Counter.builder("ds.alert.send.count")
+                    .tag("status", "fail")
+                    .description("Alert failure count")
                     .register(Metrics.globalRegistry);
 
-    public static void incMasterOverload() {
-        MASTER_OVERLOAD_COUNTER.increment();
+    public void registerPendingAlertGauge(final Supplier<Number> supplier) {
+        Gauge.builder("ds.alert.pending", supplier)
+                .description("Number of pending alert")
+                .register(Metrics.globalRegistry);
+    }
+
+    public void incAlertSuccessCount() {
+        alertSuccessCounter.increment();
     }
 
-    public static void incMasterConsumeCommand(int commandCount) {
-        MASTER_CONSUME_COMMAND_COUNTER.increment(commandCount);
+    public void incAlertFailCount() {
+        alertFailCounter.increment();
     }
 
 }
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 2a2ba20dc9..0ed522e7cd 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
@@ -19,17 +19,15 @@ package org.apache.dolphinscheduler.server.master.metrics;
 
 import io.micrometer.core.instrument.Counter;
 import io.micrometer.core.instrument.Metrics;
+import lombok.experimental.UtilityClass;
 
-public final class MasterServerMetrics {
-
-    private MasterServerMetrics() {
-        throw new UnsupportedOperationException("Utility class");
-    }
+@UtilityClass
+public class MasterServerMetrics {
 
     /**
      * Used to measure the master server is overload.
      */
-    private static final Counter MASTER_OVERLOAD_COUNTER =
+    private final Counter masterOverloadCounter =
             Counter.builder("ds.master.overload.count")
                     .description("Master server overload count")
                     .register(Metrics.globalRegistry);
@@ -37,17 +35,17 @@ public final class MasterServerMetrics {
     /**
      * Used to measure the number of process command consumed by master.
      */
-    private static final Counter MASTER_CONSUME_COMMAND_COUNTER =
+    private final Counter masterConsumeCommandCounter =
             Counter.builder("ds.master.consume.command.count")
                     .description("Master server consume command count")
                     .register(Metrics.globalRegistry);
 
-    public static void incMasterOverload() {
-        MASTER_OVERLOAD_COUNTER.increment();
+    public void incMasterOverload() {
+        masterOverloadCounter.increment();
     }
 
-    public static void incMasterConsumeCommand(int commandCount) {
-        MASTER_CONSUME_COMMAND_COUNTER.increment(commandCount);
+    public void incMasterConsumeCommand(int commandCount) {
+        masterConsumeCommandCounter.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
index 4cd8b3715a..867f8ae548 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
@@ -29,21 +29,19 @@ import io.micrometer.core.instrument.Counter;
 import io.micrometer.core.instrument.Gauge;
 import io.micrometer.core.instrument.Metrics;
 import io.micrometer.core.instrument.Timer;
+import lombok.experimental.UtilityClass;
 
-public final class ProcessInstanceMetrics {
+@UtilityClass
+public class ProcessInstanceMetrics {
 
-    private ProcessInstanceMetrics() {
-        throw new UnsupportedOperationException("Utility class");
-    }
-
-    private static Map<String, Counter> PROCESS_INSTANCE_COUNTERS = new HashMap<>();
+    private final Map<String, Counter> processInstanceCounters = new HashMap<>();
 
-    private static final Set<String> PROCESS_INSTANCE_STATES = ImmutableSet.of(
+    private final Set<String> processInstanceStates = ImmutableSet.of(
             "submit", "timeout", "finish", "failover", "success", "fail", "stop");
 
     static {
-        for (final String state : PROCESS_INSTANCE_STATES) {
-            PROCESS_INSTANCE_COUNTERS.put(
+        for (final String state : processInstanceStates) {
+            processInstanceCounters.put(
                     state,
                     Counter.builder("ds.workflow.instance.count")
                             .tag("state", state)
@@ -54,38 +52,38 @@ public final class ProcessInstanceMetrics {
 
     }
 
-    private static final Timer COMMAND_QUERY_TIMETER =
+    private final Timer commandQueryTimer =
         Timer.builder("ds.workflow.command.query.duration")
             .description("Command query duration")
             .register(Metrics.globalRegistry);
 
-    private static final Timer PROCESS_INSTANCE_GENERATE_TIMER =
+    private final Timer processInstanceGenerateTimer =
         Timer.builder("ds.workflow.instance.generate.duration")
             .description("Process instance generated duration")
             .register(Metrics.globalRegistry);
 
-    public static void recordCommandQueryTime(long milliseconds) {
-        COMMAND_QUERY_TIMETER.record(milliseconds, TimeUnit.MILLISECONDS);
+    public void recordCommandQueryTime(long milliseconds) {
+        commandQueryTimer.record(milliseconds, TimeUnit.MILLISECONDS);
     }
 
-    public static void recordProcessInstanceGenerateTime(long milliseconds) {
-        PROCESS_INSTANCE_GENERATE_TIMER.record(milliseconds, TimeUnit.MILLISECONDS);
+    public void recordProcessInstanceGenerateTime(long milliseconds) {
+        processInstanceGenerateTimer.record(milliseconds, TimeUnit.MILLISECONDS);
     }
 
-    public static synchronized void registerProcessInstanceRunningGauge(Supplier<Number> function) {
+    public synchronized void registerProcessInstanceRunningGauge(Supplier<Number> function) {
         Gauge.builder("ds.workflow.instance.running", function)
             .description("The current running process instance count")
             .register(Metrics.globalRegistry);
     }
 
-    public static synchronized void registerProcessInstanceResubmitGauge(Supplier<Number> function) {
+    public synchronized void registerProcessInstanceResubmitGauge(Supplier<Number> function) {
         Gauge.builder("ds.workflow.instance.resubmit", function)
             .description("The current process instance need to resubmit count")
             .register(Metrics.globalRegistry);
     }
 
-    public static void incProcessInstanceByState(final String state) {
-        PROCESS_INSTANCE_COUNTERS.get(state).increment();
+    public void incProcessInstanceByState(final String state) {
+        processInstanceCounters.get(state).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
index 5b20c59bac..2b71343a45 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
@@ -23,25 +23,23 @@ import java.util.Set;
 import java.util.function.Supplier;
 
 import com.facebook.presto.jdbc.internal.guava.collect.ImmutableSet;
+
 import io.micrometer.core.instrument.Counter;
 import io.micrometer.core.instrument.Gauge;
 import io.micrometer.core.instrument.Metrics;
+import lombok.experimental.UtilityClass;
 
+@UtilityClass
+public class TaskMetrics {
 
-public final class TaskMetrics {
-    private TaskMetrics() {
-        throw new UnsupportedOperationException("Utility class");
-    }
-
-
-    private static Map<String, Counter> TASK_INSTANCE_COUNTERS = new HashMap<>();
+    private final Map<String, Counter> taskInstanceCounters = new HashMap<>();
 
-    private static final Set<String> TASK_INSTANCE_STATES = ImmutableSet.of(
+    private final Set<String> taskInstanceStates = ImmutableSet.of(
             "submit", "timeout", "finish", "failover", "retry", "dispatch", "success", "fail", "stop");
 
     static {
-        for (final String state : TASK_INSTANCE_STATES) {
-            TASK_INSTANCE_COUNTERS.put(
+        for (final String state : taskInstanceStates) {
+            taskInstanceCounters.put(
                     state,
                     Counter.builder("ds.task.instance.count")
                             .tags("state", state)
@@ -52,41 +50,41 @@ public final class TaskMetrics {
 
     }
 
-    private static final Counter TASK_DISPATCH_COUNTER =
+    private final Counter taskDispatchCounter =
             Counter.builder("ds.task.dispatch.count")
                     .description("Task dispatch count")
                     .register(Metrics.globalRegistry);
 
-    private static final Counter TASK_DISPATCHER_FAILED =
+    private final Counter taskDispatchFailCounter =
             Counter.builder("ds.task.dispatch.failure.count")
                     .description("Task dispatch failures count, retried ones included")
                     .register(Metrics.globalRegistry);
 
-    private static final Counter TASK_DISPATCH_ERROR =
+    private final Counter taskDispatchErrorCounter =
             Counter.builder("ds.task.dispatch.error.count")
                     .description("Number of errors during task dispatch")
                     .register(Metrics.globalRegistry);
 
-    public synchronized static void registerTaskPrepared(Supplier<Number> consumer) {
+    public synchronized void registerTaskPrepared(Supplier<Number> consumer) {
         Gauge.builder("ds.task.prepared", consumer)
                 .description("Task prepared count")
                 .register(Metrics.globalRegistry);
     }
 
-    public static void incTaskDispatchFailed(int failedCount) {
-        TASK_DISPATCHER_FAILED.increment(failedCount);
+    public void incTaskDispatchFailed(int failedCount) {
+        taskDispatchFailCounter.increment(failedCount);
     }
 
-    public static void incTaskDispatchError() {
-        TASK_DISPATCH_ERROR.increment();
+    public void incTaskDispatchError() {
+        taskDispatchErrorCounter.increment();
     }
 
-    public static void incTaskDispatch() {
-        TASK_DISPATCH_COUNTER.increment();
+    public void incTaskDispatch() {
+        taskDispatchCounter.increment();
     }
 
-    public static void incTaskInstanceByState(final String state) {
-        TASK_INSTANCE_COUNTERS.get(state).increment();
+    public void incTaskInstanceByState(final String state) {
+        taskInstanceCounters.get(state).increment();
     }
 
 }
diff --git a/dolphinscheduler-meter/src/main/resources/grafana/DolphinSchedulerAlert.json b/dolphinscheduler-meter/src/main/resources/grafana/DolphinSchedulerAlert.json
new file mode 100644
index 0000000000..31c7c36aed
--- /dev/null
+++ b/dolphinscheduler-meter/src/main/resources/grafana/DolphinSchedulerAlert.json
@@ -0,0 +1,416 @@
+{
+  "annotations": {
+    "list": [
+      {
+        "builtIn": 1,
+        "datasource": {
+          "type": "datasource",
+          "uid": "grafana"
+        },
+        "enable": true,
+        "hide": true,
+        "iconColor": "rgba(0, 211, 255, 1)",
+        "limit": 100,
+        "name": "Annotations & Alerts",
+        "showIn": 0,
+        "target": {
+          "limit": 100,
+          "matchAny": false,
+          "tags": [],
+          "type": "dashboard"
+        },
+        "type": "dashboard"
+      },
+      {
+        "datasource": {
+          "type": "prometheus",
+          "uid": "PBFA97CFB590B2093"
+        },
+        "enable": true,
+        "expr": "resets(process_uptime_seconds{application=\"$application\", instance=\"$instance\"}[1m]) > 0",
+        "iconColor": "rgba(255, 96, 96, 1)",
+        "name": "Restart Detection",
+        "showIn": 0,
+        "step": "1m",
+        "tagKeys": "restart-tag",
+        "textFormat": "uptime reset",
+        "titleFormat": "Restart"
+      }
+    ]
+  },
+  "description": "Dashboard for Micrometer instrumented applications (Java, Spring Boot, Micronaut)",
+  "editable": true,
+  "fiscalYearStartMonth": 0,
+  "gnetId": 4701,
+  "graphTooltip": 1,
+  "links": [],
+  "liveNow": false,
+  "panels": [
+    {
+      "collapsed": false,
+      "datasource": {
+        "type": "datasource",
+        "uid": "grafana"
+      },
+      "gridPos": {
+        "h": 1,
+        "w": 24,
+        "x": 0,
+        "y": 0
+      },
+      "id": 126,
+      "panels": [],
+      "title": "Alert Server",
+      "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": 12,
+        "x": 0,
+        "y": 1
+      },
+      "id": 152,
+      "options": {
+        "legend": {
+          "calcs": [],
+          "displayMode": "list",
+          "placement": "bottom"
+        },
+        "tooltip": {
+          "mode": "single",
+          "sort": "none"
+        }
+      },
+      "targets": [
+        {
+          "datasource": {
+            "type": "prometheus",
+            "uid": "PBFA97CFB590B2093"
+          },
+          "expr": "sum(increase(ds_alert_send_count_total{}[1m]))",
+          "refId": "A"
+        },
+        {
+          "datasource": {
+            "type": "prometheus",
+            "uid": "PBFA97CFB590B2093"
+          },
+          "editorMode": "code",
+          "expr": "sum(increase(ds_alert_send_count_total{status=\"success\"}[1m]))",
+          "hide": false,
+          "legendFormat": "__auto",
+          "range": true,
+          "refId": "B"
+        },
+        {
+          "datasource": {
+            "type": "prometheus",
+            "uid": "PBFA97CFB590B2093"
+          },
+          "expr": "sum(increase(ds_alert_send_count_total{status=\"fail\"}[1m]))",
+          "hide": false,
+          "refId": "C"
+        }
+      ],
+      "title": "Alerts Sent Count/1m",
+      "type": "timeseries"
+    },
+    {
+      "datasource": {
+        "type": "prometheus",
+        "uid": "PBFA97CFB590B2093"
+      },
+      "fieldConfig": {
+        "defaults": {
+          "color": {
+            "mode": "thresholds"
+          },
+          "mappings": [],
+          "thresholds": {
+            "mode": "absolute",
+            "steps": [
+              {
+                "color": "red",
+                "value": null
+              },
+              {
+                "color": "green",
+                "value": 80
+              }
+            ]
+          },
+          "unit": "none"
+        },
+        "overrides": []
+      },
+      "gridPos": {
+        "h": 8,
+        "w": 12,
+        "x": 12,
+        "y": 1
+      },
+      "id": 144,
+      "links": [],
+      "maxDataPoints": 100,
+      "options": {
+        "orientation": "horizontal",
+        "reduceOptions": {
+          "calcs": [
+            "lastNotNull"
+          ],
+          "fields": "",
+          "values": false
+        },
+        "showThresholdLabels": false,
+        "showThresholdMarkers": true,
+        "text": {}
+      },
+      "pluginVersion": "9.0.5",
+      "targets": [
+        {
+          "datasource": {
+            "type": "prometheus",
+            "uid": "PBFA97CFB590B2093"
+          },
+          "editorMode": "builder",
+          "exemplar": true,
+          "expr": "ds_alert_pending",
+          "format": "time_series",
+          "interval": "",
+          "intervalFactor": 1,
+          "legendFormat": "",
+          "metric": "",
+          "range": true,
+          "refId": "A",
+          "step": 14400
+        }
+      ],
+      "title": "Pending Alerts",
+      "type": "gauge"
+    }
+  ],
+  "refresh": "5s",
+  "schemaVersion": 36,
+  "style": "dark",
+  "tags": [],
+  "templating": {
+    "list": [
+      {
+        "current": {
+          "selected": false,
+          "text": "standalone-server",
+          "value": "standalone-server"
+        },
+        "datasource": {
+          "type": "prometheus",
+          "uid": "PBFA97CFB590B2093"
+        },
+        "definition": "",
+        "hide": 0,
+        "includeAll": false,
+        "label": "Application",
+        "multi": false,
+        "name": "application",
+        "options": [],
+        "query": {
+          "query": "label_values(application)",
+          "refId": "Prometheus-application-Variable-Query"
+        },
+        "refresh": 2,
+        "regex": "",
+        "skipUrlSync": false,
+        "sort": 0,
+        "tagValuesQuery": "",
+        "tagsQuery": "",
+        "type": "query",
+        "useTags": false
+      },
+      {
+        "allFormat": "glob",
+        "current": {
+          "selected": false,
+          "text": "host.docker.internal:12345",
+          "value": "host.docker.internal:12345"
+        },
+        "datasource": {
+          "type": "prometheus",
+          "uid": "PBFA97CFB590B2093"
+        },
+        "definition": "",
+        "hide": 0,
+        "includeAll": false,
+        "label": "Instance",
+        "multi": false,
+        "multiFormat": "glob",
+        "name": "instance",
+        "options": [],
+        "query": {
+          "query": "label_values(jvm_memory_used_bytes{application=\"$application\"}, instance)",
+          "refId": "Prometheus-instance-Variable-Query"
+        },
+        "refresh": 2,
+        "regex": "",
+        "skipUrlSync": false,
+        "sort": 0,
+        "tagValuesQuery": "",
+        "tagsQuery": "",
+        "type": "query",
+        "useTags": false
+      },
+      {
+        "allFormat": "glob",
+        "current": {
+          "selected": false,
+          "text": "All",
+          "value": "$__all"
+        },
+        "datasource": {
+          "type": "prometheus",
+          "uid": "PBFA97CFB590B2093"
+        },
+        "definition": "",
+        "hide": 0,
+        "includeAll": true,
+        "label": "JVM Memory Pools Heap",
+        "multi": false,
+        "multiFormat": "glob",
+        "name": "jvm_memory_pool_heap",
+        "options": [],
+        "query": {
+          "query": "label_values(jvm_memory_used_bytes{application=\"$application\", instance=\"$instance\", area=\"heap\"},id)",
+          "refId": "Prometheus-jvm_memory_pool_heap-Variable-Query"
+        },
+        "refresh": 1,
+        "regex": "",
+        "skipUrlSync": false,
+        "sort": 1,
+        "tagValuesQuery": "",
+        "tagsQuery": "",
+        "type": "query",
+        "useTags": false
+      },
+      {
+        "allFormat": "glob",
+        "current": {
+          "selected": false,
+          "text": "All",
+          "value": "$__all"
+        },
+        "datasource": {
+          "type": "prometheus",
+          "uid": "PBFA97CFB590B2093"
+        },
+        "definition": "",
+        "hide": 0,
+        "includeAll": true,
+        "label": "JVM Memory Pools Non-Heap",
+        "multi": false,
+        "multiFormat": "glob",
+        "name": "jvm_memory_pool_nonheap",
+        "options": [],
+        "query": {
+          "query": "label_values(jvm_memory_used_bytes{application=\"$application\", instance=\"$instance\", area=\"nonheap\"},id)",
+          "refId": "Prometheus-jvm_memory_pool_nonheap-Variable-Query"
+        },
+        "refresh": 1,
+        "regex": "",
+        "skipUrlSync": false,
+        "sort": 2,
+        "tagValuesQuery": "",
+        "tagsQuery": "",
+        "type": "query",
+        "useTags": false
+      }
+    ]
+  },
+  "time": {
+    "from": "now-30m",
+    "to": "now"
+  },
+  "timepicker": {
+    "now": true,
+    "refresh_intervals": [
+      "5s",
+      "10s",
+      "30s",
+      "1m",
+      "5m",
+      "15m",
+      "30m",
+      "1h",
+      "2h",
+      "1d"
+    ],
+    "time_options": [
+      "5m",
+      "15m",
+      "1h",
+      "6h",
+      "12h",
+      "24h",
+      "2d",
+      "7d",
+      "30d"
+    ]
+  },
+  "timezone": "browser",
+  "title": "Alert",
+  "uid": "7ZgATOcnz",
+  "version": 3,
+  "weekStart": ""
+}
\ No newline at end of file
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 6ec6a3a3cf..6d3de8e146 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
@@ -17,6 +17,7 @@
 
 package org.apache.dolphinscheduler.server.worker.metrics;
 
+
 import org.apache.dolphinscheduler.plugin.task.api.TaskChannelFactory;
 
 import java.util.HashMap;
@@ -25,15 +26,13 @@ import java.util.ServiceLoader;
 
 import io.micrometer.core.instrument.Counter;
 import io.micrometer.core.instrument.Metrics;
+import lombok.experimental.UtilityClass;
 
-public final class TaskMetrics {
-
-    private TaskMetrics() {
-        throw new UnsupportedOperationException("Utility class");
-    }
+@UtilityClass
+public class TaskMetrics {
 
-    private static Map<String, Counter> TASK_TYPE_EXECUTE_COUNTER = new HashMap<>();
-    private static final Counter UNKNOWN_TASK_EXECUTE_COUNTER =
+    private final Map<String, Counter> taskTypeExecutionCounter = new HashMap<>();
+    private final Counter taskUnknownTypeExecutionCounter =
             Counter.builder("ds.task.execution.count.by.type")
                     .tag("task_type", "unknown")
                     .description("task execution counter by type")
@@ -41,7 +40,7 @@ public final class TaskMetrics {
 
     static {
         for (TaskChannelFactory taskChannelFactory : ServiceLoader.load(TaskChannelFactory.class)) {
-            TASK_TYPE_EXECUTE_COUNTER.put(
+            taskTypeExecutionCounter.put(
                     taskChannelFactory.getName(),
                     Counter.builder("ds.task.execution.count.by.type")
                             .tag("task_type", taskChannelFactory.getName())
@@ -51,8 +50,8 @@ public final class TaskMetrics {
         }
     }
 
-    public static void incrTaskTypeExecuteCount(String taskType) {
-        TASK_TYPE_EXECUTE_COUNTER.getOrDefault(taskType, UNKNOWN_TASK_EXECUTE_COUNTER).increment();
+    public void incrTaskTypeExecuteCount(String taskType) {
+        taskTypeExecutionCounter.getOrDefault(taskType, taskUnknownTypeExecutionCounter).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
index c0ae07269b..abe5f3fbb2 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,36 +30,36 @@ import lombok.experimental.UtilityClass;
 @UtilityClass
 public class WorkerServerMetrics {
 
-    private static final Counter WORKER_OVERLOAD_COUNTER =
+    private final Counter workerOverloadCounter =
         Counter.builder("ds.worker.overload.count")
             .description("overloaded workers count")
             .register(Metrics.globalRegistry);
 
-    private static final Counter WORKER_SUBMIT_QUEUE_IS_FULL_COUNTER =
+    private final Counter workerFullSubmitQueueCounter =
         Counter.builder("ds.worker.full.submit.queue.count")
             .description("full worker submit queues count")
             .register(Metrics.globalRegistry);
 
-    private static final Counter WORKER_RESOURCE_DOWNLOAD_SUCCESS_COUNTER =
+    private final Counter workerResourceDownloadSuccessCounter =
             Counter.builder("ds.worker.resource.download.count")
                     .tag("status", "success")
                     .description("worker resource download success count")
                     .register(Metrics.globalRegistry);
 
-    private static final Counter WORKER_RESOURCE_DOWNLOAD_FAILURE_COUNTER =
+    private final Counter workerResourceDownloadFailCounter =
             Counter.builder("ds.worker.resource.download.count")
                     .tag("status", "fail")
                     .description("worker resource download failure count")
                     .register(Metrics.globalRegistry);
 
-    private static final Timer WORKER_RESOURCE_DOWNLOAD_DURATION_TIMER =
+    private final Timer workerResourceDownloadDurationTimer =
             Timer.builder("ds.worker.resource.download.duration")
                     .publishPercentiles(0.5, 0.75, 0.95, 0.99)
                     .publishPercentileHistogram()
                     .description("time cost of resource download on workers")
                     .register(Metrics.globalRegistry);
 
-    private static final DistributionSummary WORKER_RESOURCE_DOWNLOAD_SIZE_DISTRIBUTION =
+    private final DistributionSummary workerResourceDownloadSizeDistribution =
             DistributionSummary.builder("ds.worker.resource.download.size")
             .baseUnit("bytes")
             .publishPercentiles(0.5, 0.75, 0.95, 0.99)
@@ -67,31 +67,31 @@ public class WorkerServerMetrics {
             .description("size of downloaded resource files on worker")
             .register(Metrics.globalRegistry);
 
-    public static void incWorkerOverloadCount() {
-        WORKER_OVERLOAD_COUNTER.increment();
+    public void incWorkerOverloadCount() {
+        workerOverloadCounter.increment();
     }
 
-    public static void incWorkerSubmitQueueIsFullCount() {
-        WORKER_SUBMIT_QUEUE_IS_FULL_COUNTER.increment();
+    public void incWorkerSubmitQueueIsFullCount() {
+        workerFullSubmitQueueCounter.increment();
     }
 
-    public static void incWorkerResourceDownloadSuccessCount() {
-        WORKER_RESOURCE_DOWNLOAD_SUCCESS_COUNTER.increment();
+    public void incWorkerResourceDownloadSuccessCount() {
+        workerResourceDownloadSuccessCounter.increment();
     }
 
-    public static void incWorkerResourceDownloadFailureCount() {
-        WORKER_RESOURCE_DOWNLOAD_FAILURE_COUNTER.increment();
+    public void incWorkerResourceDownloadFailureCount() {
+        workerResourceDownloadFailCounter.increment();
     }
 
-    public static void recordWorkerResourceDownloadTime(final long milliseconds) {
-        WORKER_RESOURCE_DOWNLOAD_DURATION_TIMER.record(milliseconds, TimeUnit.MILLISECONDS);
+    public void recordWorkerResourceDownloadTime(final long milliseconds) {
+        workerResourceDownloadDurationTimer.record(milliseconds, TimeUnit.MILLISECONDS);
     }
 
-    public static void recordWorkerResourceDownloadSize(final long size) {
-        WORKER_RESOURCE_DOWNLOAD_SIZE_DISTRIBUTION.record(size);
+    public void recordWorkerResourceDownloadSize(final long size) {
+        workerResourceDownloadSizeDistribution.record(size);
     }
 
-    public static void registerWorkerRunningTaskGauge(final Supplier<Number> supplier) {
+    public void registerWorkerRunningTaskGauge(final Supplier<Number> supplier) {
         Gauge.builder("ds.task.running", supplier)
             .description("number of running tasks on workers")
             .register(Metrics.globalRegistry);