Browse Source

Merge pull request #10495 from gyuho/zap-logger

*: define default zap log configuration
Gyuho Lee 6 years ago
parent
commit
918f0414dd

+ 1 - 1
clientv3/balancer/balancer.go

@@ -36,7 +36,7 @@ func RegisterBuilder(cfg Config) {
 	bb := &builder{cfg}
 	balancer.Register(bb)
 
-	bb.cfg.Logger.Info(
+	bb.cfg.Logger.Debug(
 		"registered balancer",
 		zap.String("policy", bb.cfg.Policy.String()),
 		zap.String("name", bb.cfg.Name),

+ 4 - 3
clientv3/client.go

@@ -32,6 +32,7 @@ import (
 	"go.etcd.io/etcd/clientv3/balancer/picker"
 	"go.etcd.io/etcd/clientv3/balancer/resolver/endpoint"
 	"go.etcd.io/etcd/etcdserver/api/v3rpc/rpctypes"
+	"go.etcd.io/etcd/pkg/logutil"
 	"go.uber.org/zap"
 	"google.golang.org/grpc"
 	"google.golang.org/grpc/codes"
@@ -446,7 +447,7 @@ func newClient(cfg *Config) (*Client, error) {
 		callOpts: defaultCallOpts,
 	}
 
-	lcfg := DefaultLogConfig
+	lcfg := logutil.DefaultZapLoggerConfig
 	if cfg.LogConfig != nil {
 		lcfg = *cfg.LogConfig
 	}
@@ -530,10 +531,10 @@ func (c *Client) roundRobinQuorumBackoff(waitBetween time.Duration, jitterFracti
 		n := uint(len(c.Endpoints()))
 		quorum := (n/2 + 1)
 		if attempt%quorum == 0 {
-			c.lg.Info("backoff", zap.Uint("attempt", attempt), zap.Uint("quorum", quorum), zap.Duration("waitBetween", waitBetween), zap.Float64("jitterFraction", jitterFraction))
+			c.lg.Debug("backoff", zap.Uint("attempt", attempt), zap.Uint("quorum", quorum), zap.Duration("waitBetween", waitBetween), zap.Float64("jitterFraction", jitterFraction))
 			return jitterUp(waitBetween, jitterFraction)
 		}
-		c.lg.Info("backoff skipped", zap.Uint("attempt", attempt), zap.Uint("quorum", quorum))
+		c.lg.Debug("backoff skipped", zap.Uint("attempt", attempt), zap.Uint("quorum", quorum))
 		return 0
 	}
 }

+ 0 - 18
clientv3/config.go

@@ -82,21 +82,3 @@ type Config struct {
 	// PermitWithoutStream when set will allow client to send keepalive pings to server without any active streams(RPCs).
 	PermitWithoutStream bool `json:"permit-without-stream"`
 }
-
-// DefaultLogConfig is the default client logging configuration.
-// Default log level is "Warn". Use "zap.InfoLevel" for debugging.
-// Use "/dev/null" for output paths, to discard all logs.
-var DefaultLogConfig = zap.Config{
-	Level:       zap.NewAtomicLevelAt(zap.WarnLevel),
-	Development: false,
-	Sampling: &zap.SamplingConfig{
-		Initial:    100,
-		Thereafter: 100,
-	},
-	Encoding:      "json",
-	EncoderConfig: zap.NewProductionEncoderConfig(),
-
-	// Use "/dev/null" to discard all
-	OutputPaths:      []string{"stderr"},
-	ErrorOutputPaths: []string{"stderr"},
-}

+ 3 - 3
clientv3/retry_interceptor.go

@@ -48,7 +48,7 @@ func (c *Client) unaryClientInterceptor(logger *zap.Logger, optFuncs ...retryOpt
 			if err := waitRetryBackoff(ctx, attempt, callOpts); err != nil {
 				return err
 			}
-			logger.Info(
+			logger.Debug(
 				"retrying of unary invoker",
 				zap.String("target", cc.Target()),
 				zap.Uint("attempt", attempt),
@@ -112,7 +112,7 @@ func (c *Client) streamClientInterceptor(logger *zap.Logger, optFuncs ...retryOp
 			return nil, grpc.Errorf(codes.Unimplemented, "clientv3/retry_interceptor: cannot retry on ClientStreams, set Disable()")
 		}
 		newStreamer, err := streamer(ctx, desc, cc, method, grpcOpts...)
-		logger.Info("retry stream intercept", zap.Error(err))
+		logger.Warn("retry stream intercept", zap.Error(err))
 		if err != nil {
 			// TODO(mwitkow): Maybe dial and transport errors should be retriable?
 			return nil, err
@@ -228,7 +228,7 @@ func (s *serverStreamingRetryingStream) receiveMsgAndIndicateRetry(m interface{}
 	if s.callOpts.retryAuth && rpctypes.Error(err) == rpctypes.ErrInvalidAuthToken {
 		gterr := s.client.getToken(s.ctx)
 		if gterr != nil {
-			s.client.lg.Info("retry failed to fetch new auth token", zap.Error(gterr))
+			s.client.lg.Warn("retry failed to fetch new auth token", zap.Error(gterr))
 			return false, err // return the original error for simplicity
 		}
 		return true, err

+ 17 - 38
embed/config_logging.go

@@ -21,7 +21,6 @@ import (
 	"io/ioutil"
 	"os"
 	"reflect"
-	"sort"
 	"sync"
 
 	"go.etcd.io/etcd/pkg/logutil"
@@ -131,77 +130,55 @@ func (cfg *Config) setupLogging() error {
 			}
 		}
 
-		// TODO: use zapcore to support more features?
-		lcfg := zap.Config{
-			Level:       zap.NewAtomicLevelAt(zap.InfoLevel),
-			Development: false,
-			Sampling: &zap.SamplingConfig{
-				Initial:    100,
-				Thereafter: 100,
-			},
-			Encoding:      "json",
-			EncoderConfig: zap.NewProductionEncoderConfig(),
-
-			OutputPaths:      make([]string, 0),
-			ErrorOutputPaths: make([]string, 0),
-		}
-
-		outputPaths, errOutputPaths := make(map[string]struct{}), make(map[string]struct{})
+		outputPaths, errOutputPaths := make([]string, 0), make([]string, 0)
 		isJournal := false
 		for _, v := range cfg.LogOutputs {
 			switch v {
 			case DefaultLogOutput:
-				outputPaths[StdErrLogOutput] = struct{}{}
-				errOutputPaths[StdErrLogOutput] = struct{}{}
+				outputPaths = append(outputPaths, StdErrLogOutput)
+				errOutputPaths = append(errOutputPaths, StdErrLogOutput)
 
 			case JournalLogOutput:
 				isJournal = true
 
 			case StdErrLogOutput:
-				outputPaths[StdErrLogOutput] = struct{}{}
-				errOutputPaths[StdErrLogOutput] = struct{}{}
+				outputPaths = append(outputPaths, StdErrLogOutput)
+				errOutputPaths = append(errOutputPaths, StdErrLogOutput)
 
 			case StdOutLogOutput:
-				outputPaths[StdOutLogOutput] = struct{}{}
-				errOutputPaths[StdOutLogOutput] = struct{}{}
+				outputPaths = append(outputPaths, StdOutLogOutput)
+				errOutputPaths = append(errOutputPaths, StdOutLogOutput)
 
 			default:
-				outputPaths[v] = struct{}{}
-				errOutputPaths[v] = struct{}{}
+				outputPaths = append(outputPaths, v)
+				errOutputPaths = append(errOutputPaths, v)
 			}
 		}
 
 		if !isJournal {
-			for v := range outputPaths {
-				lcfg.OutputPaths = append(lcfg.OutputPaths, v)
-			}
-			for v := range errOutputPaths {
-				lcfg.ErrorOutputPaths = append(lcfg.ErrorOutputPaths, v)
-			}
-			sort.Strings(lcfg.OutputPaths)
-			sort.Strings(lcfg.ErrorOutputPaths)
+			copied := logutil.AddOutputPaths(logutil.DefaultZapLoggerConfig, outputPaths, errOutputPaths)
 
 			if cfg.Debug {
-				lcfg.Level = zap.NewAtomicLevelAt(zap.DebugLevel)
+				copied.Level = zap.NewAtomicLevelAt(zap.DebugLevel)
 				grpc.EnableTracing = true
 			}
 			if cfg.ZapLoggerBuilder == nil {
 				cfg.ZapLoggerBuilder = func(c *Config) error {
 					var err error
-					c.logger, err = lcfg.Build()
+					c.logger, err = copied.Build()
 					if err != nil {
 						return err
 					}
 					c.loggerMu.Lock()
 					defer c.loggerMu.Unlock()
-					c.loggerConfig = &lcfg
+					c.loggerConfig = &copied
 					c.loggerCore = nil
 					c.loggerWriteSyncer = nil
 					grpcLogOnce.Do(func() {
 						// debug true, enable info, warning, error
 						// debug false, only discard info
 						var gl grpclog.LoggerV2
-						gl, err = logutil.NewGRPCLoggerV2(lcfg)
+						gl, err = logutil.NewGRPCLoggerV2(copied)
 						if err == nil {
 							grpclog.SetLoggerV2(gl)
 						}
@@ -233,7 +210,7 @@ func (cfg *Config) setupLogging() error {
 			// WARN: do not change field names in encoder config
 			// journald logging writer assumes field names of "level" and "caller"
 			cr := zapcore.NewCore(
-				zapcore.NewJSONEncoder(zap.NewProductionEncoderConfig()),
+				zapcore.NewJSONEncoder(logutil.DefaultZapLoggerConfig.EncoderConfig),
 				syncer,
 				lvl,
 			)
@@ -253,10 +230,12 @@ func (cfg *Config) setupLogging() error {
 				}
 			}
 		}
+
 		err := cfg.ZapLoggerBuilder(cfg)
 		if err != nil {
 			return err
 		}
+
 		logTLSHandshakeFailure := func(conn *tls.Conn, err error) {
 			state := conn.ConnectionState()
 			remoteAddr := conn.RemoteAddr().String()

+ 1 - 13
etcdmain/grpc_proxy.go

@@ -151,19 +151,7 @@ func newGRPCProxyStartCommand() *cobra.Command {
 func startGRPCProxy(cmd *cobra.Command, args []string) {
 	checkArgs()
 
-	lcfg := zap.Config{
-		Level:       zap.NewAtomicLevelAt(zap.InfoLevel),
-		Development: false,
-		Sampling: &zap.SamplingConfig{
-			Initial:    100,
-			Thereafter: 100,
-		},
-		Encoding:      "json",
-		EncoderConfig: zap.NewProductionEncoderConfig(),
-
-		OutputPaths:      []string{"stderr"},
-		ErrorOutputPaths: []string{"stderr"},
-	}
+	lcfg := logutil.DefaultZapLoggerConfig
 	if grpcProxyDebug {
 		lcfg.Level = zap.NewAtomicLevelAt(zap.DebugLevel)
 		grpc.EnableTracing = true

+ 2 - 14
etcdserver/raft.go

@@ -58,20 +58,8 @@ var (
 )
 
 func init() {
-	lcfg := &zap.Config{
-		Level:       zap.NewAtomicLevelAt(zap.InfoLevel),
-		Development: false,
-		Sampling: &zap.SamplingConfig{
-			Initial:    100,
-			Thereafter: 100,
-		},
-		Encoding:      "json",
-		EncoderConfig: zap.NewProductionEncoderConfig(),
-
-		OutputPaths:      []string{"stderr"},
-		ErrorOutputPaths: []string{"stderr"},
-	}
-	lg, err := logutil.NewRaftLogger(lcfg)
+	lcfg := logutil.DefaultZapLoggerConfig
+	lg, err := logutil.NewRaftLogger(&lcfg)
 	if err != nil {
 		log.Fatalf("cannot create raft logger %v", err)
 	}

+ 5 - 13
integration/cluster.go

@@ -47,6 +47,7 @@ import (
 	lockpb "go.etcd.io/etcd/etcdserver/api/v3lock/v3lockpb"
 	"go.etcd.io/etcd/etcdserver/api/v3rpc"
 	pb "go.etcd.io/etcd/etcdserver/etcdserverpb"
+	"go.etcd.io/etcd/pkg/logutil"
 	"go.etcd.io/etcd/pkg/testutil"
 	"go.etcd.io/etcd/pkg/tlsutil"
 	"go.etcd.io/etcd/pkg/transport"
@@ -673,19 +674,10 @@ func mustNewMember(t testing.TB, mcfg memberConfig) *member {
 
 	m.InitialCorruptCheck = true
 
-	m.LoggerConfig = &zap.Config{
-		Level:       zap.NewAtomicLevelAt(zap.InfoLevel),
-		Development: false,
-		Sampling: &zap.SamplingConfig{
-			Initial:    100,
-			Thereafter: 100,
-		},
-		Encoding:      "json",
-		EncoderConfig: zap.NewProductionEncoderConfig(),
-
-		OutputPaths:      []string{"/dev/null"},
-		ErrorOutputPaths: []string{"/dev/null"},
-	}
+	lcfg := logutil.DefaultZapLoggerConfig
+	m.LoggerConfig = &lcfg
+	m.LoggerConfig.OutputPaths = []string{"/dev/null"}
+	m.LoggerConfig.ErrorOutputPaths = []string{"/dev/null"}
 	if os.Getenv("CLUSTER_DEBUG") != "" {
 		m.LoggerConfig.OutputPaths = []string{"stderr"}
 		m.LoggerConfig.ErrorOutputPaths = []string{"stderr"}

+ 97 - 0
pkg/logutil/zap.go

@@ -0,0 +1,97 @@
+// 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 logutil
+
+import (
+	"sort"
+
+	"go.uber.org/zap"
+	"go.uber.org/zap/zapcore"
+)
+
+// DefaultZapLoggerConfig defines default zap logger configuration.
+var DefaultZapLoggerConfig = zap.Config{
+	Level: zap.NewAtomicLevelAt(zap.InfoLevel),
+
+	Development: false,
+	Sampling: &zap.SamplingConfig{
+		Initial:    100,
+		Thereafter: 100,
+	},
+
+	Encoding: "json",
+
+	// copied from "zap.NewProductionEncoderConfig" with some updates
+	EncoderConfig: zapcore.EncoderConfig{
+		TimeKey:        "ts",
+		LevelKey:       "level",
+		NameKey:        "logger",
+		CallerKey:      "caller",
+		MessageKey:     "msg",
+		StacktraceKey:  "stacktrace",
+		LineEnding:     zapcore.DefaultLineEnding,
+		EncodeLevel:    zapcore.LowercaseLevelEncoder,
+		EncodeTime:     zapcore.ISO8601TimeEncoder,
+		EncodeDuration: zapcore.StringDurationEncoder,
+		EncodeCaller:   zapcore.ShortCallerEncoder,
+	},
+
+	// Use "/dev/null" to discard all
+	OutputPaths:      []string{"stderr"},
+	ErrorOutputPaths: []string{"stderr"},
+}
+
+// AddOutputPaths adds output paths to the existing output paths, resolving conflicts.
+func AddOutputPaths(cfg zap.Config, outputPaths, errorOutputPaths []string) zap.Config {
+	outputs := make(map[string]struct{})
+	for _, v := range cfg.OutputPaths {
+		outputs[v] = struct{}{}
+	}
+	for _, v := range outputPaths {
+		outputs[v] = struct{}{}
+	}
+	outputSlice := make([]string, 0)
+	if _, ok := outputs["/dev/null"]; ok {
+		// "/dev/null" to discard all
+		outputSlice = []string{"/dev/null"}
+	} else {
+		for k := range outputs {
+			outputSlice = append(outputSlice, k)
+		}
+	}
+	cfg.OutputPaths = outputSlice
+	sort.Strings(cfg.OutputPaths)
+
+	errOutputs := make(map[string]struct{})
+	for _, v := range cfg.ErrorOutputPaths {
+		errOutputs[v] = struct{}{}
+	}
+	for _, v := range errorOutputPaths {
+		errOutputs[v] = struct{}{}
+	}
+	errOutputSlice := make([]string, 0)
+	if _, ok := errOutputs["/dev/null"]; ok {
+		// "/dev/null" to discard all
+		errOutputSlice = []string{"/dev/null"}
+	} else {
+		for k := range errOutputs {
+			errOutputSlice = append(errOutputSlice, k)
+		}
+	}
+	cfg.ErrorOutputPaths = errOutputSlice
+	sort.Strings(cfg.ErrorOutputPaths)
+
+	return cfg
+}

+ 1 - 1
pkg/logutil/zap_grpc_test.go

@@ -40,7 +40,7 @@ func TestNewGRPCLoggerV2(t *testing.T) {
 			Thereafter: 100,
 		},
 		Encoding:         "json",
-		EncoderConfig:    zap.NewProductionEncoderConfig(),
+		EncoderConfig:    DefaultZapLoggerConfig.EncoderConfig,
 		OutputPaths:      []string{logPath},
 		ErrorOutputPaths: []string{logPath},
 	}

+ 1 - 1
pkg/logutil/zap_journal_test.go

@@ -34,7 +34,7 @@ func TestNewJournalWriter(t *testing.T) {
 	syncer := zapcore.AddSync(jw)
 
 	cr := zapcore.NewCore(
-		zapcore.NewJSONEncoder(zap.NewProductionEncoderConfig()),
+		zapcore.NewJSONEncoder(DefaultZapLoggerConfig.EncoderConfig),
 		syncer,
 		zap.NewAtomicLevelAt(zap.InfoLevel),
 	)

+ 1 - 1
pkg/logutil/zap_raft_test.go

@@ -40,7 +40,7 @@ func TestNewRaftLogger(t *testing.T) {
 			Thereafter: 100,
 		},
 		Encoding:         "json",
-		EncoderConfig:    zap.NewProductionEncoderConfig(),
+		EncoderConfig:    DefaultZapLoggerConfig.EncoderConfig,
 		OutputPaths:      []string{logPath},
 		ErrorOutputPaths: []string{logPath},
 	}