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 2019/03/25 05:08:53 UTC

[incubator-skywalking] 01/01: Try to make disable work.

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

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

commit ea35c7b81900de99d7acb8c75946512f5d91a32c
Author: Wu Sheng <wu...@foxmail.com>
AuthorDate: Sun Mar 24 22:08:20 2019 -0700

    Try to make disable work.
---
 .../apache/skywalking/oal/tool/grammar/OALLexer.g4 |  9 +++++
 .../skywalking/oal/tool/grammar/OALParser.g4       | 11 +++++-
 .../java/org/apache/skywalking/oal/tool/Main.java  |  5 ++-
 .../skywalking/oal/tool/output/FileGenerator.java  | 15 ++++++--
 .../oal/tool/parser/DisableCollection.java         | 34 ++++++++++++++++++
 .../skywalking/oal/tool/parser/OALListener.java    | 15 ++++++--
 .../skywalking/oal/tool/parser/OALScripts.java     | 33 +++++++++++++++++
 .../skywalking/oal/tool/parser/ScriptParser.java   |  9 ++---
 .../code-templates/DisableSourceDefinition.ftl     | 33 +++++++++++++++++
 .../oal/tool/output/FileGeneratorTest.java         | 12 +++----
 .../oal/tool/parser/ScriptParserTest.java          | 18 +++++++---
 .../src/main/resources/official_analysis.oal       |  5 +++
 .../oap/server/core/CoreModuleProvider.java        |  2 ++
 .../oap/server/core/analysis/Disable.java          | 31 ++++++++++++++++
 .../oap/server/core/analysis/DisableRegister.java  | 42 ++++++++++++++++++++++
 .../oap/server/core/analysis/MultipleDisable.java  | 27 ++++++++++++++
 .../core/analysis/worker/IndicatorProcess.java     |  5 ++-
 .../server/core/analysis/worker/RecordProcess.java |  5 ++-
 .../server/core/analysis/worker/TopNProcess.java   |  9 +++--
 19 files changed, 292 insertions(+), 28 deletions(-)

diff --git a/oap-server/generate-tool-grammar/src/main/antlr4/org/apache/skywalking/oal/tool/grammar/OALLexer.g4 b/oap-server/generate-tool-grammar/src/main/antlr4/org/apache/skywalking/oal/tool/grammar/OALLexer.g4
index af1e3a7..8888c5e 100644
--- a/oap-server/generate-tool-grammar/src/main/antlr4/org/apache/skywalking/oal/tool/grammar/OALLexer.g4
+++ b/oap-server/generate-tool-grammar/src/main/antlr4/org/apache/skywalking/oal/tool/grammar/OALLexer.g4
@@ -26,6 +26,7 @@ lexer grammar OALLexer;
 
 FROM: 'from';
 FILTER: 'filter';
+DISABLE: 'disable';
 SRC_ALL: 'All';
 SRC_SERVICE: 'Service';
 SRC_SERVICE_INSTANCE: 'ServiceInstance';
@@ -43,6 +44,14 @@ SRC_SERVICE_INSTANCE_CLR_GC: 'ServiceInstanceCLRGC';
 SRC_SERVICE_INSTANCE_CLR_THREAD: 'ServiceInstanceCLRThread';
 SRC_ENVOY_INSTANCE_METRIC: 'EnvoyInstanceMetric';
 
+//hard code sources, only used when need to be deactived.
+SRC_SEGMENT: 'segment';
+SRC_TOP_N_DB_STATEMENT: 'top_n_database_statement';
+SRC_ENDPOINT_RELATION_SERVER_SIDE: 'endpoint_relation_server_side';
+SRC_SERVICE_RELATION_SERVER_SIDE: 'service_relation_server_side';
+SRC_SERVICE_RELATION_CLIENT_SIDE: 'service_relation_client_side';
+SRC_ALARM_RECORD: 'alarm_record';
+
 // Literals
 
 BOOL_LITERAL:       'true'
diff --git a/oap-server/generate-tool-grammar/src/main/antlr4/org/apache/skywalking/oal/tool/grammar/OALParser.g4 b/oap-server/generate-tool-grammar/src/main/antlr4/org/apache/skywalking/oal/tool/grammar/OALParser.g4
index 35b20bd..74fa294 100644
--- a/oap-server/generate-tool-grammar/src/main/antlr4/org/apache/skywalking/oal/tool/grammar/OALParser.g4
+++ b/oap-server/generate-tool-grammar/src/main/antlr4/org/apache/skywalking/oal/tool/grammar/OALParser.g4
@@ -26,13 +26,17 @@ options { tokenVocab=OALLexer; }
 // Top Level Description
 
 root
-    : (aggregationStatement)*
+    : (aggregationStatement | disableStatement)*
     ;
 
 aggregationStatement
     : variable (SPACE)? EQUAL (SPACE)? metricStatement DelimitedComment? LineComment? (SEMI|EOF)
     ;
 
+disableStatement
+    : DISABLE LR_BRACKET disableSource RR_BRACKET DelimitedComment? LineComment? (SEMI|EOF)
+    ;
+
 metricStatement
     : FROM LR_BRACKET source  DOT sourceAttribute RR_BRACKET (filterStatement+)? DOT aggregateFunction
     ;
@@ -53,6 +57,11 @@ source
       SRC_ENVOY_INSTANCE_METRIC
     ;
 
+disableSource
+    : SRC_SEGMENT | SRC_TOP_N_DB_STATEMENT | SRC_ENDPOINT_RELATION_SERVER_SIDE | SRC_SERVICE_RELATION_SERVER_SIDE |
+      SRC_SERVICE_RELATION_CLIENT_SIDE | SRC_ALARM_RECORD
+    ;
+
 sourceAttribute
     : IDENTIFIER | ALL
     ;
diff --git a/oap-server/generate-tool/src/main/java/org/apache/skywalking/oal/tool/Main.java b/oap-server/generate-tool/src/main/java/org/apache/skywalking/oal/tool/Main.java
index 3c960d7..ea35936 100644
--- a/oap-server/generate-tool/src/main/java/org/apache/skywalking/oal/tool/Main.java
+++ b/oap-server/generate-tool/src/main/java/org/apache/skywalking/oal/tool/Main.java
@@ -20,7 +20,6 @@ package org.apache.skywalking.oal.tool;
 
 import freemarker.template.TemplateException;
 import java.io.*;
-import java.util.List;
 import org.apache.skywalking.apm.util.StringUtil;
 import org.apache.skywalking.oal.tool.meta.*;
 import org.apache.skywalking.oal.tool.output.FileGenerator;
@@ -59,9 +58,9 @@ public class Main {
         SourceColumnsFactory.setSettings(metaSettings);
 
         ScriptParser scriptParser = ScriptParser.createFromFile(scriptFilePath);
-        List<AnalysisResult> analysisResults = scriptParser.parse();
+        OALScripts oalScripts = scriptParser.parse();
 
-        FileGenerator generator = new FileGenerator(analysisResults, outputPath);
+        FileGenerator generator = new FileGenerator(oalScripts, outputPath);
         generator.generate();
     }
 }
diff --git a/oap-server/generate-tool/src/main/java/org/apache/skywalking/oal/tool/output/FileGenerator.java b/oap-server/generate-tool/src/main/java/org/apache/skywalking/oal/tool/output/FileGenerator.java
index 469eb97..e2a1db3 100644
--- a/oap-server/generate-tool/src/main/java/org/apache/skywalking/oal/tool/output/FileGenerator.java
+++ b/oap-server/generate-tool/src/main/java/org/apache/skywalking/oal/tool/output/FileGenerator.java
@@ -21,16 +21,18 @@ package org.apache.skywalking.oal.tool.output;
 import freemarker.template.*;
 import java.io.*;
 import java.util.*;
-import org.apache.skywalking.oal.tool.parser.AnalysisResult;
+import org.apache.skywalking.oal.tool.parser.*;
 
 public class FileGenerator {
     private List<AnalysisResult> results;
+    private DisableCollection collection;
     private String outputPath;
     private Configuration configuration;
     private AllDispatcherContext allDispatcherContext;
 
-    public FileGenerator(List<AnalysisResult> results, String outputPath) {
-        this.results = results;
+    public FileGenerator(OALScripts oalScripts, String outputPath) {
+        this.results = oalScripts.getIndicatorStmts();
+        this.collection = oalScripts.getDisableCollection();
         this.outputPath = outputPath;
         configuration = new Configuration(new Version("2.3.28"));
         configuration.setEncoding(Locale.ENGLISH, "UTF-8");
@@ -48,6 +50,7 @@ public class FileGenerator {
             createFile(file);
             generateDispatcher(result, new FileWriter(file));
         }
+        generateDisable();
     }
 
     private void generate(AnalysisResult result, String fileSuffix,
@@ -103,4 +106,10 @@ public class FileGenerator {
             context.getIndicators().add(result);
         }
     }
+
+    private void generateDisable() throws IOException, TemplateException {
+        File file = new File(outputPath, "generated/DisableSourceDefinition.java");
+        createFile(file);
+        configuration.getTemplate("DisableSourceDefinition.ftl").process(collection, new FileWriter(file));
+    }
 }
diff --git a/oap-server/generate-tool/src/main/java/org/apache/skywalking/oal/tool/parser/DisableCollection.java b/oap-server/generate-tool/src/main/java/org/apache/skywalking/oal/tool/parser/DisableCollection.java
new file mode 100644
index 0000000..08987c2
--- /dev/null
+++ b/oap-server/generate-tool/src/main/java/org/apache/skywalking/oal/tool/parser/DisableCollection.java
@@ -0,0 +1,34 @@
+/*
+ * 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.oal.tool.parser;
+
+import java.util.*;
+import lombok.Getter;
+
+/**
+ * @author wusheng
+ */
+@Getter
+public class DisableCollection {
+    private List<String> allDisableSources = new ArrayList<>();
+
+    public void add(String source) {
+        allDisableSources.add(source);
+    }
+}
diff --git a/oap-server/generate-tool/src/main/java/org/apache/skywalking/oal/tool/parser/OALListener.java b/oap-server/generate-tool/src/main/java/org/apache/skywalking/oal/tool/parser/OALListener.java
index a52cc74..6433032 100644
--- a/oap-server/generate-tool/src/main/java/org/apache/skywalking/oal/tool/parser/OALListener.java
+++ b/oap-server/generate-tool/src/main/java/org/apache/skywalking/oal/tool/parser/OALListener.java
@@ -26,11 +26,13 @@ import org.apache.skywalking.oap.server.core.source.DefaultScopeDefine;
 public class OALListener extends OALParserBaseListener {
     private List<AnalysisResult> results;
     private AnalysisResult current;
+    private DisableCollection collection;
 
     private ConditionExpression conditionExpression;
 
-    public OALListener(List<AnalysisResult> results) {
-        this.results = results;
+    public OALListener(OALScripts scripts) {
+        this.results = scripts.getIndicatorStmts();
+        this.collection = scripts.getDisableCollection();
     }
 
     @Override
@@ -150,6 +152,15 @@ public class OALListener extends OALParserBaseListener {
         return source;
     }
 
+    /**
+     * Disable source
+     *
+     * @param ctx
+     */
+    @Override public void enterDisableSource(OALParser.DisableSourceContext ctx) {
+        collection.add(ctx.getText());
+    }
+
     private String firstLetterUpper(String source) {
         return source.substring(0, 1).toUpperCase() + source.substring(1);
     }
diff --git a/oap-server/generate-tool/src/main/java/org/apache/skywalking/oal/tool/parser/OALScripts.java b/oap-server/generate-tool/src/main/java/org/apache/skywalking/oal/tool/parser/OALScripts.java
new file mode 100644
index 0000000..8d5b4fc
--- /dev/null
+++ b/oap-server/generate-tool/src/main/java/org/apache/skywalking/oal/tool/parser/OALScripts.java
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.skywalking.oal.tool.parser;
+
+import java.util.*;
+import lombok.*;
+
+@Getter
+public class OALScripts {
+    private List<AnalysisResult> indicatorStmts;
+    private DisableCollection disableCollection;
+
+    public OALScripts() {
+        indicatorStmts = new LinkedList<>();
+        disableCollection = new DisableCollection();
+    }
+}
diff --git a/oap-server/generate-tool/src/main/java/org/apache/skywalking/oal/tool/parser/ScriptParser.java b/oap-server/generate-tool/src/main/java/org/apache/skywalking/oal/tool/parser/ScriptParser.java
index 8cce36a..9091fbd 100644
--- a/oap-server/generate-tool/src/main/java/org/apache/skywalking/oal/tool/parser/ScriptParser.java
+++ b/oap-server/generate-tool/src/main/java/org/apache/skywalking/oal/tool/parser/ScriptParser.java
@@ -43,8 +43,9 @@ public class ScriptParser {
         return parser;
     }
 
-    public List<AnalysisResult> parse() throws IOException {
-        List<AnalysisResult> results = new LinkedList<>();
+    public OALScripts parse() throws IOException {
+        OALScripts scripts = new OALScripts();
+
         CommonTokenStream tokens = new CommonTokenStream(lexer);
 
         OALParser parser = new OALParser(tokens);
@@ -52,9 +53,9 @@ public class ScriptParser {
         ParseTree tree = parser.root();
         ParseTreeWalker walker = new ParseTreeWalker();
 
-        walker.walk(new OALListener(results), tree);
+        walker.walk(new OALListener(scripts), tree);
 
-        return results;
+        return scripts;
     }
 
     public void close() {
diff --git a/oap-server/generate-tool/src/main/resources/code-templates/DisableSourceDefinition.ftl b/oap-server/generate-tool/src/main/resources/code-templates/DisableSourceDefinition.ftl
new file mode 100644
index 0000000..b1c9be5
--- /dev/null
+++ b/oap-server/generate-tool/src/main/resources/code-templates/DisableSourceDefinition.ftl
@@ -0,0 +1,33 @@
+/*
+* Licensed to the Apache Software Foundation (ASF) under one or more
+* contributor license agreements.  See the NOTICE file distributed with
+* this work for additional information regarding copyright ownership.
+* The ASF licenses this file to You under the Apache License, Version 2.0
+* (the "License"); you may not use this file except in compliance with
+* the License.  You may obtain a copy of the License at
+*
+*     http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*
+*/
+
+package org.apache.skywalking.oap.server.core.analysis.generated;
+<#if (allDisableSources?size>0)>
+import org.apache.skywalking.oap.server.core.analysis.Disable;
+</#if>
+
+/**
+* This class is auto generated. Please don't change this class manually.
+*
+* @author Observability Analysis Language code generator
+*/
+<#list allDisableSources as disableSource>
+@Disable("${disableSource}")
+</#list>
+public class DisableSourceDefinition {
+}
\ No newline at end of file
diff --git a/oap-server/generate-tool/src/test/java/org/apache/skywalking/oal/tool/output/FileGeneratorTest.java b/oap-server/generate-tool/src/test/java/org/apache/skywalking/oal/tool/output/FileGeneratorTest.java
index 7207a50..1a70826 100644
--- a/oap-server/generate-tool/src/test/java/org/apache/skywalking/oal/tool/output/FileGeneratorTest.java
+++ b/oap-server/generate-tool/src/test/java/org/apache/skywalking/oal/tool/output/FileGeneratorTest.java
@@ -79,10 +79,10 @@ public class FileGeneratorTest {
     public void testGenerateIndicatorImplementor() throws IOException, TemplateException {
         AnalysisResult result = buildResult();
 
-        List<AnalysisResult> results = new LinkedList<>();
-        results.add(result);
+        OALScripts oalScripts = new OALScripts();
+        oalScripts.getIndicatorStmts().add(result);
 
-        FileGenerator fileGenerator = new FileGenerator(results, ".");
+        FileGenerator fileGenerator = new FileGenerator(oalScripts, ".");
         StringWriter writer = new StringWriter();
         fileGenerator.generateIndicatorImplementor(result, writer);
         Assert.assertEquals(readExpectedFile("IndicatorImplementorExpected.java"), writer.toString());
@@ -94,10 +94,10 @@ public class FileGeneratorTest {
     public void testServiceDispatcher() throws IOException, TemplateException {
         AnalysisResult result = buildResult();
 
-        List<AnalysisResult> results = new LinkedList<>();
-        results.add(result);
+        OALScripts oalScripts = new OALScripts();
+        oalScripts.getIndicatorStmts().add(result);
 
-        FileGenerator fileGenerator = new FileGenerator(results, ".");
+        FileGenerator fileGenerator = new FileGenerator(oalScripts, ".");
         StringWriter writer = new StringWriter();
         fileGenerator.generateDispatcher(result, writer);
         Assert.assertEquals(readExpectedFile("ServiceDispatcherExpected.java"), writer.toString());
diff --git a/oap-server/generate-tool/src/test/java/org/apache/skywalking/oal/tool/parser/ScriptParserTest.java b/oap-server/generate-tool/src/test/java/org/apache/skywalking/oal/tool/parser/ScriptParserTest.java
index cab3a93..041f0e7 100644
--- a/oap-server/generate-tool/src/test/java/org/apache/skywalking/oal/tool/parser/ScriptParserTest.java
+++ b/oap-server/generate-tool/src/test/java/org/apache/skywalking/oal/tool/parser/ScriptParserTest.java
@@ -50,7 +50,7 @@ public class ScriptParserTest {
             "Endpoint_avg = from(Endpoint.latency).longAvg(); //comment test" + "\n" +
                 "Service_avg = from(Service.latency).longAvg()"
         );
-        List<AnalysisResult> results = parser.parse();
+        List<AnalysisResult> results = parser.parse().getIndicatorStmts();
 
         Assert.assertEquals(2, results.size());
 
@@ -72,7 +72,7 @@ public class ScriptParserTest {
         ScriptParser parser = ScriptParser.createFromScriptText(
             "Endpoint_percent = from(Endpoint.*).percent(status == true);"
         );
-        List<AnalysisResult> results = parser.parse();
+        List<AnalysisResult> results = parser.parse().getIndicatorStmts();
 
         AnalysisResult endpointPercent = results.get(0);
         Assert.assertEquals("EndpointPercent", endpointPercent.getMetricName());
@@ -91,7 +91,7 @@ public class ScriptParserTest {
         ScriptParser parser = ScriptParser.createFromScriptText(
             "Endpoint_percent = from(Endpoint.*).filter(status == true).filter(name == \"/product/abc\").longAvg();"
         );
-        List<AnalysisResult> results = parser.parse();
+        List<AnalysisResult> results = parser.parse().getIndicatorStmts();
 
         AnalysisResult endpointPercent = results.get(0);
         Assert.assertEquals("EndpointPercent", endpointPercent.getMetricName());
@@ -121,7 +121,7 @@ public class ScriptParserTest {
                 "service_response_s3_summary = from(Service.latency).filter(latency >= 3000).sum();" + "\n" +
                 "service_response_s4_summary = from(Service.latency).filter(latency <= 4000).sum();"
         );
-        List<AnalysisResult> results = parser.parse();
+        List<AnalysisResult> results = parser.parse().getIndicatorStmts();
 
         AnalysisResult responseSummary = results.get(0);
         Assert.assertEquals("ServiceResponseS1Summary", responseSummary.getMetricName());
@@ -167,4 +167,14 @@ public class ScriptParserTest {
         Assert.assertEquals("4000", booleanMatchExp.getValue());
         Assert.assertEquals("lessEqualMatch", booleanMatchExp.getExpressionType());
     }
+
+    @Test
+    public void testDisable() throws IOException {
+        ScriptParser parser = ScriptParser.createFromScriptText(
+            "disable(segment);");
+        DisableCollection collection = parser.parse().getDisableCollection();
+        List<String> sources = collection.getAllDisableSources();
+        Assert.assertEquals(1, sources.size());
+        Assert.assertEquals("segment", sources.get(0));
+    }
 }
diff --git a/oap-server/generated-analysis/src/main/resources/official_analysis.oal b/oap-server/generated-analysis/src/main/resources/official_analysis.oal
index 6c4079f..3b11ce9 100644
--- a/oap-server/generated-analysis/src/main/resources/official_analysis.oal
+++ b/oap-server/generated-analysis/src/main/resources/official_analysis.oal
@@ -96,3 +96,8 @@ instance_clr_max_worker_threads = from(ServiceInstanceCLRThread.maxWorkerThreads
 envoy_heap_memory_max_used = from(EnvoyInstanceMetric.value).filter(metricName == "server.memory_heap_size").maxDouble();
 envoy_total_connections_used = from(EnvoyInstanceMetric.value).filter(metricName == "server.total_connections").maxDouble();
 envoy_parent_connections_used = from(EnvoyInstanceMetric.value).filter(metricName == "server.parent_connections").maxDouble();
+
+// Disable unnecessary hard core sources
+disable(segment);
+disable(endpoint_relation_server_side);
+disable(top_n_database_statement);
\ No newline at end of file
diff --git a/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/CoreModuleProvider.java b/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/CoreModuleProvider.java
index 0cfc956..00d0b39 100644
--- a/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/CoreModuleProvider.java
+++ b/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/CoreModuleProvider.java
@@ -19,6 +19,7 @@
 package org.apache.skywalking.oap.server.core;
 
 import java.io.IOException;
+import org.apache.skywalking.oap.server.core.analysis.DisableRegister;
 import org.apache.skywalking.oap.server.core.analysis.indicator.annotation.IndicatorTypeListener;
 import org.apache.skywalking.oap.server.core.analysis.record.annotation.RecordTypeListener;
 import org.apache.skywalking.oap.server.core.analysis.topn.annotation.TopNTypeListener;
@@ -148,6 +149,7 @@ public class CoreModuleProvider extends ModuleProvider {
         annotationScan.registerListener(new InventoryTypeListener(getManager()));
         annotationScan.registerListener(new RecordTypeListener(getManager()));
         annotationScan.registerListener(new TopNTypeListener(getManager()));
+        annotationScan.registerListener(DisableRegister.INSTANCE);
 
         this.remoteClientManager = new RemoteClientManager(getManager());
         this.registerServiceImplementation(RemoteClientManager.class, remoteClientManager);
diff --git a/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/analysis/Disable.java b/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/analysis/Disable.java
new file mode 100644
index 0000000..4363798
--- /dev/null
+++ b/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/analysis/Disable.java
@@ -0,0 +1,31 @@
+/*
+ * 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.core.analysis;
+
+import java.lang.annotation.*;
+
+/**
+ * @author wusheng
+ */
+@Target(ElementType.TYPE)
+@Retention(RetentionPolicy.RUNTIME)
+@Repeatable(MultipleDisable.class)
+public @interface Disable {
+    String value();
+}
diff --git a/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/analysis/DisableRegister.java b/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/analysis/DisableRegister.java
new file mode 100644
index 0000000..92e99c1
--- /dev/null
+++ b/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/analysis/DisableRegister.java
@@ -0,0 +1,42 @@
+/*
+ * 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.core.analysis;
+
+import java.lang.annotation.Annotation;
+import org.apache.skywalking.oap.server.core.annotation.AnnotationListener;
+
+/**
+ * Find disable definition.
+ *
+ * @author wusheng
+ */
+public class DisableRegister implements AnnotationListener {
+    public static DisableRegister INSTANCE = new DisableRegister();
+
+    private DisableRegister() {
+    }
+
+    @Override public Class<? extends Annotation> annotation() {
+        return MultipleDisable.class;
+    }
+
+    @Override public void notify(Class aClass) {
+
+    }
+}
diff --git a/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/analysis/MultipleDisable.java b/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/analysis/MultipleDisable.java
new file mode 100644
index 0000000..53c829f
--- /dev/null
+++ b/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/analysis/MultipleDisable.java
@@ -0,0 +1,27 @@
+/*
+ * 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.core.analysis;
+
+import java.lang.annotation.*;
+
+@Target(ElementType.TYPE)
+@Retention(RetentionPolicy.RUNTIME)
+public @interface MultipleDisable {
+    Disable[] value();
+}
diff --git a/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/analysis/worker/IndicatorProcess.java b/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/analysis/worker/IndicatorProcess.java
index 69fbfff..1863238 100644
--- a/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/analysis/worker/IndicatorProcess.java
+++ b/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/analysis/worker/IndicatorProcess.java
@@ -37,7 +37,10 @@ public enum IndicatorProcess {
     @Getter private List<IndicatorPersistentWorker> persistentWorkers = new ArrayList<>();
 
     public void in(Indicator indicator) {
-        entryWorkers.get(indicator.getClass()).in(indicator);
+        IndicatorAggregateWorker worker = entryWorkers.get(indicator.getClass());
+        if (worker != null) {
+            worker.in(indicator);
+        }
     }
 
     public void create(ModuleManager moduleManager, Class<? extends Indicator> indicatorClass) {
diff --git a/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/analysis/worker/RecordProcess.java b/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/analysis/worker/RecordProcess.java
index b384ccd..cc26dd8 100644
--- a/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/analysis/worker/RecordProcess.java
+++ b/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/analysis/worker/RecordProcess.java
@@ -36,7 +36,10 @@ public enum RecordProcess {
     private Map<Class<? extends Record>, RecordPersistentWorker> workers = new HashMap<>();
 
     public void in(Record record) {
-        workers.get(record.getClass()).in(record);
+        RecordPersistentWorker worker = workers.get(record.getClass());
+        if (worker != null) {
+            worker.in(record);
+        }
     }
 
     @Getter private List<RecordPersistentWorker> persistentWorkers = new ArrayList<>();
diff --git a/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/analysis/worker/TopNProcess.java b/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/analysis/worker/TopNProcess.java
index b83851a..f62fc18 100644
--- a/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/analysis/worker/TopNProcess.java
+++ b/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/analysis/worker/TopNProcess.java
@@ -30,8 +30,8 @@ import org.apache.skywalking.oap.server.core.worker.*;
 import org.apache.skywalking.oap.server.library.module.ModuleManager;
 
 /**
- * TopN is a special process, which hold a certain size of windows,
- * and cache all top N records, save to the persistence in low frequence.
+ * TopN is a special process, which hold a certain size of windows, and cache all top N records, save to the persistence
+ * in low frequence.
  *
  * @author wusheng
  */
@@ -61,6 +61,9 @@ public enum TopNProcess {
     }
 
     public void in(TopNDatabaseStatement statement) {
-        workers.get(statement.getClass()).in(statement);
+        TopNWorker worker = workers.get(statement.getClass());
+        if (worker != null) {
+            worker.in(statement);
+        }
     }
 }