mirror of
https://github.com/etcd-io/etcd.git
synced 2024-09-27 06:25:44 +00:00
Merge pull request #10495 from gyuho/zap-logger
*: define default zap log configuration
This commit is contained in:
commit
918f0414dd
@ -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),
|
||||
|
@ -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
|
||||
}
|
||||
}
|
||||
|
@ -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"},
|
||||
}
|
||||
|
@ -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
|
||||
|
@ -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()
|
||||
|
@ -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
|
||||
|
@ -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)
|
||||
}
|
||||
|
@ -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
pkg/logutil/zap.go
Normal file
97
pkg/logutil/zap.go
Normal file
@ -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
|
||||
}
|
@ -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},
|
||||
}
|
||||
|
@ -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),
|
||||
)
|
||||
|
@ -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},
|
||||
}
|
||||
|
Loading…
x
Reference in New Issue
Block a user