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/21 20:14:36 UTC

[incubator-skywalking] 01/01: Support grpc exporter.

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

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

commit 0006465087ec9d52efa8a729e2a7b16c8da0adbb
Author: Wu Sheng <wu...@foxmail.com>
AuthorDate: Thu Mar 21 13:13:51 2019 -0700

    Support grpc exporter.
---
 LICENSE                                            |   1 -
 docs/en/guides/How-to-build.md                     |   1 +
 oap-server/exporter/pom.xml                        |  30 ++++
 .../server/exporter/provider/MetricFormatter.java  |  47 ++++++
 .../exporter/provider/grpc/GRPCExporter.java       | 175 +++++++++++++++++++++
 .../provider/grpc/GRPCExporterProvider.java        |  64 ++++++++
 .../provider/grpc/GRPCExporterSetting.java         |  34 ++++
 .../exporter/src/main/proto/metric-exporter.proto  |  46 ++++++
 ...alking.oap.server.library.module.ModuleProvider |   2 +
 .../provider/grpc/ExporterMockReceiver.java        |  55 +++++++
 .../envoy-metrics-receiver-plugin/pom.xml          |   8 -
 oap-server/server-starter/pom.xml                  |  12 ++
 .../src/main/resources/application.yml             |   4 +
 13 files changed, 470 insertions(+), 9 deletions(-)

diff --git a/LICENSE b/LICENSE
index 945dc24..46b7d40 100644
--- a/LICENSE
+++ b/LICENSE
@@ -220,7 +220,6 @@ The text of each license is the standard Apache 2.0 license.
    proto files from envoyproxy/data-plane-api: https://github.com/envoyproxy/data-plane-api  Apache 2.0
    proto files from prometheus/client_model: https://github.com/prometheus/client_model Apache 2.0
    proto files from lyft/protoc-gen-validate: https://github.com/lyft/protoc-gen-validate Apache 2.0
-   proto files from
 
 
 ========================================================================
diff --git a/docs/en/guides/How-to-build.md b/docs/en/guides/How-to-build.md
index aa2aca0..55cf2c2 100644
--- a/docs/en/guides/How-to-build.md
+++ b/docs/en/guides/How-to-build.md
@@ -31,6 +31,7 @@ For each official Apache release, there is a complete and independent source cod
     * `grpc-java` and `java` folders in **oap-server/server-core/target/generated-sources/protobuf**
     * `grpc-java` and `java` folders in **oap-server/server-receiver-plugin/skywalking-istio-telemetry-receiver-plugin/target/generated-sources/protobuf**
     * `grpc-java` and `java` folders in **oap-server/server-receiver-plugin/envoy-metrics-receiver-plugin/target/generated-sources/protobuf**
+    * `grpc-java` and `java` folders in **oap-server/exporter/target/generated-sources/protobuf**
     * `antlr4` folder in **oap-server/generate-tool-grammar/target/generated-sources**
     * `oal` folder in **oap-server/generated-analysis/target/generated-sources**
     
diff --git a/oap-server/exporter/pom.xml b/oap-server/exporter/pom.xml
index 492a8e2..73a0caf 100644
--- a/oap-server/exporter/pom.xml
+++ b/oap-server/exporter/pom.xml
@@ -36,4 +36,34 @@
             <version>${project.version}</version>
         </dependency>
     </dependencies>
+
+    <build>
+        <plugins>
+            <plugin>
+                <groupId>org.xolstice.maven.plugins</groupId>
+                <artifactId>protobuf-maven-plugin</artifactId>
+                <version>0.5.0</version>
+                <configuration>
+                    <!--
+                      The version of protoc must match protobuf-java. If you don't depend on
+                      protobuf-java directly, you will be transitively depending on the
+                      protobuf-java version that grpc depends on.
+                    -->
+                    <protocArtifact>com.google.protobuf:protoc:3.3.0:exe:${os.detected.classifier}
+                    </protocArtifact>
+                    <pluginId>grpc-java</pluginId>
+                    <pluginArtifact>io.grpc:protoc-gen-grpc-java:1.8.0:exe:${os.detected.classifier}
+                    </pluginArtifact>
+                </configuration>
+                <executions>
+                    <execution>
+                        <goals>
+                            <goal>compile</goal>
+                            <goal>compile-custom</goal>
+                        </goals>
+                    </execution>
+                </executions>
+            </plugin>
+        </plugins>
+    </build>
 </project>
\ No newline at end of file
diff --git a/oap-server/exporter/src/main/java/org/apache/skywalking/oap/server/exporter/provider/MetricFormatter.java b/oap-server/exporter/src/main/java/org/apache/skywalking/oap/server/exporter/provider/MetricFormatter.java
new file mode 100644
index 0000000..eafb9f9
--- /dev/null
+++ b/oap-server/exporter/src/main/java/org/apache/skywalking/oap/server/exporter/provider/MetricFormatter.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.server.exporter.provider;
+
+import lombok.*;
+import org.apache.skywalking.oap.server.core.analysis.indicator.*;
+import org.apache.skywalking.oap.server.core.cache.*;
+import org.apache.skywalking.oap.server.core.source.DefaultScopeDefine;
+
+/**
+ * @author wusheng
+ */
+@Setter
+public class MetricFormatter {
+    private ServiceInventoryCache serviceInventoryCache;
+    private ServiceInstanceInventoryCache serviceInstanceInventoryCache;
+    private EndpointInventoryCache endpointInventoryCache;
+
+    protected String getEntityName(IndicatorMetaInfo meta) {
+        int scope = meta.getScope();
+        if (DefaultScopeDefine.inServiceCatalog(scope)) {
+            return serviceInventoryCache.get(scope).getName();
+        } else if (DefaultScopeDefine.inServiceInstanceCatalog(scope)) {
+            return serviceInstanceInventoryCache.get(scope).getName();
+        } else if (DefaultScopeDefine.inEndpointCatalog(scope)) {
+            return endpointInventoryCache.get(scope).getName();
+        } else {
+            return null;
+        }
+    }
+}
diff --git a/oap-server/exporter/src/main/java/org/apache/skywalking/oap/server/exporter/provider/grpc/GRPCExporter.java b/oap-server/exporter/src/main/java/org/apache/skywalking/oap/server/exporter/provider/grpc/GRPCExporter.java
new file mode 100644
index 0000000..3869ca9
--- /dev/null
+++ b/oap-server/exporter/src/main/java/org/apache/skywalking/oap/server/exporter/provider/grpc/GRPCExporter.java
@@ -0,0 +1,175 @@
+/*
+ * 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.exporter.provider.grpc;
+
+import io.grpc.ManagedChannel;
+import io.grpc.stub.StreamObserver;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicInteger;
+import lombok.*;
+import org.apache.skywalking.apm.commons.datacarrier.DataCarrier;
+import org.apache.skywalking.apm.commons.datacarrier.consumer.IConsumer;
+import org.apache.skywalking.oap.server.core.analysis.indicator.*;
+import org.apache.skywalking.oap.server.core.exporter.MetricValuesExportService;
+import org.apache.skywalking.oap.server.exporter.grpc.*;
+import org.apache.skywalking.oap.server.exporter.provider.MetricFormatter;
+import org.apache.skywalking.oap.server.library.client.grpc.GRPCClient;
+import org.slf4j.*;
+
+/**
+ * @author wusheng
+ */
+public class GRPCExporter extends MetricFormatter implements MetricValuesExportService, IConsumer<GRPCExporter.ExportData> {
+    private static final Logger logger = LoggerFactory.getLogger(GRPCExporter.class);
+
+    private GRPCExporterSetting setting;
+    private MetricExportServiceGrpc.MetricExportServiceStub exportServiceFutureStub;
+    private final DataCarrier exportBuffer;
+
+    public GRPCExporter(GRPCExporterSetting setting) {
+        this.setting = setting;
+        GRPCClient client = new GRPCClient(setting.getTargetHost(), setting.getTargetPort());
+        client.connect();
+        ManagedChannel channel = client.getChannel();
+        exportServiceFutureStub = MetricExportServiceGrpc.newStub(channel);
+        exportBuffer = new DataCarrier<ExportData>(setting.getBufferChannelNum(), setting.getBufferChannelSize());
+        exportBuffer.consume(this, 1, 200);
+    }
+
+    @Override public void export(IndicatorMetaInfo meta, Indicator indicator) {
+        exportBuffer.produce(new ExportData(meta, indicator));
+    }
+
+    @Override public void init() {
+
+    }
+
+    @Override public void consume(List<ExportData> data) {
+        if (data.size() == 0) {
+            return;
+        }
+
+        ExportStatus status = new ExportStatus();
+        StreamObserver<ExportMetricValue> streamObserver = exportServiceFutureStub.export(
+            new StreamObserver<ExportResponse>() {
+                @Override public void onNext(ExportResponse response) {
+
+                }
+
+                @Override public void onError(Throwable throwable) {
+                    status.done();
+                }
+
+                @Override public void onCompleted() {
+                    status.done();
+                }
+            }
+        );
+        AtomicInteger exportNum = new AtomicInteger();
+        data.forEach(row -> {
+            ExportMetricValue.Builder builder = ExportMetricValue.newBuilder();
+
+            Indicator indicator = row.getIndicator();
+            if (indicator instanceof LongValueHolder) {
+                long value = ((LongValueHolder)indicator).getValue();
+                builder.setLongValue(value);
+                builder.setType(ValueType.LONG);
+            } else if (indicator instanceof IntValueHolder) {
+                long value = ((IntValueHolder)indicator).getValue();
+                builder.setLongValue(value);
+                builder.setType(ValueType.LONG);
+            } else if (indicator instanceof DoubleValueHolder) {
+                double value = ((DoubleValueHolder)indicator).getValue();
+                builder.setDoubleValue(value);
+                builder.setType(ValueType.DOUBLE);
+            } else {
+                return;
+            }
+
+            IndicatorMetaInfo meta = row.getMeta();
+            builder.setMetricName(meta.getIndicatorName());
+            String entityName = getEntityName(meta);
+            if (entityName == null) {
+                return;
+            }
+            builder.setEntityName(entityName);
+            builder.setEntityId(meta.getId());
+
+            builder.setTimeBucket(indicator.getTimeBucket());
+
+            streamObserver.onNext(builder.build());
+            exportNum.getAndIncrement();
+        });
+
+        streamObserver.onCompleted();
+
+        long sleepTime = 0;
+        long cycle = 100L;
+        /**
+         * For memory safe of oap, we must wait for the peer confirmation.
+         */
+        while (!status.isDone()) {
+            try {
+                sleepTime += cycle;
+                Thread.sleep(cycle);
+            } catch (InterruptedException e) {
+            }
+
+            if (sleepTime > 2000L) {
+                logger.warn("Export {} metric(s) to {}:{}, wait {} milliseconds.",
+                    exportNum.get(), setting.getTargetHost(), setting.getTargetPort(), sleepTime);
+                cycle = 2000L;
+            }
+        }
+
+        logger.debug("Exported {} metric(s) to {}:{} in {} milliseconds.",
+            exportNum.get(), setting.getTargetHost(), setting.getTargetPort(), sleepTime);
+    }
+
+    @Override public void onError(List<ExportData> data, Throwable t) {
+        logger.error(t.getMessage(), t);
+    }
+
+    @Override public void onExit() {
+
+    }
+
+    @Getter(AccessLevel.PRIVATE)
+    public class ExportData {
+        private IndicatorMetaInfo meta;
+        private Indicator indicator;
+
+        public ExportData(IndicatorMetaInfo meta, Indicator indicator) {
+            this.meta = meta;
+            this.indicator = indicator;
+        }
+    }
+
+    private class ExportStatus {
+        private boolean done = false;
+
+        private void done() {
+            done = true;
+        }
+
+        public boolean isDone() {
+            return done;
+        }
+    }
+}
diff --git a/oap-server/exporter/src/main/java/org/apache/skywalking/oap/server/exporter/provider/grpc/GRPCExporterProvider.java b/oap-server/exporter/src/main/java/org/apache/skywalking/oap/server/exporter/provider/grpc/GRPCExporterProvider.java
new file mode 100644
index 0000000..ac11bb3
--- /dev/null
+++ b/oap-server/exporter/src/main/java/org/apache/skywalking/oap/server/exporter/provider/grpc/GRPCExporterProvider.java
@@ -0,0 +1,64 @@
+/*
+ * 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.exporter.provider.grpc;
+
+import org.apache.skywalking.oap.server.core.CoreModule;
+import org.apache.skywalking.oap.server.core.cache.*;
+import org.apache.skywalking.oap.server.core.exporter.*;
+import org.apache.skywalking.oap.server.library.module.*;
+
+/**
+ * @author wusheng
+ */
+public class GRPCExporterProvider extends ModuleProvider {
+    private GRPCExporterSetting setting;
+    private GRPCExporter exporter;
+
+    @Override public String name() {
+        return "grpc";
+    }
+
+    @Override public Class<? extends ModuleDefine> module() {
+        return ExporterModule.class;
+    }
+
+    @Override public ModuleConfig createConfigBeanIfAbsent() {
+        setting = new GRPCExporterSetting();
+        return setting;
+    }
+
+    @Override public void prepare() throws ServiceNotProvidedException, ModuleStartException {
+        exporter = new GRPCExporter(setting);
+        this.registerServiceImplementation(MetricValuesExportService.class, exporter);
+    }
+
+    @Override public void start() throws ServiceNotProvidedException, ModuleStartException {
+
+    }
+
+    @Override public void notifyAfterCompleted() throws ServiceNotProvidedException, ModuleStartException {
+        exporter.setServiceInventoryCache(getManager().find(CoreModule.NAME).provider().getService(ServiceInventoryCache.class));
+        exporter.setServiceInstanceInventoryCache(getManager().find(CoreModule.NAME).provider().getService(ServiceInstanceInventoryCache.class));
+        exporter.setEndpointInventoryCache(getManager().find(CoreModule.NAME).provider().getService(EndpointInventoryCache.class));
+    }
+
+    @Override public String[] requiredModules() {
+        return new String[] {CoreModule.NAME};
+    }
+}
diff --git a/oap-server/exporter/src/main/java/org/apache/skywalking/oap/server/exporter/provider/grpc/GRPCExporterSetting.java b/oap-server/exporter/src/main/java/org/apache/skywalking/oap/server/exporter/provider/grpc/GRPCExporterSetting.java
new file mode 100644
index 0000000..41b5846
--- /dev/null
+++ b/oap-server/exporter/src/main/java/org/apache/skywalking/oap/server/exporter/provider/grpc/GRPCExporterSetting.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.server.exporter.provider.grpc;
+
+import lombok.*;
+import org.apache.skywalking.oap.server.library.module.ModuleConfig;
+
+/**
+ * @author wusheng
+ */
+@Setter
+@Getter
+public class GRPCExporterSetting extends ModuleConfig {
+    private String targetHost;
+    private int targetPort;
+    private int bufferChannelSize = 20000;
+    private int bufferChannelNum = 2;
+}
diff --git a/oap-server/exporter/src/main/proto/metric-exporter.proto b/oap-server/exporter/src/main/proto/metric-exporter.proto
new file mode 100644
index 0000000..4e1201c
--- /dev/null
+++ b/oap-server/exporter/src/main/proto/metric-exporter.proto
@@ -0,0 +1,46 @@
+/*
+ * 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.
+ *
+ */
+
+syntax = "proto3";
+
+option java_multiple_files = true;
+option java_package = "org.apache.skywalking.oap.server.exporter.grpc";
+
+
+service MetricExportService {
+    rpc export (stream ExportMetricValue) returns (ExportResponse) {
+    }
+}
+
+message ExportMetricValue {
+    string metricName = 1;
+    string entityName = 2;
+    string entityId = 3;
+    ValueType type = 5;
+    int64 timeBucket = 6;
+    int64 longValue = 7;
+    double doubleValue = 8;
+}
+
+enum ValueType {
+    LONG = 0;
+    DOUBLE = 1;
+}
+
+message ExportResponse {
+}
diff --git a/oap-server/exporter/src/main/resources/META-INF/services/org.apache.skywalking.oap.server.library.module.ModuleProvider b/oap-server/exporter/src/main/resources/META-INF/services/org.apache.skywalking.oap.server.library.module.ModuleProvider
index eafb553..0e65d5b 100644
--- a/oap-server/exporter/src/main/resources/META-INF/services/org.apache.skywalking.oap.server.library.module.ModuleProvider
+++ b/oap-server/exporter/src/main/resources/META-INF/services/org.apache.skywalking.oap.server.library.module.ModuleProvider
@@ -15,3 +15,5 @@
 # limitations under the License.
 #
 #
+
+org.apache.skywalking.oap.server.exporter.provider.grpc.GRPCExporterProvider
\ No newline at end of file
diff --git a/oap-server/exporter/src/test/java/org/apache/skywalking/oap/server/exporter/provider/grpc/ExporterMockReceiver.java b/oap-server/exporter/src/test/java/org/apache/skywalking/oap/server/exporter/provider/grpc/ExporterMockReceiver.java
new file mode 100644
index 0000000..d9c5ab9
--- /dev/null
+++ b/oap-server/exporter/src/test/java/org/apache/skywalking/oap/server/exporter/provider/grpc/ExporterMockReceiver.java
@@ -0,0 +1,55 @@
+/*
+ * 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.exporter.provider.grpc;
+
+import io.grpc.stub.StreamObserver;
+import org.apache.skywalking.oap.server.exporter.grpc.*;
+import org.apache.skywalking.oap.server.library.server.ServerException;
+import org.apache.skywalking.oap.server.library.server.grpc.*;
+
+public class ExporterMockReceiver {
+    public static void main(String[] args) throws ServerException, InterruptedException {
+        GRPCServer server = new GRPCServer("127.0.0.1", 9870);
+        server.initialize();
+        server.addHandler(new MockHandler());
+        server.start();
+
+        while (true){
+            Thread.sleep(20000L);
+        }
+    }
+
+    public static class MockHandler extends MetricExportServiceGrpc.MetricExportServiceImplBase implements GRPCHandler {
+        @Override public StreamObserver<ExportMetricValue> export(StreamObserver<ExportResponse> responseObserver) {
+            return new StreamObserver<ExportMetricValue>() {
+                @Override public void onNext(ExportMetricValue value) {
+                    System.out.println(value);
+                }
+
+                @Override public void onError(Throwable throwable) {
+                    responseObserver.onError(throwable);
+                }
+
+                @Override public void onCompleted() {
+                    responseObserver.onCompleted();
+                }
+            };
+        }
+    }
+}
diff --git a/oap-server/server-receiver-plugin/envoy-metrics-receiver-plugin/pom.xml b/oap-server/server-receiver-plugin/envoy-metrics-receiver-plugin/pom.xml
index 2baf0cf..91653a8 100644
--- a/oap-server/server-receiver-plugin/envoy-metrics-receiver-plugin/pom.xml
+++ b/oap-server/server-receiver-plugin/envoy-metrics-receiver-plugin/pom.xml
@@ -39,14 +39,6 @@
     <build>
         <plugins>
             <plugin>
-                <groupId>org.apache.maven.plugins</groupId>
-                <artifactId>maven-resources-plugin</artifactId>
-                <version>2.4.3</version>
-                <configuration>
-                    <encoding>${project.build.sourceEncoding}</encoding>
-                </configuration>
-            </plugin>
-            <plugin>
                 <groupId>org.xolstice.maven.plugins</groupId>
                 <artifactId>protobuf-maven-plugin</artifactId>
                 <version>0.5.0</version>
diff --git a/oap-server/server-starter/pom.xml b/oap-server/server-starter/pom.xml
index 36d8192..5e1d5a6 100644
--- a/oap-server/server-starter/pom.xml
+++ b/oap-server/server-starter/pom.xml
@@ -108,6 +108,11 @@
             <artifactId>zipkin-receiver-plugin</artifactId>
             <version>${project.version}</version>
         </dependency>
+        <dependency>
+            <groupId>org.apache.skywalking</groupId>
+            <artifactId>skywalking-clr-receiver-plugin</artifactId>
+            <version>${project.version}</version>
+        </dependency>
         <!-- receiver module -->
 
         <!-- storage module -->
@@ -144,6 +149,13 @@
             <artifactId>telemetry-prometheus</artifactId>
             <version>${project.version}</version>
         </dependency>
+
+        <!-- exporter -->
+        <dependency>
+            <groupId>org.apache.skywalking</groupId>
+            <artifactId>exporter</artifactId>
+            <version>${project.version}</version>
+        </dependency>
     </dependencies>
     <build>
         <finalName>skywalking-oap</finalName>
diff --git a/oap-server/server-starter/src/main/resources/application.yml b/oap-server/server-starter/src/main/resources/application.yml
index 58f59da..fd21669 100644
--- a/oap-server/server-starter/src/main/resources/application.yml
+++ b/oap-server/server-starter/src/main/resources/application.yml
@@ -110,3 +110,7 @@ alarm:
   default:
 telemetry:
   prometheus:
+exporter:
+  grpc:
+    targetHost: 127.0.0.1
+    targetPort: 9870
\ No newline at end of file