Browse Source

Merge pull request #2898 from xiang90/raft_log

raft use leveled logger
Xiang Li 10 years ago
parent
commit
2bf64b4adf

+ 2 - 2
Godeps/Godeps.json

@@ -1,6 +1,6 @@
 {
 	"ImportPath": "github.com/coreos/etcd",
-	"GoVersion": "go1.4.2",
+	"GoVersion": "go1.4.1",
 	"Packages": [
 		"./..."
 	],
@@ -35,7 +35,7 @@
 		},
 		{
 			"ImportPath": "github.com/coreos/pkg/capnslog",
-			"Rev": "9d5dd4632f9ece71bdf83d31253593a633e73df5"
+			"Rev": "84b359ff90d62d7b5a5b9dfb96400c08f0cc6642"
 		},
 		{
 			"ImportPath": "github.com/gogo/protobuf/proto",

+ 12 - 5
Godeps/_workspace/src/github.com/coreos/pkg/capnslog/example/hello_dolly.go

@@ -1,27 +1,34 @@
 package main
 
 import (
+	"flag"
 	oldlog "log"
 	"os"
 
-	"github.com/coreos/etcd/Godeps/_workspace/src/github.com/coreos/pkg/capnslog"
+	"github.com/coreos/pkg/capnslog"
 )
 
+var logLevel = capnslog.INFO
 var log = capnslog.NewPackageLogger("github.com/coreos/pkg/capnslog/cmd", "main")
 var dlog = capnslog.NewPackageLogger("github.com/coreos/pkg/capnslog/cmd", "dolly")
 
+func init() {
+	flag.Var(&logLevel, "log-level", "Global log level.")
+}
+
 func main() {
 	rl := capnslog.MustRepoLogger("github.com/coreos/pkg/capnslog/cmd")
 	capnslog.SetFormatter(capnslog.NewStringFormatter(os.Stderr))
 
 	// We can parse the log level configs from the command line
-	if len(os.Args) > 1 {
-		cfg, err := rl.ParseLogLevelConfig(os.Args[1])
+	flag.Parse()
+	if flag.NArg() > 1 {
+		cfg, err := rl.ParseLogLevelConfig(flag.Arg(1))
 		if err != nil {
 			log.Fatal(err)
 		}
 		rl.SetLogLevel(cfg)
-		log.Infof("Setting output to %s", os.Args[1])
+		log.Infof("Setting output to %s", flag.Arg(1))
 	}
 
 	// Send some messages at different levels to the different packages
@@ -32,7 +39,7 @@ func main() {
 	dlog.Tracef("I can tell, Dolly")
 
 	// We also have control over the built-in "log" package.
-	capnslog.SetGlobalLogLevel(capnslog.INFO)
+	capnslog.SetGlobalLogLevel(logLevel)
 	oldlog.Println("You're still glowin', you're still crowin', you're still lookin' strong")
 	log.Fatalf("Dolly'll never go away again")
 }

+ 6 - 12
Godeps/_workspace/src/github.com/coreos/pkg/capnslog/formatters.go

@@ -9,7 +9,7 @@ import (
 )
 
 type Formatter interface {
-	Format(pkg string, level LogLevel, depth int, entries ...LogEntry)
+	Format(pkg string, level LogLevel, depth int, entries ...interface{})
 	Flush()
 }
 
@@ -23,7 +23,7 @@ type StringFormatter struct {
 	w *bufio.Writer
 }
 
-func (s *StringFormatter) Format(pkg string, l LogLevel, i int, entries ...LogEntry) {
+func (s *StringFormatter) Format(pkg string, l LogLevel, i int, entries ...interface{}) {
 	now := time.Now()
 	y, m, d := now.Date()
 	h, min, sec := now.Clock()
@@ -31,19 +31,13 @@ func (s *StringFormatter) Format(pkg string, l LogLevel, i int, entries ...LogEn
 	s.writeEntries(pkg, l, i, entries...)
 }
 
-func (s *StringFormatter) writeEntries(pkg string, _ LogLevel, _ int, entries ...LogEntry) {
+func (s *StringFormatter) writeEntries(pkg string, _ LogLevel, _ int, entries ...interface{}) {
 	if pkg != "" {
 		s.w.WriteString(pkg + ": ")
 	}
-	endsInNL := false
-	for i, v := range entries {
-		if i != 0 {
-			s.w.WriteByte(' ')
-		}
-		str := v.LogString()
-		endsInNL = strings.HasSuffix(str, "\n")
-		s.w.WriteString(str)
-	}
+	str := fmt.Sprint(entries...)
+	endsInNL := strings.HasSuffix(str, "\n")
+	s.w.WriteString(str)
 	if !endsInNL {
 		s.w.WriteString("\n")
 	}

+ 1 - 1
Godeps/_workspace/src/github.com/coreos/pkg/capnslog/glog_formatter.go

@@ -23,7 +23,7 @@ func NewGlogFormatter(w io.Writer) *GlogFormatter {
 	return g
 }
 
-func (g GlogFormatter) Format(pkg string, level LogLevel, depth int, entries ...LogEntry) {
+func (g GlogFormatter) Format(pkg string, level LogLevel, depth int, entries ...interface{}) {
 	g.w.Write(GlogHeader(level, depth+1))
 	g.StringFormatter.Format(pkg, level, depth+1, entries...)
 }

+ 1 - 1
Godeps/_workspace/src/github.com/coreos/pkg/capnslog/log_hijack.go

@@ -20,6 +20,6 @@ func (p packageWriter) Write(b []byte) (int, error) {
 	if p.pl.level < INFO {
 		return 0, nil
 	}
-	p.pl.internalLog(calldepth+2, INFO, BaseLogEntry(string(b)))
+	p.pl.internalLog(calldepth+2, INFO, string(b))
 	return len(b), nil
 }

+ 40 - 20
Godeps/_workspace/src/github.com/coreos/pkg/capnslog/logmap.go

@@ -11,19 +11,19 @@ type LogLevel int8
 
 const (
 	// CRITICAL is the lowest log level; only errors which will end the program will be propagated.
-	CRITICAL LogLevel = -1
+	CRITICAL LogLevel = iota - 1
 	// ERROR is for errors that are not fatal but lead to troubling behavior.
-	ERROR = 0
+	ERROR
 	// WARNING is for errors which are not fatal and not errors, but are unusual. Often sourced from misconfigurations.
-	WARNING = 1
+	WARNING
 	// NOTICE is for normal but significant conditions.
-	NOTICE = 2
+	NOTICE
 	// INFO is a log level for common, everyday log updates.
-	INFO = 3
+	INFO
 	// DEBUG is the default hidden level for more verbose updates about internal processes.
-	DEBUG = 4
+	DEBUG
 	// TRACE is for (potentially) call by call tracing of programs.
-	TRACE = 5
+	TRACE
 )
 
 // Char returns a single-character representation of the log level.
@@ -48,6 +48,39 @@ func (l LogLevel) Char() string {
 	}
 }
 
+// String returns a multi-character representation of the log level.
+func (l LogLevel) String() string {
+	switch l {
+	case CRITICAL:
+		return "CRITICAL"
+	case ERROR:
+		return "ERROR"
+	case WARNING:
+		return "WARNING"
+	case NOTICE:
+		return "NOTICE"
+	case INFO:
+		return "INFO"
+	case DEBUG:
+		return "DEBUG"
+	case TRACE:
+		return "TRACE"
+	default:
+		panic("Unhandled loglevel")
+	}
+}
+
+// Update using the given string value. Fulfills the flag.Value interface.
+func (l *LogLevel) Set(s string) error {
+	value, err := ParseLevel(s)
+	if err != nil {
+		return err
+	}
+
+	*l = value
+	return nil
+}
+
 // ParseLevel translates some potential loglevel strings into their corresponding levels.
 func ParseLevel(s string) (LogLevel, error) {
 	switch s {
@@ -71,13 +104,6 @@ func ParseLevel(s string) (LogLevel, error) {
 
 type RepoLogger map[string]*PackageLogger
 
-// LogEntry is the generic interface for things which can be logged.
-// Implementing the single method LogString() on your objects allows you to
-// format them for logs/debugging as necessary.
-type LogEntry interface {
-	LogString() string
-}
-
 type loggerStruct struct {
 	sync.Mutex
 	repoMap   map[string]RepoLogger
@@ -198,9 +224,3 @@ func NewPackageLogger(repo string, pkg string) (p *PackageLogger) {
 	}
 	return
 }
-
-type BaseLogEntry string
-
-func (b BaseLogEntry) LogString() string {
-	return string(b)
-}

+ 33 - 65
Godeps/_workspace/src/github.com/coreos/pkg/capnslog/pkg_logger.go

@@ -12,7 +12,10 @@ type PackageLogger struct {
 
 const calldepth = 3
 
-func (p *PackageLogger) internalLog(depth int, inLevel LogLevel, entries ...LogEntry) {
+func (p *PackageLogger) internalLog(depth int, inLevel LogLevel, entries ...interface{}) {
+	if inLevel != CRITICAL && p.level < inLevel {
+		return
+	}
 	logger.Lock()
 	defer logger.Unlock()
 	if logger.formatter != nil {
@@ -24,148 +27,113 @@ func (p *PackageLogger) LevelAt(l LogLevel) bool {
 	return p.level >= l
 }
 
+// Log a formatted string at any level between ERROR and TRACE
+func (p *PackageLogger) Logf(l LogLevel, format string, args ...interface{}) {
+	p.internalLog(calldepth, l, fmt.Sprintf(format, args...))
+}
+
+// Log a message at any level between ERROR and TRACE
+func (p *PackageLogger) Log(l LogLevel, args ...interface{}) {
+	p.internalLog(calldepth, l, fmt.Sprint(args...))
+}
+
 // log stdlib compatibility
 
 func (p *PackageLogger) Println(args ...interface{}) {
-	if p.level < INFO {
-		return
-	}
-	p.internalLog(calldepth, INFO, BaseLogEntry(fmt.Sprintln(args...)))
+	p.internalLog(calldepth, INFO, fmt.Sprintln(args...))
 }
 
 func (p *PackageLogger) Printf(format string, args ...interface{}) {
-	if p.level < INFO {
-		return
-	}
-	p.internalLog(calldepth, INFO, BaseLogEntry(fmt.Sprintf(format, args...)))
+	p.internalLog(calldepth, INFO, fmt.Sprintf(format, args...))
 }
 
 func (p *PackageLogger) Print(args ...interface{}) {
-	if p.level < INFO {
-		return
-	}
-	p.internalLog(calldepth, INFO, BaseLogEntry(fmt.Sprint(args...)))
+	p.internalLog(calldepth, INFO, fmt.Sprint(args...))
 }
 
 // Panic and fatal
 
 func (p *PackageLogger) Panicf(format string, args ...interface{}) {
 	s := fmt.Sprintf(format, args...)
-	p.internalLog(calldepth, CRITICAL, BaseLogEntry(s))
+	p.internalLog(calldepth, CRITICAL, s)
 	panic(s)
 }
 
 func (p *PackageLogger) Panic(args ...interface{}) {
 	s := fmt.Sprint(args...)
-	p.internalLog(calldepth, CRITICAL, BaseLogEntry(s))
+	p.internalLog(calldepth, CRITICAL, s)
 	panic(s)
 }
 
 func (p *PackageLogger) Fatalf(format string, args ...interface{}) {
 	s := fmt.Sprintf(format, args...)
-	p.internalLog(calldepth, CRITICAL, BaseLogEntry(s))
+	p.internalLog(calldepth, CRITICAL, s)
 	os.Exit(1)
 }
 
 func (p *PackageLogger) Fatal(args ...interface{}) {
 	s := fmt.Sprint(args...)
-	p.internalLog(calldepth, CRITICAL, BaseLogEntry(s))
+	p.internalLog(calldepth, CRITICAL, s)
 	os.Exit(1)
 }
 
 // Error Functions
 
 func (p *PackageLogger) Errorf(format string, args ...interface{}) {
-	if p.level < ERROR {
-		return
-	}
-	p.internalLog(calldepth, ERROR, BaseLogEntry(fmt.Sprintf(format, args...)))
+	p.internalLog(calldepth, ERROR, fmt.Sprintf(format, args...))
 }
 
-func (p *PackageLogger) Error(entries ...LogEntry) {
-	if p.level < ERROR {
-		return
-	}
+func (p *PackageLogger) Error(entries ...interface{}) {
 	p.internalLog(calldepth, ERROR, entries...)
 }
 
 // Warning Functions
 
 func (p *PackageLogger) Warningf(format string, args ...interface{}) {
-	if p.level < WARNING {
-		return
-	}
-	p.internalLog(calldepth, WARNING, BaseLogEntry(fmt.Sprintf(format, args...)))
+	p.internalLog(calldepth, WARNING, fmt.Sprintf(format, args...))
 }
 
-func (p *PackageLogger) Warning(entries ...LogEntry) {
-	if p.level < WARNING {
-		return
-	}
+func (p *PackageLogger) Warning(entries ...interface{}) {
 	p.internalLog(calldepth, WARNING, entries...)
 }
 
 // Notice Functions
 
 func (p *PackageLogger) Noticef(format string, args ...interface{}) {
-	if p.level < NOTICE {
-		return
-	}
-	p.internalLog(calldepth, NOTICE, BaseLogEntry(fmt.Sprintf(format, args...)))
+	p.internalLog(calldepth, NOTICE, fmt.Sprintf(format, args...))
 }
 
-func (p *PackageLogger) Notice(entries ...LogEntry) {
-	if p.level < NOTICE {
-		return
-	}
+func (p *PackageLogger) Notice(entries ...interface{}) {
 	p.internalLog(calldepth, NOTICE, entries...)
 }
 
 // Info Functions
 
 func (p *PackageLogger) Infof(format string, args ...interface{}) {
-	if p.level < INFO {
-		return
-	}
-	p.internalLog(calldepth, INFO, BaseLogEntry(fmt.Sprintf(format, args...)))
+	p.internalLog(calldepth, INFO, fmt.Sprintf(format, args...))
 }
 
-func (p *PackageLogger) Info(entries ...LogEntry) {
-	if p.level < INFO {
-		return
-	}
+func (p *PackageLogger) Info(entries ...interface{}) {
 	p.internalLog(calldepth, INFO, entries...)
 }
 
 // Debug Functions
 
 func (p *PackageLogger) Debugf(format string, args ...interface{}) {
-	if p.level < DEBUG {
-		return
-	}
-	p.internalLog(calldepth, DEBUG, BaseLogEntry(fmt.Sprintf(format, args...)))
+	p.internalLog(calldepth, DEBUG, fmt.Sprintf(format, args...))
 }
 
-func (p *PackageLogger) Debug(entries ...LogEntry) {
-	if p.level < DEBUG {
-		return
-	}
+func (p *PackageLogger) Debug(entries ...interface{}) {
 	p.internalLog(calldepth, DEBUG, entries...)
 }
 
 // Trace Functions
 
 func (p *PackageLogger) Tracef(format string, args ...interface{}) {
-	if p.level < TRACE {
-		return
-	}
-	p.internalLog(calldepth, TRACE, BaseLogEntry(fmt.Sprintf(format, args...)))
+	p.internalLog(calldepth, TRACE, fmt.Sprintf(format, args...))
 }
 
-func (p *PackageLogger) Trace(entries ...LogEntry) {
-	if p.level < TRACE {
-		return
-	}
+func (p *PackageLogger) Trace(entries ...interface{}) {
 	p.internalLog(calldepth, TRACE, entries...)
 }
 

+ 3 - 2
Godeps/_workspace/src/github.com/coreos/pkg/capnslog/syslog_formatter.go

@@ -1,6 +1,7 @@
 package capnslog
 
 import (
+	"fmt"
 	"log/syslog"
 )
 
@@ -20,9 +21,9 @@ type syslogFormatter struct {
 	w *syslog.Writer
 }
 
-func (s *syslogFormatter) Format(pkg string, l LogLevel, _ int, entries ...LogEntry) {
+func (s *syslogFormatter) Format(pkg string, l LogLevel, _ int, entries ...interface{}) {
 	for _, entry := range entries {
-		str := entry.LogString()
+		str := fmt.Sprint(entry)
 		switch l {
 		case CRITICAL:
 			s.w.Crit(str)

+ 3 - 0
etcdserver/raft.go

@@ -31,6 +31,8 @@ import (
 	"github.com/coreos/etcd/rafthttp"
 	"github.com/coreos/etcd/wal"
 	"github.com/coreos/etcd/wal/walpb"
+
+	"github.com/coreos/etcd/Godeps/_workspace/src/github.com/coreos/pkg/capnslog"
 )
 
 const (
@@ -59,6 +61,7 @@ var (
 )
 
 func init() {
+	raft.SetLogger(capnslog.NewPackageLogger("github.com/coreos/etcd", "raft"))
 	expvar.Publish("raft.status", expvar.Func(func() interface{} { return raftStatus() }))
 }
 

+ 4 - 4
raft/log.go

@@ -113,7 +113,7 @@ func (l *raftLog) findConflict(ents []pb.Entry) uint64 {
 	for _, ne := range ents {
 		if !l.matchTerm(ne.Index, ne.Term) {
 			if ne.Index <= l.lastIndex() {
-				raftLogger.Infof("raftlog: found conflict at index %d [existing term: %d, conflicting term: %d]",
+				raftLogger.Infof("found conflict at index %d [existing term: %d, conflicting term: %d]",
 					ne.Index, l.term(ne.Index), ne.Term)
 			}
 			return ne.Index
@@ -247,7 +247,7 @@ func (l *raftLog) maybeCommit(maxIndex, term uint64) bool {
 }
 
 func (l *raftLog) restore(s pb.Snapshot) {
-	raftLogger.Infof("raftlog: log [%s] starts to restore snapshot [index: %d, term: %d]", l, s.Metadata.Index, s.Metadata.Term)
+	raftLogger.Infof("log [%s] starts to restore snapshot [index: %d, term: %d]", l, s.Metadata.Index, s.Metadata.Term)
 	l.committed = s.Metadata.Index
 	l.unstable.restore(s)
 }
@@ -292,10 +292,10 @@ func (l *raftLog) slice(lo, hi, maxSize uint64) []pb.Entry {
 // l.firstIndex <= lo <= hi <= l.firstIndex + len(l.entries)
 func (l *raftLog) mustCheckOutOfBounds(lo, hi uint64) {
 	if lo > hi {
-		raftLogger.Panicf("raft: invalid slice %d > %d", lo, hi)
+		raftLogger.Panicf("invalid slice %d > %d", lo, hi)
 	}
 	length := l.lastIndex() - l.firstIndex() + 1
 	if lo < l.firstIndex() || hi > l.firstIndex()+length {
-		raftLogger.Panicf("raft: slice[%d,%d) out of bound [%d,%d]", lo, hi, l.firstIndex(), l.lastIndex())
+		raftLogger.Panicf("slice[%d,%d) out of bound [%d,%d]", lo, hi, l.firstIndex(), l.lastIndex())
 	}
 }

+ 4 - 4
raft/log_unstable.go

@@ -106,7 +106,7 @@ func (u *unstable) truncateAndAppend(ents []pb.Entry) {
 		// directly append
 		u.entries = append(u.entries, ents...)
 	case after < u.offset:
-		raftLogger.Infof("raftlog: replace the unstable entries from index %d", after+1)
+		raftLogger.Infof("replace the unstable entries from index %d", after+1)
 		// The log is being truncated to before our current offset
 		// portion, so set the offset and replace the entries
 		u.offset = after + 1
@@ -114,7 +114,7 @@ func (u *unstable) truncateAndAppend(ents []pb.Entry) {
 	default:
 		// truncate to after and copy to u.entries
 		// then append
-		raftLogger.Infof("raftlog: truncate the unstable entries to index %d", after)
+		raftLogger.Infof("truncate the unstable entries to index %d", after)
 		u.entries = append([]pb.Entry{}, u.slice(u.offset, after+1)...)
 		u.entries = append(u.entries, ents...)
 	}
@@ -128,10 +128,10 @@ func (u *unstable) slice(lo uint64, hi uint64) []pb.Entry {
 // u.offset <= lo <= hi <= u.offset+len(u.offset)
 func (u *unstable) mustCheckOutOfBounds(lo, hi uint64) {
 	if lo > hi {
-		raftLogger.Panicf("raft: invalid unstable.slice %d > %d", lo, hi)
+		raftLogger.Panicf("invalid unstable.slice %d > %d", lo, hi)
 	}
 	upper := u.offset + uint64(len(u.entries))
 	if lo < u.offset || hi > upper {
-		raftLogger.Panicf("raft: unstable.slice[%d,%d) out of bound [%d,%d]", lo, hi, u.offset, upper)
+		raftLogger.Panicf("unstable.slice[%d,%d) out of bound [%d,%d]", lo, hi, u.offset, upper)
 	}
 }

+ 1 - 1
raft/logger.go

@@ -44,7 +44,7 @@ type Logger interface {
 func SetLogger(l Logger) { raftLogger = l }
 
 var (
-	defaultLogger = &DefaultLogger{Logger: log.New(os.Stderr, "", log.LstdFlags)}
+	defaultLogger = &DefaultLogger{Logger: log.New(os.Stderr, "raft", log.LstdFlags)}
 	discardLogger = &DefaultLogger{Logger: log.New(ioutil.Discard, "", 0)}
 	raftLogger    = Logger(defaultLogger)
 )

+ 34 - 34
raft/raft.go

@@ -203,7 +203,7 @@ func newRaft(c *Config) *raft {
 		nodesStrs = append(nodesStrs, fmt.Sprintf("%x", n))
 	}
 
-	raftLogger.Infof("raft: newRaft %x [peers: [%s], term: %d, commit: %d, applied: %d, lastindex: %d, lastterm: %d]",
+	raftLogger.Infof("newRaft %x [peers: [%s], term: %d, commit: %d, applied: %d, lastindex: %d, lastterm: %d]",
 		r.id, strings.Join(nodesStrs, ","), r.Term, r.raftLog.committed, r.raftLog.applied, r.raftLog.lastIndex(), r.raftLog.lastTerm())
 	return r
 }
@@ -254,10 +254,10 @@ func (r *raft) sendAppend(to uint64) {
 		}
 		m.Snapshot = snapshot
 		sindex, sterm := snapshot.Metadata.Index, snapshot.Metadata.Term
-		raftLogger.Infof("raft: %x [firstindex: %d, commit: %d] sent snapshot[index: %d, term: %d] to %x [%s]",
+		raftLogger.Infof("%x [firstindex: %d, commit: %d] sent snapshot[index: %d, term: %d] to %x [%s]",
 			r.id, r.raftLog.firstIndex(), r.Commit, sindex, sterm, to, pr)
 		pr.becomeSnapshot(sindex)
-		raftLogger.Infof("raft: %x paused sending replication messages to %x [%s]", r.id, to, pr)
+		raftLogger.Infof("%x paused sending replication messages to %x [%s]", r.id, to, pr)
 	} else {
 		m.Type = pb.MsgApp
 		m.Index = pr.Next - 1
@@ -274,7 +274,7 @@ func (r *raft) sendAppend(to uint64) {
 			case ProgressStateProbe:
 				pr.pause()
 			default:
-				raftLogger.Panicf("raft: %x is sending append in unhandled state %s", r.id, pr.State)
+				raftLogger.Panicf("%x is sending append in unhandled state %s", r.id, pr.State)
 			}
 		}
 	}
@@ -387,7 +387,7 @@ func (r *raft) becomeFollower(term uint64, lead uint64) {
 	r.tick = r.tickElection
 	r.lead = lead
 	r.state = StateFollower
-	raftLogger.Infof("raft: %x became follower at term %d", r.id, r.Term)
+	raftLogger.Infof("%x became follower at term %d", r.id, r.Term)
 }
 
 func (r *raft) becomeCandidate() {
@@ -400,7 +400,7 @@ func (r *raft) becomeCandidate() {
 	r.tick = r.tickElection
 	r.Vote = r.id
 	r.state = StateCandidate
-	raftLogger.Infof("raft: %x became candidate at term %d", r.id, r.Term)
+	raftLogger.Infof("%x became candidate at term %d", r.id, r.Term)
 }
 
 func (r *raft) becomeLeader() {
@@ -423,7 +423,7 @@ func (r *raft) becomeLeader() {
 		r.pendingConf = true
 	}
 	r.appendEntry(pb.Entry{Data: nil})
-	raftLogger.Infof("raft: %x became leader at term %d", r.id, r.Term)
+	raftLogger.Infof("%x became leader at term %d", r.id, r.Term)
 }
 
 func (r *raft) campaign() {
@@ -436,7 +436,7 @@ func (r *raft) campaign() {
 		if i == r.id {
 			continue
 		}
-		raftLogger.Infof("raft: %x [logterm: %d, index: %d] sent vote request to %x at term %d",
+		raftLogger.Infof("%x [logterm: %d, index: %d] sent vote request to %x at term %d",
 			r.id, r.raftLog.lastTerm(), r.raftLog.lastIndex(), i, r.Term)
 		r.send(pb.Message{To: i, Type: pb.MsgVote, Index: r.raftLog.lastIndex(), LogTerm: r.raftLog.lastTerm()})
 	}
@@ -444,9 +444,9 @@ func (r *raft) campaign() {
 
 func (r *raft) poll(id uint64, v bool) (granted int) {
 	if v {
-		raftLogger.Infof("raft: %x received vote from %x at term %d", r.id, id, r.Term)
+		raftLogger.Infof("%x received vote from %x at term %d", r.id, id, r.Term)
 	} else {
-		raftLogger.Infof("raft: %x received vote rejection from %x at term %d", r.id, id, r.Term)
+		raftLogger.Infof("%x received vote rejection from %x at term %d", r.id, id, r.Term)
 	}
 	if _, ok := r.votes[id]; !ok {
 		r.votes[id] = v
@@ -461,7 +461,7 @@ func (r *raft) poll(id uint64, v bool) (granted int) {
 
 func (r *raft) Step(m pb.Message) error {
 	if m.Type == pb.MsgHup {
-		raftLogger.Infof("raft: %x is starting a new election at term %d", r.id, r.Term)
+		raftLogger.Infof("%x is starting a new election at term %d", r.id, r.Term)
 		r.campaign()
 		r.Commit = r.raftLog.committed
 		return nil
@@ -475,12 +475,12 @@ func (r *raft) Step(m pb.Message) error {
 		if m.Type == pb.MsgVote {
 			lead = None
 		}
-		raftLogger.Infof("raft: %x [term: %d] received a %s message with higher term from %x [term: %d]",
+		raftLogger.Infof("%x [term: %d] received a %s message with higher term from %x [term: %d]",
 			r.id, r.Term, m.Type, m.From, m.Term)
 		r.becomeFollower(m.Term, lead)
 	case m.Term < r.Term:
 		// ignore
-		raftLogger.Infof("raft: %x [term: %d] ignored a %s message with lower term from %x [term: %d]",
+		raftLogger.Infof("%x [term: %d] ignored a %s message with lower term from %x [term: %d]",
 			r.id, r.Term, m.Type, m.From, m.Term)
 		return nil
 	}
@@ -499,7 +499,7 @@ func stepLeader(r *raft, m pb.Message) {
 		r.bcastHeartbeat()
 	case pb.MsgProp:
 		if len(m.Entries) == 0 {
-			raftLogger.Panicf("raft: %x stepped empty MsgProp", r.id)
+			raftLogger.Panicf("%x stepped empty MsgProp", r.id)
 		}
 		for i, e := range m.Entries {
 			if e.Type == pb.EntryConfChange {
@@ -513,10 +513,10 @@ func stepLeader(r *raft, m pb.Message) {
 		r.bcastAppend()
 	case pb.MsgAppResp:
 		if m.Reject {
-			raftLogger.Infof("raft: %x received msgApp rejection(lastindex: %d) from %x for index %d",
+			raftLogger.Infof("%x received msgApp rejection(lastindex: %d) from %x for index %d",
 				r.id, m.RejectHint, m.From, m.Index)
 			if pr.maybeDecrTo(m.Index, m.RejectHint) {
-				raftLogger.Infof("raft: %x decreased progress of %x to [%s]", r.id, m.From, pr)
+				raftLogger.Infof("%x decreased progress of %x to [%s]", r.id, m.From, pr)
 				if pr.State == ProgressStateReplicate {
 					pr.becomeProbe()
 				}
@@ -529,7 +529,7 @@ func stepLeader(r *raft, m pb.Message) {
 				case pr.State == ProgressStateProbe:
 					pr.becomeReplicate()
 				case pr.State == ProgressStateSnapshot && pr.maybeSnapshotAbort():
-					raftLogger.Infof("raft: %x snapshot aborted, resumed sending replication messages to %x [%s]", r.id, m.From, pr)
+					raftLogger.Infof("%x snapshot aborted, resumed sending replication messages to %x [%s]", r.id, m.From, pr)
 					pr.becomeProbe()
 				case pr.State == ProgressStateReplicate:
 					pr.ins.freeTo(m.Index)
@@ -553,7 +553,7 @@ func stepLeader(r *raft, m pb.Message) {
 			r.sendAppend(m.From)
 		}
 	case pb.MsgVote:
-		raftLogger.Infof("raft: %x [logterm: %d, index: %d, vote: %x] rejected vote from %x [logterm: %d, index: %d] at term %d",
+		raftLogger.Infof("%x [logterm: %d, index: %d, vote: %x] rejected vote from %x [logterm: %d, index: %d] at term %d",
 			r.id, r.raftLog.lastTerm(), r.raftLog.lastIndex(), r.Vote, m.From, m.LogTerm, m.Index, r.Term)
 		r.send(pb.Message{To: m.From, Type: pb.MsgVoteResp, Reject: true})
 	case pb.MsgSnapStatus:
@@ -562,11 +562,11 @@ func stepLeader(r *raft, m pb.Message) {
 		}
 		if !m.Reject {
 			pr.becomeProbe()
-			raftLogger.Infof("raft: %x snapshot succeeded, resumed sending replication messages to %x [%s]", r.id, m.From, pr)
+			raftLogger.Infof("%x snapshot succeeded, resumed sending replication messages to %x [%s]", r.id, m.From, pr)
 		} else {
 			pr.snapshotFailure()
 			pr.becomeProbe()
-			raftLogger.Infof("raft: %x snapshot failed, resumed sending replication messages to %x [%s]", r.id, m.From, pr)
+			raftLogger.Infof("%x snapshot failed, resumed sending replication messages to %x [%s]", r.id, m.From, pr)
 		}
 		// If snapshot finish, wait for the msgAppResp from the remote node before sending
 		// out the next msgApp.
@@ -578,14 +578,14 @@ func stepLeader(r *raft, m pb.Message) {
 		if pr.State == ProgressStateReplicate {
 			pr.becomeProbe()
 		}
-		raftLogger.Infof("raft: %x failed to send message to %x because it is unreachable [%s]", r.id, m.From, pr)
+		raftLogger.Infof("%x failed to send message to %x because it is unreachable [%s]", r.id, m.From, pr)
 	}
 }
 
 func stepCandidate(r *raft, m pb.Message) {
 	switch m.Type {
 	case pb.MsgProp:
-		raftLogger.Infof("raft: %x no leader at term %d; dropping proposal", r.id, r.Term)
+		raftLogger.Infof("%x no leader at term %d; dropping proposal", r.id, r.Term)
 		return
 	case pb.MsgApp:
 		r.becomeFollower(r.Term, m.From)
@@ -597,12 +597,12 @@ func stepCandidate(r *raft, m pb.Message) {
 		r.becomeFollower(m.Term, m.From)
 		r.handleSnapshot(m)
 	case pb.MsgVote:
-		raftLogger.Infof("raft: %x [logterm: %d, index: %d, vote: %x] rejected vote from %x [logterm: %d, index: %d] at term %x",
+		raftLogger.Infof("%x [logterm: %d, index: %d, vote: %x] rejected vote from %x [logterm: %d, index: %d] at term %x",
 			r.id, r.raftLog.lastTerm(), r.raftLog.lastIndex(), r.Vote, m.From, m.LogTerm, m.Index, r.Term)
 		r.send(pb.Message{To: m.From, Type: pb.MsgVoteResp, Reject: true})
 	case pb.MsgVoteResp:
 		gr := r.poll(m.From, !m.Reject)
-		raftLogger.Infof("raft: %x [q:%d] has received %d votes and %d vote rejections", r.id, r.q(), gr, len(r.votes)-gr)
+		raftLogger.Infof("%x [q:%d] has received %d votes and %d vote rejections", r.id, r.q(), gr, len(r.votes)-gr)
 		switch r.q() {
 		case gr:
 			r.becomeLeader()
@@ -617,7 +617,7 @@ func stepFollower(r *raft, m pb.Message) {
 	switch m.Type {
 	case pb.MsgProp:
 		if r.lead == None {
-			raftLogger.Infof("raft: %x no leader at term %d; dropping proposal", r.id, r.Term)
+			raftLogger.Infof("%x no leader at term %d; dropping proposal", r.id, r.Term)
 			return
 		}
 		m.To = r.lead
@@ -636,12 +636,12 @@ func stepFollower(r *raft, m pb.Message) {
 	case pb.MsgVote:
 		if (r.Vote == None || r.Vote == m.From) && r.raftLog.isUpToDate(m.Index, m.LogTerm) {
 			r.elapsed = 0
-			raftLogger.Infof("raft: %x [logterm: %d, index: %d, vote: %x] voted for %x [logterm: %d, index: %d] at term %d",
+			raftLogger.Infof("%x [logterm: %d, index: %d, vote: %x] voted for %x [logterm: %d, index: %d] at term %d",
 				r.id, r.raftLog.lastTerm(), r.raftLog.lastIndex(), r.Vote, m.From, m.LogTerm, m.Index, r.Term)
 			r.Vote = m.From
 			r.send(pb.Message{To: m.From, Type: pb.MsgVoteResp})
 		} else {
-			raftLogger.Infof("raft: %x [logterm: %d, index: %d, vote: %x] rejected vote from %x [logterm: %d, index: %d] at term %d",
+			raftLogger.Infof("%x [logterm: %d, index: %d, vote: %x] rejected vote from %x [logterm: %d, index: %d] at term %d",
 				r.id, r.raftLog.lastTerm(), r.raftLog.lastIndex(), r.Vote, m.From, m.LogTerm, m.Index, r.Term)
 			r.send(pb.Message{To: m.From, Type: pb.MsgVoteResp, Reject: true})
 		}
@@ -657,7 +657,7 @@ func (r *raft) handleAppendEntries(m pb.Message) {
 	if mlastIndex, ok := r.raftLog.maybeAppend(m.Index, m.LogTerm, m.Commit, m.Entries...); ok {
 		r.send(pb.Message{To: m.From, Type: pb.MsgAppResp, Index: mlastIndex})
 	} else {
-		raftLogger.Infof("raft: %x [logterm: %d, index: %d] rejected msgApp [logterm: %d, index: %d] from %x",
+		raftLogger.Infof("%x [logterm: %d, index: %d] rejected msgApp [logterm: %d, index: %d] from %x",
 			r.id, r.raftLog.term(m.Index), m.Index, m.LogTerm, m.Index, m.From)
 		r.send(pb.Message{To: m.From, Type: pb.MsgAppResp, Index: m.Index, Reject: true, RejectHint: r.raftLog.lastIndex()})
 	}
@@ -671,11 +671,11 @@ func (r *raft) handleHeartbeat(m pb.Message) {
 func (r *raft) handleSnapshot(m pb.Message) {
 	sindex, sterm := m.Snapshot.Metadata.Index, m.Snapshot.Metadata.Term
 	if r.restore(m.Snapshot) {
-		raftLogger.Infof("raft: %x [commit: %d] restored snapshot [index: %d, term: %d]",
+		raftLogger.Infof("%x [commit: %d] restored snapshot [index: %d, term: %d]",
 			r.id, r.Commit, sindex, sterm)
 		r.send(pb.Message{To: m.From, Type: pb.MsgAppResp, Index: r.raftLog.lastIndex()})
 	} else {
-		raftLogger.Infof("raft: %x [commit: %d] ignored snapshot [index: %d, term: %d]",
+		raftLogger.Infof("%x [commit: %d] ignored snapshot [index: %d, term: %d]",
 			r.id, r.Commit, sindex, sterm)
 		r.send(pb.Message{To: m.From, Type: pb.MsgAppResp, Index: r.raftLog.committed})
 	}
@@ -688,13 +688,13 @@ func (r *raft) restore(s pb.Snapshot) bool {
 		return false
 	}
 	if r.raftLog.matchTerm(s.Metadata.Index, s.Metadata.Term) {
-		raftLogger.Infof("raft: %x [commit: %d, lastindex: %d, lastterm: %d] fast-forwarded commit to snapshot [index: %d, term: %d]",
+		raftLogger.Infof("%x [commit: %d, lastindex: %d, lastterm: %d] fast-forwarded commit to snapshot [index: %d, term: %d]",
 			r.id, r.Commit, r.raftLog.lastIndex(), r.raftLog.lastTerm(), s.Metadata.Index, s.Metadata.Term)
 		r.raftLog.commitTo(s.Metadata.Index)
 		return false
 	}
 
-	raftLogger.Infof("raft: %x [commit: %d, lastindex: %d, lastterm: %d] starts to restore snapshot [index: %d, term: %d]",
+	raftLogger.Infof("%x [commit: %d, lastindex: %d, lastterm: %d] starts to restore snapshot [index: %d, term: %d]",
 		r.id, r.Commit, r.raftLog.lastIndex(), r.raftLog.lastTerm(), s.Metadata.Index, s.Metadata.Term)
 
 	r.raftLog.restore(s)
@@ -707,7 +707,7 @@ func (r *raft) restore(s pb.Snapshot) bool {
 			match = 0
 		}
 		r.setProgress(n, match, next)
-		raftLogger.Infof("raft: %x restored progress of %x [%s]", r.id, n, r.prs[n])
+		raftLogger.Infof("%x restored progress of %x [%s]", r.id, n, r.prs[n])
 	}
 	return true
 }
@@ -751,7 +751,7 @@ func (r *raft) delProgress(id uint64) {
 
 func (r *raft) loadState(state pb.HardState) {
 	if state.Commit < r.raftLog.committed || state.Commit > r.raftLog.lastIndex() {
-		raftLogger.Panicf("raft: %x state.commit %d is out of range [%d, %d]", r.id, state.Commit, r.raftLog.committed, r.raftLog.lastIndex())
+		raftLogger.Panicf("%x state.commit %d is out of range [%d, %d]", r.id, state.Commit, r.raftLog.committed, r.raftLog.lastIndex())
 	}
 	r.raftLog.committed = state.Commit
 	r.Term = state.Term