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/03 01:19:08 UTC

incubator-htrace git commit: HTRACE-286. htraced: improvements to logging, daemon startup, and configuration (cmccabe)

Repository: incubator-htrace
Updated Branches:
  refs/heads/master d852c448b -> 2e2757f83


HTRACE-286. htraced: improvements to logging, daemon startup, and configuration (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/2e2757f8
Tree: http://git-wip-us.apache.org/repos/asf/incubator-htrace/tree/2e2757f8
Diff: http://git-wip-us.apache.org/repos/asf/incubator-htrace/diff/2e2757f8

Branch: refs/heads/master
Commit: 2e2757f83cf539d265fa189ba726d7c92fcf3f04
Parents: d852c44
Author: Colin P. Mccabe <cm...@apache.org>
Authored: Mon Nov 2 16:15:45 2015 -0800
Committer: Colin P. Mccabe <cm...@apache.org>
Committed: Mon Nov 2 16:15:45 2015 -0800

----------------------------------------------------------------------
 htrace-htraced/example/htraced-conf.xml         |  4 +--
 .../go/src/org/apache/htrace/common/process.go  | 15 ----------
 .../go/src/org/apache/htrace/conf/config.go     | 28 ++++++++++++++++--
 .../go/src/org/apache/htrace/htrace/cmd.go      |  8 ++++-
 .../src/org/apache/htrace/htraced/datastore.go  |  4 +--
 .../go/src/org/apache/htrace/htraced/htraced.go | 31 ++++++++++++++++++--
 .../org/apache/htrace/htraced/mini_htraced.go   | 13 +++++++-
 .../go/src/org/apache/htrace/htraced/rest.go    | 21 ++++---------
 8 files changed, 81 insertions(+), 43 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-htrace/blob/2e2757f8/htrace-htraced/example/htraced-conf.xml
----------------------------------------------------------------------
diff --git a/htrace-htraced/example/htraced-conf.xml b/htrace-htraced/example/htraced-conf.xml
index ceb23a3..ba23e6b 100644
--- a/htrace-htraced/example/htraced-conf.xml
+++ b/htrace-htraced/example/htraced-conf.xml
@@ -18,11 +18,11 @@
 <?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
 <configuration>
   <property>
-    <name>log.level</name>
+    <name>htraced.log.level</name>
     <value>TRACE</value>
   </property>
   <property>
-    <name>log.path</name>
+    <name>htraced.log.path</name>
     <value>/tmp/htraced.log</value>
   </property>
   <property>

http://git-wip-us.apache.org/repos/asf/incubator-htrace/blob/2e2757f8/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 aad6ca1..2127ecf 100644
--- a/htrace-htraced/go/src/org/apache/htrace/common/process.go
+++ b/htrace-htraced/go/src/org/apache/htrace/common/process.go
@@ -20,7 +20,6 @@
 package common
 
 import (
-	"bufio"
 	"org/apache/htrace/conf"
 	"os"
 	"os/signal"
@@ -29,20 +28,6 @@ import (
 	"syscall"
 )
 
-func LoadApplicationConfig() *conf.Config {
-	cnf, dlog := conf.LoadApplicationConfig()
-	lg := NewLogger("conf", cnf)
-	defer lg.Close()
-	if lg.Level <= DEBUG {
-		// Print out the debug information from loading the configuration.
-		scanner := bufio.NewScanner(dlog)
-		for scanner.Scan() {
-			lg.Debugf(scanner.Text() + "\n")
-		}
-	}
-	return cnf
-}
-
 func InstallSignalHandlers(cnf *conf.Config) {
 	fatalSigs := []os.Signal{
 		os.Interrupt,

http://git-wip-us.apache.org/repos/asf/incubator-htrace/blob/2e2757f8/htrace-htraced/go/src/org/apache/htrace/conf/config.go
----------------------------------------------------------------------
diff --git a/htrace-htraced/go/src/org/apache/htrace/conf/config.go b/htrace-htraced/go/src/org/apache/htrace/conf/config.go
index d038723..0ce69a8 100644
--- a/htrace-htraced/go/src/org/apache/htrace/conf/config.go
+++ b/htrace-htraced/go/src/org/apache/htrace/conf/config.go
@@ -66,6 +66,10 @@ type Builder struct {
 
 	// If non-nil, the command-line arguments to use.
 	Argv []string
+
+	// The name of the application.  Configuration keys that start with this
+	// string will be converted to their unprefixed forms.
+	AppPrefix string
 }
 
 func getDefaultHTracedConfDir() string {
@@ -86,7 +90,7 @@ func getHTracedConfDirs(dlog io.Writer) []string {
 
 // Load a configuration from the application's argv, configuration file, and the standard
 // defaults.
-func LoadApplicationConfig() (*Config, io.Reader) {
+func LoadApplicationConfig(appPrefix string) (*Config, io.Reader) {
 	dlog := new(bytes.Buffer)
 	reader := openFile(CONFIG_FILE_NAME, getHTracedConfDirs(dlog), dlog)
 	bld := Builder{}
@@ -96,6 +100,7 @@ func LoadApplicationConfig() (*Config, io.Reader) {
 	}
 	bld.Argv = os.Args[1:]
 	bld.Defaults = DEFAULTS
+	bld.AppPrefix = appPrefix
 	cnf, err := bld.Build()
 	if err != nil {
 		log.Fatal("Error building configuration: " + err.Error())
@@ -106,9 +111,12 @@ func LoadApplicationConfig() (*Config, io.Reader) {
 		keys = append(keys, k)
 	}
 	sort.Sort(keys)
+	prefix := ""
+	io.WriteString(dlog, "Read configuration: ")
 	for i := range keys {
-		io.WriteString(dlog, fmt.Sprintf("%s = %s\n",
-			keys[i], cnf.settings[keys[i]]))
+		io.WriteString(dlog, fmt.Sprintf(`%s%s = "%s"`,
+			prefix, keys[i], cnf.settings[keys[i]]))
+		prefix = ", "
 	}
 	return cnf, dlog
 }
@@ -184,9 +192,23 @@ func (bld *Builder) Build() (*Config, error) {
 			i++
 		}
 	}
+	cnf.settings = bld.removeApplicationPrefixes(cnf.settings)
+	cnf.defaults = bld.removeApplicationPrefixes(cnf.defaults)
 	return &cnf, nil
 }
 
+func (bld *Builder) removeApplicationPrefixes(in map[string]string) map[string]string {
+	out := make(map[string]string)
+	for k, v := range(in) {
+		if strings.HasPrefix(k, bld.AppPrefix) {
+			out[k[len(bld.AppPrefix):]] = v
+		} else {
+			out[k] = v
+		}
+	}
+	return out
+}
+
 // Returns true if the configuration has a non-default value for the given key.
 func (cnf *Config) Contains(key string) bool {
 	_, ok := cnf.settings[key]

http://git-wip-us.apache.org/repos/asf/incubator-htrace/blob/2e2757f8/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 e7286ff..f3668ea 100644
--- a/htrace-htraced/go/src/org/apache/htrace/htrace/cmd.go
+++ b/htrace-htraced/go/src/org/apache/htrace/htrace/cmd.go
@@ -55,7 +55,13 @@ the defaults will be used.
 
 func main() {
 	// Load htraced configuration
-	cnf := common.LoadApplicationConfig()
+	cnf, cnfLog := conf.LoadApplicationConfig("htrace.tool.")
+	lg := common.NewLogger("conf", cnf)
+	defer lg.Close()
+	scanner := bufio.NewScanner(cnfLog)
+	for scanner.Scan() {
+		lg.Debugf(scanner.Text() + "\n")
+	}
 
 	// Parse argv
 	app := kingpin.New(os.Args[0], USAGE)

http://git-wip-us.apache.org/repos/asf/incubator-htrace/blob/2e2757f8/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 5d5559a..749b5ab 100644
--- a/htrace-htraced/go/src/org/apache/htrace/htraced/datastore.go
+++ b/htrace-htraced/go/src/org/apache/htrace/htraced/datastore.go
@@ -1234,8 +1234,8 @@ func (store *dataStore) ServerStats() *common.ServerStats {
 		serverStats.Dirs[shardIdx].ApproxNumSpans = vals[0]
 		serverStats.Dirs[shardIdx].LevelDbStats =
 			shard.ldb.PropertyValue("leveldb.stats")
-		store.lg.Infof("shard.ldb.PropertyValue(leveldb.stats)=%s\n",
-			shard.ldb.PropertyValue("leveldb.stats"))
+		store.lg.Infof("levedb.stats for %s: %s\n",
+			shard.path, shard.ldb.PropertyValue("leveldb.stats"))
 	}
 	serverStats.HostSpanMetrics = store.msink.AccessTotals()
 	serverStats.LastStartMs = store.startMs

http://git-wip-us.apache.org/repos/asf/incubator-htrace/blob/2e2757f8/htrace-htraced/go/src/org/apache/htrace/htraced/htraced.go
----------------------------------------------------------------------
diff --git a/htrace-htraced/go/src/org/apache/htrace/htraced/htraced.go b/htrace-htraced/go/src/org/apache/htrace/htraced/htraced.go
index 64da457..b482aa3 100644
--- a/htrace-htraced/go/src/org/apache/htrace/htraced/htraced.go
+++ b/htrace-htraced/go/src/org/apache/htrace/htraced/htraced.go
@@ -20,6 +20,7 @@
 package main
 
 import (
+	"bufio"
 	"encoding/json"
 	"fmt"
 	"net"
@@ -63,17 +64,41 @@ func main() {
 			os.Exit(0)
 		}
 	}
-	cnf := common.LoadApplicationConfig()
-	common.InstallSignalHandlers(cnf)
+
+	// Load the htraced configuration.
+	cnf, cnfLog := conf.LoadApplicationConfig("htraced.")
+
+	// Open the HTTP port.
+	// We want to do this first, before initializing the datastore or setting up
+	// logging.  That way, if someone accidentally starts two daemons with the
+	// same config file, the second invocation will exit with a "port in use"
+	// error rather than potentially disrupting the first invocation.
+	rstListener, listenErr := net.Listen("tcp", cnf.Get(conf.HTRACE_WEB_ADDRESS))
+	if listenErr != nil {
+		fmt.Fprintf(os.Stderr, "Error opening HTTP port: %s\n",
+			listenErr.Error())
+		os.Exit(1)
+	}
+
+	// Print out the startup banner and information about the daemon
+	// configuration.
 	lg := common.NewLogger("main", cnf)
 	defer lg.Close()
+	lg.Infof("*** Starting htraced ***\n")
+	scanner := bufio.NewScanner(cnfLog)
+	for scanner.Scan() {
+		lg.Infof(scanner.Text() + "\n")
+	}
+	common.InstallSignalHandlers(cnf)
+
+	// Initialize the datastore.
 	store, err := CreateDataStore(cnf, nil)
 	if err != nil {
 		lg.Errorf("Error creating datastore: %s\n", err.Error())
 		os.Exit(1)
 	}
 	var rsv *RestServer
-	rsv, err = CreateRestServer(cnf, store)
+	rsv, err = CreateRestServer(cnf, store, rstListener)
 	if err != nil {
 		lg.Errorf("Error creating REST server: %s\n", err.Error())
 		os.Exit(1)

http://git-wip-us.apache.org/repos/asf/incubator-htrace/blob/2e2757f8/htrace-htraced/go/src/org/apache/htrace/htraced/mini_htraced.go
----------------------------------------------------------------------
diff --git a/htrace-htraced/go/src/org/apache/htrace/htraced/mini_htraced.go b/htrace-htraced/go/src/org/apache/htrace/htraced/mini_htraced.go
index c2300c4..80df676 100644
--- a/htrace-htraced/go/src/org/apache/htrace/htraced/mini_htraced.go
+++ b/htrace-htraced/go/src/org/apache/htrace/htraced/mini_htraced.go
@@ -22,6 +22,7 @@ package main
 import (
 	"fmt"
 	"io/ioutil"
+	"net"
 	"org/apache/htrace/common"
 	"org/apache/htrace/conf"
 	"os"
@@ -126,10 +127,20 @@ func (bld *MiniHTracedBuilder) Build() (*MiniHTraced, error) {
 	if err != nil {
 		return nil, err
 	}
-	rsv, err = CreateRestServer(cnf, store)
+	rstListener, listenErr := net.Listen("tcp", cnf.Get(conf.HTRACE_WEB_ADDRESS))
+	if listenErr != nil {
+		return nil, listenErr
+	}
+	defer func() {
+		if rstListener != nil {
+			rstListener.Close()
+		}
+	}()
+	rsv, err = CreateRestServer(cnf, store, rstListener)
 	if err != nil {
 		return nil, err
 	}
+	rstListener = nil
 	hsv, err = CreateHrpcServer(cnf, store)
 	if err != nil {
 		return nil, err

http://git-wip-us.apache.org/repos/asf/incubator-htrace/blob/2e2757f8/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 eca3f08..2a10805 100644
--- a/htrace-htraced/go/src/org/apache/htrace/htraced/rest.go
+++ b/htrace-htraced/go/src/org/apache/htrace/htraced/rest.go
@@ -295,19 +295,10 @@ type RestServer struct {
 	lg       *common.Logger
 }
 
-func CreateRestServer(cnf *conf.Config, store *dataStore) (*RestServer, error) {
+func CreateRestServer(cnf *conf.Config, store *dataStore,
+		listener net.Listener) (*RestServer, error) {
 	var err error
 	rsv := &RestServer{}
-	rsv.listener, err = net.Listen("tcp", cnf.Get(conf.HTRACE_WEB_ADDRESS))
-	if err != nil {
-		return nil, err
-	}
-	var success bool
-	defer func() {
-		if !success {
-			rsv.Close()
-		}
-	}()
 	rsv.lg = common.NewLogger("rest", cnf)
 
 	r := mux.NewRouter().StrictSlash(false)
@@ -340,22 +331,20 @@ func CreateRestServer(cnf *conf.Config, store *dataStore) (*RestServer, error) {
 	webdir := os.Getenv("HTRACED_WEB_DIR")
 	if webdir == "" {
 		webdir, err = filepath.Abs(filepath.Join(filepath.Dir(os.Args[0]), "..", "web"))
-
 		if err != nil {
 			return nil, err
 		}
 	}
 
-	rsv.lg.Infof("Serving static files from %s\n.", webdir)
+	rsv.lg.Infof(`Serving static files from "%s"\n`, webdir)
 	r.PathPrefix("/").Handler(http.FileServer(http.Dir(webdir))).Methods("GET")
 
 	// Log an error message for unknown non-GET requests.
 	r.PathPrefix("/").Handler(&logErrorHandler{lg: rsv.lg})
 
+	rsv.listener = listener
 	go http.Serve(rsv.listener, r)
-
-	rsv.lg.Infof("Started REST server on %s...\n", rsv.listener.Addr().String())
-	success = true
+	rsv.lg.Infof("Started REST server on %s\n", rsv.listener.Addr().String())
 	return rsv, nil
 }