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

[incubator-skywalking] branch sw6-header updated: Support BASE64 in v2 header and make entryOperationName and parentOperationName optional in ContextCarrier or ContextSnapshot

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

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


The following commit(s) were added to refs/heads/sw6-header by this push:
     new f7792a5  Support BASE64 in v2 header and make entryOperationName and parentOperationName optional in ContextCarrier or ContextSnapshot
f7792a5 is described below

commit f7792a5128dc12c2f646bae3c5ee70c7ea130795
Author: Wu Sheng <wu...@foxmail.com>
AuthorDate: Thu Nov 8 16:07:03 2018 +0800

    Support BASE64 in v2 header and make entryOperationName and parentOperationName optional in ContextCarrier or ContextSnapshot
---
 .../skywalking/apm/agent/core/base64/Base64.java   | 45 +++++++++++----
 .../apm/agent/core/context/ContextCarrier.java     | 21 +++----
 .../apm/agent/core/context/ContextSnapshot.java    |  7 +--
 .../apm/agent/core/context/TracingContext.java     | 29 +++++-----
 .../agent/core/context/trace/TraceSegmentRef.java  | 20 ++++---
 .../core/context/ContextCarrierV2HeaderTest.java   | 66 +++++++++++++++++++---
 6 files changed, 132 insertions(+), 56 deletions(-)

diff --git a/apm-sniffer/apm-agent-core/src/main/java/org/apache/skywalking/apm/agent/core/base64/Base64.java b/apm-sniffer/apm-agent-core/src/main/java/org/apache/skywalking/apm/agent/core/base64/Base64.java
index 6467862..2a526da 100644
--- a/apm-sniffer/apm-agent-core/src/main/java/org/apache/skywalking/apm/agent/core/base64/Base64.java
+++ b/apm-sniffer/apm-agent-core/src/main/java/org/apache/skywalking/apm/agent/core/base64/Base64.java
@@ -18,17 +18,30 @@
 package org.apache.skywalking.apm.agent.core.base64;
 
 import java.io.UnsupportedEncodingException;
+import org.apache.skywalking.apm.agent.core.logging.api.ILog;
+import org.apache.skywalking.apm.agent.core.logging.api.LogManager;
 
 /**
- * Copied from {@code zipkin.internal.Base64}, adapted from {@code okio.Base64}
- * as JRE 6 doesn't have a base64Url encoder.
+ * Copied from {@code zipkin.internal.Base64}, adapted from {@code okio.Base64} as JRE 6 doesn't have a base64Url
+ * encoder.
  *
  * @author okio cited the original author as Alexander Y. Kleymenov
  */
 public final class Base64 {
+    private static final ILog logger = LogManager.getLogger(Base64.class);
+
     private Base64() {
     }
 
+    public static String decode2UTFString(String in) {
+        try {
+            return new String(decode(in), "utf-8");
+        } catch (UnsupportedEncodingException e) {
+            logger.error(e, "Can't decode BASE64 text {}", in);
+            return "";
+        }
+    }
+
     public static byte[] decode(String in) {
         // Ignore trailing '=' padding and whitespace from the input.
         int limit = in.length();
@@ -40,7 +53,7 @@ public final class Base64 {
         }
 
         // If the input includes whitespace, this output array will be longer than necessary.
-        byte[] out = new byte[(int) (limit * 6L / 8L)];
+        byte[] out = new byte[(int)(limit * 6L / 8L)];
         int outCount = 0;
         int inCount = 0;
 
@@ -75,14 +88,14 @@ public final class Base64 {
             }
 
             // Append this char's 6 bits to the word.
-            word = (word << 6) | (byte) bits;
+            word = (word << 6) | (byte)bits;
 
             // For every 4 chars of input, we accumulate 24 bits of output. Emit 3 bytes.
             inCount++;
             if (inCount % 4 == 0) {
-                out[outCount++] = (byte) (word >> 16);
-                out[outCount++] = (byte) (word >> 8);
-                out[outCount++] = (byte) word;
+                out[outCount++] = (byte)(word >> 16);
+                out[outCount++] = (byte)(word >> 8);
+                out[outCount++] = (byte)word;
             }
         }
 
@@ -93,16 +106,17 @@ public final class Base64 {
         } else if (lastWordChars == 2) {
             // We read 2 chars followed by "==". Emit 1 byte with 8 of those 12 bits.
             word = word << 12;
-            out[outCount++] = (byte) (word >> 16);
+            out[outCount++] = (byte)(word >> 16);
         } else if (lastWordChars == 3) {
             // We read 3 chars, followed by "=". Emit 2 bytes for 16 of those 18 bits.
             word = word << 6;
-            out[outCount++] = (byte) (word >> 16);
-            out[outCount++] = (byte) (word >> 8);
+            out[outCount++] = (byte)(word >> 16);
+            out[outCount++] = (byte)(word >> 8);
         }
 
         // If we sized our out array perfectly, we're done.
-        if (outCount == out.length) return out;
+        if (outCount == out.length)
+            return out;
 
         // Copy the decoded bytes to a new, right-sized array.
         byte[] prefix = new byte[outCount];
@@ -124,6 +138,15 @@ public final class Base64 {
         '5', '6', '7', '8', '9', '-', '_'
     };
 
+    public static String encode(String text) {
+        try {
+            return encode(text.getBytes("utf-8"));
+        } catch (UnsupportedEncodingException e) {
+            logger.error(e, "Can't encode {} in BASE64", text);
+            return "";
+        }
+    }
+
     public static String encode(byte[] in) {
         return encode(in, MAP);
     }
diff --git a/apm-sniffer/apm-agent-core/src/main/java/org/apache/skywalking/apm/agent/core/context/ContextCarrier.java b/apm-sniffer/apm-agent-core/src/main/java/org/apache/skywalking/apm/agent/core/context/ContextCarrier.java
index f3948fe..52c6a6c 100644
--- a/apm-sniffer/apm-agent-core/src/main/java/org/apache/skywalking/apm/agent/core/context/ContextCarrier.java
+++ b/apm-sniffer/apm-agent-core/src/main/java/org/apache/skywalking/apm/agent/core/context/ContextCarrier.java
@@ -20,6 +20,7 @@ package org.apache.skywalking.apm.agent.core.context;
 
 import java.io.Serializable;
 import java.util.List;
+import org.apache.skywalking.apm.agent.core.base64.Base64;
 import org.apache.skywalking.apm.agent.core.conf.Config;
 import org.apache.skywalking.apm.agent.core.context.ids.DistributedTraceId;
 import org.apache.skywalking.apm.agent.core.context.ids.ID;
@@ -118,14 +119,14 @@ public class ContextCarrier implements Serializable {
                 if (Config.Agent.ACTIVE_V2_HEADER) {
                     return StringUtil.join('-',
                         "1",
-                        this.getTraceSegmentId().encode(),
+                        Base64.encode(this.getPrimaryDistributedTraceId().encode()),
+                        Base64.encode(this.getTraceSegmentId().encode()),
                         this.getSpanId() + "",
                         this.getParentApplicationInstanceId() + "",
                         this.getEntryApplicationInstanceId() + "",
-                        this.getPeerHost(),
-                        this.getPrimaryDistributedTraceId().encode(),
-                        this.getEntryOperationName(),
-                        this.getParentOperationName());
+                        Base64.encode(this.getPeerHost()),
+                        Base64.encode(this.getEntryOperationName()),
+                        Base64.encode(this.getParentOperationName()));
                 } else {
                     return "";
                 }
@@ -167,14 +168,14 @@ public class ContextCarrier implements Serializable {
                 if (parts.length == 9) {
                     try {
                         // parts[0] is sample flag, always trace if header exists.
-                        this.primaryDistributedTraceId = new PropagatedTraceId(parts[1]);
-                        this.traceSegmentId = new ID(parts[2]);
+                        this.primaryDistributedTraceId = new PropagatedTraceId(Base64.decode2UTFString(parts[1]));
+                        this.traceSegmentId = new ID(Base64.decode2UTFString(parts[2]));
                         this.spanId = Integer.parseInt(parts[3]);
                         this.parentApplicationInstanceId = Integer.parseInt(parts[4]);
                         this.entryApplicationInstanceId = Integer.parseInt(parts[5]);
-                        this.peerHost = parts[6];
-                        this.entryOperationName = parts[7];
-                        this.parentOperationName = parts[8];
+                        this.peerHost = Base64.decode2UTFString(parts[6]);
+                        this.entryOperationName = Base64.decode2UTFString(parts[7]);
+                        this.parentOperationName = Base64.decode2UTFString(parts[8]);
                     } catch (NumberFormatException e) {
 
                     }
diff --git a/apm-sniffer/apm-agent-core/src/main/java/org/apache/skywalking/apm/agent/core/context/ContextSnapshot.java b/apm-sniffer/apm-agent-core/src/main/java/org/apache/skywalking/apm/agent/core/context/ContextSnapshot.java
index a235dfe..8a8cdbe 100644
--- a/apm-sniffer/apm-agent-core/src/main/java/org/apache/skywalking/apm/agent/core/context/ContextSnapshot.java
+++ b/apm-sniffer/apm-agent-core/src/main/java/org/apache/skywalking/apm/agent/core/context/ContextSnapshot.java
@@ -16,7 +16,6 @@
  *
  */
 
-
 package org.apache.skywalking.apm.agent.core.context;
 
 import java.util.List;
@@ -98,9 +97,7 @@ public class ContextSnapshot {
         return traceSegmentId != null
             && spanId > -1
             && entryApplicationInstanceId != DictionaryUtil.nullValue()
-            && primaryDistributedTraceId != null
-            && !StringUtil.isEmpty(entryOperationName)
-            && !StringUtil.isEmpty(parentOperationName);
+            && primaryDistributedTraceId != null;
     }
 
     public String getEntryOperationName() {
@@ -114,7 +111,7 @@ public class ContextSnapshot {
     public int getEntryApplicationInstanceId() {
         return entryApplicationInstanceId;
     }
-    
+
     public boolean isFromCurrent() {
         return traceSegmentId.equals(ContextManager.capture().getTraceSegmentId());
     }
diff --git a/apm-sniffer/apm-agent-core/src/main/java/org/apache/skywalking/apm/agent/core/context/TracingContext.java b/apm-sniffer/apm-agent-core/src/main/java/org/apache/skywalking/apm/agent/core/context/TracingContext.java
index 84552bd..c4a3bf1 100644
--- a/apm-sniffer/apm-agent-core/src/main/java/org/apache/skywalking/apm/agent/core/context/TracingContext.java
+++ b/apm-sniffer/apm-agent-core/src/main/java/org/apache/skywalking/apm/agent/core/context/TracingContext.java
@@ -38,6 +38,7 @@ import org.apache.skywalking.apm.agent.core.dictionary.PossibleFound;
 import org.apache.skywalking.apm.agent.core.logging.api.ILog;
 import org.apache.skywalking.apm.agent.core.logging.api.LogManager;
 import org.apache.skywalking.apm.agent.core.sampling.SamplingService;
+import org.apache.skywalking.apm.util.StringUtil;
 
 /**
  * The <code>TracingContext</code> represents a core tracing logic controller. It build the final {@link
@@ -94,8 +95,8 @@ public class TracingContext implements AbstractTracerContext {
      * Inject the context into the given carrier, only when the active span is an exit one.
      *
      * @param carrier to carry the context for crossing process.
-     * @throws IllegalStateException if the active span isn't an exit one.
-     * Ref to {@link AbstractTracerContext#inject(ContextCarrier)}
+     * @throws IllegalStateException if the active span isn't an exit one. Ref to {@link
+     * AbstractTracerContext#inject(ContextCarrier)}
      */
     @Override
     public void inject(ContextCarrier carrier) {
@@ -136,7 +137,9 @@ public class TracingContext implements AbstractTracerContext {
         carrier.setEntryApplicationInstanceId(entryApplicationInstanceId);
 
         if (operationId == DictionaryUtil.nullValue()) {
-            carrier.setEntryOperationName(operationName);
+            if (!StringUtil.isEmpty(operationName)) {
+                carrier.setEntryOperationName(operationName);
+            }
         } else {
             carrier.setEntryOperationId(operationId);
         }
@@ -154,8 +157,8 @@ public class TracingContext implements AbstractTracerContext {
     /**
      * Extract the carrier to build the reference for the pre segment.
      *
-     * @param carrier carried the context from a cross-process segment.
-     * Ref to {@link AbstractTracerContext#extract(ContextCarrier)}
+     * @param carrier carried the context from a cross-process segment. Ref to {@link
+     * AbstractTracerContext#extract(ContextCarrier)}
      */
     @Override
     public void extract(ContextCarrier carrier) {
@@ -171,8 +174,7 @@ public class TracingContext implements AbstractTracerContext {
     /**
      * Capture the snapshot of current context.
      *
-     * @return the snapshot of context for cross-thread propagation
-     * Ref to {@link AbstractTracerContext#capture()}
+     * @return the snapshot of context for cross-thread propagation Ref to {@link AbstractTracerContext#capture()}
      */
     @Override
     public ContextSnapshot capture() {
@@ -197,7 +199,9 @@ public class TracingContext implements AbstractTracerContext {
         snapshot.setEntryApplicationInstanceId(entryApplicationInstanceId);
 
         if (entryOperationId == DictionaryUtil.nullValue()) {
-            snapshot.setEntryOperationName(entryOperationName);
+            if (!StringUtil.isEmpty(entryOperationName)) {
+                snapshot.setEntryOperationName(entryOperationName);
+            }
         } else {
             snapshot.setEntryOperationId(entryOperationId);
         }
@@ -213,8 +217,7 @@ public class TracingContext implements AbstractTracerContext {
     /**
      * Continue the context from the given snapshot of parent thread.
      *
-     * @param snapshot from {@link #capture()} in the parent thread.
-     * Ref to {@link AbstractTracerContext#continued(ContextSnapshot)}
+     * @param snapshot from {@link #capture()} in the parent thread. Ref to {@link AbstractTracerContext#continued(ContextSnapshot)}
      */
     @Override
     public void continued(ContextSnapshot snapshot) {
@@ -236,8 +239,7 @@ public class TracingContext implements AbstractTracerContext {
      * Create an entry span
      *
      * @param operationName most likely a service name
-     * @return span instance.
-     * Ref to {@link EntrySpan}
+     * @return span instance. Ref to {@link EntrySpan}
      */
     @Override
     public AbstractSpan createEntrySpan(final String operationName) {
@@ -282,8 +284,7 @@ public class TracingContext implements AbstractTracerContext {
      * Create a local span
      *
      * @param operationName most likely a local method signature, or business name.
-     * @return the span represents a local logic block.
-     * Ref to {@link LocalSpan}
+     * @return the span represents a local logic block. Ref to {@link LocalSpan}
      */
     @Override
     public AbstractSpan createLocalSpan(final String operationName) {
diff --git a/apm-sniffer/apm-agent-core/src/main/java/org/apache/skywalking/apm/agent/core/context/trace/TraceSegmentRef.java b/apm-sniffer/apm-agent-core/src/main/java/org/apache/skywalking/apm/agent/core/context/trace/TraceSegmentRef.java
index 6fab03e..ef3f287 100644
--- a/apm-sniffer/apm-agent-core/src/main/java/org/apache/skywalking/apm/agent/core/context/trace/TraceSegmentRef.java
+++ b/apm-sniffer/apm-agent-core/src/main/java/org/apache/skywalking/apm/agent/core/context/trace/TraceSegmentRef.java
@@ -98,16 +98,20 @@ public class TraceSegmentRef {
         this.parentApplicationInstanceId = RemoteDownstreamConfig.Agent.APPLICATION_INSTANCE_ID;
         this.entryApplicationInstanceId = snapshot.getEntryApplicationInstanceId();
         String entryOperationName = snapshot.getEntryOperationName();
-        if (entryOperationName.charAt(0) == '#') {
-            this.entryOperationName = entryOperationName.substring(1);
-        } else {
-            this.entryOperationId = Integer.parseInt(entryOperationName);
+        if (!StringUtil.isEmpty(entryOperationName)) {
+            if (entryOperationName.charAt(0) == '#') {
+                this.entryOperationName = entryOperationName.substring(1);
+            } else {
+                this.entryOperationId = Integer.parseInt(entryOperationName);
+            }
         }
         String parentOperationName = snapshot.getParentOperationName();
-        if (parentOperationName.charAt(0) == '#') {
-            this.parentOperationName = parentOperationName.substring(1);
-        } else {
-            this.parentOperationId = Integer.parseInt(parentOperationName);
+        if (!StringUtil.isEmpty(parentOperationName)) {
+            if (parentOperationName.charAt(0) == '#') {
+                this.parentOperationName = parentOperationName.substring(1);
+            } else {
+                this.parentOperationId = Integer.parseInt(parentOperationName);
+            }
         }
     }
 
diff --git a/apm-sniffer/apm-agent-core/src/test/java/org/apache/skywalking/apm/agent/core/context/ContextCarrierV2HeaderTest.java b/apm-sniffer/apm-agent-core/src/test/java/org/apache/skywalking/apm/agent/core/context/ContextCarrierV2HeaderTest.java
index c4f4777..00f4d9f 100644
--- a/apm-sniffer/apm-agent-core/src/test/java/org/apache/skywalking/apm/agent/core/context/ContextCarrierV2HeaderTest.java
+++ b/apm-sniffer/apm-agent-core/src/test/java/org/apache/skywalking/apm/agent/core/context/ContextCarrierV2HeaderTest.java
@@ -53,9 +53,6 @@ public class ContextCarrierV2HeaderTest {
         Assert.assertTrue(hasSW6);
     }
 
-    /**
-     * sampleFlag-segmentId-parentAppInstId-entryAppInstId-peerHost-traceId-entryEndpoint-parentEndpoint
-     */
     @Test
     public void testDeserializeV2Header() {
         ContextCarrier contextCarrier = new ContextCarrier();
@@ -64,8 +61,7 @@ public class ContextCarrierV2HeaderTest {
             next = next.next();
             if (next.getHeadKey().equals("sw3")) {
             } else if (next.getHeadKey().equals("sw6")) {
-                //TODO, wait for base64 solution
-                next.setHeadValue("1-3.4.5-1.2.3-2-10-11-#127.0.0.1:8080--");
+                next.setHeadValue("1-My40LjU=-MS4yLjM=-4-1-1-IzEyNy4wLjAuMTo4MDgw--");
             } else {
                 Assert.fail("unexpected key");
             }
@@ -114,8 +110,12 @@ public class ContextCarrierV2HeaderTest {
             if (next.getHeadKey().equals("sw3")) {
                 Assert.assertEquals("", next.getHeadValue());
             } else if (next.getHeadKey().equals("sw6")) {
-                //TODO, no BASE64
-                Assert.assertEquals("1-1.2.3-4-1-1-#127.0.0.1:8080-3.4.5-#/portal-123", next.getHeadValue());
+                /**
+                 * sampleFlag-traceId-segmentId-spanId-parentAppInstId-entryAppInstId-peerHost-entryEndpoint-parentEndpoint
+                 *
+                 * "1-3.4.5-1.2.3-4-1-1-#127.0.0.1:8080-#/portal-123"
+                 */
+                Assert.assertEquals("1-My40LjU=-MS4yLjM=-4-1-1-IzEyNy4wLjAuMTo4MDgw-Iy9wb3J0YWw=-MTIz", next.getHeadValue());
             } else {
                 Assert.fail("unexpected key");
             }
@@ -130,7 +130,7 @@ public class ContextCarrierV2HeaderTest {
                     Assert.assertEquals("1.2.3|4|1|1|#127.0.0.1:8080|#/portal|123|3.4.5", next.getHeadValue());
                 } else if (next.getHeadKey().equals("sw6")) {
                     //TODO, no BASE64
-                    Assert.assertEquals("1-1.2.3-4-1-1-#127.0.0.1:8080-3.4.5-#/portal-123", next.getHeadValue());
+                    Assert.assertEquals("1-My40LjU=-MS4yLjM=-4-1-1-IzEyNy4wLjAuMTo4MDgw-Iy9wb3J0YWw=-MTIz", next.getHeadValue());
                 } else {
                     Assert.fail("unexpected key");
                 }
@@ -142,4 +142,54 @@ public class ContextCarrierV2HeaderTest {
 
         Assert.assertTrue(contextCarrier.isValid());
     }
+
+    @Test
+    public void testV2HeaderAccurate() {
+        List<DistributedTraceId> distributedTraceIds = new ArrayList<DistributedTraceId>();
+        distributedTraceIds.add(new PropagatedTraceId("3.4.5"));
+
+        ContextCarrier contextCarrier = new ContextCarrier();
+        contextCarrier.setTraceSegmentId(new ID(1, 2, 3));
+        contextCarrier.setDistributedTraceIds(distributedTraceIds);
+        contextCarrier.setSpanId(4);
+        contextCarrier.setEntryApplicationInstanceId(1);
+        contextCarrier.setParentApplicationInstanceId(1);
+        contextCarrier.setPeerHost("127.0.0.1:8080");
+        contextCarrier.setEntryOperationName("/portal");
+        contextCarrier.setParentOperationId(123);
+
+        CarrierItem next = contextCarrier.items();
+        String headerValue = null;
+        while (next.hasNext()) {
+            next = next.next();
+            if (next.getHeadKey().equals("sw3")) {
+                Assert.assertEquals("", next.getHeadValue());
+            } else if (next.getHeadKey().equals("sw6")) {
+                headerValue = next.getHeadValue();
+            } else {
+                Assert.fail("unexpected key");
+            }
+        }
+
+        ContextCarrier contextCarrier2 = new ContextCarrier();
+        next = contextCarrier2.items();
+        while (next.hasNext()) {
+            next = next.next();
+            if (next.getHeadKey().equals("sw3")) {
+            } else if (next.getHeadKey().equals("sw6")) {
+                next.setHeadValue(headerValue);
+            } else {
+                Assert.fail("unexpected key");
+            }
+        }
+
+        Assert.assertTrue(contextCarrier2.isValid());
+        Assert.assertEquals(contextCarrier.getSpanId(), contextCarrier2.getSpanId());
+        Assert.assertEquals(contextCarrier.getPeerHost(), contextCarrier2.getPeerHost());
+        Assert.assertEquals(contextCarrier.getDistributedTraceId(), contextCarrier2.getDistributedTraceId());
+        Assert.assertEquals(contextCarrier.getTraceSegmentId(), contextCarrier2.getTraceSegmentId());
+        Assert.assertEquals(contextCarrier.getEntryOperationName(), contextCarrier2.getEntryOperationName());
+        Assert.assertEquals(contextCarrier.getEntryApplicationInstanceId(), contextCarrier2.getEntryApplicationInstanceId());
+        Assert.assertEquals(contextCarrier.getParentApplicationInstanceId(), contextCarrier2.getParentApplicationInstanceId());
+    }
 }