You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@htrace.apache.org by cm...@apache.org on 2014/12/30 01:56:13 UTC

incubator-htrace git commit: HTRACE-27. Make Span serialization to JSON consistent across htraced and the java client (cmccabe)

Repository: incubator-htrace
Updated Branches:
  refs/heads/master 3bdab29c6 -> 10e0bc2cf


HTRACE-27. Make Span serialization to JSON consistent across htraced and the java client (cmccabe)


Project: http://git-wip-us.apache.org/repos/asf/incubator-htrace/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-htrace/commit/10e0bc2c
Tree: http://git-wip-us.apache.org/repos/asf/incubator-htrace/tree/10e0bc2c
Diff: http://git-wip-us.apache.org/repos/asf/incubator-htrace/diff/10e0bc2c

Branch: refs/heads/master
Commit: 10e0bc2cfd46de55925fea720c6593716dce8b8f
Parents: 3bdab29
Author: Colin P. Mccabe <cm...@apache.org>
Authored: Mon Dec 29 16:55:36 2014 -0800
Committer: Colin P. Mccabe <cm...@apache.org>
Committed: Mon Dec 29 16:55:36 2014 -0800

----------------------------------------------------------------------
 .../src/go/src/org/apache/htrace/common/span.go | 64 ++++++++++----------
 .../src/org/apache/htrace/common/span_test.go   | 20 +++---
 .../src/org/apache/htrace/htraced/datastore.go  | 18 ++++--
 .../org/apache/htrace/htraced/datastore_test.go | 28 ++++-----
 .../src/go/src/org/apache/htrace/test/random.go | 14 ++---
 .../src/main/java/org/apache/htrace/Span.java   |  4 +-
 .../java/org/apache/htrace/impl/MilliSpan.java  | 43 +++++++------
 7 files changed, 103 insertions(+), 88 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-htrace/blob/10e0bc2c/htrace-core/src/go/src/org/apache/htrace/common/span.go
----------------------------------------------------------------------
diff --git a/htrace-core/src/go/src/org/apache/htrace/common/span.go b/htrace-core/src/go/src/org/apache/htrace/common/span.go
index e682201..81fe0e8 100644
--- a/htrace-core/src/go/src/org/apache/htrace/common/span.go
+++ b/htrace-core/src/go/src/org/apache/htrace/common/span.go
@@ -21,25 +21,19 @@ package common
 
 import (
 	"encoding/json"
+	"fmt"
+	"strconv"
 )
 
 //
 // Represents a trace span.
 //
 // Compatibility notes:
-// We use signed numbers here, even in cases where unsigned would make more sense.  This is because
-// Java doesn't support unsigned integers, and we'd like to match the representation used by the
-// Java client.  For example, if we log a message about a span id in the Java client, it would be
-// nice if we could match it up with a log message about the same span id in this server, without
-// doing a mental conversion from signed to unsigned.
+// When converting to JSON, we store the 64-bit numbers as hexadecimal strings rather than as
+// integers.  This is because JavaScript lacks the ability to handle 64-bit integers.  Numbers above
+// about 55 bits will be rounded by Javascript.  Since the Javascript UI is a primary consumer of
+// this JSON data, we have to simply pass it as a string.
 //
-// When converting to JSON, we store the 64-bit numbers as strings rather than as integers.  This is
-// because JavaScript lacks the ability to handle 64-bit integers.  Numbers above about 55 bits will
-// be rounded by Javascript.  Since the Javascript UI is a primary consumer of this JSON data, we
-// have to simply pass it as a string.
-//
-
-const INVALID_SPAN_ID = 0
 
 type TraceInfoMap map[string][]byte
 
@@ -48,21 +42,37 @@ type TimelineAnnotation struct {
 	Msg  string `json:"msg"`
 }
 
-type SpanIdSlice []int64
+type SpanId int64
+
+func (id SpanId) String() string {
+	return fmt.Sprintf("%08x", id)
+}
+
+func (id SpanId) Val() int64 {
+	return int64(id)
+}
+
+func (id SpanId) MarshalJSON() ([]byte, error) {
+	return []byte(`"` + fmt.Sprintf("%016x", uint64(id)) + `"`), nil
+}
+
+func (id SpanId) UnMarshalJSON() ([]byte, error) {
+	return []byte(`"` + strconv.FormatUint(uint64(id), 16) + `"`), nil
+}
 
 type SpanData struct {
-	Start               int64                `json:"start,string"`
-	Stop                int64                `json:"stop,string"`
-	Description         string               `json:"desc"`
-	TraceId             int64                `json:"tid,string"`
-	ParentId            int64                `json:"prid,string"`
-	Info                TraceInfoMap         `json:"info,omitempty"`
-	ProcessId           string               `json:"pid"`
-	TimelineAnnotations []TimelineAnnotation `json:"ta,omitempty"`
+	Begin               int64                `json:"b,string"`
+	End                 int64                `json:"e,string"`
+	Description         string               `json:"d"`
+	TraceId             SpanId               `json:"i"`
+	Parents             []SpanId             `json:"p"`
+	Info                TraceInfoMap         `json:"n,omitempty"`
+	ProcessId           string               `json:"r"`
+	TimelineAnnotations []TimelineAnnotation `json:"t,omitempty"`
 }
 
 type Span struct {
-	SpanId int64 `json:"sid,string"`
+	Id SpanId `json:"s,string"`
 	SpanData
 }
 
@@ -73,13 +83,3 @@ func (span *Span) ToJson() []byte {
 	}
 	return jbytes
 }
-
-type SpanSlice []Span
-
-func (spans SpanSlice) ToJson() []byte {
-	jbytes, err := json.Marshal(spans)
-	if err != nil {
-		panic(err)
-	}
-	return jbytes
-}

http://git-wip-us.apache.org/repos/asf/incubator-htrace/blob/10e0bc2c/htrace-core/src/go/src/org/apache/htrace/common/span_test.go
----------------------------------------------------------------------
diff --git a/htrace-core/src/go/src/org/apache/htrace/common/span_test.go b/htrace-core/src/go/src/org/apache/htrace/common/span_test.go
index deef5bf..1d098fc 100644
--- a/htrace-core/src/go/src/org/apache/htrace/common/span_test.go
+++ b/htrace-core/src/go/src/org/apache/htrace/common/span_test.go
@@ -25,29 +25,29 @@ import (
 
 func TestSpanToJson(t *testing.T) {
 	t.Parallel()
-	span := Span{SpanId: 2305843009213693952,
+	span := Span{Id: 2305843009213693952,
 		SpanData: SpanData{
-			Start:       123,
-			Stop:        456,
+			Begin:       123,
+			End:         456,
 			Description: "getFileDescriptors",
 			TraceId:     999,
-			ParentId:    INVALID_SPAN_ID,
+			Parents:     []SpanId{},
 			ProcessId:   "testProcessId",
 		}}
 	ExpectStrEqual(t,
-		`{"sid":"2305843009213693952","start":"123","stop":"456","desc":"getFileDescriptors","tid":"999","prid":"0","pid":"testProcessId"}`,
+		`{"s":"2000000000000000","b":"123","e":"456","d":"getFileDescriptors","i":"00000000000003e7","p":[],"r":"testProcessId"}`,
 		string(span.ToJson()))
 }
 
 func TestAnnotatedSpanToJson(t *testing.T) {
 	t.Parallel()
-	span := Span{SpanId: 1305813009213693952,
+	span := Span{Id: 1305813009213693952,
 		SpanData: SpanData{
-			Start:       1234,
-			Stop:        4567,
+			Begin:       1234,
+			End:         4567,
 			Description: "getFileDescriptors2",
 			TraceId:     999,
-			ParentId:    INVALID_SPAN_ID,
+			Parents:     []SpanId{},
 			ProcessId:   "testAnnotatedProcessId",
 			TimelineAnnotations: []TimelineAnnotation{
 				TimelineAnnotation{
@@ -61,6 +61,6 @@ func TestAnnotatedSpanToJson(t *testing.T) {
 			},
 		}}
 	ExpectStrEqual(t,
-		`{"sid":"1305813009213693952","start":"1234","stop":"4567","desc":"getFileDescriptors2","tid":"999","prid":"0","pid":"testAnnotatedProcessId","ta":[{"time":"7777","msg":"contactedServer"},{"time":"8888","msg":"passedFd"}]}`,
+		`{"s":"121f2e036d442000","b":"1234","e":"4567","d":"getFileDescriptors2","i":"00000000000003e7","p":[],"r":"testAnnotatedProcessId","t":[{"time":"7777","msg":"contactedServer"},{"time":"8888","msg":"passedFd"}]}`,
 		string(span.ToJson()))
 }

http://git-wip-us.apache.org/repos/asf/incubator-htrace/blob/10e0bc2c/htrace-core/src/go/src/org/apache/htrace/htraced/datastore.go
----------------------------------------------------------------------
diff --git a/htrace-core/src/go/src/org/apache/htrace/htraced/datastore.go b/htrace-core/src/go/src/org/apache/htrace/htraced/datastore.go
index d78e369..b43d4ce 100644
--- a/htrace-core/src/go/src/org/apache/htrace/htraced/datastore.go
+++ b/htrace-core/src/go/src/org/apache/htrace/htraced/datastore.go
@@ -189,13 +189,15 @@ func (shd *shard) writeSpan(span *common.Span) error {
 	if err != nil {
 		return err
 	}
-	batch.Put(makeKey('s', span.SpanId), spanDataBuf.Bytes())
+	batch.Put(makeKey('s', span.Id.Val()), spanDataBuf.Bytes())
 
 	// Add this to the parent index.
-	batch.Put(makeSecondaryKey('p', span.ParentId, span.SpanId), EMPTY_BYTE_BUF)
+	for parentIdx := range span.Parents {
+		batch.Put(makeSecondaryKey('p', span.Parents[parentIdx].Val(), span.Id.Val()), EMPTY_BYTE_BUF)
+	}
 
 	// Add this to the timeline index.
-	batch.Put(makeSecondaryKey('t', span.ParentId, span.SpanId), EMPTY_BYTE_BUF)
+	batch.Put(makeSecondaryKey('t', span.Begin, span.Id.Val()), EMPTY_BYTE_BUF)
 
 	err = shd.ldb.Write(shd.store.writeOpts, batch)
 	if err != nil {
@@ -375,7 +377,7 @@ func (store *dataStore) getShardIndex(spanId int64) int {
 }
 
 func (store *dataStore) WriteSpan(span *common.Span) {
-	store.shards[store.getShardIndex(span.SpanId)].incoming <- span
+	store.shards[store.getShardIndex(span.Id.Val())].incoming <- span
 }
 
 func (store *dataStore) FindSpan(sid int64) *common.Span {
@@ -392,7 +394,6 @@ func (shd *shard) FindSpan(sid int64) *common.Span {
 			shd.path, sid, err.Error())
 		return nil
 	}
-	// check for empty buf here?
 	r := bytes.NewBuffer(buf)
 	decoder := gob.NewDecoder(r)
 	data := common.SpanData{}
@@ -402,7 +403,12 @@ func (shd *shard) FindSpan(sid int64) *common.Span {
 			shd.path, sid, err.Error())
 		return nil
 	}
-	return &common.Span{SpanId: sid, SpanData: data}
+	// Gob encoding translates empty slices to nil.  Reverse this so that we're always dealing with
+	// non-nil slices.
+	if data.Parents == nil {
+		data.Parents = []common.SpanId{}
+	}
+	return &common.Span{Id: common.SpanId(sid), SpanData: data}
 }
 
 // Find the children of a given span id.

http://git-wip-us.apache.org/repos/asf/incubator-htrace/blob/10e0bc2c/htrace-core/src/go/src/org/apache/htrace/htraced/datastore_test.go
----------------------------------------------------------------------
diff --git a/htrace-core/src/go/src/org/apache/htrace/htraced/datastore_test.go b/htrace-core/src/go/src/org/apache/htrace/htraced/datastore_test.go
index bbfd688..f037145 100644
--- a/htrace-core/src/go/src/org/apache/htrace/htraced/datastore_test.go
+++ b/htrace-core/src/go/src/org/apache/htrace/htraced/datastore_test.go
@@ -38,31 +38,31 @@ func TestCreateDatastore(t *testing.T) {
 }
 
 var SIMPLE_TEST_SPANS []common.Span = []common.Span{
-	common.Span{SpanId: 1,
+	common.Span{Id: 1,
 		SpanData: common.SpanData{
-			Start:       123,
-			Stop:        456,
+			Begin:       123,
+			End:         456,
 			Description: "getFileDescriptors",
 			TraceId:     999,
-			ParentId:    common.INVALID_SPAN_ID,
+			Parents:     []common.SpanId{},
 			ProcessId:   "firstd",
 		}},
-	common.Span{SpanId: 2,
+	common.Span{Id: 2,
 		SpanData: common.SpanData{
-			Start:       125,
-			Stop:        200,
+			Begin:       125,
+			End:         200,
 			Description: "openFd",
 			TraceId:     999,
-			ParentId:    1,
+			Parents:     []common.SpanId{1},
 			ProcessId:   "secondd",
 		}},
-	common.Span{SpanId: 3,
+	common.Span{Id: 3,
 		SpanData: common.SpanData{
-			Start:       200,
-			Stop:        456,
+			Begin:       200,
+			End:         456,
 			Description: "passFd",
 			TraceId:     999,
-			ParentId:    1,
+			Parents:     []common.SpanId{1},
 			ProcessId:   "thirdd",
 		}},
 }
@@ -95,10 +95,10 @@ func TestDatastoreWriteAndRead(t *testing.T) {
 	if span == nil {
 		t.Fatal()
 	}
-	if span.SpanId != 1 {
+	if span.Id != 1 {
 		t.Fatal()
 	}
-	common.ExpectSpansEqual(t, span, &SIMPLE_TEST_SPANS[0])
+	common.ExpectSpansEqual(t, &SIMPLE_TEST_SPANS[0], span)
 	children := ht.Store.FindChildren(1, 1)
 	if len(children) != 1 {
 		t.Fatalf("expected 1 child, but got %d\n", len(children))

http://git-wip-us.apache.org/repos/asf/incubator-htrace/blob/10e0bc2c/htrace-core/src/go/src/org/apache/htrace/test/random.go
----------------------------------------------------------------------
diff --git a/htrace-core/src/go/src/org/apache/htrace/test/random.go b/htrace-core/src/go/src/org/apache/htrace/test/random.go
index 8605b70..d10e2f9 100644
--- a/htrace-core/src/go/src/org/apache/htrace/test/random.go
+++ b/htrace-core/src/go/src/org/apache/htrace/test/random.go
@@ -53,20 +53,20 @@ func NonZeroRand32(rnd *rand.Rand) int32 {
 
 // Create a random span.
 func NewRandomSpan(rnd *rand.Rand, potentialParents []*common.Span) *common.Span {
-	var parentId int64 = common.INVALID_SPAN_ID
+	parents := []common.SpanId{}
 	if potentialParents != nil {
 		parentIdx := rnd.Intn(len(potentialParents) + 1)
 		if parentIdx < len(potentialParents) {
-			parentId = potentialParents[parentIdx].SpanId
+			parents = []common.SpanId{potentialParents[parentIdx].Id}
 		}
 	}
-	return &common.Span{SpanId: NonZeroRand64(rnd),
+	return &common.Span{Id: common.SpanId(NonZeroRand64(rnd)),
 		SpanData: common.SpanData{
-			Start:       NonZeroRand64(rnd),
-			Stop:        NonZeroRand64(rnd),
+			Begin:       NonZeroRand64(rnd),
+			End:         NonZeroRand64(rnd),
 			Description: "getFileDescriptors",
-			TraceId:     NonZeroRand64(rnd),
-			ParentId:    parentId,
+			TraceId:     common.SpanId(NonZeroRand64(rnd)),
+			Parents:     parents,
 			ProcessId:   fmt.Sprintf("process%d", NonZeroRand32(rnd)),
 		}}
 }

http://git-wip-us.apache.org/repos/asf/incubator-htrace/blob/10e0bc2c/htrace-core/src/main/java/org/apache/htrace/Span.java
----------------------------------------------------------------------
diff --git a/htrace-core/src/main/java/org/apache/htrace/Span.java b/htrace-core/src/main/java/org/apache/htrace/Span.java
index b08cfc8..50cec7c 100644
--- a/htrace-core/src/main/java/org/apache/htrace/Span.java
+++ b/htrace-core/src/main/java/org/apache/htrace/Span.java
@@ -81,8 +81,8 @@ public interface Span {
   String toString();
 
   /**
-   * Return the pseudo-unique (random) number of the parent span, returns
-   * ROOT_SPAN_ID if this is the root span
+   * Return the pseudo-unique (random) number of the first parent span, returns
+   * ROOT_SPAN_ID if there are no parents.
    */
   long getParentId();
 

http://git-wip-us.apache.org/repos/asf/incubator-htrace/blob/10e0bc2c/htrace-core/src/main/java/org/apache/htrace/impl/MilliSpan.java
----------------------------------------------------------------------
diff --git a/htrace-core/src/main/java/org/apache/htrace/impl/MilliSpan.java b/htrace-core/src/main/java/org/apache/htrace/impl/MilliSpan.java
index f313e61..b58839b 100644
--- a/htrace-core/src/main/java/org/apache/htrace/impl/MilliSpan.java
+++ b/htrace-core/src/main/java/org/apache/htrace/impl/MilliSpan.java
@@ -41,7 +41,7 @@ public class MilliSpan implements Span {
   private long stop;
   private final String description;
   private final long traceId;
-  private final long parentSpanId;
+  private final long parents[];
   private final long spanId;
   private Map<byte[], byte[]> traceInfo = null;
   private final String processId;
@@ -55,7 +55,11 @@ public class MilliSpan implements Span {
   public MilliSpan(String description, long traceId, long parentSpanId, long spanId, String processId) {
     this.description = description;
     this.traceId = traceId;
-    this.parentSpanId = parentSpanId;
+    if (parentSpanId == Span.ROOT_SPAN_ID) {
+      this.parents = new long[0];
+    } else {
+      this.parents = new long[] { parentSpanId };
+    } 
     this.spanId = spanId;
     this.start = System.currentTimeMillis();
     this.stop = 0;
@@ -93,7 +97,7 @@ public class MilliSpan implements Span {
 
   @Override
   public String toString() {
-    return String.format("Span{Id:0x%16x,parentId:0x%16x,desc:%s}", spanId, parentSpanId, description);
+    return toJson();
   }
 
   @Override
@@ -106,9 +110,14 @@ public class MilliSpan implements Span {
     return spanId;
   }
 
+  // TODO: Fix API callers to deal with multiple parents, and get rid of
+  // Span.ROOT_SPAN_ID.
   @Override
   public long getParentId() {
-    return parentSpanId;
+    if (parents.length == 0) {
+      return Span.ROOT_SPAN_ID;
+    }
+    return parents[0];
   }
 
   @Override
@@ -160,26 +169,26 @@ public class MilliSpan implements Span {
   public String getProcessId() {
     return processId;
   }
-  
+
   @Override
   public String toJson() {
     Map<String, Object> values = new LinkedHashMap<String, Object>();
-    values.put("TraceID", traceId);
-    values.put("SpanID", spanId);
-    values.put("ParentID", parentSpanId);
-    if (processId != null) {
-      values.put("ProcessID", processId);
-    }
-    values.put("Start", start);
-    values.put("Stop", stop);
-    if (description != null) {
-      values.put("Description", description);
+    values.put("i", String.format("%016x", traceId));
+    values.put("s", String.format("%016x", spanId));
+    String parentStrs[] = new String[parents.length];
+    for (int parentIdx = 0; parentIdx < parents.length; parentIdx++) {
+      parentStrs[parentIdx] = String.format("%016x", parents[parentIdx]);
     }
+    values.put("p", parentStrs);
+    values.put("r", processId);
+    values.put("b", Long.toString(start));
+    values.put("e", Long.toString(stop));
+    values.put("d", description);
     if (timeline != null) {
-      values.put("TLAnnotations", timeline);
+      values.put("t", timeline);
     }
     if (traceInfo != null){
-      values.put("KVAnnotations", traceInfo);
+      values.put("n", traceInfo);
     }
     return JSON.toString(values);
   }