You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@skywalking.apache.org by wu...@apache.org on 2018/10/28 14:28:25 UTC

[incubator-skywalking] branch master updated: Revert #1790 reset feature and provide default alarm settings. (#1833)

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

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


The following commit(s) were added to refs/heads/master by this push:
     new 8cd327d  Revert #1790 reset feature and provide default alarm settings. (#1833)
8cd327d is described below

commit 8cd327d8bba0853571ff2b9f323512bac6cff887
Author: 吴晟 Wu Sheng <wu...@foxmail.com>
AuthorDate: Sun Oct 28 22:28:21 2018 +0800

    Revert #1790 reset feature and provide default alarm settings. (#1833)
    
    * Revert reset feature.
    
    * Set default dist alarm settings and make dist package setting files right.
    
    * Support empty list.
---
 .../apm/commons/datacarrier/DataCarrier.java       |   6 -
 .../apm/commons/datacarrier/buffer/Buffer.java     |   6 -
 .../skywalking/apm/agent/core/conf/Config.java     |  18 +--
 .../agent/core/conf/SnifferConfigInitializer.java  |  23 +--
 .../core/dictionary/NetworkAddressDictionary.java  |   7 +-
 .../core/dictionary/OperationNameDictionary.java   |  11 --
 .../apm/agent/core/listener/ResetConfListener.java |  84 -----------
 .../apm/agent/core/listener/ResetStatus.java       |  35 -----
 .../apm/agent/core/listener/Reseter.java           | 159 ---------------------
 .../core/remote/AppAndServiceRegisterClient.java   |  49 +++----
 .../core/remote/TraceSegmentServiceClient.java     |   9 +-
 ...ache.skywalking.apm.agent.core.boot.BootService |   1 -
 .../apm/agent/core/boot/ServiceManagerTest.java    |   2 +-
 apm-sniffer/config/agent.config                    |   9 --
 .../server/core/alarm/provider/RunningRule.java    |  31 +++-
 .../service/ServiceInstanceInventoryRegister.java  |   2 +-
 .../register/provider/RegisterModuleProvider.java  |   1 -
 .../handler/v5/ServiceInstancePingPkgHandler.java  |  60 --------
 oap-server/server-starter/pom.xml                  |   2 +
 .../src/main/assembly/alarm-settings.yml           |  44 ++++--
 20 files changed, 94 insertions(+), 465 deletions(-)

diff --git a/apm-commons/apm-datacarrier/src/main/java/org/apache/skywalking/apm/commons/datacarrier/DataCarrier.java b/apm-commons/apm-datacarrier/src/main/java/org/apache/skywalking/apm/commons/datacarrier/DataCarrier.java
index bd9f9a6..76ad609 100644
--- a/apm-commons/apm-datacarrier/src/main/java/org/apache/skywalking/apm/commons/datacarrier/DataCarrier.java
+++ b/apm-commons/apm-datacarrier/src/main/java/org/apache/skywalking/apm/commons/datacarrier/DataCarrier.java
@@ -152,10 +152,4 @@ public class DataCarrier<T> {
             consumerPool.close();
         }
     }
-
-    public void clear() {
-        for (int i = 0; i < channels.getChannelSize(); i++) {
-            channels.getBuffer(i).clear();
-        }
-    }
 }
diff --git a/apm-commons/apm-datacarrier/src/main/java/org/apache/skywalking/apm/commons/datacarrier/buffer/Buffer.java b/apm-commons/apm-datacarrier/src/main/java/org/apache/skywalking/apm/commons/datacarrier/buffer/Buffer.java
index d9cbb34..4fbb478 100644
--- a/apm-commons/apm-datacarrier/src/main/java/org/apache/skywalking/apm/commons/datacarrier/buffer/Buffer.java
+++ b/apm-commons/apm-datacarrier/src/main/java/org/apache/skywalking/apm/commons/datacarrier/buffer/Buffer.java
@@ -91,10 +91,4 @@ public class Buffer<T> {
         return result;
     }
 
-    public void clear() {
-        for (Object obj : buffer) {
-            obj = null;
-        }
-    }
-
 }
diff --git a/apm-sniffer/apm-agent-core/src/main/java/org/apache/skywalking/apm/agent/core/conf/Config.java b/apm-sniffer/apm-agent-core/src/main/java/org/apache/skywalking/apm/agent/core/conf/Config.java
index c5e305b..dfdea53 100644
--- a/apm-sniffer/apm-agent-core/src/main/java/org/apache/skywalking/apm/agent/core/conf/Config.java
+++ b/apm-sniffer/apm-agent-core/src/main/java/org/apache/skywalking/apm/agent/core/conf/Config.java
@@ -16,6 +16,7 @@
  *
  */
 
+
 package org.apache.skywalking.apm.agent.core.conf;
 
 import org.apache.skywalking.apm.agent.core.context.trace.TraceSegment;
@@ -69,21 +70,6 @@ public class Config {
          * Skywalking team may ask for these files in order to resolve compatible problem.
          */
         public static boolean IS_OPEN_DEBUGGING_CLASS = false;
-
-        /**
-         * Specify register.status dir ,This is an option, the default is AGENT_HOME/option/reset.status.
-         */
-        public static String REGISTER_STATUS_DIR = "skywalking-agent/option";
-
-        /**
-         * Specify instance_uuid to ensure that the whole show is unique, for example: applicationName_ip_12
-         */
-        public static String INSTANCE_UUID = "";
-        /**
-         * enabled means that the reset function is enabled, and disabled means that the reset function is not enabled. A reset can be triggered by modifying the configuration file only if the reset feature is enabled.
-         */
-        public static String RESETER_LISTENER = "disabled";
-
     }
 
     public static class Collector {
@@ -94,7 +80,7 @@ public class Config {
         /**
          * application and service registry check interval
          */
-        public static long SERVICE_AND_ENDPOINT_REGISTER_CHECK_INTERVAL = 3;
+        public static long APP_AND_SERVICE_REGISTER_CHECK_INTERVAL = 3;
         /**
          * Collector skywalking trace receiver service addresses.
          */
diff --git a/apm-sniffer/apm-agent-core/src/main/java/org/apache/skywalking/apm/agent/core/conf/SnifferConfigInitializer.java b/apm-sniffer/apm-agent-core/src/main/java/org/apache/skywalking/apm/agent/core/conf/SnifferConfigInitializer.java
index 3d8ef63..d33d304 100644
--- a/apm-sniffer/apm-agent-core/src/main/java/org/apache/skywalking/apm/agent/core/conf/SnifferConfigInitializer.java
+++ b/apm-sniffer/apm-agent-core/src/main/java/org/apache/skywalking/apm/agent/core/conf/SnifferConfigInitializer.java
@@ -41,23 +41,24 @@ import org.apache.skywalking.apm.util.StringUtil;
  */
 public class SnifferConfigInitializer {
     private static final ILog logger = LogManager.getLogger(SnifferConfigInitializer.class);
-    private static final String SPECIFIED_CONFIG_PATH = "skywalking_config";
-    private static final String DEFAULT_CONFIG_FILE_NAME = "/config/agent.config";
-    private static final String ENV_KEY_PREFIX = "skywalking.";
-    private static final String INSTANCE_UUID_NAME = "agent.instance_uuid";
-    private static final String REGISTER_STATUS_DIR = "agent.register_status_dir";
+    private static String SPECIFIED_CONFIG_PATH = "skywalking_config";
+    private static String DEFAULT_CONFIG_FILE_NAME = "/config/agent.config";
+    private static String ENV_KEY_PREFIX = "skywalking.";
     private static boolean IS_INIT_COMPLETED = false;
 
     /**
-     * If the specified agent config path is set, the agent will try to locate the specified agent config. If the
-     * specified agent config path is not set , the agent will try to locate `agent.config`, which should be in the
-     * /config dictionary of agent package. <p> Also try to override the config by system.env and system.properties. All
-     * the keys in these two places should start with {@link #ENV_KEY_PREFIX}. e.g. in env
-     * `skywalking.agent.application_code=yourAppName` to override `agent.application_code` in config file. <p> At the
-     * end, `agent.application_code` and `collector.servers` must be not blank.
+     * If the specified agent config path is set, the agent will try to locate the specified agent config.
+     * If the specified agent config path is not set , the agent will try to locate `agent.config`, which should be in the /config dictionary of agent package.
+     * <p>
+     * Also try to override the config by system.env and system.properties. All the keys in these two places should
+     * start with {@link #ENV_KEY_PREFIX}. e.g. in env `skywalking.agent.application_code=yourAppName` to override
+     * `agent.application_code` in config file.
+     * <p>
+     * At the end, `agent.application_code` and `collector.servers` must be not blank.
      */
     public static void initialize() throws ConfigNotFoundException, AgentPackageNotFoundException {
         InputStreamReader configFileStream;
+
         try {
             configFileStream = loadConfig();
             Properties properties = new Properties();
diff --git a/apm-sniffer/apm-agent-core/src/main/java/org/apache/skywalking/apm/agent/core/dictionary/NetworkAddressDictionary.java b/apm-sniffer/apm-agent-core/src/main/java/org/apache/skywalking/apm/agent/core/dictionary/NetworkAddressDictionary.java
index ca4d3b5..208ba17 100644
--- a/apm-sniffer/apm-agent-core/src/main/java/org/apache/skywalking/apm/agent/core/dictionary/NetworkAddressDictionary.java
+++ b/apm-sniffer/apm-agent-core/src/main/java/org/apache/skywalking/apm/agent/core/dictionary/NetworkAddressDictionary.java
@@ -16,6 +16,7 @@
  *
  */
 
+
 package org.apache.skywalking.apm.agent.core.dictionary;
 
 import io.netty.util.internal.ConcurrentSet;
@@ -48,12 +49,6 @@ public enum NetworkAddressDictionary {
         }
     }
 
-    public void clearApplicationDictionary() {
-        unRegisterApplications.clear();
-        applicationDictionary.clear();
-
-    }
-
     public void syncRemoteDictionary(
         NetworkAddressRegisterServiceGrpc.NetworkAddressRegisterServiceBlockingStub networkAddressRegisterServiceBlockingStub) {
         if (unRegisterApplications.size() > 0) {
diff --git a/apm-sniffer/apm-agent-core/src/main/java/org/apache/skywalking/apm/agent/core/dictionary/OperationNameDictionary.java b/apm-sniffer/apm-agent-core/src/main/java/org/apache/skywalking/apm/agent/core/dictionary/OperationNameDictionary.java
index 337bafb..7f743b5 100644
--- a/apm-sniffer/apm-agent-core/src/main/java/org/apache/skywalking/apm/agent/core/dictionary/OperationNameDictionary.java
+++ b/apm-sniffer/apm-agent-core/src/main/java/org/apache/skywalking/apm/agent/core/dictionary/OperationNameDictionary.java
@@ -61,13 +61,6 @@ public enum OperationNameDictionary {
         }
     }
 
-    public void clearOperationNameDictionary() {
-        unRegisterOperationNames.clear();
-        operationNameDictionary.clear();
-
-    }
-
-
     public void syncRemoteDictionary(
         ServiceNameDiscoveryServiceGrpc.ServiceNameDiscoveryServiceBlockingStub serviceNameDiscoveryServiceBlockingStub) {
         if (unRegisterOperationNames.size() > 0) {
@@ -113,10 +106,6 @@ public enum OperationNameDictionary {
             return applicationId;
         }
 
-        public void setApplicationId(int applicationId) {
-            this.applicationId = applicationId;
-        }
-
         public String getOperationName() {
             return operationName;
         }
diff --git a/apm-sniffer/apm-agent-core/src/main/java/org/apache/skywalking/apm/agent/core/listener/ResetConfListener.java b/apm-sniffer/apm-agent-core/src/main/java/org/apache/skywalking/apm/agent/core/listener/ResetConfListener.java
deleted file mode 100644
index 7360651..0000000
--- a/apm-sniffer/apm-agent-core/src/main/java/org/apache/skywalking/apm/agent/core/listener/ResetConfListener.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.skywalking.apm.agent.core.listener;
-
-import java.io.File;
-
-import java.io.FileNotFoundException;
-import java.io.IOException;
-import java.util.concurrent.Executors;
-import java.util.concurrent.TimeUnit;
-import org.apache.skywalking.apm.agent.core.boot.BootService;
-import org.apache.skywalking.apm.agent.core.boot.DefaultImplementor;
-import org.apache.skywalking.apm.agent.core.boot.DefaultNamedThreadFactory;
-import org.apache.skywalking.apm.agent.core.conf.Config;
-import org.apache.skywalking.apm.agent.core.logging.api.ILog;
-import org.apache.skywalking.apm.agent.core.logging.api.LogManager;
-import org.apache.skywalking.apm.util.RunnableWithExceptionProtection;
-
-/**
- * @author liu-xinyuan
- **/
-@DefaultImplementor
-public class ResetConfListener implements BootService, Runnable {
-    private static final ILog logger = LogManager.getLogger(ResetConfListener.class);
-    private File configFile = null;
-
-    @Override public void prepare() throws Throwable {
-
-    }
-
-    @Override public void boot() {
-        if ("enabled".equals(Config.Agent.RESETER_LISTENER)) {
-            Executors.newSingleThreadScheduledExecutor(new DefaultNamedThreadFactory("ResetConfListener"))
-                .scheduleAtFixedRate(new RunnableWithExceptionProtection(this, new RunnableWithExceptionProtection.CallbackWhenException() {
-                    @Override
-                    public void handle(Throwable t) {
-                        logger.error("unexpected exception.", t);
-                    }
-                }), 0, Config.Collector.SERVICE_AND_ENDPOINT_REGISTER_CHECK_INTERVAL, TimeUnit.SECONDS);
-
-        } else {
-            logger.info("Since the agent.register_status variable is not set correctly, the reset service is not started -> Agent reset service is inactive.");
-        }
-    }
-    @Override public void onComplete() throws Throwable {
-
-    }
-
-    @Override public void shutdown() throws Throwable {
-
-    }
-
-    @Override public void run() {
-        logger.debug("ResetConfListener running.");
-
-        try {
-            if (Reseter.INSTANCE.predicateReset())
-                Reseter.INSTANCE.setStatus(ResetStatus.DONE).clearID().reportToRegisterFile();
-        } catch (SecurityException e) {
-            logger.warn(e, "Denise read access to the file {}", configFile);
-        } catch (FileNotFoundException e) {
-            logger.warn(e, "not found file {}", configFile);
-        } catch (IOException e) {
-            logger.warn(e.getMessage());
-        }
-
-    }
-
-}
diff --git a/apm-sniffer/apm-agent-core/src/main/java/org/apache/skywalking/apm/agent/core/listener/ResetStatus.java b/apm-sniffer/apm-agent-core/src/main/java/org/apache/skywalking/apm/agent/core/listener/ResetStatus.java
deleted file mode 100644
index ac9de48..0000000
--- a/apm-sniffer/apm-agent-core/src/main/java/org/apache/skywalking/apm/agent/core/listener/ResetStatus.java
+++ /dev/null
@@ -1,35 +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.skywalking.apm.agent.core.listener;
-
-/**
- * @author liu-xinyuan
- **/
-public enum ResetStatus {
-    OFF("OFF"), ON("ON"), DONE("DONE");
-    private String label;
-
-    ResetStatus(String label) {
-        this.label = label;
-    }
-
-    public String value() {
-        return this.label;
-    }
-
-}
\ No newline at end of file
diff --git a/apm-sniffer/apm-agent-core/src/main/java/org/apache/skywalking/apm/agent/core/listener/Reseter.java b/apm-sniffer/apm-agent-core/src/main/java/org/apache/skywalking/apm/agent/core/listener/Reseter.java
deleted file mode 100644
index cf1ecea..0000000
--- a/apm-sniffer/apm-agent-core/src/main/java/org/apache/skywalking/apm/agent/core/listener/Reseter.java
+++ /dev/null
@@ -1,159 +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.skywalking.apm.agent.core.listener;
-
-import java.io.Closeable;
-import java.io.File;
-import java.io.FileInputStream;
-import java.io.FileOutputStream;
-import java.io.IOException;
-import java.nio.channels.FileChannel;
-import java.nio.channels.FileLock;
-import java.util.Properties;
-import org.apache.skywalking.apm.agent.core.boot.ServiceManager;
-import org.apache.skywalking.apm.agent.core.conf.Config;
-import org.apache.skywalking.apm.agent.core.conf.RemoteDownstreamConfig;
-import org.apache.skywalking.apm.agent.core.dictionary.DictionaryUtil;
-import org.apache.skywalking.apm.agent.core.dictionary.NetworkAddressDictionary;
-import org.apache.skywalking.apm.agent.core.dictionary.OperationNameDictionary;
-import org.apache.skywalking.apm.agent.core.logging.api.ILog;
-import org.apache.skywalking.apm.agent.core.logging.api.LogManager;
-import org.apache.skywalking.apm.agent.core.remote.TraceSegmentServiceClient;
-
-/**
- * @author liu-xinyuan
- **/
-public enum Reseter {
-    INSTANCE;
-    private static final ILog logger = LogManager.getLogger(Reseter.class);
-    private static final String APPLICATION_ID_NAM = "application_id";
-    private static final String INSTANCE_ID_NAME = "instance_id";
-    private static final String STATUS_NAME = "status";
-    private static final String RESET_CHILD_DIR = "/reset.status";
-    private static final String COMMENT = "#Status has three values: ON (trigger reset), DONE(reset complete), OFF(agent fist boot).\n" +
-        "Application_id: application_id of the current agent.\n" +
-        "Instance_id: the instanceid of the current agent.";
-    private volatile Properties properties = new Properties();
-    private String resetPath;
-    private ResetStatus status = ResetStatus.OFF;
-    private int isFirstRun = 0;
-    private int detectDuration = 5;
-
-    public Reseter setStatus(ResetStatus status) {
-        this.status = status;
-        return this;
-    }
-
-    public String getResetPath() throws IOException {
-        if (isFirstRun == 0) {
-            File statusDir = new File(Config.Agent.REGISTER_STATUS_DIR);
-
-            if (statusDir.exists() && statusDir.isDirectory()) {
-                resetPath = statusDir.getAbsolutePath() + RESET_CHILD_DIR;
-            } else {
-                statusDir.mkdir();
-            }
-            init();
-            isFirstRun = 1;
-        }
-        return resetPath;
-    }
-
-    public void reportToRegisterFile() throws IOException {
-        FileOutputStream outputStream = null;
-        try {
-            File configFile = new File(resetPath);
-            properties.setProperty(APPLICATION_ID_NAM, RemoteDownstreamConfig.Agent.APPLICATION_ID + "");
-            properties.setProperty(INSTANCE_ID_NAME, RemoteDownstreamConfig.Agent.APPLICATION_INSTANCE_ID + "");
-            properties.setProperty(STATUS_NAME, status.value());
-            outputStream = new FileOutputStream(configFile);
-            properties.store(outputStream, COMMENT);
-        } finally {
-            closeFileStream(outputStream);
-        }
-    }
-
-    public Reseter clearID() {
-        RemoteDownstreamConfig.Agent.APPLICATION_ID = DictionaryUtil.nullValue();
-        RemoteDownstreamConfig.Agent.APPLICATION_INSTANCE_ID = DictionaryUtil.nullValue();
-        OperationNameDictionary.INSTANCE.clearOperationNameDictionary();
-        NetworkAddressDictionary.INSTANCE.clearApplicationDictionary();
-        ServiceManager.INSTANCE.findService(TraceSegmentServiceClient.class).clearCache();
-        status = ResetStatus.DONE;
-        logger.info("clear id successfully,begin trigger reset.");
-        return this;
-    }
-
-    public Boolean predicateReset() throws IOException {
-        File resetFile = new File(getResetPath());
-        FileInputStream inputStream = null;
-        FileLock fileLock = null;
-        FileChannel fileChannel = null;
-        if (System.currentTimeMillis() - resetFile.lastModified() < detectDuration * 1000) {
-            try {
-                logger.info("The file reset.status was detected to have been modified in the last {} seconds.", detectDuration);
-                inputStream = new FileInputStream(resetFile);
-                fileChannel = inputStream.getChannel();
-                fileLock = fileChannel.tryLock(0, resetFile.length(), true);
-                if (fileLock == null) {
-                    return false;
-                }
-                properties.clear();
-                properties.load(inputStream);
-            } finally {
-                fileLock.release();
-                fileChannel.close();
-                closeFileStream(inputStream);
-            }
-            if (properties.get(STATUS_NAME) != null && properties.getProperty(STATUS_NAME).equals(ResetStatus.ON.value())) {
-                return true;
-            }
-        }
-        return false;
-
-    }
-
-    public void init() throws IOException {
-        FileOutputStream outputStream = null;
-        try {
-            properties.setProperty(APPLICATION_ID_NAM, RemoteDownstreamConfig.Agent.APPLICATION_ID + "");
-            properties.setProperty(INSTANCE_ID_NAME, RemoteDownstreamConfig.Agent.APPLICATION_INSTANCE_ID + "");
-            properties.setProperty(STATUS_NAME, status.value());
-            File file = new File(resetPath);
-            if (!file.getParentFile().exists()) {
-                file.getParentFile().mkdir();
-            }
-            outputStream = new FileOutputStream(file);
-            properties.store(outputStream, COMMENT);
-        } finally {
-            closeFileStream(outputStream);
-        }
-    }
-
-    public void closeFileStream(Closeable stream) throws IOException {
-        if (stream != null) {
-            try {
-                stream.close();
-            } catch (IOException e) {
-                throw new IOException("file close failed.", e);
-            }
-        } else {
-            throw new IOException("create file outputstream failed");
-        }
-    }
-}
diff --git a/apm-sniffer/apm-agent-core/src/main/java/org/apache/skywalking/apm/agent/core/remote/AppAndServiceRegisterClient.java b/apm-sniffer/apm-agent-core/src/main/java/org/apache/skywalking/apm/agent/core/remote/AppAndServiceRegisterClient.java
index 15459c6..3ce38d9 100644
--- a/apm-sniffer/apm-agent-core/src/main/java/org/apache/skywalking/apm/agent/core/remote/AppAndServiceRegisterClient.java
+++ b/apm-sniffer/apm-agent-core/src/main/java/org/apache/skywalking/apm/agent/core/remote/AppAndServiceRegisterClient.java
@@ -31,21 +31,16 @@ import org.apache.skywalking.apm.agent.core.context.trace.TraceSegment;
 import org.apache.skywalking.apm.agent.core.dictionary.DictionaryUtil;
 import org.apache.skywalking.apm.agent.core.dictionary.NetworkAddressDictionary;
 import org.apache.skywalking.apm.agent.core.dictionary.OperationNameDictionary;
-import org.apache.skywalking.apm.agent.core.listener.ResetStatus;
-import org.apache.skywalking.apm.agent.core.listener.Reseter;
 import org.apache.skywalking.apm.agent.core.logging.api.ILog;
 import org.apache.skywalking.apm.agent.core.logging.api.LogManager;
 import org.apache.skywalking.apm.agent.core.os.OSUtil;
 import org.apache.skywalking.apm.network.language.agent.*;
-import org.apache.skywalking.apm.network.register.ServiceInstancePingGrpc;
-import org.apache.skywalking.apm.network.register.ServiceInstancePingPkg;
 import org.apache.skywalking.apm.util.RunnableWithExceptionProtection;
 
 import java.util.UUID;
 import java.util.concurrent.Executors;
 import java.util.concurrent.ScheduledFuture;
 import java.util.concurrent.TimeUnit;
-import org.apache.skywalking.apm.util.StringUtil;
 
 /**
  * @author wusheng
@@ -53,13 +48,13 @@ import org.apache.skywalking.apm.util.StringUtil;
 @DefaultImplementor
 public class AppAndServiceRegisterClient implements BootService, GRPCChannelListener, Runnable, TracingContextListener {
     private static final ILog logger = LogManager.getLogger(AppAndServiceRegisterClient.class);
+    private static final String PROCESS_UUID = UUID.randomUUID().toString().replaceAll("-", "");
 
     private volatile GRPCChannelStatus status = GRPCChannelStatus.DISCONNECT;
     private volatile ApplicationRegisterServiceGrpc.ApplicationRegisterServiceBlockingStub applicationRegisterServiceBlockingStub;
     private volatile InstanceDiscoveryServiceGrpc.InstanceDiscoveryServiceBlockingStub instanceDiscoveryServiceBlockingStub;
     private volatile ServiceNameDiscoveryServiceGrpc.ServiceNameDiscoveryServiceBlockingStub serviceNameDiscoveryServiceBlockingStub;
     private volatile NetworkAddressRegisterServiceGrpc.NetworkAddressRegisterServiceBlockingStub networkAddressRegisterServiceBlockingStub;
-    private volatile ServiceInstancePingGrpc.ServiceInstancePingBlockingStub serviceInstancePingBlockingStub;
     private volatile ScheduledFuture<?> applicationRegisterFuture;
     private volatile long lastSegmentTime = -1;
 
@@ -71,7 +66,6 @@ public class AppAndServiceRegisterClient implements BootService, GRPCChannelList
             instanceDiscoveryServiceBlockingStub = InstanceDiscoveryServiceGrpc.newBlockingStub(channel);
             serviceNameDiscoveryServiceBlockingStub = ServiceNameDiscoveryServiceGrpc.newBlockingStub(channel);
             networkAddressRegisterServiceBlockingStub = NetworkAddressRegisterServiceGrpc.newBlockingStub(channel);
-            serviceInstancePingBlockingStub = ServiceInstancePingGrpc.newBlockingStub(channel);
         } else {
             applicationRegisterServiceBlockingStub = null;
             instanceDiscoveryServiceBlockingStub = null;
@@ -88,13 +82,13 @@ public class AppAndServiceRegisterClient implements BootService, GRPCChannelList
     @Override
     public void boot() throws Throwable {
         applicationRegisterFuture = Executors
-            .newSingleThreadScheduledExecutor(new DefaultNamedThreadFactory("AppAndServiceRegisterClient"))
-            .scheduleAtFixedRate(new RunnableWithExceptionProtection(this, new RunnableWithExceptionProtection.CallbackWhenException() {
-                @Override
-                public void handle(Throwable t) {
-                    logger.error("unexpected exception.", t);
-                }
-            }), 0, Config.Collector.SERVICE_AND_ENDPOINT_REGISTER_CHECK_INTERVAL, TimeUnit.SECONDS);
+                .newSingleThreadScheduledExecutor(new DefaultNamedThreadFactory("AppAndServiceRegisterClient"))
+                .scheduleAtFixedRate(new RunnableWithExceptionProtection(this, new RunnableWithExceptionProtection.CallbackWhenException() {
+                    @Override
+                    public void handle(Throwable t) {
+                        logger.error("unexpected exception.", t);
+                    }
+                }), 0, Config.Collector.APP_AND_SERVICE_REGISTER_CHECK_INTERVAL, TimeUnit.SECONDS);
     }
 
     @Override
@@ -111,17 +105,15 @@ public class AppAndServiceRegisterClient implements BootService, GRPCChannelList
     public void run() {
         logger.debug("AppAndServiceRegisterClient running, status:{}.", status);
         boolean shouldTry = true;
-        String instanceUUID = StringUtil.isEmpty(Config.Agent.INSTANCE_UUID) ? UUID.randomUUID().toString().replaceAll("-", "") : Config.Agent.INSTANCE_UUID;
         while (GRPCChannelStatus.CONNECTED.equals(status) && shouldTry) {
             shouldTry = false;
             try {
                 if (RemoteDownstreamConfig.Agent.APPLICATION_ID == DictionaryUtil.nullValue()) {
                     if (applicationRegisterServiceBlockingStub != null) {
                         ApplicationMapping applicationMapping = applicationRegisterServiceBlockingStub.applicationCodeRegister(
-                            Application.newBuilder().setApplicationCode(Config.Agent.APPLICATION_CODE).build());
+                                Application.newBuilder().setApplicationCode(Config.Agent.APPLICATION_CODE).build());
                         if (applicationMapping != null) {
                             RemoteDownstreamConfig.Agent.APPLICATION_ID = applicationMapping.getApplication().getValue();
-                            Reseter.INSTANCE.reportToRegisterFile();
                             shouldTry = true;
                         }
                     }
@@ -130,24 +122,21 @@ public class AppAndServiceRegisterClient implements BootService, GRPCChannelList
                         if (RemoteDownstreamConfig.Agent.APPLICATION_INSTANCE_ID == DictionaryUtil.nullValue()) {
 
                             ApplicationInstanceMapping instanceMapping = instanceDiscoveryServiceBlockingStub.registerInstance(ApplicationInstance.newBuilder()
-                                .setApplicationId(RemoteDownstreamConfig.Agent.APPLICATION_ID)
-                                .setAgentUUID(instanceUUID)
-                                .setRegisterTime(System.currentTimeMillis())
-                                .setOsinfo(OSUtil.buildOSInfo())
-                                .build());
+                                    .setApplicationId(RemoteDownstreamConfig.Agent.APPLICATION_ID)
+                                    .setAgentUUID(PROCESS_UUID)
+                                    .setRegisterTime(System.currentTimeMillis())
+                                    .setOsinfo(OSUtil.buildOSInfo())
+                                    .build());
                             if (instanceMapping.getApplicationInstanceId() != DictionaryUtil.nullValue()) {
                                 RemoteDownstreamConfig.Agent.APPLICATION_INSTANCE_ID
-                                    = instanceMapping.getApplicationInstanceId();
-                                Reseter.INSTANCE.setStatus(ResetStatus.OFF).reportToRegisterFile();
-
+                                        = instanceMapping.getApplicationInstanceId();
                             }
                         } else {
                             if (lastSegmentTime - System.currentTimeMillis() > 60 * 1000) {
-                                serviceInstancePingBlockingStub.doPing(ServiceInstancePingPkg.newBuilder()
-                                    .setServiceInstanceId(RemoteDownstreamConfig.Agent.APPLICATION_INSTANCE_ID)
-                                    .setServiceInstanceUUID(instanceUUID)
-                                    .setTime(System.currentTimeMillis())
-                                    .build());
+                                instanceDiscoveryServiceBlockingStub.heartbeat(ApplicationInstanceHeartbeat.newBuilder()
+                                        .setApplicationInstanceId(RemoteDownstreamConfig.Agent.APPLICATION_INSTANCE_ID)
+                                        .setHeartbeatTime(System.currentTimeMillis())
+                                        .build());
                             }
 
                             NetworkAddressDictionary.INSTANCE.syncRemoteDictionary(networkAddressRegisterServiceBlockingStub);
diff --git a/apm-sniffer/apm-agent-core/src/main/java/org/apache/skywalking/apm/agent/core/remote/TraceSegmentServiceClient.java b/apm-sniffer/apm-agent-core/src/main/java/org/apache/skywalking/apm/agent/core/remote/TraceSegmentServiceClient.java
index 2c277fe..697d747 100644
--- a/apm-sniffer/apm-agent-core/src/main/java/org/apache/skywalking/apm/agent/core/remote/TraceSegmentServiceClient.java
+++ b/apm-sniffer/apm-agent-core/src/main/java/org/apache/skywalking/apm/agent/core/remote/TraceSegmentServiceClient.java
@@ -25,11 +25,9 @@ import io.grpc.stub.StreamObserver;
 import org.apache.skywalking.apm.agent.core.boot.BootService;
 import org.apache.skywalking.apm.agent.core.boot.DefaultImplementor;
 import org.apache.skywalking.apm.agent.core.boot.ServiceManager;
-import org.apache.skywalking.apm.agent.core.conf.RemoteDownstreamConfig;
 import org.apache.skywalking.apm.agent.core.context.TracingContext;
 import org.apache.skywalking.apm.agent.core.context.TracingContextListener;
 import org.apache.skywalking.apm.agent.core.context.trace.TraceSegment;
-import org.apache.skywalking.apm.agent.core.dictionary.DictionaryUtil;
 import org.apache.skywalking.apm.agent.core.logging.api.ILog;
 import org.apache.skywalking.apm.agent.core.logging.api.LogManager;
 import org.apache.skywalking.apm.commons.datacarrier.DataCarrier;
@@ -157,7 +155,7 @@ public class TraceSegmentServiceClient implements BootService, IConsumer<TraceSe
 
     @Override
     public void afterFinished(TraceSegment traceSegment) {
-        if (traceSegment.isIgnore() || RemoteDownstreamConfig.Agent.APPLICATION_INSTANCE_ID == DictionaryUtil.nullValue() || RemoteDownstreamConfig.Agent.APPLICATION_ID == DictionaryUtil.nullValue()) {
+        if (traceSegment.isIgnore()) {
             return;
         }
         if (!carrier.produce(traceSegment)) {
@@ -175,9 +173,4 @@ public class TraceSegmentServiceClient implements BootService, IConsumer<TraceSe
         }
         this.status = status;
     }
-
-    public void clearCache() {
-        carrier.clear();
-    }
-
 }
diff --git a/apm-sniffer/apm-agent-core/src/main/resources/META-INF/services/org.apache.skywalking.apm.agent.core.boot.BootService b/apm-sniffer/apm-agent-core/src/main/resources/META-INF/services/org.apache.skywalking.apm.agent.core.boot.BootService
index 14bef1c..9beec08 100644
--- a/apm-sniffer/apm-agent-core/src/main/resources/META-INF/services/org.apache.skywalking.apm.agent.core.boot.BootService
+++ b/apm-sniffer/apm-agent-core/src/main/resources/META-INF/services/org.apache.skywalking.apm.agent.core.boot.BootService
@@ -23,4 +23,3 @@ org.apache.skywalking.apm.agent.core.remote.GRPCChannelManager
 org.apache.skywalking.apm.agent.core.jvm.JVMService
 org.apache.skywalking.apm.agent.core.remote.AppAndServiceRegisterClient
 org.apache.skywalking.apm.agent.core.context.ContextManagerExtendService
-org.apache.skywalking.apm.agent.core.listener.ResetConfListener
\ No newline at end of file
diff --git a/apm-sniffer/apm-agent-core/src/test/java/org/apache/skywalking/apm/agent/core/boot/ServiceManagerTest.java b/apm-sniffer/apm-agent-core/src/test/java/org/apache/skywalking/apm/agent/core/boot/ServiceManagerTest.java
index 46544f3..3009f4b 100644
--- a/apm-sniffer/apm-agent-core/src/test/java/org/apache/skywalking/apm/agent/core/boot/ServiceManagerTest.java
+++ b/apm-sniffer/apm-agent-core/src/test/java/org/apache/skywalking/apm/agent/core/boot/ServiceManagerTest.java
@@ -55,7 +55,7 @@ public class ServiceManagerTest {
     public void testServiceDependencies() throws Exception {
         HashMap<Class, BootService> registryService = getFieldValue(ServiceManager.INSTANCE, "bootedServices");
 
-        assertThat(registryService.size(), is(8));
+        assertThat(registryService.size(), is(7));
 
         assertTraceSegmentServiceClient(ServiceManager.INSTANCE.findService(TraceSegmentServiceClient.class));
         assertContextManager(ServiceManager.INSTANCE.findService(ContextManager.class));
diff --git a/apm-sniffer/config/agent.config b/apm-sniffer/config/agent.config
index 6c16f8f..fbf0c6b 100644
--- a/apm-sniffer/config/agent.config
+++ b/apm-sniffer/config/agent.config
@@ -43,12 +43,3 @@ collector.backend_service=127.0.0.1:10800
 
 # Logging level
 logging.level=DEBUG
-
-#Specify register.status dir,if dir not exists or it is a file then default AGENT_HOME/option
-#agent.register_status_dir=register_dir
-
-#Specify instance_uuid to ensure that the whole show is unique, for example: applicationName_ip_12
-#agent.instance_uuid = applicationName_ip_1
-
-#enabled means that the reset function is enabled, and disabled means that the reset function is not enabled. A reset can be triggered by modifying the configuration file only if the reset feature is enabled.
-#agent.reseter_listener = disabled
diff --git a/oap-server/server-alarm-plugin/src/main/java/org/apache/skywalking/oap/server/core/alarm/provider/RunningRule.java b/oap-server/server-alarm-plugin/src/main/java/org/apache/skywalking/oap/server/core/alarm/provider/RunningRule.java
index 3f8c6ea..a010392 100644
--- a/oap-server/server-alarm-plugin/src/main/java/org/apache/skywalking/oap/server/core/alarm/provider/RunningRule.java
+++ b/oap-server/server-alarm-plugin/src/main/java/org/apache/skywalking/oap/server/core/alarm/provider/RunningRule.java
@@ -18,15 +18,26 @@
 
 package org.apache.skywalking.oap.server.core.alarm.provider;
 
-import java.util.*;
+import java.util.ArrayList;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.locks.ReentrantLock;
-import org.apache.skywalking.oap.server.core.alarm.*;
-import org.apache.skywalking.oap.server.core.analysis.indicator.*;
+import org.apache.skywalking.oap.server.core.alarm.AlarmMessage;
+import org.apache.skywalking.oap.server.core.alarm.MetaInAlarm;
+import org.apache.skywalking.oap.server.core.analysis.indicator.DoubleValueHolder;
+import org.apache.skywalking.oap.server.core.analysis.indicator.Indicator;
+import org.apache.skywalking.oap.server.core.analysis.indicator.IntValueHolder;
+import org.apache.skywalking.oap.server.core.analysis.indicator.LongValueHolder;
 import org.apache.skywalking.oap.server.core.source.Scope;
-import org.joda.time.*;
-import org.joda.time.format.*;
-import org.slf4j.*;
+import org.apache.skywalking.oap.server.library.util.CollectionUtils;
+import org.joda.time.LocalDateTime;
+import org.joda.time.Minutes;
+import org.joda.time.format.DateTimeFormat;
+import org.joda.time.format.DateTimeFormatter;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * RunningRule represents each rule in running status. Based on the {@link AlarmRule} definition,
@@ -81,6 +92,12 @@ public class RunningRule {
             return;
         }
 
+        if (CollectionUtils.isNotEmpty(includeNames)) {
+            if (!includeNames.contains(meta.getName())) {
+                return;
+            }
+        }
+
         if (valueType == null) {
             if (indicator instanceof LongValueHolder) {
                 valueType = IndicatorValueType.LONG;
@@ -91,6 +108,8 @@ public class RunningRule {
             } else if (indicator instanceof DoubleValueHolder) {
                 valueType = IndicatorValueType.DOUBLE;
                 threshold.setType(IndicatorValueType.DOUBLE);
+            } else {
+                return;
             }
             targetScope = meta.getScope();
         }
diff --git a/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/register/service/ServiceInstanceInventoryRegister.java b/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/register/service/ServiceInstanceInventoryRegister.java
index 1fd7851..af02992 100644
--- a/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/register/service/ServiceInstanceInventoryRegister.java
+++ b/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/register/service/ServiceInstanceInventoryRegister.java
@@ -109,4 +109,4 @@ public class ServiceInstanceInventoryRegister implements IServiceInstanceInvento
             logger.warn("Service instance {} heartbeat, but not found in storage.");
         }
     }
-}
\ No newline at end of file
+}
diff --git a/oap-server/server-receiver-plugin/skywalking-register-receiver-plugin/src/main/java/org/apache/skywalking/oap/server/receiver/register/provider/RegisterModuleProvider.java b/oap-server/server-receiver-plugin/skywalking-register-receiver-plugin/src/main/java/org/apache/skywalking/oap/server/receiver/register/provider/RegisterModuleProvider.java
index 10cafe8..ffaa49c 100644
--- a/oap-server/server-receiver-plugin/skywalking-register-receiver-plugin/src/main/java/org/apache/skywalking/oap/server/receiver/register/provider/RegisterModuleProvider.java
+++ b/oap-server/server-receiver-plugin/skywalking-register-receiver-plugin/src/main/java/org/apache/skywalking/oap/server/receiver/register/provider/RegisterModuleProvider.java
@@ -55,7 +55,6 @@ public class RegisterModuleProvider extends ModuleProvider {
         grpcHandlerRegister.addHandler(new ServiceNameDiscoveryHandler(getManager()));
         grpcHandlerRegister.addHandler(new NetworkAddressRegisterServiceHandler(getManager()));
 
-
         JettyHandlerRegister jettyHandlerRegister = getManager().find(CoreModule.NAME).getService(JettyHandlerRegister.class);
         jettyHandlerRegister.addHandler(new ApplicationRegisterServletHandler(getManager()));
         jettyHandlerRegister.addHandler(new InstanceDiscoveryServletHandler(getManager()));
diff --git a/oap-server/server-receiver-plugin/skywalking-register-receiver-plugin/src/main/java/org/apache/skywalking/oap/server/receiver/register/provider/handler/v5/ServiceInstancePingPkgHandler.java b/oap-server/server-receiver-plugin/skywalking-register-receiver-plugin/src/main/java/org/apache/skywalking/oap/server/receiver/register/provider/handler/v5/ServiceInstancePingPkgHandler.java
deleted file mode 100644
index 15d50c2..0000000
--- a/oap-server/server-receiver-plugin/skywalking-register-receiver-plugin/src/main/java/org/apache/skywalking/oap/server/receiver/register/provider/handler/v5/ServiceInstancePingPkgHandler.java
+++ /dev/null
@@ -1,60 +0,0 @@
-/*
- *
- *  * Licensed to the Apache Software Foundation (ASF) under one or more
- *  * contributor license agreements.  See the NOTICE file distributed with
- *  * this work for additional information regarding copyright ownership.
- *  * The ASF licenses this file to You under the Apache License, Version 2.0
- *  * (the "License"); you may not use this file except in compliance with
- *  * the License.  You may obtain a copy of the License at
- *  *
- *  *     http://www.apache.org/licenses/LICENSE-2.0
- *  *
- *  * Unless required by applicable law or agreed to in writing, software
- *  * distributed under the License is distributed on an "AS IS" BASIS,
- *  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- *  * See the License for the specific language governing permissions and
- *  * limitations under the License.
- *  *
- *
- */
-
-package org.apache.skywalking.oap.server.receiver.register.provider.handler.v5;
-
-import io.grpc.stub.StreamObserver;
-import org.apache.skywalking.apm.network.common.Commands;
-import org.apache.skywalking.apm.network.register.ServiceInstancePingGrpc;
-import org.apache.skywalking.apm.network.register.ServiceInstancePingPkg;
-import org.apache.skywalking.oap.server.core.Const;
-import org.apache.skywalking.oap.server.core.CoreModule;
-import org.apache.skywalking.oap.server.core.register.ServiceInstanceInventory;
-import org.apache.skywalking.oap.server.core.register.service.IServiceInstanceInventoryRegister;
-import org.apache.skywalking.oap.server.core.storage.StorageModule;
-import org.apache.skywalking.oap.server.core.storage.cache.IServiceInstanceInventoryCacheDAO;
-import org.apache.skywalking.oap.server.library.module.ModuleManager;
-import org.apache.skywalking.oap.server.library.server.grpc.GRPCHandler;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public class ServiceInstancePingPkgHandler extends ServiceInstancePingGrpc.ServiceInstancePingImplBase implements GRPCHandler {
-    private static final Logger logger = LoggerFactory.getLogger(ServiceInstancePingPkgHandler.class);
-
-    private final IServiceInstanceInventoryCacheDAO instanceInventoryCacheDAO;
-    private final IServiceInstanceInventoryRegister serviceInstanceInventoryRegister;
-
-    public ServiceInstancePingPkgHandler(ModuleManager moduleManager) {
-        this.instanceInventoryCacheDAO = moduleManager.find(StorageModule.NAME).getService(IServiceInstanceInventoryCacheDAO.class);
-        this.serviceInstanceInventoryRegister = moduleManager.find(CoreModule.NAME).getService(IServiceInstanceInventoryRegister.class);
-
-    }
-
-    @Override public void doPing(ServiceInstancePingPkg request, StreamObserver<Commands> responseObserver) {
-
-        ServiceInstanceInventory serviceInstanceInventory = instanceInventoryCacheDAO.get(request.getServiceInstanceId());
-        if (request.getServiceInstanceUUID().equals(serviceInstanceInventory.getName()) || serviceInstanceInventory.getServiceId() == Const.NONE) {
-            logger.error("Your metadata loss,please set the status in reset.status in the agent {} to ON to trigger a reset!", request.getServiceInstanceUUID());
-        } else {
-            serviceInstanceInventoryRegister.heartbeat(request.getServiceInstanceId(), request.getTime());
-        }
-
-    }
-}
diff --git a/oap-server/server-starter/pom.xml b/oap-server/server-starter/pom.xml
index 9bbe894..91bf853 100644
--- a/oap-server/server-starter/pom.xml
+++ b/oap-server/server-starter/pom.xml
@@ -136,6 +136,8 @@
                     <excludes>
                         <exclude>application.yml</exclude>
                         <exclude>log4j2.xml</exclude>
+                        <exclude>alarm-settings.yml</exclude>
+                        <exclude>component-libraries.yml</exclude>
                     </excludes>
                 </configuration>
             </plugin>
diff --git a/oap-server/server-starter/src/main/assembly/alarm-settings.yml b/oap-server/server-starter/src/main/assembly/alarm-settings.yml
index 6c69f89..5b839fb 100644
--- a/oap-server/server-starter/src/main/assembly/alarm-settings.yml
+++ b/oap-server/server-starter/src/main/assembly/alarm-settings.yml
@@ -16,20 +16,36 @@
 
 # Sample alarm rules.
 rules:
-# Sample alarm rules.
-#  # Rule unique name, must be ended with `_rule`.
-#  endpoint_percent_rule:
-#    # Indicator value need to be long, double or int
-#    indicator-name: endpoint_percent
-#    threshold: 75
-#    op: <
-#    # The length of time to evaluate the metric
-#    period: 10
-#    # How many times after the metric match the condition, will trigger alarm
-#    count: 3
-#    # How many times of checks, the alarm keeps silence after alarm triggered, default as same as period.
-#    silence-period: 10
-#    message: Successful rate of endpoint {name} is lower than 75%
+  # Rule unique name, must be ended with `_rule`.
+  service_resp_time_rule:
+    indicator-name: service_resp_time
+    op: ">"
+    threshold: 1000
+    period: 10
+    count: 3
+    silence-period: 5
+    message: Response time of service {name} is more than 2000ms.
+  service_sla_rule:
+    # Indicator value need to be long, double or int
+    indicator-name: service_sla
+    op: "<"
+    threshold: 80
+    # The length of time to evaluate the metric
+    period: 10
+    # How many times after the metric match the condition, will trigger alarm
+    count: 2
+    # How many times of checks, the alarm keeps silence after alarm triggered, default as same as period.
+    silence-period: 3
+    message: Successful rate of service {name} is lower than 80%
+  service_p90_sla_rule:
+    # Indicator value need to be long, double or int
+    indicator-name: service_sla
+    op: ">"
+    threshold: 1000
+    period: 10
+    count: 3
+    silence-period: 5
+    message: 90% response time of service {name} is lower than 80%
 
 webhooks:
 #  - http://127.0.0.1/notify/