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 2015/09/24 20:08:48 UTC

incubator-htrace git commit: HTRACE-129: htraced: add /server/stats REST endpoint (cmccabe)

Repository: incubator-htrace
Updated Branches:
  refs/heads/master f52ea334d -> 8bc64091c


HTRACE-129: htraced: add /server/stats REST endpoint (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/8bc64091
Tree: http://git-wip-us.apache.org/repos/asf/incubator-htrace/tree/8bc64091
Diff: http://git-wip-us.apache.org/repos/asf/incubator-htrace/diff/8bc64091

Branch: refs/heads/master
Commit: 8bc64091c2737a78c663dd55f0f1ae1d469ff3c1
Parents: f52ea33
Author: Colin P. Mccabe <cm...@apache.org>
Authored: Thu Sep 24 11:05:44 2015 -0700
Committer: Colin P. Mccabe <cm...@apache.org>
Committed: Thu Sep 24 11:05:44 2015 -0700

----------------------------------------------------------------------
 .../go/src/org/apache/htrace/client/client.go   | 15 +++++++
 .../go/src/org/apache/htrace/common/rest.go     | 16 +++++++
 .../go/src/org/apache/htrace/htrace/cmd.go      | 44 ++++++++++++++++++++
 .../src/org/apache/htrace/htraced/datastore.go  | 23 ++++++++++
 .../go/src/org/apache/htrace/htraced/rest.go    | 43 ++++++++++++++++++-
 5 files changed, 140 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-htrace/blob/8bc64091/htrace-htraced/go/src/org/apache/htrace/client/client.go
----------------------------------------------------------------------
diff --git a/htrace-htraced/go/src/org/apache/htrace/client/client.go b/htrace-htraced/go/src/org/apache/htrace/client/client.go
index 5051d94..2ac8a1e 100644
--- a/htrace-htraced/go/src/org/apache/htrace/client/client.go
+++ b/htrace-htraced/go/src/org/apache/htrace/client/client.go
@@ -67,6 +67,21 @@ func (hcl *Client) GetServerInfo() (*common.ServerInfo, error) {
 	return &info, nil
 }
 
+// Get the htraced server statistics.
+func (hcl *Client) GetServerStats() (*common.ServerStats, error) {
+	buf, _, err := hcl.makeGetRequest("server/stats")
+	if err != nil {
+		return nil, err
+	}
+	var stats common.ServerStats
+	err = json.Unmarshal(buf, &stats)
+	if err != nil {
+		return nil, errors.New(fmt.Sprintf("Error: error unmarshalling response "+
+			"body %s: %s", string(buf), err.Error()))
+	}
+	return &stats, nil
+}
+
 // Get information about a trace span.  Returns nil, nil if the span was not found.
 func (hcl *Client) FindSpan(sid common.SpanId) (*common.Span, error) {
 	buf, rc, err := hcl.makeGetRequest(fmt.Sprintf("span/%s", sid.String()))

http://git-wip-us.apache.org/repos/asf/incubator-htrace/blob/8bc64091/htrace-htraced/go/src/org/apache/htrace/common/rest.go
----------------------------------------------------------------------
diff --git a/htrace-htraced/go/src/org/apache/htrace/common/rest.go b/htrace-htraced/go/src/org/apache/htrace/common/rest.go
index b898ca4..b367ed1 100644
--- a/htrace-htraced/go/src/org/apache/htrace/common/rest.go
+++ b/htrace-htraced/go/src/org/apache/htrace/common/rest.go
@@ -27,3 +27,19 @@ type ServerInfo struct {
 	// The git hash that this software was built with.
 	GitVersion string
 }
+
+// Info returned by /server/stats
+type ServerStats struct {
+	Shards []ShardStats
+}
+
+type ShardStats struct {
+	Path string
+
+	// The approximate number of spans present in this shard.  This may be an
+	// underestimate.
+	ApproxNumSpans uint64
+
+	// leveldb.stats information
+	LevelDbStats string
+}

http://git-wip-us.apache.org/repos/asf/incubator-htrace/blob/8bc64091/htrace-htraced/go/src/org/apache/htrace/htrace/cmd.go
----------------------------------------------------------------------
diff --git a/htrace-htraced/go/src/org/apache/htrace/htrace/cmd.go b/htrace-htraced/go/src/org/apache/htrace/htrace/cmd.go
index 8fd7067..8bc0c64 100644
--- a/htrace-htraced/go/src/org/apache/htrace/htrace/cmd.go
+++ b/htrace-htraced/go/src/org/apache/htrace/htrace/cmd.go
@@ -32,6 +32,7 @@ import (
 	"org/apache/htrace/conf"
 	"os"
 	"time"
+	"strings"
 )
 
 var RELEASE_VERSION string
@@ -62,6 +63,8 @@ func main() {
 	verbose = app.Flag("verbose", "Verbose.").Default("false").Bool()
 	version := app.Command("version", "Print the version of this program.")
 	serverInfo := app.Command("serverInfo", "Print information retrieved from an htraced server.")
+	serverStats := app.Command("serverStats", "Print statistics retrieved from the htraced server.")
+	serverStatsJson := serverStats.Flag("json", "Display statistics as raw JSON.").Default("false").Bool()
 	findSpan := app.Command("findSpan", "Print information about a trace span with a given ID.")
 	findSpanId := findSpan.Arg("id", "Span ID to find. Example: be305e54-4534-2110-a0b2-e06b9effe112").Required().String()
 	findChildren := app.Command("findChildren", "Print out the span IDs that are children of a given span ID.")
@@ -122,6 +125,12 @@ func main() {
 		os.Exit(printVersion())
 	case serverInfo.FullCommand():
 		os.Exit(printServerInfo(hcl))
+	case serverStats.FullCommand():
+		if (*serverStatsJson) {
+			os.Exit(printServerStatsJson(hcl))
+		} else {
+			os.Exit(printServerStats(hcl))
+		}
 	case findSpan.FullCommand():
 		var id *common.SpanId
 		id.FromString(*findSpanId)
@@ -177,6 +186,41 @@ func printServerInfo(hcl *htrace.Client) int {
 	return EXIT_SUCCESS
 }
 
+// Print information retrieved from an htraced server via /server/info
+func printServerStats(hcl *htrace.Client) int {
+	stats, err := hcl.GetServerStats()
+	if err != nil {
+		fmt.Println(err.Error())
+		return EXIT_FAILURE
+	}
+	fmt.Printf("HTraced server stats:\n")
+	fmt.Printf("%d leveldb shards.\n", len(stats.Shards))
+	for i := range(stats.Shards) {
+		shard := stats.Shards[i]
+		fmt.Printf("==== %s ===\n", shard.Path)
+		fmt.Printf("Approximate number of spans: %d\n", shard.ApproxNumSpans)
+		stats := strings.Replace(shard.LevelDbStats, "\\n", "\n", -1)
+		fmt.Printf("%s\n", stats)
+	}
+	return EXIT_SUCCESS
+}
+
+// Print information retrieved from an htraced server via /server/info as JSON
+func printServerStatsJson(hcl *htrace.Client) int {
+	stats, err := hcl.GetServerStats()
+	if err != nil {
+		fmt.Println(err.Error())
+		return EXIT_FAILURE
+	}
+	buf, err := json.MarshalIndent(stats, "", "  ")
+	if err != nil {
+		fmt.Printf("Error marshalling server stats: %s", err.Error())
+		return EXIT_FAILURE
+	}
+	fmt.Printf("%s\n", string(buf))
+	return EXIT_SUCCESS
+}
+
 // Print information about a trace span.
 func doFindSpan(hcl *htrace.Client, sid common.SpanId) int {
 	span, err := hcl.FindSpan(sid)

http://git-wip-us.apache.org/repos/asf/incubator-htrace/blob/8bc64091/htrace-htraced/go/src/org/apache/htrace/htraced/datastore.go
----------------------------------------------------------------------
diff --git a/htrace-htraced/go/src/org/apache/htrace/htraced/datastore.go b/htrace-htraced/go/src/org/apache/htrace/htraced/datastore.go
index 5885168..0595d36 100644
--- a/htrace-htraced/go/src/org/apache/htrace/htraced/datastore.go
+++ b/htrace-htraced/go/src/org/apache/htrace/htraced/datastore.go
@@ -951,3 +951,26 @@ func (store *dataStore) HandleQuery(query *common.Query) ([]*common.Span, error)
 	}
 	return ret, nil
 }
+
+func (store *dataStore) ServerStats() *common.ServerStats {
+	serverStats := common.ServerStats {
+		Shards : make([]common.ShardStats, len(store.shards)),
+	}
+	for shardIdx := range(store.shards) {
+		shard := store.shards[shardIdx]
+		serverStats.Shards[shardIdx].Path = shard.path
+		r := levigo.Range {
+			Start : append([]byte{SPAN_ID_INDEX_PREFIX},
+				common.INVALID_SPAN_ID.Val()...),
+			Limit : append([]byte{SPAN_ID_INDEX_PREFIX + 1},
+				common.INVALID_SPAN_ID.Val()...),
+		}
+		vals := shard.ldb.GetApproximateSizes([]levigo.Range { r })
+		serverStats.Shards[shardIdx].ApproxNumSpans = vals[0]
+		serverStats.Shards[shardIdx].LevelDbStats =
+			shard.ldb.PropertyValue("leveldb.stats")
+		store.lg.Infof("shard.ldb.PropertyValue(leveldb.stats)=%s\n",
+			shard.ldb.PropertyValue("leveldb.stats"))
+	}
+	return &serverStats
+}

http://git-wip-us.apache.org/repos/asf/incubator-htrace/blob/8bc64091/htrace-htraced/go/src/org/apache/htrace/htraced/rest.go
----------------------------------------------------------------------
diff --git a/htrace-htraced/go/src/org/apache/htrace/htraced/rest.go b/htrace-htraced/go/src/org/apache/htrace/htraced/rest.go
index 66f78f8..97b2bca 100644
--- a/htrace-htraced/go/src/org/apache/htrace/htraced/rest.go
+++ b/htrace-htraced/go/src/org/apache/htrace/htraced/rest.go
@@ -25,6 +25,7 @@ import (
 	"fmt"
 	"github.com/gorilla/mux"
 	"io"
+	"io/ioutil"
 	"net"
 	"net/http"
 	"org/apache/htrace/common"
@@ -69,6 +70,24 @@ func (hand *serverInfoHandler) ServeHTTP(w http.ResponseWriter, req *http.Reques
 	w.Write(buf)
 }
 
+type serverStatsHandler struct {
+	dataStoreHandler
+}
+
+func (hand *serverStatsHandler) ServeHTTP(w http.ResponseWriter, req *http.Request) {
+	setResponseHeaders(w.Header())
+	hand.lg.Debugf("serverStatsHandler\n")
+	stats := hand.store.ServerStats()
+	buf, err := json.Marshal(&stats)
+	if err != nil {
+		writeError(hand.lg, w, http.StatusInternalServerError,
+			fmt.Sprintf("error marshalling ServerStats: %s\n", err.Error()))
+		return
+	}
+	hand.lg.Debugf("Returned ServerStats %s\n", string(buf))
+	w.Write(buf)
+}
+
 type dataStoreHandler struct {
 	lg    *common.Logger
 	store *dataStore
@@ -161,7 +180,19 @@ type writeSpansHandler struct {
 
 func (hand *writeSpansHandler) ServeHTTP(w http.ResponseWriter, req *http.Request) {
 	setResponseHeaders(w.Header())
-	dec := json.NewDecoder(req.Body)
+	var dec *json.Decoder
+	if hand.lg.TraceEnabled() {
+		b, err := ioutil.ReadAll(req.Body)
+		if err != nil {
+			writeError(hand.lg, w, http.StatusBadRequest,
+				fmt.Sprintf("Error reading span data: %s", err.Error()))
+			return
+		}
+		hand.lg.Tracef("writeSpansHandler: read %s\n", string(b))
+		dec = json.NewDecoder(bytes.NewBuffer(b))
+	} else {
+		dec = json.NewDecoder(req.Body)
+	}
 	spans := make([]*common.Span, 0, 32)
 	defaultTrid := req.Header.Get("htrace-trid")
 	for {
@@ -175,6 +206,12 @@ func (hand *writeSpansHandler) ServeHTTP(w http.ResponseWriter, req *http.Reques
 			}
 			break
 		}
+		spanIdProblem := span.Id.FindProblem()
+		if spanIdProblem != "" {
+			writeError(hand.lg, w, http.StatusBadRequest,
+				fmt.Sprintf("Invalid span ID: %s", spanIdProblem))
+			return
+		}
 		if span.TracerId == "" {
 			span.TracerId = defaultTrid
 		}
@@ -262,6 +299,10 @@ func CreateRestServer(cnf *conf.Config, store *dataStore) (*RestServer, error) {
 
 	r.Handle("/server/info", &serverInfoHandler{lg: rsv.lg}).Methods("GET")
 
+	serverStatsH := &serverStatsHandler{dataStoreHandler: dataStoreHandler{
+		store: store, lg: rsv.lg}}
+	r.Handle("/server/stats", serverStatsH).Methods("GET")
+
 	writeSpansH := &writeSpansHandler{dataStoreHandler: dataStoreHandler{
 		store: store, lg: rsv.lg}}
 	r.Handle("/writeSpans", writeSpansH).Methods("POST")