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/11/17 23:19:51 UTC

incubator-htrace git commit: HTRACE-299. htraced: add /server/debugInfo REST endpoint to get stack traces and GC stats (cmccabe)

Repository: incubator-htrace
Updated Branches:
  refs/heads/master 3845a187e -> fc0d8f38f


HTRACE-299. htraced: add /server/debugInfo REST endpoint to get stack traces and GC stats (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/fc0d8f38
Tree: http://git-wip-us.apache.org/repos/asf/incubator-htrace/tree/fc0d8f38
Diff: http://git-wip-us.apache.org/repos/asf/incubator-htrace/diff/fc0d8f38

Branch: refs/heads/master
Commit: fc0d8f38f6302b2eb66caf6727887893a01a7ed5
Parents: 3845a18
Author: Colin Patrick Mccabe <cm...@cloudera.com>
Authored: Tue Nov 17 14:10:26 2015 -0800
Committer: Colin Patrick Mccabe <cm...@cloudera.com>
Committed: Tue Nov 17 14:10:26 2015 -0800

----------------------------------------------------------------------
 .../go/src/org/apache/htrace/client/client.go   | 15 +++++
 .../go/src/org/apache/htrace/common/process.go  | 58 ++++++++++++--------
 .../go/src/org/apache/htrace/common/rpc.go      | 11 ++++
 .../org/apache/htrace/htraced/client_test.go    | 26 +++++++++
 .../go/src/org/apache/htrace/htraced/rest.go    | 23 ++++++++
 .../go/src/org/apache/htrace/htracedTool/cmd.go | 19 +++++++
 .../src/main/webapp/app/server_debug_info.js    | 30 ++++++++++
 .../src/main/webapp/app/server_info_view.js     | 21 +++++++
 htrace-webapp/src/main/webapp/index.html        |  2 +
 9 files changed, 182 insertions(+), 23 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-htrace/blob/fc0d8f38/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 28b9e29..1140209 100644
--- a/htrace-htraced/go/src/org/apache/htrace/client/client.go
+++ b/htrace-htraced/go/src/org/apache/htrace/client/client.go
@@ -69,6 +69,21 @@ func (hcl *Client) GetServerVersion() (*common.ServerVersion, error) {
 	return &info, nil
 }
 
+// Get the htraced server debug information.
+func (hcl *Client) GetServerDebugInfo() (*common.ServerDebugInfo, error) {
+	buf, _, err := hcl.makeGetRequest("server/debugInfo")
+	if err != nil {
+		return nil, err
+	}
+	var debugInfo common.ServerDebugInfo
+	err = json.Unmarshal(buf, &debugInfo)
+	if err != nil {
+		return nil, errors.New(fmt.Sprintf("Error: error unmarshalling response "+
+			"body %s: %s", string(buf), err.Error()))
+	}
+	return &debugInfo, nil
+}
+
 // Get the htraced server statistics.
 func (hcl *Client) GetServerStats() (*common.ServerStats, error) {
 	buf, _, err := hcl.makeGetRequest("server/stats")

http://git-wip-us.apache.org/repos/asf/incubator-htrace/blob/fc0d8f38/htrace-htraced/go/src/org/apache/htrace/common/process.go
----------------------------------------------------------------------
diff --git a/htrace-htraced/go/src/org/apache/htrace/common/process.go b/htrace-htraced/go/src/org/apache/htrace/common/process.go
index 2127ecf..dbe9b93 100644
--- a/htrace-htraced/go/src/org/apache/htrace/common/process.go
+++ b/htrace-htraced/go/src/org/apache/htrace/common/process.go
@@ -20,6 +20,8 @@
 package common
 
 import (
+	"bytes"
+	"fmt"
 	"org/apache/htrace/conf"
 	"os"
 	"os/signal"
@@ -54,36 +56,46 @@ func InstallSignalHandlers(cnf *conf.Config) {
 	sigQuitChan := make(chan os.Signal, 1)
 	signal.Notify(sigQuitChan, syscall.SIGQUIT)
 	go func() {
-		bufSize := 1 << 20
-		buf := make([]byte, bufSize)
+		stackTraceBuf := make([]byte, 1 << 20)
 		for {
 			<-sigQuitChan
-			neededBytes := runtime.Stack(buf, true)
-			if neededBytes > bufSize {
-				bufSize = neededBytes
-				buf = make([]byte, bufSize)
-				runtime.Stack(buf, true)
-			}
+			GetStackTraces(&stackTraceBuf)
 			lg.Info("=== received SIGQUIT ===\n")
 			lg.Info("=== GOROUTINE STACKS ===\n")
-			lg.Info(string(buf[:neededBytes]))
+			lg.Info(string(stackTraceBuf))
 			lg.Info("\n=== END GOROUTINE STACKS ===\n")
-			gcs := debug.GCStats{}
-			debug.ReadGCStats(&gcs)
 			lg.Info("=== GC STATISTICS ===\n")
-			lg.Infof("LastGC: %s\n", gcs.LastGC.UTC().String())
-			lg.Infof("NumGC: %d\n", gcs.NumGC)
-			lg.Infof("PauseTotal: %v\n", gcs.PauseTotal)
-			if gcs.Pause != nil {
-				pauseStr := ""
-				prefix := ""
-				for p := range gcs.Pause {
-					pauseStr += prefix + gcs.Pause[p].String()
-					prefix = ", "
-				}
-				lg.Infof("Pause History: %s\n", pauseStr)
-			}
+			lg.Info(GetGCStats())
 			lg.Info("=== END GC STATISTICS ===\n")
 		}
 	}()
 }
+
+func GetStackTraces(buf *[]byte) {
+	*buf = (*buf)[0:cap(*buf)]
+	neededBytes := runtime.Stack(*buf, true)
+	for ;neededBytes > len(*buf); {
+		*buf = make([]byte, neededBytes)
+		runtime.Stack(*buf, true)
+	}
+	*buf = (*buf)[0:neededBytes]
+}
+
+func GetGCStats() string {
+	gcs := debug.GCStats{}
+	debug.ReadGCStats(&gcs)
+	var buf bytes.Buffer
+	buf.WriteString(fmt.Sprintf("LastGC: %s\n", gcs.LastGC.UTC().String()))
+	buf.WriteString(fmt.Sprintf("NumGC: %d\n", gcs.NumGC))
+	buf.WriteString(fmt.Sprintf("PauseTotal: %v\n", gcs.PauseTotal))
+	if gcs.Pause != nil {
+		pauseStr := ""
+		prefix := ""
+		for p := range gcs.Pause {
+			pauseStr += prefix + gcs.Pause[p].String()
+			prefix = ", "
+		}
+		buf.WriteString(fmt.Sprintf("Pause History: %s\n", pauseStr))
+	}
+	return buf.String()
+}

http://git-wip-us.apache.org/repos/asf/incubator-htrace/blob/fc0d8f38/htrace-htraced/go/src/org/apache/htrace/common/rpc.go
----------------------------------------------------------------------
diff --git a/htrace-htraced/go/src/org/apache/htrace/common/rpc.go b/htrace-htraced/go/src/org/apache/htrace/common/rpc.go
index 74008bc..2627c26 100644
--- a/htrace-htraced/go/src/org/apache/htrace/common/rpc.go
+++ b/htrace-htraced/go/src/org/apache/htrace/common/rpc.go
@@ -150,3 +150,14 @@ type StorageDirectoryStats struct {
 	// leveldb.stats information
 	LevelDbStats string
 }
+
+type ServerDebugInfoReq struct {
+}
+
+type ServerDebugInfo struct {
+	// Stack traces from all goroutines
+	StackTraces string
+
+	// Garbage collection statistics
+	GCStats string
+}

http://git-wip-us.apache.org/repos/asf/incubator-htrace/blob/fc0d8f38/htrace-htraced/go/src/org/apache/htrace/htraced/client_test.go
----------------------------------------------------------------------
diff --git a/htrace-htraced/go/src/org/apache/htrace/htraced/client_test.go b/htrace-htraced/go/src/org/apache/htrace/htraced/client_test.go
index 0b38481..9a51cd4 100644
--- a/htrace-htraced/go/src/org/apache/htrace/htraced/client_test.go
+++ b/htrace-htraced/go/src/org/apache/htrace/htraced/client_test.go
@@ -49,6 +49,32 @@ func TestClientGetServerVersion(t *testing.T) {
 	}
 }
 
+func TestClientGetServerDebugInfo(t *testing.T) {
+	htraceBld := &MiniHTracedBuilder{Name: "TestClientGetServerDebugInfo",
+		DataDirs: make([]string, 2)}
+	ht, err := htraceBld.Build()
+	if err != nil {
+		t.Fatalf("failed to create datastore: %s", err.Error())
+	}
+	defer ht.Close()
+	var hcl *htrace.Client
+	hcl, err = htrace.NewClient(ht.ClientConf())
+	if err != nil {
+		t.Fatalf("failed to create client: %s", err.Error())
+	}
+	defer hcl.Close()
+	debugInfo, err := hcl.GetServerDebugInfo()
+	if err != nil {
+		t.Fatalf("failed to call GetServerDebugInfo: %s", err.Error())
+	}
+	if debugInfo.StackTraces == "" {
+		t.Fatalf(`debugInfo.StackTraces == ""`)
+	}
+	if debugInfo.GCStats == "" {
+		t.Fatalf(`debugInfo.GCStats == ""`)
+	}
+}
+
 func createRandomTestSpans(amount int) common.SpanSlice {
 	rnd := rand.New(rand.NewSource(2))
 	allSpans := make(common.SpanSlice, amount)

http://git-wip-us.apache.org/repos/asf/incubator-htrace/blob/fc0d8f38/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 9b78d15..432375d 100644
--- a/htrace-htraced/go/src/org/apache/htrace/htraced/rest.go
+++ b/htrace-htraced/go/src/org/apache/htrace/htraced/rest.go
@@ -71,6 +71,28 @@ func (hand *serverVersionHandler) ServeHTTP(w http.ResponseWriter, req *http.Req
 	w.Write(buf)
 }
 
+type serverDebugInfoHandler struct {
+	lg *common.Logger
+}
+
+func (hand *serverDebugInfoHandler) ServeHTTP(w http.ResponseWriter, req *http.Request) {
+	setResponseHeaders(w.Header())
+	buf := make([]byte, 1 << 20)
+	common.GetStackTraces(&buf)
+	resp := common.ServerDebugInfo{
+		StackTraces: string(buf),
+		GCStats: common.GetGCStats(),
+	}
+	buf, err := json.Marshal(&resp)
+	if err != nil {
+		writeError(hand.lg, w, http.StatusInternalServerError,
+			fmt.Sprintf("error marshalling ServerDebugInfo: %s\n", err.Error()))
+		return
+	}
+	w.Write(buf)
+	hand.lg.Info("Returned ServerDebugInfo\n")
+}
+
 type serverStatsHandler struct {
 	dataStoreHandler
 }
@@ -317,6 +339,7 @@ func CreateRestServer(cnf *conf.Config, store *dataStore,
 
 	r.Handle("/server/info", &serverVersionHandler{lg: rsv.lg}).Methods("GET")
 	r.Handle("/server/version", &serverVersionHandler{lg: rsv.lg}).Methods("GET")
+	r.Handle("/server/debugInfo", &serverDebugInfoHandler{lg: rsv.lg}).Methods("GET")
 
 	serverStatsH := &serverStatsHandler{dataStoreHandler: dataStoreHandler{
 		store: store, lg: rsv.lg}}

http://git-wip-us.apache.org/repos/asf/incubator-htrace/blob/fc0d8f38/htrace-htraced/go/src/org/apache/htrace/htracedTool/cmd.go
----------------------------------------------------------------------
diff --git a/htrace-htraced/go/src/org/apache/htrace/htracedTool/cmd.go b/htrace-htraced/go/src/org/apache/htrace/htracedTool/cmd.go
index 88071c7..394e1c1 100644
--- a/htrace-htraced/go/src/org/apache/htrace/htracedTool/cmd.go
+++ b/htrace-htraced/go/src/org/apache/htrace/htracedTool/cmd.go
@@ -73,6 +73,7 @@ func main() {
 	serverVersion := app.Command("serverVersion", "Print the version of the 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()
+	serverDebugInfo := app.Command("serverDebugInfo", "Print the debug info of the htraced server.")
 	serverConf := app.Command("serverConf", "Print the server configuration retrieved from the htraced server.")
 	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()
@@ -140,6 +141,8 @@ func main() {
 		} else {
 			os.Exit(printServerStats(hcl))
 		}
+	case serverDebugInfo.FullCommand():
+		os.Exit(printServerDebugInfo(hcl))
 	case serverConf.FullCommand():
 		os.Exit(printServerConfJson(hcl))
 	case findSpan.FullCommand():
@@ -264,6 +267,22 @@ func printServerStatsJson(hcl *htrace.Client) int {
 	return EXIT_SUCCESS
 }
 
+// Print information retrieved from an htraced server via /server/debugInfo
+func printServerDebugInfo(hcl *htrace.Client) int {
+	stats, err := hcl.GetServerDebugInfo()
+	if err != nil {
+		fmt.Println(err.Error())
+		return EXIT_FAILURE
+	}
+	fmt.Println("=== GOROUTINE STACKS ===")
+	fmt.Print(stats.StackTraces)
+	fmt.Println("=== END GOROUTINE STACKS ===")
+	fmt.Println("=== GC STATISTICS ===")
+	fmt.Print(stats.GCStats)
+	fmt.Println("=== END GC STATISTICS ===")
+	return EXIT_SUCCESS
+}
+
 // Print information retrieved from an htraced server via /server/conf as JSON
 func printServerConfJson(hcl *htrace.Client) int {
 	cnf, err := hcl.GetServerConf()

http://git-wip-us.apache.org/repos/asf/incubator-htrace/blob/fc0d8f38/htrace-webapp/src/main/webapp/app/server_debug_info.js
----------------------------------------------------------------------
diff --git a/htrace-webapp/src/main/webapp/app/server_debug_info.js b/htrace-webapp/src/main/webapp/app/server_debug_info.js
new file mode 100644
index 0000000..2331f7c
--- /dev/null
+++ b/htrace-webapp/src/main/webapp/app/server_debug_info.js
@@ -0,0 +1,30 @@
+/*
+ * 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.
+ */
+
+// htraced server debug info.  See rest.go.
+htrace.ServerDebugInfo = Backbone.Model.extend({
+  defaults: {
+    "StackTraces": "(unknown)",
+    "GCStats": "(unknown)"
+  },
+
+  url: function() {
+    return "server/debugInfo";
+  }
+});

http://git-wip-us.apache.org/repos/asf/incubator-htrace/blob/fc0d8f38/htrace-webapp/src/main/webapp/app/server_info_view.js
----------------------------------------------------------------------
diff --git a/htrace-webapp/src/main/webapp/app/server_info_view.js b/htrace-webapp/src/main/webapp/app/server_info_view.js
index 4255995..62775e8 100644
--- a/htrace-webapp/src/main/webapp/app/server_info_view.js
+++ b/htrace-webapp/src/main/webapp/app/server_info_view.js
@@ -23,6 +23,7 @@ htrace.ServerInfoView = Backbone.View.extend({
   events: {
     "click .serverConfigurationButton": "showServerConfigurationModal",
     "click .storageDirectoryStatsButton": "showStorageDirectoryStatsModal",
+    "click .debugInfoButton": "showDebugInfoModal",
   },
 
   render: function() {
@@ -124,5 +125,25 @@ htrace.ServerInfoView = Backbone.View.extend({
     }
     htrace.showModal(_.template($("#modal-table-template").html())(
           {title: "HTraced Storage Directory Statistics", body: out}));
+  },
+
+  showDebugInfoModal: function() {
+    var config = new htrace.ServerDebugInfo();
+    var view = this;
+    config.fetch({
+      "error": function(model, response, options) {
+        window.alert("Failed to fetch htraced server debug info: " +
+                     JSON.stringify(response));
+      },
+      "success": function(model, response, options) {
+        var out = "";
+        out += "<h2>Stack Traces</h2>";
+        out += "<pre>" + model.get("StackTraces") + "</pre>";
+        out += "<h2>GC Stats</h2>";
+        out += "<pre>" + model.get("GCStats") + "</pre>";
+        htrace.showModal(_.template($("#modal-table-template").html())(
+            {title: "HTraced Debug Info", body: out}));
+      }
+    })
   }
 });

http://git-wip-us.apache.org/repos/asf/incubator-htrace/blob/fc0d8f38/htrace-webapp/src/main/webapp/index.html
----------------------------------------------------------------------
diff --git a/htrace-webapp/src/main/webapp/index.html b/htrace-webapp/src/main/webapp/index.html
index a59282a..16fd2f9 100644
--- a/htrace-webapp/src/main/webapp/index.html
+++ b/htrace-webapp/src/main/webapp/index.html
@@ -110,6 +110,7 @@
           </div>
           <button type="button" class="btn btn-info serverConfigurationButton">Server Configuration</button>
           <button type="button" class="btn btn-success storageDirectoryStatsButton">Storage Directory Stats</button>
+          <button type="button" class="btn btn-warning debugInfoButton">Debug Info</button>
           <br/>
           <p/>
         </div>
@@ -304,6 +305,7 @@
     <script src="app/server_configuration.js" type="text/javascript"></script>
     <script src="app/server_stats.js" type="text/javascript"></script>
     <script src="app/server_version.js" type="text/javascript"></script>
+    <script src="app/server_debug_info.js" type="text/javascript"></script>
 
     <script src="app/router.js" type="text/javascript"></script>
   </body>