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

ningjiang pushed a commit to branch master
in repository 
https://gitbox.apache.org/repos/asf/incubator-servicecomb-java-chassis.git

commit c205240cd9cc784868da2bd1874dc2ee3f4d745e
Author: zhengyangyong <yangyong.zh...@huawei.com>
AuthorDate: Fri Feb 23 20:08:56 2018 +0800

    SCB-327 refactor metrics simplify monitors in order to improve implement
    
    Signed-off-by: zhengyangyong <yangyong.zh...@huawei.com>
---
 .../common/rest/AbstractRestInvocation.java        |   2 +-
 .../org/apache/servicecomb/core/Invocation.java    |  16 +-
 .../core/metrics/InvocationFinishedEvent.java      |  13 +-
 .../metrics/InvocationStartExecutionEvent.java     |  14 +-
 .../metrics/InvocationStartProcessingEvent.java    |  41 -----
 .../core/metrics/InvocationStartedEvent.java       |   3 +-
 .../demo/springmvc/client/SpringmvcClient.java     |   4 +-
 .../org/apache/servicecomb/demo/perf/PerfMain.java |   3 +-
 .../demo/perf/PerfMetricsFilePublisher.java        |  63 +++----
 .../foundation/common/event/EventBus.java          |  15 +-
 .../foundation/common/event/EventListener.java     |   7 +-
 .../foundation/common/utils/EventUtils.java        |  11 +-
 .../foundation/common/event/TestEventBus.java      |  26 +--
 .../foundation/metrics/MetricsConst.java           |   8 +-
 .../foundation/metrics/publish/Metric.java         |  29 ++-
 .../foundation/metrics/publish/MetricNode.java     |  29 ++-
 .../foundation/metrics/publish/MetricsLoader.java  |   6 +-
 .../foundation/metrics/publish/TestMetricNode.java |  65 +++++++
 .../metrics/publish/TestMetricsLoader.java         |   6 +-
 .../servicecomb/metrics/core/MetricsConfig.java    |   2 +-
 .../metrics/core/MetricsDataSource.java            | 137 --------------
 .../servicecomb/metrics/core/MonitorManager.java   | 181 +++++++++++++++++++
 .../servicecomb/metrics/core/SystemMetrics.java    |  83 +++++++++
 .../event/InvocationFinishedEventListener.java     |  58 +++---
 ... => InvocationStartExecutionEventListener.java} |  23 +--
 .../InvocationStartProcessingEventListener.java    |  48 -----
 .../core/event/InvocationStartedEventListener.java |  31 ++--
 .../core/health/DefaultHealthCheckExtraData.java   |  69 -------
 .../health/DefaultMicroserviceHealthChecker.java   |  63 -------
 .../metrics/core/monitor/CallMonitor.java          |  92 ----------
 .../core/monitor/ConsumerInvocationMonitor.java    |  49 -----
 .../metrics/core/monitor/DefaultSystemMonitor.java | 126 -------------
 .../core/monitor/ProducerInvocationMonitor.java    |  84 ---------
 .../metrics/core/monitor/RegistryMonitor.java      |  72 --------
 .../metrics/core/monitor/SystemMonitor.java        |  44 -----
 .../metrics/core/monitor/TimerMonitor.java         | 108 -----------
 .../core/publish/HealthCheckerPublisher.java       |  26 ++-
 .../metrics/core/publish/MetricsPublisher.java     |  21 ++-
 .../metrics/core/utils/MonitorUtils.java           |  53 ------
 ....servicecomb.metrics.core.monitor.SystemMonitor |  18 --
 .../metrics/core/TestEventAndRunner.java           | 199 +++++++++------------
 .../metrics/core/TestHealthCheckerPublisher.java   |  86 +++------
 .../metrics/core/TestMetricsPublisher.java         |   2 +-
 .../metrics/prometheus/MetricsCollector.java       |   7 +-
 .../samples/mwf/WriteFileInitializer.java          |  11 +-
 tracing/tracing-zipkin/pom.xml                     |   1 +
 .../transport/highway/HighwayServerInvoke.java     |   2 +-
 47 files changed, 672 insertions(+), 1385 deletions(-)

diff --git 
a/common/common-rest/src/main/java/org/apache/servicecomb/common/rest/AbstractRestInvocation.java
 
b/common/common-rest/src/main/java/org/apache/servicecomb/common/rest/AbstractRestInvocation.java
index 3cef720..4e1ce24 100644
--- 
a/common/common-rest/src/main/java/org/apache/servicecomb/common/rest/AbstractRestInvocation.java
+++ 
b/common/common-rest/src/main/java/org/apache/servicecomb/common/rest/AbstractRestInvocation.java
@@ -145,7 +145,7 @@ public abstract class AbstractRestInvocation {
 
     //立刻设置开始时间,否则Finished时无法计算TotalTime
     invocation.setStartTime(startedEvent.getStartedTime());
-    invocation.triggerStartProcessingEvent();
+    invocation.triggerStartExecutionEvent();
 
     invoke();
   }
diff --git a/core/src/main/java/org/apache/servicecomb/core/Invocation.java 
b/core/src/main/java/org/apache/servicecomb/core/Invocation.java
index 882b9e6..6db9279 100644
--- a/core/src/main/java/org/apache/servicecomb/core/Invocation.java
+++ b/core/src/main/java/org/apache/servicecomb/core/Invocation.java
@@ -25,7 +25,7 @@ import java.util.concurrent.Executor;
 import org.apache.servicecomb.core.definition.OperationMeta;
 import org.apache.servicecomb.core.definition.SchemaMeta;
 import org.apache.servicecomb.core.metrics.InvocationFinishedEvent;
-import org.apache.servicecomb.core.metrics.InvocationStartProcessingEvent;
+import org.apache.servicecomb.core.metrics.InvocationStartExecutionEvent;
 import org.apache.servicecomb.core.provider.consumer.ReferenceConfig;
 import org.apache.servicecomb.foundation.common.utils.EventUtils;
 import org.apache.servicecomb.swagger.invocation.AsyncResponse;
@@ -60,7 +60,7 @@ public class Invocation extends SwaggerInvocation {
 
   private long startTime;
 
-  private long startProcessingTime;
+  private long startExecutionTime;
 
   private boolean sync = true;
 
@@ -185,18 +185,18 @@ public class Invocation extends SwaggerInvocation {
     return operationMeta.getMicroserviceQualifiedName();
   }
 
-  public void triggerStartProcessingEvent() {
-    this.startProcessingTime = System.nanoTime();
-    EventUtils.triggerEvent(new InvocationStartProcessingEvent(
-        operationMeta.getMicroserviceQualifiedName(), this.invocationType));
+  public void triggerStartExecutionEvent() {
+    if (InvocationType.PRODUCER.equals(invocationType)) {
+      this.startExecutionTime = System.nanoTime();
+      EventUtils.triggerEvent(new 
InvocationStartExecutionEvent(operationMeta.getMicroserviceQualifiedName()));
+    }
   }
 
   public void triggerFinishedEvent(int statusCode) {
     long finishedTime = System.nanoTime();
     EventUtils
         .triggerEvent(new 
InvocationFinishedEvent(operationMeta.getMicroserviceQualifiedName(), 
this.invocationType,
-            startProcessingTime - startTime,
-            finishedTime - startProcessingTime,
+            startExecutionTime - startTime, finishedTime - startExecutionTime,
             finishedTime - startTime, statusCode));
   }
 
diff --git 
a/core/src/main/java/org/apache/servicecomb/core/metrics/InvocationFinishedEvent.java
 
b/core/src/main/java/org/apache/servicecomb/core/metrics/InvocationFinishedEvent.java
index ee7c684..e0a10d1 100644
--- 
a/core/src/main/java/org/apache/servicecomb/core/metrics/InvocationFinishedEvent.java
+++ 
b/core/src/main/java/org/apache/servicecomb/core/metrics/InvocationFinishedEvent.java
@@ -17,17 +17,16 @@
 
 package org.apache.servicecomb.core.metrics;
 
-import org.apache.servicecomb.foundation.common.event.Event;
 import org.apache.servicecomb.swagger.invocation.InvocationType;
 
-public class InvocationFinishedEvent implements Event {
+public class InvocationFinishedEvent {
   private final String operationName;
 
   private final InvocationType invocationType;
 
   private final long inQueueNanoTime;
 
-  private final long processElapsedNanoTime;
+  private final long executionElapsedNanoTime;
 
   private final long totalElapsedNanoTime;
 
@@ -45,8 +44,8 @@ public class InvocationFinishedEvent implements Event {
     return inQueueNanoTime;
   }
 
-  public long getProcessElapsedNanoTime() {
-    return processElapsedNanoTime;
+  public long getExecutionElapsedNanoTime() {
+    return executionElapsedNanoTime;
   }
 
   public long getTotalElapsedNanoTime() {
@@ -58,11 +57,11 @@ public class InvocationFinishedEvent implements Event {
   }
 
   public InvocationFinishedEvent(String operationName, InvocationType 
invocationType,
-      long inQueueNanoTime, long processElapsedNanoTime, long 
totalElapsedNanoTime, int statusCode) {
+      long inQueueNanoTime, long executionElapsedNanoTime, long 
totalElapsedNanoTime, int statusCode) {
     this.operationName = operationName;
     this.invocationType = invocationType;
     this.inQueueNanoTime = inQueueNanoTime;
-    this.processElapsedNanoTime = processElapsedNanoTime;
+    this.executionElapsedNanoTime = executionElapsedNanoTime;
     this.totalElapsedNanoTime = totalElapsedNanoTime;
     this.statusCode = statusCode;
   }
diff --git 
a/foundations/foundation-common/src/main/java/org/apache/servicecomb/foundation/common/event/Event.java
 
b/core/src/main/java/org/apache/servicecomb/core/metrics/InvocationStartExecutionEvent.java
similarity index 72%
rename from 
foundations/foundation-common/src/main/java/org/apache/servicecomb/foundation/common/event/Event.java
rename to 
core/src/main/java/org/apache/servicecomb/core/metrics/InvocationStartExecutionEvent.java
index d089f01..2d7c5a9 100644
--- 
a/foundations/foundation-common/src/main/java/org/apache/servicecomb/foundation/common/event/Event.java
+++ 
b/core/src/main/java/org/apache/servicecomb/core/metrics/InvocationStartExecutionEvent.java
@@ -15,8 +15,16 @@
  * limitations under the License.
  */
 
-package org.apache.servicecomb.foundation.common.event;
+package org.apache.servicecomb.core.metrics;
 
-//Common event interface for event extension
-public interface Event {
+public class InvocationStartExecutionEvent {
+  private final String operationName;
+
+  public String getOperationName() {
+    return operationName;
+  }
+
+  public InvocationStartExecutionEvent(String operationName) {
+    this.operationName = operationName;
+  }
 }
diff --git 
a/core/src/main/java/org/apache/servicecomb/core/metrics/InvocationStartProcessingEvent.java
 
b/core/src/main/java/org/apache/servicecomb/core/metrics/InvocationStartProcessingEvent.java
deleted file mode 100644
index 3e7ed6b..0000000
--- 
a/core/src/main/java/org/apache/servicecomb/core/metrics/InvocationStartProcessingEvent.java
+++ /dev/null
@@ -1,41 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.servicecomb.core.metrics;
-
-import org.apache.servicecomb.foundation.common.event.Event;
-import org.apache.servicecomb.swagger.invocation.InvocationType;
-
-public class InvocationStartProcessingEvent implements Event {
-  private final String operationName;
-
-  private final InvocationType invocationType;
-
-  public String getOperationName() {
-    return operationName;
-  }
-
-  public InvocationType getInvocationType() {
-    return invocationType;
-  }
-
-
-  public InvocationStartProcessingEvent(String operationName, InvocationType 
invocationType) {
-    this.operationName = operationName;
-    this.invocationType = invocationType;
-  }
-}
diff --git 
a/core/src/main/java/org/apache/servicecomb/core/metrics/InvocationStartedEvent.java
 
b/core/src/main/java/org/apache/servicecomb/core/metrics/InvocationStartedEvent.java
index 4bc4a8f..9ce7759 100644
--- 
a/core/src/main/java/org/apache/servicecomb/core/metrics/InvocationStartedEvent.java
+++ 
b/core/src/main/java/org/apache/servicecomb/core/metrics/InvocationStartedEvent.java
@@ -17,10 +17,9 @@
 
 package org.apache.servicecomb.core.metrics;
 
-import org.apache.servicecomb.foundation.common.event.Event;
 import org.apache.servicecomb.swagger.invocation.InvocationType;
 
-public class InvocationStartedEvent implements Event {
+public class InvocationStartedEvent {
   private final String operationName;
 
   private final InvocationType invocationType;
diff --git 
a/demo/demo-springmvc/springmvc-client/src/main/java/org/apache/servicecomb/demo/springmvc/client/SpringmvcClient.java
 
b/demo/demo-springmvc/springmvc-client/src/main/java/org/apache/servicecomb/demo/springmvc/client/SpringmvcClient.java
index 5298f16..6c938e5 100644
--- 
a/demo/demo-springmvc/springmvc-client/src/main/java/org/apache/servicecomb/demo/springmvc/client/SpringmvcClient.java
+++ 
b/demo/demo-springmvc/springmvc-client/src/main/java/org/apache/servicecomb/demo/springmvc/client/SpringmvcClient.java
@@ -101,10 +101,10 @@ public class SpringmvcClient {
       Map<String, Double> metrics = restTemplate.getForObject(prefix + 
"/metrics", Map.class);
 
       TestMgr
-          .check(true, metrics.get("jvm(statistic=gauge,name=heapUsed)") != 0);
+          .check(true, metrics.get("jvm(name=heapUsed,statistic=gauge)") != 0);
       TestMgr.check(true, metrics.size() > 0);
       TestMgr.check(true, metrics.get(
-          
"servicecomb.invocation(operation=springmvc.codeFirst.saySomething,role=producer,stage=whole,statistic=count,status=200)")
+          
"servicecomb.invocation(operation=springmvc.codeFirst.saySomething,role=PRODUCER,stage=total,statistic=count,status=200)")
           >= 0);
     } catch (Exception e) {
       TestMgr.check("true", "false");
diff --git 
a/demo/perf/src/main/java/org/apache/servicecomb/demo/perf/PerfMain.java 
b/demo/perf/src/main/java/org/apache/servicecomb/demo/perf/PerfMain.java
index 16a0bed..c067577 100644
--- a/demo/perf/src/main/java/org/apache/servicecomb/demo/perf/PerfMain.java
+++ b/demo/perf/src/main/java/org/apache/servicecomb/demo/perf/PerfMain.java
@@ -24,7 +24,6 @@ import java.util.concurrent.TimeUnit;
 
 import org.apache.servicecomb.foundation.common.utils.BeanUtils;
 import org.apache.servicecomb.foundation.vertx.VertxUtils;
-import org.apache.servicecomb.metrics.core.MetricsDataSource;
 
 public class PerfMain {
 
@@ -36,7 +35,7 @@ public class PerfMain {
 
     // metrics
     //DataSource dataSource = BeanUtils.getContext().getBean(Def.class);
-    PerfMetricsFilePublisher metricsLog = new 
PerfMetricsFilePublisher(MetricsDataSource.getInstance());
+    PerfMetricsFilePublisher metricsLog = new PerfMetricsFilePublisher();
     
Executors.newScheduledThreadPool(1).scheduleWithFixedDelay(metricsLog::onCycle, 
0, 1, TimeUnit.SECONDS);
 
     List<String> argList = Arrays.asList(args);
diff --git 
a/demo/perf/src/main/java/org/apache/servicecomb/demo/perf/PerfMetricsFilePublisher.java
 
b/demo/perf/src/main/java/org/apache/servicecomb/demo/perf/PerfMetricsFilePublisher.java
index f17b04d..2c6fc42 100644
--- 
a/demo/perf/src/main/java/org/apache/servicecomb/demo/perf/PerfMetricsFilePublisher.java
+++ 
b/demo/perf/src/main/java/org/apache/servicecomb/demo/perf/PerfMetricsFilePublisher.java
@@ -18,30 +18,25 @@ package org.apache.servicecomb.demo.perf;
 
 import java.util.Map;
 import java.util.Map.Entry;
+import java.util.concurrent.TimeUnit;
 
+import 
org.apache.servicecomb.foundation.common.exceptions.ServiceCombException;
 import org.apache.servicecomb.foundation.metrics.MetricsConst;
 import org.apache.servicecomb.foundation.metrics.publish.MetricNode;
 import org.apache.servicecomb.foundation.metrics.publish.MetricsLoader;
 import org.apache.servicecomb.foundation.vertx.VertxUtils;
-import org.apache.servicecomb.metrics.core.MetricsDataSource;
+import org.apache.servicecomb.metrics.core.MonitorManager;
+import org.apache.servicecomb.swagger.invocation.InvocationType;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import com.google.common.collect.Lists;
-
 import io.vertx.core.impl.VertxImplEx;
 
 public class PerfMetricsFilePublisher {
   private static final Logger LOGGER = 
LoggerFactory.getLogger(PerfMetricsFilePublisher.class);
 
-  private MetricsDataSource dataSource;
-
-  public PerfMetricsFilePublisher(MetricsDataSource dataSource) {
-    this.dataSource = dataSource;
-  }
-
   public void onCycle() {
-    Map<String, Double> metrics = 
dataSource.measure(dataSource.getAppliedWindowTime().get(0), true);
+    Map<String, Double> metrics = MonitorManager.getInstance().measure();
     MetricsLoader loader = new MetricsLoader(metrics);
 
     StringBuilder sb = new StringBuilder();
@@ -75,12 +70,19 @@ public class PerfMetricsFilePublisher {
   }
 
   private void collectMetrics(MetricsLoader loader, StringBuilder sb) {
-    MetricNode treeNode = loader
-        .getMetricTree(MetricsConst.SERVICECOMB_INVOCATION, 
MetricsConst.TAG_ROLE, MetricsConst.TAG_OPERATION,
-            MetricsConst.TAG_STATUS);
+    MetricNode treeNode;
+    try {
+      treeNode = loader
+          .getMetricTree(MetricsConst.SERVICECOMB_INVOCATION, 
MetricsConst.TAG_ROLE, MetricsConst.TAG_OPERATION,
+              MetricsConst.TAG_STATUS);
+    }
+    //before receive any request,there are no 
MetricsConst.SERVICECOMB_INVOCATION,so getMetricTree will throw 
ServiceCombException
+    catch (ServiceCombException ignored) {
+      return;
+    }
 
     if (treeNode != null && treeNode.getChildren().size() != 0) {
-      MetricNode consumerNode = 
treeNode.getChildren().get(MetricsConst.ROLE_CONSUMER);
+      MetricNode consumerNode = 
treeNode.getChildren().get(String.valueOf(InvocationType.CONSUMER));
       if (consumerNode != null) {
         sb.append("consumer:\n");
         sb.append("  tps     latency(ms) status  operation\n");
@@ -88,18 +90,17 @@ public class PerfMetricsFilePublisher {
           for (Entry<String, MetricNode> statusNode : 
operationNode.getValue().getChildren().entrySet()) {
             sb.append(String.format("  %-7.0f %-11.3f %-9s %s\n",
                 statusNode.getValue()
-                    
.getFirstMatchMetricValue(Lists.newArrayList(MetricsConst.TAG_STAGE, 
MetricsConst.TAG_STATISTIC),
-                        Lists.newArrayList(MetricsConst.STAGE_WHOLE, "tps")),
+                    .getFirstMatchMetricValue(MetricsConst.TAG_STAGE, 
MetricsConst.STAGE_TOTAL,
+                        MetricsConst.TAG_STATISTIC, "tps"),
                 statusNode.getValue()
-                    
.getFirstMatchMetricValue(Lists.newArrayList(MetricsConst.TAG_STAGE, 
MetricsConst.TAG_STATISTIC),
-                        Lists.newArrayList(MetricsConst.STAGE_WHOLE, 
"latency")),
-                statusNode.getKey(),
-                operationNode.getKey()));
+                    .getFirstMatchMetricValue(TimeUnit.MILLISECONDS, 
MetricsConst.TAG_STAGE, MetricsConst.STAGE_TOTAL,
+                        MetricsConst.TAG_STATISTIC, "latency"),
+                statusNode.getKey(), operationNode.getKey()));
           }
         }
       }
 
-      MetricNode producerNode = 
treeNode.getChildren().get(MetricsConst.ROLE_PRODUCER);
+      MetricNode producerNode = 
treeNode.getChildren().get(String.valueOf(InvocationType.PRODUCER));
       if (producerNode != null) {
         sb.append("producer:\n");
         sb.append("  tps     latency(ms) queue(ms) execute(ms) status  
operation\n");
@@ -107,19 +108,19 @@ public class PerfMetricsFilePublisher {
           for (Entry<String, MetricNode> statusNode : 
operationNode.getValue().getChildren().entrySet()) {
             sb.append(String.format("  %-7.0f %-11.3f %-9.3f %-11.3f %-7s 
%s\n",
                 statusNode.getValue()
-                    
.getFirstMatchMetricValue(Lists.newArrayList(MetricsConst.TAG_STAGE, 
MetricsConst.TAG_STATISTIC),
-                        Lists.newArrayList(MetricsConst.STAGE_WHOLE, "tps")),
+                    .getFirstMatchMetricValue(MetricsConst.TAG_STAGE, 
MetricsConst.STAGE_TOTAL,
+                        MetricsConst.TAG_STATISTIC, "tps"),
                 statusNode.getValue()
-                    
.getFirstMatchMetricValue(Lists.newArrayList(MetricsConst.TAG_STAGE, 
MetricsConst.TAG_STATISTIC),
-                        Lists.newArrayList(MetricsConst.STAGE_WHOLE, 
"latency")),
+                    .getFirstMatchMetricValue(TimeUnit.MILLISECONDS, 
MetricsConst.TAG_STAGE, MetricsConst.STAGE_TOTAL,
+                        MetricsConst.TAG_STATISTIC, "latency"),
                 statusNode.getValue()
-                    
.getFirstMatchMetricValue(Lists.newArrayList(MetricsConst.TAG_STAGE, 
MetricsConst.TAG_STATISTIC),
-                        Lists.newArrayList(MetricsConst.STAGE_QUEUE, 
"latency")),
+                    .getFirstMatchMetricValue(TimeUnit.MILLISECONDS, 
MetricsConst.TAG_STAGE, MetricsConst.STAGE_QUEUE,
+                        MetricsConst.TAG_STATISTIC, "latency"),
                 statusNode.getValue()
-                    
.getFirstMatchMetricValue(Lists.newArrayList(MetricsConst.TAG_STAGE, 
MetricsConst.TAG_STATISTIC),
-                        Lists.newArrayList(MetricsConst.STAGE_EXECUTION, 
"latency")),
-                statusNode.getKey(),
-                operationNode.getKey()));
+                    .getFirstMatchMetricValue(TimeUnit.MILLISECONDS, 
MetricsConst.TAG_STAGE,
+                        MetricsConst.STAGE_EXECUTION,
+                        MetricsConst.TAG_STATISTIC, "latency"),
+                statusNode.getKey(), operationNode.getKey()));
           }
         }
       }
diff --git 
a/foundations/foundation-common/src/main/java/org/apache/servicecomb/foundation/common/event/EventBus.java
 
b/foundations/foundation-common/src/main/java/org/apache/servicecomb/foundation/common/event/EventBus.java
index 4305ace..c76b179 100644
--- 
a/foundations/foundation-common/src/main/java/org/apache/servicecomb/foundation/common/event/EventBus.java
+++ 
b/foundations/foundation-common/src/main/java/org/apache/servicecomb/foundation/common/event/EventBus.java
@@ -20,27 +20,28 @@ package org.apache.servicecomb.foundation.common.event;
 import java.util.Collections;
 import java.util.List;
 import java.util.Map;
-import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.CopyOnWriteArrayList;
 
+import org.apache.servicecomb.foundation.common.concurrent.ConcurrentHashMapEx;
+
 public class EventBus {
-  private final Map<Class<? extends Event>, List<EventListener>> 
allEventListeners = new ConcurrentHashMap<>();
+  private final Map<Class, List<EventListener>> allEventListeners = new 
ConcurrentHashMapEx<>();
 
-  public void registerEventListener(EventListener eventListener) {
+  public <T> void registerEventListener(Class<T> cls, EventListener<T> 
eventListener) {
     List<EventListener> eventListeners = allEventListeners
-        .computeIfAbsent(eventListener.getConcernedEvent(), f -> new 
CopyOnWriteArrayList<>());
+        .computeIfAbsent(cls, f -> new CopyOnWriteArrayList<>());
     eventListeners.add(eventListener);
   }
 
-  public void unregisterEventListener(EventListener eventListener) {
+  public <T> void unregisterEventListener(Class<T> cls, EventListener<T> 
eventListener) {
     List<EventListener> eventListeners = allEventListeners
-        .computeIfAbsent(eventListener.getConcernedEvent(), f -> new 
CopyOnWriteArrayList<>());
+        .computeIfAbsent(cls, f -> new CopyOnWriteArrayList<>());
     if (eventListeners.contains(eventListener)) {
       eventListeners.remove(eventListener);
     }
   }
 
-  public void triggerEvent(Event event) {
+  public <T> void triggerEvent(T event) {
     List<EventListener> eventListeners = 
allEventListeners.getOrDefault(event.getClass(), Collections.emptyList());
     for (EventListener eventListener : eventListeners) {
       eventListener.process(event);
diff --git 
a/foundations/foundation-common/src/main/java/org/apache/servicecomb/foundation/common/event/EventListener.java
 
b/foundations/foundation-common/src/main/java/org/apache/servicecomb/foundation/common/event/EventListener.java
index 5f208d6..475ed2d 100644
--- 
a/foundations/foundation-common/src/main/java/org/apache/servicecomb/foundation/common/event/EventListener.java
+++ 
b/foundations/foundation-common/src/main/java/org/apache/servicecomb/foundation/common/event/EventListener.java
@@ -18,11 +18,8 @@
 package org.apache.servicecomb.foundation.common.event;
 
 //Common event listener interface,java chassis component can trigger event let 
high level component perceive data change.
-public interface EventListener {
-
-  //what is type event this listener concerned
-  Class<? extends Event> getConcernedEvent();
+public interface EventListener<T> {
 
   //process event data
-  void process(Event data);
+  void process(T data);
 }
diff --git 
a/foundations/foundation-common/src/main/java/org/apache/servicecomb/foundation/common/utils/EventUtils.java
 
b/foundations/foundation-common/src/main/java/org/apache/servicecomb/foundation/common/utils/EventUtils.java
index 731d8ff..26902db 100644
--- 
a/foundations/foundation-common/src/main/java/org/apache/servicecomb/foundation/common/utils/EventUtils.java
+++ 
b/foundations/foundation-common/src/main/java/org/apache/servicecomb/foundation/common/utils/EventUtils.java
@@ -17,22 +17,21 @@
 
 package org.apache.servicecomb.foundation.common.utils;
 
-import org.apache.servicecomb.foundation.common.event.Event;
 import org.apache.servicecomb.foundation.common.event.EventBus;
 import org.apache.servicecomb.foundation.common.event.EventListener;
 
 public final class EventUtils {
   private static final EventBus eventBus = new EventBus();
 
-  public static void registerEventListener(EventListener eventListener) {
-    eventBus.registerEventListener(eventListener);
+  public static <T> void registerEventListener(Class<T> cls, EventListener<T> 
eventListener) {
+    eventBus.registerEventListener(cls, eventListener);
   }
 
-  public static void unregisterEventListener(EventListener eventListener) {
-    eventBus.unregisterEventListener(eventListener);
+  public static <T> void unregisterEventListener(Class<T> cls, 
EventListener<T> eventListener) {
+    eventBus.unregisterEventListener(cls, eventListener);
   }
 
-  public static void triggerEvent(Event event) {
+  public static <T> void triggerEvent(T event) {
     eventBus.triggerEvent(event);
   }
 }
diff --git 
a/foundations/foundation-common/src/test/java/org/apache/servicecomb/foundation/common/event/TestEventBus.java
 
b/foundations/foundation-common/src/test/java/org/apache/servicecomb/foundation/common/event/TestEventBus.java
index 28b9575..14b508a 100644
--- 
a/foundations/foundation-common/src/test/java/org/apache/servicecomb/foundation/common/event/TestEventBus.java
+++ 
b/foundations/foundation-common/src/test/java/org/apache/servicecomb/foundation/common/event/TestEventBus.java
@@ -32,32 +32,20 @@ public class TestEventBus {
   public void test() throws InterruptedException {
     AtomicBoolean eventReceived = new AtomicBoolean(false);
 
-    EventListener listener = new EventListener() {
-      @Override
-      public Class<? extends Event> getConcernedEvent() {
-        return TestEvent.class;
-      }
-
-      @Override
-      public void process(Event data) {
-        eventReceived.set(true);
-      }
-    };
-
-    EventUtils.registerEventListener(listener);
-    EventUtils.triggerEvent(new TestEvent());
+    EventListener<String> listener = data -> eventReceived.set(true);
+
+    EventUtils.registerEventListener(String.class, listener);
+
+    EventUtils.triggerEvent("xxx");
     await().atMost(1, TimeUnit.SECONDS)
         .until(eventReceived::get);
     Assert.assertTrue(eventReceived.get());
 
     eventReceived.set(false);
 
-    EventUtils.unregisterEventListener(listener);
-    EventUtils.triggerEvent(new TestEvent());
+    EventUtils.unregisterEventListener(String.class, listener);
+    EventUtils.triggerEvent("xxx");
     Thread.sleep(1000);
     Assert.assertFalse(eventReceived.get());
   }
-
-  private class TestEvent implements Event {
-  }
 }
diff --git 
a/foundations/foundation-metrics/src/main/java/org/apache/servicecomb/foundation/metrics/MetricsConst.java
 
b/foundations/foundation-metrics/src/main/java/org/apache/servicecomb/foundation/metrics/MetricsConst.java
index 7bb0198..fe82da4 100644
--- 
a/foundations/foundation-metrics/src/main/java/org/apache/servicecomb/foundation/metrics/MetricsConst.java
+++ 
b/foundations/foundation-metrics/src/main/java/org/apache/servicecomb/foundation/metrics/MetricsConst.java
@@ -34,13 +34,11 @@ public class MetricsConst {
 
   public static final String TAG_STATISTIC = "statistic";
 
-  public static final String STAGE_WHOLE = "whole";
+  public static final String TAG_UNIT = "unit";
+
+  public static final String STAGE_TOTAL = "total";
 
   public static final String STAGE_QUEUE = "queue";
 
   public static final String STAGE_EXECUTION = "execution";
-
-  public static final String ROLE_CONSUMER = "consumer";
-
-  public static final String ROLE_PRODUCER = "producer";
 }
diff --git 
a/foundations/foundation-metrics/src/main/java/org/apache/servicecomb/foundation/metrics/publish/Metric.java
 
b/foundations/foundation-metrics/src/main/java/org/apache/servicecomb/foundation/metrics/publish/Metric.java
index 3a01644..5402054 100644
--- 
a/foundations/foundation-metrics/src/main/java/org/apache/servicecomb/foundation/metrics/publish/Metric.java
+++ 
b/foundations/foundation-metrics/src/main/java/org/apache/servicecomb/foundation/metrics/publish/Metric.java
@@ -18,8 +18,10 @@
 package org.apache.servicecomb.foundation.metrics.publish;
 
 import java.util.HashMap;
-import java.util.List;
 import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.servicecomb.foundation.metrics.MetricsConst;
 
 public class Metric {
   private final String name;
@@ -40,6 +42,15 @@ public class Metric {
     return value;
   }
 
+  public double getValue(TimeUnit unit) {
+    if (tags.containsKey(MetricsConst.TAG_UNIT)) {
+      if (!tags.get(MetricsConst.TAG_UNIT).equals(String.valueOf(unit))) {
+        return unit.convert((long) value, 
TimeUnit.valueOf(tags.get(MetricsConst.TAG_UNIT)));
+      }
+    }
+    return value;
+  }
+
   public Metric(String id, double value) {
     String[] nameAndTag = id.split("\\(");
     this.tags = new HashMap<>();
@@ -51,16 +62,16 @@ public class Metric {
     this.value = value;
   }
 
-  public boolean containTag(List<String> tagKeys, List<String> tagValues) {
-    for (int i = 0; i < tagKeys.size(); i++) {
-      if (!containTag(tagKeys.get(i), tagValues.get(i))) {
+  public boolean containTag(String tagKey, String tagValue) {
+    return tags.containsKey(tagKey) && tagValue.equals(tags.get(tagKey));
+  }
+
+  public boolean containTag(String... tags) {
+    for (int i = 0; i < tags.length; i += 2) {
+      if (!containTag(tags[i], tags[i + 1])) {
         return false;
       }
     }
     return true;
   }
-
-  public boolean containTag(String tagKey, String tagValue) {
-    return tags.containsKey(tagKey) && tagValue.equals(tags.get(tagKey));
-  }
-}
+}
\ No newline at end of file
diff --git 
a/foundations/foundation-metrics/src/main/java/org/apache/servicecomb/foundation/metrics/publish/MetricNode.java
 
b/foundations/foundation-metrics/src/main/java/org/apache/servicecomb/foundation/metrics/publish/MetricNode.java
index c23d9e2..6042de6 100644
--- 
a/foundations/foundation-metrics/src/main/java/org/apache/servicecomb/foundation/metrics/publish/MetricNode.java
+++ 
b/foundations/foundation-metrics/src/main/java/org/apache/servicecomb/foundation/metrics/publish/MetricNode.java
@@ -23,6 +23,7 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
+import java.util.concurrent.TimeUnit;
 
 import org.apache.servicecomb.foundation.metrics.MetricsConst;
 
@@ -54,19 +55,41 @@ public class MetricNode {
     return Double.NaN;
   }
 
-  public Double getFirstMatchMetricValue(List<String> tagKeys, List<String> 
tagValues) {
+  public Double getFirstMatchMetricValue(TimeUnit unit, String tagKey, String 
tagValue) {
     for (Metric metric : this.metrics) {
-      if (metric.containTag(tagKeys, tagValues)) {
+      if (metric.containTag(tagKey, tagValue)) {
+        return metric.getValue(unit);
+      }
+    }
+    return Double.NaN;
+  }
+
+  public Double getFirstMatchMetricValue(String... tags) {
+    for (Metric metric : this.metrics) {
+      if (metric.containTag(tags)) {
         return metric.getValue();
       }
     }
     return Double.NaN;
   }
 
+  public Double getFirstMatchMetricValue(TimeUnit unit, String... tags) {
+    for (Metric metric : this.metrics) {
+      if (metric.containTag(tags)) {
+        return metric.getValue(unit);
+      }
+    }
+    return Double.NaN;
+  }
+
   public double getMatchStatisticMetricValue(String statisticValue) {
     return getFirstMatchMetricValue(MetricsConst.TAG_STATISTIC, 
statisticValue);
   }
 
+  public double getMatchStatisticMetricValue(TimeUnit unit, String 
statisticValue) {
+    return getFirstMatchMetricValue(unit, MetricsConst.TAG_STATISTIC, 
statisticValue);
+  }
+
   public MetricNode(List<Metric> metrics, String... groupTagKeys) {
     if (groupTagKeys == null || groupTagKeys.length == 0) {
       this.tagKey = null;
@@ -105,4 +128,4 @@ public class MetricNode {
     }
     return groups;
   }
-}
+}
\ No newline at end of file
diff --git 
a/foundations/foundation-metrics/src/main/java/org/apache/servicecomb/foundation/metrics/publish/MetricsLoader.java
 
b/foundations/foundation-metrics/src/main/java/org/apache/servicecomb/foundation/metrics/publish/MetricsLoader.java
index 073eaf6..2aac335 100644
--- 
a/foundations/foundation-metrics/src/main/java/org/apache/servicecomb/foundation/metrics/publish/MetricsLoader.java
+++ 
b/foundations/foundation-metrics/src/main/java/org/apache/servicecomb/foundation/metrics/publish/MetricsLoader.java
@@ -23,6 +23,8 @@ import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
 
+import 
org.apache.servicecomb.foundation.common.exceptions.ServiceCombException;
+
 //load origin metrics value and publish tree
 public class MetricsLoader {
 
@@ -40,7 +42,7 @@ public class MetricsLoader {
     if (metrics.containsKey(id)) {
       return new MetricNode(metrics.get(id), groupTagKeys);
     }
-    return null;
+    throw new ServiceCombException("no such id : " + id);
   }
 
   public double getFirstMatchMetricValue(String name, String tagKey, String 
tagValue) {
@@ -53,4 +55,4 @@ public class MetricsLoader {
     }
     return Double.NaN;
   }
-}
+}
\ No newline at end of file
diff --git 
a/foundations/foundation-metrics/src/test/java/org/apache/servicecomb/foundation/metrics/publish/TestMetricNode.java
 
b/foundations/foundation-metrics/src/test/java/org/apache/servicecomb/foundation/metrics/publish/TestMetricNode.java
new file mode 100644
index 0000000..543d697
--- /dev/null
+++ 
b/foundations/foundation-metrics/src/test/java/org/apache/servicecomb/foundation/metrics/publish/TestMetricNode.java
@@ -0,0 +1,65 @@
+/*
+ * 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.servicecomb.foundation.metrics.publish;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestMetricNode {
+  @Test
+  public void test() {
+    Map<String, Double> metrics = new HashMap<>();
+    metrics.put("X(K1=1,K2=2,K3=3,unit=SECONDS,statistic=A)", 100.0);
+    metrics.put("X(K1=1,K2=2,K3=30000,unit=SECONDS,statistic=AA)", 110.0);
+    metrics.put("X(K1=1,K2=20,K3=30,unit=SECONDS,statistic=B)", 200.0);
+    metrics.put("X(K1=2,K2=200,K3=300,unit=SECONDS,statistic=C)", 300.0);
+    metrics.put("X(K1=2,K2=2000,K3=3000,unit=SECONDS,statistic=D)", 400.0);
+
+    metrics.put("Y(K1=1,K2=2,K3=3)", 500.0);
+    metrics.put("Y(K1=10,K2=20,K3=30)", 600.0);
+    metrics.put("Y(K1=100,K2=200,K3=300)", 700.0);
+    metrics.put("Y(K1=1000,K2=2000,K3=3000)", 800.0);
+
+    MetricsLoader loader = new MetricsLoader(metrics);
+
+    MetricNode node = loader.getMetricTree("X", "K1");
+
+    MetricNode node_k1 = node.getChildrenNode("1");
+    Assert.assertEquals(3, node_k1.getMetrics().size());
+    Assert.assertEquals(100, node_k1.getFirstMatchMetricValue("K2", "2"), 0);
+    Assert.assertEquals(100 * 1000, 
node_k1.getFirstMatchMetricValue(TimeUnit.MILLISECONDS, "K2", "2"), 0);
+    Assert.assertEquals(100 * 1000, 
node_k1.getFirstMatchMetricValue(TimeUnit.MILLISECONDS, "K2", "2"), 0);
+
+    Assert.assertEquals(200, node_k1.getFirstMatchMetricValue("K3", "30", 
"K2", "20"), 0);
+    Assert.assertEquals(200 * 1000, 
node_k1.getFirstMatchMetricValue(TimeUnit.MILLISECONDS, "K3", "30", "K2", 
"20"), 0);
+    Assert.assertEquals(110.0, node_k1.getFirstMatchMetricValue("K2", "2", 
"K3", "30000"), 0);
+    Assert
+        .assertEquals(110 * 1000, 
node_k1.getFirstMatchMetricValue(TimeUnit.MILLISECONDS, "K2", "2", "K3", 
"30000"), 0);
+
+    Assert.assertEquals(200, node_k1.getFirstMatchMetricValue("K3", "30", 
"K2", "20"), 0);
+    Assert.assertEquals(200 * 1000, 
node_k1.getFirstMatchMetricValue(TimeUnit.MILLISECONDS, "K3", "30", "K2", 
"20"), 0);
+
+    Assert.assertEquals(100, node_k1.getMatchStatisticMetricValue("A"), 0);
+    Assert.assertEquals(100 * 1000, 
node_k1.getMatchStatisticMetricValue(TimeUnit.MILLISECONDS, "A"), 0);
+    Assert.assertEquals(100 * 1000, 
node_k1.getMatchStatisticMetricValue(TimeUnit.MILLISECONDS, "A"), 0);
+  }
+}
\ No newline at end of file
diff --git 
a/foundations/foundation-metrics/src/test/java/org/apache/servicecomb/foundation/metrics/publish/TestMetricsLoader.java
 
b/foundations/foundation-metrics/src/test/java/org/apache/servicecomb/foundation/metrics/publish/TestMetricsLoader.java
index 9d5254a..4305b76 100644
--- 
a/foundations/foundation-metrics/src/test/java/org/apache/servicecomb/foundation/metrics/publish/TestMetricsLoader.java
+++ 
b/foundations/foundation-metrics/src/test/java/org/apache/servicecomb/foundation/metrics/publish/TestMetricsLoader.java
@@ -23,8 +23,6 @@ import java.util.Map;
 import org.junit.Assert;
 import org.junit.Test;
 
-import com.google.common.collect.Lists;
-
 public class TestMetricsLoader {
   @Test
   public void test() {
@@ -49,9 +47,7 @@ public class TestMetricsLoader {
 
     MetricNode node_k1 = node.getChildrenNode("1");
 
-    Assert.assertEquals(200, node_k1.getFirstMatchMetricValue(
-        Lists.newArrayList("K2", "K3"),
-        Lists.newArrayList("20", "30")), 0);
+    Assert.assertEquals(200, node_k1.getFirstMatchMetricValue("K2", "20", 
"K3", "30"), 0);
 
     Assert.assertEquals(100, node_k1.getFirstMatchMetricValue("K2", "2"), 0);
 
diff --git 
a/metrics/metrics-core/src/main/java/org/apache/servicecomb/metrics/core/MetricsConfig.java
 
b/metrics/metrics-core/src/main/java/org/apache/servicecomb/metrics/core/MetricsConfig.java
index b903f15..2bccfcf 100644
--- 
a/metrics/metrics-core/src/main/java/org/apache/servicecomb/metrics/core/MetricsConfig.java
+++ 
b/metrics/metrics-core/src/main/java/org/apache/servicecomb/metrics/core/MetricsConfig.java
@@ -18,5 +18,5 @@
 package org.apache.servicecomb.metrics.core;
 
 public class MetricsConfig {
-  public static final String METRICS_POLLING_TIME = 
"servicecomb.metrics.window_time";
+  public static final String METRICS_WINDOW_TIME = 
"servicecomb.metrics.window_time";
 }
diff --git 
a/metrics/metrics-core/src/main/java/org/apache/servicecomb/metrics/core/MetricsDataSource.java
 
b/metrics/metrics-core/src/main/java/org/apache/servicecomb/metrics/core/MetricsDataSource.java
deleted file mode 100644
index 3854456..0000000
--- 
a/metrics/metrics-core/src/main/java/org/apache/servicecomb/metrics/core/MetricsDataSource.java
+++ /dev/null
@@ -1,137 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.servicecomb.metrics.core;
-
-import static javax.ws.rs.core.Response.Status.BAD_REQUEST;
-
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-
-import 
org.apache.servicecomb.foundation.common.exceptions.ServiceCombException;
-import org.apache.servicecomb.metrics.core.monitor.RegistryMonitor;
-import org.apache.servicecomb.swagger.invocation.exception.InvocationException;
-
-import com.google.common.collect.Lists;
-import com.netflix.config.DynamicPropertyFactory;
-import com.netflix.servo.util.Strings;
-
-public class MetricsDataSource {
-  private RegistryMonitor registryMonitor;
-
-  private Map<Long, Integer> appliedWindowTimes = new HashMap<>();
-
-  private static final MetricsDataSource INSTANCE = new MetricsDataSource();
-
-  public static MetricsDataSource getInstance() {
-    return INSTANCE;
-  }
-
-  private MetricsDataSource() {
-    this.init(RegistryMonitor.getInstance(), DynamicPropertyFactory
-        .getInstance().getStringProperty(MetricsConfig.METRICS_POLLING_TIME, 
"5000").get());
-  }
-
-  public MetricsDataSource(RegistryMonitor registryMonitor, String 
pollingSettings) {
-    this.init(registryMonitor, pollingSettings);
-  }
-
-  private void init(RegistryMonitor registryMonitor, String pollingSettings) {
-    this.registryMonitor = registryMonitor;
-
-    String[] pollingSettingStrings = pollingSettings.split(",");
-    Set<Long> parsePollingSettings = new HashSet<>();
-    for (String singlePollingSetting : pollingSettingStrings) {
-      try {
-        long settingValue = Long.parseLong(singlePollingSetting);
-        if (settingValue > 0) {
-          parsePollingSettings.add(settingValue);
-        } else {
-          throw new ServiceCombException(
-              "bad format servicecomb.metrics.window_time : " + 
String.valueOf(settingValue));
-        }
-      } catch (NumberFormatException e) {
-        throw new ServiceCombException("bad format 
servicecomb.metrics.window_time", e);
-      }
-    }
-
-    List<Long> sortedPollingSettings = 
Lists.newArrayList(parsePollingSettings);
-    System.getProperties().setProperty("servo.pollers", Strings.join(",", 
sortedPollingSettings.iterator()));
-    for (int i = 0; i < sortedPollingSettings.size(); i++) {
-      this.appliedWindowTimes.put(sortedPollingSettings.get(i), i);
-    }
-  }
-
-
-  /**  What's the WindowTime ?
-   We all know there are two major type of metric :
-   1.Time-unrelated,you can get the latest value any time immediately:
-   Counter -> increase or decrease
-   Guage -> set a certain one value
-   2.Time-related,only after a centain time pass you can compute the right 
value,"a centain time" called WindowTime
-   Max & Min -> the max value or min value in a centain time
-   Average -> average value, the simplest algorithm is f = sum / count
-   Rate -> like TPS,algorithm is f = sum / second
-
-   Will be return "servicecomb.metrics.window_time" setting in 
microservice.yaml
-   */
-  public List<Long> getAppliedWindowTime() {
-    return Lists.newArrayList(appliedWindowTimes.keySet());
-  }
-
-  //same as call measure(getAppliedWindowTime().get(0),false)
-  public Map<String, Double> measure() {
-    return measure(getAppliedWindowTime().get(0));
-  }
-
-  //same as call measure(windowTime,false)
-  public Map<String, Double> measure(long windowTime) {
-    return measure(windowTime, false);
-  }
-
-  /**
-   * windowTime usage example:
-   * if there is two window time set in "servicecomb.metrics.window_time" like 
1000,2000
-   * then windowTime = 1000 will return result of the setting 1000(1 second)
-   * windowTime = 2000 will return result of the setting 2000(2 second)
-   *
-   * there are three monitor of max,min,total
-   * 0----------1----------2----------3----------  <-time line (second)
-   *   100,200    300,400                          <-value record
-   *
-   *                 ↑ measure(1000) will return max=200 min=100 total=300
-   *                   measure(2000) will return max=0 min=0 total=0
-   *                             ↑ measure(1000) will return max=300 min=400 
total=700
-   *                               measure(2000) will return max=400 min=100 
total=1000
-   *
-   * @param windowTime getAppliedWindowTime() item
-   * @param calculateLatency need output latency
-   * @return Map<String               ,               Double>
-   */
-  public Map<String, Double> measure(long windowTime, boolean 
calculateLatency) {
-    Integer index = appliedWindowTimes.get(windowTime);
-    if (index != null) {
-      return registryMonitor.measure(index, calculateLatency);
-    }
-    throw new InvocationException(BAD_REQUEST,
-        "windowTime : " + windowTime + " unset in 
servicecomb.metrics.window_time,current available are : " +
-            Strings.join(",", getAppliedWindowTime().iterator()));
-  }
-}
diff --git 
a/metrics/metrics-core/src/main/java/org/apache/servicecomb/metrics/core/MonitorManager.java
 
b/metrics/metrics-core/src/main/java/org/apache/servicecomb/metrics/core/MonitorManager.java
new file mode 100644
index 0000000..5bdd409
--- /dev/null
+++ 
b/metrics/metrics-core/src/main/java/org/apache/servicecomb/metrics/core/MonitorManager.java
@@ -0,0 +1,181 @@
+/*
+ * 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.servicecomb.metrics.core;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.SortedMap;
+import java.util.TreeMap;
+import java.util.concurrent.Callable;
+
+import org.apache.servicecomb.foundation.common.concurrent.ConcurrentHashMapEx;
+import org.apache.servicecomb.foundation.metrics.MetricsConst;
+
+import com.netflix.config.DynamicPropertyFactory;
+import com.netflix.servo.BasicMonitorRegistry;
+import com.netflix.servo.MonitorRegistry;
+import com.netflix.servo.monitor.BasicCounter;
+import com.netflix.servo.monitor.BasicGauge;
+import com.netflix.servo.monitor.BasicTimer;
+import com.netflix.servo.monitor.Counter;
+import com.netflix.servo.monitor.Gauge;
+import com.netflix.servo.monitor.MaxGauge;
+import com.netflix.servo.monitor.Monitor;
+import com.netflix.servo.monitor.MonitorConfig;
+import com.netflix.servo.monitor.MonitorConfig.Builder;
+import com.netflix.servo.monitor.StepCounter;
+import com.netflix.servo.monitor.Timer;
+import com.netflix.servo.tag.Tag;
+import com.netflix.servo.tag.TagList;
+
+public class MonitorManager {
+
+  private final Map<String, Counter> counters;
+
+  private final Map<String, MaxGauge> maxGauges;
+
+  private final Map<String, Gauge> gauges;
+
+  private final Map<String, Timer> timers;
+
+  private final MonitorRegistry basicMonitorRegistry;
+
+  private static final MonitorManager INSTANCE = new MonitorManager();
+
+  public static MonitorManager getInstance() {
+    return INSTANCE;
+  }
+
+  private MonitorManager() {
+    this.counters = new ConcurrentHashMapEx<>();
+    this.maxGauges = new ConcurrentHashMapEx<>();
+    this.gauges = new ConcurrentHashMapEx<>();
+    this.timers = new ConcurrentHashMapEx<>();
+    this.basicMonitorRegistry = new BasicMonitorRegistry();
+    setupWindowTime();
+    registerSystemMetrics();
+  }
+
+  private void setupWindowTime() {
+    int time = 
DynamicPropertyFactory.getInstance().getIntProperty(MetricsConfig.METRICS_WINDOW_TIME,
 5000).get();
+    System.getProperties().setProperty("servo.pollers", time > 0 ? 
String.valueOf(time) : "5000");
+  }
+
+  public Counter getCounter(boolean isStepCounter, String name, String... 
tags) {
+    return counters.computeIfAbsent(getMonitorKey(name, tags), f -> {
+      Counter counter =
+          isStepCounter ? new StepCounter(getConfig(name, tags)) : new 
BasicCounter(getConfig(name, tags));
+      basicMonitorRegistry.register(counter);
+      return counter;
+    });
+  }
+
+  public MaxGauge getMaxGauge(String name, String... tags) {
+    return maxGauges.computeIfAbsent(getMonitorKey(name, tags), f -> {
+      MaxGauge maxGauge = new MaxGauge(getConfig(name, tags));
+      basicMonitorRegistry.register(maxGauge);
+      return maxGauge;
+    });
+  }
+
+  public <V extends Number> Gauge getGauge(Callable<V> callable, String name, 
String... tags) {
+    return gauges.computeIfAbsent(getMonitorKey(name, tags), f -> {
+      Gauge gauge = new BasicGauge<>(getConfig(name, tags), callable);
+      basicMonitorRegistry.register(gauge);
+      return gauge;
+    });
+  }
+
+  public Timer getTimer(String name, String... tags) {
+    return timers.computeIfAbsent(getMonitorKey(name, tags), f -> {
+      Timer timer = new BasicTimer(getConfig(name, tags));
+      basicMonitorRegistry.register(timer);
+      return timer;
+    });
+  }
+
+  public Map<String, Double> measure() {
+    Map<String, Double> measurements = new HashMap<>();
+    List<Monitor> monitors = new 
ArrayList<>(basicMonitorRegistry.getRegisteredMonitors());
+    for (Monitor monitor : monitors) {
+      measurements.put(getMonitorKey(monitor.getConfig()),
+          ((Number) monitor.getValue(0)).doubleValue());
+    }
+    return measurements;
+  }
+
+  private MonitorConfig getConfig(String name, String... tags) {
+    Builder builder = MonitorConfig.builder(name);
+    for (int i = 0; i < tags.length; i += 2) {
+      builder.withTag(tags[i], tags[i + 1]);
+    }
+    return builder.build();
+  }
+
+  private String getMonitorKey(String name, String... tags) {
+    if (tags.length != 0) {
+      SortedMap<String, String> tagMap = new TreeMap<>();
+      for (int i = 0; i < tags.length; i += 2) {
+        tagMap.put(tags[i], tags[i + 1]);
+      }
+      StringBuilder builder = new StringBuilder("(");
+      for (Entry<String, String> entry : tagMap.entrySet()) {
+        builder.append(String.format("%s=%s,", entry.getKey(), 
entry.getValue()));
+      }
+      builder.deleteCharAt(builder.length() - 1);
+      builder.append(")");
+      return name + builder.toString();
+    }
+    return name;
+  }
+
+  private static String getMonitorKey(MonitorConfig config) {
+    TagList tags = config.getTags();
+    StringBuilder tagPart = new StringBuilder("(");
+    for (Tag tag : tags) {
+      if (!"type".equals(tag.getKey())) {
+        tagPart.append(String.format("%s=%s,", tag.getKey(), tag.getValue()));
+      }
+    }
+    tagPart.deleteCharAt(tagPart.length() - 1);
+    tagPart.append(")");
+    return config.getName() + tagPart.toString();
+  }
+
+
+  private void registerSystemMetrics() {
+    SystemMetrics resource = new SystemMetrics();
+    registerSystemMetricItem(resource::getCpuLoad, "cpuLoad");
+    registerSystemMetricItem(resource::getCpuRunningThreads, 
"cpuRunningThreads");
+    registerSystemMetricItem(resource::getHeapInit, "heapInit");
+    registerSystemMetricItem(resource::getHeapCommit, "heapCommit");
+    registerSystemMetricItem(resource::getHeapUsed, "heapUsed");
+    registerSystemMetricItem(resource::getHeapMax, "heapMax");
+    registerSystemMetricItem(resource::getNonHeapInit, "nonHeapInit");
+    registerSystemMetricItem(resource::getNonHeapCommit, "nonHeapCommit");
+    registerSystemMetricItem(resource::getNonHeapUsed, "nonHeapUsed");
+    registerSystemMetricItem(resource::getNonHeapMax, "nonHeapMax");
+  }
+
+  private <V extends Number> void registerSystemMetricItem(Callable<V> 
callable, String name) {
+    this.getGauge(callable, MetricsConst.JVM, MetricsConst.TAG_STATISTIC, 
"gauge", MetricsConst.TAG_NAME, name);
+  }
+}
\ No newline at end of file
diff --git 
a/metrics/metrics-core/src/main/java/org/apache/servicecomb/metrics/core/SystemMetrics.java
 
b/metrics/metrics-core/src/main/java/org/apache/servicecomb/metrics/core/SystemMetrics.java
new file mode 100644
index 0000000..132412d
--- /dev/null
+++ 
b/metrics/metrics-core/src/main/java/org/apache/servicecomb/metrics/core/SystemMetrics.java
@@ -0,0 +1,83 @@
+/*
+ * 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.servicecomb.metrics.core;
+
+import java.lang.management.ManagementFactory;
+import java.lang.management.MemoryMXBean;
+import java.lang.management.OperatingSystemMXBean;
+import java.lang.management.ThreadMXBean;
+
+public class SystemMetrics {
+  private final OperatingSystemMXBean systemMXBean;
+
+  private final ThreadMXBean threadMXBean;
+
+  private final MemoryMXBean memoryMXBean;
+
+  public SystemMetrics() {
+    this(ManagementFactory.getOperatingSystemMXBean(), 
ManagementFactory.getThreadMXBean(),
+        ManagementFactory.getMemoryMXBean());
+  }
+
+  public SystemMetrics(OperatingSystemMXBean systemMXBean, ThreadMXBean 
threadMXBean,
+      MemoryMXBean memoryMXBean) {
+    this.systemMXBean = systemMXBean;
+    this.threadMXBean = threadMXBean;
+    this.memoryMXBean = memoryMXBean;
+  }
+
+  public double getCpuLoad() {
+    return systemMXBean.getSystemLoadAverage();
+  }
+
+  public int getCpuRunningThreads() {
+    return threadMXBean.getThreadCount();
+  }
+
+  public long getHeapInit() {
+    return memoryMXBean.getHeapMemoryUsage().getInit();
+  }
+
+  public long getHeapMax() {
+    return memoryMXBean.getHeapMemoryUsage().getMax();
+  }
+
+  public long getHeapCommit() {
+    return memoryMXBean.getHeapMemoryUsage().getCommitted();
+  }
+
+  public long getHeapUsed() {
+    return memoryMXBean.getHeapMemoryUsage().getUsed();
+  }
+
+  public long getNonHeapInit() {
+    return memoryMXBean.getNonHeapMemoryUsage().getInit();
+  }
+
+  public long getNonHeapMax() {
+    return memoryMXBean.getNonHeapMemoryUsage().getMax();
+  }
+
+  public long getNonHeapCommit() {
+    return memoryMXBean.getNonHeapMemoryUsage().getCommitted();
+  }
+
+  public long getNonHeapUsed() {
+    return memoryMXBean.getNonHeapMemoryUsage().getUsed();
+  }
+}
diff --git 
a/metrics/metrics-core/src/main/java/org/apache/servicecomb/metrics/core/event/InvocationFinishedEventListener.java
 
b/metrics/metrics-core/src/main/java/org/apache/servicecomb/metrics/core/event/InvocationFinishedEventListener.java
index 948b631..4f83d7c 100644
--- 
a/metrics/metrics-core/src/main/java/org/apache/servicecomb/metrics/core/event/InvocationFinishedEventListener.java
+++ 
b/metrics/metrics-core/src/main/java/org/apache/servicecomb/metrics/core/event/InvocationFinishedEventListener.java
@@ -17,39 +17,45 @@
 
 package org.apache.servicecomb.metrics.core.event;
 
+import java.util.concurrent.TimeUnit;
+
+import org.apache.commons.lang3.ArrayUtils;
 import org.apache.servicecomb.core.metrics.InvocationFinishedEvent;
-import org.apache.servicecomb.foundation.common.event.Event;
 import org.apache.servicecomb.foundation.common.event.EventListener;
-import org.apache.servicecomb.metrics.core.monitor.ConsumerInvocationMonitor;
-import org.apache.servicecomb.metrics.core.monitor.ProducerInvocationMonitor;
-import org.apache.servicecomb.metrics.core.monitor.RegistryMonitor;
+import org.apache.servicecomb.foundation.metrics.MetricsConst;
+import org.apache.servicecomb.metrics.core.MonitorManager;
 import org.apache.servicecomb.swagger.invocation.InvocationType;
 
-public class InvocationFinishedEventListener implements EventListener {
-  private final RegistryMonitor registryMonitor;
-
-  public InvocationFinishedEventListener(RegistryMonitor registryMonitor) {
-    this.registryMonitor = registryMonitor;
+public class InvocationFinishedEventListener implements 
EventListener<InvocationFinishedEvent> {
+  @Override
+  public void process(InvocationFinishedEvent data) {
+    String[] tags = new String[] {MetricsConst.TAG_OPERATION, 
data.getOperationName(),
+        MetricsConst.TAG_ROLE, String.valueOf(data.getInvocationType()),
+        MetricsConst.TAG_STATUS, String.valueOf(data.getStatusCode())};
+    this.updateLatency(MetricsConst.STAGE_TOTAL, 
data.getTotalElapsedNanoTime(), tags);
+    this.updateCount(tags);
+    if (InvocationType.PRODUCER.equals(data.getInvocationType())) {
+      this.updateLatency(MetricsConst.STAGE_QUEUE, data.getInQueueNanoTime(), 
tags);
+      this.updateLatency(MetricsConst.STAGE_EXECUTION, 
data.getExecutionElapsedNanoTime(), tags);
+    }
   }
 
-  @Override
-  public Class<? extends Event> getConcernedEvent() {
-    return InvocationFinishedEvent.class;
+  private void updateLatency(String stage, long value, String... basicTags) {
+    String[] tags = ArrayUtils
+        .addAll(basicTags, MetricsConst.TAG_STAGE, stage, 
MetricsConst.TAG_UNIT, String.valueOf(TimeUnit.MILLISECONDS));
+    MonitorManager.getInstance()
+        .getTimer(MetricsConst.SERVICECOMB_INVOCATION, ArrayUtils.addAll(tags, 
MetricsConst.TAG_STATISTIC, "latency"))
+        .record(value, TimeUnit.NANOSECONDS);
+    MonitorManager.getInstance()
+        .getMaxGauge(MetricsConst.SERVICECOMB_INVOCATION, 
ArrayUtils.addAll(tags, MetricsConst.TAG_STATISTIC, "max"))
+        .update(TimeUnit.NANOSECONDS.toMillis(value));
   }
 
-  @Override
-  public void process(Event data) {
-    InvocationFinishedEvent event = (InvocationFinishedEvent) data;
-    if (InvocationType.PRODUCER.equals(event.getInvocationType())) {
-      ProducerInvocationMonitor monitor = 
registryMonitor.getProducerInvocationMonitor(event.getOperationName());
-      monitor.getLifeTimeInQueue().update(event.getInQueueNanoTime(), 
String.valueOf(event.getStatusCode()));
-      monitor.getExecutionTime().update(event.getProcessElapsedNanoTime(), 
String.valueOf(event.getStatusCode()));
-      monitor.getProducerLatency().update(event.getTotalElapsedNanoTime(), 
String.valueOf(event.getStatusCode()));
-      
monitor.getProducerCall().increment(String.valueOf(event.getStatusCode()));
-    } else {
-      ConsumerInvocationMonitor monitor = 
registryMonitor.getConsumerInvocationMonitor(event.getOperationName());
-      monitor.getConsumerLatency().update(event.getTotalElapsedNanoTime(), 
String.valueOf(event.getStatusCode()));
-      
monitor.getConsumerCall().increment(String.valueOf(event.getStatusCode()));
-    }
+  private void updateCount(String... basicTags) {
+    String[] tags = ArrayUtils.addAll(basicTags, MetricsConst.TAG_STAGE, 
MetricsConst.STAGE_TOTAL);
+    MonitorManager.getInstance().getCounter(true, 
MetricsConst.SERVICECOMB_INVOCATION,
+        ArrayUtils.addAll(tags, MetricsConst.TAG_STATISTIC, 
"tps")).increment();
+    MonitorManager.getInstance().getCounter(false, 
MetricsConst.SERVICECOMB_INVOCATION,
+        ArrayUtils.addAll(tags, MetricsConst.TAG_STATISTIC, 
"count")).increment();
   }
 }
diff --git 
a/metrics/metrics-core/src/main/java/org/apache/servicecomb/metrics/core/event/EventListenerManager.java
 
b/metrics/metrics-core/src/main/java/org/apache/servicecomb/metrics/core/event/InvocationStartExecutionEventListener.java
similarity index 54%
rename from 
metrics/metrics-core/src/main/java/org/apache/servicecomb/metrics/core/event/EventListenerManager.java
rename to 
metrics/metrics-core/src/main/java/org/apache/servicecomb/metrics/core/event/InvocationStartExecutionEventListener.java
index da4af45..af3eeb4 100644
--- 
a/metrics/metrics-core/src/main/java/org/apache/servicecomb/metrics/core/event/EventListenerManager.java
+++ 
b/metrics/metrics-core/src/main/java/org/apache/servicecomb/metrics/core/event/InvocationStartExecutionEventListener.java
@@ -17,18 +17,19 @@
 
 package org.apache.servicecomb.metrics.core.event;
 
+import org.apache.servicecomb.core.metrics.InvocationStartExecutionEvent;
 import org.apache.servicecomb.foundation.common.event.EventListener;
-import org.apache.servicecomb.foundation.common.utils.EventUtils;
-import org.apache.servicecomb.metrics.core.monitor.RegistryMonitor;
+import org.apache.servicecomb.foundation.metrics.MetricsConst;
+import org.apache.servicecomb.metrics.core.MonitorManager;
+import org.apache.servicecomb.swagger.invocation.InvocationType;
 
-public class EventListenerManager {
-  public EventListenerManager(RegistryMonitor registryMonitor) {
-    this.registerEventListener(new 
InvocationStartedEventListener(registryMonitor));
-    this.registerEventListener(new 
InvocationStartProcessingEventListener(registryMonitor));
-    this.registerEventListener(new 
InvocationFinishedEventListener(registryMonitor));
-  }
-
-  public void registerEventListener(EventListener listener) {
-    EventUtils.registerEventListener(listener);
+public class InvocationStartExecutionEventListener implements 
EventListener<InvocationStartExecutionEvent> {
+  @Override
+  public void process(InvocationStartExecutionEvent data) {
+    MonitorManager.getInstance().getCounter(false, 
MetricsConst.SERVICECOMB_INVOCATION,
+        MetricsConst.TAG_OPERATION, data.getOperationName(),
+        MetricsConst.TAG_STAGE, MetricsConst.STAGE_QUEUE,
+        MetricsConst.TAG_ROLE, String.valueOf(InvocationType.PRODUCER),
+        MetricsConst.TAG_STATISTIC, "waitInQueue").increment(-1);
   }
 }
diff --git 
a/metrics/metrics-core/src/main/java/org/apache/servicecomb/metrics/core/event/InvocationStartProcessingEventListener.java
 
b/metrics/metrics-core/src/main/java/org/apache/servicecomb/metrics/core/event/InvocationStartProcessingEventListener.java
deleted file mode 100644
index 771810d..0000000
--- 
a/metrics/metrics-core/src/main/java/org/apache/servicecomb/metrics/core/event/InvocationStartProcessingEventListener.java
+++ /dev/null
@@ -1,48 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.servicecomb.metrics.core.event;
-
-import org.apache.servicecomb.core.metrics.InvocationStartProcessingEvent;
-import org.apache.servicecomb.foundation.common.event.Event;
-import org.apache.servicecomb.foundation.common.event.EventListener;
-import org.apache.servicecomb.metrics.core.monitor.ProducerInvocationMonitor;
-import org.apache.servicecomb.metrics.core.monitor.RegistryMonitor;
-import org.apache.servicecomb.swagger.invocation.InvocationType;
-
-public class InvocationStartProcessingEventListener implements EventListener {
-
-  private final RegistryMonitor registryMonitor;
-
-  public InvocationStartProcessingEventListener(RegistryMonitor 
registryMonitor) {
-    this.registryMonitor = registryMonitor;
-  }
-
-  @Override
-  public Class<? extends Event> getConcernedEvent() {
-    return InvocationStartProcessingEvent.class;
-  }
-
-  @Override
-  public void process(Event data) {
-    InvocationStartProcessingEvent event = (InvocationStartProcessingEvent) 
data;
-    if (InvocationType.PRODUCER.equals(event.getInvocationType())) {
-      ProducerInvocationMonitor monitor = 
registryMonitor.getProducerInvocationMonitor(event.getOperationName());
-      monitor.getWaitInQueue().increment(-1);
-    }
-  }
-}
diff --git 
a/metrics/metrics-core/src/main/java/org/apache/servicecomb/metrics/core/event/InvocationStartedEventListener.java
 
b/metrics/metrics-core/src/main/java/org/apache/servicecomb/metrics/core/event/InvocationStartedEventListener.java
index 68defae..e3ce689 100644
--- 
a/metrics/metrics-core/src/main/java/org/apache/servicecomb/metrics/core/event/InvocationStartedEventListener.java
+++ 
b/metrics/metrics-core/src/main/java/org/apache/servicecomb/metrics/core/event/InvocationStartedEventListener.java
@@ -18,31 +18,20 @@
 package org.apache.servicecomb.metrics.core.event;
 
 import org.apache.servicecomb.core.metrics.InvocationStartedEvent;
-import org.apache.servicecomb.foundation.common.event.Event;
 import org.apache.servicecomb.foundation.common.event.EventListener;
-import org.apache.servicecomb.metrics.core.monitor.ProducerInvocationMonitor;
-import org.apache.servicecomb.metrics.core.monitor.RegistryMonitor;
+import org.apache.servicecomb.foundation.metrics.MetricsConst;
+import org.apache.servicecomb.metrics.core.MonitorManager;
 import org.apache.servicecomb.swagger.invocation.InvocationType;
 
-public class InvocationStartedEventListener implements EventListener {
-
-  private final RegistryMonitor registryMonitor;
-
-  public InvocationStartedEventListener(RegistryMonitor registryMonitor) {
-    this.registryMonitor = registryMonitor;
-  }
-
-  @Override
-  public Class<? extends Event> getConcernedEvent() {
-    return InvocationStartedEvent.class;
-  }
-
+public class InvocationStartedEventListener implements 
EventListener<InvocationStartedEvent> {
   @Override
-  public void process(Event data) {
-    InvocationStartedEvent event = (InvocationStartedEvent) data;
-    if (InvocationType.PRODUCER.equals(event.getInvocationType())) {
-      ProducerInvocationMonitor monitor = 
registryMonitor.getProducerInvocationMonitor(event.getOperationName());
-      monitor.getWaitInQueue().increment();
+  public void process(InvocationStartedEvent data) {
+    if (InvocationType.PRODUCER.equals(data.getInvocationType())) {
+      MonitorManager.getInstance().getCounter(false, 
MetricsConst.SERVICECOMB_INVOCATION,
+          MetricsConst.TAG_OPERATION, data.getOperationName(),
+          MetricsConst.TAG_STAGE, MetricsConst.STAGE_QUEUE,
+          MetricsConst.TAG_ROLE, String.valueOf(InvocationType.PRODUCER),
+          MetricsConst.TAG_STATISTIC, "waitInQueue").increment();
     }
   }
 }
diff --git 
a/metrics/metrics-core/src/main/java/org/apache/servicecomb/metrics/core/health/DefaultHealthCheckExtraData.java
 
b/metrics/metrics-core/src/main/java/org/apache/servicecomb/metrics/core/health/DefaultHealthCheckExtraData.java
deleted file mode 100644
index 8a78fd1..0000000
--- 
a/metrics/metrics-core/src/main/java/org/apache/servicecomb/metrics/core/health/DefaultHealthCheckExtraData.java
+++ /dev/null
@@ -1,69 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.servicecomb.metrics.core.health;
-
-public class DefaultHealthCheckExtraData {
-  private String instanceId;
-
-  private String hostName;
-
-  private String appId;
-
-  private String serviceName;
-
-  private String serviceVersion;
-
-  private String endpoints;
-
-  public String getInstanceId() {
-    return instanceId;
-  }
-
-  public String getHostName() {
-    return hostName;
-  }
-
-  public String getAppId() {
-    return appId;
-  }
-
-  public String getServiceName() {
-    return serviceName;
-  }
-
-  public String getServiceVersion() {
-    return serviceVersion;
-  }
-
-  public String getEndpoints() {
-    return endpoints;
-  }
-
-  public DefaultHealthCheckExtraData() {
-  }
-
-  public DefaultHealthCheckExtraData(String instanceId, String hostName, 
String appId, String serviceName,
-      String serviceVersion, String endpoints) {
-    this.instanceId = instanceId;
-    this.hostName = hostName;
-    this.appId = appId;
-    this.serviceName = serviceName;
-    this.serviceVersion = serviceVersion;
-    this.endpoints = endpoints;
-  }
-}
diff --git 
a/metrics/metrics-core/src/main/java/org/apache/servicecomb/metrics/core/health/DefaultMicroserviceHealthChecker.java
 
b/metrics/metrics-core/src/main/java/org/apache/servicecomb/metrics/core/health/DefaultMicroserviceHealthChecker.java
deleted file mode 100644
index 03e2913..0000000
--- 
a/metrics/metrics-core/src/main/java/org/apache/servicecomb/metrics/core/health/DefaultMicroserviceHealthChecker.java
+++ /dev/null
@@ -1,63 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.servicecomb.metrics.core.health;
-
-import javax.ws.rs.core.Response.Status;
-
-import org.apache.servicecomb.foundation.common.utils.JsonUtils;
-import org.apache.servicecomb.foundation.metrics.health.HealthCheckResult;
-import org.apache.servicecomb.foundation.metrics.health.HealthChecker;
-import org.apache.servicecomb.serviceregistry.RegistryUtils;
-import org.apache.servicecomb.serviceregistry.api.registry.Microservice;
-import 
org.apache.servicecomb.serviceregistry.api.registry.MicroserviceInstance;
-import org.apache.servicecomb.swagger.invocation.exception.InvocationException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public class DefaultMicroserviceHealthChecker implements HealthChecker {
-
-  private static Logger logger = 
LoggerFactory.getLogger(DefaultMicroserviceHealthChecker.class);
-
-  @Override
-  public String getName() {
-    return "default";
-  }
-
-  @Override
-  public HealthCheckResult check() {
-    return new HealthCheckResult(true, "", getExtraData());
-  }
-
-  private String getExtraData() {
-    try {
-      Microservice microservice = RegistryUtils.getMicroservice();
-      MicroserviceInstance instance = RegistryUtils.getMicroserviceInstance();
-      return JsonUtils.writeValueAsString(new DefaultHealthCheckExtraData(
-          instance.getInstanceId(),
-          instance.getHostName(),
-          microservice.getAppId(),
-          microservice.getServiceName(),
-          microservice.getVersion(),
-          String.join(",", instance.getEndpoints())));
-    } catch (Exception e) {
-      String error = "unable load microservice info from RegistryUtils";
-      logger.error(error, e);
-      throw new InvocationException(Status.INTERNAL_SERVER_ERROR, error);
-    }
-  }
-}
diff --git 
a/metrics/metrics-core/src/main/java/org/apache/servicecomb/metrics/core/monitor/CallMonitor.java
 
b/metrics/metrics-core/src/main/java/org/apache/servicecomb/metrics/core/monitor/CallMonitor.java
deleted file mode 100644
index 93a3903..0000000
--- 
a/metrics/metrics-core/src/main/java/org/apache/servicecomb/metrics/core/monitor/CallMonitor.java
+++ /dev/null
@@ -1,92 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.servicecomb.metrics.core.monitor;
-
-import java.util.HashMap;
-import java.util.Map;
-
-import org.apache.servicecomb.foundation.common.concurrent.ConcurrentHashMapEx;
-import org.apache.servicecomb.foundation.metrics.MetricsConst;
-import org.apache.servicecomb.metrics.core.utils.MonitorUtils;
-
-import com.netflix.servo.monitor.BasicCounter;
-import com.netflix.servo.monitor.MonitorConfig;
-import com.netflix.servo.monitor.StepCounter;
-import com.netflix.servo.tag.Tags;
-
-public class CallMonitor {
-  private final Map<String, StatusCounter> statusCounters;
-
-  private final String operation;
-
-  private final String stage;
-
-  private final String role;
-
-  public CallMonitor(String operation, String stage, String role) {
-    this.operation = operation;
-    this.stage = stage;
-    this.role = role;
-
-    this.statusCounters = new ConcurrentHashMapEx<>();
-  }
-
-  public void increment(String statusCode) {
-    StatusCounter counter = statusCounters
-        .computeIfAbsent(statusCode, d -> new StatusCounter(operation, stage, 
role, statusCode));
-    counter.increment();
-  }
-
-  public Map<String, Double> measure(int windowTimeIndex) {
-    Map<String, Double> metrics = new HashMap<>();
-    for (StatusCounter counter : statusCounters.values()) {
-      metrics.putAll(counter.measure(windowTimeIndex));
-    }
-    return metrics;
-  }
-
-  class StatusCounter {
-    private final BasicCounter totalCount;
-
-    private final StepCounter tps;
-
-    public StatusCounter(String operation, String stage, String role, String 
statusCode) {
-      MonitorConfig config = 
MonitorConfig.builder(MetricsConst.SERVICECOMB_INVOCATION)
-          .withTag(MetricsConst.TAG_STATUS, 
statusCode).withTag(MetricsConst.TAG_OPERATION, operation)
-          .withTag(MetricsConst.TAG_STAGE, 
stage).withTag(MetricsConst.TAG_ROLE, role).build();
-
-      this.totalCount = new BasicCounter(
-          config.withAdditionalTag(Tags.newTag(MetricsConst.TAG_STATISTIC, 
"totalCount")));
-      this.tps = new 
StepCounter(config.withAdditionalTag(Tags.newTag(MetricsConst.TAG_STATISTIC, 
"tps")));
-    }
-
-    public void increment() {
-      totalCount.increment();
-      tps.increment();
-    }
-
-    public Map<String, Double> measure(int windowTimeIndex) {
-      Map<String, Double> measurements = new HashMap<>();
-      
measurements.put(MonitorUtils.getMonitorName(this.totalCount.getConfig()),
-          this.totalCount.getValue(windowTimeIndex).doubleValue());
-      measurements.put(MonitorUtils.getMonitorName(this.tps.getConfig()),
-          this.tps.getValue(windowTimeIndex).doubleValue());
-      return measurements;
-    }
-  }
-}
diff --git 
a/metrics/metrics-core/src/main/java/org/apache/servicecomb/metrics/core/monitor/ConsumerInvocationMonitor.java
 
b/metrics/metrics-core/src/main/java/org/apache/servicecomb/metrics/core/monitor/ConsumerInvocationMonitor.java
deleted file mode 100644
index eb18198..0000000
--- 
a/metrics/metrics-core/src/main/java/org/apache/servicecomb/metrics/core/monitor/ConsumerInvocationMonitor.java
+++ /dev/null
@@ -1,49 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.servicecomb.metrics.core.monitor;
-
-import java.util.HashMap;
-import java.util.Map;
-
-import org.apache.servicecomb.foundation.metrics.MetricsConst;
-
-public class ConsumerInvocationMonitor {
-  private final TimerMonitor consumerLatency;
-
-  private final CallMonitor consumerCall;
-
-  public TimerMonitor getConsumerLatency() {
-    return consumerLatency;
-  }
-
-  public CallMonitor getConsumerCall() {
-    return consumerCall;
-  }
-
-  public ConsumerInvocationMonitor(String operation) {
-    this.consumerLatency = new TimerMonitor(operation, 
MetricsConst.STAGE_WHOLE, MetricsConst.ROLE_CONSUMER);
-    this.consumerCall = new CallMonitor(operation, MetricsConst.STAGE_WHOLE, 
MetricsConst.ROLE_CONSUMER);
-  }
-
-  public Map<String, Double> measure(int windowTimeIndex, boolean 
calculateLatency) {
-    Map<String, Double> measurements = new HashMap<>();
-    measurements.putAll(consumerCall.measure(windowTimeIndex));
-    measurements.putAll(consumerLatency.measure(windowTimeIndex, 
calculateLatency));
-    return measurements;
-  }
-}
diff --git 
a/metrics/metrics-core/src/main/java/org/apache/servicecomb/metrics/core/monitor/DefaultSystemMonitor.java
 
b/metrics/metrics-core/src/main/java/org/apache/servicecomb/metrics/core/monitor/DefaultSystemMonitor.java
deleted file mode 100644
index e38a143..0000000
--- 
a/metrics/metrics-core/src/main/java/org/apache/servicecomb/metrics/core/monitor/DefaultSystemMonitor.java
+++ /dev/null
@@ -1,126 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.servicecomb.metrics.core.monitor;
-
-import java.lang.management.ManagementFactory;
-import java.lang.management.MemoryMXBean;
-import java.lang.management.OperatingSystemMXBean;
-import java.lang.management.ThreadMXBean;
-import java.util.HashMap;
-import java.util.Map;
-
-import org.apache.servicecomb.foundation.metrics.MetricsConst;
-import org.springframework.stereotype.Component;
-
-@Component
-public class DefaultSystemMonitor implements SystemMonitor {
-
-  private final OperatingSystemMXBean systemMXBean;
-
-  private final ThreadMXBean threadMXBean;
-
-  private final MemoryMXBean memoryMXBean;
-
-  public DefaultSystemMonitor() {
-    this(ManagementFactory.getOperatingSystemMXBean(), 
ManagementFactory.getThreadMXBean(),
-        ManagementFactory.getMemoryMXBean());
-  }
-
-  public DefaultSystemMonitor(OperatingSystemMXBean systemMXBean, ThreadMXBean 
threadMXBean,
-      MemoryMXBean memoryMXBean) {
-    this.systemMXBean = systemMXBean;
-    this.threadMXBean = threadMXBean;
-    this.memoryMXBean = memoryMXBean;
-  }
-
-  @Override
-  public double getCpuLoad() {
-    return systemMXBean.getSystemLoadAverage();
-  }
-
-  @Override
-  public int getCpuRunningThreads() {
-    return threadMXBean.getThreadCount();
-  }
-
-  @Override
-  public long getHeapInit() {
-    return memoryMXBean.getHeapMemoryUsage().getInit();
-  }
-
-  @Override
-  public long getHeapMax() {
-    return memoryMXBean.getHeapMemoryUsage().getMax();
-  }
-
-  @Override
-  public long getHeapCommit() {
-    return memoryMXBean.getHeapMemoryUsage().getCommitted();
-  }
-
-  @Override
-  public long getHeapUsed() {
-    return memoryMXBean.getHeapMemoryUsage().getUsed();
-  }
-
-  @Override
-  public long getNonHeapInit() {
-    return memoryMXBean.getNonHeapMemoryUsage().getInit();
-  }
-
-  @Override
-  public long getNonHeapMax() {
-    return memoryMXBean.getNonHeapMemoryUsage().getMax();
-  }
-
-  @Override
-  public long getNonHeapCommit() {
-    return memoryMXBean.getNonHeapMemoryUsage().getCommitted();
-  }
-
-  @Override
-  public long getNonHeapUsed() {
-    return memoryMXBean.getNonHeapMemoryUsage().getUsed();
-  }
-
-  @Override
-  public Map<String, Double> measure() {
-    Map<String, Double> measurements = new HashMap<>();
-    measurements.put(String.format(MetricsConst.JVM + "(%s=%s,%s=%s)", 
MetricsConst.TAG_STATISTIC, "gauge",
-        MetricsConst.TAG_NAME, "cpuLoad"), getCpuLoad());
-    measurements.put(String.format(MetricsConst.JVM + "(%s=%s,%s=%s)", 
MetricsConst.TAG_STATISTIC, "gauge",
-        MetricsConst.TAG_NAME, "cpuRunningThreads"), (double) 
getCpuRunningThreads());
-    measurements.put(String.format(MetricsConst.JVM + "(%s=%s,%s=%s)", 
MetricsConst.TAG_STATISTIC, "gauge",
-        MetricsConst.TAG_NAME, "heapInit"), (double) getHeapInit());
-    measurements.put(String.format(MetricsConst.JVM + "(%s=%s,%s=%s)", 
MetricsConst.TAG_STATISTIC, "gauge",
-        MetricsConst.TAG_NAME, "heapMax"), (double) getHeapMax());
-    measurements.put(String.format(MetricsConst.JVM + "(%s=%s,%s=%s)", 
MetricsConst.TAG_STATISTIC, "gauge",
-        MetricsConst.TAG_NAME, "heapCommit"), (double) getHeapCommit());
-    measurements.put(String.format(MetricsConst.JVM + "(%s=%s,%s=%s)", 
MetricsConst.TAG_STATISTIC, "gauge",
-        MetricsConst.TAG_NAME, "heapUsed"), (double) getHeapUsed());
-    measurements.put(String.format(MetricsConst.JVM + "(%s=%s,%s=%s)", 
MetricsConst.TAG_STATISTIC, "gauge",
-        MetricsConst.TAG_NAME, "nonHeapInit"), (double) getNonHeapInit());
-    measurements.put(String.format(MetricsConst.JVM + "(%s=%s,%s=%s)", 
MetricsConst.TAG_STATISTIC, "gauge",
-        MetricsConst.TAG_NAME, "nonHeapMax"), (double) getNonHeapMax());
-    measurements.put(String.format(MetricsConst.JVM + "(%s=%s,%s=%s)", 
MetricsConst.TAG_STATISTIC, "gauge",
-        MetricsConst.TAG_NAME, "nonHeapCommit"), (double) getNonHeapCommit());
-    measurements.put(String.format(MetricsConst.JVM + "(%s=%s,%s=%s)", 
MetricsConst.TAG_STATISTIC, "gauge",
-        MetricsConst.TAG_NAME, "nonHeapUsed"), (double) getNonHeapUsed());
-    return measurements;
-  }
-}
diff --git 
a/metrics/metrics-core/src/main/java/org/apache/servicecomb/metrics/core/monitor/ProducerInvocationMonitor.java
 
b/metrics/metrics-core/src/main/java/org/apache/servicecomb/metrics/core/monitor/ProducerInvocationMonitor.java
deleted file mode 100644
index cb75e57..0000000
--- 
a/metrics/metrics-core/src/main/java/org/apache/servicecomb/metrics/core/monitor/ProducerInvocationMonitor.java
+++ /dev/null
@@ -1,84 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.servicecomb.metrics.core.monitor;
-
-import java.util.HashMap;
-import java.util.Map;
-
-import org.apache.servicecomb.foundation.metrics.MetricsConst;
-import org.apache.servicecomb.metrics.core.utils.MonitorUtils;
-
-import com.netflix.servo.monitor.BasicCounter;
-import com.netflix.servo.monitor.MonitorConfig;
-
-public class ProducerInvocationMonitor {
-  private final BasicCounter waitInQueue;
-
-  private final TimerMonitor lifeTimeInQueue;
-
-  private final TimerMonitor executionTime;
-
-  private final TimerMonitor producerLatency;
-
-  private final CallMonitor producerCall;
-
-  public BasicCounter getWaitInQueue() {
-    return waitInQueue;
-  }
-
-  public TimerMonitor getLifeTimeInQueue() {
-    return lifeTimeInQueue;
-  }
-
-  public TimerMonitor getExecutionTime() {
-    return executionTime;
-  }
-
-  public TimerMonitor getProducerLatency() {
-    return producerLatency;
-  }
-
-  public CallMonitor getProducerCall() {
-    return producerCall;
-  }
-
-  public ProducerInvocationMonitor(String operation) {
-    this.waitInQueue = new 
BasicCounter(MonitorConfig.builder(MetricsConst.SERVICECOMB_INVOCATION)
-        .withTag(MetricsConst.TAG_OPERATION, operation)
-        .withTag(MetricsConst.TAG_STAGE, MetricsConst.STAGE_QUEUE)
-        .withTag(MetricsConst.TAG_ROLE, MetricsConst.ROLE_PRODUCER)
-        .withTag(MetricsConst.TAG_STATISTIC, "waitInQueue")
-        .build());
-
-    this.lifeTimeInQueue = new TimerMonitor(operation, 
MetricsConst.STAGE_QUEUE, MetricsConst.ROLE_PRODUCER);
-    this.executionTime = new TimerMonitor(operation, 
MetricsConst.STAGE_EXECUTION, MetricsConst.ROLE_PRODUCER);
-    this.producerLatency = new TimerMonitor(operation, 
MetricsConst.STAGE_WHOLE, MetricsConst.ROLE_PRODUCER);
-    this.producerCall = new CallMonitor(operation, MetricsConst.STAGE_WHOLE, 
MetricsConst.ROLE_PRODUCER);
-  }
-
-  public Map<String, Double> measure(int windowTimeIndex, boolean 
calculateLatency) {
-    Map<String, Double> measurements = new HashMap<>();
-    measurements.put(MonitorUtils.getMonitorName(waitInQueue.getConfig()),
-        waitInQueue.getValue(windowTimeIndex).doubleValue());
-    measurements.putAll(lifeTimeInQueue.measure(windowTimeIndex, 
calculateLatency));
-    measurements.putAll(executionTime.measure(windowTimeIndex, 
calculateLatency));
-    measurements.putAll(producerLatency.measure(windowTimeIndex, 
calculateLatency));
-    measurements.putAll(producerCall.measure(windowTimeIndex));
-    return measurements;
-  }
-}
diff --git 
a/metrics/metrics-core/src/main/java/org/apache/servicecomb/metrics/core/monitor/RegistryMonitor.java
 
b/metrics/metrics-core/src/main/java/org/apache/servicecomb/metrics/core/monitor/RegistryMonitor.java
deleted file mode 100644
index 4607d59..0000000
--- 
a/metrics/metrics-core/src/main/java/org/apache/servicecomb/metrics/core/monitor/RegistryMonitor.java
+++ /dev/null
@@ -1,72 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.servicecomb.metrics.core.monitor;
-
-import java.util.HashMap;
-import java.util.Map;
-
-import org.apache.servicecomb.foundation.common.concurrent.ConcurrentHashMapEx;
-import org.apache.servicecomb.foundation.common.utils.SPIServiceUtils;
-
-public class RegistryMonitor {
-
-  private SystemMonitor systemMonitor;
-
-  private Map<String, ConsumerInvocationMonitor> consumerInvocationMonitors;
-
-  private Map<String, ProducerInvocationMonitor> producerInvocationMonitors;
-
-  private static final RegistryMonitor INSTANCE = new RegistryMonitor();
-
-  public static RegistryMonitor getInstance() {
-    return INSTANCE;
-  }
-
-  private RegistryMonitor() {
-    init(SPIServiceUtils.getTargetService(SystemMonitor.class));
-  }
-
-  public RegistryMonitor(SystemMonitor systemMonitor) {
-    init(systemMonitor);
-  }
-
-  private void init(SystemMonitor systemMonitor) {
-    this.systemMonitor = systemMonitor;
-    this.consumerInvocationMonitors = new ConcurrentHashMapEx<>();
-    this.producerInvocationMonitors = new ConcurrentHashMapEx<>();
-  }
-
-  public ConsumerInvocationMonitor getConsumerInvocationMonitor(String 
operationName) {
-    return consumerInvocationMonitors.computeIfAbsent(operationName, i -> new 
ConsumerInvocationMonitor(operationName));
-  }
-
-  public ProducerInvocationMonitor getProducerInvocationMonitor(String 
operationName) {
-    return producerInvocationMonitors.computeIfAbsent(operationName, i -> new 
ProducerInvocationMonitor(operationName));
-  }
-
-  public Map<String, Double> measure(int windowTimeIndex, boolean 
calculateLatency) {
-    Map<String, Double> measurements = new HashMap<>(systemMonitor.measure());
-    for (ConsumerInvocationMonitor monitor : 
this.consumerInvocationMonitors.values()) {
-      measurements.putAll(monitor.measure(windowTimeIndex, calculateLatency));
-    }
-    for (ProducerInvocationMonitor monitor : 
this.producerInvocationMonitors.values()) {
-      measurements.putAll(monitor.measure(windowTimeIndex, calculateLatency));
-    }
-    return measurements;
-  }
-}
diff --git 
a/metrics/metrics-core/src/main/java/org/apache/servicecomb/metrics/core/monitor/SystemMonitor.java
 
b/metrics/metrics-core/src/main/java/org/apache/servicecomb/metrics/core/monitor/SystemMonitor.java
deleted file mode 100644
index 098d685..0000000
--- 
a/metrics/metrics-core/src/main/java/org/apache/servicecomb/metrics/core/monitor/SystemMonitor.java
+++ /dev/null
@@ -1,44 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.servicecomb.metrics.core.monitor;
-
-import java.util.Map;
-
-public interface SystemMonitor {
-  double getCpuLoad();
-
-  int getCpuRunningThreads();
-
-  long getHeapInit();
-
-  long getHeapMax();
-
-  long getHeapCommit();
-
-  long getHeapUsed();
-
-  long getNonHeapInit();
-
-  long getNonHeapMax();
-
-  long getNonHeapCommit();
-
-  long getNonHeapUsed();
-
-  Map<String, Double> measure();
-}
diff --git 
a/metrics/metrics-core/src/main/java/org/apache/servicecomb/metrics/core/monitor/TimerMonitor.java
 
b/metrics/metrics-core/src/main/java/org/apache/servicecomb/metrics/core/monitor/TimerMonitor.java
deleted file mode 100644
index 6c63bfe..0000000
--- 
a/metrics/metrics-core/src/main/java/org/apache/servicecomb/metrics/core/monitor/TimerMonitor.java
+++ /dev/null
@@ -1,108 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.servicecomb.metrics.core.monitor;
-
-import java.util.HashMap;
-import java.util.Map;
-
-import org.apache.servicecomb.foundation.common.concurrent.ConcurrentHashMapEx;
-import org.apache.servicecomb.foundation.metrics.MetricsConst;
-import org.apache.servicecomb.metrics.core.utils.MonitorUtils;
-
-import com.netflix.servo.monitor.MaxGauge;
-import com.netflix.servo.monitor.MonitorConfig;
-import com.netflix.servo.monitor.StepCounter;
-import com.netflix.servo.tag.Tags;
-
-public class TimerMonitor {
-  private final Map<String, StatusCounter> statusCounters;
-
-  private final String operation;
-
-  private final String stage;
-
-  private final String role;
-
-  public TimerMonitor(String operation, String stage, String role) {
-    this.operation = operation;
-    this.stage = stage;
-    this.role = role;
-
-    this.statusCounters = new ConcurrentHashMapEx<>();
-  }
-
-  public void update(long value, String statusCode) {
-    StatusCounter counter = statusCounters
-        .computeIfAbsent(statusCode, d -> new StatusCounter(operation, stage, 
role, statusCode));
-    counter.update(value);
-  }
-
-  public Map<String, Double> measure(int windowTimeIndex, boolean 
calculateLatency) {
-    Map<String, Double> measurements = new HashMap<>();
-    for (StatusCounter counter : statusCounters.values()) {
-      measurements.putAll(counter.measure(windowTimeIndex, calculateLatency));
-    }
-    return measurements;
-  }
-
-  class StatusCounter {
-    //nanosecond sum
-    private final StepCounter totalTime;
-
-    private final StepCounter count;
-
-    //nanosecond max
-    private final MaxGauge max;
-
-    private final MonitorConfig latency;
-
-    public StatusCounter(String operation, String stage, String role, String 
statusCode) {
-      MonitorConfig config = 
MonitorConfig.builder(MetricsConst.SERVICECOMB_INVOCATION)
-          .withTag(MetricsConst.TAG_STATUS, 
statusCode).withTag(MetricsConst.TAG_OPERATION, operation)
-          .withTag(MetricsConst.TAG_STAGE, 
stage).withTag(MetricsConst.TAG_ROLE, role).build();
-
-      this.latency = 
config.withAdditionalTag(Tags.newTag(MetricsConst.TAG_STATISTIC, "latency"));
-      this.totalTime = new 
StepCounter(config.withAdditionalTag(Tags.newTag(MetricsConst.TAG_STATISTIC, 
"totalTime")));
-      this.count = new 
StepCounter(config.withAdditionalTag(Tags.newTag(MetricsConst.TAG_STATISTIC, 
"count")));
-      this.max = new 
MaxGauge(config.withAdditionalTag(Tags.newTag(MetricsConst.TAG_STATISTIC, 
"max")));
-    }
-
-    public void update(long value) {
-      if (value > 0) {
-        totalTime.increment(value);
-        count.increment();
-        max.update(value);
-      }
-    }
-
-    public Map<String, Double> measure(int windowTimeIndex, boolean 
calculateLatency) {
-      Map<String, Double> measurements = new HashMap<>();
-      double totalTime = (double) MonitorUtils.convertNanosecondToMillisecond(
-          MonitorUtils.adjustValue(this.totalTime.getCount(windowTimeIndex)));
-      double count = (double) 
MonitorUtils.adjustValue(this.count.getCount(windowTimeIndex));
-      
measurements.put(MonitorUtils.getMonitorName(this.totalTime.getConfig()), 
totalTime);
-      measurements.put(MonitorUtils.getMonitorName(this.count.getConfig()), 
count);
-      measurements.put(MonitorUtils.getMonitorName(this.max.getConfig()), 
(double) MonitorUtils
-          
.convertNanosecondToMillisecond(MonitorUtils.adjustValue(this.max.getValue(windowTimeIndex))));
-      if (calculateLatency) {
-        measurements.put(MonitorUtils.getMonitorName(latency), totalTime / 
count);
-      }
-      return measurements;
-    }
-  }
-}
diff --git 
a/metrics/metrics-core/src/main/java/org/apache/servicecomb/metrics/core/publish/HealthCheckerPublisher.java
 
b/metrics/metrics-core/src/main/java/org/apache/servicecomb/metrics/core/publish/HealthCheckerPublisher.java
index 792eb36..8497936 100644
--- 
a/metrics/metrics-core/src/main/java/org/apache/servicecomb/metrics/core/publish/HealthCheckerPublisher.java
+++ 
b/metrics/metrics-core/src/main/java/org/apache/servicecomb/metrics/core/publish/HealthCheckerPublisher.java
@@ -24,16 +24,11 @@ import 
org.apache.servicecomb.foundation.common.utils.SPIServiceUtils;
 import org.apache.servicecomb.foundation.metrics.health.HealthCheckResult;
 import org.apache.servicecomb.foundation.metrics.health.HealthChecker;
 import org.apache.servicecomb.foundation.metrics.health.HealthCheckerManager;
-import 
org.apache.servicecomb.metrics.core.health.DefaultMicroserviceHealthChecker;
 import org.apache.servicecomb.provider.rest.common.RestSchema;
 import org.springframework.web.bind.annotation.CrossOrigin;
-import org.springframework.web.bind.annotation.PathVariable;
 import org.springframework.web.bind.annotation.RequestMapping;
 import org.springframework.web.bind.annotation.RequestMethod;
 
-import io.swagger.annotations.ApiResponse;
-import io.swagger.annotations.ApiResponses;
-
 @RestSchema(schemaId = "healthEndpoint")
 @RequestMapping(path = "/health")
 public class HealthCheckerPublisher {
@@ -50,8 +45,6 @@ public class HealthCheckerPublisher {
 
   private void init(HealthCheckerManager manager) {
     this.manager = manager;
-
-    this.manager.register(new DefaultMicroserviceHealthChecker());
     List<HealthChecker> checkers = 
SPIServiceUtils.getAllService(HealthChecker.class);
     for (HealthChecker checker : checkers) {
       this.manager.register(checker);
@@ -60,16 +53,19 @@ public class HealthCheckerPublisher {
 
   @RequestMapping(path = "/", method = RequestMethod.GET)
   @CrossOrigin
-  public Map<String, HealthCheckResult> health() {
-    return manager.check();
+  public boolean checkHealth() {
+    Map<String, HealthCheckResult> results = manager.check();
+    for (HealthCheckResult result : results.values()) {
+      if (!result.isHealthy()) {
+        return false;
+      }
+    }
+    return true;
   }
 
-  @ApiResponses({
-      @ApiResponse(code = 400, response = String.class, message = "illegal 
request content"),
-  })
-  @RequestMapping(path = "/{name}", method = RequestMethod.GET)
+  @RequestMapping(path = "/detail", method = RequestMethod.GET)
   @CrossOrigin
-  public HealthCheckResult healthWithName(@PathVariable(name = "name") String 
name) {
-    return manager.check(name);
+  public Map<String, HealthCheckResult> checkHealthDetail() {
+    return manager.check();
   }
 }
diff --git 
a/metrics/metrics-core/src/main/java/org/apache/servicecomb/metrics/core/publish/MetricsPublisher.java
 
b/metrics/metrics-core/src/main/java/org/apache/servicecomb/metrics/core/publish/MetricsPublisher.java
index 1369323..17bb613 100644
--- 
a/metrics/metrics-core/src/main/java/org/apache/servicecomb/metrics/core/publish/MetricsPublisher.java
+++ 
b/metrics/metrics-core/src/main/java/org/apache/servicecomb/metrics/core/publish/MetricsPublisher.java
@@ -19,9 +19,14 @@ package org.apache.servicecomb.metrics.core.publish;
 
 import java.util.Map;
 
-import org.apache.servicecomb.metrics.core.MetricsDataSource;
-import org.apache.servicecomb.metrics.core.event.EventListenerManager;
-import org.apache.servicecomb.metrics.core.monitor.RegistryMonitor;
+import org.apache.servicecomb.core.metrics.InvocationFinishedEvent;
+import org.apache.servicecomb.core.metrics.InvocationStartExecutionEvent;
+import org.apache.servicecomb.core.metrics.InvocationStartedEvent;
+import org.apache.servicecomb.foundation.common.utils.EventUtils;
+import org.apache.servicecomb.metrics.core.MonitorManager;
+import 
org.apache.servicecomb.metrics.core.event.InvocationFinishedEventListener;
+import 
org.apache.servicecomb.metrics.core.event.InvocationStartExecutionEventListener;
+import 
org.apache.servicecomb.metrics.core.event.InvocationStartedEventListener;
 import org.apache.servicecomb.provider.rest.common.RestSchema;
 import org.springframework.web.bind.annotation.CrossOrigin;
 import org.springframework.web.bind.annotation.RequestMapping;
@@ -35,7 +40,11 @@ import io.swagger.annotations.ApiResponses;
 public class MetricsPublisher {
 
   public MetricsPublisher() {
-    new EventListenerManager(RegistryMonitor.getInstance());
+    //init
+    EventUtils.registerEventListener(InvocationFinishedEvent.class, new 
InvocationFinishedEventListener());
+    EventUtils.registerEventListener(InvocationStartExecutionEvent.class, new 
InvocationStartExecutionEventListener());
+    EventUtils.registerEventListener(InvocationStartedEvent.class, new 
InvocationStartedEventListener());
+    MonitorManager.getInstance();
   }
 
   @ApiResponses({
@@ -43,7 +52,7 @@ public class MetricsPublisher {
   })
   @RequestMapping(path = "/", method = RequestMethod.GET)
   @CrossOrigin
-  public Map<String, Double> metrics() {
-    return MetricsDataSource.getInstance().measure();
+  public Map<String, Double> measure() {
+    return MonitorManager.getInstance().measure();
   }
 }
diff --git 
a/metrics/metrics-core/src/main/java/org/apache/servicecomb/metrics/core/utils/MonitorUtils.java
 
b/metrics/metrics-core/src/main/java/org/apache/servicecomb/metrics/core/utils/MonitorUtils.java
deleted file mode 100644
index 26183db..0000000
--- 
a/metrics/metrics-core/src/main/java/org/apache/servicecomb/metrics/core/utils/MonitorUtils.java
+++ /dev/null
@@ -1,53 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.servicecomb.metrics.core.utils;
-
-import java.util.concurrent.TimeUnit;
-
-import com.netflix.servo.monitor.MonitorConfig;
-import com.netflix.servo.tag.Tag;
-import com.netflix.servo.tag.TagList;
-
-public class MonitorUtils {
-
-  //for time-related monitor type, if stop poll value over one window time,
-  //the value may return -1 because servo can't known precise value of 
previous step
-  //so must change to return 0
-  public static long adjustValue(long value) {
-    return value < 0 ? 0 : value;
-  }
-
-  //Counting use System.nano get more precise time
-  //so we need change unit to millisecond when ouput
-  public static long convertNanosecondToMillisecond(long nanoValue) {
-    return TimeUnit.NANOSECONDS.toMillis(nanoValue);
-  }
-
-  public static String getMonitorName(MonitorConfig config) {
-    TagList tags = config.getTags();
-    StringBuilder tagPart = new StringBuilder("(");
-    for (Tag tag : tags) {
-      if (!"type".equals(tag.getKey())) {
-        tagPart.append(String.format("%s=%s,", tag.getKey(), tag.getValue()));
-      }
-    }
-    tagPart.deleteCharAt(tagPart.length() - 1);
-    tagPart.append(")");
-    return config.getName() + tagPart.toString();
-  }
-}
diff --git 
a/metrics/metrics-core/src/main/resources/META-INF/services/org.apache.servicecomb.metrics.core.monitor.SystemMonitor
 
b/metrics/metrics-core/src/main/resources/META-INF/services/org.apache.servicecomb.metrics.core.monitor.SystemMonitor
deleted file mode 100644
index 792668d..0000000
--- 
a/metrics/metrics-core/src/main/resources/META-INF/services/org.apache.servicecomb.metrics.core.monitor.SystemMonitor
+++ /dev/null
@@ -1,18 +0,0 @@
-#
-# Licensed to the Apache Software Foundation (ASF) under one or more
-# contributor license agreements.  See the NOTICE file distributed with
-# this work for additional information regarding copyright ownership.
-# The ASF licenses this file to You under the Apache License, Version 2.0
-# (the "License"); you may not use this file except in compliance with
-# the License.  You may obtain a copy of the License at
-#
-#     http://www.apache.org/licenses/LICENSE-2.0
-#
-# Unless required by applicable law or agreed to in writing, software
-# distributed under the License is distributed on an "AS IS" BASIS,
-# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-# See the License for the specific language governing permissions and
-# limitations under the License.
-#
-
-org.apache.servicecomb.metrics.core.monitor.DefaultSystemMonitor
\ No newline at end of file
diff --git 
a/metrics/metrics-core/src/test/java/org/apache/servicecomb/metrics/core/TestEventAndRunner.java
 
b/metrics/metrics-core/src/test/java/org/apache/servicecomb/metrics/core/TestEventAndRunner.java
index 00e02fd..d179202 100644
--- 
a/metrics/metrics-core/src/test/java/org/apache/servicecomb/metrics/core/TestEventAndRunner.java
+++ 
b/metrics/metrics-core/src/test/java/org/apache/servicecomb/metrics/core/TestEventAndRunner.java
@@ -17,103 +17,65 @@
 
 package org.apache.servicecomb.metrics.core;
 
-import static org.hamcrest.Matchers.containsInAnyOrder;
-import static org.mockito.Mockito.when;
-
-import java.lang.management.MemoryMXBean;
-import java.lang.management.MemoryUsage;
-import java.lang.management.OperatingSystemMXBean;
-import java.lang.management.ThreadMXBean;
-import java.util.Arrays;
-import java.util.List;
 import java.util.Map;
 import java.util.concurrent.TimeUnit;
 
 import org.apache.servicecomb.core.metrics.InvocationFinishedEvent;
-import org.apache.servicecomb.core.metrics.InvocationStartProcessingEvent;
+import org.apache.servicecomb.core.metrics.InvocationStartExecutionEvent;
 import org.apache.servicecomb.core.metrics.InvocationStartedEvent;
 import org.apache.servicecomb.foundation.common.utils.EventUtils;
 import org.apache.servicecomb.foundation.metrics.MetricsConst;
 import org.apache.servicecomb.foundation.metrics.publish.MetricNode;
 import org.apache.servicecomb.foundation.metrics.publish.MetricsLoader;
-import org.apache.servicecomb.metrics.core.event.EventListenerManager;
-import org.apache.servicecomb.metrics.core.monitor.DefaultSystemMonitor;
-import org.apache.servicecomb.metrics.core.monitor.RegistryMonitor;
+import 
org.apache.servicecomb.metrics.core.event.InvocationFinishedEventListener;
+import 
org.apache.servicecomb.metrics.core.event.InvocationStartExecutionEventListener;
+import 
org.apache.servicecomb.metrics.core.event.InvocationStartedEventListener;
 import org.apache.servicecomb.swagger.invocation.InvocationType;
 import org.junit.Assert;
 import org.junit.Test;
-import org.mockito.Mockito;
 
 public class TestEventAndRunner {
 
   @Test
   public void test() throws InterruptedException {
-    OperatingSystemMXBean systemMXBean = 
Mockito.mock(OperatingSystemMXBean.class);
-    when(systemMXBean.getSystemLoadAverage()).thenReturn(1.0);
-    ThreadMXBean threadMXBean = Mockito.mock(ThreadMXBean.class);
-    when(threadMXBean.getThreadCount()).thenReturn(2);
-    MemoryMXBean memoryMXBean = Mockito.mock(MemoryMXBean.class);
-    MemoryUsage heap = Mockito.mock(MemoryUsage.class);
-    when(memoryMXBean.getHeapMemoryUsage()).thenReturn(heap);
-    when(heap.getCommitted()).thenReturn(100L);
-    when(heap.getInit()).thenReturn(200L);
-    when(heap.getMax()).thenReturn(300L);
-    when(heap.getUsed()).thenReturn(400L);
-    MemoryUsage nonHeap = Mockito.mock(MemoryUsage.class);
-    when(memoryMXBean.getNonHeapMemoryUsage()).thenReturn(nonHeap);
-    when(nonHeap.getCommitted()).thenReturn(500L);
-    when(nonHeap.getInit()).thenReturn(600L);
-    when(nonHeap.getMax()).thenReturn(700L);
-    when(nonHeap.getUsed()).thenReturn(800L);
-
-    DefaultSystemMonitor systemMonitor = new 
DefaultSystemMonitor(systemMXBean, threadMXBean, memoryMXBean);
-    RegistryMonitor monitor = new RegistryMonitor(systemMonitor);
-    MetricsDataSource dataSource = new MetricsDataSource(monitor, 
"1000,2000,4000");
-
-    List<Long> intervals = dataSource.getAppliedWindowTime();
-    Assert.assertEquals(intervals.size(), 3);
-    Assert.assertThat(intervals, containsInAnyOrder(Arrays.asList(1000L, 
2000L, 4000L).toArray()));
-
-    new EventListenerManager(monitor);
+    //init
+    System.getProperties().setProperty(MetricsConfig.METRICS_WINDOW_TIME, 
"2000");
+    EventUtils.registerEventListener(InvocationFinishedEvent.class, new 
InvocationFinishedEventListener());
+    EventUtils.registerEventListener(InvocationStartExecutionEvent.class, new 
InvocationStartExecutionEventListener());
+    EventUtils.registerEventListener(InvocationStartedEvent.class, new 
InvocationStartedEventListener());
+    MonitorManager.getInstance();
 
     
//==========================================================================
     //fun1 is a PRODUCER invocation call 2 time and all is completed
     //two time success
     EventUtils.triggerEvent(new InvocationStartedEvent("fun1", 
InvocationType.PRODUCER, System.nanoTime()));
-    EventUtils.triggerEvent(
-        new InvocationStartProcessingEvent("fun1", InvocationType.PRODUCER));
+    EventUtils.triggerEvent(new InvocationStartExecutionEvent("fun1"));
     EventUtils
         .triggerEvent(new InvocationFinishedEvent("fun1", 
InvocationType.PRODUCER, TimeUnit.MILLISECONDS.toNanos(100),
             TimeUnit.MILLISECONDS.toNanos(200), 
TimeUnit.MILLISECONDS.toNanos(300), 200));
 
     EventUtils.triggerEvent(new InvocationStartedEvent("fun1", 
InvocationType.PRODUCER, System.nanoTime()));
-    EventUtils.triggerEvent(
-        new InvocationStartProcessingEvent("fun1", InvocationType.PRODUCER));
+    EventUtils.triggerEvent(new InvocationStartExecutionEvent("fun1"));
     EventUtils
         .triggerEvent(new InvocationFinishedEvent("fun1", 
InvocationType.PRODUCER, TimeUnit.MILLISECONDS.toNanos(300),
             TimeUnit.MILLISECONDS.toNanos(400), 
TimeUnit.MILLISECONDS.toNanos(700), 200));
 
     EventUtils.triggerEvent(new InvocationStartedEvent("fun1", 
InvocationType.PRODUCER, System.nanoTime()));
-    EventUtils.triggerEvent(
-        new InvocationStartProcessingEvent("fun1", InvocationType.PRODUCER));
+    EventUtils.triggerEvent(new InvocationStartExecutionEvent("fun1"));
     EventUtils
         .triggerEvent(new InvocationFinishedEvent("fun1", 
InvocationType.PRODUCER, TimeUnit.MILLISECONDS.toNanos(300),
             TimeUnit.MILLISECONDS.toNanos(400), 
TimeUnit.MILLISECONDS.toNanos(700), 500));
 
     
//==========================================================================
     //fun2 is a CONSUMER invocation call once and completed
-    EventUtils.triggerEvent(new InvocationStartedEvent("fun2", 
InvocationType.CONSUMER, System.nanoTime()));
-    EventUtils.triggerEvent(
-        new InvocationStartProcessingEvent("fun2", InvocationType.CONSUMER));
     EventUtils
-        .triggerEvent(new InvocationFinishedEvent("fun2", 
InvocationType.CONSUMER, TimeUnit.MILLISECONDS.toNanos(100),
-            TimeUnit.MILLISECONDS.toNanos(200), 
TimeUnit.MILLISECONDS.toNanos(300), 200));
+        .triggerEvent(new InvocationFinishedEvent("fun2", 
InvocationType.CONSUMER, 0, 0,
+            TimeUnit.MILLISECONDS.toNanos(300), 200));
 
     
//==========================================================================
     //fun3 is a PRODUCER invocation call uncompleted
     EventUtils.triggerEvent(new InvocationStartedEvent("fun3", 
InvocationType.PRODUCER, System.nanoTime()));
-    EventUtils.triggerEvent(
-        new InvocationStartProcessingEvent("fun3", InvocationType.PRODUCER));
+    EventUtils.triggerEvent(new InvocationStartExecutionEvent("fun3"));
 
     
//==========================================================================
     //fun4 is a PRODUCER call only started and no processing start and finished
@@ -121,90 +83,103 @@ public class TestEventAndRunner {
 
     
//==========================================================================
 
-    //sim at lease one window time
-    //increase up to 4 seconds in order to prevent low-probability test failed 
on ci
-    Thread.sleep(4000);
-
-    Map<String, Double> metrics = dataSource.measure(4000, true);
-
+    Map<String, Double> metrics = MonitorManager.getInstance().measure();
     MetricsLoader loader = new MetricsLoader(metrics);
-
     MetricNode node = loader
         .getMetricTree(MetricsConst.SERVICECOMB_INVOCATION, 
MetricsConst.TAG_OPERATION, MetricsConst.TAG_ROLE,
             MetricsConst.TAG_STAGE);
-
-    //check ProducerMetrics
+    //checkHealth ProducerMetrics
     //fun1
-    MetricNode node1_queue = 
node.getChildrenNode("fun1").getChildrenNode(MetricsConst.ROLE_PRODUCER)
+    MetricNode node1_queue = 
node.getChildrenNode("fun1").getChildrenNode(String.valueOf(InvocationType.PRODUCER))
         .getChildrenNode(MetricsConst.STAGE_QUEUE);
     Assert.assertEquals(0, 
node1_queue.getMatchStatisticMetricValue("waitInQueue"), 0);
     MetricNode node1_queue_status = new MetricNode(node1_queue.getMetrics(), 
MetricsConst.TAG_STATUS);
-    Assert.assertEquals(300, 
node1_queue_status.getChildrenNode("200").getMatchStatisticMetricValue("max"), 
0);
-    Assert.assertEquals(2, 
node1_queue_status.getChildrenNode("200").getMatchStatisticMetricValue("count"),
 0);
-    Assert.assertEquals(400, 
node1_queue_status.getChildrenNode("200").getMatchStatisticMetricValue("totalTime"),
 0);
-    Assert.assertEquals(300, 
node1_queue_status.getChildrenNode("500").getMatchStatisticMetricValue("max"), 
0);
-    Assert.assertEquals(1, 
node1_queue_status.getChildrenNode("500").getMatchStatisticMetricValue("count"),
 0);
-    Assert.assertEquals(300, 
node1_queue_status.getChildrenNode("500").getMatchStatisticMetricValue("totalTime"),
 0);
-
-    MetricNode node1_exec = 
node.getChildrenNode("fun1").getChildrenNode(MetricsConst.ROLE_PRODUCER)
+    Assert.assertEquals(200,
+        
node1_queue_status.getChildrenNode("200").getMatchStatisticMetricValue(TimeUnit.MILLISECONDS,
 "latency"), 0);
+    Assert.assertEquals(300,
+        
node1_queue_status.getChildrenNode("500").getMatchStatisticMetricValue(TimeUnit.MILLISECONDS,
 "latency"), 0);
+
+    MetricNode node1_exec = 
node.getChildrenNode("fun1").getChildrenNode(String.valueOf(InvocationType.PRODUCER))
         .getChildrenNode(MetricsConst.STAGE_EXECUTION);
     MetricNode node1_exec_status = new MetricNode(node1_exec.getMetrics(), 
MetricsConst.TAG_STATUS);
-    Assert.assertEquals(400, 
node1_exec_status.getChildrenNode("200").getMatchStatisticMetricValue("max"), 
0);
-    Assert.assertEquals(2, 
node1_exec_status.getChildrenNode("200").getMatchStatisticMetricValue("count"), 
0);
-    Assert.assertEquals(600, 
node1_exec_status.getChildrenNode("200").getMatchStatisticMetricValue("totalTime"),
 0);
-    Assert.assertEquals(400, 
node1_exec_status.getChildrenNode("500").getMatchStatisticMetricValue("max"), 
0);
-    Assert.assertEquals(1, 
node1_exec_status.getChildrenNode("500").getMatchStatisticMetricValue("count"), 
0);
-    Assert.assertEquals(400, 
node1_exec_status.getChildrenNode("500").getMatchStatisticMetricValue("totalTime"),
 0);
-
-    MetricNode node1_whole = 
node.getChildrenNode("fun1").getChildrenNode(MetricsConst.ROLE_PRODUCER)
-        .getChildrenNode(MetricsConst.STAGE_WHOLE);
+    Assert.assertEquals(300,
+        
node1_exec_status.getChildrenNode("200").getMatchStatisticMetricValue(TimeUnit.MILLISECONDS,
 "latency"), 0);
+    Assert.assertEquals(400,
+        
node1_exec_status.getChildrenNode("500").getMatchStatisticMetricValue(TimeUnit.MILLISECONDS,
 "latency"), 0);
+
+    MetricNode node1_whole = 
node.getChildrenNode("fun1").getChildrenNode(String.valueOf(InvocationType.PRODUCER))
+        .getChildrenNode(MetricsConst.STAGE_TOTAL);
     MetricNode node1_whole_status = new MetricNode(node1_whole.getMetrics(), 
MetricsConst.TAG_STATUS);
-    Assert.assertEquals(700, 
node1_whole_status.getChildrenNode("200").getMatchStatisticMetricValue("max"), 
0);
+    Assert.assertEquals(500,
+        
node1_whole_status.getChildrenNode("200").getMatchStatisticMetricValue(TimeUnit.MILLISECONDS,
 "latency"), 0);
+    Assert.assertEquals(700,
+        
node1_whole_status.getChildrenNode("500").getMatchStatisticMetricValue(TimeUnit.MILLISECONDS,
 "latency"), 0);
     Assert.assertEquals(2, 
node1_whole_status.getChildrenNode("200").getMatchStatisticMetricValue("count"),
 0);
-    Assert.assertEquals(1000, 
node1_whole_status.getChildrenNode("200").getMatchStatisticMetricValue("totalTime"),
 0);
-    Assert.assertEquals(700, 
node1_whole_status.getChildrenNode("500").getMatchStatisticMetricValue("max"), 
0);
     Assert.assertEquals(1, 
node1_whole_status.getChildrenNode("500").getMatchStatisticMetricValue("count"),
 0);
-    Assert.assertEquals(700, 
node1_whole_status.getChildrenNode("500").getMatchStatisticMetricValue("totalTime"),
 0);
-    Assert.assertEquals(0.5, 
node1_whole_status.getChildrenNode("200").getMatchStatisticMetricValue("tps"), 
0);
-    Assert.assertEquals(2, 
node1_whole_status.getChildrenNode("200").getMatchStatisticMetricValue("totalCount"),
 0);
-    Assert.assertEquals(0.25, 
node1_whole_status.getChildrenNode("500").getMatchStatisticMetricValue("tps"), 
0);
-    Assert.assertEquals(1, 
node1_whole_status.getChildrenNode("500").getMatchStatisticMetricValue("totalCount"),
 0);
 
-    //check ConsumerMetrics
+    //checkHealth ConsumerMetrics
     //fun2
-    MetricNode node2_whole = 
node.getChildrenNode("fun2").getChildrenNode(MetricsConst.ROLE_CONSUMER)
-        .getChildrenNode(MetricsConst.STAGE_WHOLE);
+    MetricNode node2_whole = 
node.getChildrenNode("fun2").getChildrenNode(String.valueOf(InvocationType.CONSUMER))
+        .getChildrenNode(MetricsConst.STAGE_TOTAL);
     MetricNode node2_whole_status = new MetricNode(node2_whole.getMetrics(), 
MetricsConst.TAG_STATUS);
-    Assert.assertEquals(300, 
node2_whole_status.getChildrenNode("200").getMatchStatisticMetricValue("max"), 
0);
+    Assert.assertEquals(300,
+        
node2_whole_status.getChildrenNode("200").getMatchStatisticMetricValue(TimeUnit.MILLISECONDS,
 "latency"), 0);
     Assert.assertEquals(1, 
node2_whole_status.getChildrenNode("200").getMatchStatisticMetricValue("count"),
 0);
-    Assert.assertEquals(300, 
node2_whole_status.getChildrenNode("200").getMatchStatisticMetricValue("totalTime"),
 0);
-    Assert.assertEquals(0.25, 
node2_whole_status.getChildrenNode("200").getMatchStatisticMetricValue("tps"), 
0);
-    Assert.assertEquals(1, 
node2_whole_status.getChildrenNode("200").getMatchStatisticMetricValue("totalCount"),
 0);
 
     //fun3
-    MetricNode node3_queue = 
node.getChildrenNode("fun3").getChildrenNode(MetricsConst.ROLE_PRODUCER)
+    MetricNode node3_queue = 
node.getChildrenNode("fun3").getChildrenNode(String.valueOf(InvocationType.PRODUCER))
         .getChildrenNode(MetricsConst.STAGE_QUEUE);
     Assert.assertEquals(0, 
node3_queue.getMatchStatisticMetricValue("waitInQueue"), 0);
 
     //fun4
-    MetricNode node4_queue = 
node.getChildrenNode("fun4").getChildrenNode(MetricsConst.ROLE_PRODUCER)
+    MetricNode node4_queue = 
node.getChildrenNode("fun4").getChildrenNode(String.valueOf(InvocationType.PRODUCER))
         .getChildrenNode(MetricsConst.STAGE_QUEUE);
     Assert.assertEquals(1, 
node4_queue.getMatchStatisticMetricValue("waitInQueue"), 0);
 
-    //System metrics
-    Assert.assertEquals(1.0, getSystemMetric(loader, "cpuLoad"), 0);
-    Assert.assertEquals(2, getSystemMetric(loader, "cpuRunningThreads"), 0);
-    Assert.assertEquals(100, getSystemMetric(loader, "heapCommit"), 0);
-    Assert.assertEquals(200, getSystemMetric(loader, "heapInit"), 0);
-    Assert.assertEquals(300, getSystemMetric(loader, "heapMax"), 0);
-    Assert.assertEquals(400, getSystemMetric(loader, "heapUsed"), 0);
-    Assert.assertEquals(500, getSystemMetric(loader, "nonHeapCommit"), 0);
-    Assert.assertEquals(600, getSystemMetric(loader, "nonHeapInit"), 0);
-    Assert.assertEquals(700, getSystemMetric(loader, "nonHeapMax"), 0);
-    Assert.assertEquals(800, getSystemMetric(loader, "nonHeapUsed"), 0);
-  }
+    //sim at lease one window time
+    Thread.sleep(2000);
+
+    metrics = MonitorManager.getInstance().measure();
+    loader = new MetricsLoader(metrics);
+    node = loader
+        .getMetricTree(MetricsConst.SERVICECOMB_INVOCATION, 
MetricsConst.TAG_OPERATION, MetricsConst.TAG_ROLE,
+            MetricsConst.TAG_STAGE);
+
+    node1_queue = 
node.getChildrenNode("fun1").getChildrenNode(String.valueOf(InvocationType.PRODUCER))
+        .getChildrenNode(MetricsConst.STAGE_QUEUE);
+    node1_queue_status = new MetricNode(node1_queue.getMetrics(), 
MetricsConst.TAG_STATUS);
+    Assert.assertEquals(300,
+        
node1_queue_status.getChildrenNode("200").getMatchStatisticMetricValue(TimeUnit.MILLISECONDS,
 "max"), 0);
+    Assert.assertEquals(300,
+        
node1_queue_status.getChildrenNode("500").getMatchStatisticMetricValue(TimeUnit.MILLISECONDS,
 "max"), 0);
+
+    node1_exec = 
node.getChildrenNode("fun1").getChildrenNode(String.valueOf(InvocationType.PRODUCER))
+        .getChildrenNode(MetricsConst.STAGE_EXECUTION);
+    node1_exec_status = new MetricNode(node1_exec.getMetrics(), 
MetricsConst.TAG_STATUS);
+    Assert.assertEquals(400,
+        
node1_exec_status.getChildrenNode("200").getMatchStatisticMetricValue(TimeUnit.MILLISECONDS,
 "max"), 0);
+    Assert.assertEquals(400,
+        
node1_exec_status.getChildrenNode("500").getMatchStatisticMetricValue(TimeUnit.MILLISECONDS,
 "max"), 0);
+
+    node1_whole = 
node.getChildrenNode("fun1").getChildrenNode(String.valueOf(InvocationType.PRODUCER))
+        .getChildrenNode(MetricsConst.STAGE_TOTAL);
+    node1_whole_status = new MetricNode(node1_whole.getMetrics(), 
MetricsConst.TAG_STATUS);
+    Assert.assertEquals(700,
+        
node1_whole_status.getChildrenNode("200").getMatchStatisticMetricValue(TimeUnit.MILLISECONDS,
 "max"), 0);
+    Assert.assertEquals(700,
+        
node1_whole_status.getChildrenNode("500").getMatchStatisticMetricValue(TimeUnit.MILLISECONDS,
 "max"), 0);
+
+    Assert.assertEquals(1, 
node1_whole_status.getChildrenNode("200").getMatchStatisticMetricValue("tps"), 
0);
+    Assert.assertEquals(0.5, 
node1_whole_status.getChildrenNode("500").getMatchStatisticMetricValue("tps"), 
0);
+
+    //checkHealth ConsumerMetrics
+    //fun2
+    node2_whole = 
node.getChildrenNode("fun2").getChildrenNode(String.valueOf(InvocationType.CONSUMER))
+        .getChildrenNode(MetricsConst.STAGE_TOTAL);
+    node2_whole_status = new MetricNode(node2_whole.getMetrics(), 
MetricsConst.TAG_STATUS);
+    Assert.assertEquals(300,
+        
node2_whole_status.getChildrenNode("200").getMatchStatisticMetricValue(TimeUnit.MILLISECONDS,
 "max"), 0);
 
-  private Double getSystemMetric(MetricsLoader loader, String name) {
-    return loader.getFirstMatchMetricValue(MetricsConst.JVM, 
MetricsConst.TAG_NAME, name);
+    Assert.assertEquals(0.5, 
node2_whole_status.getChildrenNode("200").getMatchStatisticMetricValue("tps"), 
0);
   }
 }
\ No newline at end of file
diff --git 
a/metrics/metrics-core/src/test/java/org/apache/servicecomb/metrics/core/TestHealthCheckerPublisher.java
 
b/metrics/metrics-core/src/test/java/org/apache/servicecomb/metrics/core/TestHealthCheckerPublisher.java
index 3585193..a29bac6 100644
--- 
a/metrics/metrics-core/src/test/java/org/apache/servicecomb/metrics/core/TestHealthCheckerPublisher.java
+++ 
b/metrics/metrics-core/src/test/java/org/apache/servicecomb/metrics/core/TestHealthCheckerPublisher.java
@@ -17,86 +17,52 @@
 
 package org.apache.servicecomb.metrics.core;
 
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-
-import java.io.IOException;
-import java.util.HashMap;
 import java.util.Map;
 
-import org.apache.servicecomb.foundation.common.utils.JsonUtils;
 import org.apache.servicecomb.foundation.metrics.health.HealthCheckResult;
 import org.apache.servicecomb.foundation.metrics.health.HealthChecker;
 import org.apache.servicecomb.foundation.metrics.health.HealthCheckerManager;
-import org.apache.servicecomb.metrics.core.health.DefaultHealthCheckExtraData;
-import 
org.apache.servicecomb.metrics.core.health.DefaultMicroserviceHealthChecker;
 import org.apache.servicecomb.metrics.core.publish.HealthCheckerPublisher;
-import org.apache.servicecomb.serviceregistry.RegistryUtils;
-import org.apache.servicecomb.serviceregistry.api.registry.Microservice;
-import 
org.apache.servicecomb.serviceregistry.api.registry.MicroserviceInstance;
 import org.junit.Assert;
 import org.junit.Test;
 
-import com.google.common.collect.Lists;
-
-import mockit.Expectations;
-
 public class TestHealthCheckerPublisher {
 
   @Test
-  public void testPublisher() throws IOException {
-    Microservice microservice = new Microservice();
-    microservice.setAppId("appId");
-    microservice.setServiceName("serviceName");
-    microservice.setVersion("0.0.1");
-
-    MicroserviceInstance microserviceInstance = new MicroserviceInstance();
-    microserviceInstance.setEndpoints(Lists.newArrayList("127.0.0.1", 
"192.168.0.100"));
-    microserviceInstance.setInstanceId("001");
-    microserviceInstance.setHostName("localhost");
-
-    new Expectations(RegistryUtils.class) {
-      {
-        RegistryUtils.getMicroservice();
-        result = microservice;
+  public void test() {
+    HealthCheckerManager manager = new HealthCheckerManager();
+    manager.register(new HealthChecker() {
+      @Override
+      public String getName() {
+        return "test";
       }
-    };
 
-    new Expectations(RegistryUtils.class) {
-      {
-        RegistryUtils.getMicroserviceInstance();
-        result = microserviceInstance;
+      @Override
+      public HealthCheckResult check() {
+        return new HealthCheckResult(true, "info", "extra data");
       }
-    };
+    });
 
-    HealthCheckerManager manager = mock(HealthCheckerManager.class);
-
-    Map<String, HealthCheckResult> results = new HashMap<>();
-    HealthChecker result = new DefaultMicroserviceHealthChecker();
-    results.put("default", result.check());
+    manager.register(new HealthChecker() {
+      @Override
+      public String getName() {
+        return "test2";
+      }
 
-    when(manager.check()).thenReturn(results);
-    when(manager.check("default")).thenReturn(result.check());
+      @Override
+      public HealthCheckResult check() {
+        return new HealthCheckResult(false, "info2", "extra data 2");
+      }
+    });
 
     HealthCheckerPublisher publisher = new HealthCheckerPublisher(manager);
-    Map<String, HealthCheckResult> content = publisher.health();
 
-    DefaultHealthCheckExtraData data = JsonUtils.OBJ_MAPPER
-        .readValue(content.get("default").getExtraData(), 
DefaultHealthCheckExtraData.class);
-    Assert.assertEquals("appId", data.getAppId());
-    Assert.assertEquals("serviceName", data.getServiceName());
-    Assert.assertEquals("0.0.1", data.getServiceVersion());
-    Assert.assertEquals("001", data.getInstanceId());
-    Assert.assertEquals("localhost", data.getHostName());
-    Assert.assertEquals("127.0.0.1,192.168.0.100", data.getEndpoints());
+    Assert.assertEquals(false, publisher.checkHealth());
+
+    Map<String, HealthCheckResult> content = publisher.checkHealthDetail();
 
-    data = JsonUtils.OBJ_MAPPER
-        .readValue(publisher.healthWithName("default").getExtraData(), 
DefaultHealthCheckExtraData.class);
-    Assert.assertEquals("appId", data.getAppId());
-    Assert.assertEquals("serviceName", data.getServiceName());
-    Assert.assertEquals("0.0.1", data.getServiceVersion());
-    Assert.assertEquals("001", data.getInstanceId());
-    Assert.assertEquals("localhost", data.getHostName());
-    Assert.assertEquals("127.0.0.1,192.168.0.100", data.getEndpoints());
+    Assert.assertEquals(true, content.get("test").isHealthy());
+    Assert.assertEquals("info", content.get("test").getInformation());
+    Assert.assertEquals("extra data", content.get("test").getExtraData());
   }
 }
diff --git 
a/metrics/metrics-core/src/test/java/org/apache/servicecomb/metrics/core/TestMetricsPublisher.java
 
b/metrics/metrics-core/src/test/java/org/apache/servicecomb/metrics/core/TestMetricsPublisher.java
index 48211ed..c310dd6 100644
--- 
a/metrics/metrics-core/src/test/java/org/apache/servicecomb/metrics/core/TestMetricsPublisher.java
+++ 
b/metrics/metrics-core/src/test/java/org/apache/servicecomb/metrics/core/TestMetricsPublisher.java
@@ -27,7 +27,7 @@ public class TestMetricsPublisher {
   @Test
   public void test() {
     MetricsPublisher publisher = new MetricsPublisher();
-    Map<String, Double> metrics = publisher.metrics();
+    Map<String, Double> metrics = publisher.measure();
     //10 jvm metrics get
     Assert.assertEquals(10, metrics.size());
   }
diff --git 
a/metrics/metrics-integration/metrics-prometheus/src/main/java/org/apache/servicecomb/metrics/prometheus/MetricsCollector.java
 
b/metrics/metrics-integration/metrics-prometheus/src/main/java/org/apache/servicecomb/metrics/prometheus/MetricsCollector.java
index c3bf275..a99cce4 100644
--- 
a/metrics/metrics-integration/metrics-prometheus/src/main/java/org/apache/servicecomb/metrics/prometheus/MetricsCollector.java
+++ 
b/metrics/metrics-integration/metrics-prometheus/src/main/java/org/apache/servicecomb/metrics/prometheus/MetricsCollector.java
@@ -23,7 +23,7 @@ import java.util.Map;
 import java.util.Map.Entry;
 
 import org.apache.servicecomb.foundation.metrics.MetricsConst;
-import org.apache.servicecomb.metrics.core.MetricsDataSource;
+import org.apache.servicecomb.metrics.core.MonitorManager;
 
 import io.prometheus.client.Collector;
 import io.prometheus.client.Collector.MetricFamilySamples.Sample;
@@ -41,12 +41,11 @@ public class MetricsCollector extends Collector implements 
Collector.Describable
   }
 
   private List<MetricFamilySamples> load() {
-    Map<String, Double> registryMetric = MetricsDataSource.getInstance()
-        
.measure(MetricsDataSource.getInstance().getAppliedWindowTime().get(0), true);
+    Map<String, Double> metrics = MonitorManager.getInstance().measure();
     List<MetricFamilySamples> familySamples = new ArrayList<>();
 
     List<Sample> samples = new ArrayList<>();
-    for (Entry<String, Double> metric : registryMetric.entrySet()) {
+    for (Entry<String, Double> metric : metrics.entrySet()) {
       List<String> tagNames = new ArrayList<>();
       List<String> tagValues = new ArrayList<>();
       String name = metric.getKey();
diff --git 
a/samples/metrics-write-file-sample/metrics-write-file/src/main/java/org/apache/servicecomb/samples/mwf/WriteFileInitializer.java
 
b/samples/metrics-write-file-sample/metrics-write-file/src/main/java/org/apache/servicecomb/samples/mwf/WriteFileInitializer.java
index a91ced6..d5486b8 100644
--- 
a/samples/metrics-write-file-sample/metrics-write-file/src/main/java/org/apache/servicecomb/samples/mwf/WriteFileInitializer.java
+++ 
b/samples/metrics-write-file-sample/metrics-write-file/src/main/java/org/apache/servicecomb/samples/mwf/WriteFileInitializer.java
@@ -25,7 +25,7 @@ import java.util.concurrent.Executors;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.servicecomb.foundation.common.net.NetUtils;
 import org.apache.servicecomb.metrics.core.MetricsConfig;
-import org.apache.servicecomb.metrics.core.MetricsDataSource;
+import org.apache.servicecomb.metrics.core.MonitorManager;
 import org.apache.servicecomb.serviceregistry.RegistryUtils;
 import org.apache.servicecomb.serviceregistry.api.registry.Microservice;
 import org.springframework.beans.factory.annotation.Autowired;
@@ -49,13 +49,13 @@ public class WriteFileInitializer {
 
   @Autowired
   public WriteFileInitializer(MetricsFileWriter fileWriter) {
-    metricPoll = 
DynamicPropertyFactory.getInstance().getIntProperty(MetricsConfig.METRICS_POLLING_TIME,
 5000).get();
+    metricPoll = 
DynamicPropertyFactory.getInstance().getIntProperty(MetricsConfig.METRICS_WINDOW_TIME,
 5000).get();
     this.fileWriter = fileWriter;
     this.convertor = new SimpleFileContentConvertor();
   }
 
   public WriteFileInitializer(MetricsFileWriter fileWriter, String hostName, 
String filePrefix) {
-    metricPoll = 
DynamicPropertyFactory.getInstance().getIntProperty(MetricsConfig.METRICS_POLLING_TIME,
 5000).get();
+    metricPoll = 
DynamicPropertyFactory.getInstance().getIntProperty(MetricsConfig.METRICS_WINDOW_TIME,
 5000).get();
     this.fileWriter = fileWriter;
     this.hostName = hostName;
     this.filePrefix = filePrefix;
@@ -83,9 +83,8 @@ public class WriteFileInitializer {
   }
 
   private void run() {
-    Map<String, Double> registryMetric = MetricsDataSource.getInstance()
-        
.measure(MetricsDataSource.getInstance().getAppliedWindowTime().get(0), true);
-    Map<String, String> convertedMetrics = convertor.convert(registryMetric);
+    Map<String, Double> metrics = MonitorManager.getInstance().measure();
+    Map<String, String> convertedMetrics = convertor.convert(metrics);
     Map<String, String> formattedMetrics = formatter.format(convertedMetrics);
 
     for (String metricName : formattedMetrics.keySet()) {
diff --git a/tracing/tracing-zipkin/pom.xml b/tracing/tracing-zipkin/pom.xml
index 70b3314..6910faf 100644
--- a/tracing/tracing-zipkin/pom.xml
+++ b/tracing/tracing-zipkin/pom.xml
@@ -27,6 +27,7 @@
   <modelVersion>4.0.0</modelVersion>
 
   <artifactId>tracing-zipkin</artifactId>
+  <name>Java Chassis::Tracing::Zipkin</name>
 
   <dependencies>
     <dependency>
diff --git 
a/transports/transport-highway/src/main/java/org/apache/servicecomb/transport/highway/HighwayServerInvoke.java
 
b/transports/transport-highway/src/main/java/org/apache/servicecomb/transport/highway/HighwayServerInvoke.java
index 37bdcfc..d340aa4 100644
--- 
a/transports/transport-highway/src/main/java/org/apache/servicecomb/transport/highway/HighwayServerInvoke.java
+++ 
b/transports/transport-highway/src/main/java/org/apache/servicecomb/transport/highway/HighwayServerInvoke.java
@@ -125,7 +125,7 @@ public class HighwayServerInvoke {
     invocation.getHandlerContext().put(Const.REMOTE_ADDRESS, 
this.connection.getNetSocket().remoteAddress());
     //立刻设置开始时间,否则Finished时无法计算TotalTime
     invocation.setStartTime(startedEvent.getStartedTime());
-    invocation.triggerStartProcessingEvent();
+    invocation.triggerStartExecutionEvent();
 
     invocation.next(response -> {
       sendResponse(invocation.getContext(), response);

-- 
To stop receiving notification emails like this one, please contact
ningji...@apache.org.

Reply via email to