Browse Source

pkg: use zap logger to format the structure log output.

yoyinzyc 6 years ago
parent
commit
1d6ef8370e
5 changed files with 110 additions and 171 deletions
  1. 2 2
      etcdserver/apply.go
  2. 11 11
      etcdserver/v3_server.go
  3. 2 2
      mvcc/kvstore_txn.go
  4. 39 37
      pkg/traceutil/trace.go
  5. 56 119
      pkg/traceutil/trace_test.go

+ 2 - 2
etcdserver/apply.go

@@ -331,7 +331,7 @@ func (a *applierV3backend) Range(ctx context.Context, txn mvcc.TxnRead, r *pb.Ra
 		rr.KVs = rr.KVs[:r.Limit]
 		resp.More = true
 	}
-	trace.Step("Filter and sort the key-value pairs.")
+	trace.Step("filter and sort the key-value pairs")
 	resp.Header.Revision = rr.Rev
 	resp.Count = int64(rr.Count)
 	resp.Kvs = make([]*mvccpb.KeyValue, len(rr.KVs))
@@ -341,7 +341,7 @@ func (a *applierV3backend) Range(ctx context.Context, txn mvcc.TxnRead, r *pb.Ra
 		}
 		resp.Kvs[i] = &rr.KVs[i]
 	}
-	trace.Step("Assemble the response.")
+	trace.Step("assemble the response")
 	return resp, nil
 }
 

+ 11 - 11
etcdserver/v3_server.go

@@ -86,11 +86,12 @@ type Authenticator interface {
 }
 
 func (s *EtcdServer) Range(ctx context.Context, r *pb.RangeRequest) (*pb.RangeResponse, error) {
-	trace := traceutil.New("Range",
-		traceutil.Field{Key: "RangeBegin", Value: string(r.Key)},
-		traceutil.Field{Key: "RangeEnd", Value: string(r.RangeEnd)},
+	trace := traceutil.New("range",
+		s.getLogger(),
+		traceutil.Field{Key: "range_begin", Value: string(r.Key)},
+		traceutil.Field{Key: "range_end", Value: string(r.RangeEnd)},
 	)
-	ctx = context.WithValue(ctx, "trace", trace)
+	ctx = context.WithValue(ctx, traceutil.CtxKey, trace)
 
 	var resp *pb.RangeResponse
 	var err error
@@ -98,16 +99,16 @@ func (s *EtcdServer) Range(ctx context.Context, r *pb.RangeRequest) (*pb.RangeRe
 		warnOfExpensiveReadOnlyRangeRequest(s.getLogger(), start, r, resp, err)
 		if resp != nil {
 			trace.AddField(
-				traceutil.Field{Key: "ResponseCount", Value: len(resp.Kvs)},
-				traceutil.Field{Key: "ResponseRevision", Value: resp.Header.Revision},
+				traceutil.Field{Key: "response_count", Value: len(resp.Kvs)},
+				traceutil.Field{Key: "response_revision", Value: resp.Header.Revision},
 			)
 		}
-		trace.LogIfLong(rangeTraceThreshold, s.getLogger())
+		trace.LogIfLong(rangeTraceThreshold)
 	}(time.Now())
 
 	if !r.Serializable {
 		err = s.linearizableReadNotify(ctx)
-		trace.Step("Agreement among raft nodes before linearized reading.")
+		trace.Step("agreement among raft nodes before linearized reading")
 		if err != nil {
 			return nil, err
 		}
@@ -562,6 +563,7 @@ func (s *EtcdServer) raftRequest(ctx context.Context, r pb.InternalRaftRequest)
 
 // doSerialize handles the auth logic, with permissions checked by "chk", for a serialized request "get". Returns a non-nil error on authentication failure.
 func (s *EtcdServer) doSerialize(ctx context.Context, chk func(*auth.AuthInfo) error, get func()) error {
+	trace := traceutil.Get(ctx)
 	ai, err := s.AuthInfoFromCtx(ctx)
 	if err != nil {
 		return err
@@ -573,9 +575,7 @@ func (s *EtcdServer) doSerialize(ctx context.Context, chk func(*auth.AuthInfo) e
 	if err = chk(ai); err != nil {
 		return err
 	}
-
-	trace := traceutil.Get(ctx)
-	trace.Step("Authentication.")
+	trace.Step("get authentication metadata")
 	// fetch response for serialized request
 	get()
 	// check for stale token revision in case the auth store was updated while

+ 2 - 2
mvcc/kvstore_txn.go

@@ -127,7 +127,7 @@ func (tr *storeTxnRead) rangeKeys(key, end []byte, curRev int64, ro RangeOptions
 	}
 
 	revpairs := tr.s.kvindex.Revisions(key, end, rev)
-	tr.trace.Step("Range keys from in-memory index tree.")
+	tr.trace.Step("range keys from in-memory index tree")
 	if len(revpairs) == 0 {
 		return &RangeResult{KVs: nil, Count: 0, Rev: curRev}, nil
 	}
@@ -167,7 +167,7 @@ func (tr *storeTxnRead) rangeKeys(key, end []byte, curRev int64, ro RangeOptions
 			}
 		}
 	}
-	tr.trace.Step("Range keys from bolt db.")
+	tr.trace.Step("range keys from bolt db")
 	return &RangeResult{KVs: kvs, Count: len(revpairs), Rev: curRev}, nil
 }
 

+ 39 - 37
pkg/traceutil/trace.go

@@ -1,3 +1,18 @@
+// Copyright 2019 The etcd Authors
+//
+// Licensed 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.
+
+// Package traceutil implements tracing utilities using "context".
 package traceutil
 
 import (
@@ -7,13 +22,10 @@ import (
 	"math/rand"
 	"time"
 
-	"github.com/coreos/pkg/capnslog"
 	"go.uber.org/zap"
 )
 
-var (
-	plog = capnslog.NewPackageLogger("go.etcd.io/etcd", "trace")
-)
+const CtxKey = "trace"
 
 // Field is a kv pair to record additional details of the trace.
 type Field struct {
@@ -40,6 +52,7 @@ func writeFields(fields []Field) string {
 
 type Trace struct {
 	operation string
+	lg        *zap.Logger
 	fields    []Field
 	startTime time.Time
 	steps     []step
@@ -51,32 +64,22 @@ type step struct {
 	fields []Field
 }
 
-func New(op string, fields ...Field) *Trace {
-	return &Trace{operation: op, startTime: time.Now(), fields: fields}
+func New(op string, lg *zap.Logger, fields ...Field) *Trace {
+	return &Trace{operation: op, lg: lg, startTime: time.Now(), fields: fields}
 }
 
-// traceutil.TODO() returns a non-nil, empty Trace
+// TODO returns a non-nil, empty Trace
 func TODO() *Trace {
 	return &Trace{}
 }
 
 func Get(ctx context.Context) *Trace {
-	if trace, ok := ctx.Value("trace").(*Trace); ok && trace != nil {
+	if trace, ok := ctx.Value(CtxKey).(*Trace); ok && trace != nil {
 		return trace
 	}
 	return TODO()
 }
 
-func GetOrCreate(ctx context.Context, op string, fields ...Field) (context.Context, *Trace) {
-	trace, ok := ctx.Value("trace").(*Trace)
-	if !ok || trace == nil {
-		trace = New(op)
-		trace.fields = fields
-		ctx = context.WithValue(ctx, "trace", trace)
-	}
-	return ctx, trace
-}
-
 func (t *Trace) Step(msg string, fields ...Field) {
 	t.steps = append(t.steps, step{time: time.Now(), msg: msg, fields: fields})
 }
@@ -88,48 +91,47 @@ func (t *Trace) AddField(fields ...Field) {
 }
 
 // Log dumps all steps in the Trace
-func (t *Trace) Log(lg *zap.Logger) {
-	t.LogWithStepThreshold(0, lg)
+func (t *Trace) Log() {
+	t.LogWithStepThreshold(0)
 }
 
 // LogIfLong dumps logs if the duration is longer than threshold
-func (t *Trace) LogIfLong(threshold time.Duration, lg *zap.Logger) {
+func (t *Trace) LogIfLong(threshold time.Duration) {
 	if time.Since(t.startTime) > threshold {
 		stepThreshold := threshold / time.Duration(len(t.steps)+1)
-		t.LogWithStepThreshold(stepThreshold, lg)
+		t.LogWithStepThreshold(stepThreshold)
 	}
 }
 
 // LogWithStepThreshold only dumps step whose duration is longer than step threshold
-func (t *Trace) LogWithStepThreshold(threshold time.Duration, lg *zap.Logger) {
-	s := t.format(threshold)
-	if lg != nil {
-		lg.Info(s)
-	} else {
-		plog.Info(s)
+func (t *Trace) LogWithStepThreshold(threshold time.Duration) {
+	msg, fs := t.logInfo(threshold)
+	if t.lg != nil {
+		t.lg.Info(msg, fs...)
 	}
 }
 
-func (t *Trace) format(threshold time.Duration) string {
+func (t *Trace) logInfo(threshold time.Duration) (string, []zap.Field) {
 	endTime := time.Now()
 	totalDuration := endTime.Sub(t.startTime)
-	var buf bytes.Buffer
 	traceNum := rand.Int31()
+	msg := fmt.Sprintf("trace[%d] %s", traceNum, t.operation)
 
-	buf.WriteString(fmt.Sprintf("Trace[%d] \"%v\" %s (duration: %v, start: %v)\n",
-		traceNum, t.operation, writeFields(t.fields), totalDuration,
-		t.startTime.Format("2006-01-02 15:04:05.000")))
+	var steps []string
 	lastStepTime := t.startTime
 	for _, step := range t.steps {
 		stepDuration := step.time.Sub(lastStepTime)
 		if stepDuration > threshold {
-			buf.WriteString(fmt.Sprintf("Trace[%d] Step \"%v\" %s (duration: %v)\n",
+			steps = append(steps, fmt.Sprintf("trace[%d] step '%v' %s (duration: %v)",
 				traceNum, step.msg, writeFields(step.fields), stepDuration))
 		}
 		lastStepTime = step.time
 	}
-	buf.WriteString(fmt.Sprintf("Trace[%d] End %v\n", traceNum,
-		endTime.Format("2006-01-02 15:04:05.000")))
 
-	return buf.String()
+	fs := []zap.Field{zap.String("detail", writeFields(t.fields)),
+		zap.Duration("duration", totalDuration),
+		zap.Time("start", t.startTime),
+		zap.Time("end", endTime),
+		zap.Strings("steps", steps)}
+	return msg, fs
 }

+ 56 - 119
pkg/traceutil/trace_test.go

@@ -1,3 +1,17 @@
+// Copyright 2019 The etcd Authors
+//
+// Licensed 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.
+
 package traceutil
 
 import (
@@ -7,8 +21,6 @@ import (
 	"io/ioutil"
 	"os"
 	"path/filepath"
-	"regexp"
-	"strings"
 	"testing"
 	"time"
 
@@ -29,7 +41,7 @@ func TestGet(t *testing.T) {
 		},
 		{
 			name:        "When the context has trace",
-			inputCtx:    context.WithValue(context.Background(), "trace", traceForTest),
+			inputCtx:    context.WithValue(context.Background(), CtxKey, traceForTest),
 			outputTrace: traceForTest,
 		},
 	}
@@ -38,43 +50,10 @@ func TestGet(t *testing.T) {
 		t.Run(tt.name, func(t *testing.T) {
 			trace := Get(tt.inputCtx)
 			if trace == nil {
-				t.Errorf("Expected %v; Got nil\n", tt.outputTrace)
+				t.Errorf("Expected %v; Got nil", tt.outputTrace)
 			}
 			if trace.operation != tt.outputTrace.operation {
-				t.Errorf("Expected %v; Got %v\n", tt.outputTrace, trace)
-			}
-		})
-	}
-}
-
-func TestGetOrCreate(t *testing.T) {
-	tests := []struct {
-		name          string
-		inputCtx      context.Context
-		outputTraceOp string
-	}{
-		{
-			name:          "When the context does not have trace",
-			inputCtx:      context.TODO(),
-			outputTraceOp: "test",
-		},
-		{
-			name:          "When the context has trace",
-			inputCtx:      context.WithValue(context.Background(), "trace", &Trace{operation: "test"}),
-			outputTraceOp: "test",
-		},
-	}
-
-	for _, tt := range tests {
-		t.Run(tt.name, func(t *testing.T) {
-			ctx, trace := GetOrCreate(tt.inputCtx, "test")
-			if trace == nil {
-				t.Errorf("Expected trace object; Got nil\n")
-			} else if trace.operation != tt.outputTraceOp {
-				t.Errorf("Expected %v; Got %v\n", tt.outputTraceOp, trace.operation)
-			}
-			if ctx.Value("trace") == nil {
-				t.Errorf("Expected context has attached trace; Got nil\n")
+				t.Errorf("Expected %v; Got %v", tt.outputTrace, trace)
 			}
 		})
 	}
@@ -94,16 +73,16 @@ func TestCreate(t *testing.T) {
 		}
 	)
 
-	trace := New(op, fields[0], fields[1])
+	trace := New(op, nil, fields[0], fields[1])
 	if trace.operation != op {
-		t.Errorf("Expected %v; Got %v\n", op, trace.operation)
+		t.Errorf("Expected %v; Got %v", op, trace.operation)
 	}
 	for i, f := range trace.fields {
 		if f.Key != fields[i].Key {
-			t.Errorf("Expected %v; Got %v\n", fields[i].Key, f.Key)
+			t.Errorf("Expected %v; Got %v", fields[i].Key, f.Key)
 		}
 		if f.Value != fields[i].Value {
-			t.Errorf("Expected %v; Got %v\n", fields[i].Value, f.Value)
+			t.Errorf("Expected %v; Got %v", fields[i].Value, f.Value)
 		}
 	}
 
@@ -113,67 +92,38 @@ func TestCreate(t *testing.T) {
 
 	for i, v := range trace.steps {
 		if steps[i] != v.msg {
-			t.Errorf("Expected %v, got %v\n.", steps[i], v.msg)
+			t.Errorf("Expected %v; Got %v", steps[i], v.msg)
 		}
 		if stepFields[i].Key != v.fields[0].Key {
-			t.Errorf("Expected %v; Got %v\n", stepFields[i].Key, v.fields[0].Key)
+			t.Errorf("Expected %v; Got %v", stepFields[i].Key, v.fields[0].Key)
 		}
 		if stepFields[i].Value != v.fields[0].Value {
-			t.Errorf("Expected %v; Got %v\n", stepFields[i].Value, v.fields[0].Value)
+			t.Errorf("Expected %v; Got %v", stepFields[i].Value, v.fields[0].Value)
 		}
 	}
 }
 
 func TestLog(t *testing.T) {
-	test := struct {
-		name        string
-		trace       *Trace
-		expectedMsg []string
-	}{
-		name: "When dump all logs",
-		trace: &Trace{
-			operation: "Test",
-			startTime: time.Now().Add(-100 * time.Millisecond),
-			steps: []step{
-				{time: time.Now().Add(-80 * time.Millisecond), msg: "msg1"},
-				{time: time.Now().Add(-50 * time.Millisecond), msg: "msg2"},
-			},
-		},
-		expectedMsg: []string{
-			"msg1", "msg2",
-		},
-	}
-
-	t.Run(test.name, func(t *testing.T) {
-		logPath := filepath.Join(os.TempDir(), fmt.Sprintf("test-log-%d", time.Now().UnixNano()))
-		defer os.RemoveAll(logPath)
-
-		lcfg := zap.NewProductionConfig()
-		lcfg.OutputPaths = []string{logPath}
-		lcfg.ErrorOutputPaths = []string{logPath}
-		lg, _ := lcfg.Build()
-
-		test.trace.Log(lg)
-		data, err := ioutil.ReadFile(logPath)
-		if err != nil {
-			t.Fatal(err)
-		}
-
-		for _, msg := range test.expectedMsg {
-			if !bytes.Contains(data, []byte(msg)) {
-				t.Errorf("Expected to find %v in log.\n", msg)
-			}
-		}
-	})
-}
-
-func TestTraceFormat(t *testing.T) {
 	tests := []struct {
 		name        string
 		trace       *Trace
 		fields      []Field
 		expectedMsg []string
 	}{
+		{
+			name: "When dump all logs",
+			trace: &Trace{
+				operation: "Test",
+				startTime: time.Now().Add(-100 * time.Millisecond),
+				steps: []step{
+					{time: time.Now().Add(-80 * time.Millisecond), msg: "msg1"},
+					{time: time.Now().Add(-50 * time.Millisecond), msg: "msg2"},
+				},
+			},
+			expectedMsg: []string{
+				"msg1", "msg2",
+			},
+		},
 		{
 			name: "When trace has fields",
 			trace: &Trace{
@@ -203,45 +153,31 @@ func TestTraceFormat(t *testing.T) {
 				"stepKey1:stepValue1", "stepKey2:stepValue2",
 			},
 		},
-		{
-			name: "When trace has no field",
-			trace: &Trace{
-				operation: "Test",
-				startTime: time.Now().Add(-100 * time.Millisecond),
-				steps: []step{
-					{time: time.Now().Add(-80 * time.Millisecond), msg: "msg1"},
-					{time: time.Now().Add(-50 * time.Millisecond), msg: "msg2"},
-				},
-			},
-			fields: []Field{},
-			expectedMsg: []string{
-				"Test",
-				"msg1", "msg2",
-			},
-		},
 	}
 
 	for _, tt := range tests {
 		t.Run(tt.name, func(t *testing.T) {
+			logPath := filepath.Join(os.TempDir(), fmt.Sprintf("test-log-%d", time.Now().UnixNano()))
+			defer os.RemoveAll(logPath)
+
+			lcfg := zap.NewProductionConfig()
+			lcfg.OutputPaths = []string{logPath}
+			lcfg.ErrorOutputPaths = []string{logPath}
+			lg, _ := lcfg.Build()
+
 			for _, f := range tt.fields {
 				tt.trace.AddField(f)
 			}
-			s := tt.trace.format(0)
-			var buf bytes.Buffer
-			buf.WriteString(`Trace\[(\d*)?\](.+)\(duration(.+)start(.+)\)\n`)
-			for range tt.trace.steps {
-				buf.WriteString(`Trace\[(\d*)?\](.+)Step(.+)\(duration(.+)\)\n`)
+			tt.trace.lg = lg
+			tt.trace.Log()
+			data, err := ioutil.ReadFile(logPath)
+			if err != nil {
+				t.Fatal(err)
 			}
-			buf.WriteString(`Trace\[(\d*)?\](.+)End(.+)\n`)
-			pattern := buf.String()
 
-			r, _ := regexp.Compile(pattern)
-			if !r.MatchString(s) {
-				t.Errorf("Wrong log format.\n")
-			}
 			for _, msg := range tt.expectedMsg {
-				if !strings.Contains(s, msg) {
-					t.Errorf("Expected to find %v in log.\n", msg)
+				if !bytes.Contains(data, []byte(msg)) {
+					t.Errorf("Expected to find %v in log", msg)
 				}
 			}
 		})
@@ -310,14 +246,15 @@ func TestLogIfLong(t *testing.T) {
 			lcfg.ErrorOutputPaths = []string{logPath}
 			lg, _ := lcfg.Build()
 
-			tt.trace.LogIfLong(tt.threshold, lg)
+			tt.trace.lg = lg
+			tt.trace.LogIfLong(tt.threshold)
 			data, err := ioutil.ReadFile(logPath)
 			if err != nil {
 				t.Fatal(err)
 			}
 			for _, msg := range tt.expectedMsg {
 				if !bytes.Contains(data, []byte(msg)) {
-					t.Errorf("Expected to find %v in log\n", msg)
+					t.Errorf("Expected to find %v in log", msg)
 				}
 			}
 		})