You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@inlong.apache.org by go...@apache.org on 2021/12/05 07:09:13 UTC

[incubator-inlong] branch master updated: [INLONG-1872]The Broker of TubeMQ supports metrics with JMX (#1907)

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

gosonzhang pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-inlong.git


The following commit(s) were added to refs/heads/master by this push:
     new 925774f  [INLONG-1872]The Broker of TubeMQ supports metrics with JMX (#1907)
925774f is described below

commit 925774f6c906e82721315f8ffe69d2d1b2dd10f4
Author: gosonzhang <46...@qq.com>
AuthorDate: Sun Dec 5 15:09:05 2021 +0800

    [INLONG-1872]The Broker of TubeMQ supports metrics with JMX (#1907)
---
 .../tubemq/corebase/metric/AbsMetricItem.java      | 77 +++++++++++++++++
 .../tubemq/corebase/metric/CountMetricItem.java    | 36 ++++++++
 .../tubemq/corebase/metric/GaugeMaxMetricItem.java | 39 +++++++++
 .../tubemq/corebase/metric/GaugeMinMetricItem.java | 39 +++++++++
 .../corebase/metric/GaugeNormMetricItem.java       | 36 ++++++++
 .../inlong/tubemq/corebase/metric/MetricType.java  | 49 +++++++++++
 .../inlong/tubemq/corebase/metric/MetricValue.java | 45 ++++++++++
 .../tubemq/corebase/metric/MetricValueType.java    | 49 +++++++++++
 .../tubemq/corebase/metric/MetricValues.java       | 40 +++++++++
 .../tubemq/server/broker/BrokerServiceServer.java  | 18 +++-
 .../inlong/tubemq/server/broker/TubeBroker.java    |  6 ++
 .../server/broker/metrics/BrokerMetricMXBean.java  | 33 ++++++++
 .../server/broker/metrics/BrokerMetrics.java       | 96 ++++++++++++++++++++++
 .../server/broker/metrics/BrokerMetricsHolder.java | 48 +++++++++++
 .../server/broker/msgstore/disk/FileSegment.java   |  5 ++
 .../server/broker/msgstore/disk/MsgFileStore.java  |  4 +
 .../server/broker/msgstore/mem/MsgMemStore.java    |  5 ++
 .../server/broker/offset/DefaultOffsetManager.java |  5 ++
 .../common/offsetstorage/ZkOffsetStorage.java      |  5 ++
 .../tubemq/server/broker/BrokerMetricsTest.java    | 75 +++++++++++++++++
 20 files changed, 706 insertions(+), 4 deletions(-)

diff --git a/inlong-tubemq/tubemq-core/src/main/java/org/apache/inlong/tubemq/corebase/metric/AbsMetricItem.java b/inlong-tubemq/tubemq-core/src/main/java/org/apache/inlong/tubemq/corebase/metric/AbsMetricItem.java
new file mode 100644
index 0000000..3bf0b43
--- /dev/null
+++ b/inlong-tubemq/tubemq-core/src/main/java/org/apache/inlong/tubemq/corebase/metric/AbsMetricItem.java
@@ -0,0 +1,77 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.inlong.tubemq.corebase.metric;
+
+import java.util.concurrent.atomic.AtomicLong;
+import org.apache.inlong.tubemq.corebase.utils.Tuple2;
+
+public abstract class AbsMetricItem {
+
+    protected final MetricType metricType;
+    protected final MetricValueType valueType;
+    protected final String name;
+    protected final AtomicLong value = new AtomicLong(0);
+
+    public AbsMetricItem(MetricType metricType, String name) {
+        this(metricType, MetricValueType.NORMAL, name, 0);
+    }
+
+    public AbsMetricItem(MetricType metricType, MetricValueType valueType,
+                         String name, long initialValue) {
+        this.metricType = metricType;
+        this.valueType = valueType;
+        this.name = name;
+        this.value.set(initialValue);
+    }
+
+    public String getName() {
+        return name;
+    }
+
+    public long getValue() {
+        return value.get();
+    }
+
+    public boolean isCounterMetric() {
+        return metricType == MetricType.COUNTER;
+    }
+
+    public MetricType getMetricType() {
+        return metricType;
+    }
+
+    public MetricValueType getMetricValueType() {
+        return valueType;
+    }
+
+    public Tuple2<String, Long> getNameValue() {
+        return new Tuple2<>(name, value.get());
+    }
+
+    public long incrementAndGet() {
+        return value.incrementAndGet();
+    }
+
+    public long decrementAndGet() {
+        return value.decrementAndGet();
+    }
+
+    public abstract long getAndSet();
+
+    public abstract boolean update(long newValue);
+}
diff --git a/inlong-tubemq/tubemq-core/src/main/java/org/apache/inlong/tubemq/corebase/metric/CountMetricItem.java b/inlong-tubemq/tubemq-core/src/main/java/org/apache/inlong/tubemq/corebase/metric/CountMetricItem.java
new file mode 100644
index 0000000..67f2c62
--- /dev/null
+++ b/inlong-tubemq/tubemq-core/src/main/java/org/apache/inlong/tubemq/corebase/metric/CountMetricItem.java
@@ -0,0 +1,36 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.inlong.tubemq.corebase.metric;
+
+public class CountMetricItem extends AbsMetricItem {
+
+    public CountMetricItem(String name) {
+        super(MetricType.COUNTER, name);
+    }
+
+    @Override
+    public long getAndSet() {
+        return value.getAndSet(0);
+    }
+
+    @Override
+    public boolean update(long newValue) {
+        value.set(newValue);
+        return true;
+    }
+}
diff --git a/inlong-tubemq/tubemq-core/src/main/java/org/apache/inlong/tubemq/corebase/metric/GaugeMaxMetricItem.java b/inlong-tubemq/tubemq-core/src/main/java/org/apache/inlong/tubemq/corebase/metric/GaugeMaxMetricItem.java
new file mode 100644
index 0000000..20d9577
--- /dev/null
+++ b/inlong-tubemq/tubemq-core/src/main/java/org/apache/inlong/tubemq/corebase/metric/GaugeMaxMetricItem.java
@@ -0,0 +1,39 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.inlong.tubemq.corebase.metric;
+
+public class GaugeMaxMetricItem extends AbsMetricItem {
+
+    public GaugeMaxMetricItem(String name) {
+        super(MetricType.GAUGE, MetricValueType.MAX, name, 0);
+    }
+
+    @Override
+    public long getAndSet() {
+        return value.getAndSet(0);
+    }
+
+    @Override
+    public boolean update(long newValue) {
+        long curValue = value.get();
+        if (newValue > curValue) {
+            return value.compareAndSet(curValue, newValue);
+        }
+        return false;
+    }
+}
diff --git a/inlong-tubemq/tubemq-core/src/main/java/org/apache/inlong/tubemq/corebase/metric/GaugeMinMetricItem.java b/inlong-tubemq/tubemq-core/src/main/java/org/apache/inlong/tubemq/corebase/metric/GaugeMinMetricItem.java
new file mode 100644
index 0000000..9ccdcb3
--- /dev/null
+++ b/inlong-tubemq/tubemq-core/src/main/java/org/apache/inlong/tubemq/corebase/metric/GaugeMinMetricItem.java
@@ -0,0 +1,39 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.inlong.tubemq.corebase.metric;
+
+public class GaugeMinMetricItem extends AbsMetricItem {
+
+    public GaugeMinMetricItem(String name) {
+        super(MetricType.GAUGE, MetricValueType.MIN, name, Long.MAX_VALUE);
+    }
+
+    @Override
+    public long getAndSet() {
+        return value.getAndSet(Long.MAX_VALUE);
+    }
+
+    @Override
+    public boolean update(long newValue) {
+        long curValue = value.get();
+        if (newValue < curValue) {
+            return value.compareAndSet(curValue, newValue);
+        }
+        return false;
+    }
+}
diff --git a/inlong-tubemq/tubemq-core/src/main/java/org/apache/inlong/tubemq/corebase/metric/GaugeNormMetricItem.java b/inlong-tubemq/tubemq-core/src/main/java/org/apache/inlong/tubemq/corebase/metric/GaugeNormMetricItem.java
new file mode 100644
index 0000000..2005bab
--- /dev/null
+++ b/inlong-tubemq/tubemq-core/src/main/java/org/apache/inlong/tubemq/corebase/metric/GaugeNormMetricItem.java
@@ -0,0 +1,36 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.inlong.tubemq.corebase.metric;
+
+public class GaugeNormMetricItem extends AbsMetricItem {
+
+    public GaugeNormMetricItem(String name) {
+        super(MetricType.GAUGE, MetricValueType.MIN, name, 0);
+    }
+
+    @Override
+    public long getAndSet() {
+        return value.get();
+    }
+
+    @Override
+    public boolean update(long newValue) {
+        value.set(newValue);
+        return true;
+    }
+}
diff --git a/inlong-tubemq/tubemq-core/src/main/java/org/apache/inlong/tubemq/corebase/metric/MetricType.java b/inlong-tubemq/tubemq-core/src/main/java/org/apache/inlong/tubemq/corebase/metric/MetricType.java
new file mode 100644
index 0000000..e17cbfc
--- /dev/null
+++ b/inlong-tubemq/tubemq-core/src/main/java/org/apache/inlong/tubemq/corebase/metric/MetricType.java
@@ -0,0 +1,49 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.inlong.tubemq.corebase.metric;
+
+public enum MetricType {
+    UNKNOWN(0, "Unknown"),
+    COUNTER(1, "Counter"),
+    GAUGE(2, "Gauge");
+
+    MetricType(int id, String name) {
+        this.id = id;
+        this.name = name;
+    }
+
+    public int getId() {
+        return id;
+    }
+
+    public String getName() {
+        return name;
+    }
+
+    public static MetricType valueOf(int value) {
+        for (MetricType metricType : MetricType.values()) {
+            if (metricType.getId() == value) {
+                return metricType;
+            }
+        }
+        return UNKNOWN;
+    }
+
+    private final int id;
+    private final String name;
+}
diff --git a/inlong-tubemq/tubemq-core/src/main/java/org/apache/inlong/tubemq/corebase/metric/MetricValue.java b/inlong-tubemq/tubemq-core/src/main/java/org/apache/inlong/tubemq/corebase/metric/MetricValue.java
new file mode 100644
index 0000000..4ff4aa4
--- /dev/null
+++ b/inlong-tubemq/tubemq-core/src/main/java/org/apache/inlong/tubemq/corebase/metric/MetricValue.java
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.inlong.tubemq.corebase.metric;
+
+import java.beans.ConstructorProperties;
+
+public class MetricValue {
+    private final String type;
+    private final String name;
+    private final long value;
+
+    @ConstructorProperties({"type", "name", "value"})
+    public MetricValue(String type, String name, long value) {
+        this.name = name;
+        this.type = type;
+        this.value = value;
+    }
+
+    public String getName() {
+        return name;
+    }
+
+    public String getType() {
+        return type;
+    }
+
+    public long getValue() {
+        return value;
+    }
+}
diff --git a/inlong-tubemq/tubemq-core/src/main/java/org/apache/inlong/tubemq/corebase/metric/MetricValueType.java b/inlong-tubemq/tubemq-core/src/main/java/org/apache/inlong/tubemq/corebase/metric/MetricValueType.java
new file mode 100644
index 0000000..64d972c
--- /dev/null
+++ b/inlong-tubemq/tubemq-core/src/main/java/org/apache/inlong/tubemq/corebase/metric/MetricValueType.java
@@ -0,0 +1,49 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.inlong.tubemq.corebase.metric;
+
+public enum MetricValueType {
+    NORMAL(0, "Normal"),
+    MIN(1, "Min"),
+    MAX(2, "Max");
+
+    MetricValueType(int id, String name) {
+        this.id = id;
+        this.name = name;
+    }
+
+    public int getId() {
+        return id;
+    }
+
+    public String getName() {
+        return name;
+    }
+
+    public static MetricValueType valueOf(int value) {
+        for (MetricValueType valueType : MetricValueType.values()) {
+            if (valueType.getId() == value) {
+                return valueType;
+            }
+        }
+        return NORMAL;
+    }
+
+    private final int id;
+    private final String name;
+}
diff --git a/inlong-tubemq/tubemq-core/src/main/java/org/apache/inlong/tubemq/corebase/metric/MetricValues.java b/inlong-tubemq/tubemq-core/src/main/java/org/apache/inlong/tubemq/corebase/metric/MetricValues.java
new file mode 100644
index 0000000..34c771b
--- /dev/null
+++ b/inlong-tubemq/tubemq-core/src/main/java/org/apache/inlong/tubemq/corebase/metric/MetricValues.java
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.inlong.tubemq.corebase.metric;
+
+import java.beans.ConstructorProperties;
+import java.util.Map;
+
+public class MetricValues {
+    private final String lastResetTime;
+    private final Map<String, Long> metricValues;
+
+    @ConstructorProperties({"lastResetTime", "metricValues"})
+    public MetricValues(String lastResetTime, Map<String, Long> metricValues) {
+        this.lastResetTime = lastResetTime;
+        this.metricValues = metricValues;
+    }
+
+    public String getLastResetTime() {
+        return lastResetTime;
+    }
+
+    public Map<String, Long> getMetricValues() {
+        return metricValues;
+    }
+}
diff --git a/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/broker/BrokerServiceServer.java b/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/broker/BrokerServiceServer.java
index beca6c2..6fe76f4 100644
--- a/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/broker/BrokerServiceServer.java
+++ b/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/broker/BrokerServiceServer.java
@@ -58,6 +58,7 @@ import org.apache.inlong.tubemq.corerpc.service.BrokerWriteService;
 import org.apache.inlong.tubemq.server.Server;
 import org.apache.inlong.tubemq.server.broker.metadata.MetadataManager;
 import org.apache.inlong.tubemq.server.broker.metadata.TopicMetadata;
+import org.apache.inlong.tubemq.server.broker.metrics.BrokerMetricsHolder;
 import org.apache.inlong.tubemq.server.broker.msgstore.MessageStore;
 import org.apache.inlong.tubemq.server.broker.msgstore.MessageStoreManager;
 import org.apache.inlong.tubemq.server.broker.msgstore.disk.GetMessageResult;
@@ -843,7 +844,9 @@ public class BrokerServiceServer implements BrokerReadService, BrokerWriteServic
                     ? request.getQryPriorityId() : TBaseConstants.META_VALUE_UNDEFINED;
             consumerNodeInfo = new ConsumerNodeInfo(storeManager, reqQryPriorityId,
                     clientId, filterCondSet, reqSessionKey, reqSessionTime, true, partStr);
-            consumerRegisterMap.put(partStr, consumerNodeInfo);
+            if (consumerRegisterMap.put(partStr, consumerNodeInfo) == null) {
+                BrokerMetricsHolder.METRICS.consumerOnlineCnt.incrementAndGet();
+            }
             heartbeatManager.regConsumerNode(getHeartbeatNodeId(clientId, partStr), clientId, partStr);
             MessageStore dataStore = null;
             try {
@@ -887,7 +890,9 @@ public class BrokerServiceServer implements BrokerReadService, BrokerWriteServic
             TimeoutInfo timeoutInfo =
                     heartbeatManager.getConsumerRegMap().get(getHeartbeatNodeId(consumerId, partStr));
             if (timeoutInfo == null || System.currentTimeMillis() >= timeoutInfo.getTimeoutTime()) {
-                consumerRegisterMap.remove(partStr);
+                if (consumerRegisterMap.remove(partStr) != null) {
+                    BrokerMetricsHolder.METRICS.consumerOnlineCnt.decrementAndGet();
+                }
                 strBuffer.append("[Duplicated Register] Remove Invalid Consumer Register ")
                         .append(consumerId).append(TokenConstants.SEGMENT_SEP).append(partStr);
             } else {
@@ -951,7 +956,9 @@ public class BrokerServiceServer implements BrokerReadService, BrokerWriteServic
                     .append(groupName).append(" topic:").append(topicName).append(" partition:")
                     .append(request.getPartitionId()).append(" updatedOffset:").append(updatedOffset).toString());
             strBuffer.delete(0, strBuffer.length());
-            consumerRegisterMap.remove(partStr);
+            if (consumerRegisterMap.remove(partStr) != null) {
+                BrokerMetricsHolder.METRICS.consumerOnlineCnt.decrementAndGet();
+            }
             heartbeatManager.unRegConsumerNode(
                     getHeartbeatNodeId(clientId, partStr));
         } catch (Exception e) {
@@ -1237,7 +1244,10 @@ public class BrokerServiceServer implements BrokerReadService, BrokerWriteServic
                         return;
                     }
                     if (consumerNodeInfo.getConsumerId().equalsIgnoreCase(nodeInfo.getSecondKey())) {
-                        consumerRegisterMap.remove(nodeInfo.getThirdKey());
+                        if (consumerRegisterMap.remove(nodeInfo.getThirdKey()) != null) {
+                            BrokerMetricsHolder.METRICS.consumerOnlineCnt.decrementAndGet();
+                            BrokerMetricsHolder.METRICS.consumerTmoTotCnt.decrementAndGet();
+                        }
                         String[] groupTopicPart =
                                 consumerNodeInfo.getPartStr().split(TokenConstants.ATTR_SEP);
                         long updatedOffset =
diff --git a/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/broker/TubeBroker.java b/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/broker/TubeBroker.java
index cbf8767..474ddfb 100644
--- a/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/broker/TubeBroker.java
+++ b/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/broker/TubeBroker.java
@@ -47,6 +47,7 @@ import org.apache.inlong.tubemq.server.broker.exception.StartupException;
 import org.apache.inlong.tubemq.server.broker.metadata.BrokerMetadataManager;
 import org.apache.inlong.tubemq.server.broker.metadata.ClusterConfigHolder;
 import org.apache.inlong.tubemq.server.broker.metadata.MetadataManager;
+import org.apache.inlong.tubemq.server.broker.metrics.BrokerMetricsHolder;
 import org.apache.inlong.tubemq.server.broker.msgstore.MessageStoreManager;
 import org.apache.inlong.tubemq.server.broker.nodeinfo.ConsumerNodeInfo;
 import org.apache.inlong.tubemq.server.broker.offset.DefaultOffsetManager;
@@ -104,6 +105,8 @@ public class TubeBroker implements Stoppable {
         java.security.Security.setProperty("networkaddress.cache.negative.ttl", "1");
         this.tubeConfig = tubeConfig;
         this.brokerId = generateBrokerClientId();
+        // register metric bean
+        BrokerMetricsHolder.registerMXBean();
         this.metadataManager = new BrokerMetadataManager();
         this.offsetManager = new DefaultOffsetManager(tubeConfig);
         this.storeManager = new MessageStoreManager(this, tubeConfig);
@@ -215,6 +218,8 @@ public class TubeBroker implements Stoppable {
                             if (!response.getSuccess()) {
                                 isKeepAlive.set(false);
                                 if (response.getErrCode() == TErrCodeConstants.HB_NO_NODE) {
+                                    BrokerMetricsHolder.METRICS
+                                            .masterNoNodeCnt.incrementAndGet();
                                     register2Master();
                                     heartbeatErrors.set(0);
                                     logger.info("Re-register to master successfully!");
@@ -229,6 +234,7 @@ public class TubeBroker implements Stoppable {
                             isKeepAlive.set(false);
                             heartbeatErrors.incrementAndGet();
                             samplePrintCtrl.printExceptionCaught(t);
+                            BrokerMetricsHolder.METRICS.hbExceptionCnt.incrementAndGet();
                         }
                     }
                 }
diff --git a/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/broker/metrics/BrokerMetricMXBean.java b/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/broker/metrics/BrokerMetricMXBean.java
new file mode 100644
index 0000000..f8f0c38
--- /dev/null
+++ b/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/broker/metrics/BrokerMetricMXBean.java
@@ -0,0 +1,33 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.inlong.tubemq.server.broker.metrics;
+
+import org.apache.inlong.tubemq.corebase.metric.MetricValues;
+
+/**
+ * BrokerMonitorMXBean
+ * Provide access interface of a metric item with JMX.<br>
+ * Decouple between metric item and monitor system, in particular scene, <br>
+ * inlong can depend on user-defined monitor system.
+ */
+public interface BrokerMetricMXBean {
+
+    MetricValues getMetrics();
+
+    MetricValues getAndReSetMetrics();
+}
diff --git a/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/broker/metrics/BrokerMetrics.java b/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/broker/metrics/BrokerMetrics.java
new file mode 100644
index 0000000..f7588c8
--- /dev/null
+++ b/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/broker/metrics/BrokerMetrics.java
@@ -0,0 +1,96 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.inlong.tubemq.server.broker.metrics;
+
+import java.util.Date;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicLong;
+import org.apache.inlong.tubemq.corebase.metric.AbsMetricItem;
+import org.apache.inlong.tubemq.corebase.metric.CountMetricItem;
+import org.apache.inlong.tubemq.corebase.metric.GaugeMaxMetricItem;
+import org.apache.inlong.tubemq.corebase.metric.GaugeMinMetricItem;
+import org.apache.inlong.tubemq.corebase.metric.GaugeNormMetricItem;
+import org.apache.inlong.tubemq.corebase.metric.MetricValues;
+import org.apache.inlong.tubemq.server.common.utils.WebParameterUtils;
+
+public class BrokerMetrics implements BrokerMetricMXBean {
+
+    private final AtomicLong lastResetTime =
+            new AtomicLong(System.currentTimeMillis());
+    public final AbsMetricItem syncDataDurMin =
+            new GaugeMinMetricItem("fSync_latency_min");
+    public final AbsMetricItem syncDataDurMax =
+            new GaugeMaxMetricItem("fSync_latency_max");
+    public final AbsMetricItem syncZkDurMin =
+            new GaugeMinMetricItem("zkSync_latency_min");
+    public final AbsMetricItem syncZkDurMax =
+            new GaugeMaxMetricItem("zkSync_latency_max");
+    public final AbsMetricItem zkExceptionCnt =
+            new CountMetricItem("zk_exception_cnt");
+    public final AbsMetricItem masterNoNodeCnt =
+            new CountMetricItem("online_timeout_cnt");
+    public final AbsMetricItem hbExceptionCnt =
+            new CountMetricItem("hb_master_exception_cnt");
+    public final AbsMetricItem ioExceptionCnt =
+            new CountMetricItem("io_exception_cnt");
+    public final AbsMetricItem consumerOnlineCnt =
+            new GaugeNormMetricItem("consumer_online_cnt");
+    public final AbsMetricItem consumerTmoTotCnt =
+            new CountMetricItem("consumer_timeout_cnt");
+
+    public BrokerMetrics() {
+        this.lastResetTime.set(System.currentTimeMillis());
+    }
+
+    @Override
+    public MetricValues getMetrics() {
+        Map<String, Long> metricValues = new HashMap<>();
+        metricValues.put(syncDataDurMin.getName(), syncDataDurMin.getValue());
+        metricValues.put(syncDataDurMax.getName(), syncDataDurMax.getValue());
+        metricValues.put(syncZkDurMin.getName(), syncZkDurMin.getValue());
+        metricValues.put(syncZkDurMax.getName(), syncZkDurMax.getValue());
+        metricValues.put(zkExceptionCnt.getName(), zkExceptionCnt.getValue());
+        metricValues.put(masterNoNodeCnt.getName(), masterNoNodeCnt.getValue());
+        metricValues.put(hbExceptionCnt.getName(), hbExceptionCnt.getValue());
+        metricValues.put(ioExceptionCnt.getName(), ioExceptionCnt.getValue());
+        metricValues.put(consumerOnlineCnt.getName(), consumerOnlineCnt.getValue());
+        metricValues.put(consumerTmoTotCnt.getName(), consumerTmoTotCnt.getValue());
+        return new MetricValues(WebParameterUtils.date2yyyyMMddHHmmss(
+                new Date(lastResetTime.get())), metricValues);
+    }
+
+    @Override
+    public MetricValues getAndReSetMetrics() {
+        Map<String, Long> metricValues = new HashMap<>();
+        metricValues.put(syncDataDurMin.getName(), syncDataDurMin.getAndSet());
+        metricValues.put(syncDataDurMax.getName(), syncDataDurMax.getAndSet());
+        metricValues.put(syncZkDurMin.getName(), syncZkDurMin.getAndSet());
+        metricValues.put(syncZkDurMax.getName(), syncZkDurMax.getAndSet());
+        metricValues.put(zkExceptionCnt.getName(), zkExceptionCnt.getAndSet());
+        metricValues.put(masterNoNodeCnt.getName(), masterNoNodeCnt.getAndSet());
+        metricValues.put(hbExceptionCnt.getName(), hbExceptionCnt.getAndSet());
+        metricValues.put(ioExceptionCnt.getName(), ioExceptionCnt.getAndSet());
+        metricValues.put(consumerOnlineCnt.getName(), consumerOnlineCnt.getAndSet());
+        metricValues.put(consumerTmoTotCnt.getName(), consumerTmoTotCnt.getAndSet());
+        long befTime = lastResetTime.getAndSet(System.currentTimeMillis());
+        return new MetricValues(
+                WebParameterUtils.date2yyyyMMddHHmmss(new Date(befTime)), metricValues);
+    }
+}
+
diff --git a/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/broker/metrics/BrokerMetricsHolder.java b/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/broker/metrics/BrokerMetricsHolder.java
new file mode 100644
index 0000000..845f14d
--- /dev/null
+++ b/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/broker/metrics/BrokerMetricsHolder.java
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.inlong.tubemq.server.broker.metrics;
+
+import java.lang.management.ManagementFactory;
+import java.util.concurrent.atomic.AtomicBoolean;
+import javax.management.MBeanServer;
+import javax.management.ObjectName;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class BrokerMetricsHolder {
+    private static final Logger logger =
+            LoggerFactory.getLogger(BrokerMetricsHolder.class);
+
+    private static final AtomicBoolean registered = new AtomicBoolean(false);
+    public static final BrokerMetrics METRICS = new BrokerMetrics();
+
+    public static void registerMXBean() {
+        if (!registered.compareAndSet(false, true)) {
+            return;
+        }
+        try {
+            MBeanServer mbs = ManagementFactory.getPlatformMBeanServer();
+            ObjectName mxBeanName =
+                    new ObjectName("org.apache.inlong.tubemq.server.broker:type=brokerMetrics");
+            mbs.registerMBean(METRICS, mxBeanName);
+        } catch (Exception ex) {
+            logger.error("Register BrokerMXBean error: ", ex);
+        }
+    }
+}
+
diff --git a/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/broker/msgstore/disk/FileSegment.java b/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/broker/msgstore/disk/FileSegment.java
index 54c0b47..a6202cf 100644
--- a/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/broker/msgstore/disk/FileSegment.java
+++ b/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/broker/msgstore/disk/FileSegment.java
@@ -26,6 +26,7 @@ import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicLong;
 import org.apache.inlong.tubemq.corebase.utils.CheckSum;
 import org.apache.inlong.tubemq.corebase.utils.ServiceStatusHolder;
+import org.apache.inlong.tubemq.server.broker.metrics.BrokerMetricsHolder;
 import org.apache.inlong.tubemq.server.broker.utils.DataStoreUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -114,6 +115,7 @@ public class FileSegment implements Segment {
             } catch (final Exception e) {
                 if (e instanceof IOException) {
                     ServiceStatusHolder.addReadIOErrCnt();
+                    BrokerMetricsHolder.METRICS.ioExceptionCnt.incrementAndGet();
                 }
                 if (this.segmentType == SegmentType.DATA) {
                     logger.error("[File Store] Set DATA Segment cachedSize error", e);
@@ -138,6 +140,7 @@ public class FileSegment implements Segment {
             } catch (Throwable ee) {
                 if (ee instanceof IOException) {
                     ServiceStatusHolder.addReadIOErrCnt();
+                    BrokerMetricsHolder.METRICS.ioExceptionCnt.incrementAndGet();
                 }
                 logger.error(new StringBuilder(512).append("[File Store] Close ")
                         .append(this.file.getAbsoluteFile().toString())
@@ -160,6 +163,7 @@ public class FileSegment implements Segment {
         } catch (Throwable e1) {
             if (e1 instanceof IOException) {
                 ServiceStatusHolder.addReadIOErrCnt();
+                BrokerMetricsHolder.METRICS.ioExceptionCnt.incrementAndGet();
             }
             logger.error("[File Store] failure to close channel ", e1);
         }
@@ -171,6 +175,7 @@ public class FileSegment implements Segment {
         } catch (Throwable ee) {
             if (ee instanceof IOException) {
                 ServiceStatusHolder.addReadIOErrCnt();
+                BrokerMetricsHolder.METRICS.ioExceptionCnt.incrementAndGet();
             }
             logger.error("[File Store] failure to delete file ", ee);
         }
diff --git a/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/broker/msgstore/disk/MsgFileStore.java b/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/broker/msgstore/disk/MsgFileStore.java
index 0de1588..2b116dd 100644
--- a/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/broker/msgstore/disk/MsgFileStore.java
+++ b/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/broker/msgstore/disk/MsgFileStore.java
@@ -35,6 +35,7 @@ import org.apache.inlong.tubemq.corebase.TErrCodeConstants;
 import org.apache.inlong.tubemq.corebase.protobuf.generated.ClientBroker;
 import org.apache.inlong.tubemq.corebase.utils.ServiceStatusHolder;
 import org.apache.inlong.tubemq.server.broker.BrokerConfig;
+import org.apache.inlong.tubemq.server.broker.metrics.BrokerMetricsHolder;
 import org.apache.inlong.tubemq.server.broker.msgstore.MessageStore;
 import org.apache.inlong.tubemq.server.broker.stats.CountItem;
 import org.apache.inlong.tubemq.server.broker.utils.DataStoreUtils;
@@ -196,6 +197,7 @@ public class MsgFileStore implements Closeable {
             // print abnormal information
             if (inIndexOffset != indexOffset || inDataOffset != dataOffset) {
                 ServiceStatusHolder.addWriteIOErrCnt();
+                BrokerMetricsHolder.METRICS.ioExceptionCnt.incrementAndGet();
                 logger.error(sb.append("[File Store]: appendMsg data Error, storekey=")
                     .append(this.storeKey).append(",msgCnt=").append(msgCnt)
                     .append(",indexSize=").append(indexSize)
@@ -209,6 +211,7 @@ public class MsgFileStore implements Closeable {
         } catch (Throwable e) {
             if (!closed.get()) {
                 ServiceStatusHolder.addWriteIOErrCnt();
+                BrokerMetricsHolder.METRICS.ioExceptionCnt.incrementAndGet();
             }
             samplePrintCtrl.printExceptionCaught(e);
         } finally {
@@ -324,6 +327,7 @@ public class MsgFileStore implements Closeable {
             } catch (Throwable e2) {
                 if (e2 instanceof IOException) {
                     ServiceStatusHolder.addReadIOErrCnt();
+                    BrokerMetricsHolder.METRICS.ioExceptionCnt.incrementAndGet();
                 }
                 samplePrintCtrl.printExceptionCaught(e2,
                     messageStore.getStoreKey(), String.valueOf(partitionId));
diff --git a/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/broker/msgstore/mem/MsgMemStore.java b/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/broker/msgstore/mem/MsgMemStore.java
index afac6f3..f37dfa2 100644
--- a/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/broker/msgstore/mem/MsgMemStore.java
+++ b/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/broker/msgstore/mem/MsgMemStore.java
@@ -30,6 +30,7 @@ import org.apache.inlong.tubemq.corebase.TBaseConstants;
 import org.apache.inlong.tubemq.corebase.TErrCodeConstants;
 import org.apache.inlong.tubemq.server.broker.BrokerConfig;
 import org.apache.inlong.tubemq.server.broker.metadata.ClusterConfigHolder;
+import org.apache.inlong.tubemq.server.broker.metrics.BrokerMetricsHolder;
 import org.apache.inlong.tubemq.server.broker.msgstore.disk.MsgFileStore;
 import org.apache.inlong.tubemq.server.broker.utils.DataStoreUtils;
 import org.apache.inlong.tubemq.server.common.utils.AppendResult;
@@ -271,8 +272,12 @@ public class MsgMemStore implements Closeable {
         final ByteBuffer tmpDataReadBuf = this.cacheDataSegment.asReadOnlyBuffer();
         tmpIndexBuffer.flip();
         tmpDataReadBuf.flip();
+        long tmpValue = System.currentTimeMillis();
         msgFileStore.batchAppendMsg(strBuffer, curMessageCount.get(),
             cacheIndexOffset.get(), tmpIndexBuffer, cacheDataOffset.get(), tmpDataReadBuf);
+        long dltTime = System.currentTimeMillis() - tmpValue;
+        BrokerMetricsHolder.METRICS.syncDataDurMin.update(dltTime);
+        BrokerMetricsHolder.METRICS.syncDataDurMax.update(dltTime);
         return true;
     }
 
diff --git a/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/broker/offset/DefaultOffsetManager.java b/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/broker/offset/DefaultOffsetManager.java
index 897ba3d..a4f1b7a 100644
--- a/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/broker/offset/DefaultOffsetManager.java
+++ b/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/broker/offset/DefaultOffsetManager.java
@@ -31,6 +31,7 @@ import org.apache.inlong.tubemq.corebase.utils.TStringUtils;
 import org.apache.inlong.tubemq.corebase.utils.Tuple2;
 import org.apache.inlong.tubemq.corebase.utils.Tuple3;
 import org.apache.inlong.tubemq.server.broker.BrokerConfig;
+import org.apache.inlong.tubemq.server.broker.metrics.BrokerMetricsHolder;
 import org.apache.inlong.tubemq.server.broker.msgstore.MessageStore;
 import org.apache.inlong.tubemq.server.broker.utils.DataStoreUtils;
 import org.apache.inlong.tubemq.server.common.offsetstorage.OffsetStorage;
@@ -613,6 +614,7 @@ public class DefaultOffsetManager extends AbstractDaemonService implements Offse
     }
 
     private void commitCfmOffsets(boolean retryable) {
+        long tmpValue = System.currentTimeMillis();
         for (Map.Entry<String, ConcurrentHashMap<String, OffsetStorageInfo>> entry : cfmOffsetMap.entrySet()) {
             if (TStringUtils.isBlank(entry.getKey())
                     || entry.getValue() == null || entry.getValue().isEmpty()) {
@@ -620,6 +622,9 @@ public class DefaultOffsetManager extends AbstractDaemonService implements Offse
             }
             zkOffsetStorage.commitOffset(entry.getKey(), entry.getValue().values(), retryable);
         }
+        long dltTime = System.currentTimeMillis() - tmpValue;
+        BrokerMetricsHolder.METRICS.syncZkDurMin.update(dltTime);
+        BrokerMetricsHolder.METRICS.syncZkDurMax.update(dltTime);
     }
 
     /***
diff --git a/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/common/offsetstorage/ZkOffsetStorage.java b/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/common/offsetstorage/ZkOffsetStorage.java
index f199d01..30eb1d1 100644
--- a/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/common/offsetstorage/ZkOffsetStorage.java
+++ b/inlong-tubemq/tubemq-server/src/main/java/org/apache/inlong/tubemq/server/common/offsetstorage/ZkOffsetStorage.java
@@ -27,6 +27,7 @@ import java.util.Set;
 
 import org.apache.inlong.tubemq.corebase.TokenConstants;
 import org.apache.inlong.tubemq.server.broker.exception.OffsetStoreException;
+import org.apache.inlong.tubemq.server.broker.metrics.BrokerMetricsHolder;
 import org.apache.inlong.tubemq.server.common.TServerConstants;
 import org.apache.inlong.tubemq.server.common.fileconfig.ZKConfig;
 import org.apache.inlong.tubemq.server.common.offsetstorage.zookeeper.ZKUtil;
@@ -78,6 +79,7 @@ public class ZkOffsetStorage implements OffsetStorage {
         try {
             this.zkw = new ZooKeeperWatcher(zkConfig);
         } catch (Throwable e) {
+            BrokerMetricsHolder.METRICS.zkExceptionCnt.incrementAndGet();
             logger.error(new StringBuilder(256)
                     .append("[ZkOffsetStorage] Failed to connect ZooKeeper server (")
                     .append(this.zkConfig.getZkServerAddr()).append(") !").toString(), e);
@@ -141,6 +143,7 @@ public class ZkOffsetStorage implements OffsetStorage {
         try {
             offsetZkInfo = ZKUtil.readDataMaybeNull(this.zkw, znode);
         } catch (KeeperException e) {
+            BrokerMetricsHolder.METRICS.zkExceptionCnt.incrementAndGet();
             logger.error("KeeperException during load offsets from ZooKeeper", e);
             return null;
         }
@@ -180,6 +183,7 @@ public class ZkOffsetStorage implements OffsetStorage {
             try {
                 ZKUtil.updatePersistentPath(this.zkw, offsetPath, offsetData);
             } catch (final Throwable t) {
+                BrokerMetricsHolder.METRICS.zkExceptionCnt.incrementAndGet();
                 logger.error("Exception during commit offsets to ZooKeeper", t);
                 throw new OffsetStoreException(t);
             }
@@ -220,6 +224,7 @@ public class ZkOffsetStorage implements OffsetStorage {
                     offsetMap.put(partitionId, Long.parseLong(offsetInfoStrs[1]));
                 }
             } catch (Throwable e) {
+                BrokerMetricsHolder.METRICS.zkExceptionCnt.incrementAndGet();
                 offsetMap.put(partitionId, null);
             }
         }
diff --git a/inlong-tubemq/tubemq-server/src/test/java/org/apache/inlong/tubemq/server/broker/BrokerMetricsTest.java b/inlong-tubemq/tubemq-server/src/test/java/org/apache/inlong/tubemq/server/broker/BrokerMetricsTest.java
new file mode 100644
index 0000000..be7640c
--- /dev/null
+++ b/inlong-tubemq/tubemq-server/src/test/java/org/apache/inlong/tubemq/server/broker/BrokerMetricsTest.java
@@ -0,0 +1,75 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.inlong.tubemq.server.broker;
+
+import org.apache.inlong.tubemq.corebase.metric.MetricValues;
+import org.apache.inlong.tubemq.server.broker.metrics.BrokerMetrics;
+import org.junit.Assert;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class BrokerMetricsTest {
+    private static final Logger logger =
+            LoggerFactory.getLogger(BrokerMetricsTest.class);
+
+    @Test
+    public void testAgentMetrics() {
+        try {
+            BrokerMetrics metrics = new BrokerMetrics();
+            metrics.zkExceptionCnt.incrementAndGet();
+            metrics.consumerTmoTotCnt.incrementAndGet();
+            metrics.syncDataDurMax.update(10000);
+            metrics.syncDataDurMin.update(2000);
+            metrics.syncDataDurMax.update(20000);
+            metrics.syncDataDurMin.update(1000);
+            metrics.syncDataDurMin.update(3000);
+            metrics.syncDataDurMax.update(30000);
+            MetricValues result1 = metrics.getMetrics();
+            Assert.assertEquals(Long.valueOf(1000),
+                    result1.getMetricValues().get(metrics.syncDataDurMin.getName()));
+            Assert.assertEquals(Long.valueOf(30000),
+                    result1.getMetricValues().get(metrics.syncDataDurMax.getName()));
+            Assert.assertEquals(Long.valueOf(1),
+                    result1.getMetricValues().get(metrics.zkExceptionCnt.getName()));
+            Assert.assertEquals(Long.valueOf(1),
+                    result1.getMetricValues().get(metrics.consumerTmoTotCnt.getName()));
+            // get and reset value
+            final MetricValues result2 = metrics.getAndReSetMetrics();
+            metrics.zkExceptionCnt.incrementAndGet();
+            metrics.zkExceptionCnt.getAndSet();
+            metrics.consumerTmoTotCnt.incrementAndGet();
+            metrics.consumerTmoTotCnt.update(10);
+            metrics.syncDataDurMax.update(20000);
+            metrics.syncDataDurMin.update(2000);
+            MetricValues result3 = metrics.getMetrics();
+            Assert.assertEquals(result1.getLastResetTime(),
+                    result2.getLastResetTime());
+            Assert.assertEquals(Long.valueOf(2000),
+                    result3.getMetricValues().get(metrics.syncDataDurMin.getName()));
+            Assert.assertEquals(Long.valueOf(20000),
+                    result3.getMetricValues().get(metrics.syncDataDurMax.getName()));
+            Assert.assertEquals(Long.valueOf(0),
+                    result3.getMetricValues().get(metrics.zkExceptionCnt.getName()));
+            Assert.assertEquals(Long.valueOf(10),
+                    result3.getMetricValues().get(metrics.consumerTmoTotCnt.getName()));
+        } catch (Exception ex) {
+            logger.error("error happens" + ex);
+        }
+    }
+}