You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@skywalking.apache.org by ha...@apache.org on 2020/09/21 16:12:43 UTC

[skywalking] 01/01: Document MAL

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

hanahmily pushed a commit to branch meter-mal
in repository https://gitbox.apache.org/repos/asf/skywalking.git

commit 42324480fa68d482c58c1e9546facffeffdde0c0
Author: Gao Hongtao <ha...@gmail.com>
AuthorDate: Tue Sep 22 00:11:45 2020 +0800

    Document MAL
    
    Signed-off-by: Gao Hongtao <ha...@gmail.com>
---
 docs/en/concepts-and-designs/mal.md                | 188 +++++++++++++++++++++
 docs/en/concepts-and-designs/meter.md              |   2 +
 oap-server/analyzer/{ => meter-analyzer}/pom.xml   |  29 ++--
 .../skywalking/oap/meter/analyzer/dsl/DSL.java     |  34 ++++
 .../oap/meter/analyzer/dsl/Expression.java         |  62 +++++++
 .../skywalking/oap/meter/analyzer/dsl/Result.java  |  47 ++++++
 .../skywalking/oap/meter/analyzer/dsl/Sample.java  |  42 +++++
 .../oap/meter/analyzer/dsl/SampleFamily.java       | 163 ++++++++++++++++++
 .../skywalking/oap/meter/analyzer/dsl/DSLTest.java | 104 ++++++++++++
 oap-server/analyzer/pom.xml                        |   1 +
 10 files changed, 653 insertions(+), 19 deletions(-)

diff --git a/docs/en/concepts-and-designs/mal.md b/docs/en/concepts-and-designs/mal.md
new file mode 100644
index 0000000..0012d3b
--- /dev/null
+++ b/docs/en/concepts-and-designs/mal.md
@@ -0,0 +1,188 @@
+# Meter Analysis Language
+
+Meter system provides a functional analysis language called MAL(Meter Analysis Language) that lets the user analyze and 
+aggregate meter metric data in OAP streaming system. The result of an expression can either be ingested by agent analyzer,
+or OC/Prometheus analyzer.
+
+## Language data type
+
+In MAL, an expression or sub-expression can evaluate to one of two types:
+
+ - Sample family -  a set of samples(metrics) containing a range of metrics whose name is identical.
+ - Scalar - a simple numeric value. it supports integer/long, floating/double,
+
+## Sample family
+
+A set of samples, which is as the basic unit in MAL. For example:
+
+```
+instance_trace_count
+```
+
+The above sample family might contains following simples which are provided by external modules, for instance, agent analyzer:
+
+```
+instance_trace_count{region="us-west",az="az-1"} 100
+instance_trace_count{region="us-east",az="az-3"} 20
+instance_trace_count{region="asia-north",az="az-1"} 33
+```
+
+### Tag filter
+
+MAL support four type operations to filter samples in a sample family:
+
+ - tagEqual: Filter tags that are exactly equal to the provided string.
+ - tagNotEqual: Filter tags that are not equal to the provided string.
+ - tagMatch: Filter tags that regex-match the provided string.
+ - tagNotMatch: Filter labels that do not regex-match the provided string.
+
+For example, this filters all instance_trace_count samples for us-west and asia-north region and az-1 az:
+
+```
+instance_trace_count.tagMatch("region", "us-west|asia-north").tagEqual("az", "az-1")
+```
+
+### Binary operators
+
+The following binary arithmetic operators are available in MAL:
+
+ - + (addition)
+ - - (subtraction)
+ - * (multiplication)
+ - / (division)
+
+Binary operators are defined between scalar/scalar, sampleFamily/scalar and sampleFamily/sampleFamily value pairs.
+
+Between two scalars: they evaluate to another scalar that is the result of the operator applied to both scalar operands:
+
+```
+1 + 2
+```
+
+Between a sample family and a scalar, the operator is applied to the value of every sample in the smaple family. For example:
+
+```
+instance_trace_count + 2
+``` 
+
+results in
+
+```
+instance_trace_count{region="us-west",az="az-1"} 102 // 100 + 2
+instance_trace_count{region="us-east",az="az-3"} 22 // 20 + 2
+instance_trace_count{region="asia-north",az="az-1"} 35 // 33 + 2
+```
+
+Between two sample families, a binary operator is applied to each sample in the left-hand side sample family and 
+its matching sample in the right-hand sample family. A new sample family with empty name will be generated.
+Only the matched tags will be reserved. Samples for which no matching sample in the right-hand sample family are not in the result.
+
+Another sample family `instance_trace_analysis_error_count` is 
+
+```
+instance_trace_analysis_error_count{region="us-west",az="az-1"} 20
+instance_trace_analysis_error_count{region="asia-north",az="az-1"} 11 
+```
+
+Example expression:
+
+```
+instance_trace_analysis_error_count / instance_trace_count
+```
+
+This returns a result sample family containing the error rate of trace analysis. The samples with region us-west and az az-3 
+have no match and will not show up in the result:
+
+```
+{region="us-west",az="az-1"} 0.8  // 20 / 100
+{region="asia-north",az="az-1"} 0.3333  // 11 / 33
+```
+
+### Aggregation Operation
+
+Sample family supports the following aggregation operations that can be used to aggregate the samples of a single sample family,
+resulting in a new sample family of fewer samples(even single one) with aggregated values:
+
+ - sum (calculate sum over dimensions)
+ - min (select minimum over dimensions)
+ - max (select maximum over dimensions)
+ - avg (calculate the average over dimensions)
+ 
+These operations can be used to aggregate over all label dimensions or preserve distinct dimensions by inputting `by` parameter. 
+
+```
+<aggr-op>(by: <tag1, tag2, ...>)
+```
+
+Example expression:
+
+```
+instance_trace_count.sum(by: ['az'])
+```
+
+will output a result:
+
+```
+instance_trace_count{az="az-1"} 133 // 100 + 33
+instance_trace_count{az="az-3"} 20
+```
+
+### Function
+
+`Duraton` is a textual representation of a time range. The formats accepted are based on the ISO-8601 duration format {@code PnDTnHnMn.nS}
+ with days considered to be exactly 24 hours.
+
+Examples:
+ - "PT20.345S" -- parses as "20.345 seconds"
+ - "PT15M"     -- parses as "15 minutes" (where a minute is 60 seconds)
+ - "PT10H"     -- parses as "10 hours" (where an hour is 3600 seconds)
+ - "P2D"       -- parses as "2 days" (where a day is 24 hours or 86400 seconds)
+ - "P2DT3H4M"  -- parses as "2 days, 3 hours and 4 minutes"
+ - "P-6H3M"    -- parses as "-6 hours and +3 minutes"
+ - "-P6H3M"    -- parses as "-6 hours and -3 minutes"
+ - "-P-6H+3M"  -- parses as "+6 hours and -3 minutes"
+
+#### increase
+`increase(Duration)`. Calculates the increase in the time range.
+
+#### rate
+`rate(Duration)`. Calculates the per-second average rate of increase of the time range.
+
+#### irate
+`irate()`. Calculates the per-second instant rate of increase of the time range.
+
+#### tag
+`tag({allTags -> })`. Update tags of samples. User can add, drop, rename and update tags.
+
+#### histogram
+`histogram(le: '<the tag name of le>')`. Transforms less based histogram buckets to meter system histogram buckets. 
+`le` parameter hints the tag name of a bucket. 
+
+#### histogram_quantile
+`histogram_quantile([<φ scalar>])`. Hints meter-system to calculates the φ-quantile (0 ≤ φ ≤ 100) from the buckets. 
+
+## Down Sampling Operation
+MAL should instruct meter-system how to do downsampling for metrics. It doesn't only refer to aggregate raw samples to 
+`minute` level, but also hints data from `minute` to higher levels, for instance, `hour` and `day`. 
+
+Down sampling operations are as global function in MAL:
+
+ - avg
+ - latest
+ - min
+ - max
+ - mean
+ - sum
+ - count
+
+The default one is `avg` if not specific an operation.
+
+If user want get latest time from `last_server_state_sync_time_in_seconds`:
+
+```
+latest(last_server_state_sync_time_in_seconds.tagEqual('production', 'catalog'))
+
+or
+
+latest last_server_state_sync_time_in_seconds.tagEqual('production', 'catalog')
+```
diff --git a/docs/en/concepts-and-designs/meter.md b/docs/en/concepts-and-designs/meter.md
index d07160d..270d347 100644
--- a/docs/en/concepts-and-designs/meter.md
+++ b/docs/en/concepts-and-designs/meter.md
@@ -22,3 +22,5 @@ Meter System supports following binding functions
 - **percentile**. Read [percentile in WIKI](https://en.wikipedia.org/wiki/Percentile). Unlike in the OAL, we provide
 50/75/90/95/99 in default, in the meter system function, percentile function accepts several ranks, which should be in
 the (0, 100) range.
+
+## 
diff --git a/oap-server/analyzer/pom.xml b/oap-server/analyzer/meter-analyzer/pom.xml
similarity index 56%
copy from oap-server/analyzer/pom.xml
copy to oap-server/analyzer/meter-analyzer/pom.xml
index 90c758d..0957a11 100644
--- a/oap-server/analyzer/pom.xml
+++ b/oap-server/analyzer/meter-analyzer/pom.xml
@@ -17,36 +17,27 @@
   ~
   -->
 
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
     <parent>
-        <artifactId>oap-server</artifactId>
+        <artifactId>analyzer</artifactId>
         <groupId>org.apache.skywalking</groupId>
         <version>8.2.0-SNAPSHOT</version>
     </parent>
     <modelVersion>4.0.0</modelVersion>
 
-    <artifactId>analyzer</artifactId>
-    <packaging>pom</packaging>
-
-    <modules>
-        <module>agent-analyzer</module>
-    </modules>
+    <artifactId>meter-analyzer</artifactId>
 
     <dependencies>
         <dependency>
-            <groupId>org.apache.skywalking</groupId>
-            <artifactId>apm-network</artifactId>
-            <version>${project.version}</version>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.skywalking</groupId>
-            <artifactId>library-module</artifactId>
-            <version>${project.version}</version>
+            <groupId>org.codehaus.groovy</groupId>
+            <artifactId>groovy</artifactId>
         </dependency>
         <dependency>
-            <groupId>org.apache.skywalking</groupId>
-            <artifactId>library-util</artifactId>
-            <version>${project.version}</version>
+            <groupId>io.vavr</groupId>
+            <artifactId>vavr</artifactId>
         </dependency>
     </dependencies>
+
 </project>
\ No newline at end of file
diff --git a/oap-server/analyzer/meter-analyzer/src/main/java/org/apache/skywalking/oap/meter/analyzer/dsl/DSL.java b/oap-server/analyzer/meter-analyzer/src/main/java/org/apache/skywalking/oap/meter/analyzer/dsl/DSL.java
new file mode 100644
index 0000000..24a287f
--- /dev/null
+++ b/oap-server/analyzer/meter-analyzer/src/main/java/org/apache/skywalking/oap/meter/analyzer/dsl/DSL.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.oap.meter.analyzer.dsl;
+
+import groovy.lang.Binding;
+import groovy.lang.GroovyShell;
+import groovy.util.DelegatingScript;
+import org.codehaus.groovy.control.CompilerConfiguration;
+
+public final class DSL {
+    public static Expression parse(final String expression) {
+        CompilerConfiguration cc = new CompilerConfiguration();
+        cc.setScriptBaseClass(DelegatingScript.class.getName());
+        GroovyShell sh = new GroovyShell(new Binding(), cc);
+        DelegatingScript script = (DelegatingScript) sh.parse(expression);
+        return new Expression(script);
+    }
+}
diff --git a/oap-server/analyzer/meter-analyzer/src/main/java/org/apache/skywalking/oap/meter/analyzer/dsl/Expression.java b/oap-server/analyzer/meter-analyzer/src/main/java/org/apache/skywalking/oap/meter/analyzer/dsl/Expression.java
new file mode 100644
index 0000000..c58f19b
--- /dev/null
+++ b/oap-server/analyzer/meter-analyzer/src/main/java/org/apache/skywalking/oap/meter/analyzer/dsl/Expression.java
@@ -0,0 +1,62 @@
+/*
+ * 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.meter.analyzer.dsl;
+
+import com.google.common.collect.ImmutableMap;
+import groovy.lang.GroovyObjectSupport;
+import groovy.util.DelegatingScript;
+import lombok.extern.slf4j.Slf4j;
+
+@Slf4j
+public class Expression {
+
+    private final DelegatingScript expression;
+
+    Expression(DelegatingScript expression) {
+        this.expression = expression;
+    }
+
+    public Result run(final ImmutableMap<String, SampleFamily> sampleFamilies) {
+        expression.setDelegate(new GroovyObjectSupport() {
+
+            public SampleFamily propertyMissing(String metricName) {
+                if (sampleFamilies.containsKey(metricName)) {
+                    return sampleFamilies.get(metricName);
+                }
+                if (log.isDebugEnabled()) {
+                    log.debug("{} doesn't exist in {}", metricName, sampleFamilies.keySet());
+                }
+                throw new IllegalArgumentException("[" + metricName + "] can't be found");
+            }
+
+            @Override public Object invokeMethod(String name, Object args) {
+                //TODO: Validate the name is one of meter functions
+                return super.invokeMethod(name, args);
+            }
+
+        });
+        try {
+            SampleFamily sf = (SampleFamily) expression.run();
+            return Result.success(sf);
+        } catch (Throwable t) {
+            return Result.fail(t);
+        }
+    }
+
+}
diff --git a/oap-server/analyzer/meter-analyzer/src/main/java/org/apache/skywalking/oap/meter/analyzer/dsl/Result.java b/oap-server/analyzer/meter-analyzer/src/main/java/org/apache/skywalking/oap/meter/analyzer/dsl/Result.java
new file mode 100644
index 0000000..4c1b2d0
--- /dev/null
+++ b/oap-server/analyzer/meter-analyzer/src/main/java/org/apache/skywalking/oap/meter/analyzer/dsl/Result.java
@@ -0,0 +1,47 @@
+/*
+ * 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.meter.analyzer.dsl;
+
+import lombok.AccessLevel;
+import lombok.EqualsAndHashCode;
+import lombok.RequiredArgsConstructor;
+import lombok.ToString;
+import lombok.extern.slf4j.Slf4j;
+
+@Slf4j
+@RequiredArgsConstructor(access = AccessLevel.PRIVATE)
+@EqualsAndHashCode
+@ToString
+public class Result {
+    public static Result fail(final Throwable throwable) {
+        log.error("Result is failure", throwable);
+        return new Result(false, SampleFamily.EMPTY);
+    }
+
+    public static Result success(SampleFamily sf) {
+        if (log.isDebugEnabled()) {
+            log.debug("Result is successful, sample family is {}", sf);
+        }
+        return new Result(true, sf);
+    }
+
+    private final boolean success;
+
+    private final SampleFamily data;
+}
diff --git a/oap-server/analyzer/meter-analyzer/src/main/java/org/apache/skywalking/oap/meter/analyzer/dsl/Sample.java b/oap-server/analyzer/meter-analyzer/src/main/java/org/apache/skywalking/oap/meter/analyzer/dsl/Sample.java
new file mode 100644
index 0000000..6daf65a
--- /dev/null
+++ b/oap-server/analyzer/meter-analyzer/src/main/java/org/apache/skywalking/oap/meter/analyzer/dsl/Sample.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.meter.analyzer.dsl;
+
+import com.google.common.collect.ImmutableMap;
+import java.util.function.Function;
+import lombok.Builder;
+import lombok.EqualsAndHashCode;
+import lombok.ToString;
+
+@Builder
+@EqualsAndHashCode
+@ToString
+public class Sample {
+    final ImmutableMap<String, String> labels;
+    final double value;
+    final long timeStamp;
+
+    Sample newValue(Function<Double, Double> transform) {
+        Sample.SampleBuilder b =  Sample.builder();
+        b.timeStamp(timeStamp);
+        b.labels(labels);
+        b.value(transform.apply(value));
+        return b.build();
+    }
+}
diff --git a/oap-server/analyzer/meter-analyzer/src/main/java/org/apache/skywalking/oap/meter/analyzer/dsl/SampleFamily.java b/oap-server/analyzer/meter-analyzer/src/main/java/org/apache/skywalking/oap/meter/analyzer/dsl/SampleFamily.java
new file mode 100644
index 0000000..5c31905
--- /dev/null
+++ b/oap-server/analyzer/meter-analyzer/src/main/java/org/apache/skywalking/oap/meter/analyzer/dsl/SampleFamily.java
@@ -0,0 +1,163 @@
+/*
+ * 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.meter.analyzer.dsl;
+
+import com.google.common.base.Preconditions;
+import groovy.lang.Closure;
+import io.vavr.Function2;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.function.Function;
+import java.util.stream.Stream;
+import lombok.AccessLevel;
+import lombok.EqualsAndHashCode;
+import lombok.RequiredArgsConstructor;
+import lombok.ToString;
+
+@RequiredArgsConstructor(access = AccessLevel.PRIVATE)
+@EqualsAndHashCode
+@ToString
+public class SampleFamily {
+    public static final SampleFamily EMPTY = new SampleFamily(new Sample[0]);
+
+    public static SampleFamily build(Sample... samples) {
+        return new SampleFamily(samples);
+    }
+
+    private final Sample[] samples;
+
+    /**
+     * Following operations are used in DSL
+     */
+
+    /* tag filter operations*/
+    public SampleFamily tagEqual(String... labels) {
+        return match(labels, String::equals);
+    }
+
+    public SampleFamily tagNotEqual(String[] labels) {
+        return match(labels, (sv, lv) -> !sv.equals(lv));
+    }
+
+    public SampleFamily tagMatch(String[] labels) {
+        return match(labels, (sv, lv) -> lv.matches(sv));
+    }
+
+    public SampleFamily tagNotMatch(String[] labels) {
+        return match(labels, (sv, lv) -> !lv.matches(sv));
+    }
+
+    /* Operator overloading*/
+    public SampleFamily plus(Number number) {
+        return newValue(v -> v + number.doubleValue());
+    }
+
+    public SampleFamily minus(Number number) {
+        return newValue(v -> v - number.doubleValue());
+    }
+
+    public SampleFamily multiply(Number number) {
+        return newValue(v -> v * number.doubleValue());
+    }
+
+    public SampleFamily div(Number number) {
+        return newValue(v -> v / number.doubleValue());
+    }
+
+    public SampleFamily plus(SampleFamily number) {
+        return this;
+    }
+
+    public SampleFamily minus(SampleFamily number) {
+        return this;
+    }
+
+    public SampleFamily multiply(SampleFamily number) {
+        return this;
+    }
+
+    public SampleFamily div(SampleFamily number) {
+        return this;
+    }
+
+    /* Aggregation operators */
+    @RequiredArgsConstructor
+    public static class AggregationParameter {
+        private final String[] by;
+    }
+
+    public SampleFamily sum(AggregationParameter parameter) {
+        return this;
+    }
+
+    /* Function */
+    public SampleFamily increase(String range) {
+        return this;
+    }
+
+    public SampleFamily rate(String range) {
+        return this;
+    }
+
+    public SampleFamily irate() {
+        return this;
+    }
+
+    public SampleFamily tag(Closure cl) {
+        return this;
+    }
+
+    @RequiredArgsConstructor
+    public static class HistogramParameter {
+        private final String le;
+    }
+
+    public  SampleFamily histogram(HistogramParameter parameter) {
+       return this;
+    }
+
+    public SampleFamily histogram_quantile(int[] range) {
+        return  this;
+    }
+
+    private SampleFamily match(String[] labels, Function2<String, String, Boolean> op) {
+        Preconditions.checkArgument(labels.length % 2 == 0);
+        Map<String, String> ll = new HashMap<>(labels.length / 2);
+        for (int i = 0; i < labels.length; i += 2) {
+            ll.put(labels[i], labels[i + 1]);
+        }
+        Stream<Sample> ss = Arrays.stream(samples).filter(sample ->
+            ll.entrySet().stream().allMatch(entry ->
+                sample.labels.containsKey(entry.getKey()) && op.apply(sample.labels.get(entry.getKey()), entry.getValue())));
+        Sample[] sArr = ss.toArray(Sample[]::new);
+        if (sArr.length < 1) {
+            return SampleFamily.EMPTY;
+        }
+        return SampleFamily.build(sArr);
+    }
+
+    private SampleFamily newValue(Function<Double, Double> transform) {
+        Sample[] ss = new Sample[samples.length];
+        for (int i = 0; i < ss.length; i++) {
+            ss[i] = samples[i].newValue(transform);
+        }
+        return SampleFamily.build(ss);
+    }
+}
diff --git a/oap-server/analyzer/meter-analyzer/src/test/java/org/apache/skywalking/oap/meter/analyzer/dsl/DSLTest.java b/oap-server/analyzer/meter-analyzer/src/test/java/org/apache/skywalking/oap/meter/analyzer/dsl/DSLTest.java
new file mode 100644
index 0000000..5324ec8
--- /dev/null
+++ b/oap-server/analyzer/meter-analyzer/src/test/java/org/apache/skywalking/oap/meter/analyzer/dsl/DSLTest.java
@@ -0,0 +1,104 @@
+/*
+ * 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.meter.analyzer.dsl;
+
+import com.google.common.collect.ImmutableMap;
+import java.util.Arrays;
+import java.util.Collection;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import static com.google.common.collect.ImmutableMap.of;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.CoreMatchers.is;
+
+@RunWith(Parameterized.class)
+public class DSLTest {
+
+    @Parameterized.Parameter
+    public String name;
+
+    @Parameterized.Parameter(1)
+    public ImmutableMap<String, SampleFamily> input;
+
+    @Parameterized.Parameter(2)
+    public String expression;
+
+    @Parameterized.Parameter(3)
+    public Result want;
+
+    @Parameterized.Parameters(name = "{index}: {0}")
+    public static Collection<Object[]> data() {
+        return Arrays.asList(new Object[][]{
+            {
+               "default",
+                of("instance_cpu_percentage", SampleFamily.EMPTY),
+                "instance_cpu_percentage",
+                Result.success(SampleFamily.EMPTY),
+            },
+            {
+                "single-value",
+                of("instance_cpu_percentage", SampleFamily.build(Sample.builder().value(1600592418480.0).build())),
+                "instance_cpu_percentage",
+                Result.success(SampleFamily.build(Sample.builder().value(1600592418480.0).build())),
+            },
+            {
+                "label-equal",
+                of("instance_cpu_percentage", SampleFamily.build(
+                    Sample.builder().labels(of("idc", "t1")).value(1600592418480.0).build(),
+                    Sample.builder().labels(of("idc", "t2")).value(1600592418481.0).build()
+                )),
+                "instance_cpu_percentage.tagEqual('idc','t1')",
+                Result.success(SampleFamily.build(
+                    Sample.builder().labels(of("idc", "t1")).value(1600592418480.0).build()
+                )),
+            },
+            {
+                "label-not-equal",
+                of("instance_cpu_percentage", SampleFamily.build(
+                    Sample.builder().labels(of("idc", "t1")).value(1600592418480.0).build(),
+                    Sample.builder().labels(of("idc", "t2")).value(1600592418481.0).build()
+                )),
+                "instance_cpu_percentage.tagNotEqual('idc','t2')",
+                Result.success(SampleFamily.build(
+                    Sample.builder().labels(of("idc", "t1")).value(1600592418480.0).build()
+                )),
+            },
+            {
+                "plus",
+                of("instance_cpu_percentage", SampleFamily.build(
+                    Sample.builder().labels(of("idc", "t1")).value(1600592418480.0).build(),
+                    Sample.builder().labels(of("idc", "t2")).value(1600592418481.0).build()
+                )),
+                "instance_cpu_percentage.tagEqual('idc','t1') + 1000",
+                Result.success(SampleFamily.build(
+                    Sample.builder().labels(of("idc", "t1")).value(1600592419480.0).build()
+                )),
+            },
+        });
+    }
+
+    @Test
+    public void test() {
+        Expression e = DSL.parse(expression);
+        Result r = e.run(input);
+        assertThat(r, is(want));
+    }
+}
\ No newline at end of file
diff --git a/oap-server/analyzer/pom.xml b/oap-server/analyzer/pom.xml
index 90c758d..b147071 100644
--- a/oap-server/analyzer/pom.xml
+++ b/oap-server/analyzer/pom.xml
@@ -30,6 +30,7 @@
 
     <modules>
         <module>agent-analyzer</module>
+        <module>meter-analyzer</module>
     </modules>
 
     <dependencies>