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/05 13:49:55 UTC

[incubator-skywalking] branch envoy-metric created (now 9b8ced0)

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

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


      at 9b8ced0  Extend envoy metric service, scope and OAL lexer/parser.

This branch includes the following new commits:

     new 9b8ced0  Extend envoy metric service, scope and OAL lexer/parser.

The 1 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.



[incubator-skywalking] 01/01: Extend envoy metric service, scope and OAL lexer/parser.

Posted by wu...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 9b8ced0dde041e6ae967e79c7c4300ebfaa2616f
Author: Wu Sheng <wu...@foxmail.com>
AuthorDate: Tue Mar 5 21:49:47 2019 +0800

    Extend envoy metric service, scope and OAL lexer/parser.
---
 .../apache/skywalking/oal/tool/grammar/OALLexer.g4 |  1 +
 .../skywalking/oal/tool/grammar/OALParser.g4       |  3 +-
 .../oap/server/core/source/DefaultScopeDefine.java |  7 ++
 .../server/core/source/EnvoyInstanceMetric.java    | 51 ++++++++++++++
 .../receiver/envoy/EnvoyMetricReceiverModule.java  | 38 +++++++++++
 .../envoy/EnvoyMetricReceiverProvider.java         | 58 ++++++++++++++++
 .../receiver/envoy/MetricServiceGRPCHandler.java   | 78 ++++++++++++++++++++++
 ...ywalking.oap.server.library.module.ModuleDefine | 20 ++++++
 ...alking.oap.server.library.module.ModuleProvider | 19 ++++++
 9 files changed, 274 insertions(+), 1 deletion(-)

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 9eefd44..c9fb889 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
@@ -41,6 +41,7 @@ SRC_DATABASE_ACCESS: 'DatabaseAccess';
 SRC_SERVICE_INSTANCE_CLR_CPU: 'ServiceInstanceCLRCPU';
 SRC_SERVICE_INSTANCE_CLR_GC: 'ServiceInstanceCLRGC';
 SRC_SERVICE_INSTANCE_CLR_THREAD: 'ServiceInstanceCLRThread';
+SRC_ENVOY_INSTANCE_METRIC: 'EnvoyInstanceMetric';
 
 // Literals
 
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 a94dbe2..086348c 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
@@ -49,7 +49,8 @@ source
     : SRC_ALL | SRC_SERVICE | SRC_DATABASE_ACCESS | SRC_SERVICE_INSTANCE | SRC_ENDPOINT |
       SRC_SERVICE_RELATION | SRC_SERVICE_INSTANCE_RELATION | SRC_ENDPOINT_RELATION |
       SRC_SERVICE_INSTANCE_JVM_CPU | SRC_SERVICE_INSTANCE_JVM_MEMORY | SRC_SERVICE_INSTANCE_JVM_MEMORY_POOL | SRC_SERVICE_INSTANCE_JVM_GC |// JVM source of service instance
-      SRC_SERVICE_INSTANCE_CLR_CPU | SRC_SERVICE_INSTANCE_CLR_GC | SRC_SERVICE_INSTANCE_CLR_THREAD
+      SRC_SERVICE_INSTANCE_CLR_CPU | SRC_SERVICE_INSTANCE_CLR_GC | SRC_SERVICE_INSTANCE_CLR_THREAD |
+      SRC_ENVOY_INSTANCE_METRIC
     ;
 
 sourceAttribute
diff --git a/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/source/DefaultScopeDefine.java b/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/source/DefaultScopeDefine.java
index 9cccb2e..d6576e3 100644
--- a/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/source/DefaultScopeDefine.java
+++ b/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/source/DefaultScopeDefine.java
@@ -53,6 +53,13 @@ public class DefaultScopeDefine {
     public static final int SERVICE_INSTANCE_CLR_CPU = 19;
     public static final int SERVICE_INSTANCE_CLR_GC = 20;
     public static final int SERVICE_INSTANCE_CLR_THREAD = 21;
+    public static final int ENVOY_INSTANCE_METRIC = 22;
+
+    /**
+     * All metric IDs in [0, 10,000) are reserved in Apache SkyWalking.
+     *
+     * If you want to extend the scope, recommend to start with 10,000.
+     */
 
     public static class Listener implements AnnotationListener {
         @Override public Class<? extends Annotation> annotation() {
diff --git a/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/source/EnvoyInstanceMetric.java b/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/source/EnvoyInstanceMetric.java
new file mode 100644
index 0000000..0a5eafb
--- /dev/null
+++ b/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/source/EnvoyInstanceMetric.java
@@ -0,0 +1,51 @@
+/*
+ * 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.source;
+
+import lombok.*;
+
+import static org.apache.skywalking.oap.server.core.source.DefaultScopeDefine.ENVOY_INSTANCE_METRIC;
+
+/**
+ * The envoy metrics. This group of metrics are in Prometheus metric format family.
+ *
+ * This metric source supports Counter and Gauge types.
+ *
+ * @author wusheng
+ */
+@ScopeDeclaration(id = ENVOY_INSTANCE_METRIC, name = "EnvoyInstanceMetric")
+public class EnvoyInstanceMetric extends Source {
+    @Override public int scope() {
+        return ENVOY_INSTANCE_METRIC;
+    }
+
+    @Override public String getEntityId() {
+        return String.valueOf(id);
+    }
+
+    /**
+     * Instance id
+     */
+    @Getter @Setter private int id;
+    @Getter @Setter private int serviceId;
+    @Getter @Setter private String name;
+    @Getter @Setter private String serviceName;
+    @Getter @Setter private String metricName;
+    @Getter @Setter private double value;
+}
diff --git a/oap-server/server-receiver-plugin/envoy-metrics-receiver-plugin/src/main/java/org/apache/skywalking/oap/server/receiver/envoy/EnvoyMetricReceiverModule.java b/oap-server/server-receiver-plugin/envoy-metrics-receiver-plugin/src/main/java/org/apache/skywalking/oap/server/receiver/envoy/EnvoyMetricReceiverModule.java
new file mode 100644
index 0000000..686f923
--- /dev/null
+++ b/oap-server/server-receiver-plugin/envoy-metrics-receiver-plugin/src/main/java/org/apache/skywalking/oap/server/receiver/envoy/EnvoyMetricReceiverModule.java
@@ -0,0 +1,38 @@
+/*
+ * 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.envoy;
+
+import org.apache.skywalking.oap.server.library.module.ModuleDefine;
+
+/**
+ * Envoy metric receiver module
+ *
+ * @author wusheng
+ */
+public class EnvoyMetricReceiverModule extends ModuleDefine {
+    public static final String NAME = "envoy-metric";
+
+    public EnvoyMetricReceiverModule() {
+        super(NAME);
+    }
+
+    @Override public Class[] services() {
+        return new Class[0];
+    }
+}
diff --git a/oap-server/server-receiver-plugin/envoy-metrics-receiver-plugin/src/main/java/org/apache/skywalking/oap/server/receiver/envoy/EnvoyMetricReceiverProvider.java b/oap-server/server-receiver-plugin/envoy-metrics-receiver-plugin/src/main/java/org/apache/skywalking/oap/server/receiver/envoy/EnvoyMetricReceiverProvider.java
new file mode 100644
index 0000000..1631615
--- /dev/null
+++ b/oap-server/server-receiver-plugin/envoy-metrics-receiver-plugin/src/main/java/org/apache/skywalking/oap/server/receiver/envoy/EnvoyMetricReceiverProvider.java
@@ -0,0 +1,58 @@
+/*
+ * 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.envoy;
+
+import org.apache.skywalking.oap.server.core.CoreModule;
+import org.apache.skywalking.oap.server.core.server.GRPCHandlerRegister;
+import org.apache.skywalking.oap.server.library.module.*;
+import org.apache.skywalking.oap.server.telemetry.TelemetryModule;
+
+/**
+ * @author wusheng
+ */
+public class EnvoyMetricReceiverProvider extends ModuleProvider {
+    @Override public String name() {
+        return "default";
+    }
+
+    @Override public Class<? extends ModuleDefine> module() {
+        return EnvoyMetricReceiverModule.class;
+    }
+
+    @Override public ModuleConfig createConfigBeanIfAbsent() {
+        return null;
+    }
+
+    @Override public void prepare() throws ServiceNotProvidedException, ModuleStartException {
+
+    }
+
+    @Override public void start() throws ServiceNotProvidedException, ModuleStartException {
+        GRPCHandlerRegister service = getManager().find(CoreModule.NAME).provider().getService(GRPCHandlerRegister.class);
+        service.addHandler(new MetricServiceGRPCHandler(getManager()));
+    }
+
+    @Override public void notifyAfterCompleted() throws ServiceNotProvidedException, ModuleStartException {
+
+    }
+
+    @Override public String[] requiredModules() {
+        return new String[] {TelemetryModule.NAME, CoreModule.NAME};
+    }
+}
diff --git a/oap-server/server-receiver-plugin/envoy-metrics-receiver-plugin/src/main/java/org/apache/skywalking/oap/server/receiver/envoy/MetricServiceGRPCHandler.java b/oap-server/server-receiver-plugin/envoy-metrics-receiver-plugin/src/main/java/org/apache/skywalking/oap/server/receiver/envoy/MetricServiceGRPCHandler.java
new file mode 100644
index 0000000..3bc6fe3
--- /dev/null
+++ b/oap-server/server-receiver-plugin/envoy-metrics-receiver-plugin/src/main/java/org/apache/skywalking/oap/server/receiver/envoy/MetricServiceGRPCHandler.java
@@ -0,0 +1,78 @@
+/*
+ * 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.envoy;
+
+import io.envoyproxy.envoy.api.v2.core.Node;
+import io.envoyproxy.envoy.service.metrics.v2.*;
+import io.grpc.stub.StreamObserver;
+import io.prometheus.client.Metrics;
+import org.apache.skywalking.apm.util.StringUtil;
+import org.apache.skywalking.oap.server.library.module.ModuleManager;
+import org.slf4j.*;
+
+/**
+ * @author wusheng
+ */
+public class MetricServiceGRPCHandler extends MetricsServiceGrpc.MetricsServiceImplBase {
+    private static final Logger logger = LoggerFactory.getLogger(MetricServiceGRPCHandler.class);
+
+    public MetricServiceGRPCHandler(ModuleManager moduleManager
+    ) {
+    }
+
+    @Override
+    public StreamObserver<StreamMetricsMessage> streamMetrics(StreamObserver<StreamMetricsResponse> responseObserver) {
+        return new StreamObserver<StreamMetricsMessage>() {
+            private boolean isFirst = true;
+            private String serviceName = null;
+
+            @Override public void onNext(StreamMetricsMessage message) {
+                if (isFirst) {
+                    isFirst = false;
+                    StreamMetricsMessage.Identifier identifier = message.getIdentifier();
+                    Node node = identifier.getNode();
+                    if (node != null) {
+                        String nodeId = node.getId();
+                        if (!StringUtil.isEmpty(nodeId)) {
+                            serviceName = nodeId;
+                            String cluster = node.getCluster();
+                            if (!StringUtil.isEmpty(cluster)) {
+                                serviceName = nodeId + "." + cluster;
+                            }
+                        }
+                    }
+                }
+
+                if (serviceName != null) {
+                    for (Metrics.MetricFamily metricFamily : message.getEnvoyMetricsList()) {
+
+                    }
+                }
+            }
+
+            @Override public void onError(Throwable throwable) {
+
+            }
+
+            @Override public void onCompleted() {
+
+            }
+        };
+    }
+}
diff --git a/oap-server/server-receiver-plugin/envoy-metrics-receiver-plugin/src/main/resources/META-INF/services/org.apache.skywalking.oap.server.library.module.ModuleDefine b/oap-server/server-receiver-plugin/envoy-metrics-receiver-plugin/src/main/resources/META-INF/services/org.apache.skywalking.oap.server.library.module.ModuleDefine
new file mode 100644
index 0000000..47b2778
--- /dev/null
+++ b/oap-server/server-receiver-plugin/envoy-metrics-receiver-plugin/src/main/resources/META-INF/services/org.apache.skywalking.oap.server.library.module.ModuleDefine
@@ -0,0 +1,20 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+#
+
+
+org.apache.skywalking.oap.server.receiver.envoy.EnvoyMetricReceiverModule
\ No newline at end of file
diff --git a/oap-server/server-receiver-plugin/envoy-metrics-receiver-plugin/src/main/resources/META-INF/services/org.apache.skywalking.oap.server.library.module.ModuleProvider b/oap-server/server-receiver-plugin/envoy-metrics-receiver-plugin/src/main/resources/META-INF/services/org.apache.skywalking.oap.server.library.module.ModuleProvider
new file mode 100644
index 0000000..699cff6
--- /dev/null
+++ b/oap-server/server-receiver-plugin/envoy-metrics-receiver-plugin/src/main/resources/META-INF/services/org.apache.skywalking.oap.server.library.module.ModuleProvider
@@ -0,0 +1,19 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+#
+
+org.apache.skywalking.oap.server.receiver.envoy.EnvoyMetricReceiverProvider
\ No newline at end of file