You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@skywalking.apache.org by wu...@apache.org on 2018/11/17 09:21:53 UTC

[incubator-skywalking] branch zipkin-receiver updated: Fix document and fix for Zipkin receiver.

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

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


The following commit(s) were added to refs/heads/zipkin-receiver by this push:
     new f953d2f  Fix document and fix for Zipkin receiver.
f953d2f is described below

commit f953d2f514b6730c093c2a2d118045d441135840
Author: Wu Sheng <wu...@foxmail.com>
AuthorDate: Sat Nov 17 17:21:43 2018 +0800

    Fix document and fix for Zipkin receiver.
---
 docs/en/setup/backend/backend-receivers.md         |  15 +-
 .../trace/provider/TraceModuleProvider.java        |  14 +-
 .../listener/endpoint/MultiScopesSpanListener.java |   7 +-
 .../server/receiver/zipkin/CoreRegisterLinker.java |   9 -
 .../receiver/zipkin/ZipkinReceiverProvider.java    |  12 +-
 .../receiver/zipkin/transform/SegmentBuilder.java  |   7 +-
 ...walking.oap.server.library.module.ModuleDefine} |  18 --
 ...lking.oap.server.library.module.ModuleProvider} |  18 --
 .../transform/SpringSleuthSegmentBuilderTest.java  | 202 +++++++++++++++++++++
 oap-server/server-starter/pom.xml                  |   5 +
 .../src/main/assembly/application.yml              |   5 +
 .../src/main/resources/application.yml             |   5 +
 12 files changed, 255 insertions(+), 62 deletions(-)

diff --git a/docs/en/setup/backend/backend-receivers.md b/docs/en/setup/backend/backend-receivers.md
index 768020f..b808acb 100644
--- a/docs/en/setup/backend/backend-receivers.md
+++ b/docs/en/setup/backend/backend-receivers.md
@@ -10,23 +10,32 @@ We have following receivers, and `default` implementors are provided in our Apac
 1. **service-mesh**. gRPC services accept data from inbound mesh probes.
 1. **istio-telemetry**. Istio telemetry is from Istio official bypass adaptor, this receiver match its gRPC services.
 1. **receiver-jvm**. gRPC services accept JVM metric data.
+1. **receiver_zipkin**. HTTP service accepts Span in Zipkin v1 and v2 formats. Notice, this receiver only
+works as expected in backend single node mode. Cluster mode is not supported. Welcome anyone to improve this.
 
 The sample settings of these receivers should be already in default `application.yml`, and also list here
 ```yaml
+receiver-register:
+  default:
 receiver-trace:
   default:
-    bufferPath: ../buffer/  # Path to trace buffer files, suggest to use absolute path
+    bufferPath: ../trace-buffer/  # Path to trace buffer files, suggest to use absolute path
     bufferOffsetMaxFileSize: 100 # Unit is MB
     bufferDataMaxFileSize: 500 # Unit is MB
-    bufferFileCleanWhenRestart: false # Clean buffer file when backend restart.
+    bufferFileCleanWhenRestart: false
 receiver-jvm:
   default:
 service-mesh:
   default:
-    bufferPath: ../mesh-buffer/  # Path to mesh telemetry data buffer files, suggest to use absolute path
+    bufferPath: ../mesh-buffer/  # Path to trace buffer files, suggest to use absolute path
     bufferOffsetMaxFileSize: 100 # Unit is MB
     bufferDataMaxFileSize: 500 # Unit is MB
     bufferFileCleanWhenRestart: false
 istio-telemetry:
   default:
+receiver_zipkin:
+  default:
+    host: 0.0.0.0
+    port: 9411
+    contextPath: /
 ```
\ No newline at end of file
diff --git a/oap-server/server-receiver-plugin/skywalking-trace-receiver-plugin/src/main/java/org/apache/skywalking/oap/server/receiver/trace/provider/TraceModuleProvider.java b/oap-server/server-receiver-plugin/skywalking-trace-receiver-plugin/src/main/java/org/apache/skywalking/oap/server/receiver/trace/provider/TraceModuleProvider.java
index 47bc6ec..17b3795 100644
--- a/oap-server/server-receiver-plugin/skywalking-trace-receiver-plugin/src/main/java/org/apache/skywalking/oap/server/receiver/trace/provider/TraceModuleProvider.java
+++ b/oap-server/server-receiver-plugin/skywalking-trace-receiver-plugin/src/main/java/org/apache/skywalking/oap/server/receiver/trace/provider/TraceModuleProvider.java
@@ -45,6 +45,7 @@ import org.apache.skywalking.oap.server.receiver.trace.provider.parser.standardi
 public class TraceModuleProvider extends ModuleProvider {
 
     private final TraceServiceModuleConfig moduleConfig;
+    private SegmentParse.Producer segmentProducer;
 
     public TraceModuleProvider() {
         this.moduleConfig = new TraceServiceModuleConfig();
@@ -62,20 +63,21 @@ public class TraceModuleProvider extends ModuleProvider {
         return moduleConfig;
     }
 
-    @Override public void prepare() {
-    }
-
-    @Override public void start() throws ModuleStartException, ServiceNotProvidedException {
+    @Override public void prepare() throws ServiceNotProvidedException {
         SegmentParserListenerManager listenerManager = new SegmentParserListenerManager();
         listenerManager.add(new MultiScopesSpanListener.Factory());
         listenerManager.add(new ServiceMappingSpanListener.Factory());
         listenerManager.add(new SegmentSpanListener.Factory());
 
+        segmentProducer = new SegmentParse.Producer(getManager(), listenerManager);
+        this.registerServiceImplementation(ISegmentParserService.class, new SegmentParserServiceImpl(segmentProducer));
+    }
+
+    @Override public void start() throws ModuleStartException {
         GRPCHandlerRegister grpcHandlerRegister = getManager().find(CoreModule.NAME).getService(GRPCHandlerRegister.class);
         JettyHandlerRegister jettyHandlerRegister = getManager().find(CoreModule.NAME).getService(JettyHandlerRegister.class);
         try {
-            SegmentParse.Producer segmentProducer = new SegmentParse.Producer(getManager(), listenerManager);
-            this.registerServiceImplementation(ISegmentParserService.class, new SegmentParserServiceImpl(segmentProducer));
+
             grpcHandlerRegister.addHandler(new TraceSegmentServiceHandler(segmentProducer));
             jettyHandlerRegister.addHandler(new TraceSegmentServletHandler(segmentProducer));
 
diff --git a/oap-server/server-receiver-plugin/skywalking-trace-receiver-plugin/src/main/java/org/apache/skywalking/oap/server/receiver/trace/provider/parser/listener/endpoint/MultiScopesSpanListener.java b/oap-server/server-receiver-plugin/skywalking-trace-receiver-plugin/src/main/java/org/apache/skywalking/oap/server/receiver/trace/provider/parser/listener/endpoint/MultiScopesSpanListener.java
index e5434ae..c0a936e 100644
--- a/oap-server/server-receiver-plugin/skywalking-trace-receiver-plugin/src/main/java/org/apache/skywalking/oap/server/receiver/trace/provider/parser/listener/endpoint/MultiScopesSpanListener.java
+++ b/oap-server/server-receiver-plugin/skywalking-trace-receiver-plugin/src/main/java/org/apache/skywalking/oap/server/receiver/trace/provider/parser/listener/endpoint/MultiScopesSpanListener.java
@@ -35,9 +35,7 @@ import static java.util.Objects.nonNull;
  *
  * v5        |   v6
  *
- * 1. Application == Service
- * 2. Server == Service Instance
- * 3. Service == Endpoint
+ * 1. Application == Service 2. Server == Service Instance 3. Service == Endpoint
  *
  * @author peng-yongsheng, wusheng
  */
@@ -120,6 +118,9 @@ public class MultiScopesSpanListener implements EntrySpanListener, ExitSpanListe
         SourceBuilder sourceBuilder = new SourceBuilder();
 
         int peerId = spanDecorator.getPeerId();
+        if (peerId == 0) {
+            return;
+        }
         int destServiceId = serviceInventoryCache.getServiceId(peerId);
         int mappingServiceId = serviceInventoryCache.get(destServiceId).getMappingServiceId();
         int destInstanceId = instanceInventoryCache.getServiceInstanceId(destServiceId, peerId);
diff --git a/oap-server/server-receiver-plugin/zipkin-receiver-plugin/src/main/java/org/apache/skywalking/oap/server/receiver/zipkin/CoreRegisterLinker.java b/oap-server/server-receiver-plugin/zipkin-receiver-plugin/src/main/java/org/apache/skywalking/oap/server/receiver/zipkin/CoreRegisterLinker.java
index b974fe1..4bbcea3 100644
--- a/oap-server/server-receiver-plugin/zipkin-receiver-plugin/src/main/java/org/apache/skywalking/oap/server/receiver/zipkin/CoreRegisterLinker.java
+++ b/oap-server/server-receiver-plugin/zipkin-receiver-plugin/src/main/java/org/apache/skywalking/oap/server/receiver/zipkin/CoreRegisterLinker.java
@@ -19,7 +19,6 @@
 package org.apache.skywalking.oap.server.receiver.zipkin;
 
 import org.apache.skywalking.oap.server.core.CoreModule;
-import org.apache.skywalking.oap.server.core.register.service.IEndpointInventoryRegister;
 import org.apache.skywalking.oap.server.core.register.service.IServiceInstanceInventoryRegister;
 import org.apache.skywalking.oap.server.core.register.service.IServiceInventoryRegister;
 import org.apache.skywalking.oap.server.library.module.ModuleManager;
@@ -28,7 +27,6 @@ public class CoreRegisterLinker {
     private static volatile ModuleManager MODULE_MANAGER;
     private static volatile IServiceInventoryRegister SERVICE_INVENTORY_REGISTER;
     private static volatile IServiceInstanceInventoryRegister SERVICE_INSTANCE_INVENTORY_REGISTER;
-    private static volatile IEndpointInventoryRegister ENDPOINT_INVENTORY_REGISTER;
 
     public static void setModuleManager(ModuleManager moduleManager) {
         CoreRegisterLinker.MODULE_MANAGER = moduleManager;
@@ -47,11 +45,4 @@ public class CoreRegisterLinker {
         }
         return SERVICE_INSTANCE_INVENTORY_REGISTER;
     }
-
-    public static IEndpointInventoryRegister getEndpointInventoryRegister() {
-        if (ENDPOINT_INVENTORY_REGISTER == null) {
-            ENDPOINT_INVENTORY_REGISTER = MODULE_MANAGER.find(CoreModule.NAME).getService(IEndpointInventoryRegister.class);
-        }
-        return ENDPOINT_INVENTORY_REGISTER;
-    }
 }
diff --git a/oap-server/server-receiver-plugin/zipkin-receiver-plugin/src/main/java/org/apache/skywalking/oap/server/receiver/zipkin/ZipkinReceiverProvider.java b/oap-server/server-receiver-plugin/zipkin-receiver-plugin/src/main/java/org/apache/skywalking/oap/server/receiver/zipkin/ZipkinReceiverProvider.java
index 82d3bb9..c86af50 100644
--- a/oap-server/server-receiver-plugin/zipkin-receiver-plugin/src/main/java/org/apache/skywalking/oap/server/receiver/zipkin/ZipkinReceiverProvider.java
+++ b/oap-server/server-receiver-plugin/zipkin-receiver-plugin/src/main/java/org/apache/skywalking/oap/server/receiver/zipkin/ZipkinReceiverProvider.java
@@ -23,6 +23,7 @@ import org.apache.skywalking.oap.server.library.module.ModuleDefine;
 import org.apache.skywalking.oap.server.library.module.ModuleProvider;
 import org.apache.skywalking.oap.server.library.module.ModuleStartException;
 import org.apache.skywalking.oap.server.library.module.ServiceNotProvidedException;
+import org.apache.skywalking.oap.server.library.server.ServerException;
 import org.apache.skywalking.oap.server.library.server.jetty.JettyServer;
 import org.apache.skywalking.oap.server.receiver.trace.module.TraceModule;
 import org.apache.skywalking.oap.server.receiver.trace.provider.parser.ISegmentParserService;
@@ -36,6 +37,7 @@ import org.apache.skywalking.oap.server.receiver.zipkin.transform.Zipkin2SkyWalk
 public class ZipkinReceiverProvider extends ModuleProvider {
     public static final String NAME = "default";
     private ZipkinReceiverConfig config;
+    private JettyServer jettyServer;
 
     public ZipkinReceiverProvider() {
         config = new ZipkinReceiverConfig();
@@ -60,7 +62,7 @@ public class ZipkinReceiverProvider extends ModuleProvider {
     @Override public void start() throws ServiceNotProvidedException, ModuleStartException {
         CoreRegisterLinker.setModuleManager(getManager());
 
-        JettyServer jettyServer = new JettyServer(config.getHost(), config.getPort(), config.getContextPath());
+        jettyServer = new JettyServer(config.getHost(), config.getPort(), config.getContextPath());
         jettyServer.initialize();
 
         jettyServer.addHandler(new SpanV1JettyHandler(config));
@@ -71,8 +73,12 @@ public class ZipkinReceiverProvider extends ModuleProvider {
         Zipkin2SkyWalkingTransfer.INSTANCE.addListener(bridge);
     }
 
-    @Override public void notifyAfterCompleted() throws ServiceNotProvidedException {
-
+    @Override public void notifyAfterCompleted() throws ModuleStartException {
+        try {
+            jettyServer.start();
+        } catch (ServerException e) {
+            throw new ModuleStartException(e.getMessage(), e);
+        }
     }
 
     @Override public String[] requiredModules() {
diff --git a/oap-server/server-receiver-plugin/zipkin-receiver-plugin/src/main/java/org/apache/skywalking/oap/server/receiver/zipkin/transform/SegmentBuilder.java b/oap-server/server-receiver-plugin/zipkin-receiver-plugin/src/main/java/org/apache/skywalking/oap/server/receiver/zipkin/transform/SegmentBuilder.java
index 31e73d4..973d832 100644
--- a/oap-server/server-receiver-plugin/zipkin-receiver-plugin/src/main/java/org/apache/skywalking/oap/server/receiver/zipkin/transform/SegmentBuilder.java
+++ b/oap-server/server-receiver-plugin/zipkin-receiver-plugin/src/main/java/org/apache/skywalking/oap/server/receiver/zipkin/transform/SegmentBuilder.java
@@ -238,8 +238,11 @@ public class SegmentBuilder {
             // using same span id in client and server for RPC
             // SkyWalking will build both sides of span
             ClientSideSpan clientSideSpan = clientPartSpan.get(span.id());
-            parentSegmentSpan = clientSideSpan.getBuilder();
-            parentSpan = clientSideSpan.getSpan();
+            if (clientSideSpan != null) {
+                // For the root span, there may be no ref, because of no parent.
+                parentSegmentSpan = clientSideSpan.getBuilder();
+                parentSpan = clientSideSpan.getSpan();
+            }
         }
 
         String ip = null;
diff --git a/oap-server/server-receiver-plugin/zipkin-receiver-plugin/src/main/resources/META-INF/services/org.apache.skywalking.apm.collector.core.module.ModuleDefine b/oap-server/server-receiver-plugin/zipkin-receiver-plugin/src/main/resources/META-INF/services/org.apache.skywalking.oap.server.library.module.ModuleDefine
similarity index 52%
rename from oap-server/server-receiver-plugin/zipkin-receiver-plugin/src/main/resources/META-INF/services/org.apache.skywalking.apm.collector.core.module.ModuleDefine
rename to oap-server/server-receiver-plugin/zipkin-receiver-plugin/src/main/resources/META-INF/services/org.apache.skywalking.oap.server.library.module.ModuleDefine
index 0f1527e..2be863b 100644
--- a/oap-server/server-receiver-plugin/zipkin-receiver-plugin/src/main/resources/META-INF/services/org.apache.skywalking.apm.collector.core.module.ModuleDefine
+++ b/oap-server/server-receiver-plugin/zipkin-receiver-plugin/src/main/resources/META-INF/services/org.apache.skywalking.oap.server.library.module.ModuleDefine
@@ -16,22 +16,4 @@
 #
 #
 
-#
-# 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.zipkin.ZipkinReceiverModule
diff --git a/oap-server/server-receiver-plugin/zipkin-receiver-plugin/src/main/resources/META-INF/services/org.apache.skywalking.apm.collector.core.module.ModuleProvider b/oap-server/server-receiver-plugin/zipkin-receiver-plugin/src/main/resources/META-INF/services/org.apache.skywalking.oap.server.library.module.ModuleProvider
similarity index 52%
rename from oap-server/server-receiver-plugin/zipkin-receiver-plugin/src/main/resources/META-INF/services/org.apache.skywalking.apm.collector.core.module.ModuleProvider
rename to oap-server/server-receiver-plugin/zipkin-receiver-plugin/src/main/resources/META-INF/services/org.apache.skywalking.oap.server.library.module.ModuleProvider
index c3b5897..349942c 100644
--- a/oap-server/server-receiver-plugin/zipkin-receiver-plugin/src/main/resources/META-INF/services/org.apache.skywalking.apm.collector.core.module.ModuleProvider
+++ b/oap-server/server-receiver-plugin/zipkin-receiver-plugin/src/main/resources/META-INF/services/org.apache.skywalking.oap.server.library.module.ModuleProvider
@@ -16,22 +16,4 @@
 #
 #
 
-#
-# 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.zipkin.ZipkinReceiverProvider
diff --git a/oap-server/server-receiver-plugin/zipkin-receiver-plugin/src/test/java/org/apache/skywalking/oap/server/receiver/zipkin/transform/SpringSleuthSegmentBuilderTest.java b/oap-server/server-receiver-plugin/zipkin-receiver-plugin/src/test/java/org/apache/skywalking/oap/server/receiver/zipkin/transform/SpringSleuthSegmentBuilderTest.java
new file mode 100644
index 0000000..59f1cb2
--- /dev/null
+++ b/oap-server/server-receiver-plugin/zipkin-receiver-plugin/src/test/java/org/apache/skywalking/oap/server/receiver/zipkin/transform/SpringSleuthSegmentBuilderTest.java
@@ -0,0 +1,202 @@
+/*
+ * 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.zipkin.transform;
+
+import org.apache.skywalking.apm.network.language.agent.SpanObject;
+import org.apache.skywalking.apm.network.language.agent.SpanType;
+import org.apache.skywalking.apm.network.language.agent.TraceSegmentObject;
+import org.apache.skywalking.apm.network.language.agent.TraceSegmentReference;
+import org.apache.skywalking.oap.server.core.register.ServiceInstanceInventory;
+import org.apache.skywalking.oap.server.core.register.service.IServiceInstanceInventoryRegister;
+import org.apache.skywalking.oap.server.core.register.service.IServiceInventoryRegister;
+import org.apache.skywalking.oap.server.receiver.zipkin.CoreRegisterLinker;
+import org.apache.skywalking.oap.server.receiver.zipkin.data.SkyWalkingTrace;
+import org.apache.skywalking.oap.server.receiver.zipkin.data.ZipkinTrace;
+import org.junit.Assert;
+import org.junit.Test;
+import org.powermock.reflect.Whitebox;
+import zipkin2.Span;
+import zipkin2.codec.SpanBytesDecoder;
+
+import java.io.UnsupportedEncodingException;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * @author wusheng
+ */
+public class SpringSleuthSegmentBuilderTest implements SegmentListener {
+    private Map<String, Integer> applicationInstRegister = new HashMap<>();
+    private Map<String, Integer> applicationRegister = new HashMap<>();
+    private int appIdSeg = 1;
+    private int appInstIdSeq = 1;
+
+    @Test
+    public void testTransform() throws Exception {
+
+        IServiceInventoryRegister applicationIDService = new IServiceInventoryRegister() {
+            @Override public int getOrCreate(String serviceName) {
+                String key = "AppCode:" + serviceName;
+                if (applicationRegister.containsKey(key)) {
+                    return applicationRegister.get(key);
+                } else {
+                    int id = appIdSeg++;
+                    applicationRegister.put(key, id);
+                    return id;
+                }
+            }
+
+            @Override public int getOrCreate(int addressId, String serviceName) {
+                String key = "Address:" + serviceName;
+                if (applicationRegister.containsKey(key)) {
+                    return applicationRegister.get(key);
+                } else {
+                    int id = appIdSeg++;
+                    applicationRegister.put(key, id);
+                    return id;
+                }
+            }
+
+            @Override public void heartbeat(int serviceId, long heartBeatTime) {
+
+            }
+
+            @Override public void updateMapping(int serviceId, int mappingServiceId) {
+
+            }
+
+
+        };
+
+        IServiceInstanceInventoryRegister instanceIDService = new IServiceInstanceInventoryRegister() {
+            @Override public int getOrCreate(int serviceId, String serviceInstanceName, String uuid, long registerTime,
+                ServiceInstanceInventory.AgentOsInfo osInfo) {
+                String key = "AppCode:" + serviceId + ",UUID:" + uuid;
+                if (applicationInstRegister.containsKey(key)) {
+                    return applicationInstRegister.get(key);
+                } else {
+                    int id = appInstIdSeq++;
+                    applicationInstRegister.put(key, id);
+                    return id;
+                }
+            }
+
+            @Override public int getOrCreate(int serviceId, int addressId, long registerTime) {
+                String key = "VitualAppCode:" + serviceId + ",address:" + addressId;
+                if (applicationInstRegister.containsKey(key)) {
+                    return applicationInstRegister.get(key);
+                } else {
+                    int id = appInstIdSeq++;
+                    applicationInstRegister.put(key, id);
+                    return id;
+                }
+            }
+
+            @Override public void heartbeat(int serviceInstanceId, long heartBeatTime) {
+
+            }
+        };
+
+        Whitebox.setInternalState(CoreRegisterLinker.class, "SERVICE_INVENTORY_REGISTER", applicationIDService);
+        Whitebox.setInternalState(CoreRegisterLinker.class, "SERVICE_INSTANCE_INVENTORY_REGISTER", instanceIDService);
+
+        Zipkin2SkyWalkingTransfer.INSTANCE.addListener(this);
+
+        List<Span> spanList = buildSpringSleuthExampleTrace();
+        Assert.assertEquals(3, spanList.size());
+
+        ZipkinTrace trace = new ZipkinTrace();
+        spanList.forEach(span -> trace.addSpan(span));
+
+        Zipkin2SkyWalkingTransfer.INSTANCE.transfer(trace);
+    }
+
+    private List<Span> buildSpringSleuthExampleTrace() throws UnsupportedEncodingException {
+        List<Span> spans = new LinkedList<>();
+        String span = "{\"traceId\":\"5b0e64354eea4fa71a8a1b5bdd791b8a\",\"id\":\"1a8a1b5bdd791b8a\",\"kind\":\"SERVER\",\"name\":\"get /\",\"timestamp\":1527669813700123,\"duration\":11295,\"localEndpoint\":{\"serviceName\":\"frontend\",\"ipv4\":\"192.168.72.220\"},\"remoteEndpoint\":{\"ipv6\":\"::1\",\"port\":55146},\"tags\":{\"http.method\":\"GET\",\"http.path\":\"/\",\"mvc.controller.class\":\"Frontend\",\"mvc.controller.method\":\"callBackend\"}}";
+        spans.add(SpanBytesDecoder.JSON_V2.decodeOne(span.getBytes("UTF-8")));
+        span = "{\"traceId\":\"5b0e64354eea4fa71a8a1b5bdd791b8a\",\"parentId\":\"1a8a1b5bdd791b8a\",\"id\":\"d7d5b93dcda767c8\",\"kind\":\"CLIENT\",\"name\":\"get\",\"timestamp\":1527669813702456,\"duration\":6672,\"localEndpoint\":{\"serviceName\":\"frontend\",\"ipv4\":\"192.168.72.220\"},\"tags\":{\"http.method\":\"GET\",\"http.path\":\"/api\"}}";
+        spans.add(SpanBytesDecoder.JSON_V2.decodeOne(span.getBytes("UTF-8")));
+        span = "{\"traceId\":\"5b0e64354eea4fa71a8a1b5bdd791b8a\",\"parentId\":\"1a8a1b5bdd791b8a\",\"id\":\"d7d5b93dcda767c8\",\"kind\":\"SERVER\",\"name\":\"get /api\",\"timestamp\":1527669813705106,\"duration\":4802,\"localEndpoint\":{\"serviceName\":\"backend\",\"ipv4\":\"192.168.72.220\"},\"remoteEndpoint\":{\"ipv4\":\"127.0.0.1\",\"port\":55147},\"tags\":{\"http.method\":\"GET\",\"http.path\":\"/api\",\"mvc.controller.class\":\"Backend\",\"mvc.controller.method\":\"printDate\"},\"s [...]
+        spans.add(SpanBytesDecoder.JSON_V2.decodeOne(span.getBytes("UTF-8")));
+
+        return SpanBytesDecoder.JSON_V2.decodeList(spans.toString().getBytes("UTF-8"));
+    }
+
+    @Override
+    public void notify(SkyWalkingTrace trace) {
+        List<TraceSegmentObject.Builder> segments = trace.getSegmentList();
+        Assert.assertEquals(2, segments.size());
+        TraceSegmentObject.Builder builder = segments.get(0);
+        TraceSegmentObject.Builder builder1 = segments.get(1);
+        TraceSegmentObject.Builder front, end;
+        if (builder.getApplicationId() == applicationRegister.get("AppCode:frontend")) {
+            front = builder;
+            end = builder1;
+            Assert.assertEquals(applicationRegister.get("AppCode:backend").longValue(), builder1.getApplicationId());
+        } else if (builder.getApplicationId() == applicationRegister.get("AppCode:backend")) {
+            end = builder;
+            front = builder1;
+            Assert.assertEquals(applicationRegister.get("AppCode:frontend").longValue(), builder1.getApplicationId());
+        } else {
+            Assert.fail("Can't find frontend and backend applications. ");
+            return;
+        }
+
+        Assert.assertEquals(2, front.getSpansCount());
+        Assert.assertEquals(1, end.getSpansCount());
+
+        front.getSpansList().forEach(spanObject -> {
+            if (spanObject.getSpanId() == 0) {
+                // span id = 1, means incoming http of frontend
+                Assert.assertEquals(SpanType.Entry, spanObject.getSpanType());
+                Assert.assertEquals("get /", spanObject.getOperationName());
+                Assert.assertEquals(0, spanObject.getSpanId());
+                Assert.assertEquals(-1, spanObject.getParentSpanId());
+            } else if (spanObject.getSpanId() == 1) {
+                Assert.assertEquals("192.168.72.220", spanObject.getPeer());
+                Assert.assertEquals(SpanType.Exit, spanObject.getSpanType());
+                Assert.assertEquals(1, spanObject.getSpanId());
+                Assert.assertEquals(0, spanObject.getParentSpanId());
+            } else {
+                Assert.fail("Only two spans expected");
+            }
+            Assert.assertTrue(spanObject.getTagsCount() > 0);
+        });
+
+        SpanObject spanObject = end.getSpans(0);
+
+        Assert.assertEquals(1, spanObject.getRefsCount());
+        TraceSegmentReference spanObjectRef = spanObject.getRefs(0);
+        Assert.assertEquals("get", spanObjectRef.getEntryServiceName());
+        Assert.assertEquals("get", spanObjectRef.getParentServiceName());
+        //Assert.assertEquals("192.168.72.220", spanObjectRef.getNetworkAddress());
+        Assert.assertEquals(1, spanObjectRef.getParentSpanId());
+        Assert.assertEquals(front.getTraceSegmentId(), spanObjectRef.getParentTraceSegmentId());
+
+        Assert.assertTrue(spanObject.getTagsCount() > 0);
+
+        Assert.assertEquals("get /api", spanObject.getOperationName());
+        Assert.assertEquals(0, spanObject.getSpanId());
+        Assert.assertEquals(-1, spanObject.getParentSpanId());
+        Assert.assertEquals(SpanType.Entry, spanObject.getSpanType());
+    }
+}
\ No newline at end of file
diff --git a/oap-server/server-starter/pom.xml b/oap-server/server-starter/pom.xml
index 153a871..f1295a5 100644
--- a/oap-server/server-starter/pom.xml
+++ b/oap-server/server-starter/pom.xml
@@ -78,6 +78,11 @@
             <artifactId>skywalking-trace-receiver-plugin</artifactId>
             <version>${project.version}</version>
         </dependency>
+        <dependency>
+            <groupId>org.apache.skywalking</groupId>
+            <artifactId>zipkin-receiver-plugin</artifactId>
+            <version>${project.version}</version>
+        </dependency>
         <!-- receiver module -->
 
         <!-- storage module -->
diff --git a/oap-server/server-starter/src/main/assembly/application.yml b/oap-server/server-starter/src/main/assembly/application.yml
index c4fc574..e1f1406 100644
--- a/oap-server/server-starter/src/main/assembly/application.yml
+++ b/oap-server/server-starter/src/main/assembly/application.yml
@@ -75,6 +75,11 @@ service-mesh:
     bufferFileCleanWhenRestart: false
 istio-telemetry:
   default:
+#receiver_zipkin:
+#  default:
+#    host: 0.0.0.0
+#    port: 9411
+#    contextPath: /
 query:
   graphql:
     path: /graphql
diff --git a/oap-server/server-starter/src/main/resources/application.yml b/oap-server/server-starter/src/main/resources/application.yml
index 2236cfc..91ee05d 100644
--- a/oap-server/server-starter/src/main/resources/application.yml
+++ b/oap-server/server-starter/src/main/resources/application.yml
@@ -75,6 +75,11 @@ service-mesh:
     bufferFileCleanWhenRestart: false
 istio-telemetry:
   default:
+receiver_zipkin:
+  default:
+    host: 0.0.0.0
+    port: 9411
+    contextPath: /
 query:
   graphql:
     path: /graphql