You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@skywalking.apache.org by GitBox <gi...@apache.org> on 2020/01/05 20:17:43 UTC

[GitHub] [skywalking] devkanro opened a new pull request #4177: Enhance gRPC plugin

devkanro opened a new pull request #4177: Enhance gRPC plugin
URL: https://github.com/apache/skywalking/pull/4177
 
 
   # Skywalking with enhanced gRPC
   This PR enhances the original gRPC plugin of Skywalking.  
   I created PR early for the request for comments. @GuoDuanLZ and me will accept opinion and improve this PR.
   
   ## Checklist
   This PR still in WIP state, you can merge this PR after all check is done.
   - [x] Implementation
   - [ ] Test
   - [ ] Documents
   - [ ] CI build pass
   - [ ] Review
   
   ## Description
   The original gRPC plugin just provided very basically tracing function, the exception of business code will not be traced.
   
   We refactored this plugin for those functions:
   01. Server and Client tracing has the same operations.
   02. Provide the internal or external gRPC server tracing mode.
   03. Streamline span timeline.
   04. The error caused by business code will make the "/Response/onClose" fail.
   05. Reduce interface point and optimize the code.
   
   ## Operations
   WIP
   
   ## Configure plugin
   WIP
   
   ## Screenshots
   WIP

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [skywalking] wu-sheng commented on a change in pull request #4177: Enhance gRPC plugin

Posted by GitBox <gi...@apache.org>.
wu-sheng commented on a change in pull request #4177: Enhance gRPC plugin
URL: https://github.com/apache/skywalking/pull/4177#discussion_r366913385
 
 

 ##########
 File path: apm-sniffer/apm-sdk-plugin/grpc-1.x-plugin/src/main/java/org/apache/skywalking/apm/plugin/grpc/v1/client/BlockingCallInterceptor.java
 ##########
 @@ -16,43 +16,48 @@
  *
  */
 
-package org.apache.skywalking.apm.plugin.grpc.v1;
+package org.apache.skywalking.apm.plugin.grpc.v1.client;
 
 import io.grpc.Channel;
 import io.grpc.MethodDescriptor;
-import java.lang.reflect.Method;
 import org.apache.skywalking.apm.agent.core.context.ContextManager;
 import org.apache.skywalking.apm.agent.core.context.trace.AbstractSpan;
 import org.apache.skywalking.apm.agent.core.context.trace.SpanLayer;
 import org.apache.skywalking.apm.agent.core.plugin.interceptor.enhance.MethodInterceptResult;
 import org.apache.skywalking.apm.agent.core.plugin.interceptor.enhance.StaticMethodsAroundInterceptor;
 import org.apache.skywalking.apm.network.trace.component.ComponentsDefine;
 
+import java.lang.reflect.Method;
+
+import static org.apache.skywalking.apm.plugin.grpc.v1.Constants.BLOCKING_CALL_EXIT_SPAN;
 import static org.apache.skywalking.apm.plugin.grpc.v1.OperationNameFormatUtil.formatOperationName;
 
 /**
- * @author zhang xin
+ * @author zhang xin, kanro
  */
 public class BlockingCallInterceptor implements StaticMethodsAroundInterceptor {
 
-    @Override public void beforeMethod(Class clazz, Method method, Object[] allArguments, Class<?>[] parameterTypes,
-        MethodInterceptResult result) {
-        Channel channel = (Channel)allArguments[0];
-        MethodDescriptor methodDescriptor = (MethodDescriptor)allArguments[1];
+    @Override
+    public void beforeMethod(Class clazz, Method method, Object[] allArguments, Class<?>[] parameterTypes,
+                             MethodInterceptResult result) {
+        Channel channel = (Channel) allArguments[0];
+        MethodDescriptor<?, ?> methodDescriptor = (MethodDescriptor<?, ?>) allArguments[1];
         final AbstractSpan span = ContextManager.createExitSpan(formatOperationName(methodDescriptor), channel.authority());
         span.setComponent(ComponentsDefine.GRPC);
-        SpanLayer.asRPCFramework(span);
+        span.setLayer(SpanLayer.RPC_FRAMEWORK);
+        ContextManager.getRuntimeContext().put(BLOCKING_CALL_EXIT_SPAN, span);
 
 Review comment:
   When will remove this from the RuntimeContext?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [skywalking] wu-sheng merged pull request #4177: Enhance gRPC plugin

Posted by GitBox <gi...@apache.org>.
wu-sheng merged pull request #4177: Enhance gRPC plugin
URL: https://github.com/apache/skywalking/pull/4177
 
 
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [skywalking] wu-sheng commented on a change in pull request #4177: Enhance gRPC plugin

Posted by GitBox <gi...@apache.org>.
wu-sheng commented on a change in pull request #4177: Enhance gRPC plugin
URL: https://github.com/apache/skywalking/pull/4177#discussion_r371104922
 
 

 ##########
 File path: apm-sniffer/apm-sdk-plugin/grpc-1.x-plugin/src/main/java/org/apache/skywalking/apm/plugin/grpc/v1/server/TracingServerCallListener.java
 ##########
 @@ -0,0 +1,102 @@
+/*
+ * 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.apm.plugin.grpc.v1.server;
+
+import io.grpc.ForwardingServerCallListener;
+import io.grpc.MethodDescriptor;
+import io.grpc.ServerCall;
+import org.apache.skywalking.apm.agent.core.context.ContextManager;
+import org.apache.skywalking.apm.agent.core.context.ContextSnapshot;
+import org.apache.skywalking.apm.agent.core.context.trace.AbstractSpan;
+import org.apache.skywalking.apm.agent.core.context.trace.SpanLayer;
+import org.apache.skywalking.apm.network.trace.component.ComponentsDefine;
+import org.apache.skywalking.apm.plugin.grpc.v1.OperationNameFormatUtil;
+
+import static org.apache.skywalking.apm.plugin.grpc.v1.Constants.*;
+
+/**
+ * @author wang zheng, kanro
+ */
+public class TracingServerCallListener<REQUEST> extends ForwardingServerCallListener.SimpleForwardingServerCallListener<REQUEST> {
+
+    private final ContextSnapshot contextSnapshot;
+    private final MethodDescriptor.MethodType methodType;
+    private final String operationPrefix;
+
+    protected TracingServerCallListener(ServerCall.Listener<REQUEST> delegate, MethodDescriptor<REQUEST, ?> descriptor,
+                                        ContextSnapshot contextSnapshot) {
+        super(delegate);
+        this.contextSnapshot = contextSnapshot;
+        this.methodType = descriptor.getType();
+        this.operationPrefix = OperationNameFormatUtil.formatOperationName(descriptor) + SERVER;
+    }
+
+    @Override
+    public void onMessage(REQUEST message) {
+        // We just create the request on message span for client stream calls.
+        if (!methodType.clientSendsOneMessage()) {
+            try {
+                final AbstractSpan span = ContextManager.createLocalSpan(operationPrefix + REQUEST_ON_MESSAGE_OPERATION_NAME);
+                span.setComponent(ComponentsDefine.GRPC);
+                span.setLayer(SpanLayer.RPC_FRAMEWORK);
+                ContextManager.continued(contextSnapshot);
+                super.onMessage(message);
 
 Review comment:
   A similar suggestion, `try/final` should be added here.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [skywalking] wu-sheng commented on a change in pull request #4177: Enhance gRPC plugin

Posted by GitBox <gi...@apache.org>.
wu-sheng commented on a change in pull request #4177: Enhance gRPC plugin
URL: https://github.com/apache/skywalking/pull/4177#discussion_r371645672
 
 

 ##########
 File path: apm-sniffer/apm-sdk-plugin/grpc-1.x-plugin/src/main/java/org/apache/skywalking/apm/plugin/grpc/v1/server/TracingServerCall.java
 ##########
 @@ -0,0 +1,111 @@
+/*
+ * 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.apm.plugin.grpc.v1.server;
+
+import io.grpc.ForwardingServerCall;
+import io.grpc.Metadata;
+import io.grpc.ServerCall;
+import io.grpc.Status;
+import org.apache.skywalking.apm.agent.core.context.ContextManager;
+import org.apache.skywalking.apm.agent.core.context.ContextSnapshot;
+import org.apache.skywalking.apm.agent.core.context.tag.Tags;
+import org.apache.skywalking.apm.agent.core.context.trace.AbstractSpan;
+import org.apache.skywalking.apm.agent.core.context.trace.SpanLayer;
+import org.apache.skywalking.apm.network.trace.component.ComponentsDefine;
+import org.apache.skywalking.apm.plugin.grpc.v1.OperationNameFormatUtil;
+
+import static org.apache.skywalking.apm.plugin.grpc.v1.Constants.*;
+
+/**
+ * @author wang zheng, kanro
+ */
+public class TracingServerCall<REQUEST, RESPONSE> extends ForwardingServerCall.SimpleForwardingServerCall<REQUEST, RESPONSE> {
+
+    private final String operationPrefix;
+    private final ContextSnapshot contextSnapshot;
+
+    protected TracingServerCall(ServerCall<REQUEST, RESPONSE> delegate, ContextSnapshot contextSnapshot) {
+        super(delegate);
+        this.operationPrefix = OperationNameFormatUtil.formatOperationName(delegate.getMethodDescriptor()) + SERVER;
+        this.contextSnapshot = contextSnapshot;
+    }
+
+    @Override
+    public void sendMessage(RESPONSE message) {
+        // We just create the request on message span for server stream calls.
+        if (!getMethodDescriptor().getType().serverSendsOneMessage()) {
+            try {
+                final AbstractSpan span = ContextManager.createLocalSpan(operationPrefix + RESPONSE_ON_MESSAGE_OPERATION_NAME);
+                span.setComponent(ComponentsDefine.GRPC);
+                span.setLayer(SpanLayer.RPC_FRAMEWORK);
+                ContextManager.continued(contextSnapshot);
+            } catch (Throwable t) {
+                ContextManager.activeSpan().errorOccurred().log(t);
+                throw t;
+            } finally {
+                super.sendMessage(message);
+                ContextManager.stopSpan();
+            }
+        } else {
+            super.sendMessage(message);
+        }
+    }
+
+    @Override
+    public void close(Status status, Metadata trailers) {
+        try {
+            final AbstractSpan span = ContextManager.createLocalSpan(operationPrefix + RESPONSE_ON_CLOSE_OPERATION_NAME);
+            span.setComponent(ComponentsDefine.GRPC);
+            span.setLayer(SpanLayer.RPC_FRAMEWORK);
+            ContextManager.continued(contextSnapshot);
+
+            switch (status.getCode()) {
+                case OK:
+                    break;
+                // UNKNOWN/INTERNAL status code will case error in this span.
+                // Those status code means some unexpected error occurred in server.
+                // Similar to 5XX in HTTP status.
+                case UNKNOWN:
+                case INTERNAL:
+                    if (status.getCause() == null) {
+                        span.errorOccurred().log(status.asRuntimeException());
+                    } else {
+                        span.errorOccurred().log(status.getCause());
+                    }
+                    break;
+                // Other status code means some predictable error occurred in server.
+                // Like PERMISSION_DENIED or UNAUTHENTICATED somethings.
+                // Similar to 4XX in HTTP status.
+                default:
+                    // But if the status still has cause exception, we will log it too.
+                    if (status.getCause() != null) {
+                        span.errorOccurred().log(status.getCause());
+                    }
+                    break;
+            }
+            Tags.STATUS_CODE.set(span, status.getCode().name());
+        } catch (Throwable t) {
 
 Review comment:
   What is this catch for? I can't see any code happens except the plugin logic. We should not log an error caused by our own codes, rather than users.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [skywalking] wu-sheng commented on issue #4177: Enhance gRPC plugin

Posted by GitBox <gi...@apache.org>.
wu-sheng commented on issue #4177: Enhance gRPC plugin
URL: https://github.com/apache/skywalking/pull/4177#issuecomment-575198294
 
 
   Got it. Waiting for you update.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [skywalking] GuoDuanLZ commented on a change in pull request #4177: Enhance gRPC plugin

Posted by GitBox <gi...@apache.org>.
GuoDuanLZ commented on a change in pull request #4177: Enhance gRPC plugin
URL: https://github.com/apache/skywalking/pull/4177#discussion_r371717507
 
 

 ##########
 File path: test/plugin/scenarios/grpc-scenario/config/expectedData.yaml
 ##########
 @@ -15,239 +15,524 @@
 # limitations under the License.
 registryItems:
   applications:
-    - {grpc-scenario: 2}
+  - { grpc-scenario: 2 }
   instances:
-    - {grpc-scenario: 1}
+  - { grpc-scenario: 1 }
   operationNames:
-    - grpc-scenario: [Greeter.sayHello, GreeterBlocking.sayHello,
-                      /grpc-scenario/case/grpc-scenario]
+  - grpc-scenario: [Greeter.sayHello, GreeterBlocking.sayHello, GreeterBlockingError.sayHello, /grpc-scenario/case/grpc-scenario]
   heartbeat: []
 segmentItems:
-  - applicationCode: grpc-scenario
-    segmentSize: gt 10
-    segments:
-      - segmentId: not null
-        spans:
-          - operationName: GreeterBlocking.sayHello
-            operationId: 0
-            parentSpanId: -1
-            spanId: 0
-            spanLayer: RPCFramework
-            startTime: nq 0
-            endTime: nq 0
-            componentId: 23
-            componentName: ''
-            isError: false
-            spanType: Entry
-            peer: ''
-            peerId: 0
-            refs:
-              - {parentEndpointId: 0, parentEndpoint: /grpc-scenario/case/grpc-scenario, networkAddressId: 0,
-                 entryEndpointId: 0, refType: CrossProcess, parentSpanId: 2, parentTraceSegmentId: not null,
-                 parentServiceInstanceId: 1, networkAddress: '127.0.0.1:18080', entryEndpoint: /grpc-scenario/case/grpc-scenario,
-                 entryServiceInstanceId: 1}
-      - segmentId: not null
-        spans:
-          - operationName: Greeter.sayHello
-            operationId: 0
-            parentSpanId: -1
-            spanId: 0
-            spanLayer: RPCFramework
-            startTime: nq 0
-            endTime: nq 0
-            componentId: 23
-            componentName: ''
-            isError: false
-            spanType: Entry
-            peer: ''
-            peerId: 0
-            refs:
-              - {parentEndpointId: 0, parentEndpoint: /grpc-scenario/case/grpc-scenario, networkAddressId: 0,
-                 entryEndpointId: 0, refType: CrossProcess, parentSpanId: 1, parentTraceSegmentId: not null,
-                 parentServiceInstanceId: 1, networkAddress: '127.0.0.1:18080', entryEndpoint: /grpc-scenario/case/grpc-scenario,
-                 entryServiceInstanceId: 1}
-      - segmentId: not null
-        spans:
-          - operationName: GreeterBlocking.sayHello/server/RequestObserver/onNext
-            operationId: 0
-            parentSpanId: -1
-            spanId: 0
-            spanLayer: Unknown
-            startTime: nq 0
-            endTime: nq 0
-            componentId: 0
-            componentName: ''
-            isError: false
-            spanType: Local
-            peer: ''
-            peerId: 0
-            refs:
-              - {parentEndpointId: 0, parentEndpoint: GreeterBlocking.sayHello, networkAddressId: 0,
-                 entryEndpointId: 0, refType: CrossThread, parentSpanId: 0, parentTraceSegmentId: not null,
-                 parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
-                 entryServiceInstanceId: 1}
-      - segmentId: not null
-        spans:
-          - operationName: Greeter.sayHello/server/RequestObserver/onNext
-            operationId: 0
-            parentSpanId: -1
-            spanId: 0
-            spanLayer: Unknown
-            startTime: nq 0
-            endTime: nq 0
-            componentId: 0
-            componentName: ''
-            isError: false
-            spanType: Local
-            peer: ''
-            peerId: 0
-            refs:
-              - {parentEndpointId: 0, parentEndpoint: Greeter.sayHello, networkAddressId: 0,
-                 entryEndpointId: 0, refType: CrossThread, parentSpanId: 0, parentTraceSegmentId: not null,
-                 parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
-                 entryServiceInstanceId: 1}
-      - segmentId: not null
-        spans:
-          - operationName: Greeter.sayHello/client/ResponseObserver/onNext
-            operationId: 0
-            parentSpanId: -1
-            spanId: 0
-            spanLayer: Unknown
-            startTime: nq 0
-            endTime: nq 0
-            componentId: 0
-            componentName: ''
-            isError: false
-            spanType: Local
-            peer: ''
-            peerId: 0
-            refs:
-              - {parentEndpointId: 0, parentEndpoint: /grpc-scenario/case/grpc-scenario, networkAddressId: 0,
-                 entryEndpointId: 0, refType: CrossThread, parentSpanId: 1, parentTraceSegmentId: not null,
-                 parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
-                 entryServiceInstanceId: 1}
-      - segmentId: not null
-        spans:
-          - operationName: Greeter.sayHello/client/ResponseObserver/onComplete
-            operationId: 0
-            parentSpanId: -1
-            spanId: 0
-            spanLayer: Unknown
-            startTime: nq 0
-            endTime: nq 0
-            componentId: 0
-            componentName: ''
-            isError: false
-            spanType: Local
-            peer: ''
-            peerId: 0
-            refs:
-              - {parentEndpointId: 0, parentEndpoint: /grpc-scenario/case/grpc-scenario, networkAddressId: 0,
-                 entryEndpointId: 0, refType: CrossThread, parentSpanId: 1, parentTraceSegmentId: not null,
-                 parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
-                 entryServiceInstanceId: 1}
-      - segmentId: not null
-        spans:
-          - operationName: Greeter.sayHello/server/RequestObserver/onNext
-            operationId: 0
-            parentSpanId: -1
-            spanId: 0
-            spanLayer: Unknown
-            startTime: nq 0
-            endTime: nq 0
-            componentId: 0
-            componentName: ''
-            isError: false
-            spanType: Local
-            peer: ''
-            peerId: 0
-            refs:
-              - {parentEndpointId: 0, parentEndpoint: Greeter.sayHello, networkAddressId: 0,
-                 entryEndpointId: 0, refType: CrossThread, parentSpanId: 0, parentTraceSegmentId: not null,
-                 parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
-                 entryServiceInstanceId: 1}
-      - segmentId: not null
-        spans:
-          - operationName: Greeter.sayHello/server/RequestObserver/onComplete
-            operationId: 0
-            parentSpanId: -1
-            spanId: 0
-            spanLayer: Unknown
-            startTime: nq 0
-            endTime: nq 0
-            componentId: 0
-            componentName: ''
-            isError: false
-            spanType: Local
-            peer: ''
-            peerId: 0
-            refs:
-              - {parentEndpointId: 0, parentEndpoint: Greeter.sayHello, networkAddressId: 0,
-                 entryEndpointId: 0, refType: CrossThread, parentSpanId: 0, parentTraceSegmentId: not null,
-                 parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
-                 entryServiceInstanceId: 1}
-      - segmentId: not null
-        spans:
-          - operationName: Greeter.sayHello
-            operationId: 0
-            parentSpanId: 0
-            spanId: 1
-            spanLayer: RPCFramework
-            startTime: nq 0
-            endTime: nq 0
-            componentId: 23
-            componentName: ''
-            isError: false
-            spanType: Exit
-            peer: '127.0.0.1:18080'
-            peerId: 0
-          - operationName: GreeterBlocking.sayHello
-            operationId: 0
-            parentSpanId: 0
-            spanId: 2
-            spanLayer: RPCFramework
-            startTime: nq 0
-            endTime: nq 0
-            componentId: 23
-            componentName: ''
-            isError: false
-            spanType: Exit
-            peer: '127.0.0.1:18080'
-            peerId: 0
-          - operationName: /grpc-scenario/case/grpc-scenario
-            operationId: 0
-            parentSpanId: -1
-            spanId: 0
-            spanLayer: Http
-            startTime: nq 0
-            endTime: nq 0
-            componentId: 1
-            componentName: ''
-            isError: false
-            spanType: Entry
-            peer: ''
-            peerId: 0
-            tags:
-              - {key: url, value: 'http://localhost:8080/grpc-scenario/case/grpc-scenario'}
-              - {key: http.method, value: GET}
-      - segmentId: not null
-        spans:
-          - operationName: Greeter.sayHello/client/ResponseObserver/onNext
-            operationId: 0
-            parentSpanId: -1
-            spanId: 0
-            spanLayer: Unknown
-            startTime: nq 0
-            endTime: nq 0
-            componentId: 0
-            componentName: ''
-            isError: false
-            spanType: Local
-            peer: ''
-            peerId: 0
-            refs:
-              - {parentEndpointId: 0, parentEndpoint: /grpc-scenario/case/grpc-scenario, networkAddressId: 0,
-                 entryEndpointId: 0, refType: CrossThread, parentSpanId: 1, parentTraceSegmentId: not null,
-                 parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
-                 entryServiceInstanceId: 1}
-
+- applicationCode: grpc-scenario
+  segmentSize: gt 10
+  segments:
+  - segmentId: not null
+    spans:
+    - operationName: GreeterBlocking.sayHello
+      operationId: 0
+      parentSpanId: -1
+      spanId: 0
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Entry
+      peer: ''
+      peerId: 0
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: /grpc-scenario/case/grpc-scenario, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossProcess, parentSpanId: 2, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '127.0.0.1:18080', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+  - segmentId: not null
+    spans:
+    - operationName: Greeter.sayHello/client/Request/onMessage
+      operationId: 0
+      parentSpanId: -1
+      spanId: 0
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Local
+      peer: ''
+      peerId: 0
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: /grpc-scenario/case/grpc-scenario, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossThread, parentSpanId: 1, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+  - segmentId: not null
+    spans:
+    - operationName: Greeter.sayHello/client/Request/onMessage
+      operationId: 0
+      parentSpanId: -1
+      spanId: 0
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Local
+      peer: ''
+      peerId: 0
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: /grpc-scenario/case/grpc-scenario, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossThread, parentSpanId: 1, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+  - segmentId: not null
+    spans:
+    - operationName: Greeter.sayHello/client/Request/onComplete
+      operationId: 0
+      parentSpanId: -1
+      spanId: 0
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Local
+      peer: ''
+      peerId: 0
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: /grpc-scenario/case/grpc-scenario, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossThread, parentSpanId: 1, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+  - segmentId: not null
+    spans:
+    - operationName: Greeter.sayHello
+      operationId: 0
+      parentSpanId: -1
+      spanId: 0
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Entry
+      peer: ''
+      peerId: 0
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: /grpc-scenario/case/grpc-scenario, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossProcess, parentSpanId: 1, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '127.0.0.1:18080', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+  - segmentId: not null
+    spans:
+    - operationName: Greeter.sayHello/server/Response/onMessage
+      operationId: 0
+      parentSpanId: 0
+      spanId: 1
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Local
+      peer: ''
+      peerId: 0
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: Greeter.sayHello, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossThread, parentSpanId: 0, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+    - operationName: Greeter.sayHello/server/Request/onMessage
+      operationId: 0
+      parentSpanId: -1
+      spanId: 0
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Local
+      peer: ''
+      peerId: 0
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: Greeter.sayHello, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossThread, parentSpanId: 0, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+  - segmentId: not null
+    spans:
+    - operationName: Greeter.sayHello/client/Response/onMessage
+      operationId: 0
+      parentSpanId: -1
+      spanId: 0
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Local
+      peer: ''
+      peerId: 0
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: /grpc-scenario/case/grpc-scenario, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossThread, parentSpanId: 1, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+  - segmentId: not null
+    spans:
+    - operationName: GreeterBlocking.sayHello/server/Response/onClose
+      operationId: 0
+      parentSpanId: 0
+      spanId: 1
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Local
+      peer: ''
+      peerId: 0
+      tags:
+      - { key: status_code, value: OK }
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: GreeterBlocking.sayHello, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossThread, parentSpanId: 0, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+    - operationName: GreeterBlocking.sayHello/server/Request/onComplete
+      operationId: 0
+      parentSpanId: -1
+      spanId: 0
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Local
+      peer: ''
+      peerId: 0
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: GreeterBlocking.sayHello, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossThread, parentSpanId: 0, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+  - segmentId: not null
+    spans:
+    - operationName: Greeter.sayHello/server/Response/onMessage
+      operationId: 0
+      parentSpanId: 0
+      spanId: 1
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Local
+      peer: ''
+      peerId: 0
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: Greeter.sayHello, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossThread, parentSpanId: 0, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+    - operationName: Greeter.sayHello/server/Request/onMessage
+      operationId: 0
+      parentSpanId: -1
+      spanId: 0
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Local
+      peer: ''
+      peerId: 0
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: Greeter.sayHello, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossThread, parentSpanId: 0, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+  - segmentId: not null
+    spans:
+    - operationName: Greeter.sayHello/server/Response/onClose
+      operationId: 0
+      parentSpanId: 0
+      spanId: 1
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Local
+      peer: ''
+      peerId: 0
+      tags:
+      - {key: status_code, value: OK}
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: Greeter.sayHello, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossThread, parentSpanId: 0, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+    - operationName: Greeter.sayHello/server/Request/onComplete
+      operationId: 0
+      parentSpanId: -1
+      spanId: 0
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Local
+      peer: ''
+      peerId: 0
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: Greeter.sayHello, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossThread, parentSpanId: 0, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+  - segmentId: not null
+    spans:
+    - operationName: Greeter.sayHello/client/Response/onMessage
+      operationId: 0
+      parentSpanId: -1
+      spanId: 0
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Local
+      peer: ''
+      peerId: 0
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: /grpc-scenario/case/grpc-scenario, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossThread, parentSpanId: 1, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+  - segmentId: not null
+    spans:
+    - operationName: Greeter.sayHello/client/Response/onClose
+      operationId: 0
+      parentSpanId: -1
+      spanId: 0
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Local
+      peer: ''
+      peerId: 0
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: /grpc-scenario/case/grpc-scenario, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossThread, parentSpanId: 1, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+  - segmentId: not null
+    spans:
+    - operationName: GreeterBlockingError.sayHello
+      operationId: 0
+      parentSpanId: -1
+      spanId: 0
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Entry
+      peer: ''
+      peerId: 0
+      refs:
+        - {parentEndpointId: 0, parentEndpoint: /grpc-scenario/case/grpc-scenario, networkAddressId: 0,
+           entryEndpointId: 0, refType: CrossProcess, parentSpanId: 5, parentTraceSegmentId: not null,
+           parentServiceInstanceId: 1, networkAddress: '127.0.0.1:18080', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+           entryServiceInstanceId: 1}
+  - segmentId: not null
+    spans:
+    - operationName: GreeterBlockingError.sayHello/server/Response/onClose
+      operationId: 0
+      parentSpanId: 0
+      spanId: 1
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: true
+      spanType: Local
+      peer: ''
+      peerId: 0
+      tags:
+      - { key: status_code, value: UNKNOWN }
+      logs:
+      - logEvent:
+        - { key: event, value: error }
+        - { key: error.kind, value: java.lang.Exception }
+        - { key: message, value: '' }
+        - key: stack
+          value: not null
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: GreeterBlockingError.sayHello, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossThread, parentSpanId: 0, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+    - operationName: GreeterBlockingError.sayHello/server/Request/onComplete
+      operationId: 0
+      parentSpanId: -1
+      spanId: 0
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Local
+      peer: ''
+      peerId: 0
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: GreeterBlockingError.sayHello, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossThread, parentSpanId: 0, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+  - segmentId: not null
+    spans:
+    - operationName: Greeter.sayHello
+      operationId: 0
+      parentSpanId: 0
+      spanId: 1
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Exit
+      peer: '127.0.0.1:18080'
+      peerId: 0
+    - {operationName: GreeterBlocking.sayHello/client/Request/onComplete, operationId: 0,
+       parentSpanId: 2, spanId: 3, spanLayer: RPCFramework, startTime: nq 0,
+       endTime: nq 0, componentId: 23, componentName: '', isError: false,
+       spanType: Local, peer: '', peerId: 0}
+    - {operationName: GreeterBlocking.sayHello/client/Response/onClose, operationId: 0,
 
 Review comment:
   When the span is shorter, it is a simplified format.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [skywalking] codecov-io edited a comment on issue #4177: Enhance gRPC plugin

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #4177: Enhance gRPC plugin
URL: https://github.com/apache/skywalking/pull/4177#issuecomment-575098453
 
 
   # [Codecov](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=h1) Report
   > Merging [#4177](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=desc) into [master](https://codecov.io/gh/apache/skywalking/commit/5f00e93c0b53eb0b116bb769799c7bd650743bf1?src=pr&el=desc) will **not change** coverage.
   > The diff coverage is `n/a`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/skywalking/pull/4177/graphs/tree.svg?width=650&token=qrILxY5yA8&height=150&src=pr)](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=tree)
   
   ```diff
   @@           Coverage Diff           @@
   ##           master    #4177   +/-   ##
   =======================================
     Coverage   26.83%   26.83%           
   =======================================
     Files        1162     1162           
     Lines       25448    25448           
     Branches     3630     3630           
   =======================================
     Hits         6829     6829           
     Misses      18014    18014           
     Partials      605      605
   ```
   
   
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=footer). Last update [5f00e93...23be984](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [skywalking] wu-sheng commented on a change in pull request #4177: Enhance gRPC plugin

Posted by GitBox <gi...@apache.org>.
wu-sheng commented on a change in pull request #4177: Enhance gRPC plugin
URL: https://github.com/apache/skywalking/pull/4177#discussion_r371669444
 
 

 ##########
 File path: test/plugin/scenarios/grpc-scenario/config/expectedData.yaml
 ##########
 @@ -15,239 +15,524 @@
 # limitations under the License.
 registryItems:
   applications:
-    - {grpc-scenario: 2}
+  - { grpc-scenario: 2 }
   instances:
-    - {grpc-scenario: 1}
+  - { grpc-scenario: 1 }
   operationNames:
-    - grpc-scenario: [Greeter.sayHello, GreeterBlocking.sayHello,
-                      /grpc-scenario/case/grpc-scenario]
+  - grpc-scenario: [Greeter.sayHello, GreeterBlocking.sayHello, GreeterBlockingError.sayHello, /grpc-scenario/case/grpc-scenario]
   heartbeat: []
 segmentItems:
-  - applicationCode: grpc-scenario
-    segmentSize: gt 10
-    segments:
-      - segmentId: not null
-        spans:
-          - operationName: GreeterBlocking.sayHello
-            operationId: 0
-            parentSpanId: -1
-            spanId: 0
-            spanLayer: RPCFramework
-            startTime: nq 0
-            endTime: nq 0
-            componentId: 23
-            componentName: ''
-            isError: false
-            spanType: Entry
-            peer: ''
-            peerId: 0
-            refs:
-              - {parentEndpointId: 0, parentEndpoint: /grpc-scenario/case/grpc-scenario, networkAddressId: 0,
-                 entryEndpointId: 0, refType: CrossProcess, parentSpanId: 2, parentTraceSegmentId: not null,
-                 parentServiceInstanceId: 1, networkAddress: '127.0.0.1:18080', entryEndpoint: /grpc-scenario/case/grpc-scenario,
-                 entryServiceInstanceId: 1}
-      - segmentId: not null
-        spans:
-          - operationName: Greeter.sayHello
-            operationId: 0
-            parentSpanId: -1
-            spanId: 0
-            spanLayer: RPCFramework
-            startTime: nq 0
-            endTime: nq 0
-            componentId: 23
-            componentName: ''
-            isError: false
-            spanType: Entry
-            peer: ''
-            peerId: 0
-            refs:
-              - {parentEndpointId: 0, parentEndpoint: /grpc-scenario/case/grpc-scenario, networkAddressId: 0,
-                 entryEndpointId: 0, refType: CrossProcess, parentSpanId: 1, parentTraceSegmentId: not null,
-                 parentServiceInstanceId: 1, networkAddress: '127.0.0.1:18080', entryEndpoint: /grpc-scenario/case/grpc-scenario,
-                 entryServiceInstanceId: 1}
-      - segmentId: not null
-        spans:
-          - operationName: GreeterBlocking.sayHello/server/RequestObserver/onNext
-            operationId: 0
-            parentSpanId: -1
-            spanId: 0
-            spanLayer: Unknown
-            startTime: nq 0
-            endTime: nq 0
-            componentId: 0
-            componentName: ''
-            isError: false
-            spanType: Local
-            peer: ''
-            peerId: 0
-            refs:
-              - {parentEndpointId: 0, parentEndpoint: GreeterBlocking.sayHello, networkAddressId: 0,
-                 entryEndpointId: 0, refType: CrossThread, parentSpanId: 0, parentTraceSegmentId: not null,
-                 parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
-                 entryServiceInstanceId: 1}
-      - segmentId: not null
-        spans:
-          - operationName: Greeter.sayHello/server/RequestObserver/onNext
-            operationId: 0
-            parentSpanId: -1
-            spanId: 0
-            spanLayer: Unknown
-            startTime: nq 0
-            endTime: nq 0
-            componentId: 0
-            componentName: ''
-            isError: false
-            spanType: Local
-            peer: ''
-            peerId: 0
-            refs:
-              - {parentEndpointId: 0, parentEndpoint: Greeter.sayHello, networkAddressId: 0,
-                 entryEndpointId: 0, refType: CrossThread, parentSpanId: 0, parentTraceSegmentId: not null,
-                 parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
-                 entryServiceInstanceId: 1}
-      - segmentId: not null
-        spans:
-          - operationName: Greeter.sayHello/client/ResponseObserver/onNext
-            operationId: 0
-            parentSpanId: -1
-            spanId: 0
-            spanLayer: Unknown
-            startTime: nq 0
-            endTime: nq 0
-            componentId: 0
-            componentName: ''
-            isError: false
-            spanType: Local
-            peer: ''
-            peerId: 0
-            refs:
-              - {parentEndpointId: 0, parentEndpoint: /grpc-scenario/case/grpc-scenario, networkAddressId: 0,
-                 entryEndpointId: 0, refType: CrossThread, parentSpanId: 1, parentTraceSegmentId: not null,
-                 parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
-                 entryServiceInstanceId: 1}
-      - segmentId: not null
-        spans:
-          - operationName: Greeter.sayHello/client/ResponseObserver/onComplete
-            operationId: 0
-            parentSpanId: -1
-            spanId: 0
-            spanLayer: Unknown
-            startTime: nq 0
-            endTime: nq 0
-            componentId: 0
-            componentName: ''
-            isError: false
-            spanType: Local
-            peer: ''
-            peerId: 0
-            refs:
-              - {parentEndpointId: 0, parentEndpoint: /grpc-scenario/case/grpc-scenario, networkAddressId: 0,
-                 entryEndpointId: 0, refType: CrossThread, parentSpanId: 1, parentTraceSegmentId: not null,
-                 parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
-                 entryServiceInstanceId: 1}
-      - segmentId: not null
-        spans:
-          - operationName: Greeter.sayHello/server/RequestObserver/onNext
-            operationId: 0
-            parentSpanId: -1
-            spanId: 0
-            spanLayer: Unknown
-            startTime: nq 0
-            endTime: nq 0
-            componentId: 0
-            componentName: ''
-            isError: false
-            spanType: Local
-            peer: ''
-            peerId: 0
-            refs:
-              - {parentEndpointId: 0, parentEndpoint: Greeter.sayHello, networkAddressId: 0,
-                 entryEndpointId: 0, refType: CrossThread, parentSpanId: 0, parentTraceSegmentId: not null,
-                 parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
-                 entryServiceInstanceId: 1}
-      - segmentId: not null
-        spans:
-          - operationName: Greeter.sayHello/server/RequestObserver/onComplete
-            operationId: 0
-            parentSpanId: -1
-            spanId: 0
-            spanLayer: Unknown
-            startTime: nq 0
-            endTime: nq 0
-            componentId: 0
-            componentName: ''
-            isError: false
-            spanType: Local
-            peer: ''
-            peerId: 0
-            refs:
-              - {parentEndpointId: 0, parentEndpoint: Greeter.sayHello, networkAddressId: 0,
-                 entryEndpointId: 0, refType: CrossThread, parentSpanId: 0, parentTraceSegmentId: not null,
-                 parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
-                 entryServiceInstanceId: 1}
-      - segmentId: not null
-        spans:
-          - operationName: Greeter.sayHello
-            operationId: 0
-            parentSpanId: 0
-            spanId: 1
-            spanLayer: RPCFramework
-            startTime: nq 0
-            endTime: nq 0
-            componentId: 23
-            componentName: ''
-            isError: false
-            spanType: Exit
-            peer: '127.0.0.1:18080'
-            peerId: 0
-          - operationName: GreeterBlocking.sayHello
-            operationId: 0
-            parentSpanId: 0
-            spanId: 2
-            spanLayer: RPCFramework
-            startTime: nq 0
-            endTime: nq 0
-            componentId: 23
-            componentName: ''
-            isError: false
-            spanType: Exit
-            peer: '127.0.0.1:18080'
-            peerId: 0
-          - operationName: /grpc-scenario/case/grpc-scenario
-            operationId: 0
-            parentSpanId: -1
-            spanId: 0
-            spanLayer: Http
-            startTime: nq 0
-            endTime: nq 0
-            componentId: 1
-            componentName: ''
-            isError: false
-            spanType: Entry
-            peer: ''
-            peerId: 0
-            tags:
-              - {key: url, value: 'http://localhost:8080/grpc-scenario/case/grpc-scenario'}
-              - {key: http.method, value: GET}
-      - segmentId: not null
-        spans:
-          - operationName: Greeter.sayHello/client/ResponseObserver/onNext
-            operationId: 0
-            parentSpanId: -1
-            spanId: 0
-            spanLayer: Unknown
-            startTime: nq 0
-            endTime: nq 0
-            componentId: 0
-            componentName: ''
-            isError: false
-            spanType: Local
-            peer: ''
-            peerId: 0
-            refs:
-              - {parentEndpointId: 0, parentEndpoint: /grpc-scenario/case/grpc-scenario, networkAddressId: 0,
-                 entryEndpointId: 0, refType: CrossThread, parentSpanId: 1, parentTraceSegmentId: not null,
-                 parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
-                 entryServiceInstanceId: 1}
-
+- applicationCode: grpc-scenario
+  segmentSize: gt 10
+  segments:
+  - segmentId: not null
+    spans:
+    - operationName: GreeterBlocking.sayHello
+      operationId: 0
+      parentSpanId: -1
+      spanId: 0
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Entry
+      peer: ''
+      peerId: 0
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: /grpc-scenario/case/grpc-scenario, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossProcess, parentSpanId: 2, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '127.0.0.1:18080', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+  - segmentId: not null
+    spans:
+    - operationName: Greeter.sayHello/client/Request/onMessage
+      operationId: 0
+      parentSpanId: -1
+      spanId: 0
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Local
+      peer: ''
+      peerId: 0
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: /grpc-scenario/case/grpc-scenario, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossThread, parentSpanId: 1, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+  - segmentId: not null
+    spans:
+    - operationName: Greeter.sayHello/client/Request/onMessage
+      operationId: 0
+      parentSpanId: -1
+      spanId: 0
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Local
+      peer: ''
+      peerId: 0
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: /grpc-scenario/case/grpc-scenario, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossThread, parentSpanId: 1, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+  - segmentId: not null
+    spans:
+    - operationName: Greeter.sayHello/client/Request/onComplete
+      operationId: 0
+      parentSpanId: -1
+      spanId: 0
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Local
+      peer: ''
+      peerId: 0
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: /grpc-scenario/case/grpc-scenario, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossThread, parentSpanId: 1, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+  - segmentId: not null
+    spans:
+    - operationName: Greeter.sayHello
+      operationId: 0
+      parentSpanId: -1
+      spanId: 0
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Entry
+      peer: ''
+      peerId: 0
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: /grpc-scenario/case/grpc-scenario, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossProcess, parentSpanId: 1, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '127.0.0.1:18080', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+  - segmentId: not null
+    spans:
+    - operationName: Greeter.sayHello/server/Response/onMessage
+      operationId: 0
+      parentSpanId: 0
+      spanId: 1
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Local
+      peer: ''
+      peerId: 0
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: Greeter.sayHello, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossThread, parentSpanId: 0, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+    - operationName: Greeter.sayHello/server/Request/onMessage
+      operationId: 0
+      parentSpanId: -1
+      spanId: 0
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Local
+      peer: ''
+      peerId: 0
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: Greeter.sayHello, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossThread, parentSpanId: 0, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+  - segmentId: not null
+    spans:
+    - operationName: Greeter.sayHello/client/Response/onMessage
+      operationId: 0
+      parentSpanId: -1
+      spanId: 0
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Local
+      peer: ''
+      peerId: 0
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: /grpc-scenario/case/grpc-scenario, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossThread, parentSpanId: 1, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+  - segmentId: not null
+    spans:
+    - operationName: GreeterBlocking.sayHello/server/Response/onClose
+      operationId: 0
+      parentSpanId: 0
+      spanId: 1
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Local
+      peer: ''
+      peerId: 0
+      tags:
+      - { key: status_code, value: OK }
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: GreeterBlocking.sayHello, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossThread, parentSpanId: 0, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+    - operationName: GreeterBlocking.sayHello/server/Request/onComplete
+      operationId: 0
+      parentSpanId: -1
+      spanId: 0
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Local
+      peer: ''
+      peerId: 0
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: GreeterBlocking.sayHello, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossThread, parentSpanId: 0, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+  - segmentId: not null
+    spans:
+    - operationName: Greeter.sayHello/server/Response/onMessage
+      operationId: 0
+      parentSpanId: 0
+      spanId: 1
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Local
+      peer: ''
+      peerId: 0
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: Greeter.sayHello, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossThread, parentSpanId: 0, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+    - operationName: Greeter.sayHello/server/Request/onMessage
+      operationId: 0
+      parentSpanId: -1
+      spanId: 0
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Local
+      peer: ''
+      peerId: 0
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: Greeter.sayHello, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossThread, parentSpanId: 0, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+  - segmentId: not null
+    spans:
+    - operationName: Greeter.sayHello/server/Response/onClose
+      operationId: 0
+      parentSpanId: 0
+      spanId: 1
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Local
+      peer: ''
+      peerId: 0
+      tags:
+      - {key: status_code, value: OK}
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: Greeter.sayHello, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossThread, parentSpanId: 0, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+    - operationName: Greeter.sayHello/server/Request/onComplete
+      operationId: 0
+      parentSpanId: -1
+      spanId: 0
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Local
+      peer: ''
+      peerId: 0
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: Greeter.sayHello, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossThread, parentSpanId: 0, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+  - segmentId: not null
+    spans:
+    - operationName: Greeter.sayHello/client/Response/onMessage
+      operationId: 0
+      parentSpanId: -1
+      spanId: 0
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Local
+      peer: ''
+      peerId: 0
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: /grpc-scenario/case/grpc-scenario, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossThread, parentSpanId: 1, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+  - segmentId: not null
+    spans:
+    - operationName: Greeter.sayHello/client/Response/onClose
+      operationId: 0
+      parentSpanId: -1
+      spanId: 0
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Local
+      peer: ''
+      peerId: 0
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: /grpc-scenario/case/grpc-scenario, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossThread, parentSpanId: 1, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+  - segmentId: not null
+    spans:
+    - operationName: GreeterBlockingError.sayHello
+      operationId: 0
+      parentSpanId: -1
+      spanId: 0
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Entry
+      peer: ''
+      peerId: 0
+      refs:
+        - {parentEndpointId: 0, parentEndpoint: /grpc-scenario/case/grpc-scenario, networkAddressId: 0,
+           entryEndpointId: 0, refType: CrossProcess, parentSpanId: 5, parentTraceSegmentId: not null,
+           parentServiceInstanceId: 1, networkAddress: '127.0.0.1:18080', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+           entryServiceInstanceId: 1}
+  - segmentId: not null
+    spans:
+    - operationName: GreeterBlockingError.sayHello/server/Response/onClose
+      operationId: 0
+      parentSpanId: 0
+      spanId: 1
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: true
+      spanType: Local
+      peer: ''
+      peerId: 0
+      tags:
+      - { key: status_code, value: UNKNOWN }
+      logs:
+      - logEvent:
+        - { key: event, value: error }
+        - { key: error.kind, value: java.lang.Exception }
+        - { key: message, value: '' }
+        - key: stack
+          value: not null
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: GreeterBlockingError.sayHello, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossThread, parentSpanId: 0, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+    - operationName: GreeterBlockingError.sayHello/server/Request/onComplete
+      operationId: 0
+      parentSpanId: -1
+      spanId: 0
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Local
+      peer: ''
+      peerId: 0
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: GreeterBlockingError.sayHello, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossThread, parentSpanId: 0, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+  - segmentId: not null
+    spans:
+    - operationName: Greeter.sayHello
+      operationId: 0
+      parentSpanId: 0
+      spanId: 1
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Exit
+      peer: '127.0.0.1:18080'
+      peerId: 0
+    - {operationName: GreeterBlocking.sayHello/client/Request/onComplete, operationId: 0,
+       parentSpanId: 2, spanId: 3, spanLayer: RPCFramework, startTime: nq 0,
+       endTime: nq 0, componentId: 23, componentName: '', isError: false,
+       spanType: Local, peer: '', peerId: 0}
+    - {operationName: GreeterBlocking.sayHello/client/Response/onClose, operationId: 0,
 
 Review comment:
   Why do these two use different formats? I know they may be right but it seems strange.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [skywalking] codecov-io edited a comment on issue #4177: Enhance gRPC plugin

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #4177: Enhance gRPC plugin
URL: https://github.com/apache/skywalking/pull/4177#issuecomment-575098453
 
 
   # [Codecov](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=h1) Report
   > :exclamation: No coverage uploaded for pull request base (`master@5ca626e`). [Click here to learn what that means](https://docs.codecov.io/docs/error-reference#section-missing-base-commit).
   > The diff coverage is `16.56%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/skywalking/pull/4177/graphs/tree.svg?width=650&token=qrILxY5yA8&height=150&src=pr)](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff            @@
   ##             master    #4177   +/-   ##
   =========================================
     Coverage          ?   26.75%           
   =========================================
     Files             ?     1162           
     Lines             ?    25456           
     Branches          ?     3632           
   =========================================
     Hits              ?     6810           
     Misses            ?    18041           
     Partials          ?      605
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [...ing/oap/server/core/source/DefaultScopeDefine.java](https://codecov.io/gh/apache/skywalking/pull/4177/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvc2t5d2Fsa2luZy9vYXAvc2VydmVyL2NvcmUvc291cmNlL0RlZmF1bHRTY29wZURlZmluZS5qYXZh) | `0% <ø> (ø)` | |
   | [...erver/receiver/jvm/provider/JVMModuleProvider.java](https://codecov.io/gh/apache/skywalking/pull/4177/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItcmVjZWl2ZXItcGx1Z2luL3NreXdhbGtpbmctanZtLXJlY2VpdmVyLXBsdWdpbi9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvc2t5d2Fsa2luZy9vYXAvc2VydmVyL3JlY2VpdmVyL2p2bS9wcm92aWRlci9KVk1Nb2R1bGVQcm92aWRlci5qYXZh) | `0% <ø> (ø)` | |
   | [...alking/apm/agent/core/context/trace/EntrySpan.java](https://codecov.io/gh/apache/skywalking/pull/4177/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLWFnZW50LWNvcmUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvYXBtL2FnZW50L2NvcmUvY29udGV4dC90cmFjZS9FbnRyeVNwYW4uamF2YQ==) | `54.83% <ø> (ø)` | |
   | [...lking/oap/server/core/query/TraceQueryService.java](https://codecov.io/gh/apache/skywalking/pull/4177/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvc2t5d2Fsa2luZy9vYXAvc2VydmVyL2NvcmUvcXVlcnkvVHJhY2VRdWVyeVNlcnZpY2UuamF2YQ==) | `0% <ø> (ø)` | |
   | [...walking/apm/agent/core/context/trace/ExitSpan.java](https://codecov.io/gh/apache/skywalking/pull/4177/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLWFnZW50LWNvcmUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvYXBtL2FnZW50L2NvcmUvY29udGV4dC90cmFjZS9FeGl0U3Bhbi5qYXZh) | `53.57% <ø> (ø)` | |
   | [.../provider/parser/SegmentParserListenerManager.java](https://codecov.io/gh/apache/skywalking/pull/4177/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItcmVjZWl2ZXItcGx1Z2luL3NreXdhbGtpbmctdHJhY2UtcmVjZWl2ZXItcGx1Z2luL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9za3l3YWxraW5nL29hcC9zZXJ2ZXIvcmVjZWl2ZXIvdHJhY2UvcHJvdmlkZXIvcGFyc2VyL1NlZ21lbnRQYXJzZXJMaXN0ZW5lck1hbmFnZXIuamF2YQ==) | `0% <ø> (ø)` | |
   | [...alking/apm/agent/core/context/trace/LocalSpan.java](https://codecov.io/gh/apache/skywalking/pull/4177/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLWFnZW50LWNvcmUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvYXBtL2FnZW50L2NvcmUvY29udGV4dC90cmFjZS9Mb2NhbFNwYW4uamF2YQ==) | `0% <ø> (ø)` | |
   | [...iver/register/provider/RegisterModuleProvider.java](https://codecov.io/gh/apache/skywalking/pull/4177/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItcmVjZWl2ZXItcGx1Z2luL3NreXdhbGtpbmctcmVnaXN0ZXItcmVjZWl2ZXItcGx1Z2luL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9za3l3YWxraW5nL29hcC9zZXJ2ZXIvcmVjZWl2ZXIvcmVnaXN0ZXIvcHJvdmlkZXIvUmVnaXN0ZXJNb2R1bGVQcm92aWRlci5qYXZh) | `0% <ø> (ø)` | |
   | [.../apm/agent/core/profile/TracingThreadSnapshot.java](https://codecov.io/gh/apache/skywalking/pull/4177/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLWFnZW50LWNvcmUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvYXBtL2FnZW50L2NvcmUvcHJvZmlsZS9UcmFjaW5nVGhyZWFkU25hcHNob3QuamF2YQ==) | `0% <0%> (ø)` | |
   | [...alking/oap/server/core/command/CommandService.java](https://codecov.io/gh/apache/skywalking/pull/4177/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvc2t5d2Fsa2luZy9vYXAvc2VydmVyL2NvcmUvY29tbWFuZC9Db21tYW5kU2VydmljZS5qYXZh) | `0% <0%> (ø)` | |
   | ... and [29 more](https://codecov.io/gh/apache/skywalking/pull/4177/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=footer). Last update [5ca626e...93cdc2a](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [skywalking] devkanro commented on a change in pull request #4177: Enhance gRPC plugin

Posted by GitBox <gi...@apache.org>.
devkanro commented on a change in pull request #4177: Enhance gRPC plugin
URL: https://github.com/apache/skywalking/pull/4177#discussion_r371403331
 
 

 ##########
 File path: apm-sniffer/apm-sdk-plugin/grpc-1.x-plugin/src/main/java/org/apache/skywalking/apm/plugin/grpc/v1/server/TracingServerCall.java
 ##########
 @@ -0,0 +1,111 @@
+/*
+ * 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.apm.plugin.grpc.v1.server;
+
+import io.grpc.ForwardingServerCall;
+import io.grpc.Metadata;
+import io.grpc.ServerCall;
+import io.grpc.Status;
+import org.apache.skywalking.apm.agent.core.context.ContextManager;
+import org.apache.skywalking.apm.agent.core.context.ContextSnapshot;
+import org.apache.skywalking.apm.agent.core.context.tag.Tags;
+import org.apache.skywalking.apm.agent.core.context.trace.AbstractSpan;
+import org.apache.skywalking.apm.agent.core.context.trace.SpanLayer;
+import org.apache.skywalking.apm.network.trace.component.ComponentsDefine;
+import org.apache.skywalking.apm.plugin.grpc.v1.OperationNameFormatUtil;
+
+import static org.apache.skywalking.apm.plugin.grpc.v1.Constants.*;
+
+/**
+ * @author wang zheng, kanro
+ */
+public class TracingServerCall<REQUEST, RESPONSE> extends ForwardingServerCall.SimpleForwardingServerCall<REQUEST, RESPONSE> {
+
+    private final String operationPrefix;
+    private final ContextSnapshot contextSnapshot;
+
+    protected TracingServerCall(ServerCall<REQUEST, RESPONSE> delegate, ContextSnapshot contextSnapshot) {
+        super(delegate);
+        this.operationPrefix = OperationNameFormatUtil.formatOperationName(delegate.getMethodDescriptor()) + SERVER;
+        this.contextSnapshot = contextSnapshot;
+    }
+
+    @Override
+    public void sendMessage(RESPONSE message) {
+        // We just create the request on message span for server stream calls.
+        if (!getMethodDescriptor().getType().serverSendsOneMessage()) {
+            try {
+                final AbstractSpan span = ContextManager.createLocalSpan(operationPrefix + RESPONSE_ON_MESSAGE_OPERATION_NAME);
+                span.setComponent(ComponentsDefine.GRPC);
+                span.setLayer(SpanLayer.RPC_FRAMEWORK);
+                ContextManager.continued(contextSnapshot);
+                super.sendMessage(message);
+            } catch (Throwable t) {
+                ContextManager.activeSpan().errorOccurred().log(t);
+                throw t;
+            } finally {
+                ContextManager.stopSpan();
+            }
+        } else {
+            super.sendMessage(message);
+        }
+    }
+
+    @Override
+    public void close(Status status, Metadata trailers) {
+        try {
+            final AbstractSpan span = ContextManager.createLocalSpan(operationPrefix + RESPONSE_ON_CLOSE_OPERATION_NAME);
+            span.setComponent(ComponentsDefine.GRPC);
+            span.setLayer(SpanLayer.RPC_FRAMEWORK);
+            ContextManager.continued(contextSnapshot);
+
+            switch (status.getCode()) {
+                case OK:
+                    break;
+                // UNKNOWN/INTERNAL status code will case error in this span.
+                // Those status code means some unexpected error occurred in server.
+                // Similar to 5XX in HTTP status.
+                case UNKNOWN:
+                case INTERNAL:
+                    if (status.getCause() == null) {
+                        span.errorOccurred().log(status.asRuntimeException());
+                    } else {
+                        span.errorOccurred().log(status.getCause());
+                    }
+                    break;
+                // Other status code means some predictable error occurred in server.
+                // Like PERMISSION_DENIED or UNAUTHENTICATED somethings.
+                // Similar to 4XX in HTTP status.
+                default:
 
 Review comment:
   `UNKNOWN` and `INTERNAL` is similar to the `5xx` errors in HTTP, it must case span error even status has no cause exception.
   
   `default`(others) is similar to the `4xx` errors in HTTP, we just case error when status has cause exception for user debugging.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [skywalking] GuoDuanLZ commented on a change in pull request #4177: Enhance gRPC plugin

Posted by GitBox <gi...@apache.org>.
GuoDuanLZ commented on a change in pull request #4177: Enhance gRPC plugin
URL: https://github.com/apache/skywalking/pull/4177#discussion_r371717507
 
 

 ##########
 File path: test/plugin/scenarios/grpc-scenario/config/expectedData.yaml
 ##########
 @@ -15,239 +15,524 @@
 # limitations under the License.
 registryItems:
   applications:
-    - {grpc-scenario: 2}
+  - { grpc-scenario: 2 }
   instances:
-    - {grpc-scenario: 1}
+  - { grpc-scenario: 1 }
   operationNames:
-    - grpc-scenario: [Greeter.sayHello, GreeterBlocking.sayHello,
-                      /grpc-scenario/case/grpc-scenario]
+  - grpc-scenario: [Greeter.sayHello, GreeterBlocking.sayHello, GreeterBlockingError.sayHello, /grpc-scenario/case/grpc-scenario]
   heartbeat: []
 segmentItems:
-  - applicationCode: grpc-scenario
-    segmentSize: gt 10
-    segments:
-      - segmentId: not null
-        spans:
-          - operationName: GreeterBlocking.sayHello
-            operationId: 0
-            parentSpanId: -1
-            spanId: 0
-            spanLayer: RPCFramework
-            startTime: nq 0
-            endTime: nq 0
-            componentId: 23
-            componentName: ''
-            isError: false
-            spanType: Entry
-            peer: ''
-            peerId: 0
-            refs:
-              - {parentEndpointId: 0, parentEndpoint: /grpc-scenario/case/grpc-scenario, networkAddressId: 0,
-                 entryEndpointId: 0, refType: CrossProcess, parentSpanId: 2, parentTraceSegmentId: not null,
-                 parentServiceInstanceId: 1, networkAddress: '127.0.0.1:18080', entryEndpoint: /grpc-scenario/case/grpc-scenario,
-                 entryServiceInstanceId: 1}
-      - segmentId: not null
-        spans:
-          - operationName: Greeter.sayHello
-            operationId: 0
-            parentSpanId: -1
-            spanId: 0
-            spanLayer: RPCFramework
-            startTime: nq 0
-            endTime: nq 0
-            componentId: 23
-            componentName: ''
-            isError: false
-            spanType: Entry
-            peer: ''
-            peerId: 0
-            refs:
-              - {parentEndpointId: 0, parentEndpoint: /grpc-scenario/case/grpc-scenario, networkAddressId: 0,
-                 entryEndpointId: 0, refType: CrossProcess, parentSpanId: 1, parentTraceSegmentId: not null,
-                 parentServiceInstanceId: 1, networkAddress: '127.0.0.1:18080', entryEndpoint: /grpc-scenario/case/grpc-scenario,
-                 entryServiceInstanceId: 1}
-      - segmentId: not null
-        spans:
-          - operationName: GreeterBlocking.sayHello/server/RequestObserver/onNext
-            operationId: 0
-            parentSpanId: -1
-            spanId: 0
-            spanLayer: Unknown
-            startTime: nq 0
-            endTime: nq 0
-            componentId: 0
-            componentName: ''
-            isError: false
-            spanType: Local
-            peer: ''
-            peerId: 0
-            refs:
-              - {parentEndpointId: 0, parentEndpoint: GreeterBlocking.sayHello, networkAddressId: 0,
-                 entryEndpointId: 0, refType: CrossThread, parentSpanId: 0, parentTraceSegmentId: not null,
-                 parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
-                 entryServiceInstanceId: 1}
-      - segmentId: not null
-        spans:
-          - operationName: Greeter.sayHello/server/RequestObserver/onNext
-            operationId: 0
-            parentSpanId: -1
-            spanId: 0
-            spanLayer: Unknown
-            startTime: nq 0
-            endTime: nq 0
-            componentId: 0
-            componentName: ''
-            isError: false
-            spanType: Local
-            peer: ''
-            peerId: 0
-            refs:
-              - {parentEndpointId: 0, parentEndpoint: Greeter.sayHello, networkAddressId: 0,
-                 entryEndpointId: 0, refType: CrossThread, parentSpanId: 0, parentTraceSegmentId: not null,
-                 parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
-                 entryServiceInstanceId: 1}
-      - segmentId: not null
-        spans:
-          - operationName: Greeter.sayHello/client/ResponseObserver/onNext
-            operationId: 0
-            parentSpanId: -1
-            spanId: 0
-            spanLayer: Unknown
-            startTime: nq 0
-            endTime: nq 0
-            componentId: 0
-            componentName: ''
-            isError: false
-            spanType: Local
-            peer: ''
-            peerId: 0
-            refs:
-              - {parentEndpointId: 0, parentEndpoint: /grpc-scenario/case/grpc-scenario, networkAddressId: 0,
-                 entryEndpointId: 0, refType: CrossThread, parentSpanId: 1, parentTraceSegmentId: not null,
-                 parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
-                 entryServiceInstanceId: 1}
-      - segmentId: not null
-        spans:
-          - operationName: Greeter.sayHello/client/ResponseObserver/onComplete
-            operationId: 0
-            parentSpanId: -1
-            spanId: 0
-            spanLayer: Unknown
-            startTime: nq 0
-            endTime: nq 0
-            componentId: 0
-            componentName: ''
-            isError: false
-            spanType: Local
-            peer: ''
-            peerId: 0
-            refs:
-              - {parentEndpointId: 0, parentEndpoint: /grpc-scenario/case/grpc-scenario, networkAddressId: 0,
-                 entryEndpointId: 0, refType: CrossThread, parentSpanId: 1, parentTraceSegmentId: not null,
-                 parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
-                 entryServiceInstanceId: 1}
-      - segmentId: not null
-        spans:
-          - operationName: Greeter.sayHello/server/RequestObserver/onNext
-            operationId: 0
-            parentSpanId: -1
-            spanId: 0
-            spanLayer: Unknown
-            startTime: nq 0
-            endTime: nq 0
-            componentId: 0
-            componentName: ''
-            isError: false
-            spanType: Local
-            peer: ''
-            peerId: 0
-            refs:
-              - {parentEndpointId: 0, parentEndpoint: Greeter.sayHello, networkAddressId: 0,
-                 entryEndpointId: 0, refType: CrossThread, parentSpanId: 0, parentTraceSegmentId: not null,
-                 parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
-                 entryServiceInstanceId: 1}
-      - segmentId: not null
-        spans:
-          - operationName: Greeter.sayHello/server/RequestObserver/onComplete
-            operationId: 0
-            parentSpanId: -1
-            spanId: 0
-            spanLayer: Unknown
-            startTime: nq 0
-            endTime: nq 0
-            componentId: 0
-            componentName: ''
-            isError: false
-            spanType: Local
-            peer: ''
-            peerId: 0
-            refs:
-              - {parentEndpointId: 0, parentEndpoint: Greeter.sayHello, networkAddressId: 0,
-                 entryEndpointId: 0, refType: CrossThread, parentSpanId: 0, parentTraceSegmentId: not null,
-                 parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
-                 entryServiceInstanceId: 1}
-      - segmentId: not null
-        spans:
-          - operationName: Greeter.sayHello
-            operationId: 0
-            parentSpanId: 0
-            spanId: 1
-            spanLayer: RPCFramework
-            startTime: nq 0
-            endTime: nq 0
-            componentId: 23
-            componentName: ''
-            isError: false
-            spanType: Exit
-            peer: '127.0.0.1:18080'
-            peerId: 0
-          - operationName: GreeterBlocking.sayHello
-            operationId: 0
-            parentSpanId: 0
-            spanId: 2
-            spanLayer: RPCFramework
-            startTime: nq 0
-            endTime: nq 0
-            componentId: 23
-            componentName: ''
-            isError: false
-            spanType: Exit
-            peer: '127.0.0.1:18080'
-            peerId: 0
-          - operationName: /grpc-scenario/case/grpc-scenario
-            operationId: 0
-            parentSpanId: -1
-            spanId: 0
-            spanLayer: Http
-            startTime: nq 0
-            endTime: nq 0
-            componentId: 1
-            componentName: ''
-            isError: false
-            spanType: Entry
-            peer: ''
-            peerId: 0
-            tags:
-              - {key: url, value: 'http://localhost:8080/grpc-scenario/case/grpc-scenario'}
-              - {key: http.method, value: GET}
-      - segmentId: not null
-        spans:
-          - operationName: Greeter.sayHello/client/ResponseObserver/onNext
-            operationId: 0
-            parentSpanId: -1
-            spanId: 0
-            spanLayer: Unknown
-            startTime: nq 0
-            endTime: nq 0
-            componentId: 0
-            componentName: ''
-            isError: false
-            spanType: Local
-            peer: ''
-            peerId: 0
-            refs:
-              - {parentEndpointId: 0, parentEndpoint: /grpc-scenario/case/grpc-scenario, networkAddressId: 0,
-                 entryEndpointId: 0, refType: CrossThread, parentSpanId: 1, parentTraceSegmentId: not null,
-                 parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
-                 entryServiceInstanceId: 1}
-
+- applicationCode: grpc-scenario
+  segmentSize: gt 10
+  segments:
+  - segmentId: not null
+    spans:
+    - operationName: GreeterBlocking.sayHello
+      operationId: 0
+      parentSpanId: -1
+      spanId: 0
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Entry
+      peer: ''
+      peerId: 0
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: /grpc-scenario/case/grpc-scenario, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossProcess, parentSpanId: 2, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '127.0.0.1:18080', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+  - segmentId: not null
+    spans:
+    - operationName: Greeter.sayHello/client/Request/onMessage
+      operationId: 0
+      parentSpanId: -1
+      spanId: 0
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Local
+      peer: ''
+      peerId: 0
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: /grpc-scenario/case/grpc-scenario, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossThread, parentSpanId: 1, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+  - segmentId: not null
+    spans:
+    - operationName: Greeter.sayHello/client/Request/onMessage
+      operationId: 0
+      parentSpanId: -1
+      spanId: 0
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Local
+      peer: ''
+      peerId: 0
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: /grpc-scenario/case/grpc-scenario, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossThread, parentSpanId: 1, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+  - segmentId: not null
+    spans:
+    - operationName: Greeter.sayHello/client/Request/onComplete
+      operationId: 0
+      parentSpanId: -1
+      spanId: 0
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Local
+      peer: ''
+      peerId: 0
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: /grpc-scenario/case/grpc-scenario, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossThread, parentSpanId: 1, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+  - segmentId: not null
+    spans:
+    - operationName: Greeter.sayHello
+      operationId: 0
+      parentSpanId: -1
+      spanId: 0
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Entry
+      peer: ''
+      peerId: 0
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: /grpc-scenario/case/grpc-scenario, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossProcess, parentSpanId: 1, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '127.0.0.1:18080', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+  - segmentId: not null
+    spans:
+    - operationName: Greeter.sayHello/server/Response/onMessage
+      operationId: 0
+      parentSpanId: 0
+      spanId: 1
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Local
+      peer: ''
+      peerId: 0
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: Greeter.sayHello, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossThread, parentSpanId: 0, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+    - operationName: Greeter.sayHello/server/Request/onMessage
+      operationId: 0
+      parentSpanId: -1
+      spanId: 0
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Local
+      peer: ''
+      peerId: 0
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: Greeter.sayHello, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossThread, parentSpanId: 0, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+  - segmentId: not null
+    spans:
+    - operationName: Greeter.sayHello/client/Response/onMessage
+      operationId: 0
+      parentSpanId: -1
+      spanId: 0
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Local
+      peer: ''
+      peerId: 0
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: /grpc-scenario/case/grpc-scenario, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossThread, parentSpanId: 1, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+  - segmentId: not null
+    spans:
+    - operationName: GreeterBlocking.sayHello/server/Response/onClose
+      operationId: 0
+      parentSpanId: 0
+      spanId: 1
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Local
+      peer: ''
+      peerId: 0
+      tags:
+      - { key: status_code, value: OK }
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: GreeterBlocking.sayHello, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossThread, parentSpanId: 0, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+    - operationName: GreeterBlocking.sayHello/server/Request/onComplete
+      operationId: 0
+      parentSpanId: -1
+      spanId: 0
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Local
+      peer: ''
+      peerId: 0
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: GreeterBlocking.sayHello, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossThread, parentSpanId: 0, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+  - segmentId: not null
+    spans:
+    - operationName: Greeter.sayHello/server/Response/onMessage
+      operationId: 0
+      parentSpanId: 0
+      spanId: 1
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Local
+      peer: ''
+      peerId: 0
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: Greeter.sayHello, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossThread, parentSpanId: 0, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+    - operationName: Greeter.sayHello/server/Request/onMessage
+      operationId: 0
+      parentSpanId: -1
+      spanId: 0
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Local
+      peer: ''
+      peerId: 0
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: Greeter.sayHello, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossThread, parentSpanId: 0, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+  - segmentId: not null
+    spans:
+    - operationName: Greeter.sayHello/server/Response/onClose
+      operationId: 0
+      parentSpanId: 0
+      spanId: 1
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Local
+      peer: ''
+      peerId: 0
+      tags:
+      - {key: status_code, value: OK}
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: Greeter.sayHello, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossThread, parentSpanId: 0, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+    - operationName: Greeter.sayHello/server/Request/onComplete
+      operationId: 0
+      parentSpanId: -1
+      spanId: 0
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Local
+      peer: ''
+      peerId: 0
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: Greeter.sayHello, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossThread, parentSpanId: 0, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+  - segmentId: not null
+    spans:
+    - operationName: Greeter.sayHello/client/Response/onMessage
+      operationId: 0
+      parentSpanId: -1
+      spanId: 0
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Local
+      peer: ''
+      peerId: 0
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: /grpc-scenario/case/grpc-scenario, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossThread, parentSpanId: 1, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+  - segmentId: not null
+    spans:
+    - operationName: Greeter.sayHello/client/Response/onClose
+      operationId: 0
+      parentSpanId: -1
+      spanId: 0
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Local
+      peer: ''
+      peerId: 0
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: /grpc-scenario/case/grpc-scenario, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossThread, parentSpanId: 1, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+  - segmentId: not null
+    spans:
+    - operationName: GreeterBlockingError.sayHello
+      operationId: 0
+      parentSpanId: -1
+      spanId: 0
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Entry
+      peer: ''
+      peerId: 0
+      refs:
+        - {parentEndpointId: 0, parentEndpoint: /grpc-scenario/case/grpc-scenario, networkAddressId: 0,
+           entryEndpointId: 0, refType: CrossProcess, parentSpanId: 5, parentTraceSegmentId: not null,
+           parentServiceInstanceId: 1, networkAddress: '127.0.0.1:18080', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+           entryServiceInstanceId: 1}
+  - segmentId: not null
+    spans:
+    - operationName: GreeterBlockingError.sayHello/server/Response/onClose
+      operationId: 0
+      parentSpanId: 0
+      spanId: 1
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: true
+      spanType: Local
+      peer: ''
+      peerId: 0
+      tags:
+      - { key: status_code, value: UNKNOWN }
+      logs:
+      - logEvent:
+        - { key: event, value: error }
+        - { key: error.kind, value: java.lang.Exception }
+        - { key: message, value: '' }
+        - key: stack
+          value: not null
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: GreeterBlockingError.sayHello, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossThread, parentSpanId: 0, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+    - operationName: GreeterBlockingError.sayHello/server/Request/onComplete
+      operationId: 0
+      parentSpanId: -1
+      spanId: 0
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Local
+      peer: ''
+      peerId: 0
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: GreeterBlockingError.sayHello, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossThread, parentSpanId: 0, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+  - segmentId: not null
+    spans:
+    - operationName: Greeter.sayHello
+      operationId: 0
+      parentSpanId: 0
+      spanId: 1
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Exit
+      peer: '127.0.0.1:18080'
+      peerId: 0
+    - {operationName: GreeterBlocking.sayHello/client/Request/onComplete, operationId: 0,
+       parentSpanId: 2, spanId: 3, spanLayer: RPCFramework, startTime: nq 0,
+       endTime: nq 0, componentId: 23, componentName: '', isError: false,
+       spanType: Local, peer: '', peerId: 0}
+    - {operationName: GreeterBlocking.sayHello/client/Response/onClose, operationId: 0,
 
 Review comment:
   When the span is shorter, it is a simplified format.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [skywalking] codecov-io edited a comment on issue #4177: Enhance gRPC plugin

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #4177: Enhance gRPC plugin
URL: https://github.com/apache/skywalking/pull/4177#issuecomment-570947001
 
 
   # [Codecov](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=h1) Report
   > Merging [#4177](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=desc) into [master](https://codecov.io/gh/apache/skywalking/commit/c6b169544e199fdf6d8e27e58b8f98897c3e7ef7?src=pr&el=desc) will **decrease** coverage by `0.1%`.
   > The diff coverage is `0%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/skywalking/pull/4177/graphs/tree.svg?width=650&token=qrILxY5yA8&height=150&src=pr)](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #4177      +/-   ##
   ==========================================
   - Coverage   26.81%   26.71%   -0.11%     
   ==========================================
     Files        1163     1164       +1     
     Lines       25431    25529      +98     
     Branches     3694     3705      +11     
   ==========================================
     Hits         6819     6819              
   - Misses      18000    18098      +98     
     Partials      612      612
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [...ng/apm/plugin/grpc/v1/OperationNameFormatUtil.java](https://codecov.io/gh/apache/skywalking/pull/4177/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLXNkay1wbHVnaW4vZ3JwYy0xLngtcGx1Z2luL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9za3l3YWxraW5nL2FwbS9wbHVnaW4vZ3JwYy92MS9PcGVyYXRpb25OYW1lRm9ybWF0VXRpbC5qYXZh) | `0% <ø> (ø)` | :arrow_up: |
   | [...plugin/grpc/v1/client/BlockingCallInterceptor.java](https://codecov.io/gh/apache/skywalking/pull/4177/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLXNkay1wbHVnaW4vZ3JwYy0xLngtcGx1Z2luL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9za3l3YWxraW5nL2FwbS9wbHVnaW4vZ3JwYy92MS9jbGllbnQvQmxvY2tpbmdDYWxsSW50ZXJjZXB0b3IuamF2YQ==) | `0% <0%> (ø)` | |
   | [...g/apm/plugin/grpc/v1/client/TracingClientCall.java](https://codecov.io/gh/apache/skywalking/pull/4177/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLXNkay1wbHVnaW4vZ3JwYy0xLngtcGx1Z2luL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9za3l3YWxraW5nL2FwbS9wbHVnaW4vZ3JwYy92MS9jbGllbnQvVHJhY2luZ0NsaWVudENhbGwuamF2YQ==) | `0% <0%> (ø)` | |
   | [...g/apm/plugin/grpc/v1/server/ServerInterceptor.java](https://codecov.io/gh/apache/skywalking/pull/4177/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLXNkay1wbHVnaW4vZ3JwYy0xLngtcGx1Z2luL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9za3l3YWxraW5nL2FwbS9wbHVnaW4vZ3JwYy92MS9zZXJ2ZXIvU2VydmVySW50ZXJjZXB0b3IuamF2YQ==) | `0% <0%> (ø)` | |
   | [...g/apm/plugin/grpc/v1/server/TracingServerCall.java](https://codecov.io/gh/apache/skywalking/pull/4177/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLXNkay1wbHVnaW4vZ3JwYy0xLngtcGx1Z2luL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9za3l3YWxraW5nL2FwbS9wbHVnaW4vZ3JwYy92MS9zZXJ2ZXIvVHJhY2luZ1NlcnZlckNhbGwuamF2YQ==) | `0% <0%> (ø)` | |
   | [...plugin/grpc/v1/client/AbstractStubInterceptor.java](https://codecov.io/gh/apache/skywalking/pull/4177/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLXNkay1wbHVnaW4vZ3JwYy0xLngtcGx1Z2luL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9za3l3YWxraW5nL2FwbS9wbHVnaW4vZ3JwYy92MS9jbGllbnQvQWJzdHJhY3RTdHViSW50ZXJjZXB0b3IuamF2YQ==) | `0% <0%> (ø)` | |
   | [...ugin/grpc/v1/server/TracingServerCallListener.java](https://codecov.io/gh/apache/skywalking/pull/4177/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLXNkay1wbHVnaW4vZ3JwYy0xLngtcGx1Z2luL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9za3l3YWxraW5nL2FwbS9wbHVnaW4vZ3JwYy92MS9zZXJ2ZXIvVHJhY2luZ1NlcnZlckNhbGxMaXN0ZW5lci5qYXZh) | `0% <0%> (ø)` | |
   | [...g/apm/plugin/grpc/v1/client/ClientInterceptor.java](https://codecov.io/gh/apache/skywalking/pull/4177/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLXNkay1wbHVnaW4vZ3JwYy0xLngtcGx1Z2luL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9za3l3YWxraW5nL2FwbS9wbHVnaW4vZ3JwYy92MS9jbGllbnQvQ2xpZW50SW50ZXJjZXB0b3IuamF2YQ==) | `0% <0%> (ø)` | |
   | [.../apache/skywalking/apm/agent/core/conf/Config.java](https://codecov.io/gh/apache/skywalking/pull/4177/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLWFnZW50LWNvcmUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvYXBtL2FnZW50L2NvcmUvY29uZi9Db25maWcuamF2YQ==) | `62.96% <0%> (-3.71%)` | :arrow_down: |
   | [...plugin/grpc/v1/client/SimpleTracingClientCall.java](https://codecov.io/gh/apache/skywalking/pull/4177/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLXNkay1wbHVnaW4vZ3JwYy0xLngtcGx1Z2luL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9za3l3YWxraW5nL2FwbS9wbHVnaW4vZ3JwYy92MS9jbGllbnQvU2ltcGxlVHJhY2luZ0NsaWVudENhbGwuamF2YQ==) | `0% <0%> (ø)` | |
   | ... and [7 more](https://codecov.io/gh/apache/skywalking/pull/4177/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=footer). Last update [c6b1695...40cbd77](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [skywalking] wu-sheng commented on a change in pull request #4177: Enhance gRPC plugin

Posted by GitBox <gi...@apache.org>.
wu-sheng commented on a change in pull request #4177: Enhance gRPC plugin
URL: https://github.com/apache/skywalking/pull/4177#discussion_r371099766
 
 

 ##########
 File path: apm-sniffer/apm-sdk-plugin/grpc-1.x-plugin/src/main/java/org/apache/skywalking/apm/plugin/grpc/v1/client/TracingClientCall.java
 ##########
 @@ -0,0 +1,201 @@
+/*
+ * 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.apm.plugin.grpc.v1.client;
+
+import io.grpc.*;
+import org.apache.skywalking.apm.agent.core.context.CarrierItem;
+import org.apache.skywalking.apm.agent.core.context.ContextCarrier;
+import org.apache.skywalking.apm.agent.core.context.ContextManager;
+import org.apache.skywalking.apm.agent.core.context.ContextSnapshot;
+import org.apache.skywalking.apm.agent.core.context.tag.Tags;
+import org.apache.skywalking.apm.agent.core.context.trace.AbstractSpan;
+import org.apache.skywalking.apm.agent.core.context.trace.SpanLayer;
+import org.apache.skywalking.apm.network.trace.component.ComponentsDefine;
+import org.apache.skywalking.apm.plugin.grpc.v1.OperationNameFormatUtil;
+
+import javax.annotation.Nullable;
+
+import static org.apache.skywalking.apm.plugin.grpc.v1.Constants.*;
+import static org.apache.skywalking.apm.plugin.grpc.v1.OperationNameFormatUtil.formatOperationName;
+
+/**
+ * Fully client tracing for external gRPC server which not be registered in the same Skywalking server.
 
 Review comment:
   This comment seems not right. Do you miss the changes of old version?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [skywalking] codecov-io edited a comment on issue #4177: Enhance gRPC plugin

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #4177: Enhance gRPC plugin
URL: https://github.com/apache/skywalking/pull/4177#issuecomment-575098453
 
 
   # [Codecov](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=h1) Report
   > Merging [#4177](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=desc) into [master](https://codecov.io/gh/apache/skywalking/commit/d0ef0dff770e4342ad98a643eb19b9ccd5a5fb69?src=pr&el=desc) will **decrease** coverage by `0.01%`.
   > The diff coverage is `0%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/skywalking/pull/4177/graphs/tree.svg?width=650&token=qrILxY5yA8&height=150&src=pr)](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #4177      +/-   ##
   ==========================================
   - Coverage   26.74%   26.73%   -0.02%     
   ==========================================
     Files        1162     1162              
     Lines       25446    25448       +2     
     Branches     3630     3630              
   ==========================================
   - Hits         6806     6803       -3     
   - Misses      18034    18041       +7     
   + Partials      606      604       -2
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [...g/apm/plugin/grpc/v1/client/TracingClientCall.java](https://codecov.io/gh/apache/skywalking/pull/4177/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLXNkay1wbHVnaW4vZ3JwYy0xLngtcGx1Z2luL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9za3l3YWxraW5nL2FwbS9wbHVnaW4vZ3JwYy92MS9jbGllbnQvVHJhY2luZ0NsaWVudENhbGwuamF2YQ==) | `0% <0%> (ø)` | :arrow_up: |
   | [...ugin/grpc/v1/server/TracingServerCallListener.java](https://codecov.io/gh/apache/skywalking/pull/4177/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLXNkay1wbHVnaW4vZ3JwYy0xLngtcGx1Z2luL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9za3l3YWxraW5nL2FwbS9wbHVnaW4vZ3JwYy92MS9zZXJ2ZXIvVHJhY2luZ1NlcnZlckNhbGxMaXN0ZW5lci5qYXZh) | `0% <0%> (ø)` | :arrow_up: |
   | [...g/apm/plugin/grpc/v1/server/TracingServerCall.java](https://codecov.io/gh/apache/skywalking/pull/4177/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLXNkay1wbHVnaW4vZ3JwYy0xLngtcGx1Z2luL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9za3l3YWxraW5nL2FwbS9wbHVnaW4vZ3JwYy92MS9zZXJ2ZXIvVHJhY2luZ1NlcnZlckNhbGwuamF2YQ==) | `0% <0%> (ø)` | :arrow_up: |
   | [.../core/remote/ServiceAndEndpointRegisterClient.java](https://codecov.io/gh/apache/skywalking/pull/4177/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLWFnZW50LWNvcmUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvYXBtL2FnZW50L2NvcmUvcmVtb3RlL1NlcnZpY2VBbmRFbmRwb2ludFJlZ2lzdGVyQ2xpZW50LmphdmE=) | `28.08% <0%> (-3.38%)` | :arrow_down: |
   | [...m/agent/core/remote/TraceSegmentServiceClient.java](https://codecov.io/gh/apache/skywalking/pull/4177/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLWFnZW50LWNvcmUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvYXBtL2FnZW50L2NvcmUvcmVtb3RlL1RyYWNlU2VnbWVudFNlcnZpY2VDbGllbnQuamF2YQ==) | `80.88% <0%> (-1.48%)` | :arrow_down: |
   | [...ache/skywalking/apm/agent/core/jvm/JVMService.java](https://codecov.io/gh/apache/skywalking/pull/4177/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLWFnZW50LWNvcmUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvYXBtL2FnZW50L2NvcmUvanZtL0pWTVNlcnZpY2UuamF2YQ==) | `59.01% <0%> (ø)` | :arrow_up: |
   | [...walking/apm/agent/core/context/TracingContext.java](https://codecov.io/gh/apache/skywalking/pull/4177/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLWFnZW50LWNvcmUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvYXBtL2FnZW50L2NvcmUvY29udGV4dC9UcmFjaW5nQ29udGV4dC5qYXZh) | `61.33% <0%> (+0.44%)` | :arrow_up: |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=footer). Last update [d0ef0df...3d11d83](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [skywalking] codecov-io edited a comment on issue #4177: Enhance gRPC plugin

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #4177: Enhance gRPC plugin
URL: https://github.com/apache/skywalking/pull/4177#issuecomment-570947001
 
 
   # [Codecov](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=h1) Report
   > Merging [#4177](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=desc) into [master](https://codecov.io/gh/apache/skywalking/commit/131071964523e26243d7a2db4b3fd30022c93d9b?src=pr&el=desc) will **decrease** coverage by `0.21%`.
   > The diff coverage is `0%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/skywalking/pull/4177/graphs/tree.svg?width=650&token=qrILxY5yA8&height=150&src=pr)](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff            @@
   ##           master   #4177      +/-   ##
   =========================================
   - Coverage   26.62%   26.4%   -0.22%     
   =========================================
     Files        1180    1181       +1     
     Lines       25859   25957      +98     
     Branches     3763    3774      +11     
   =========================================
   - Hits         6884    6855      -29     
   - Misses      18367   18498     +131     
   + Partials      608     604       -4
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [...ng/apm/plugin/grpc/v1/OperationNameFormatUtil.java](https://codecov.io/gh/apache/skywalking/pull/4177/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLXNkay1wbHVnaW4vZ3JwYy0xLngtcGx1Z2luL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9za3l3YWxraW5nL2FwbS9wbHVnaW4vZ3JwYy92MS9PcGVyYXRpb25OYW1lRm9ybWF0VXRpbC5qYXZh) | `0% <ø> (ø)` | :arrow_up: |
   | [...plugin/grpc/v1/client/BlockingCallInterceptor.java](https://codecov.io/gh/apache/skywalking/pull/4177/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLXNkay1wbHVnaW4vZ3JwYy0xLngtcGx1Z2luL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9za3l3YWxraW5nL2FwbS9wbHVnaW4vZ3JwYy92MS9jbGllbnQvQmxvY2tpbmdDYWxsSW50ZXJjZXB0b3IuamF2YQ==) | `0% <0%> (ø)` | |
   | [...g/apm/plugin/grpc/v1/server/ServerInterceptor.java](https://codecov.io/gh/apache/skywalking/pull/4177/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLXNkay1wbHVnaW4vZ3JwYy0xLngtcGx1Z2luL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9za3l3YWxraW5nL2FwbS9wbHVnaW4vZ3JwYy92MS9zZXJ2ZXIvU2VydmVySW50ZXJjZXB0b3IuamF2YQ==) | `0% <0%> (ø)` | |
   | [...g/apm/plugin/grpc/v1/client/TracingClientCall.java](https://codecov.io/gh/apache/skywalking/pull/4177/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLXNkay1wbHVnaW4vZ3JwYy0xLngtcGx1Z2luL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9za3l3YWxraW5nL2FwbS9wbHVnaW4vZ3JwYy92MS9jbGllbnQvVHJhY2luZ0NsaWVudENhbGwuamF2YQ==) | `0% <0%> (ø)` | |
   | [...g/apm/plugin/grpc/v1/server/TracingServerCall.java](https://codecov.io/gh/apache/skywalking/pull/4177/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLXNkay1wbHVnaW4vZ3JwYy0xLngtcGx1Z2luL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9za3l3YWxraW5nL2FwbS9wbHVnaW4vZ3JwYy92MS9zZXJ2ZXIvVHJhY2luZ1NlcnZlckNhbGwuamF2YQ==) | `0% <0%> (ø)` | |
   | [...plugin/grpc/v1/client/AbstractStubInterceptor.java](https://codecov.io/gh/apache/skywalking/pull/4177/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLXNkay1wbHVnaW4vZ3JwYy0xLngtcGx1Z2luL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9za3l3YWxraW5nL2FwbS9wbHVnaW4vZ3JwYy92MS9jbGllbnQvQWJzdHJhY3RTdHViSW50ZXJjZXB0b3IuamF2YQ==) | `0% <0%> (ø)` | |
   | [...plugin/grpc/v1/client/SimpleTracingClientCall.java](https://codecov.io/gh/apache/skywalking/pull/4177/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLXNkay1wbHVnaW4vZ3JwYy0xLngtcGx1Z2luL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9za3l3YWxraW5nL2FwbS9wbHVnaW4vZ3JwYy92MS9jbGllbnQvU2ltcGxlVHJhY2luZ0NsaWVudENhbGwuamF2YQ==) | `0% <0%> (ø)` | |
   | [...ugin/grpc/v1/server/TracingServerCallListener.java](https://codecov.io/gh/apache/skywalking/pull/4177/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLXNkay1wbHVnaW4vZ3JwYy0xLngtcGx1Z2luL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9za3l3YWxraW5nL2FwbS9wbHVnaW4vZ3JwYy92MS9zZXJ2ZXIvVHJhY2luZ1NlcnZlckNhbGxMaXN0ZW5lci5qYXZh) | `0% <0%> (ø)` | |
   | [...1/server/AbstractServerImplBuilderInterceptor.java](https://codecov.io/gh/apache/skywalking/pull/4177/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLXNkay1wbHVnaW4vZ3JwYy0xLngtcGx1Z2luL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9za3l3YWxraW5nL2FwbS9wbHVnaW4vZ3JwYy92MS9zZXJ2ZXIvQWJzdHJhY3RTZXJ2ZXJJbXBsQnVpbGRlckludGVyY2VwdG9yLmphdmE=) | `0% <0%> (ø)` | |
   | [.../apache/skywalking/apm/agent/core/conf/Config.java](https://codecov.io/gh/apache/skywalking/pull/4177/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLWFnZW50LWNvcmUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvYXBtL2FnZW50L2NvcmUvY29uZi9Db25maWcuamF2YQ==) | `64.28% <0%> (-3.64%)` | :arrow_down: |
   | ... and [11 more](https://codecov.io/gh/apache/skywalking/pull/4177/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=footer). Last update [1310719...722f77a](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [skywalking] codecov-io commented on issue #4177: Enhance gRPC plugin

Posted by GitBox <gi...@apache.org>.
codecov-io commented on issue #4177: Enhance gRPC plugin
URL: https://github.com/apache/skywalking/pull/4177#issuecomment-575098453
 
 
   # [Codecov](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=h1) Report
   > Merging [#4177](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=desc) into [master](https://codecov.io/gh/apache/skywalking/commit/5a1d453a53bba15397d708d6558bdf28500320ae?src=pr&el=desc) will **increase** coverage by `<.01%`.
   > The diff coverage is `n/a`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/skywalking/pull/4177/graphs/tree.svg?width=650&token=qrILxY5yA8&height=150&src=pr)](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #4177      +/-   ##
   ==========================================
   + Coverage   26.36%   26.37%   +<.01%     
   ==========================================
     Files        1179     1179              
     Lines       25863    25863              
     Branches     3753     3753              
   ==========================================
   + Hits         6819     6821       +2     
   + Misses      18440    18438       -2     
     Partials      604      604
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [...pm/agent/core/profile/ProfileTaskQueryService.java](https://codecov.io/gh/apache/skywalking/pull/4177/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLWFnZW50LWNvcmUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvYXBtL2FnZW50L2NvcmUvcHJvZmlsZS9Qcm9maWxlVGFza1F1ZXJ5U2VydmljZS5qYXZh) | `48.71% <0%> (+5.12%)` | :arrow_up: |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=footer). Last update [5a1d453...aef265d](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [skywalking] codecov-io edited a comment on issue #4177: Enhance gRPC plugin

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #4177: Enhance gRPC plugin
URL: https://github.com/apache/skywalking/pull/4177#issuecomment-575098453
 
 
   # [Codecov](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=h1) Report
   > Merging [#4177](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=desc) into [master](https://codecov.io/gh/apache/skywalking/commit/23be98441c0aee38119d1beace166be57930fcbf?src=pr&el=desc) will **decrease** coverage by `0.01%`.
   > The diff coverage is `0%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/skywalking/pull/4177/graphs/tree.svg?width=650&token=qrILxY5yA8&height=150&src=pr)](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #4177      +/-   ##
   ==========================================
   - Coverage   26.83%   26.82%   -0.02%     
   ==========================================
     Files        1162     1162              
     Lines       25448    25457       +9     
     Branches     3630     3631       +1     
   ==========================================
   - Hits         6829     6828       -1     
   - Misses      18014    18023       +9     
   - Partials      605      606       +1
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [...alking/oap/query/graphql/resolver/MetricQuery.java](https://codecov.io/gh/apache/skywalking/pull/4177/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItcXVlcnktcGx1Z2luL3F1ZXJ5LWdyYXBocWwtcGx1Z2luL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9za3l3YWxraW5nL29hcC9xdWVyeS9ncmFwaHFsL3Jlc29sdmVyL01ldHJpY1F1ZXJ5LmphdmE=) | `0% <0%> (ø)` | :arrow_up: |
   | [.../storage/plugin/jdbc/h2/dao/H2MetricsQueryDAO.java](https://codecov.io/gh/apache/skywalking/pull/4177/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItc3RvcmFnZS1wbHVnaW4vc3RvcmFnZS1qZGJjLWhpa2FyaWNwLXBsdWdpbi9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvc2t5d2Fsa2luZy9vYXAvc2VydmVyL3N0b3JhZ2UvcGx1Z2luL2pkYmMvaDIvZGFvL0gyTWV0cmljc1F1ZXJ5REFPLmphdmE=) | `0% <0%> (ø)` | :arrow_up: |
   | [.../plugin/elasticsearch/query/MetricsQueryEsDAO.java](https://codecov.io/gh/apache/skywalking/pull/4177/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItc3RvcmFnZS1wbHVnaW4vc3RvcmFnZS1lbGFzdGljc2VhcmNoLXBsdWdpbi9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvc2t5d2Fsa2luZy9vYXAvc2VydmVyL3N0b3JhZ2UvcGx1Z2luL2VsYXN0aWNzZWFyY2gvcXVlcnkvTWV0cmljc1F1ZXJ5RXNEQU8uamF2YQ==) | `0% <0%> (ø)` | :arrow_up: |
   | [...king/oap/server/core/query/MetricQueryService.java](https://codecov.io/gh/apache/skywalking/pull/4177/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvc2t5d2Fsa2luZy9vYXAvc2VydmVyL2NvcmUvcXVlcnkvTWV0cmljUXVlcnlTZXJ2aWNlLmphdmE=) | `0% <0%> (ø)` | :arrow_up: |
   | [.../agent/core/profile/ProfileTaskChannelService.java](https://codecov.io/gh/apache/skywalking/pull/4177/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLWFnZW50LWNvcmUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvYXBtL2FnZW50L2NvcmUvcHJvZmlsZS9Qcm9maWxlVGFza0NoYW5uZWxTZXJ2aWNlLmphdmE=) | `35.63% <0%> (-1.15%)` | :arrow_down: |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=footer). Last update [23be984...9608d07](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [skywalking] wu-sheng commented on issue #4177: Enhance gRPC plugin

Posted by GitBox <gi...@apache.org>.
wu-sheng commented on issue #4177: Enhance gRPC plugin
URL: https://github.com/apache/skywalking/pull/4177#issuecomment-571581668
 
 
   > This is a very good pull request template for the plugin contribution. 
   
   Anyone want to discuss about new issue and pull request templates?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [skywalking] codecov-io commented on issue #4177: Enhance gRPC plugin

Posted by GitBox <gi...@apache.org>.
codecov-io commented on issue #4177: Enhance gRPC plugin
URL: https://github.com/apache/skywalking/pull/4177#issuecomment-570947001
 
 
   # [Codecov](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=h1) Report
   > Merging [#4177](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=desc) into [master](https://codecov.io/gh/apache/skywalking/commit/23b15e645bf30b29281aa34e6f37322941cae558?src=pr&el=desc) will **decrease** coverage by `0.16%`.
   > The diff coverage is `0%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/skywalking/pull/4177/graphs/tree.svg?width=650&token=qrILxY5yA8&height=150&src=pr)](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #4177      +/-   ##
   ==========================================
   - Coverage   26.82%   26.66%   -0.17%     
   ==========================================
     Files        1163     1164       +1     
     Lines       25428    25525      +97     
     Branches     3691     3702      +11     
   ==========================================
   - Hits         6820     6805      -15     
   - Misses      18000    18114     +114     
   + Partials      608      606       -2
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [...ng/apm/plugin/grpc/v1/OperationNameFormatUtil.java](https://codecov.io/gh/apache/skywalking/pull/4177/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLXNkay1wbHVnaW4vZ3JwYy0xLngtcGx1Z2luL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9za3l3YWxraW5nL2FwbS9wbHVnaW4vZ3JwYy92MS9PcGVyYXRpb25OYW1lRm9ybWF0VXRpbC5qYXZh) | `0% <ø> (ø)` | :arrow_up: |
   | [...plugin/grpc/v1/client/BlockingCallInterceptor.java](https://codecov.io/gh/apache/skywalking/pull/4177/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLXNkay1wbHVnaW4vZ3JwYy0xLngtcGx1Z2luL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9za3l3YWxraW5nL2FwbS9wbHVnaW4vZ3JwYy92MS9jbGllbnQvQmxvY2tpbmdDYWxsSW50ZXJjZXB0b3IuamF2YQ==) | `0% <0%> (ø)` | |
   | [...g/apm/plugin/grpc/v1/client/TracingClientCall.java](https://codecov.io/gh/apache/skywalking/pull/4177/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLXNkay1wbHVnaW4vZ3JwYy0xLngtcGx1Z2luL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9za3l3YWxraW5nL2FwbS9wbHVnaW4vZ3JwYy92MS9jbGllbnQvVHJhY2luZ0NsaWVudENhbGwuamF2YQ==) | `0% <0%> (ø)` | |
   | [...g/apm/plugin/grpc/v1/server/ServerInterceptor.java](https://codecov.io/gh/apache/skywalking/pull/4177/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLXNkay1wbHVnaW4vZ3JwYy0xLngtcGx1Z2luL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9za3l3YWxraW5nL2FwbS9wbHVnaW4vZ3JwYy92MS9zZXJ2ZXIvU2VydmVySW50ZXJjZXB0b3IuamF2YQ==) | `0% <0%> (ø)` | |
   | [...g/apm/plugin/grpc/v1/server/TracingServerCall.java](https://codecov.io/gh/apache/skywalking/pull/4177/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLXNkay1wbHVnaW4vZ3JwYy0xLngtcGx1Z2luL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9za3l3YWxraW5nL2FwbS9wbHVnaW4vZ3JwYy92MS9zZXJ2ZXIvVHJhY2luZ1NlcnZlckNhbGwuamF2YQ==) | `0% <0%> (ø)` | |
   | [...plugin/grpc/v1/client/AbstractStubInterceptor.java](https://codecov.io/gh/apache/skywalking/pull/4177/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLXNkay1wbHVnaW4vZ3JwYy0xLngtcGx1Z2luL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9za3l3YWxraW5nL2FwbS9wbHVnaW4vZ3JwYy92MS9jbGllbnQvQWJzdHJhY3RTdHViSW50ZXJjZXB0b3IuamF2YQ==) | `0% <0%> (ø)` | |
   | [...ugin/grpc/v1/server/TracingServerCallListener.java](https://codecov.io/gh/apache/skywalking/pull/4177/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLXNkay1wbHVnaW4vZ3JwYy0xLngtcGx1Z2luL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9za3l3YWxraW5nL2FwbS9wbHVnaW4vZ3JwYy92MS9zZXJ2ZXIvVHJhY2luZ1NlcnZlckNhbGxMaXN0ZW5lci5qYXZh) | `0% <0%> (ø)` | |
   | [...g/apm/plugin/grpc/v1/client/ClientInterceptor.java](https://codecov.io/gh/apache/skywalking/pull/4177/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLXNkay1wbHVnaW4vZ3JwYy0xLngtcGx1Z2luL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9za3l3YWxraW5nL2FwbS9wbHVnaW4vZ3JwYy92MS9jbGllbnQvQ2xpZW50SW50ZXJjZXB0b3IuamF2YQ==) | `0% <0%> (ø)` | |
   | [.../apache/skywalking/apm/agent/core/conf/Config.java](https://codecov.io/gh/apache/skywalking/pull/4177/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLWFnZW50LWNvcmUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvYXBtL2FnZW50L2NvcmUvY29uZi9Db25maWcuamF2YQ==) | `62.96% <0%> (-3.71%)` | :arrow_down: |
   | [...plugin/grpc/v1/client/SimpleTracingClientCall.java](https://codecov.io/gh/apache/skywalking/pull/4177/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLXNkay1wbHVnaW4vZ3JwYy0xLngtcGx1Z2luL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9za3l3YWxraW5nL2FwbS9wbHVnaW4vZ3JwYy92MS9jbGllbnQvU2ltcGxlVHJhY2luZ0NsaWVudENhbGwuamF2YQ==) | `0% <0%> (ø)` | |
   | ... and [10 more](https://codecov.io/gh/apache/skywalking/pull/4177/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=footer). Last update [23b15e6...337442d](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [skywalking] wu-sheng commented on issue #4177: Enhance gRPC plugin

Posted by GitBox <gi...@apache.org>.
wu-sheng commented on issue #4177: Enhance gRPC plugin
URL: https://github.com/apache/skywalking/pull/4177#issuecomment-573539623
 
 
   I will do the review after you remove the configurations and make the tracing working at client/server sides with callback. Thanks for your contributions.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [skywalking] devkanro commented on issue #4177: Enhance gRPC plugin

Posted by GitBox <gi...@apache.org>.
devkanro commented on issue #4177: Enhance gRPC plugin
URL: https://github.com/apache/skywalking/pull/4177#issuecomment-573529847
 
 
   You convinced me, the client-side tracing is also important to trace.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [skywalking] devkanro commented on issue #4177: Enhance gRPC plugin

Posted by GitBox <gi...@apache.org>.
devkanro commented on issue #4177: Enhance gRPC plugin
URL: https://github.com/apache/skywalking/pull/4177#issuecomment-570994639
 
 
   > If you enhance the plugin, i think you should adjust Grpc tests cases, then use ci to check first.
   
   Yes, it already in our checklist, and @GuoDuanLZ will help us in tests.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [skywalking] wu-sheng commented on issue #4177: Enhance gRPC plugin

Posted by GitBox <gi...@apache.org>.
wu-sheng commented on issue #4177: Enhance gRPC plugin
URL: https://github.com/apache/skywalking/pull/4177#issuecomment-573522600
 
 
   If no client-side span, there is no client-side metrics of topology. Then you would detect the error of unreachable or network perf unstable issues from trace and response time page.  The thing you posted is exactly an APM should collect.
   
   Back to my point, this is a basic design of SkyWalking. We should not argue about this in a single one plugin. If you want to discuss that, it is more than this. You need to change the design and protocol of the project.
   
   If you want to change this, I prefer you keep that in private, and only push the both sides tracing in the upstream.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [skywalking] codecov-io edited a comment on issue #4177: Enhance gRPC plugin

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #4177: Enhance gRPC plugin
URL: https://github.com/apache/skywalking/pull/4177#issuecomment-570947001
 
 
   # [Codecov](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=h1) Report
   > Merging [#4177](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=desc) into [master](https://codecov.io/gh/apache/skywalking/commit/722f77a3c9fb7cb92886687ce6d2efb93fbb3b4c?src=pr&el=desc) will **decrease** coverage by `0.08%`.
   > The diff coverage is `47.61%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/skywalking/pull/4177/graphs/tree.svg?width=650&token=qrILxY5yA8&height=150&src=pr)](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #4177      +/-   ##
   ==========================================
   - Coverage    26.4%   26.32%   -0.09%     
   ==========================================
     Files        1181     1180       -1     
     Lines       25957    25903      -54     
     Branches     3774     3760      -14     
   ==========================================
   - Hits         6855     6820      -35     
   + Misses      18498    18478      -20     
   - Partials      604      605       +1
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [...ng/oap/server/core/alarm/provider/RunningRule.java](https://codecov.io/gh/apache/skywalking/pull/4177/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItYWxhcm0tcGx1Z2luL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9za3l3YWxraW5nL29hcC9zZXJ2ZXIvY29yZS9hbGFybS9wcm92aWRlci9SdW5uaW5nUnVsZS5qYXZh) | `74.83% <44.44%> (+8.92%)` | :arrow_up: |
   | [.../skywalking/oap/server/core/alarm/provider/OP.java](https://codecov.io/gh/apache/skywalking/pull/4177/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItYWxhcm0tcGx1Z2luL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9za3l3YWxraW5nL29hcC9zZXJ2ZXIvY29yZS9hbGFybS9wcm92aWRlci9PUC5qYXZh) | `50% <50%> (-7.15%)` | :arrow_down: |
   | [...pm/agent/core/profile/ProfileTaskQueryService.java](https://codecov.io/gh/apache/skywalking/pull/4177/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLWFnZW50LWNvcmUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvYXBtL2FnZW50L2NvcmUvcHJvZmlsZS9Qcm9maWxlVGFza1F1ZXJ5U2VydmljZS5qYXZh) | `48.71% <0%> (-2.57%)` | :arrow_down: |
   | [...ache/skywalking/apm/agent/core/jvm/JVMService.java](https://codecov.io/gh/apache/skywalking/pull/4177/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLWFnZW50LWNvcmUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvYXBtL2FnZW50L2NvcmUvanZtL0pWTVNlcnZpY2UuamF2YQ==) | `75.4% <0%> (-1.64%)` | :arrow_down: |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=footer). Last update [722f77a...7ff8b91](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [skywalking] codecov-io edited a comment on issue #4177: Enhance gRPC plugin

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #4177: Enhance gRPC plugin
URL: https://github.com/apache/skywalking/pull/4177#issuecomment-575098453
 
 
   # [Codecov](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=h1) Report
   > Merging [#4177](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=desc) into [master](https://codecov.io/gh/apache/skywalking/commit/d0ef0dff770e4342ad98a643eb19b9ccd5a5fb69?src=pr&el=desc) will **increase** coverage by `0.09%`.
   > The diff coverage is `n/a`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/skywalking/pull/4177/graphs/tree.svg?width=650&token=qrILxY5yA8&height=150&src=pr)](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #4177      +/-   ##
   ==========================================
   + Coverage   26.74%   26.83%   +0.09%     
   ==========================================
     Files        1162     1162              
     Lines       25446    25446              
     Branches     3630     3630              
   ==========================================
   + Hits         6806     6829      +23     
   + Misses      18034    18012      -22     
   + Partials      606      605       -1
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [.../agent/core/profile/ProfileTaskChannelService.java](https://codecov.io/gh/apache/skywalking/pull/4177/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLWFnZW50LWNvcmUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvYXBtL2FnZW50L2NvcmUvcHJvZmlsZS9Qcm9maWxlVGFza0NoYW5uZWxTZXJ2aWNlLmphdmE=) | `36.78% <0%> (+8.04%)` | :arrow_up: |
   | [...alking/apm/agent/core/remote/AgentIDDecorator.java](https://codecov.io/gh/apache/skywalking/pull/4177/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLWFnZW50LWNvcmUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvYXBtL2FnZW50L2NvcmUvcmVtb3RlL0FnZW50SUREZWNvcmF0b3IuamF2YQ==) | `85.71% <0%> (+17.85%)` | :arrow_up: |
   | [...ache/skywalking/apm/agent/core/jvm/JVMService.java](https://codecov.io/gh/apache/skywalking/pull/4177/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLWFnZW50LWNvcmUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvYXBtL2FnZW50L2NvcmUvanZtL0pWTVNlcnZpY2UuamF2YQ==) | `77.04% <0%> (+18.03%)` | :arrow_up: |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=footer). Last update [d0ef0df...394adec](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [skywalking] devkanro removed a comment on issue #4177: Enhance gRPC plugin

Posted by GitBox <gi...@apache.org>.
devkanro removed a comment on issue #4177: Enhance gRPC plugin
URL: https://github.com/apache/skywalking/pull/4177#issuecomment-573500007
 
 
   The original plugin doesn't provide the client-side tracing, I add it for my case and optional powerful tracing.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [skywalking] codecov-io edited a comment on issue #4177: Enhance gRPC plugin

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #4177: Enhance gRPC plugin
URL: https://github.com/apache/skywalking/pull/4177#issuecomment-570947001
 
 
   # [Codecov](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=h1) Report
   > Merging [#4177](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=desc) into [master](https://codecov.io/gh/apache/skywalking/commit/337442dc2edfad63a0fe08a994e2a68641972dbe?src=pr&el=desc) will **decrease** coverage by `0.29%`.
   > The diff coverage is `53.84%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/skywalking/pull/4177/graphs/tree.svg?width=650&token=qrILxY5yA8&height=150&src=pr)](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff            @@
   ##           master    #4177     +/-   ##
   =========================================
   - Coverage   26.66%   26.36%   -0.3%     
   =========================================
     Files        1164     1179     +15     
     Lines       25525    25871    +346     
     Branches     3702     3759     +57     
   =========================================
   + Hits         6805     6820     +15     
   - Misses      18114    18437    +323     
   - Partials      606      614      +8
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [...ugin/grpc/v1/server/TracingServerCallListener.java](https://codecov.io/gh/apache/skywalking/pull/4177/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLXNkay1wbHVnaW4vZ3JwYy0xLngtcGx1Z2luL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9za3l3YWxraW5nL2FwbS9wbHVnaW4vZ3JwYy92MS9zZXJ2ZXIvVHJhY2luZ1NlcnZlckNhbGxMaXN0ZW5lci5qYXZh) | `0% <ø> (ø)` | :arrow_up: |
   | [.../java/org/apache/skywalking/oal/rt/OALRuntime.java](https://codecov.io/gh/apache/skywalking/pull/4177/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9vYWwtcnQvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvb2FsL3J0L09BTFJ1bnRpbWUuamF2YQ==) | `0% <ø> (ø)` | :arrow_up: |
   | [...g/apm/plugin/grpc/v1/server/ServerInterceptor.java](https://codecov.io/gh/apache/skywalking/pull/4177/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLXNkay1wbHVnaW4vZ3JwYy0xLngtcGx1Z2luL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9za3l3YWxraW5nL2FwbS9wbHVnaW4vZ3JwYy92MS9zZXJ2ZXIvU2VydmVySW50ZXJjZXB0b3IuamF2YQ==) | `0% <ø> (ø)` | :arrow_up: |
   | [...walking/apm/plugin/asf/dubbo/DubboInterceptor.java](https://codecov.io/gh/apache/skywalking/pull/4177/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLXNkay1wbHVnaW4vZHViYm8tMi43LngtcGx1Z2luL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9za3l3YWxraW5nL2FwbS9wbHVnaW4vYXNmL2R1YmJvL0R1YmJvSW50ZXJjZXB0b3IuamF2YQ==) | `92.45% <0%> (-3.63%)` | :arrow_down: |
   | [...g/apm/plugin/grpc/v1/server/TracingServerCall.java](https://codecov.io/gh/apache/skywalking/pull/4177/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLXNkay1wbHVnaW4vZ3JwYy0xLngtcGx1Z2luL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9za3l3YWxraW5nL2FwbS9wbHVnaW4vZ3JwYy92MS9zZXJ2ZXIvVHJhY2luZ1NlcnZlckNhbGwuamF2YQ==) | `0% <0%> (ø)` | :arrow_up: |
   | [.../skywalking/apm/plugin/dubbo/DubboInterceptor.java](https://codecov.io/gh/apache/skywalking/pull/4177/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLXNkay1wbHVnaW4vZHViYm8tcGx1Z2luL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9za3l3YWxraW5nL2FwbS9wbHVnaW4vZHViYm8vRHViYm9JbnRlcmNlcHRvci5qYXZh) | `92.45% <0%> (-3.63%)` | :arrow_down: |
   | [...alking/apm/agent/core/remote/AgentIDDecorator.java](https://codecov.io/gh/apache/skywalking/pull/4177/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLWFnZW50LWNvcmUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvYXBtL2FnZW50L2NvcmUvcmVtb3RlL0FnZW50SUREZWNvcmF0b3IuamF2YQ==) | `67.85% <87.5%> (-21.04%)` | :arrow_down: |
   | [...ache/skywalking/apm/agent/core/jvm/JVMService.java](https://codecov.io/gh/apache/skywalking/pull/4177/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLWFnZW50LWNvcmUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvYXBtL2FnZW50L2NvcmUvanZtL0pWTVNlcnZpY2UuamF2YQ==) | `57.37% <0%> (-16.4%)` | :arrow_down: |
   | [.../core/remote/ServiceAndEndpointRegisterClient.java](https://codecov.io/gh/apache/skywalking/pull/4177/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLWFnZW50LWNvcmUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvYXBtL2FnZW50L2NvcmUvcmVtb3RlL1NlcnZpY2VBbmRFbmRwb2ludFJlZ2lzdGVyQ2xpZW50LmphdmE=) | `28.08% <0%> (-3.38%)` | :arrow_down: |
   | ... and [39 more](https://codecov.io/gh/apache/skywalking/pull/4177/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=footer). Last update [337442d...577efd2](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [skywalking] codecov-io edited a comment on issue #4177: Enhance gRPC plugin

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #4177: Enhance gRPC plugin
URL: https://github.com/apache/skywalking/pull/4177#issuecomment-570947001
 
 
   # [Codecov](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=h1) Report
   > Merging [#4177](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=desc) into [master](https://codecov.io/gh/apache/skywalking/commit/7e9812e8572ec9d8cebef452be0f0a40ee0fae66?src=pr&el=desc) will **decrease** coverage by `0.29%`.
   > The diff coverage is `0%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/skywalking/pull/4177/graphs/tree.svg?width=650&token=qrILxY5yA8&height=150&src=pr)](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff            @@
   ##           master    #4177     +/-   ##
   =========================================
   - Coverage   26.66%   26.37%   -0.3%     
   =========================================
     Files        1178     1179      +1     
     Lines       25773    25871     +98     
     Branches     3748     3759     +11     
   =========================================
   - Hits         6872     6823     -49     
   - Misses      18286    18436    +150     
   + Partials      615      612      -3
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [...ng/apm/plugin/grpc/v1/OperationNameFormatUtil.java](https://codecov.io/gh/apache/skywalking/pull/4177/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLXNkay1wbHVnaW4vZ3JwYy0xLngtcGx1Z2luL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9za3l3YWxraW5nL2FwbS9wbHVnaW4vZ3JwYy92MS9PcGVyYXRpb25OYW1lRm9ybWF0VXRpbC5qYXZh) | `0% <ø> (ø)` | :arrow_up: |
   | [...plugin/grpc/v1/client/BlockingCallInterceptor.java](https://codecov.io/gh/apache/skywalking/pull/4177/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLXNkay1wbHVnaW4vZ3JwYy0xLngtcGx1Z2luL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9za3l3YWxraW5nL2FwbS9wbHVnaW4vZ3JwYy92MS9jbGllbnQvQmxvY2tpbmdDYWxsSW50ZXJjZXB0b3IuamF2YQ==) | `0% <0%> (ø)` | |
   | [...g/apm/plugin/grpc/v1/server/TracingServerCall.java](https://codecov.io/gh/apache/skywalking/pull/4177/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLXNkay1wbHVnaW4vZ3JwYy0xLngtcGx1Z2luL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9za3l3YWxraW5nL2FwbS9wbHVnaW4vZ3JwYy92MS9zZXJ2ZXIvVHJhY2luZ1NlcnZlckNhbGwuamF2YQ==) | `0% <0%> (ø)` | |
   | [...g/apm/plugin/grpc/v1/server/ServerInterceptor.java](https://codecov.io/gh/apache/skywalking/pull/4177/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLXNkay1wbHVnaW4vZ3JwYy0xLngtcGx1Z2luL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9za3l3YWxraW5nL2FwbS9wbHVnaW4vZ3JwYy92MS9zZXJ2ZXIvU2VydmVySW50ZXJjZXB0b3IuamF2YQ==) | `0% <0%> (ø)` | |
   | [...g/apm/plugin/grpc/v1/client/TracingClientCall.java](https://codecov.io/gh/apache/skywalking/pull/4177/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLXNkay1wbHVnaW4vZ3JwYy0xLngtcGx1Z2luL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9za3l3YWxraW5nL2FwbS9wbHVnaW4vZ3JwYy92MS9jbGllbnQvVHJhY2luZ0NsaWVudENhbGwuamF2YQ==) | `0% <0%> (ø)` | |
   | [...plugin/grpc/v1/client/AbstractStubInterceptor.java](https://codecov.io/gh/apache/skywalking/pull/4177/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLXNkay1wbHVnaW4vZ3JwYy0xLngtcGx1Z2luL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9za3l3YWxraW5nL2FwbS9wbHVnaW4vZ3JwYy92MS9jbGllbnQvQWJzdHJhY3RTdHViSW50ZXJjZXB0b3IuamF2YQ==) | `0% <0%> (ø)` | |
   | [...plugin/grpc/v1/client/SimpleTracingClientCall.java](https://codecov.io/gh/apache/skywalking/pull/4177/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLXNkay1wbHVnaW4vZ3JwYy0xLngtcGx1Z2luL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9za3l3YWxraW5nL2FwbS9wbHVnaW4vZ3JwYy92MS9jbGllbnQvU2ltcGxlVHJhY2luZ0NsaWVudENhbGwuamF2YQ==) | `0% <0%> (ø)` | |
   | [...ugin/grpc/v1/server/TracingServerCallListener.java](https://codecov.io/gh/apache/skywalking/pull/4177/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLXNkay1wbHVnaW4vZ3JwYy0xLngtcGx1Z2luL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9za3l3YWxraW5nL2FwbS9wbHVnaW4vZ3JwYy92MS9zZXJ2ZXIvVHJhY2luZ1NlcnZlckNhbGxMaXN0ZW5lci5qYXZh) | `0% <0%> (ø)` | |
   | [...1/server/AbstractServerImplBuilderInterceptor.java](https://codecov.io/gh/apache/skywalking/pull/4177/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLXNkay1wbHVnaW4vZ3JwYy0xLngtcGx1Z2luL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9za3l3YWxraW5nL2FwbS9wbHVnaW4vZ3JwYy92MS9zZXJ2ZXIvQWJzdHJhY3RTZXJ2ZXJJbXBsQnVpbGRlckludGVyY2VwdG9yLmphdmE=) | `0% <0%> (ø)` | |
   | [.../apache/skywalking/apm/agent/core/conf/Config.java](https://codecov.io/gh/apache/skywalking/pull/4177/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLWFnZW50LWNvcmUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvYXBtL2FnZW50L2NvcmUvY29uZi9Db25maWcuamF2YQ==) | `64.28% <0%> (-3.64%)` | :arrow_down: |
   | ... and [15 more](https://codecov.io/gh/apache/skywalking/pull/4177/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=footer). Last update [7e9812e...6547afa](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [skywalking] devkanro commented on issue #4177: Enhance gRPC plugin

Posted by GitBox <gi...@apache.org>.
devkanro commented on issue #4177: Enhance gRPC plugin
URL: https://github.com/apache/skywalking/pull/4177#issuecomment-576532610
 
 
   It seems test case passed with client tracing

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [skywalking] wu-sheng commented on a change in pull request #4177: Enhance gRPC plugin

Posted by GitBox <gi...@apache.org>.
wu-sheng commented on a change in pull request #4177: Enhance gRPC plugin
URL: https://github.com/apache/skywalking/pull/4177#discussion_r371105013
 
 

 ##########
 File path: apm-sniffer/apm-sdk-plugin/grpc-1.x-plugin/src/main/java/org/apache/skywalking/apm/plugin/grpc/v1/server/TracingServerCallListener.java
 ##########
 @@ -0,0 +1,102 @@
+/*
+ * 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.apm.plugin.grpc.v1.server;
+
+import io.grpc.ForwardingServerCallListener;
+import io.grpc.MethodDescriptor;
+import io.grpc.ServerCall;
+import org.apache.skywalking.apm.agent.core.context.ContextManager;
+import org.apache.skywalking.apm.agent.core.context.ContextSnapshot;
+import org.apache.skywalking.apm.agent.core.context.trace.AbstractSpan;
+import org.apache.skywalking.apm.agent.core.context.trace.SpanLayer;
+import org.apache.skywalking.apm.network.trace.component.ComponentsDefine;
+import org.apache.skywalking.apm.plugin.grpc.v1.OperationNameFormatUtil;
+
+import static org.apache.skywalking.apm.plugin.grpc.v1.Constants.*;
+
+/**
+ * @author wang zheng, kanro
+ */
+public class TracingServerCallListener<REQUEST> extends ForwardingServerCallListener.SimpleForwardingServerCallListener<REQUEST> {
+
+    private final ContextSnapshot contextSnapshot;
+    private final MethodDescriptor.MethodType methodType;
+    private final String operationPrefix;
+
+    protected TracingServerCallListener(ServerCall.Listener<REQUEST> delegate, MethodDescriptor<REQUEST, ?> descriptor,
+                                        ContextSnapshot contextSnapshot) {
+        super(delegate);
+        this.contextSnapshot = contextSnapshot;
+        this.methodType = descriptor.getType();
+        this.operationPrefix = OperationNameFormatUtil.formatOperationName(descriptor) + SERVER;
+    }
+
+    @Override
+    public void onMessage(REQUEST message) {
+        // We just create the request on message span for client stream calls.
+        if (!methodType.clientSendsOneMessage()) {
+            try {
+                final AbstractSpan span = ContextManager.createLocalSpan(operationPrefix + REQUEST_ON_MESSAGE_OPERATION_NAME);
+                span.setComponent(ComponentsDefine.GRPC);
+                span.setLayer(SpanLayer.RPC_FRAMEWORK);
+                ContextManager.continued(contextSnapshot);
+                super.onMessage(message);
+            } catch (Throwable t) {
+                ContextManager.activeSpan().errorOccurred().log(t);
+                throw t;
+            } finally {
+                ContextManager.stopSpan();
+            }
+        } else {
+            super.onMessage(message);
+        }
+    }
+
+    @Override
+    public void onCancel() {
+        try {
+            final AbstractSpan span = ContextManager.createLocalSpan(operationPrefix + REQUEST_ON_CANCEL_OPERATION_NAME);
+            span.setComponent(ComponentsDefine.GRPC);
+            span.setLayer(SpanLayer.RPC_FRAMEWORK);
+            ContextManager.continued(contextSnapshot);
+            super.onCancel();
 
 Review comment:
   A similar suggestion, `try/final` should be added here.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [skywalking] devkanro commented on a change in pull request #4177: Enhance gRPC plugin

Posted by GitBox <gi...@apache.org>.
devkanro commented on a change in pull request #4177: Enhance gRPC plugin
URL: https://github.com/apache/skywalking/pull/4177#discussion_r367261370
 
 

 ##########
 File path: apm-sniffer/apm-sdk-plugin/grpc-1.x-plugin/src/main/java/org/apache/skywalking/apm/plugin/grpc/v1/client/BlockingCallInterceptor.java
 ##########
 @@ -16,43 +16,48 @@
  *
  */
 
-package org.apache.skywalking.apm.plugin.grpc.v1;
+package org.apache.skywalking.apm.plugin.grpc.v1.client;
 
 import io.grpc.Channel;
 import io.grpc.MethodDescriptor;
-import java.lang.reflect.Method;
 import org.apache.skywalking.apm.agent.core.context.ContextManager;
 import org.apache.skywalking.apm.agent.core.context.trace.AbstractSpan;
 import org.apache.skywalking.apm.agent.core.context.trace.SpanLayer;
 import org.apache.skywalking.apm.agent.core.plugin.interceptor.enhance.MethodInterceptResult;
 import org.apache.skywalking.apm.agent.core.plugin.interceptor.enhance.StaticMethodsAroundInterceptor;
 import org.apache.skywalking.apm.network.trace.component.ComponentsDefine;
 
+import java.lang.reflect.Method;
+
+import static org.apache.skywalking.apm.plugin.grpc.v1.Constants.BLOCKING_CALL_EXIT_SPAN;
 import static org.apache.skywalking.apm.plugin.grpc.v1.OperationNameFormatUtil.formatOperationName;
 
 /**
- * @author zhang xin
+ * @author zhang xin, kanro
  */
 public class BlockingCallInterceptor implements StaticMethodsAroundInterceptor {
 
-    @Override public void beforeMethod(Class clazz, Method method, Object[] allArguments, Class<?>[] parameterTypes,
-        MethodInterceptResult result) {
-        Channel channel = (Channel)allArguments[0];
-        MethodDescriptor methodDescriptor = (MethodDescriptor)allArguments[1];
+    @Override
+    public void beforeMethod(Class clazz, Method method, Object[] allArguments, Class<?>[] parameterTypes,
+                             MethodInterceptResult result) {
+        Channel channel = (Channel) allArguments[0];
+        MethodDescriptor<?, ?> methodDescriptor = (MethodDescriptor<?, ?>) allArguments[1];
         final AbstractSpan span = ContextManager.createExitSpan(formatOperationName(methodDescriptor), channel.authority());
         span.setComponent(ComponentsDefine.GRPC);
-        SpanLayer.asRPCFramework(span);
+        span.setLayer(SpanLayer.RPC_FRAMEWORK);
+        ContextManager.getRuntimeContext().put(BLOCKING_CALL_EXIT_SPAN, span);
 
 Review comment:
   Yes, got it, I will fix it later

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [skywalking] wu-sheng commented on a change in pull request #4177: Enhance gRPC plugin

Posted by GitBox <gi...@apache.org>.
wu-sheng commented on a change in pull request #4177: Enhance gRPC plugin
URL: https://github.com/apache/skywalking/pull/4177#discussion_r371647780
 
 

 ##########
 File path: apm-sniffer/apm-sdk-plugin/grpc-1.x-plugin/src/main/java/org/apache/skywalking/apm/plugin/grpc/v1/server/TracingServerCallListener.java
 ##########
 @@ -0,0 +1,102 @@
+/*
+ * 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.apm.plugin.grpc.v1.server;
+
+import io.grpc.ForwardingServerCallListener;
+import io.grpc.MethodDescriptor;
+import io.grpc.ServerCall;
+import org.apache.skywalking.apm.agent.core.context.ContextManager;
+import org.apache.skywalking.apm.agent.core.context.ContextSnapshot;
+import org.apache.skywalking.apm.agent.core.context.trace.AbstractSpan;
+import org.apache.skywalking.apm.agent.core.context.trace.SpanLayer;
+import org.apache.skywalking.apm.network.trace.component.ComponentsDefine;
+import org.apache.skywalking.apm.plugin.grpc.v1.OperationNameFormatUtil;
+
+import static org.apache.skywalking.apm.plugin.grpc.v1.Constants.*;
+
+/**
+ * @author wang zheng, kanro
+ */
+public class TracingServerCallListener<REQUEST> extends ForwardingServerCallListener.SimpleForwardingServerCallListener<REQUEST> {
+
+    private final ContextSnapshot contextSnapshot;
+    private final MethodDescriptor.MethodType methodType;
+    private final String operationPrefix;
+
+    protected TracingServerCallListener(ServerCall.Listener<REQUEST> delegate, MethodDescriptor<REQUEST, ?> descriptor,
+                                        ContextSnapshot contextSnapshot) {
+        super(delegate);
+        this.contextSnapshot = contextSnapshot;
+        this.methodType = descriptor.getType();
+        this.operationPrefix = OperationNameFormatUtil.formatOperationName(descriptor) + SERVER;
+    }
+
+    @Override
+    public void onMessage(REQUEST message) {
+        // We just create the request on message span for client stream calls.
+        if (!methodType.clientSendsOneMessage()) {
+            try {
+                final AbstractSpan span = ContextManager.createLocalSpan(operationPrefix + REQUEST_ON_MESSAGE_OPERATION_NAME);
+                span.setComponent(ComponentsDefine.GRPC);
+                span.setLayer(SpanLayer.RPC_FRAMEWORK);
+                ContextManager.continued(contextSnapshot);
+            } catch (Throwable t) {
+                ContextManager.activeSpan().errorOccurred().log(t);
 
 Review comment:
   As I last comment, please just `try/final`, but not `catch`. I just want to guarantee our plugins would not break the original codes even our plugins have bugs.
   
   Please recheck the codes based on this rule. 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [skywalking] wu-sheng commented on issue #4177: Enhance gRPC plugin

Posted by GitBox <gi...@apache.org>.
wu-sheng commented on issue #4177: Enhance gRPC plugin
URL: https://github.com/apache/skywalking/pull/4177#issuecomment-579136931
 
 
   You could use another issue or mail list to discuss this. But to be honest, I can't see much difference. Many discussions have been done about this.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [skywalking] wu-sheng commented on a change in pull request #4177: Enhance gRPC plugin

Posted by GitBox <gi...@apache.org>.
wu-sheng commented on a change in pull request #4177: Enhance gRPC plugin
URL: https://github.com/apache/skywalking/pull/4177#discussion_r366914052
 
 

 ##########
 File path: apm-sniffer/apm-sdk-plugin/grpc-1.x-plugin/src/main/java/org/apache/skywalking/apm/plugin/grpc/v1/client/BlockingCallInterceptor.java
 ##########
 @@ -16,43 +16,48 @@
  *
  */
 
-package org.apache.skywalking.apm.plugin.grpc.v1;
+package org.apache.skywalking.apm.plugin.grpc.v1.client;
 
 import io.grpc.Channel;
 import io.grpc.MethodDescriptor;
-import java.lang.reflect.Method;
 import org.apache.skywalking.apm.agent.core.context.ContextManager;
 import org.apache.skywalking.apm.agent.core.context.trace.AbstractSpan;
 import org.apache.skywalking.apm.agent.core.context.trace.SpanLayer;
 import org.apache.skywalking.apm.agent.core.plugin.interceptor.enhance.MethodInterceptResult;
 import org.apache.skywalking.apm.agent.core.plugin.interceptor.enhance.StaticMethodsAroundInterceptor;
 import org.apache.skywalking.apm.network.trace.component.ComponentsDefine;
 
+import java.lang.reflect.Method;
+
+import static org.apache.skywalking.apm.plugin.grpc.v1.Constants.BLOCKING_CALL_EXIT_SPAN;
 import static org.apache.skywalking.apm.plugin.grpc.v1.OperationNameFormatUtil.formatOperationName;
 
 /**
- * @author zhang xin
+ * @author zhang xin, kanro
  */
 public class BlockingCallInterceptor implements StaticMethodsAroundInterceptor {
 
-    @Override public void beforeMethod(Class clazz, Method method, Object[] allArguments, Class<?>[] parameterTypes,
-        MethodInterceptResult result) {
-        Channel channel = (Channel)allArguments[0];
-        MethodDescriptor methodDescriptor = (MethodDescriptor)allArguments[1];
+    @Override
+    public void beforeMethod(Class clazz, Method method, Object[] allArguments, Class<?>[] parameterTypes,
+                             MethodInterceptResult result) {
+        Channel channel = (Channel) allArguments[0];
+        MethodDescriptor<?, ?> methodDescriptor = (MethodDescriptor<?, ?>) allArguments[1];
         final AbstractSpan span = ContextManager.createExitSpan(formatOperationName(methodDescriptor), channel.authority());
         span.setComponent(ComponentsDefine.GRPC);
-        SpanLayer.asRPCFramework(span);
+        span.setLayer(SpanLayer.RPC_FRAMEWORK);
+        ContextManager.getRuntimeContext().put(BLOCKING_CALL_EXIT_SPAN, span);
 
 Review comment:
   If not, whether the multiple RPCs will trigger dirty read?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [skywalking] wu-sheng commented on issue #4177: Enhance gRPC plugin

Posted by GitBox <gi...@apache.org>.
wu-sheng commented on issue #4177: Enhance gRPC plugin
URL: https://github.com/apache/skywalking/pull/4177#issuecomment-573482503
 
 
   > Internal and external gRPC server tracing
   
   It is still unclear for me about this section. Client/server side tracing are both required in nearly every RPC plugin. What makes the gRPC different? Manually setting this is very painful for the end user.
   
   cc @kezhenxu94 Could you get the point of this?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [skywalking] GuoDuanLZ commented on a change in pull request #4177: Enhance gRPC plugin

Posted by GitBox <gi...@apache.org>.
GuoDuanLZ commented on a change in pull request #4177: Enhance gRPC plugin
URL: https://github.com/apache/skywalking/pull/4177#discussion_r372171690
 
 

 ##########
 File path: test/plugin/scenarios/grpc-scenario/config/expectedData.yaml
 ##########
 @@ -15,239 +15,524 @@
 # limitations under the License.
 registryItems:
   applications:
-    - {grpc-scenario: 2}
+  - { grpc-scenario: 2 }
   instances:
-    - {grpc-scenario: 1}
+  - { grpc-scenario: 1 }
   operationNames:
-    - grpc-scenario: [Greeter.sayHello, GreeterBlocking.sayHello,
-                      /grpc-scenario/case/grpc-scenario]
+  - grpc-scenario: [Greeter.sayHello, GreeterBlocking.sayHello, GreeterBlockingError.sayHello, /grpc-scenario/case/grpc-scenario]
   heartbeat: []
 segmentItems:
-  - applicationCode: grpc-scenario
-    segmentSize: gt 10
-    segments:
-      - segmentId: not null
-        spans:
-          - operationName: GreeterBlocking.sayHello
-            operationId: 0
-            parentSpanId: -1
-            spanId: 0
-            spanLayer: RPCFramework
-            startTime: nq 0
-            endTime: nq 0
-            componentId: 23
-            componentName: ''
-            isError: false
-            spanType: Entry
-            peer: ''
-            peerId: 0
-            refs:
-              - {parentEndpointId: 0, parentEndpoint: /grpc-scenario/case/grpc-scenario, networkAddressId: 0,
-                 entryEndpointId: 0, refType: CrossProcess, parentSpanId: 2, parentTraceSegmentId: not null,
-                 parentServiceInstanceId: 1, networkAddress: '127.0.0.1:18080', entryEndpoint: /grpc-scenario/case/grpc-scenario,
-                 entryServiceInstanceId: 1}
-      - segmentId: not null
-        spans:
-          - operationName: Greeter.sayHello
-            operationId: 0
-            parentSpanId: -1
-            spanId: 0
-            spanLayer: RPCFramework
-            startTime: nq 0
-            endTime: nq 0
-            componentId: 23
-            componentName: ''
-            isError: false
-            spanType: Entry
-            peer: ''
-            peerId: 0
-            refs:
-              - {parentEndpointId: 0, parentEndpoint: /grpc-scenario/case/grpc-scenario, networkAddressId: 0,
-                 entryEndpointId: 0, refType: CrossProcess, parentSpanId: 1, parentTraceSegmentId: not null,
-                 parentServiceInstanceId: 1, networkAddress: '127.0.0.1:18080', entryEndpoint: /grpc-scenario/case/grpc-scenario,
-                 entryServiceInstanceId: 1}
-      - segmentId: not null
-        spans:
-          - operationName: GreeterBlocking.sayHello/server/RequestObserver/onNext
-            operationId: 0
-            parentSpanId: -1
-            spanId: 0
-            spanLayer: Unknown
-            startTime: nq 0
-            endTime: nq 0
-            componentId: 0
-            componentName: ''
-            isError: false
-            spanType: Local
-            peer: ''
-            peerId: 0
-            refs:
-              - {parentEndpointId: 0, parentEndpoint: GreeterBlocking.sayHello, networkAddressId: 0,
-                 entryEndpointId: 0, refType: CrossThread, parentSpanId: 0, parentTraceSegmentId: not null,
-                 parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
-                 entryServiceInstanceId: 1}
-      - segmentId: not null
-        spans:
-          - operationName: Greeter.sayHello/server/RequestObserver/onNext
-            operationId: 0
-            parentSpanId: -1
-            spanId: 0
-            spanLayer: Unknown
-            startTime: nq 0
-            endTime: nq 0
-            componentId: 0
-            componentName: ''
-            isError: false
-            spanType: Local
-            peer: ''
-            peerId: 0
-            refs:
-              - {parentEndpointId: 0, parentEndpoint: Greeter.sayHello, networkAddressId: 0,
-                 entryEndpointId: 0, refType: CrossThread, parentSpanId: 0, parentTraceSegmentId: not null,
-                 parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
-                 entryServiceInstanceId: 1}
-      - segmentId: not null
-        spans:
-          - operationName: Greeter.sayHello/client/ResponseObserver/onNext
-            operationId: 0
-            parentSpanId: -1
-            spanId: 0
-            spanLayer: Unknown
-            startTime: nq 0
-            endTime: nq 0
-            componentId: 0
-            componentName: ''
-            isError: false
-            spanType: Local
-            peer: ''
-            peerId: 0
-            refs:
-              - {parentEndpointId: 0, parentEndpoint: /grpc-scenario/case/grpc-scenario, networkAddressId: 0,
-                 entryEndpointId: 0, refType: CrossThread, parentSpanId: 1, parentTraceSegmentId: not null,
-                 parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
-                 entryServiceInstanceId: 1}
-      - segmentId: not null
-        spans:
-          - operationName: Greeter.sayHello/client/ResponseObserver/onComplete
-            operationId: 0
-            parentSpanId: -1
-            spanId: 0
-            spanLayer: Unknown
-            startTime: nq 0
-            endTime: nq 0
-            componentId: 0
-            componentName: ''
-            isError: false
-            spanType: Local
-            peer: ''
-            peerId: 0
-            refs:
-              - {parentEndpointId: 0, parentEndpoint: /grpc-scenario/case/grpc-scenario, networkAddressId: 0,
-                 entryEndpointId: 0, refType: CrossThread, parentSpanId: 1, parentTraceSegmentId: not null,
-                 parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
-                 entryServiceInstanceId: 1}
-      - segmentId: not null
-        spans:
-          - operationName: Greeter.sayHello/server/RequestObserver/onNext
-            operationId: 0
-            parentSpanId: -1
-            spanId: 0
-            spanLayer: Unknown
-            startTime: nq 0
-            endTime: nq 0
-            componentId: 0
-            componentName: ''
-            isError: false
-            spanType: Local
-            peer: ''
-            peerId: 0
-            refs:
-              - {parentEndpointId: 0, parentEndpoint: Greeter.sayHello, networkAddressId: 0,
-                 entryEndpointId: 0, refType: CrossThread, parentSpanId: 0, parentTraceSegmentId: not null,
-                 parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
-                 entryServiceInstanceId: 1}
-      - segmentId: not null
-        spans:
-          - operationName: Greeter.sayHello/server/RequestObserver/onComplete
-            operationId: 0
-            parentSpanId: -1
-            spanId: 0
-            spanLayer: Unknown
-            startTime: nq 0
-            endTime: nq 0
-            componentId: 0
-            componentName: ''
-            isError: false
-            spanType: Local
-            peer: ''
-            peerId: 0
-            refs:
-              - {parentEndpointId: 0, parentEndpoint: Greeter.sayHello, networkAddressId: 0,
-                 entryEndpointId: 0, refType: CrossThread, parentSpanId: 0, parentTraceSegmentId: not null,
-                 parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
-                 entryServiceInstanceId: 1}
-      - segmentId: not null
-        spans:
-          - operationName: Greeter.sayHello
-            operationId: 0
-            parentSpanId: 0
-            spanId: 1
-            spanLayer: RPCFramework
-            startTime: nq 0
-            endTime: nq 0
-            componentId: 23
-            componentName: ''
-            isError: false
-            spanType: Exit
-            peer: '127.0.0.1:18080'
-            peerId: 0
-          - operationName: GreeterBlocking.sayHello
-            operationId: 0
-            parentSpanId: 0
-            spanId: 2
-            spanLayer: RPCFramework
-            startTime: nq 0
-            endTime: nq 0
-            componentId: 23
-            componentName: ''
-            isError: false
-            spanType: Exit
-            peer: '127.0.0.1:18080'
-            peerId: 0
-          - operationName: /grpc-scenario/case/grpc-scenario
-            operationId: 0
-            parentSpanId: -1
-            spanId: 0
-            spanLayer: Http
-            startTime: nq 0
-            endTime: nq 0
-            componentId: 1
-            componentName: ''
-            isError: false
-            spanType: Entry
-            peer: ''
-            peerId: 0
-            tags:
-              - {key: url, value: 'http://localhost:8080/grpc-scenario/case/grpc-scenario'}
-              - {key: http.method, value: GET}
-      - segmentId: not null
-        spans:
-          - operationName: Greeter.sayHello/client/ResponseObserver/onNext
-            operationId: 0
-            parentSpanId: -1
-            spanId: 0
-            spanLayer: Unknown
-            startTime: nq 0
-            endTime: nq 0
-            componentId: 0
-            componentName: ''
-            isError: false
-            spanType: Local
-            peer: ''
-            peerId: 0
-            refs:
-              - {parentEndpointId: 0, parentEndpoint: /grpc-scenario/case/grpc-scenario, networkAddressId: 0,
-                 entryEndpointId: 0, refType: CrossThread, parentSpanId: 1, parentTraceSegmentId: not null,
-                 parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
-                 entryServiceInstanceId: 1}
-
+- applicationCode: grpc-scenario
+  segmentSize: gt 10
+  segments:
+  - segmentId: not null
+    spans:
+    - operationName: GreeterBlocking.sayHello
+      operationId: 0
+      parentSpanId: -1
+      spanId: 0
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Entry
+      peer: ''
+      peerId: 0
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: /grpc-scenario/case/grpc-scenario, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossProcess, parentSpanId: 2, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '127.0.0.1:18080', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+  - segmentId: not null
+    spans:
+    - operationName: Greeter.sayHello/client/Request/onMessage
+      operationId: 0
+      parentSpanId: -1
+      spanId: 0
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Local
+      peer: ''
+      peerId: 0
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: /grpc-scenario/case/grpc-scenario, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossThread, parentSpanId: 1, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+  - segmentId: not null
+    spans:
+    - operationName: Greeter.sayHello/client/Request/onMessage
+      operationId: 0
+      parentSpanId: -1
+      spanId: 0
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Local
+      peer: ''
+      peerId: 0
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: /grpc-scenario/case/grpc-scenario, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossThread, parentSpanId: 1, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+  - segmentId: not null
+    spans:
+    - operationName: Greeter.sayHello/client/Request/onComplete
+      operationId: 0
+      parentSpanId: -1
+      spanId: 0
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Local
+      peer: ''
+      peerId: 0
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: /grpc-scenario/case/grpc-scenario, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossThread, parentSpanId: 1, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+  - segmentId: not null
+    spans:
+    - operationName: Greeter.sayHello
+      operationId: 0
+      parentSpanId: -1
+      spanId: 0
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Entry
+      peer: ''
+      peerId: 0
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: /grpc-scenario/case/grpc-scenario, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossProcess, parentSpanId: 1, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '127.0.0.1:18080', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+  - segmentId: not null
+    spans:
+    - operationName: Greeter.sayHello/server/Response/onMessage
+      operationId: 0
+      parentSpanId: 0
+      spanId: 1
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Local
+      peer: ''
+      peerId: 0
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: Greeter.sayHello, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossThread, parentSpanId: 0, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+    - operationName: Greeter.sayHello/server/Request/onMessage
+      operationId: 0
+      parentSpanId: -1
+      spanId: 0
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Local
+      peer: ''
+      peerId: 0
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: Greeter.sayHello, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossThread, parentSpanId: 0, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+  - segmentId: not null
+    spans:
+    - operationName: Greeter.sayHello/client/Response/onMessage
+      operationId: 0
+      parentSpanId: -1
+      spanId: 0
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Local
+      peer: ''
+      peerId: 0
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: /grpc-scenario/case/grpc-scenario, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossThread, parentSpanId: 1, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+  - segmentId: not null
+    spans:
+    - operationName: GreeterBlocking.sayHello/server/Response/onClose
+      operationId: 0
+      parentSpanId: 0
+      spanId: 1
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Local
+      peer: ''
+      peerId: 0
+      tags:
+      - { key: status_code, value: OK }
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: GreeterBlocking.sayHello, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossThread, parentSpanId: 0, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+    - operationName: GreeterBlocking.sayHello/server/Request/onComplete
+      operationId: 0
+      parentSpanId: -1
+      spanId: 0
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Local
+      peer: ''
+      peerId: 0
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: GreeterBlocking.sayHello, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossThread, parentSpanId: 0, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+  - segmentId: not null
+    spans:
+    - operationName: Greeter.sayHello/server/Response/onMessage
+      operationId: 0
+      parentSpanId: 0
+      spanId: 1
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Local
+      peer: ''
+      peerId: 0
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: Greeter.sayHello, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossThread, parentSpanId: 0, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+    - operationName: Greeter.sayHello/server/Request/onMessage
+      operationId: 0
+      parentSpanId: -1
+      spanId: 0
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Local
+      peer: ''
+      peerId: 0
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: Greeter.sayHello, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossThread, parentSpanId: 0, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+  - segmentId: not null
+    spans:
+    - operationName: Greeter.sayHello/server/Response/onClose
+      operationId: 0
+      parentSpanId: 0
+      spanId: 1
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Local
+      peer: ''
+      peerId: 0
+      tags:
+      - {key: status_code, value: OK}
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: Greeter.sayHello, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossThread, parentSpanId: 0, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+    - operationName: Greeter.sayHello/server/Request/onComplete
+      operationId: 0
+      parentSpanId: -1
+      spanId: 0
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Local
+      peer: ''
+      peerId: 0
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: Greeter.sayHello, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossThread, parentSpanId: 0, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+  - segmentId: not null
+    spans:
+    - operationName: Greeter.sayHello/client/Response/onMessage
+      operationId: 0
+      parentSpanId: -1
+      spanId: 0
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Local
+      peer: ''
+      peerId: 0
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: /grpc-scenario/case/grpc-scenario, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossThread, parentSpanId: 1, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+  - segmentId: not null
+    spans:
+    - operationName: Greeter.sayHello/client/Response/onClose
+      operationId: 0
+      parentSpanId: -1
+      spanId: 0
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Local
+      peer: ''
+      peerId: 0
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: /grpc-scenario/case/grpc-scenario, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossThread, parentSpanId: 1, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+  - segmentId: not null
+    spans:
+    - operationName: GreeterBlockingError.sayHello
+      operationId: 0
+      parentSpanId: -1
+      spanId: 0
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Entry
+      peer: ''
+      peerId: 0
+      refs:
+        - {parentEndpointId: 0, parentEndpoint: /grpc-scenario/case/grpc-scenario, networkAddressId: 0,
+           entryEndpointId: 0, refType: CrossProcess, parentSpanId: 5, parentTraceSegmentId: not null,
+           parentServiceInstanceId: 1, networkAddress: '127.0.0.1:18080', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+           entryServiceInstanceId: 1}
+  - segmentId: not null
+    spans:
+    - operationName: GreeterBlockingError.sayHello/server/Response/onClose
+      operationId: 0
+      parentSpanId: 0
+      spanId: 1
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: true
+      spanType: Local
+      peer: ''
+      peerId: 0
+      tags:
+      - { key: status_code, value: UNKNOWN }
+      logs:
+      - logEvent:
+        - { key: event, value: error }
+        - { key: error.kind, value: java.lang.Exception }
+        - { key: message, value: '' }
+        - key: stack
+          value: not null
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: GreeterBlockingError.sayHello, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossThread, parentSpanId: 0, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+    - operationName: GreeterBlockingError.sayHello/server/Request/onComplete
+      operationId: 0
+      parentSpanId: -1
+      spanId: 0
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Local
+      peer: ''
+      peerId: 0
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: GreeterBlockingError.sayHello, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossThread, parentSpanId: 0, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+  - segmentId: not null
+    spans:
+    - operationName: Greeter.sayHello
+      operationId: 0
+      parentSpanId: 0
+      spanId: 1
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Exit
+      peer: '127.0.0.1:18080'
+      peerId: 0
+    - {operationName: GreeterBlocking.sayHello/client/Request/onComplete, operationId: 0,
+       parentSpanId: 2, spanId: 3, spanLayer: RPCFramework, startTime: nq 0,
+       endTime: nq 0, componentId: 23, componentName: '', isError: false,
+       spanType: Local, peer: '', peerId: 0}
+    - {operationName: GreeterBlocking.sayHello/client/Response/onClose, operationId: 0,
 
 Review comment:
   > Could you follow the same pattern? I prefer we keep this consistent same in all test cases.
   
   Yes, of course.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [skywalking] codecov-io edited a comment on issue #4177: Enhance gRPC plugin

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #4177: Enhance gRPC plugin
URL: https://github.com/apache/skywalking/pull/4177#issuecomment-575098453
 
 
   # [Codecov](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=h1) Report
   > Merging [#4177](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=desc) into [master](https://codecov.io/gh/apache/skywalking/commit/3d11d83299211f04e4cde3e4198607e3ed2dfac8?src=pr&el=desc) will **increase** coverage by `0.1%`.
   > The diff coverage is `n/a`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/skywalking/pull/4177/graphs/tree.svg?width=650&token=qrILxY5yA8&height=150&src=pr)](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff            @@
   ##           master    #4177     +/-   ##
   =========================================
   + Coverage   26.73%   26.83%   +0.1%     
   =========================================
     Files        1162     1162             
     Lines       25448    25448             
     Branches     3630     3630             
   =========================================
   + Hits         6803     6829     +26     
   + Misses      18041    18014     -27     
   - Partials      604      605      +1
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [...walking/apm/agent/core/context/TracingContext.java](https://codecov.io/gh/apache/skywalking/pull/4177/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLWFnZW50LWNvcmUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvYXBtL2FnZW50L2NvcmUvY29udGV4dC9UcmFjaW5nQ29udGV4dC5qYXZh) | `60.88% <0%> (-0.45%)` | :arrow_down: |
   | [...m/agent/core/remote/TraceSegmentServiceClient.java](https://codecov.io/gh/apache/skywalking/pull/4177/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLWFnZW50LWNvcmUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvYXBtL2FnZW50L2NvcmUvcmVtb3RlL1RyYWNlU2VnbWVudFNlcnZpY2VDbGllbnQuamF2YQ==) | `82.35% <0%> (+1.47%)` | :arrow_up: |
   | [.../core/remote/ServiceAndEndpointRegisterClient.java](https://codecov.io/gh/apache/skywalking/pull/4177/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLWFnZW50LWNvcmUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvYXBtL2FnZW50L2NvcmUvcmVtb3RlL1NlcnZpY2VBbmRFbmRwb2ludFJlZ2lzdGVyQ2xpZW50LmphdmE=) | `31.46% <0%> (+3.37%)` | :arrow_up: |
   | [.../agent/core/profile/ProfileTaskChannelService.java](https://codecov.io/gh/apache/skywalking/pull/4177/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLWFnZW50LWNvcmUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvYXBtL2FnZW50L2NvcmUvcHJvZmlsZS9Qcm9maWxlVGFza0NoYW5uZWxTZXJ2aWNlLmphdmE=) | `36.78% <0%> (+8.04%)` | :arrow_up: |
   | [...alking/apm/agent/core/remote/AgentIDDecorator.java](https://codecov.io/gh/apache/skywalking/pull/4177/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLWFnZW50LWNvcmUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvYXBtL2FnZW50L2NvcmUvcmVtb3RlL0FnZW50SUREZWNvcmF0b3IuamF2YQ==) | `85.71% <0%> (+17.85%)` | :arrow_up: |
   | [...ache/skywalking/apm/agent/core/jvm/JVMService.java](https://codecov.io/gh/apache/skywalking/pull/4177/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLWFnZW50LWNvcmUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvYXBtL2FnZW50L2NvcmUvanZtL0pWTVNlcnZpY2UuamF2YQ==) | `77.04% <0%> (+18.03%)` | :arrow_up: |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=footer). Last update [3d11d83...5f00e93](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [skywalking] wu-sheng commented on a change in pull request #4177: Enhance gRPC plugin

Posted by GitBox <gi...@apache.org>.
wu-sheng commented on a change in pull request #4177: Enhance gRPC plugin
URL: https://github.com/apache/skywalking/pull/4177#discussion_r371111799
 
 

 ##########
 File path: test/plugin/scenarios/grpc-scenario/config/expectedData.yaml
 ##########
 @@ -15,239 +15,542 @@
 # limitations under the License.
 registryItems:
   applications:
-    - {grpc-scenario: 2}
+  - { grpc-scenario: 2 }
   instances:
-    - {grpc-scenario: 1}
+  - { grpc-scenario: 1 }
   operationNames:
-    - grpc-scenario: [Greeter.sayHello, GreeterBlocking.sayHello,
-                      /grpc-scenario/case/grpc-scenario]
+  - grpc-scenario: [/grpc-scenario/case/healthCheck, Greeter.sayHello, GreeterBlocking.sayHello, GreeterBlockingError.sayHello, /grpc-scenario/case/grpc-scenario]
   heartbeat: []
 segmentItems:
-  - applicationCode: grpc-scenario
-    segmentSize: gt 10
-    segments:
-      - segmentId: not null
-        spans:
-          - operationName: GreeterBlocking.sayHello
-            operationId: 0
-            parentSpanId: -1
-            spanId: 0
-            spanLayer: RPCFramework
-            startTime: nq 0
-            endTime: nq 0
-            componentId: 23
-            componentName: ''
-            isError: false
-            spanType: Entry
-            peer: ''
-            peerId: 0
-            refs:
-              - {parentEndpointId: 0, parentEndpoint: /grpc-scenario/case/grpc-scenario, networkAddressId: 0,
-                 entryEndpointId: 0, refType: CrossProcess, parentSpanId: 2, parentTraceSegmentId: not null,
-                 parentServiceInstanceId: 1, networkAddress: '127.0.0.1:18080', entryEndpoint: /grpc-scenario/case/grpc-scenario,
-                 entryServiceInstanceId: 1}
-      - segmentId: not null
-        spans:
-          - operationName: Greeter.sayHello
-            operationId: 0
-            parentSpanId: -1
-            spanId: 0
-            spanLayer: RPCFramework
-            startTime: nq 0
-            endTime: nq 0
-            componentId: 23
-            componentName: ''
-            isError: false
-            spanType: Entry
-            peer: ''
-            peerId: 0
-            refs:
-              - {parentEndpointId: 0, parentEndpoint: /grpc-scenario/case/grpc-scenario, networkAddressId: 0,
-                 entryEndpointId: 0, refType: CrossProcess, parentSpanId: 1, parentTraceSegmentId: not null,
-                 parentServiceInstanceId: 1, networkAddress: '127.0.0.1:18080', entryEndpoint: /grpc-scenario/case/grpc-scenario,
-                 entryServiceInstanceId: 1}
-      - segmentId: not null
-        spans:
-          - operationName: GreeterBlocking.sayHello/server/RequestObserver/onNext
-            operationId: 0
-            parentSpanId: -1
-            spanId: 0
-            spanLayer: Unknown
-            startTime: nq 0
-            endTime: nq 0
-            componentId: 0
-            componentName: ''
-            isError: false
-            spanType: Local
-            peer: ''
-            peerId: 0
-            refs:
-              - {parentEndpointId: 0, parentEndpoint: GreeterBlocking.sayHello, networkAddressId: 0,
-                 entryEndpointId: 0, refType: CrossThread, parentSpanId: 0, parentTraceSegmentId: not null,
-                 parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
-                 entryServiceInstanceId: 1}
-      - segmentId: not null
-        spans:
-          - operationName: Greeter.sayHello/server/RequestObserver/onNext
-            operationId: 0
-            parentSpanId: -1
-            spanId: 0
-            spanLayer: Unknown
-            startTime: nq 0
-            endTime: nq 0
-            componentId: 0
-            componentName: ''
-            isError: false
-            spanType: Local
-            peer: ''
-            peerId: 0
-            refs:
-              - {parentEndpointId: 0, parentEndpoint: Greeter.sayHello, networkAddressId: 0,
-                 entryEndpointId: 0, refType: CrossThread, parentSpanId: 0, parentTraceSegmentId: not null,
-                 parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
-                 entryServiceInstanceId: 1}
-      - segmentId: not null
-        spans:
-          - operationName: Greeter.sayHello/client/ResponseObserver/onNext
-            operationId: 0
-            parentSpanId: -1
-            spanId: 0
-            spanLayer: Unknown
-            startTime: nq 0
-            endTime: nq 0
-            componentId: 0
-            componentName: ''
-            isError: false
-            spanType: Local
-            peer: ''
-            peerId: 0
-            refs:
-              - {parentEndpointId: 0, parentEndpoint: /grpc-scenario/case/grpc-scenario, networkAddressId: 0,
-                 entryEndpointId: 0, refType: CrossThread, parentSpanId: 1, parentTraceSegmentId: not null,
-                 parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
-                 entryServiceInstanceId: 1}
-      - segmentId: not null
-        spans:
-          - operationName: Greeter.sayHello/client/ResponseObserver/onComplete
-            operationId: 0
-            parentSpanId: -1
-            spanId: 0
-            spanLayer: Unknown
-            startTime: nq 0
-            endTime: nq 0
-            componentId: 0
-            componentName: ''
-            isError: false
-            spanType: Local
-            peer: ''
-            peerId: 0
-            refs:
-              - {parentEndpointId: 0, parentEndpoint: /grpc-scenario/case/grpc-scenario, networkAddressId: 0,
-                 entryEndpointId: 0, refType: CrossThread, parentSpanId: 1, parentTraceSegmentId: not null,
-                 parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
-                 entryServiceInstanceId: 1}
-      - segmentId: not null
-        spans:
-          - operationName: Greeter.sayHello/server/RequestObserver/onNext
-            operationId: 0
-            parentSpanId: -1
-            spanId: 0
-            spanLayer: Unknown
-            startTime: nq 0
-            endTime: nq 0
-            componentId: 0
-            componentName: ''
-            isError: false
-            spanType: Local
-            peer: ''
-            peerId: 0
-            refs:
-              - {parentEndpointId: 0, parentEndpoint: Greeter.sayHello, networkAddressId: 0,
-                 entryEndpointId: 0, refType: CrossThread, parentSpanId: 0, parentTraceSegmentId: not null,
-                 parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
-                 entryServiceInstanceId: 1}
-      - segmentId: not null
-        spans:
-          - operationName: Greeter.sayHello/server/RequestObserver/onComplete
-            operationId: 0
-            parentSpanId: -1
-            spanId: 0
-            spanLayer: Unknown
-            startTime: nq 0
-            endTime: nq 0
-            componentId: 0
-            componentName: ''
-            isError: false
-            spanType: Local
-            peer: ''
-            peerId: 0
-            refs:
-              - {parentEndpointId: 0, parentEndpoint: Greeter.sayHello, networkAddressId: 0,
-                 entryEndpointId: 0, refType: CrossThread, parentSpanId: 0, parentTraceSegmentId: not null,
-                 parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
-                 entryServiceInstanceId: 1}
-      - segmentId: not null
-        spans:
-          - operationName: Greeter.sayHello
-            operationId: 0
-            parentSpanId: 0
-            spanId: 1
-            spanLayer: RPCFramework
-            startTime: nq 0
-            endTime: nq 0
-            componentId: 23
-            componentName: ''
-            isError: false
-            spanType: Exit
-            peer: '127.0.0.1:18080'
-            peerId: 0
-          - operationName: GreeterBlocking.sayHello
-            operationId: 0
-            parentSpanId: 0
-            spanId: 2
-            spanLayer: RPCFramework
-            startTime: nq 0
-            endTime: nq 0
-            componentId: 23
-            componentName: ''
-            isError: false
-            spanType: Exit
-            peer: '127.0.0.1:18080'
-            peerId: 0
-          - operationName: /grpc-scenario/case/grpc-scenario
-            operationId: 0
-            parentSpanId: -1
-            spanId: 0
-            spanLayer: Http
-            startTime: nq 0
-            endTime: nq 0
-            componentId: 1
-            componentName: ''
-            isError: false
-            spanType: Entry
-            peer: ''
-            peerId: 0
-            tags:
-              - {key: url, value: 'http://localhost:8080/grpc-scenario/case/grpc-scenario'}
-              - {key: http.method, value: GET}
-      - segmentId: not null
-        spans:
-          - operationName: Greeter.sayHello/client/ResponseObserver/onNext
-            operationId: 0
-            parentSpanId: -1
-            spanId: 0
-            spanLayer: Unknown
-            startTime: nq 0
-            endTime: nq 0
-            componentId: 0
-            componentName: ''
-            isError: false
-            spanType: Local
-            peer: ''
-            peerId: 0
-            refs:
-              - {parentEndpointId: 0, parentEndpoint: /grpc-scenario/case/grpc-scenario, networkAddressId: 0,
-                 entryEndpointId: 0, refType: CrossThread, parentSpanId: 1, parentTraceSegmentId: not null,
-                 parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
-                 entryServiceInstanceId: 1}
-
+- applicationCode: grpc-scenario
+  segmentSize: gt 10
+  segments:
+  - segmentId: not null
+    spans:
+    - operationName: /grpc-scenario/case/healthCheck
 
 Review comment:
   Health check should not be included.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [skywalking] wu-sheng commented on a change in pull request #4177: Enhance gRPC plugin

Posted by GitBox <gi...@apache.org>.
wu-sheng commented on a change in pull request #4177: Enhance gRPC plugin
URL: https://github.com/apache/skywalking/pull/4177#discussion_r371105093
 
 

 ##########
 File path: apm-sniffer/apm-sdk-plugin/grpc-1.x-plugin/src/main/java/org/apache/skywalking/apm/plugin/grpc/v1/server/TracingServerCallListener.java
 ##########
 @@ -0,0 +1,102 @@
+/*
+ * 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.apm.plugin.grpc.v1.server;
+
+import io.grpc.ForwardingServerCallListener;
+import io.grpc.MethodDescriptor;
+import io.grpc.ServerCall;
+import org.apache.skywalking.apm.agent.core.context.ContextManager;
+import org.apache.skywalking.apm.agent.core.context.ContextSnapshot;
+import org.apache.skywalking.apm.agent.core.context.trace.AbstractSpan;
+import org.apache.skywalking.apm.agent.core.context.trace.SpanLayer;
+import org.apache.skywalking.apm.network.trace.component.ComponentsDefine;
+import org.apache.skywalking.apm.plugin.grpc.v1.OperationNameFormatUtil;
+
+import static org.apache.skywalking.apm.plugin.grpc.v1.Constants.*;
+
+/**
+ * @author wang zheng, kanro
+ */
+public class TracingServerCallListener<REQUEST> extends ForwardingServerCallListener.SimpleForwardingServerCallListener<REQUEST> {
+
+    private final ContextSnapshot contextSnapshot;
+    private final MethodDescriptor.MethodType methodType;
+    private final String operationPrefix;
+
+    protected TracingServerCallListener(ServerCall.Listener<REQUEST> delegate, MethodDescriptor<REQUEST, ?> descriptor,
+                                        ContextSnapshot contextSnapshot) {
+        super(delegate);
+        this.contextSnapshot = contextSnapshot;
+        this.methodType = descriptor.getType();
+        this.operationPrefix = OperationNameFormatUtil.formatOperationName(descriptor) + SERVER;
+    }
+
+    @Override
+    public void onMessage(REQUEST message) {
+        // We just create the request on message span for client stream calls.
+        if (!methodType.clientSendsOneMessage()) {
+            try {
+                final AbstractSpan span = ContextManager.createLocalSpan(operationPrefix + REQUEST_ON_MESSAGE_OPERATION_NAME);
+                span.setComponent(ComponentsDefine.GRPC);
+                span.setLayer(SpanLayer.RPC_FRAMEWORK);
+                ContextManager.continued(contextSnapshot);
+                super.onMessage(message);
+            } catch (Throwable t) {
+                ContextManager.activeSpan().errorOccurred().log(t);
+                throw t;
+            } finally {
+                ContextManager.stopSpan();
+            }
+        } else {
+            super.onMessage(message);
+        }
+    }
+
+    @Override
+    public void onCancel() {
+        try {
+            final AbstractSpan span = ContextManager.createLocalSpan(operationPrefix + REQUEST_ON_CANCEL_OPERATION_NAME);
+            span.setComponent(ComponentsDefine.GRPC);
+            span.setLayer(SpanLayer.RPC_FRAMEWORK);
+            ContextManager.continued(contextSnapshot);
+            super.onCancel();
+        } catch (Throwable t) {
+            ContextManager.activeSpan().errorOccurred().log(t);
+            throw t;
+        } finally {
+            ContextManager.stopSpan();
+        }
+    }
+
+    @Override
+    public void onHalfClose() {
+        try {
+            final AbstractSpan span = ContextManager.createLocalSpan(operationPrefix + REQUEST_ON_COMPLETE_OPERATION_NAME);
+            span.setComponent(ComponentsDefine.GRPC);
+            span.setLayer(SpanLayer.RPC_FRAMEWORK);
+            ContextManager.continued(contextSnapshot);
+            super.onHalfClose();
 
 Review comment:
   A similar suggestion, `try/final` should be added here.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [skywalking] codecov-io edited a comment on issue #4177: Enhance gRPC plugin

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #4177: Enhance gRPC plugin
URL: https://github.com/apache/skywalking/pull/4177#issuecomment-570947001
 
 
   # [Codecov](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=h1) Report
   > Merging [#4177](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=desc) into [master](https://codecov.io/gh/apache/skywalking/commit/9f3c9e1ab09f0cb23cf936d1ca9fd8f098bfb00d?src=pr&el=desc) will **decrease** coverage by `<.01%`.
   > The diff coverage is `n/a`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/skywalking/pull/4177/graphs/tree.svg?width=650&token=qrILxY5yA8&height=150&src=pr)](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #4177      +/-   ##
   ==========================================
   - Coverage   26.29%   26.28%   -0.01%     
   ==========================================
     Files        1179     1179              
     Lines       25863    25863              
     Branches     3753     3753              
   ==========================================
   - Hits         6800     6799       -1     
     Misses      18460    18460              
   - Partials      603      604       +1
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [...pm/agent/core/profile/ProfileTaskQueryService.java](https://codecov.io/gh/apache/skywalking/pull/4177/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLWFnZW50LWNvcmUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvYXBtL2FnZW50L2NvcmUvcHJvZmlsZS9Qcm9maWxlVGFza1F1ZXJ5U2VydmljZS5qYXZh) | `43.58% <0%> (-2.57%)` | :arrow_down: |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=footer). Last update [9f3c9e1...b559dbb](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [skywalking] wu-sheng commented on issue #4177: Enhance gRPC plugin

Posted by GitBox <gi...@apache.org>.
wu-sheng commented on issue #4177: Enhance gRPC plugin
URL: https://github.com/apache/skywalking/pull/4177#issuecomment-571876290
 
 
   > Provide the internal or external gRPC server tracing mode.
   
   What is this? Could you explain a little more?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [skywalking] codecov-io edited a comment on issue #4177: Enhance gRPC plugin

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #4177: Enhance gRPC plugin
URL: https://github.com/apache/skywalking/pull/4177#issuecomment-570947001
 
 
   # [Codecov](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=h1) Report
   > Merging [#4177](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=desc) into [master](https://codecov.io/gh/apache/skywalking/commit/5a1d453a53bba15397d708d6558bdf28500320ae?src=pr&el=desc) will **increase** coverage by `<.01%`.
   > The diff coverage is `n/a`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/skywalking/pull/4177/graphs/tree.svg?width=650&token=qrILxY5yA8&height=150&src=pr)](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #4177      +/-   ##
   ==========================================
   + Coverage   26.36%   26.37%   +<.01%     
   ==========================================
     Files        1179     1179              
     Lines       25863    25863              
     Branches     3753     3753              
   ==========================================
   + Hits         6819     6821       +2     
   + Misses      18440    18438       -2     
     Partials      604      604
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [...pm/agent/core/profile/ProfileTaskQueryService.java](https://codecov.io/gh/apache/skywalking/pull/4177/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLWFnZW50LWNvcmUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvYXBtL2FnZW50L2NvcmUvcHJvZmlsZS9Qcm9maWxlVGFza1F1ZXJ5U2VydmljZS5qYXZh) | `48.71% <0%> (+5.12%)` | :arrow_up: |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=footer). Last update [5a1d453...aef265d](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [skywalking] devkanro commented on issue #4177: Enhance gRPC plugin

Posted by GitBox <gi...@apache.org>.
devkanro commented on issue #4177: Enhance gRPC plugin
URL: https://github.com/apache/skywalking/pull/4177#issuecomment-573498339
 
 
   @wu-sheng 
   If you trace the both of client and server-side, you would get the tracing like blow of bi-streaming call:
   ```
   +
   |
   +-+(Server1) Exit: foo.bar.MyApi.echo
   | |
   | +-+(Server2) Entry: foo.bar.MyApi.echo
   |   |
   |   +-+(Server2) Local: foo.bar.MyApi.echo/server/Request/onMessage
   |   | |
   |   | +-+(Server2) Local: foo.bar.MyApi.echo/server/Response/onMessage
   |   |
   |   +-+(Server2) Local: foo.bar.MyApi.echo/server/Request/onMessage
   |   | |
   |   | +-+(Server2) Local: foo.bar.MyApi.echo/server/Response/onMessage
   |   |
   |   +-+(Server2) Local: foo.bar.MyApi.echo/server/Request/onComplete
   |     |
   |     +-+(Server2) Local: foo.bar.MyApi.echo/server/Response/onClose
   |
   +-+(Server1) Local: foo.bar.MyApi.echo/client/Request/onMessage
   |
   +-+(Server1) Local: foo.bar.MyApi.echo/client/Resoponse/onMessage
   |
   +-+(Server1) Local: foo.bar.MyApi.echo/client/Request/onMessage
   |
   +-+(Server1) Local: foo.bar.MyApi.echo/client/Resoponse/onMessage
   |
   +-+(Server1) Local: foo.bar.MyApi.echo/client/Request/onComplete
   |
   +-+(Server1) Local: foo.bar.MyApi.echo/client/Response/onClose
   ```
   
   A call will be traced both in server1(client-side), and server2(server-side). They are redundant, and the server-side is more accurate than the client-side because of the gRPC framework. You can distinguish the server active message(parent is entry span) and passive  message(parent is request message span).
   
   In the internal server case, we don't need the client-side tracing, but in the external server case, client-side tracing is the only way to tracing gRPC call.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [skywalking] devkanro commented on issue #4177: Enhance gRPC plugin

Posted by GitBox <gi...@apache.org>.
devkanro commented on issue #4177: Enhance gRPC plugin
URL: https://github.com/apache/skywalking/pull/4177#issuecomment-573524127
 
 
   I agree with you, the error of unreachable or network perf unstable issues are also important for service.
   
   I can simplify client-side tracing for this, like no `onMessage` event, only `Complete/Close` event for internal client-side tracing.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [skywalking] codecov-io edited a comment on issue #4177: Enhance gRPC plugin

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #4177: Enhance gRPC plugin
URL: https://github.com/apache/skywalking/pull/4177#issuecomment-575098453
 
 
   # [Codecov](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=h1) Report
   > Merging [#4177](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=desc) into [master](https://codecov.io/gh/apache/skywalking/commit/d0ef0dff770e4342ad98a643eb19b9ccd5a5fb69?src=pr&el=desc) will **decrease** coverage by `0.01%`.
   > The diff coverage is `0%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/skywalking/pull/4177/graphs/tree.svg?width=650&token=qrILxY5yA8&height=150&src=pr)](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #4177      +/-   ##
   ==========================================
   - Coverage   26.74%   26.72%   -0.02%     
   ==========================================
     Files        1162     1162              
     Lines       25446    25448       +2     
     Branches     3630     3630              
   ==========================================
   - Hits         6806     6802       -4     
   - Misses      18034    18041       +7     
   + Partials      606      605       -1
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [...g/apm/plugin/grpc/v1/client/TracingClientCall.java](https://codecov.io/gh/apache/skywalking/pull/4177/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLXNkay1wbHVnaW4vZ3JwYy0xLngtcGx1Z2luL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9za3l3YWxraW5nL2FwbS9wbHVnaW4vZ3JwYy92MS9jbGllbnQvVHJhY2luZ0NsaWVudENhbGwuamF2YQ==) | `0% <0%> (ø)` | :arrow_up: |
   | [...ugin/grpc/v1/server/TracingServerCallListener.java](https://codecov.io/gh/apache/skywalking/pull/4177/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLXNkay1wbHVnaW4vZ3JwYy0xLngtcGx1Z2luL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9za3l3YWxraW5nL2FwbS9wbHVnaW4vZ3JwYy92MS9zZXJ2ZXIvVHJhY2luZ1NlcnZlckNhbGxMaXN0ZW5lci5qYXZh) | `0% <0%> (ø)` | :arrow_up: |
   | [...g/apm/plugin/grpc/v1/server/TracingServerCall.java](https://codecov.io/gh/apache/skywalking/pull/4177/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLXNkay1wbHVnaW4vZ3JwYy0xLngtcGx1Z2luL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9za3l3YWxraW5nL2FwbS9wbHVnaW4vZ3JwYy92MS9zZXJ2ZXIvVHJhY2luZ1NlcnZlckNhbGwuamF2YQ==) | `0% <0%> (ø)` | :arrow_up: |
   | [.../core/remote/ServiceAndEndpointRegisterClient.java](https://codecov.io/gh/apache/skywalking/pull/4177/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLWFnZW50LWNvcmUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvYXBtL2FnZW50L2NvcmUvcmVtb3RlL1NlcnZpY2VBbmRFbmRwb2ludFJlZ2lzdGVyQ2xpZW50LmphdmE=) | `28.08% <0%> (-3.38%)` | :arrow_down: |
   | [...m/agent/core/remote/TraceSegmentServiceClient.java](https://codecov.io/gh/apache/skywalking/pull/4177/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLWFnZW50LWNvcmUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvYXBtL2FnZW50L2NvcmUvcmVtb3RlL1RyYWNlU2VnbWVudFNlcnZpY2VDbGllbnQuamF2YQ==) | `80.88% <0%> (-1.48%)` | :arrow_down: |
   | [...ache/skywalking/apm/agent/core/jvm/JVMService.java](https://codecov.io/gh/apache/skywalking/pull/4177/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLWFnZW50LWNvcmUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvYXBtL2FnZW50L2NvcmUvanZtL0pWTVNlcnZpY2UuamF2YQ==) | `59.01% <0%> (ø)` | :arrow_up: |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=footer). Last update [d0ef0df...2fae778](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [skywalking] wu-sheng commented on issue #4177: Enhance gRPC plugin

Posted by GitBox <gi...@apache.org>.
wu-sheng commented on issue #4177: Enhance gRPC plugin
URL: https://github.com/apache/skywalking/pull/4177#issuecomment-580172776
 
 
   @kezhenxu94 Could you recheck this recently?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [skywalking] wu-sheng commented on a change in pull request #4177: Enhance gRPC plugin

Posted by GitBox <gi...@apache.org>.
wu-sheng commented on a change in pull request #4177: Enhance gRPC plugin
URL: https://github.com/apache/skywalking/pull/4177#discussion_r371669532
 
 

 ##########
 File path: test/plugin/scenarios/grpc-scenario/config/expectedData.yaml
 ##########
 @@ -15,239 +15,524 @@
 # limitations under the License.
 registryItems:
   applications:
-    - {grpc-scenario: 2}
+  - { grpc-scenario: 2 }
   instances:
-    - {grpc-scenario: 1}
+  - { grpc-scenario: 1 }
   operationNames:
-    - grpc-scenario: [Greeter.sayHello, GreeterBlocking.sayHello,
-                      /grpc-scenario/case/grpc-scenario]
+  - grpc-scenario: [Greeter.sayHello, GreeterBlocking.sayHello, GreeterBlockingError.sayHello, /grpc-scenario/case/grpc-scenario]
   heartbeat: []
 segmentItems:
-  - applicationCode: grpc-scenario
-    segmentSize: gt 10
-    segments:
-      - segmentId: not null
-        spans:
-          - operationName: GreeterBlocking.sayHello
-            operationId: 0
-            parentSpanId: -1
-            spanId: 0
-            spanLayer: RPCFramework
-            startTime: nq 0
-            endTime: nq 0
-            componentId: 23
-            componentName: ''
-            isError: false
-            spanType: Entry
-            peer: ''
-            peerId: 0
-            refs:
-              - {parentEndpointId: 0, parentEndpoint: /grpc-scenario/case/grpc-scenario, networkAddressId: 0,
-                 entryEndpointId: 0, refType: CrossProcess, parentSpanId: 2, parentTraceSegmentId: not null,
-                 parentServiceInstanceId: 1, networkAddress: '127.0.0.1:18080', entryEndpoint: /grpc-scenario/case/grpc-scenario,
-                 entryServiceInstanceId: 1}
-      - segmentId: not null
-        spans:
-          - operationName: Greeter.sayHello
-            operationId: 0
-            parentSpanId: -1
-            spanId: 0
-            spanLayer: RPCFramework
-            startTime: nq 0
-            endTime: nq 0
-            componentId: 23
-            componentName: ''
-            isError: false
-            spanType: Entry
-            peer: ''
-            peerId: 0
-            refs:
-              - {parentEndpointId: 0, parentEndpoint: /grpc-scenario/case/grpc-scenario, networkAddressId: 0,
-                 entryEndpointId: 0, refType: CrossProcess, parentSpanId: 1, parentTraceSegmentId: not null,
-                 parentServiceInstanceId: 1, networkAddress: '127.0.0.1:18080', entryEndpoint: /grpc-scenario/case/grpc-scenario,
-                 entryServiceInstanceId: 1}
-      - segmentId: not null
-        spans:
-          - operationName: GreeterBlocking.sayHello/server/RequestObserver/onNext
-            operationId: 0
-            parentSpanId: -1
-            spanId: 0
-            spanLayer: Unknown
-            startTime: nq 0
-            endTime: nq 0
-            componentId: 0
-            componentName: ''
-            isError: false
-            spanType: Local
-            peer: ''
-            peerId: 0
-            refs:
-              - {parentEndpointId: 0, parentEndpoint: GreeterBlocking.sayHello, networkAddressId: 0,
-                 entryEndpointId: 0, refType: CrossThread, parentSpanId: 0, parentTraceSegmentId: not null,
-                 parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
-                 entryServiceInstanceId: 1}
-      - segmentId: not null
-        spans:
-          - operationName: Greeter.sayHello/server/RequestObserver/onNext
-            operationId: 0
-            parentSpanId: -1
-            spanId: 0
-            spanLayer: Unknown
-            startTime: nq 0
-            endTime: nq 0
-            componentId: 0
-            componentName: ''
-            isError: false
-            spanType: Local
-            peer: ''
-            peerId: 0
-            refs:
-              - {parentEndpointId: 0, parentEndpoint: Greeter.sayHello, networkAddressId: 0,
-                 entryEndpointId: 0, refType: CrossThread, parentSpanId: 0, parentTraceSegmentId: not null,
-                 parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
-                 entryServiceInstanceId: 1}
-      - segmentId: not null
-        spans:
-          - operationName: Greeter.sayHello/client/ResponseObserver/onNext
-            operationId: 0
-            parentSpanId: -1
-            spanId: 0
-            spanLayer: Unknown
-            startTime: nq 0
-            endTime: nq 0
-            componentId: 0
-            componentName: ''
-            isError: false
-            spanType: Local
-            peer: ''
-            peerId: 0
-            refs:
-              - {parentEndpointId: 0, parentEndpoint: /grpc-scenario/case/grpc-scenario, networkAddressId: 0,
-                 entryEndpointId: 0, refType: CrossThread, parentSpanId: 1, parentTraceSegmentId: not null,
-                 parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
-                 entryServiceInstanceId: 1}
-      - segmentId: not null
-        spans:
-          - operationName: Greeter.sayHello/client/ResponseObserver/onComplete
-            operationId: 0
-            parentSpanId: -1
-            spanId: 0
-            spanLayer: Unknown
-            startTime: nq 0
-            endTime: nq 0
-            componentId: 0
-            componentName: ''
-            isError: false
-            spanType: Local
-            peer: ''
-            peerId: 0
-            refs:
-              - {parentEndpointId: 0, parentEndpoint: /grpc-scenario/case/grpc-scenario, networkAddressId: 0,
-                 entryEndpointId: 0, refType: CrossThread, parentSpanId: 1, parentTraceSegmentId: not null,
-                 parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
-                 entryServiceInstanceId: 1}
-      - segmentId: not null
-        spans:
-          - operationName: Greeter.sayHello/server/RequestObserver/onNext
-            operationId: 0
-            parentSpanId: -1
-            spanId: 0
-            spanLayer: Unknown
-            startTime: nq 0
-            endTime: nq 0
-            componentId: 0
-            componentName: ''
-            isError: false
-            spanType: Local
-            peer: ''
-            peerId: 0
-            refs:
-              - {parentEndpointId: 0, parentEndpoint: Greeter.sayHello, networkAddressId: 0,
-                 entryEndpointId: 0, refType: CrossThread, parentSpanId: 0, parentTraceSegmentId: not null,
-                 parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
-                 entryServiceInstanceId: 1}
-      - segmentId: not null
-        spans:
-          - operationName: Greeter.sayHello/server/RequestObserver/onComplete
-            operationId: 0
-            parentSpanId: -1
-            spanId: 0
-            spanLayer: Unknown
-            startTime: nq 0
-            endTime: nq 0
-            componentId: 0
-            componentName: ''
-            isError: false
-            spanType: Local
-            peer: ''
-            peerId: 0
-            refs:
-              - {parentEndpointId: 0, parentEndpoint: Greeter.sayHello, networkAddressId: 0,
-                 entryEndpointId: 0, refType: CrossThread, parentSpanId: 0, parentTraceSegmentId: not null,
-                 parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
-                 entryServiceInstanceId: 1}
-      - segmentId: not null
-        spans:
-          - operationName: Greeter.sayHello
-            operationId: 0
-            parentSpanId: 0
-            spanId: 1
-            spanLayer: RPCFramework
-            startTime: nq 0
-            endTime: nq 0
-            componentId: 23
-            componentName: ''
-            isError: false
-            spanType: Exit
-            peer: '127.0.0.1:18080'
-            peerId: 0
-          - operationName: GreeterBlocking.sayHello
-            operationId: 0
-            parentSpanId: 0
-            spanId: 2
-            spanLayer: RPCFramework
-            startTime: nq 0
-            endTime: nq 0
-            componentId: 23
-            componentName: ''
-            isError: false
-            spanType: Exit
-            peer: '127.0.0.1:18080'
-            peerId: 0
-          - operationName: /grpc-scenario/case/grpc-scenario
-            operationId: 0
-            parentSpanId: -1
-            spanId: 0
-            spanLayer: Http
-            startTime: nq 0
-            endTime: nq 0
-            componentId: 1
-            componentName: ''
-            isError: false
-            spanType: Entry
-            peer: ''
-            peerId: 0
-            tags:
-              - {key: url, value: 'http://localhost:8080/grpc-scenario/case/grpc-scenario'}
-              - {key: http.method, value: GET}
-      - segmentId: not null
-        spans:
-          - operationName: Greeter.sayHello/client/ResponseObserver/onNext
-            operationId: 0
-            parentSpanId: -1
-            spanId: 0
-            spanLayer: Unknown
-            startTime: nq 0
-            endTime: nq 0
-            componentId: 0
-            componentName: ''
-            isError: false
-            spanType: Local
-            peer: ''
-            peerId: 0
-            refs:
-              - {parentEndpointId: 0, parentEndpoint: /grpc-scenario/case/grpc-scenario, networkAddressId: 0,
-                 entryEndpointId: 0, refType: CrossThread, parentSpanId: 1, parentTraceSegmentId: not null,
-                 parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
-                 entryServiceInstanceId: 1}
-
+- applicationCode: grpc-scenario
+  segmentSize: gt 10
+  segments:
+  - segmentId: not null
+    spans:
+    - operationName: GreeterBlocking.sayHello
+      operationId: 0
+      parentSpanId: -1
+      spanId: 0
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Entry
+      peer: ''
+      peerId: 0
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: /grpc-scenario/case/grpc-scenario, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossProcess, parentSpanId: 2, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '127.0.0.1:18080', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+  - segmentId: not null
+    spans:
+    - operationName: Greeter.sayHello/client/Request/onMessage
+      operationId: 0
+      parentSpanId: -1
+      spanId: 0
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Local
+      peer: ''
+      peerId: 0
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: /grpc-scenario/case/grpc-scenario, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossThread, parentSpanId: 1, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+  - segmentId: not null
+    spans:
+    - operationName: Greeter.sayHello/client/Request/onMessage
+      operationId: 0
+      parentSpanId: -1
+      spanId: 0
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Local
+      peer: ''
+      peerId: 0
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: /grpc-scenario/case/grpc-scenario, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossThread, parentSpanId: 1, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+  - segmentId: not null
+    spans:
+    - operationName: Greeter.sayHello/client/Request/onComplete
+      operationId: 0
+      parentSpanId: -1
+      spanId: 0
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Local
+      peer: ''
+      peerId: 0
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: /grpc-scenario/case/grpc-scenario, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossThread, parentSpanId: 1, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+  - segmentId: not null
+    spans:
+    - operationName: Greeter.sayHello
+      operationId: 0
+      parentSpanId: -1
+      spanId: 0
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Entry
+      peer: ''
+      peerId: 0
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: /grpc-scenario/case/grpc-scenario, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossProcess, parentSpanId: 1, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '127.0.0.1:18080', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+  - segmentId: not null
+    spans:
+    - operationName: Greeter.sayHello/server/Response/onMessage
+      operationId: 0
+      parentSpanId: 0
+      spanId: 1
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Local
+      peer: ''
+      peerId: 0
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: Greeter.sayHello, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossThread, parentSpanId: 0, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+    - operationName: Greeter.sayHello/server/Request/onMessage
+      operationId: 0
+      parentSpanId: -1
+      spanId: 0
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Local
+      peer: ''
+      peerId: 0
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: Greeter.sayHello, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossThread, parentSpanId: 0, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+  - segmentId: not null
+    spans:
+    - operationName: Greeter.sayHello/client/Response/onMessage
+      operationId: 0
+      parentSpanId: -1
+      spanId: 0
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Local
+      peer: ''
+      peerId: 0
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: /grpc-scenario/case/grpc-scenario, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossThread, parentSpanId: 1, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+  - segmentId: not null
+    spans:
+    - operationName: GreeterBlocking.sayHello/server/Response/onClose
+      operationId: 0
+      parentSpanId: 0
+      spanId: 1
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Local
+      peer: ''
+      peerId: 0
+      tags:
+      - { key: status_code, value: OK }
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: GreeterBlocking.sayHello, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossThread, parentSpanId: 0, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+    - operationName: GreeterBlocking.sayHello/server/Request/onComplete
+      operationId: 0
+      parentSpanId: -1
+      spanId: 0
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Local
+      peer: ''
+      peerId: 0
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: GreeterBlocking.sayHello, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossThread, parentSpanId: 0, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+  - segmentId: not null
+    spans:
+    - operationName: Greeter.sayHello/server/Response/onMessage
+      operationId: 0
+      parentSpanId: 0
+      spanId: 1
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Local
+      peer: ''
+      peerId: 0
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: Greeter.sayHello, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossThread, parentSpanId: 0, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+    - operationName: Greeter.sayHello/server/Request/onMessage
+      operationId: 0
+      parentSpanId: -1
+      spanId: 0
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Local
+      peer: ''
+      peerId: 0
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: Greeter.sayHello, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossThread, parentSpanId: 0, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+  - segmentId: not null
+    spans:
+    - operationName: Greeter.sayHello/server/Response/onClose
+      operationId: 0
+      parentSpanId: 0
+      spanId: 1
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Local
+      peer: ''
+      peerId: 0
+      tags:
+      - {key: status_code, value: OK}
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: Greeter.sayHello, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossThread, parentSpanId: 0, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+    - operationName: Greeter.sayHello/server/Request/onComplete
+      operationId: 0
+      parentSpanId: -1
+      spanId: 0
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Local
+      peer: ''
+      peerId: 0
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: Greeter.sayHello, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossThread, parentSpanId: 0, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+  - segmentId: not null
+    spans:
+    - operationName: Greeter.sayHello/client/Response/onMessage
+      operationId: 0
+      parentSpanId: -1
+      spanId: 0
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Local
+      peer: ''
+      peerId: 0
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: /grpc-scenario/case/grpc-scenario, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossThread, parentSpanId: 1, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+  - segmentId: not null
+    spans:
+    - operationName: Greeter.sayHello/client/Response/onClose
+      operationId: 0
+      parentSpanId: -1
+      spanId: 0
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Local
+      peer: ''
+      peerId: 0
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: /grpc-scenario/case/grpc-scenario, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossThread, parentSpanId: 1, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+  - segmentId: not null
+    spans:
+    - operationName: GreeterBlockingError.sayHello
+      operationId: 0
+      parentSpanId: -1
+      spanId: 0
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Entry
+      peer: ''
+      peerId: 0
+      refs:
+        - {parentEndpointId: 0, parentEndpoint: /grpc-scenario/case/grpc-scenario, networkAddressId: 0,
+           entryEndpointId: 0, refType: CrossProcess, parentSpanId: 5, parentTraceSegmentId: not null,
+           parentServiceInstanceId: 1, networkAddress: '127.0.0.1:18080', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+           entryServiceInstanceId: 1}
+  - segmentId: not null
+    spans:
+    - operationName: GreeterBlockingError.sayHello/server/Response/onClose
+      operationId: 0
+      parentSpanId: 0
+      spanId: 1
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: true
+      spanType: Local
+      peer: ''
+      peerId: 0
+      tags:
+      - { key: status_code, value: UNKNOWN }
+      logs:
+      - logEvent:
+        - { key: event, value: error }
+        - { key: error.kind, value: java.lang.Exception }
+        - { key: message, value: '' }
+        - key: stack
+          value: not null
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: GreeterBlockingError.sayHello, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossThread, parentSpanId: 0, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+    - operationName: GreeterBlockingError.sayHello/server/Request/onComplete
+      operationId: 0
+      parentSpanId: -1
+      spanId: 0
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Local
+      peer: ''
+      peerId: 0
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: GreeterBlockingError.sayHello, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossThread, parentSpanId: 0, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+  - segmentId: not null
+    spans:
+    - operationName: Greeter.sayHello
+      operationId: 0
+      parentSpanId: 0
+      spanId: 1
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Exit
+      peer: '127.0.0.1:18080'
+      peerId: 0
+    - {operationName: GreeterBlocking.sayHello/client/Request/onComplete, operationId: 0,
+       parentSpanId: 2, spanId: 3, spanLayer: RPCFramework, startTime: nq 0,
+       endTime: nq 0, componentId: 23, componentName: '', isError: false,
+       spanType: Local, peer: '', peerId: 0}
+    - {operationName: GreeterBlocking.sayHello/client/Response/onClose, operationId: 0,
+       parentSpanId: 2, spanId: 4, spanLayer: RPCFramework, startTime: nq 0,
+       endTime: nq 0, componentId: 23, componentName: '', isError: false,
+       spanType: Local, peer: '', peerId: 0}
+    - operationName: GreeterBlocking.sayHello
+      operationId: 0
+      parentSpanId: 0
+      spanId: 2
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Exit
+      peer: '127.0.0.1:18080'
+      peerId: 0
+    - {operationName: GreeterBlockingError.sayHello/client/Request/onComplete, operationId: 0,
 
 Review comment:
   This one too.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [skywalking] wu-sheng commented on a change in pull request #4177: Enhance gRPC plugin

Posted by GitBox <gi...@apache.org>.
wu-sheng commented on a change in pull request #4177: Enhance gRPC plugin
URL: https://github.com/apache/skywalking/pull/4177#discussion_r371777178
 
 

 ##########
 File path: test/plugin/scenarios/grpc-scenario/config/expectedData.yaml
 ##########
 @@ -15,239 +15,524 @@
 # limitations under the License.
 registryItems:
   applications:
-    - {grpc-scenario: 2}
+  - { grpc-scenario: 2 }
   instances:
-    - {grpc-scenario: 1}
+  - { grpc-scenario: 1 }
   operationNames:
-    - grpc-scenario: [Greeter.sayHello, GreeterBlocking.sayHello,
-                      /grpc-scenario/case/grpc-scenario]
+  - grpc-scenario: [Greeter.sayHello, GreeterBlocking.sayHello, GreeterBlockingError.sayHello, /grpc-scenario/case/grpc-scenario]
   heartbeat: []
 segmentItems:
-  - applicationCode: grpc-scenario
-    segmentSize: gt 10
-    segments:
-      - segmentId: not null
-        spans:
-          - operationName: GreeterBlocking.sayHello
-            operationId: 0
-            parentSpanId: -1
-            spanId: 0
-            spanLayer: RPCFramework
-            startTime: nq 0
-            endTime: nq 0
-            componentId: 23
-            componentName: ''
-            isError: false
-            spanType: Entry
-            peer: ''
-            peerId: 0
-            refs:
-              - {parentEndpointId: 0, parentEndpoint: /grpc-scenario/case/grpc-scenario, networkAddressId: 0,
-                 entryEndpointId: 0, refType: CrossProcess, parentSpanId: 2, parentTraceSegmentId: not null,
-                 parentServiceInstanceId: 1, networkAddress: '127.0.0.1:18080', entryEndpoint: /grpc-scenario/case/grpc-scenario,
-                 entryServiceInstanceId: 1}
-      - segmentId: not null
-        spans:
-          - operationName: Greeter.sayHello
-            operationId: 0
-            parentSpanId: -1
-            spanId: 0
-            spanLayer: RPCFramework
-            startTime: nq 0
-            endTime: nq 0
-            componentId: 23
-            componentName: ''
-            isError: false
-            spanType: Entry
-            peer: ''
-            peerId: 0
-            refs:
-              - {parentEndpointId: 0, parentEndpoint: /grpc-scenario/case/grpc-scenario, networkAddressId: 0,
-                 entryEndpointId: 0, refType: CrossProcess, parentSpanId: 1, parentTraceSegmentId: not null,
-                 parentServiceInstanceId: 1, networkAddress: '127.0.0.1:18080', entryEndpoint: /grpc-scenario/case/grpc-scenario,
-                 entryServiceInstanceId: 1}
-      - segmentId: not null
-        spans:
-          - operationName: GreeterBlocking.sayHello/server/RequestObserver/onNext
-            operationId: 0
-            parentSpanId: -1
-            spanId: 0
-            spanLayer: Unknown
-            startTime: nq 0
-            endTime: nq 0
-            componentId: 0
-            componentName: ''
-            isError: false
-            spanType: Local
-            peer: ''
-            peerId: 0
-            refs:
-              - {parentEndpointId: 0, parentEndpoint: GreeterBlocking.sayHello, networkAddressId: 0,
-                 entryEndpointId: 0, refType: CrossThread, parentSpanId: 0, parentTraceSegmentId: not null,
-                 parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
-                 entryServiceInstanceId: 1}
-      - segmentId: not null
-        spans:
-          - operationName: Greeter.sayHello/server/RequestObserver/onNext
-            operationId: 0
-            parentSpanId: -1
-            spanId: 0
-            spanLayer: Unknown
-            startTime: nq 0
-            endTime: nq 0
-            componentId: 0
-            componentName: ''
-            isError: false
-            spanType: Local
-            peer: ''
-            peerId: 0
-            refs:
-              - {parentEndpointId: 0, parentEndpoint: Greeter.sayHello, networkAddressId: 0,
-                 entryEndpointId: 0, refType: CrossThread, parentSpanId: 0, parentTraceSegmentId: not null,
-                 parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
-                 entryServiceInstanceId: 1}
-      - segmentId: not null
-        spans:
-          - operationName: Greeter.sayHello/client/ResponseObserver/onNext
-            operationId: 0
-            parentSpanId: -1
-            spanId: 0
-            spanLayer: Unknown
-            startTime: nq 0
-            endTime: nq 0
-            componentId: 0
-            componentName: ''
-            isError: false
-            spanType: Local
-            peer: ''
-            peerId: 0
-            refs:
-              - {parentEndpointId: 0, parentEndpoint: /grpc-scenario/case/grpc-scenario, networkAddressId: 0,
-                 entryEndpointId: 0, refType: CrossThread, parentSpanId: 1, parentTraceSegmentId: not null,
-                 parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
-                 entryServiceInstanceId: 1}
-      - segmentId: not null
-        spans:
-          - operationName: Greeter.sayHello/client/ResponseObserver/onComplete
-            operationId: 0
-            parentSpanId: -1
-            spanId: 0
-            spanLayer: Unknown
-            startTime: nq 0
-            endTime: nq 0
-            componentId: 0
-            componentName: ''
-            isError: false
-            spanType: Local
-            peer: ''
-            peerId: 0
-            refs:
-              - {parentEndpointId: 0, parentEndpoint: /grpc-scenario/case/grpc-scenario, networkAddressId: 0,
-                 entryEndpointId: 0, refType: CrossThread, parentSpanId: 1, parentTraceSegmentId: not null,
-                 parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
-                 entryServiceInstanceId: 1}
-      - segmentId: not null
-        spans:
-          - operationName: Greeter.sayHello/server/RequestObserver/onNext
-            operationId: 0
-            parentSpanId: -1
-            spanId: 0
-            spanLayer: Unknown
-            startTime: nq 0
-            endTime: nq 0
-            componentId: 0
-            componentName: ''
-            isError: false
-            spanType: Local
-            peer: ''
-            peerId: 0
-            refs:
-              - {parentEndpointId: 0, parentEndpoint: Greeter.sayHello, networkAddressId: 0,
-                 entryEndpointId: 0, refType: CrossThread, parentSpanId: 0, parentTraceSegmentId: not null,
-                 parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
-                 entryServiceInstanceId: 1}
-      - segmentId: not null
-        spans:
-          - operationName: Greeter.sayHello/server/RequestObserver/onComplete
-            operationId: 0
-            parentSpanId: -1
-            spanId: 0
-            spanLayer: Unknown
-            startTime: nq 0
-            endTime: nq 0
-            componentId: 0
-            componentName: ''
-            isError: false
-            spanType: Local
-            peer: ''
-            peerId: 0
-            refs:
-              - {parentEndpointId: 0, parentEndpoint: Greeter.sayHello, networkAddressId: 0,
-                 entryEndpointId: 0, refType: CrossThread, parentSpanId: 0, parentTraceSegmentId: not null,
-                 parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
-                 entryServiceInstanceId: 1}
-      - segmentId: not null
-        spans:
-          - operationName: Greeter.sayHello
-            operationId: 0
-            parentSpanId: 0
-            spanId: 1
-            spanLayer: RPCFramework
-            startTime: nq 0
-            endTime: nq 0
-            componentId: 23
-            componentName: ''
-            isError: false
-            spanType: Exit
-            peer: '127.0.0.1:18080'
-            peerId: 0
-          - operationName: GreeterBlocking.sayHello
-            operationId: 0
-            parentSpanId: 0
-            spanId: 2
-            spanLayer: RPCFramework
-            startTime: nq 0
-            endTime: nq 0
-            componentId: 23
-            componentName: ''
-            isError: false
-            spanType: Exit
-            peer: '127.0.0.1:18080'
-            peerId: 0
-          - operationName: /grpc-scenario/case/grpc-scenario
-            operationId: 0
-            parentSpanId: -1
-            spanId: 0
-            spanLayer: Http
-            startTime: nq 0
-            endTime: nq 0
-            componentId: 1
-            componentName: ''
-            isError: false
-            spanType: Entry
-            peer: ''
-            peerId: 0
-            tags:
-              - {key: url, value: 'http://localhost:8080/grpc-scenario/case/grpc-scenario'}
-              - {key: http.method, value: GET}
-      - segmentId: not null
-        spans:
-          - operationName: Greeter.sayHello/client/ResponseObserver/onNext
-            operationId: 0
-            parentSpanId: -1
-            spanId: 0
-            spanLayer: Unknown
-            startTime: nq 0
-            endTime: nq 0
-            componentId: 0
-            componentName: ''
-            isError: false
-            spanType: Local
-            peer: ''
-            peerId: 0
-            refs:
-              - {parentEndpointId: 0, parentEndpoint: /grpc-scenario/case/grpc-scenario, networkAddressId: 0,
-                 entryEndpointId: 0, refType: CrossThread, parentSpanId: 1, parentTraceSegmentId: not null,
-                 parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
-                 entryServiceInstanceId: 1}
-
+- applicationCode: grpc-scenario
+  segmentSize: gt 10
+  segments:
+  - segmentId: not null
+    spans:
+    - operationName: GreeterBlocking.sayHello
+      operationId: 0
+      parentSpanId: -1
+      spanId: 0
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Entry
+      peer: ''
+      peerId: 0
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: /grpc-scenario/case/grpc-scenario, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossProcess, parentSpanId: 2, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '127.0.0.1:18080', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+  - segmentId: not null
+    spans:
+    - operationName: Greeter.sayHello/client/Request/onMessage
+      operationId: 0
+      parentSpanId: -1
+      spanId: 0
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Local
+      peer: ''
+      peerId: 0
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: /grpc-scenario/case/grpc-scenario, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossThread, parentSpanId: 1, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+  - segmentId: not null
+    spans:
+    - operationName: Greeter.sayHello/client/Request/onMessage
+      operationId: 0
+      parentSpanId: -1
+      spanId: 0
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Local
+      peer: ''
+      peerId: 0
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: /grpc-scenario/case/grpc-scenario, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossThread, parentSpanId: 1, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+  - segmentId: not null
+    spans:
+    - operationName: Greeter.sayHello/client/Request/onComplete
+      operationId: 0
+      parentSpanId: -1
+      spanId: 0
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Local
+      peer: ''
+      peerId: 0
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: /grpc-scenario/case/grpc-scenario, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossThread, parentSpanId: 1, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+  - segmentId: not null
+    spans:
+    - operationName: Greeter.sayHello
+      operationId: 0
+      parentSpanId: -1
+      spanId: 0
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Entry
+      peer: ''
+      peerId: 0
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: /grpc-scenario/case/grpc-scenario, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossProcess, parentSpanId: 1, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '127.0.0.1:18080', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+  - segmentId: not null
+    spans:
+    - operationName: Greeter.sayHello/server/Response/onMessage
+      operationId: 0
+      parentSpanId: 0
+      spanId: 1
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Local
+      peer: ''
+      peerId: 0
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: Greeter.sayHello, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossThread, parentSpanId: 0, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+    - operationName: Greeter.sayHello/server/Request/onMessage
+      operationId: 0
+      parentSpanId: -1
+      spanId: 0
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Local
+      peer: ''
+      peerId: 0
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: Greeter.sayHello, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossThread, parentSpanId: 0, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+  - segmentId: not null
+    spans:
+    - operationName: Greeter.sayHello/client/Response/onMessage
+      operationId: 0
+      parentSpanId: -1
+      spanId: 0
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Local
+      peer: ''
+      peerId: 0
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: /grpc-scenario/case/grpc-scenario, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossThread, parentSpanId: 1, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+  - segmentId: not null
+    spans:
+    - operationName: GreeterBlocking.sayHello/server/Response/onClose
+      operationId: 0
+      parentSpanId: 0
+      spanId: 1
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Local
+      peer: ''
+      peerId: 0
+      tags:
+      - { key: status_code, value: OK }
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: GreeterBlocking.sayHello, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossThread, parentSpanId: 0, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+    - operationName: GreeterBlocking.sayHello/server/Request/onComplete
+      operationId: 0
+      parentSpanId: -1
+      spanId: 0
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Local
+      peer: ''
+      peerId: 0
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: GreeterBlocking.sayHello, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossThread, parentSpanId: 0, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+  - segmentId: not null
+    spans:
+    - operationName: Greeter.sayHello/server/Response/onMessage
+      operationId: 0
+      parentSpanId: 0
+      spanId: 1
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Local
+      peer: ''
+      peerId: 0
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: Greeter.sayHello, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossThread, parentSpanId: 0, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+    - operationName: Greeter.sayHello/server/Request/onMessage
+      operationId: 0
+      parentSpanId: -1
+      spanId: 0
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Local
+      peer: ''
+      peerId: 0
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: Greeter.sayHello, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossThread, parentSpanId: 0, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+  - segmentId: not null
+    spans:
+    - operationName: Greeter.sayHello/server/Response/onClose
+      operationId: 0
+      parentSpanId: 0
+      spanId: 1
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Local
+      peer: ''
+      peerId: 0
+      tags:
+      - {key: status_code, value: OK}
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: Greeter.sayHello, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossThread, parentSpanId: 0, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+    - operationName: Greeter.sayHello/server/Request/onComplete
+      operationId: 0
+      parentSpanId: -1
+      spanId: 0
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Local
+      peer: ''
+      peerId: 0
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: Greeter.sayHello, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossThread, parentSpanId: 0, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+  - segmentId: not null
+    spans:
+    - operationName: Greeter.sayHello/client/Response/onMessage
+      operationId: 0
+      parentSpanId: -1
+      spanId: 0
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Local
+      peer: ''
+      peerId: 0
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: /grpc-scenario/case/grpc-scenario, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossThread, parentSpanId: 1, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+  - segmentId: not null
+    spans:
+    - operationName: Greeter.sayHello/client/Response/onClose
+      operationId: 0
+      parentSpanId: -1
+      spanId: 0
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Local
+      peer: ''
+      peerId: 0
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: /grpc-scenario/case/grpc-scenario, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossThread, parentSpanId: 1, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+  - segmentId: not null
+    spans:
+    - operationName: GreeterBlockingError.sayHello
+      operationId: 0
+      parentSpanId: -1
+      spanId: 0
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Entry
+      peer: ''
+      peerId: 0
+      refs:
+        - {parentEndpointId: 0, parentEndpoint: /grpc-scenario/case/grpc-scenario, networkAddressId: 0,
+           entryEndpointId: 0, refType: CrossProcess, parentSpanId: 5, parentTraceSegmentId: not null,
+           parentServiceInstanceId: 1, networkAddress: '127.0.0.1:18080', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+           entryServiceInstanceId: 1}
+  - segmentId: not null
+    spans:
+    - operationName: GreeterBlockingError.sayHello/server/Response/onClose
+      operationId: 0
+      parentSpanId: 0
+      spanId: 1
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: true
+      spanType: Local
+      peer: ''
+      peerId: 0
+      tags:
+      - { key: status_code, value: UNKNOWN }
+      logs:
+      - logEvent:
+        - { key: event, value: error }
+        - { key: error.kind, value: java.lang.Exception }
+        - { key: message, value: '' }
+        - key: stack
+          value: not null
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: GreeterBlockingError.sayHello, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossThread, parentSpanId: 0, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+    - operationName: GreeterBlockingError.sayHello/server/Request/onComplete
+      operationId: 0
+      parentSpanId: -1
+      spanId: 0
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Local
+      peer: ''
+      peerId: 0
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: GreeterBlockingError.sayHello, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossThread, parentSpanId: 0, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+  - segmentId: not null
+    spans:
+    - operationName: Greeter.sayHello
+      operationId: 0
+      parentSpanId: 0
+      spanId: 1
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Exit
+      peer: '127.0.0.1:18080'
+      peerId: 0
+    - {operationName: GreeterBlocking.sayHello/client/Request/onComplete, operationId: 0,
+       parentSpanId: 2, spanId: 3, spanLayer: RPCFramework, startTime: nq 0,
+       endTime: nq 0, componentId: 23, componentName: '', isError: false,
+       spanType: Local, peer: '', peerId: 0}
+    - {operationName: GreeterBlocking.sayHello/client/Response/onClose, operationId: 0,
 
 Review comment:
   Could you follow the same pattern? I prefer we keep this consistent same in all test cases.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [skywalking] devkanro commented on issue #4177: Enhance gRPC plugin

Posted by GitBox <gi...@apache.org>.
devkanro commented on issue #4177: Enhance gRPC plugin
URL: https://github.com/apache/skywalking/pull/4177#issuecomment-573500007
 
 
   The original plugin doesn't provide the client-side tracing, I add it for my case and optional powerful tracing.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [skywalking] dmsolr commented on issue #4177: Enhance gRPC plugin

Posted by GitBox <gi...@apache.org>.
dmsolr commented on issue #4177: Enhance gRPC plugin
URL: https://github.com/apache/skywalking/pull/4177#issuecomment-571453991
 
 
   This is a very good pull request template for the plugin contribution. 👍 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [skywalking] codecov-io edited a comment on issue #4177: Enhance gRPC plugin

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #4177: Enhance gRPC plugin
URL: https://github.com/apache/skywalking/pull/4177#issuecomment-575098453
 
 
   # [Codecov](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=h1) Report
   > Merging [#4177](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=desc) into [master](https://codecov.io/gh/apache/skywalking/commit/d0ef0dff770e4342ad98a643eb19b9ccd5a5fb69?src=pr&el=desc) will **increase** coverage by `0.09%`.
   > The diff coverage is `0%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/skywalking/pull/4177/graphs/tree.svg?width=650&token=qrILxY5yA8&height=150&src=pr)](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #4177      +/-   ##
   ==========================================
   + Coverage   26.74%   26.83%   +0.09%     
   ==========================================
     Files        1162     1162              
     Lines       25446    25446              
     Branches     3630     3630              
   ==========================================
   + Hits         6806     6829      +23     
   + Misses      18034    18012      -22     
   + Partials      606      605       -1
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [...g/apm/plugin/grpc/v1/client/TracingClientCall.java](https://codecov.io/gh/apache/skywalking/pull/4177/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLXNkay1wbHVnaW4vZ3JwYy0xLngtcGx1Z2luL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9za3l3YWxraW5nL2FwbS9wbHVnaW4vZ3JwYy92MS9jbGllbnQvVHJhY2luZ0NsaWVudENhbGwuamF2YQ==) | `0% <ø> (ø)` | :arrow_up: |
   | [...ugin/grpc/v1/server/TracingServerCallListener.java](https://codecov.io/gh/apache/skywalking/pull/4177/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLXNkay1wbHVnaW4vZ3JwYy0xLngtcGx1Z2luL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9za3l3YWxraW5nL2FwbS9wbHVnaW4vZ3JwYy92MS9zZXJ2ZXIvVHJhY2luZ1NlcnZlckNhbGxMaXN0ZW5lci5qYXZh) | `0% <0%> (ø)` | :arrow_up: |
   | [...g/apm/plugin/grpc/v1/server/TracingServerCall.java](https://codecov.io/gh/apache/skywalking/pull/4177/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLXNkay1wbHVnaW4vZ3JwYy0xLngtcGx1Z2luL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9za3l3YWxraW5nL2FwbS9wbHVnaW4vZ3JwYy92MS9zZXJ2ZXIvVHJhY2luZ1NlcnZlckNhbGwuamF2YQ==) | `0% <0%> (ø)` | :arrow_up: |
   | [.../agent/core/profile/ProfileTaskChannelService.java](https://codecov.io/gh/apache/skywalking/pull/4177/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLWFnZW50LWNvcmUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvYXBtL2FnZW50L2NvcmUvcHJvZmlsZS9Qcm9maWxlVGFza0NoYW5uZWxTZXJ2aWNlLmphdmE=) | `36.78% <0%> (+8.04%)` | :arrow_up: |
   | [...alking/apm/agent/core/remote/AgentIDDecorator.java](https://codecov.io/gh/apache/skywalking/pull/4177/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLWFnZW50LWNvcmUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvYXBtL2FnZW50L2NvcmUvcmVtb3RlL0FnZW50SUREZWNvcmF0b3IuamF2YQ==) | `85.71% <0%> (+17.85%)` | :arrow_up: |
   | [...ache/skywalking/apm/agent/core/jvm/JVMService.java](https://codecov.io/gh/apache/skywalking/pull/4177/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLWFnZW50LWNvcmUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvYXBtL2FnZW50L2NvcmUvanZtL0pWTVNlcnZpY2UuamF2YQ==) | `77.04% <0%> (+18.03%)` | :arrow_up: |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=footer). Last update [d0ef0df...c601e1d](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [skywalking] GuoDuanLZ commented on a change in pull request #4177: Enhance gRPC plugin

Posted by GitBox <gi...@apache.org>.
GuoDuanLZ commented on a change in pull request #4177: Enhance gRPC plugin
URL: https://github.com/apache/skywalking/pull/4177#discussion_r371717846
 
 

 ##########
 File path: test/plugin/scenarios/grpc-scenario/config/expectedData.yaml
 ##########
 @@ -15,239 +15,524 @@
 # limitations under the License.
 registryItems:
   applications:
-    - {grpc-scenario: 2}
+  - { grpc-scenario: 2 }
   instances:
-    - {grpc-scenario: 1}
+  - { grpc-scenario: 1 }
   operationNames:
-    - grpc-scenario: [Greeter.sayHello, GreeterBlocking.sayHello,
-                      /grpc-scenario/case/grpc-scenario]
+  - grpc-scenario: [Greeter.sayHello, GreeterBlocking.sayHello, GreeterBlockingError.sayHello, /grpc-scenario/case/grpc-scenario]
   heartbeat: []
 segmentItems:
-  - applicationCode: grpc-scenario
-    segmentSize: gt 10
-    segments:
-      - segmentId: not null
-        spans:
-          - operationName: GreeterBlocking.sayHello
-            operationId: 0
-            parentSpanId: -1
-            spanId: 0
-            spanLayer: RPCFramework
-            startTime: nq 0
-            endTime: nq 0
-            componentId: 23
-            componentName: ''
-            isError: false
-            spanType: Entry
-            peer: ''
-            peerId: 0
-            refs:
-              - {parentEndpointId: 0, parentEndpoint: /grpc-scenario/case/grpc-scenario, networkAddressId: 0,
-                 entryEndpointId: 0, refType: CrossProcess, parentSpanId: 2, parentTraceSegmentId: not null,
-                 parentServiceInstanceId: 1, networkAddress: '127.0.0.1:18080', entryEndpoint: /grpc-scenario/case/grpc-scenario,
-                 entryServiceInstanceId: 1}
-      - segmentId: not null
-        spans:
-          - operationName: Greeter.sayHello
-            operationId: 0
-            parentSpanId: -1
-            spanId: 0
-            spanLayer: RPCFramework
-            startTime: nq 0
-            endTime: nq 0
-            componentId: 23
-            componentName: ''
-            isError: false
-            spanType: Entry
-            peer: ''
-            peerId: 0
-            refs:
-              - {parentEndpointId: 0, parentEndpoint: /grpc-scenario/case/grpc-scenario, networkAddressId: 0,
-                 entryEndpointId: 0, refType: CrossProcess, parentSpanId: 1, parentTraceSegmentId: not null,
-                 parentServiceInstanceId: 1, networkAddress: '127.0.0.1:18080', entryEndpoint: /grpc-scenario/case/grpc-scenario,
-                 entryServiceInstanceId: 1}
-      - segmentId: not null
-        spans:
-          - operationName: GreeterBlocking.sayHello/server/RequestObserver/onNext
-            operationId: 0
-            parentSpanId: -1
-            spanId: 0
-            spanLayer: Unknown
-            startTime: nq 0
-            endTime: nq 0
-            componentId: 0
-            componentName: ''
-            isError: false
-            spanType: Local
-            peer: ''
-            peerId: 0
-            refs:
-              - {parentEndpointId: 0, parentEndpoint: GreeterBlocking.sayHello, networkAddressId: 0,
-                 entryEndpointId: 0, refType: CrossThread, parentSpanId: 0, parentTraceSegmentId: not null,
-                 parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
-                 entryServiceInstanceId: 1}
-      - segmentId: not null
-        spans:
-          - operationName: Greeter.sayHello/server/RequestObserver/onNext
-            operationId: 0
-            parentSpanId: -1
-            spanId: 0
-            spanLayer: Unknown
-            startTime: nq 0
-            endTime: nq 0
-            componentId: 0
-            componentName: ''
-            isError: false
-            spanType: Local
-            peer: ''
-            peerId: 0
-            refs:
-              - {parentEndpointId: 0, parentEndpoint: Greeter.sayHello, networkAddressId: 0,
-                 entryEndpointId: 0, refType: CrossThread, parentSpanId: 0, parentTraceSegmentId: not null,
-                 parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
-                 entryServiceInstanceId: 1}
-      - segmentId: not null
-        spans:
-          - operationName: Greeter.sayHello/client/ResponseObserver/onNext
-            operationId: 0
-            parentSpanId: -1
-            spanId: 0
-            spanLayer: Unknown
-            startTime: nq 0
-            endTime: nq 0
-            componentId: 0
-            componentName: ''
-            isError: false
-            spanType: Local
-            peer: ''
-            peerId: 0
-            refs:
-              - {parentEndpointId: 0, parentEndpoint: /grpc-scenario/case/grpc-scenario, networkAddressId: 0,
-                 entryEndpointId: 0, refType: CrossThread, parentSpanId: 1, parentTraceSegmentId: not null,
-                 parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
-                 entryServiceInstanceId: 1}
-      - segmentId: not null
-        spans:
-          - operationName: Greeter.sayHello/client/ResponseObserver/onComplete
-            operationId: 0
-            parentSpanId: -1
-            spanId: 0
-            spanLayer: Unknown
-            startTime: nq 0
-            endTime: nq 0
-            componentId: 0
-            componentName: ''
-            isError: false
-            spanType: Local
-            peer: ''
-            peerId: 0
-            refs:
-              - {parentEndpointId: 0, parentEndpoint: /grpc-scenario/case/grpc-scenario, networkAddressId: 0,
-                 entryEndpointId: 0, refType: CrossThread, parentSpanId: 1, parentTraceSegmentId: not null,
-                 parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
-                 entryServiceInstanceId: 1}
-      - segmentId: not null
-        spans:
-          - operationName: Greeter.sayHello/server/RequestObserver/onNext
-            operationId: 0
-            parentSpanId: -1
-            spanId: 0
-            spanLayer: Unknown
-            startTime: nq 0
-            endTime: nq 0
-            componentId: 0
-            componentName: ''
-            isError: false
-            spanType: Local
-            peer: ''
-            peerId: 0
-            refs:
-              - {parentEndpointId: 0, parentEndpoint: Greeter.sayHello, networkAddressId: 0,
-                 entryEndpointId: 0, refType: CrossThread, parentSpanId: 0, parentTraceSegmentId: not null,
-                 parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
-                 entryServiceInstanceId: 1}
-      - segmentId: not null
-        spans:
-          - operationName: Greeter.sayHello/server/RequestObserver/onComplete
-            operationId: 0
-            parentSpanId: -1
-            spanId: 0
-            spanLayer: Unknown
-            startTime: nq 0
-            endTime: nq 0
-            componentId: 0
-            componentName: ''
-            isError: false
-            spanType: Local
-            peer: ''
-            peerId: 0
-            refs:
-              - {parentEndpointId: 0, parentEndpoint: Greeter.sayHello, networkAddressId: 0,
-                 entryEndpointId: 0, refType: CrossThread, parentSpanId: 0, parentTraceSegmentId: not null,
-                 parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
-                 entryServiceInstanceId: 1}
-      - segmentId: not null
-        spans:
-          - operationName: Greeter.sayHello
-            operationId: 0
-            parentSpanId: 0
-            spanId: 1
-            spanLayer: RPCFramework
-            startTime: nq 0
-            endTime: nq 0
-            componentId: 23
-            componentName: ''
-            isError: false
-            spanType: Exit
-            peer: '127.0.0.1:18080'
-            peerId: 0
-          - operationName: GreeterBlocking.sayHello
-            operationId: 0
-            parentSpanId: 0
-            spanId: 2
-            spanLayer: RPCFramework
-            startTime: nq 0
-            endTime: nq 0
-            componentId: 23
-            componentName: ''
-            isError: false
-            spanType: Exit
-            peer: '127.0.0.1:18080'
-            peerId: 0
-          - operationName: /grpc-scenario/case/grpc-scenario
-            operationId: 0
-            parentSpanId: -1
-            spanId: 0
-            spanLayer: Http
-            startTime: nq 0
-            endTime: nq 0
-            componentId: 1
-            componentName: ''
-            isError: false
-            spanType: Entry
-            peer: ''
-            peerId: 0
-            tags:
-              - {key: url, value: 'http://localhost:8080/grpc-scenario/case/grpc-scenario'}
-              - {key: http.method, value: GET}
-      - segmentId: not null
-        spans:
-          - operationName: Greeter.sayHello/client/ResponseObserver/onNext
-            operationId: 0
-            parentSpanId: -1
-            spanId: 0
-            spanLayer: Unknown
-            startTime: nq 0
-            endTime: nq 0
-            componentId: 0
-            componentName: ''
-            isError: false
-            spanType: Local
-            peer: ''
-            peerId: 0
-            refs:
-              - {parentEndpointId: 0, parentEndpoint: /grpc-scenario/case/grpc-scenario, networkAddressId: 0,
-                 entryEndpointId: 0, refType: CrossThread, parentSpanId: 1, parentTraceSegmentId: not null,
-                 parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
-                 entryServiceInstanceId: 1}
-
+- applicationCode: grpc-scenario
+  segmentSize: gt 10
+  segments:
+  - segmentId: not null
+    spans:
+    - operationName: GreeterBlocking.sayHello
+      operationId: 0
+      parentSpanId: -1
+      spanId: 0
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Entry
+      peer: ''
+      peerId: 0
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: /grpc-scenario/case/grpc-scenario, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossProcess, parentSpanId: 2, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '127.0.0.1:18080', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+  - segmentId: not null
+    spans:
+    - operationName: Greeter.sayHello/client/Request/onMessage
+      operationId: 0
+      parentSpanId: -1
+      spanId: 0
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Local
+      peer: ''
+      peerId: 0
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: /grpc-scenario/case/grpc-scenario, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossThread, parentSpanId: 1, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+  - segmentId: not null
+    spans:
+    - operationName: Greeter.sayHello/client/Request/onMessage
+      operationId: 0
+      parentSpanId: -1
+      spanId: 0
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Local
+      peer: ''
+      peerId: 0
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: /grpc-scenario/case/grpc-scenario, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossThread, parentSpanId: 1, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+  - segmentId: not null
+    spans:
+    - operationName: Greeter.sayHello/client/Request/onComplete
+      operationId: 0
+      parentSpanId: -1
+      spanId: 0
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Local
+      peer: ''
+      peerId: 0
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: /grpc-scenario/case/grpc-scenario, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossThread, parentSpanId: 1, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+  - segmentId: not null
+    spans:
+    - operationName: Greeter.sayHello
+      operationId: 0
+      parentSpanId: -1
+      spanId: 0
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Entry
+      peer: ''
+      peerId: 0
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: /grpc-scenario/case/grpc-scenario, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossProcess, parentSpanId: 1, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '127.0.0.1:18080', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+  - segmentId: not null
+    spans:
+    - operationName: Greeter.sayHello/server/Response/onMessage
+      operationId: 0
+      parentSpanId: 0
+      spanId: 1
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Local
+      peer: ''
+      peerId: 0
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: Greeter.sayHello, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossThread, parentSpanId: 0, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+    - operationName: Greeter.sayHello/server/Request/onMessage
+      operationId: 0
+      parentSpanId: -1
+      spanId: 0
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Local
+      peer: ''
+      peerId: 0
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: Greeter.sayHello, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossThread, parentSpanId: 0, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+  - segmentId: not null
+    spans:
+    - operationName: Greeter.sayHello/client/Response/onMessage
+      operationId: 0
+      parentSpanId: -1
+      spanId: 0
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Local
+      peer: ''
+      peerId: 0
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: /grpc-scenario/case/grpc-scenario, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossThread, parentSpanId: 1, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+  - segmentId: not null
+    spans:
+    - operationName: GreeterBlocking.sayHello/server/Response/onClose
+      operationId: 0
+      parentSpanId: 0
+      spanId: 1
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Local
+      peer: ''
+      peerId: 0
+      tags:
+      - { key: status_code, value: OK }
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: GreeterBlocking.sayHello, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossThread, parentSpanId: 0, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+    - operationName: GreeterBlocking.sayHello/server/Request/onComplete
+      operationId: 0
+      parentSpanId: -1
+      spanId: 0
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Local
+      peer: ''
+      peerId: 0
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: GreeterBlocking.sayHello, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossThread, parentSpanId: 0, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+  - segmentId: not null
+    spans:
+    - operationName: Greeter.sayHello/server/Response/onMessage
+      operationId: 0
+      parentSpanId: 0
+      spanId: 1
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Local
+      peer: ''
+      peerId: 0
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: Greeter.sayHello, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossThread, parentSpanId: 0, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+    - operationName: Greeter.sayHello/server/Request/onMessage
+      operationId: 0
+      parentSpanId: -1
+      spanId: 0
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Local
+      peer: ''
+      peerId: 0
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: Greeter.sayHello, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossThread, parentSpanId: 0, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+  - segmentId: not null
+    spans:
+    - operationName: Greeter.sayHello/server/Response/onClose
+      operationId: 0
+      parentSpanId: 0
+      spanId: 1
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Local
+      peer: ''
+      peerId: 0
+      tags:
+      - {key: status_code, value: OK}
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: Greeter.sayHello, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossThread, parentSpanId: 0, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+    - operationName: Greeter.sayHello/server/Request/onComplete
+      operationId: 0
+      parentSpanId: -1
+      spanId: 0
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Local
+      peer: ''
+      peerId: 0
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: Greeter.sayHello, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossThread, parentSpanId: 0, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+  - segmentId: not null
+    spans:
+    - operationName: Greeter.sayHello/client/Response/onMessage
+      operationId: 0
+      parentSpanId: -1
+      spanId: 0
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Local
+      peer: ''
+      peerId: 0
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: /grpc-scenario/case/grpc-scenario, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossThread, parentSpanId: 1, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+  - segmentId: not null
+    spans:
+    - operationName: Greeter.sayHello/client/Response/onClose
+      operationId: 0
+      parentSpanId: -1
+      spanId: 0
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Local
+      peer: ''
+      peerId: 0
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: /grpc-scenario/case/grpc-scenario, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossThread, parentSpanId: 1, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+  - segmentId: not null
+    spans:
+    - operationName: GreeterBlockingError.sayHello
+      operationId: 0
+      parentSpanId: -1
+      spanId: 0
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Entry
+      peer: ''
+      peerId: 0
+      refs:
+        - {parentEndpointId: 0, parentEndpoint: /grpc-scenario/case/grpc-scenario, networkAddressId: 0,
+           entryEndpointId: 0, refType: CrossProcess, parentSpanId: 5, parentTraceSegmentId: not null,
+           parentServiceInstanceId: 1, networkAddress: '127.0.0.1:18080', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+           entryServiceInstanceId: 1}
+  - segmentId: not null
+    spans:
+    - operationName: GreeterBlockingError.sayHello/server/Response/onClose
+      operationId: 0
+      parentSpanId: 0
+      spanId: 1
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: true
+      spanType: Local
+      peer: ''
+      peerId: 0
+      tags:
+      - { key: status_code, value: UNKNOWN }
+      logs:
+      - logEvent:
+        - { key: event, value: error }
+        - { key: error.kind, value: java.lang.Exception }
+        - { key: message, value: '' }
+        - key: stack
+          value: not null
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: GreeterBlockingError.sayHello, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossThread, parentSpanId: 0, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+    - operationName: GreeterBlockingError.sayHello/server/Request/onComplete
+      operationId: 0
+      parentSpanId: -1
+      spanId: 0
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Local
+      peer: ''
+      peerId: 0
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: GreeterBlockingError.sayHello, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossThread, parentSpanId: 0, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+  - segmentId: not null
+    spans:
+    - operationName: Greeter.sayHello
+      operationId: 0
+      parentSpanId: 0
+      spanId: 1
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Exit
+      peer: '127.0.0.1:18080'
+      peerId: 0
+    - {operationName: GreeterBlocking.sayHello/client/Request/onComplete, operationId: 0,
+       parentSpanId: 2, spanId: 3, spanLayer: RPCFramework, startTime: nq 0,
+       endTime: nq 0, componentId: 23, componentName: '', isError: false,
+       spanType: Local, peer: '', peerId: 0}
+    - {operationName: GreeterBlocking.sayHello/client/Response/onClose, operationId: 0,
 
 Review comment:
   > Why do these two use different formats? I know they may be right but it seems strange.
   
   When the span is shorter, it is a simplified format.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [skywalking] wu-sheng commented on a change in pull request #4177: Enhance gRPC plugin

Posted by GitBox <gi...@apache.org>.
wu-sheng commented on a change in pull request #4177: Enhance gRPC plugin
URL: https://github.com/apache/skywalking/pull/4177#discussion_r371102796
 
 

 ##########
 File path: apm-sniffer/apm-sdk-plugin/grpc-1.x-plugin/src/main/java/org/apache/skywalking/apm/plugin/grpc/v1/server/TracingServerCall.java
 ##########
 @@ -0,0 +1,111 @@
+/*
+ * 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.apm.plugin.grpc.v1.server;
+
+import io.grpc.ForwardingServerCall;
+import io.grpc.Metadata;
+import io.grpc.ServerCall;
+import io.grpc.Status;
+import org.apache.skywalking.apm.agent.core.context.ContextManager;
+import org.apache.skywalking.apm.agent.core.context.ContextSnapshot;
+import org.apache.skywalking.apm.agent.core.context.tag.Tags;
+import org.apache.skywalking.apm.agent.core.context.trace.AbstractSpan;
+import org.apache.skywalking.apm.agent.core.context.trace.SpanLayer;
+import org.apache.skywalking.apm.network.trace.component.ComponentsDefine;
+import org.apache.skywalking.apm.plugin.grpc.v1.OperationNameFormatUtil;
+
+import static org.apache.skywalking.apm.plugin.grpc.v1.Constants.*;
+
+/**
+ * @author wang zheng, kanro
+ */
+public class TracingServerCall<REQUEST, RESPONSE> extends ForwardingServerCall.SimpleForwardingServerCall<REQUEST, RESPONSE> {
+
+    private final String operationPrefix;
+    private final ContextSnapshot contextSnapshot;
+
+    protected TracingServerCall(ServerCall<REQUEST, RESPONSE> delegate, ContextSnapshot contextSnapshot) {
+        super(delegate);
+        this.operationPrefix = OperationNameFormatUtil.formatOperationName(delegate.getMethodDescriptor()) + SERVER;
+        this.contextSnapshot = contextSnapshot;
+    }
+
+    @Override
+    public void sendMessage(RESPONSE message) {
+        // We just create the request on message span for server stream calls.
+        if (!getMethodDescriptor().getType().serverSendsOneMessage()) {
+            try {
+                final AbstractSpan span = ContextManager.createLocalSpan(operationPrefix + RESPONSE_ON_MESSAGE_OPERATION_NAME);
+                span.setComponent(ComponentsDefine.GRPC);
+                span.setLayer(SpanLayer.RPC_FRAMEWORK);
+                ContextManager.continued(contextSnapshot);
+                super.sendMessage(message);
+            } catch (Throwable t) {
+                ContextManager.activeSpan().errorOccurred().log(t);
+                throw t;
+            } finally {
+                ContextManager.stopSpan();
+            }
+        } else {
+            super.sendMessage(message);
+        }
+    }
+
+    @Override
+    public void close(Status status, Metadata trailers) {
+        try {
+            final AbstractSpan span = ContextManager.createLocalSpan(operationPrefix + RESPONSE_ON_CLOSE_OPERATION_NAME);
+            span.setComponent(ComponentsDefine.GRPC);
+            span.setLayer(SpanLayer.RPC_FRAMEWORK);
+            ContextManager.continued(contextSnapshot);
+
+            switch (status.getCode()) {
+                case OK:
+                    break;
+                // UNKNOWN/INTERNAL status code will case error in this span.
+                // Those status code means some unexpected error occurred in server.
+                // Similar to 5XX in HTTP status.
+                case UNKNOWN:
+                case INTERNAL:
+                    if (status.getCause() == null) {
+                        span.errorOccurred().log(status.asRuntimeException());
+                    } else {
+                        span.errorOccurred().log(status.getCause());
+                    }
+                    break;
+                // Other status code means some predictable error occurred in server.
+                // Like PERMISSION_DENIED or UNAUTHENTICATED somethings.
+                // Similar to 4XX in HTTP status.
+                default:
 
 Review comment:
   `case UNKNOWN`, `case INTERNAL` and `default` seem to have the same logic. What is the difference?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [skywalking] wu-sheng commented on a change in pull request #4177: Enhance gRPC plugin

Posted by GitBox <gi...@apache.org>.
wu-sheng commented on a change in pull request #4177: Enhance gRPC plugin
URL: https://github.com/apache/skywalking/pull/4177#discussion_r371111758
 
 

 ##########
 File path: test/plugin/scenarios/grpc-scenario/config/expectedData.yaml
 ##########
 @@ -15,239 +15,542 @@
 # limitations under the License.
 registryItems:
   applications:
-    - {grpc-scenario: 2}
+  - { grpc-scenario: 2 }
   instances:
-    - {grpc-scenario: 1}
+  - { grpc-scenario: 1 }
   operationNames:
-    - grpc-scenario: [Greeter.sayHello, GreeterBlocking.sayHello,
-                      /grpc-scenario/case/grpc-scenario]
+  - grpc-scenario: [/grpc-scenario/case/healthCheck, Greeter.sayHello, GreeterBlocking.sayHello, GreeterBlockingError.sayHello, /grpc-scenario/case/grpc-scenario]
 
 Review comment:
   Health check should not be included.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [skywalking] devkanro commented on issue #4177: Enhance gRPC plugin

Posted by GitBox <gi...@apache.org>.
devkanro commented on issue #4177: Enhance gRPC plugin
URL: https://github.com/apache/skywalking/pull/4177#issuecomment-579134965
 
 
   Some discuss not about this PR.
   
   I found there is bad throughput about OAP server collect tracing.
   I have tried to use skywalking in my production env, but there are too many segments to collect.
   And I scale the OAP server to 8c16gx8 instances, there are also too many errors about collecting(gRPC client canceled.)
   The CPU looks fine about OAP server and ES server.
   
   Maybe separate collectors from OAP server is a good idea?
   Agents are not calling OAP server to collect spans directly, use the MessageQueue or log collector to do it, and make collectors consume those messages or log, one collector maybe 1c2g/2c4g, we can have many collectors with low cost.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [skywalking] devkanro edited a comment on issue #4177: Enhance gRPC plugin

Posted by GitBox <gi...@apache.org>.
devkanro edited a comment on issue #4177: Enhance gRPC plugin
URL: https://github.com/apache/skywalking/pull/4177#issuecomment-573499115
 
 
   In the default config, all gRPC calls are internal calls. No client-side tracing will be created.
   
   I think the default config is enough for most cases (most of external APIs will not be gRPC). But if you need the client-side tracing for separate peers, just add it to `INCLUDED_CLIENT_TRACING_PEERS`.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [skywalking] codecov-io edited a comment on issue #4177: Enhance gRPC plugin

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #4177: Enhance gRPC plugin
URL: https://github.com/apache/skywalking/pull/4177#issuecomment-575098453
 
 
   # [Codecov](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=h1) Report
   > :exclamation: No coverage uploaded for pull request base (`master@5ca626e`). [Click here to learn what that means](https://docs.codecov.io/docs/error-reference#section-missing-base-commit).
   > The diff coverage is `16.56%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/skywalking/pull/4177/graphs/tree.svg?width=650&token=qrILxY5yA8&height=150&src=pr)](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff            @@
   ##             master    #4177   +/-   ##
   =========================================
     Coverage          ?   26.83%           
   =========================================
     Files             ?     1162           
     Lines             ?    25457           
     Branches          ?     3632           
   =========================================
     Hits              ?     6832           
     Misses            ?    18020           
     Partials          ?      605
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [...ing/oap/server/core/source/DefaultScopeDefine.java](https://codecov.io/gh/apache/skywalking/pull/4177/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvc2t5d2Fsa2luZy9vYXAvc2VydmVyL2NvcmUvc291cmNlL0RlZmF1bHRTY29wZURlZmluZS5qYXZh) | `0% <ø> (ø)` | |
   | [...erver/receiver/jvm/provider/JVMModuleProvider.java](https://codecov.io/gh/apache/skywalking/pull/4177/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItcmVjZWl2ZXItcGx1Z2luL3NreXdhbGtpbmctanZtLXJlY2VpdmVyLXBsdWdpbi9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvc2t5d2Fsa2luZy9vYXAvc2VydmVyL3JlY2VpdmVyL2p2bS9wcm92aWRlci9KVk1Nb2R1bGVQcm92aWRlci5qYXZh) | `0% <ø> (ø)` | |
   | [...alking/apm/agent/core/context/trace/EntrySpan.java](https://codecov.io/gh/apache/skywalking/pull/4177/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLWFnZW50LWNvcmUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvYXBtL2FnZW50L2NvcmUvY29udGV4dC90cmFjZS9FbnRyeVNwYW4uamF2YQ==) | `54.83% <ø> (ø)` | |
   | [...lking/oap/server/core/query/TraceQueryService.java](https://codecov.io/gh/apache/skywalking/pull/4177/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvc2t5d2Fsa2luZy9vYXAvc2VydmVyL2NvcmUvcXVlcnkvVHJhY2VRdWVyeVNlcnZpY2UuamF2YQ==) | `0% <ø> (ø)` | |
   | [...walking/apm/agent/core/context/trace/ExitSpan.java](https://codecov.io/gh/apache/skywalking/pull/4177/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLWFnZW50LWNvcmUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvYXBtL2FnZW50L2NvcmUvY29udGV4dC90cmFjZS9FeGl0U3Bhbi5qYXZh) | `53.57% <ø> (ø)` | |
   | [.../provider/parser/SegmentParserListenerManager.java](https://codecov.io/gh/apache/skywalking/pull/4177/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItcmVjZWl2ZXItcGx1Z2luL3NreXdhbGtpbmctdHJhY2UtcmVjZWl2ZXItcGx1Z2luL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9za3l3YWxraW5nL29hcC9zZXJ2ZXIvcmVjZWl2ZXIvdHJhY2UvcHJvdmlkZXIvcGFyc2VyL1NlZ21lbnRQYXJzZXJMaXN0ZW5lck1hbmFnZXIuamF2YQ==) | `0% <ø> (ø)` | |
   | [...alking/apm/agent/core/context/trace/LocalSpan.java](https://codecov.io/gh/apache/skywalking/pull/4177/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLWFnZW50LWNvcmUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvYXBtL2FnZW50L2NvcmUvY29udGV4dC90cmFjZS9Mb2NhbFNwYW4uamF2YQ==) | `0% <ø> (ø)` | |
   | [...iver/register/provider/RegisterModuleProvider.java](https://codecov.io/gh/apache/skywalking/pull/4177/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItcmVjZWl2ZXItcGx1Z2luL3NreXdhbGtpbmctcmVnaXN0ZXItcmVjZWl2ZXItcGx1Z2luL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9za3l3YWxraW5nL29hcC9zZXJ2ZXIvcmVjZWl2ZXIvcmVnaXN0ZXIvcHJvdmlkZXIvUmVnaXN0ZXJNb2R1bGVQcm92aWRlci5qYXZh) | `0% <ø> (ø)` | |
   | [.../apm/agent/core/profile/TracingThreadSnapshot.java](https://codecov.io/gh/apache/skywalking/pull/4177/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLWFnZW50LWNvcmUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvYXBtL2FnZW50L2NvcmUvcHJvZmlsZS9UcmFjaW5nVGhyZWFkU25hcHNob3QuamF2YQ==) | `0% <0%> (ø)` | |
   | [...alking/oap/server/core/command/CommandService.java](https://codecov.io/gh/apache/skywalking/pull/4177/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvc2t5d2Fsa2luZy9vYXAvc2VydmVyL2NvcmUvY29tbWFuZC9Db21tYW5kU2VydmljZS5qYXZh) | `0% <0%> (ø)` | |
   | ... and [29 more](https://codecov.io/gh/apache/skywalking/pull/4177/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=footer). Last update [5ca626e...d44d145](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [skywalking] devkanro edited a comment on issue #4177: Enhance gRPC plugin

Posted by GitBox <gi...@apache.org>.
devkanro edited a comment on issue #4177: Enhance gRPC plugin
URL: https://github.com/apache/skywalking/pull/4177#issuecomment-579134965
 
 
   Some discuss not about this PR. 
   
   I found there is bad throughput about OAP server collect tracing.
   I have tried to use skywalking in my production env, but there are too many segments to collect.
   And I scale the OAP server to 8c16gx8 instances, there are also too many errors about collecting(gRPC client canceled.)
   The CPU looks fine about OAP server and ES server.
   
   Maybe separate collectors from OAP server is a good idea?
   Agents are not calling OAP server to collect spans directly, use the MessageQueue or log collector to do it, and make collectors consume those messages or log, one collector maybe 1c2g/2c4g, we can have many collectors with low cost.
   
   If you are interested in this, maybe we could open an issue to discuss.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [skywalking] wu-sheng commented on a change in pull request #4177: Enhance gRPC plugin

Posted by GitBox <gi...@apache.org>.
wu-sheng commented on a change in pull request #4177: Enhance gRPC plugin
URL: https://github.com/apache/skywalking/pull/4177#discussion_r366911621
 
 

 ##########
 File path: apm-sniffer/apm-agent-core/src/main/java/org/apache/skywalking/apm/agent/core/conf/Config.java
 ##########
 @@ -388,5 +390,26 @@
              */
             public static int HTTP_PARAMS_LENGTH_THRESHOLD = 1024;
         }
+
+        public static class Grpc {
+            /**
+             * If this config is false, only client spans of peer which configured in {@link Grpc#INCLUDED_CLIENT_TRACING_PEERS}
+             * will be collected. In this mode {@link Grpc#EXCLUDED_CLIENT_TRACING_PEERS} will override {@link Grpc#INCLUDED_CLIENT_TRACING_PEERS}.
+             * <p>
+             * If this config is true, only client spans of peer which configured in {@link Grpc#EXCLUDED_CLIENT_TRACING_PEERS}
+             * will not be collected. In this mode {@link Grpc#INCLUDED_CLIENT_TRACING_PEERS} will override {@link Grpc#EXCLUDED_CLIENT_TRACING_PEERS}.
+             */
+            public static boolean DEFAULT_CLIENT_TRACING_ENABLE = false;
+
+            /**
+             * Included client tracing peers. gRPC plugin will collect client spans of configured config.
+             */
+            public static List<String> INCLUDED_CLIENT_TRACING_PEERS = new LinkedList<>();
+
+            /**
+             * Excluded client tracing peers. gRPC plugin will not collect client spans of configured config.
+             */
+            public static List<String> EXCLUDED_CLIENT_TRACING_PEERS = new LinkedList<>();
 
 Review comment:
   Do you forget to remove all these?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [skywalking] devkanro commented on issue #4177: Enhance gRPC plugin

Posted by GitBox <gi...@apache.org>.
devkanro commented on issue #4177: Enhance gRPC plugin
URL: https://github.com/apache/skywalking/pull/4177#issuecomment-573499115
 
 
   In the default config, all gRPC calls are internal calls. No client-side tracing will be created. I think the default config is enough for most cases (most of external APIs will not be gRPC). But if you need the client-side tracing for separate peers, just add it to `INCLUDED_CLIENT_TRACING_PEERS`.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [skywalking] wu-sheng commented on a change in pull request #4177: Enhance gRPC plugin

Posted by GitBox <gi...@apache.org>.
wu-sheng commented on a change in pull request #4177: Enhance gRPC plugin
URL: https://github.com/apache/skywalking/pull/4177#discussion_r371104619
 
 

 ##########
 File path: apm-sniffer/apm-sdk-plugin/grpc-1.x-plugin/src/main/java/org/apache/skywalking/apm/plugin/grpc/v1/server/TracingServerCall.java
 ##########
 @@ -0,0 +1,111 @@
+/*
+ * 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.apm.plugin.grpc.v1.server;
+
+import io.grpc.ForwardingServerCall;
+import io.grpc.Metadata;
+import io.grpc.ServerCall;
+import io.grpc.Status;
+import org.apache.skywalking.apm.agent.core.context.ContextManager;
+import org.apache.skywalking.apm.agent.core.context.ContextSnapshot;
+import org.apache.skywalking.apm.agent.core.context.tag.Tags;
+import org.apache.skywalking.apm.agent.core.context.trace.AbstractSpan;
+import org.apache.skywalking.apm.agent.core.context.trace.SpanLayer;
+import org.apache.skywalking.apm.network.trace.component.ComponentsDefine;
+import org.apache.skywalking.apm.plugin.grpc.v1.OperationNameFormatUtil;
+
+import static org.apache.skywalking.apm.plugin.grpc.v1.Constants.*;
+
+/**
+ * @author wang zheng, kanro
+ */
+public class TracingServerCall<REQUEST, RESPONSE> extends ForwardingServerCall.SimpleForwardingServerCall<REQUEST, RESPONSE> {
+
+    private final String operationPrefix;
+    private final ContextSnapshot contextSnapshot;
+
+    protected TracingServerCall(ServerCall<REQUEST, RESPONSE> delegate, ContextSnapshot contextSnapshot) {
+        super(delegate);
+        this.operationPrefix = OperationNameFormatUtil.formatOperationName(delegate.getMethodDescriptor()) + SERVER;
+        this.contextSnapshot = contextSnapshot;
+    }
+
+    @Override
+    public void sendMessage(RESPONSE message) {
+        // We just create the request on message span for server stream calls.
+        if (!getMethodDescriptor().getType().serverSendsOneMessage()) {
+            try {
+                final AbstractSpan span = ContextManager.createLocalSpan(operationPrefix + RESPONSE_ON_MESSAGE_OPERATION_NAME);
+                span.setComponent(ComponentsDefine.GRPC);
+                span.setLayer(SpanLayer.RPC_FRAMEWORK);
+                ContextManager.continued(contextSnapshot);
+                super.sendMessage(message);
+            } catch (Throwable t) {
+                ContextManager.activeSpan().errorOccurred().log(t);
+                throw t;
+            } finally {
+                ContextManager.stopSpan();
+            }
+        } else {
+            super.sendMessage(message);
+        }
+    }
+
+    @Override
+    public void close(Status status, Metadata trailers) {
+        try {
+            final AbstractSpan span = ContextManager.createLocalSpan(operationPrefix + RESPONSE_ON_CLOSE_OPERATION_NAME);
+            span.setComponent(ComponentsDefine.GRPC);
+            span.setLayer(SpanLayer.RPC_FRAMEWORK);
+            ContextManager.continued(contextSnapshot);
+
+            switch (status.getCode()) {
+                case OK:
+                    break;
+                // UNKNOWN/INTERNAL status code will case error in this span.
+                // Those status code means some unexpected error occurred in server.
+                // Similar to 5XX in HTTP status.
+                case UNKNOWN:
+                case INTERNAL:
+                    if (status.getCause() == null) {
+                        span.errorOccurred().log(status.asRuntimeException());
+                    } else {
+                        span.errorOccurred().log(status.getCause());
+                    }
+                    break;
+                // Other status code means some predictable error occurred in server.
+                // Like PERMISSION_DENIED or UNAUTHENTICATED somethings.
+                // Similar to 4XX in HTTP status.
+                default:
+                    // But if the status still has cause exception, we will log it too.
+                    if (status.getCause() != null) {
+                        span.errorOccurred().log(status.getCause());
+                    }
+                    break;
+            }
+            Tags.STATUS_CODE.set(span, status.getCode().name());
+            super.close(status, trailers);
 
 Review comment:
   You should add a new `try/final` in this code block, and keep `super#close` in the final.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [skywalking] codecov-io edited a comment on issue #4177: Enhance gRPC plugin

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #4177: Enhance gRPC plugin
URL: https://github.com/apache/skywalking/pull/4177#issuecomment-575098453
 
 
   # [Codecov](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=h1) Report
   > Merging [#4177](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=desc) into [master](https://codecov.io/gh/apache/skywalking/commit/93cdc2a0e658e44fef7fc20c0cb388a1356417cd?src=pr&el=desc) will **not change** coverage.
   > The diff coverage is `84.61%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/skywalking/pull/4177/graphs/tree.svg?width=650&token=qrILxY5yA8&height=150&src=pr)](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=tree)
   
   ```diff
   @@           Coverage Diff           @@
   ##           master    #4177   +/-   ##
   =======================================
     Coverage   26.75%   26.75%           
   =======================================
     Files        1162     1162           
     Lines       25456    25456           
     Branches     3632     3632           
   =======================================
     Hits         6810     6810           
     Misses      18041    18041           
     Partials      605      605
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [...ng/oap/server/core/alarm/provider/RunningRule.java](https://codecov.io/gh/apache/skywalking/pull/4177/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItYWxhcm0tcGx1Z2luL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9za3l3YWxraW5nL29hcC9zZXJ2ZXIvY29yZS9hbGFybS9wcm92aWRlci9SdW5uaW5nUnVsZS5qYXZh) | `74% <84.61%> (ø)` | :arrow_up: |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=footer). Last update [93cdc2a...416582e](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [skywalking] wu-sheng commented on a change in pull request #4177: Enhance gRPC plugin

Posted by GitBox <gi...@apache.org>.
wu-sheng commented on a change in pull request #4177: Enhance gRPC plugin
URL: https://github.com/apache/skywalking/pull/4177#discussion_r371648214
 
 

 ##########
 File path: apm-sniffer/apm-sdk-plugin/grpc-1.x-plugin/src/main/java/org/apache/skywalking/apm/plugin/grpc/v1/server/TracingServerCall.java
 ##########
 @@ -0,0 +1,111 @@
+/*
+ * 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.apm.plugin.grpc.v1.server;
+
+import io.grpc.ForwardingServerCall;
+import io.grpc.Metadata;
+import io.grpc.ServerCall;
+import io.grpc.Status;
+import org.apache.skywalking.apm.agent.core.context.ContextManager;
+import org.apache.skywalking.apm.agent.core.context.ContextSnapshot;
+import org.apache.skywalking.apm.agent.core.context.tag.Tags;
+import org.apache.skywalking.apm.agent.core.context.trace.AbstractSpan;
+import org.apache.skywalking.apm.agent.core.context.trace.SpanLayer;
+import org.apache.skywalking.apm.network.trace.component.ComponentsDefine;
+import org.apache.skywalking.apm.plugin.grpc.v1.OperationNameFormatUtil;
+
+import static org.apache.skywalking.apm.plugin.grpc.v1.Constants.*;
+
+/**
+ * @author wang zheng, kanro
+ */
+public class TracingServerCall<REQUEST, RESPONSE> extends ForwardingServerCall.SimpleForwardingServerCall<REQUEST, RESPONSE> {
+
+    private final String operationPrefix;
+    private final ContextSnapshot contextSnapshot;
+
+    protected TracingServerCall(ServerCall<REQUEST, RESPONSE> delegate, ContextSnapshot contextSnapshot) {
+        super(delegate);
+        this.operationPrefix = OperationNameFormatUtil.formatOperationName(delegate.getMethodDescriptor()) + SERVER;
+        this.contextSnapshot = contextSnapshot;
+    }
+
+    @Override
+    public void sendMessage(RESPONSE message) {
+        // We just create the request on message span for server stream calls.
+        if (!getMethodDescriptor().getType().serverSendsOneMessage()) {
+            try {
+                final AbstractSpan span = ContextManager.createLocalSpan(operationPrefix + RESPONSE_ON_MESSAGE_OPERATION_NAME);
+                span.setComponent(ComponentsDefine.GRPC);
+                span.setLayer(SpanLayer.RPC_FRAMEWORK);
+                ContextManager.continued(contextSnapshot);
+            } catch (Throwable t) {
+                ContextManager.activeSpan().errorOccurred().log(t);
+                throw t;
+            } finally {
+                super.sendMessage(message);
+                ContextManager.stopSpan();
+            }
+        } else {
+            super.sendMessage(message);
+        }
+    }
+
+    @Override
+    public void close(Status status, Metadata trailers) {
+        try {
+            final AbstractSpan span = ContextManager.createLocalSpan(operationPrefix + RESPONSE_ON_CLOSE_OPERATION_NAME);
+            span.setComponent(ComponentsDefine.GRPC);
+            span.setLayer(SpanLayer.RPC_FRAMEWORK);
+            ContextManager.continued(contextSnapshot);
+
+            switch (status.getCode()) {
+                case OK:
+                    break;
+                // UNKNOWN/INTERNAL status code will case error in this span.
+                // Those status code means some unexpected error occurred in server.
+                // Similar to 5XX in HTTP status.
+                case UNKNOWN:
+                case INTERNAL:
+                    if (status.getCause() == null) {
+                        span.errorOccurred().log(status.asRuntimeException());
+                    } else {
+                        span.errorOccurred().log(status.getCause());
+                    }
+                    break;
+                // Other status code means some predictable error occurred in server.
+                // Like PERMISSION_DENIED or UNAUTHENTICATED somethings.
+                // Similar to 4XX in HTTP status.
+                default:
+                    // But if the status still has cause exception, we will log it too.
+                    if (status.getCause() != null) {
+                        span.errorOccurred().log(status.getCause());
+                    }
+                    break;
+            }
+            Tags.STATUS_CODE.set(span, status.getCode().name());
+        } catch (Throwable t) {
 
 Review comment:
   Please follow the next comment.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [skywalking] wu-sheng commented on issue #4177: Enhance gRPC plugin

Posted by GitBox <gi...@apache.org>.
wu-sheng commented on issue #4177: Enhance gRPC plugin
URL: https://github.com/apache/skywalking/pull/4177#issuecomment-573529288
 
 
   > I can simplify client-side tracing for this, like no onMessage event, only Complete/Close event for internal client-side tracing.
   
   I think in most cases, both of them should exist, including `onMessage/Complete/Close`. User could have further OP, such as DB or cache access in the onMessage like any other callback. 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [skywalking] codecov-io edited a comment on issue #4177: Enhance gRPC plugin

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #4177: Enhance gRPC plugin
URL: https://github.com/apache/skywalking/pull/4177#issuecomment-570947001
 
 
   # [Codecov](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=h1) Report
   > Merging [#4177](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=desc) into [master](https://codecov.io/gh/apache/skywalking/commit/c6b169544e199fdf6d8e27e58b8f98897c3e7ef7?src=pr&el=desc) will **decrease** coverage by `0.1%`.
   > The diff coverage is `0%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/skywalking/pull/4177/graphs/tree.svg?width=650&token=qrILxY5yA8&height=150&src=pr)](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #4177      +/-   ##
   ==========================================
   - Coverage   26.81%   26.71%   -0.11%     
   ==========================================
     Files        1163     1164       +1     
     Lines       25431    25528      +97     
     Branches     3694     3705      +11     
   ==========================================
     Hits         6819     6819              
   - Misses      18000    18097      +97     
     Partials      612      612
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [...ng/apm/plugin/grpc/v1/OperationNameFormatUtil.java](https://codecov.io/gh/apache/skywalking/pull/4177/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLXNkay1wbHVnaW4vZ3JwYy0xLngtcGx1Z2luL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9za3l3YWxraW5nL2FwbS9wbHVnaW4vZ3JwYy92MS9PcGVyYXRpb25OYW1lRm9ybWF0VXRpbC5qYXZh) | `0% <ø> (ø)` | :arrow_up: |
   | [...plugin/grpc/v1/client/BlockingCallInterceptor.java](https://codecov.io/gh/apache/skywalking/pull/4177/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLXNkay1wbHVnaW4vZ3JwYy0xLngtcGx1Z2luL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9za3l3YWxraW5nL2FwbS9wbHVnaW4vZ3JwYy92MS9jbGllbnQvQmxvY2tpbmdDYWxsSW50ZXJjZXB0b3IuamF2YQ==) | `0% <0%> (ø)` | |
   | [...g/apm/plugin/grpc/v1/client/TracingClientCall.java](https://codecov.io/gh/apache/skywalking/pull/4177/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLXNkay1wbHVnaW4vZ3JwYy0xLngtcGx1Z2luL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9za3l3YWxraW5nL2FwbS9wbHVnaW4vZ3JwYy92MS9jbGllbnQvVHJhY2luZ0NsaWVudENhbGwuamF2YQ==) | `0% <0%> (ø)` | |
   | [...g/apm/plugin/grpc/v1/server/ServerInterceptor.java](https://codecov.io/gh/apache/skywalking/pull/4177/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLXNkay1wbHVnaW4vZ3JwYy0xLngtcGx1Z2luL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9za3l3YWxraW5nL2FwbS9wbHVnaW4vZ3JwYy92MS9zZXJ2ZXIvU2VydmVySW50ZXJjZXB0b3IuamF2YQ==) | `0% <0%> (ø)` | |
   | [...g/apm/plugin/grpc/v1/server/TracingServerCall.java](https://codecov.io/gh/apache/skywalking/pull/4177/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLXNkay1wbHVnaW4vZ3JwYy0xLngtcGx1Z2luL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9za3l3YWxraW5nL2FwbS9wbHVnaW4vZ3JwYy92MS9zZXJ2ZXIvVHJhY2luZ1NlcnZlckNhbGwuamF2YQ==) | `0% <0%> (ø)` | |
   | [...plugin/grpc/v1/client/AbstractStubInterceptor.java](https://codecov.io/gh/apache/skywalking/pull/4177/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLXNkay1wbHVnaW4vZ3JwYy0xLngtcGx1Z2luL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9za3l3YWxraW5nL2FwbS9wbHVnaW4vZ3JwYy92MS9jbGllbnQvQWJzdHJhY3RTdHViSW50ZXJjZXB0b3IuamF2YQ==) | `0% <0%> (ø)` | |
   | [...ugin/grpc/v1/server/TracingServerCallListener.java](https://codecov.io/gh/apache/skywalking/pull/4177/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLXNkay1wbHVnaW4vZ3JwYy0xLngtcGx1Z2luL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9za3l3YWxraW5nL2FwbS9wbHVnaW4vZ3JwYy92MS9zZXJ2ZXIvVHJhY2luZ1NlcnZlckNhbGxMaXN0ZW5lci5qYXZh) | `0% <0%> (ø)` | |
   | [...g/apm/plugin/grpc/v1/client/ClientInterceptor.java](https://codecov.io/gh/apache/skywalking/pull/4177/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLXNkay1wbHVnaW4vZ3JwYy0xLngtcGx1Z2luL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9za3l3YWxraW5nL2FwbS9wbHVnaW4vZ3JwYy92MS9jbGllbnQvQ2xpZW50SW50ZXJjZXB0b3IuamF2YQ==) | `0% <0%> (ø)` | |
   | [.../apache/skywalking/apm/agent/core/conf/Config.java](https://codecov.io/gh/apache/skywalking/pull/4177/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLWFnZW50LWNvcmUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvYXBtL2FnZW50L2NvcmUvY29uZi9Db25maWcuamF2YQ==) | `62.96% <0%> (-3.71%)` | :arrow_down: |
   | [...plugin/grpc/v1/client/SimpleTracingClientCall.java](https://codecov.io/gh/apache/skywalking/pull/4177/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLXNkay1wbHVnaW4vZ3JwYy0xLngtcGx1Z2luL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9za3l3YWxraW5nL2FwbS9wbHVnaW4vZ3JwYy92MS9jbGllbnQvU2ltcGxlVHJhY2luZ0NsaWVudENhbGwuamF2YQ==) | `0% <0%> (ø)` | |
   | ... and [7 more](https://codecov.io/gh/apache/skywalking/pull/4177/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=footer). Last update [c6b1695...8e99d18](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [skywalking] GuoDuanLZ commented on a change in pull request #4177: Enhance gRPC plugin

Posted by GitBox <gi...@apache.org>.
GuoDuanLZ commented on a change in pull request #4177: Enhance gRPC plugin
URL: https://github.com/apache/skywalking/pull/4177#discussion_r371718269
 
 

 ##########
 File path: test/plugin/scenarios/grpc-scenario/config/expectedData.yaml
 ##########
 @@ -15,239 +15,524 @@
 # limitations under the License.
 registryItems:
   applications:
-    - {grpc-scenario: 2}
+  - { grpc-scenario: 2 }
   instances:
-    - {grpc-scenario: 1}
+  - { grpc-scenario: 1 }
   operationNames:
-    - grpc-scenario: [Greeter.sayHello, GreeterBlocking.sayHello,
-                      /grpc-scenario/case/grpc-scenario]
+  - grpc-scenario: [Greeter.sayHello, GreeterBlocking.sayHello, GreeterBlockingError.sayHello, /grpc-scenario/case/grpc-scenario]
   heartbeat: []
 segmentItems:
-  - applicationCode: grpc-scenario
-    segmentSize: gt 10
-    segments:
-      - segmentId: not null
-        spans:
-          - operationName: GreeterBlocking.sayHello
-            operationId: 0
-            parentSpanId: -1
-            spanId: 0
-            spanLayer: RPCFramework
-            startTime: nq 0
-            endTime: nq 0
-            componentId: 23
-            componentName: ''
-            isError: false
-            spanType: Entry
-            peer: ''
-            peerId: 0
-            refs:
-              - {parentEndpointId: 0, parentEndpoint: /grpc-scenario/case/grpc-scenario, networkAddressId: 0,
-                 entryEndpointId: 0, refType: CrossProcess, parentSpanId: 2, parentTraceSegmentId: not null,
-                 parentServiceInstanceId: 1, networkAddress: '127.0.0.1:18080', entryEndpoint: /grpc-scenario/case/grpc-scenario,
-                 entryServiceInstanceId: 1}
-      - segmentId: not null
-        spans:
-          - operationName: Greeter.sayHello
-            operationId: 0
-            parentSpanId: -1
-            spanId: 0
-            spanLayer: RPCFramework
-            startTime: nq 0
-            endTime: nq 0
-            componentId: 23
-            componentName: ''
-            isError: false
-            spanType: Entry
-            peer: ''
-            peerId: 0
-            refs:
-              - {parentEndpointId: 0, parentEndpoint: /grpc-scenario/case/grpc-scenario, networkAddressId: 0,
-                 entryEndpointId: 0, refType: CrossProcess, parentSpanId: 1, parentTraceSegmentId: not null,
-                 parentServiceInstanceId: 1, networkAddress: '127.0.0.1:18080', entryEndpoint: /grpc-scenario/case/grpc-scenario,
-                 entryServiceInstanceId: 1}
-      - segmentId: not null
-        spans:
-          - operationName: GreeterBlocking.sayHello/server/RequestObserver/onNext
-            operationId: 0
-            parentSpanId: -1
-            spanId: 0
-            spanLayer: Unknown
-            startTime: nq 0
-            endTime: nq 0
-            componentId: 0
-            componentName: ''
-            isError: false
-            spanType: Local
-            peer: ''
-            peerId: 0
-            refs:
-              - {parentEndpointId: 0, parentEndpoint: GreeterBlocking.sayHello, networkAddressId: 0,
-                 entryEndpointId: 0, refType: CrossThread, parentSpanId: 0, parentTraceSegmentId: not null,
-                 parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
-                 entryServiceInstanceId: 1}
-      - segmentId: not null
-        spans:
-          - operationName: Greeter.sayHello/server/RequestObserver/onNext
-            operationId: 0
-            parentSpanId: -1
-            spanId: 0
-            spanLayer: Unknown
-            startTime: nq 0
-            endTime: nq 0
-            componentId: 0
-            componentName: ''
-            isError: false
-            spanType: Local
-            peer: ''
-            peerId: 0
-            refs:
-              - {parentEndpointId: 0, parentEndpoint: Greeter.sayHello, networkAddressId: 0,
-                 entryEndpointId: 0, refType: CrossThread, parentSpanId: 0, parentTraceSegmentId: not null,
-                 parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
-                 entryServiceInstanceId: 1}
-      - segmentId: not null
-        spans:
-          - operationName: Greeter.sayHello/client/ResponseObserver/onNext
-            operationId: 0
-            parentSpanId: -1
-            spanId: 0
-            spanLayer: Unknown
-            startTime: nq 0
-            endTime: nq 0
-            componentId: 0
-            componentName: ''
-            isError: false
-            spanType: Local
-            peer: ''
-            peerId: 0
-            refs:
-              - {parentEndpointId: 0, parentEndpoint: /grpc-scenario/case/grpc-scenario, networkAddressId: 0,
-                 entryEndpointId: 0, refType: CrossThread, parentSpanId: 1, parentTraceSegmentId: not null,
-                 parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
-                 entryServiceInstanceId: 1}
-      - segmentId: not null
-        spans:
-          - operationName: Greeter.sayHello/client/ResponseObserver/onComplete
-            operationId: 0
-            parentSpanId: -1
-            spanId: 0
-            spanLayer: Unknown
-            startTime: nq 0
-            endTime: nq 0
-            componentId: 0
-            componentName: ''
-            isError: false
-            spanType: Local
-            peer: ''
-            peerId: 0
-            refs:
-              - {parentEndpointId: 0, parentEndpoint: /grpc-scenario/case/grpc-scenario, networkAddressId: 0,
-                 entryEndpointId: 0, refType: CrossThread, parentSpanId: 1, parentTraceSegmentId: not null,
-                 parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
-                 entryServiceInstanceId: 1}
-      - segmentId: not null
-        spans:
-          - operationName: Greeter.sayHello/server/RequestObserver/onNext
-            operationId: 0
-            parentSpanId: -1
-            spanId: 0
-            spanLayer: Unknown
-            startTime: nq 0
-            endTime: nq 0
-            componentId: 0
-            componentName: ''
-            isError: false
-            spanType: Local
-            peer: ''
-            peerId: 0
-            refs:
-              - {parentEndpointId: 0, parentEndpoint: Greeter.sayHello, networkAddressId: 0,
-                 entryEndpointId: 0, refType: CrossThread, parentSpanId: 0, parentTraceSegmentId: not null,
-                 parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
-                 entryServiceInstanceId: 1}
-      - segmentId: not null
-        spans:
-          - operationName: Greeter.sayHello/server/RequestObserver/onComplete
-            operationId: 0
-            parentSpanId: -1
-            spanId: 0
-            spanLayer: Unknown
-            startTime: nq 0
-            endTime: nq 0
-            componentId: 0
-            componentName: ''
-            isError: false
-            spanType: Local
-            peer: ''
-            peerId: 0
-            refs:
-              - {parentEndpointId: 0, parentEndpoint: Greeter.sayHello, networkAddressId: 0,
-                 entryEndpointId: 0, refType: CrossThread, parentSpanId: 0, parentTraceSegmentId: not null,
-                 parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
-                 entryServiceInstanceId: 1}
-      - segmentId: not null
-        spans:
-          - operationName: Greeter.sayHello
-            operationId: 0
-            parentSpanId: 0
-            spanId: 1
-            spanLayer: RPCFramework
-            startTime: nq 0
-            endTime: nq 0
-            componentId: 23
-            componentName: ''
-            isError: false
-            spanType: Exit
-            peer: '127.0.0.1:18080'
-            peerId: 0
-          - operationName: GreeterBlocking.sayHello
-            operationId: 0
-            parentSpanId: 0
-            spanId: 2
-            spanLayer: RPCFramework
-            startTime: nq 0
-            endTime: nq 0
-            componentId: 23
-            componentName: ''
-            isError: false
-            spanType: Exit
-            peer: '127.0.0.1:18080'
-            peerId: 0
-          - operationName: /grpc-scenario/case/grpc-scenario
-            operationId: 0
-            parentSpanId: -1
-            spanId: 0
-            spanLayer: Http
-            startTime: nq 0
-            endTime: nq 0
-            componentId: 1
-            componentName: ''
-            isError: false
-            spanType: Entry
-            peer: ''
-            peerId: 0
-            tags:
-              - {key: url, value: 'http://localhost:8080/grpc-scenario/case/grpc-scenario'}
-              - {key: http.method, value: GET}
-      - segmentId: not null
-        spans:
-          - operationName: Greeter.sayHello/client/ResponseObserver/onNext
-            operationId: 0
-            parentSpanId: -1
-            spanId: 0
-            spanLayer: Unknown
-            startTime: nq 0
-            endTime: nq 0
-            componentId: 0
-            componentName: ''
-            isError: false
-            spanType: Local
-            peer: ''
-            peerId: 0
-            refs:
-              - {parentEndpointId: 0, parentEndpoint: /grpc-scenario/case/grpc-scenario, networkAddressId: 0,
-                 entryEndpointId: 0, refType: CrossThread, parentSpanId: 1, parentTraceSegmentId: not null,
-                 parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
-                 entryServiceInstanceId: 1}
-
+- applicationCode: grpc-scenario
+  segmentSize: gt 10
+  segments:
+  - segmentId: not null
+    spans:
+    - operationName: GreeterBlocking.sayHello
+      operationId: 0
+      parentSpanId: -1
+      spanId: 0
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Entry
+      peer: ''
+      peerId: 0
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: /grpc-scenario/case/grpc-scenario, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossProcess, parentSpanId: 2, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '127.0.0.1:18080', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+  - segmentId: not null
+    spans:
+    - operationName: Greeter.sayHello/client/Request/onMessage
+      operationId: 0
+      parentSpanId: -1
+      spanId: 0
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Local
+      peer: ''
+      peerId: 0
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: /grpc-scenario/case/grpc-scenario, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossThread, parentSpanId: 1, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+  - segmentId: not null
+    spans:
+    - operationName: Greeter.sayHello/client/Request/onMessage
+      operationId: 0
+      parentSpanId: -1
+      spanId: 0
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Local
+      peer: ''
+      peerId: 0
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: /grpc-scenario/case/grpc-scenario, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossThread, parentSpanId: 1, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+  - segmentId: not null
+    spans:
+    - operationName: Greeter.sayHello/client/Request/onComplete
+      operationId: 0
+      parentSpanId: -1
+      spanId: 0
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Local
+      peer: ''
+      peerId: 0
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: /grpc-scenario/case/grpc-scenario, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossThread, parentSpanId: 1, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+  - segmentId: not null
+    spans:
+    - operationName: Greeter.sayHello
+      operationId: 0
+      parentSpanId: -1
+      spanId: 0
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Entry
+      peer: ''
+      peerId: 0
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: /grpc-scenario/case/grpc-scenario, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossProcess, parentSpanId: 1, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '127.0.0.1:18080', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+  - segmentId: not null
+    spans:
+    - operationName: Greeter.sayHello/server/Response/onMessage
+      operationId: 0
+      parentSpanId: 0
+      spanId: 1
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Local
+      peer: ''
+      peerId: 0
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: Greeter.sayHello, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossThread, parentSpanId: 0, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+    - operationName: Greeter.sayHello/server/Request/onMessage
+      operationId: 0
+      parentSpanId: -1
+      spanId: 0
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Local
+      peer: ''
+      peerId: 0
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: Greeter.sayHello, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossThread, parentSpanId: 0, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+  - segmentId: not null
+    spans:
+    - operationName: Greeter.sayHello/client/Response/onMessage
+      operationId: 0
+      parentSpanId: -1
+      spanId: 0
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Local
+      peer: ''
+      peerId: 0
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: /grpc-scenario/case/grpc-scenario, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossThread, parentSpanId: 1, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+  - segmentId: not null
+    spans:
+    - operationName: GreeterBlocking.sayHello/server/Response/onClose
+      operationId: 0
+      parentSpanId: 0
+      spanId: 1
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Local
+      peer: ''
+      peerId: 0
+      tags:
+      - { key: status_code, value: OK }
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: GreeterBlocking.sayHello, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossThread, parentSpanId: 0, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+    - operationName: GreeterBlocking.sayHello/server/Request/onComplete
+      operationId: 0
+      parentSpanId: -1
+      spanId: 0
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Local
+      peer: ''
+      peerId: 0
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: GreeterBlocking.sayHello, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossThread, parentSpanId: 0, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+  - segmentId: not null
+    spans:
+    - operationName: Greeter.sayHello/server/Response/onMessage
+      operationId: 0
+      parentSpanId: 0
+      spanId: 1
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Local
+      peer: ''
+      peerId: 0
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: Greeter.sayHello, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossThread, parentSpanId: 0, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+    - operationName: Greeter.sayHello/server/Request/onMessage
+      operationId: 0
+      parentSpanId: -1
+      spanId: 0
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Local
+      peer: ''
+      peerId: 0
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: Greeter.sayHello, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossThread, parentSpanId: 0, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+  - segmentId: not null
+    spans:
+    - operationName: Greeter.sayHello/server/Response/onClose
+      operationId: 0
+      parentSpanId: 0
+      spanId: 1
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Local
+      peer: ''
+      peerId: 0
+      tags:
+      - {key: status_code, value: OK}
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: Greeter.sayHello, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossThread, parentSpanId: 0, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+    - operationName: Greeter.sayHello/server/Request/onComplete
+      operationId: 0
+      parentSpanId: -1
+      spanId: 0
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Local
+      peer: ''
+      peerId: 0
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: Greeter.sayHello, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossThread, parentSpanId: 0, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+  - segmentId: not null
+    spans:
+    - operationName: Greeter.sayHello/client/Response/onMessage
+      operationId: 0
+      parentSpanId: -1
+      spanId: 0
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Local
+      peer: ''
+      peerId: 0
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: /grpc-scenario/case/grpc-scenario, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossThread, parentSpanId: 1, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+  - segmentId: not null
+    spans:
+    - operationName: Greeter.sayHello/client/Response/onClose
+      operationId: 0
+      parentSpanId: -1
+      spanId: 0
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Local
+      peer: ''
+      peerId: 0
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: /grpc-scenario/case/grpc-scenario, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossThread, parentSpanId: 1, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+  - segmentId: not null
+    spans:
+    - operationName: GreeterBlockingError.sayHello
+      operationId: 0
+      parentSpanId: -1
+      spanId: 0
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Entry
+      peer: ''
+      peerId: 0
+      refs:
+        - {parentEndpointId: 0, parentEndpoint: /grpc-scenario/case/grpc-scenario, networkAddressId: 0,
+           entryEndpointId: 0, refType: CrossProcess, parentSpanId: 5, parentTraceSegmentId: not null,
+           parentServiceInstanceId: 1, networkAddress: '127.0.0.1:18080', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+           entryServiceInstanceId: 1}
+  - segmentId: not null
+    spans:
+    - operationName: GreeterBlockingError.sayHello/server/Response/onClose
+      operationId: 0
+      parentSpanId: 0
+      spanId: 1
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: true
+      spanType: Local
+      peer: ''
+      peerId: 0
+      tags:
+      - { key: status_code, value: UNKNOWN }
+      logs:
+      - logEvent:
+        - { key: event, value: error }
+        - { key: error.kind, value: java.lang.Exception }
+        - { key: message, value: '' }
+        - key: stack
+          value: not null
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: GreeterBlockingError.sayHello, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossThread, parentSpanId: 0, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+    - operationName: GreeterBlockingError.sayHello/server/Request/onComplete
+      operationId: 0
+      parentSpanId: -1
+      spanId: 0
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Local
+      peer: ''
+      peerId: 0
+      refs:
+      - {parentEndpointId: 0, parentEndpoint: GreeterBlockingError.sayHello, networkAddressId: 0,
+         entryEndpointId: 0, refType: CrossThread, parentSpanId: 0, parentTraceSegmentId: not null,
+         parentServiceInstanceId: 1, networkAddress: '', entryEndpoint: /grpc-scenario/case/grpc-scenario,
+         entryServiceInstanceId: 1}
+  - segmentId: not null
+    spans:
+    - operationName: Greeter.sayHello
+      operationId: 0
+      parentSpanId: 0
+      spanId: 1
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Exit
+      peer: '127.0.0.1:18080'
+      peerId: 0
+    - {operationName: GreeterBlocking.sayHello/client/Request/onComplete, operationId: 0,
+       parentSpanId: 2, spanId: 3, spanLayer: RPCFramework, startTime: nq 0,
+       endTime: nq 0, componentId: 23, componentName: '', isError: false,
+       spanType: Local, peer: '', peerId: 0}
+    - {operationName: GreeterBlocking.sayHello/client/Response/onClose, operationId: 0,
+       parentSpanId: 2, spanId: 4, spanLayer: RPCFramework, startTime: nq 0,
+       endTime: nq 0, componentId: 23, componentName: '', isError: false,
+       spanType: Local, peer: '', peerId: 0}
+    - operationName: GreeterBlocking.sayHello
+      operationId: 0
+      parentSpanId: 0
+      spanId: 2
+      spanLayer: RPCFramework
+      startTime: nq 0
+      endTime: nq 0
+      componentId: 23
+      componentName: ''
+      isError: false
+      spanType: Exit
+      peer: '127.0.0.1:18080'
+      peerId: 0
+    - {operationName: GreeterBlockingError.sayHello/client/Request/onComplete, operationId: 0,
 
 Review comment:
   > This one too.
   
   This same as above

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [skywalking] codecov-io edited a comment on issue #4177: Enhance gRPC plugin

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #4177: Enhance gRPC plugin
URL: https://github.com/apache/skywalking/pull/4177#issuecomment-575098453
 
 
   # [Codecov](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=h1) Report
   > Merging [#4177](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=desc) into [master](https://codecov.io/gh/apache/skywalking/commit/9608d0733ff07bd69ac0ef9d6682fc81dd417aa0?src=pr&el=desc) will **decrease** coverage by `0.03%`.
   > The diff coverage is `23.88%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/skywalking/pull/4177/graphs/tree.svg?width=650&token=qrILxY5yA8&height=150&src=pr)](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #4177      +/-   ##
   ==========================================
   - Coverage   26.82%   26.79%   -0.04%     
   ==========================================
     Files        1162     1168       +6     
     Lines       25457    25524      +67     
     Branches     3631     3638       +7     
   ==========================================
   + Hits         6828     6838      +10     
   - Misses      18023    18080      +57     
     Partials      606      606
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [...ywalking/apm/plugin/avro/ResponderInterceptor.java](https://codecov.io/gh/apache/skywalking/pull/4177/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLXNkay1wbHVnaW4vYXZyby1wbHVnaW4vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvYXBtL3BsdWdpbi9hdnJvL1Jlc3BvbmRlckludGVyY2VwdG9yLmphdmE=) | `0% <0%> (ø)` | |
   | [.../apm/network/trace/component/ComponentsDefine.java](https://codecov.io/gh/apache/skywalking/pull/4177/diff?src=pr&el=tree#diff-YXBtLXByb3RvY29sL2FwbS1uZXR3b3JrL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9za3l3YWxraW5nL2FwbS9uZXR3b3JrL3RyYWNlL2NvbXBvbmVudC9Db21wb25lbnRzRGVmaW5lLmphdmE=) | `0% <0%> (ø)` | :arrow_up: |
   | [.../skywalking/apm/plugin/avro/SWClientRPCPlugin.java](https://codecov.io/gh/apache/skywalking/pull/4177/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLXNkay1wbHVnaW4vYXZyby1wbHVnaW4vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvYXBtL3BsdWdpbi9hdnJvL1NXQ2xpZW50UlBDUGx1Z2luLmphdmE=) | `0% <0%> (ø)` | |
   | [.../skywalking/apm/plugin/avro/SWServerRPCPlugin.java](https://codecov.io/gh/apache/skywalking/pull/4177/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLXNkay1wbHVnaW4vYXZyby1wbHVnaW4vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvYXBtL3BsdWdpbi9hdnJvL1NXU2VydmVyUlBDUGx1Z2luLmphdmE=) | `0% <0%> (ø)` | |
   | [.../apm/plugin/avro/SpecificRequestorInterceptor.java](https://codecov.io/gh/apache/skywalking/pull/4177/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLXNkay1wbHVnaW4vYXZyby1wbHVnaW4vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvYXBtL3BsdWdpbi9hdnJvL1NwZWNpZmljUmVxdWVzdG9ySW50ZXJjZXB0b3IuamF2YQ==) | `100% <100%> (ø)` | |
   | [...g/apm/plugin/avro/GenericRequestorInterceptor.java](https://codecov.io/gh/apache/skywalking/pull/4177/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLXNkay1wbHVnaW4vYXZyby1wbHVnaW4vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvYXBtL3BsdWdpbi9hdnJvL0dlbmVyaWNSZXF1ZXN0b3JJbnRlcmNlcHRvci5qYXZh) | `100% <100%> (ø)` | |
   | [...ng/apm/plugin/avro/AbstractRequestInterceptor.java](https://codecov.io/gh/apache/skywalking/pull/4177/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLXNkay1wbHVnaW4vYXZyby1wbHVnaW4vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvYXBtL3BsdWdpbi9hdnJvL0Fic3RyYWN0UmVxdWVzdEludGVyY2VwdG9yLmphdmE=) | `15% <15%> (ø)` | |
   | [.../agent/core/profile/ProfileTaskChannelService.java](https://codecov.io/gh/apache/skywalking/pull/4177/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLWFnZW50LWNvcmUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvYXBtL2FnZW50L2NvcmUvcHJvZmlsZS9Qcm9maWxlVGFza0NoYW5uZWxTZXJ2aWNlLmphdmE=) | `28.73% <0%> (-6.9%)` | :arrow_down: |
   | ... and [4 more](https://codecov.io/gh/apache/skywalking/pull/4177/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=footer). Last update [9608d07...a6e143e](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [skywalking] codecov-io edited a comment on issue #4177: Enhance gRPC plugin

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #4177: Enhance gRPC plugin
URL: https://github.com/apache/skywalking/pull/4177#issuecomment-575098453
 
 
   # [Codecov](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=h1) Report
   > Merging [#4177](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=desc) into [master](https://codecov.io/gh/apache/skywalking/commit/23be98441c0aee38119d1beace166be57930fcbf?src=pr&el=desc) will **not change** coverage.
   > The diff coverage is `n/a`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/skywalking/pull/4177/graphs/tree.svg?width=650&token=qrILxY5yA8&height=150&src=pr)](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=tree)
   
   ```diff
   @@           Coverage Diff           @@
   ##           master    #4177   +/-   ##
   =======================================
     Coverage   26.83%   26.83%           
   =======================================
     Files        1162     1162           
     Lines       25448    25448           
     Branches     3630     3630           
   =======================================
     Hits         6829     6829           
     Misses      18014    18014           
     Partials      605      605
   ```
   
   
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=footer). Last update [23be984...9608d07](https://codecov.io/gh/apache/skywalking/pull/4177?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [skywalking] devkanro commented on issue #4177: Enhance gRPC plugin

Posted by GitBox <gi...@apache.org>.
devkanro commented on issue #4177: Enhance gRPC plugin
URL: https://github.com/apache/skywalking/pull/4177#issuecomment-571909973
 
 
   > What is this? Could you explain a little more?
   
   I will update the doc of PR later, I am too busy to update PR in the working day.
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [skywalking] devkanro commented on issue #4177: Enhance gRPC plugin

Posted by GitBox <gi...@apache.org>.
devkanro commented on issue #4177: Enhance gRPC plugin
URL: https://github.com/apache/skywalking/pull/4177#issuecomment-573431678
 
 
   @wu-sheng @wayilau @kezhenxu94 
   This PR is ready for reviews.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [skywalking] devkanro commented on issue #4177: Enhance gRPC plugin

Posted by GitBox <gi...@apache.org>.
devkanro commented on issue #4177: Enhance gRPC plugin
URL: https://github.com/apache/skywalking/pull/4177#issuecomment-575155705
 
 
   @GuoDuanLZ 
   The test case need to update for client tracing

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services