소스 검색

Merge pull request #2911 from yichengq/rafthttp-plog

rafthttp: use leveled logger
Yicheng Qin 10 년 전
부모
커밋
1403783326
6개의 변경된 파일51개의 추가작업 그리고 44개의 파일을 삭제
  1. 17 13
      rafthttp/http.go
  2. 5 5
      rafthttp/peer.go
  3. 5 6
      rafthttp/pipeline.go
  4. 2 2
      rafthttp/remote.go
  5. 14 12
      rafthttp/stream.go
  6. 8 6
      rafthttp/transport.go

+ 17 - 13
rafthttp/http.go

@@ -17,7 +17,6 @@ package rafthttp
 import (
 import (
 	"errors"
 	"errors"
 	"io/ioutil"
 	"io/ioutil"
-	"log"
 	"net/http"
 	"net/http"
 	"path"
 	"path"
 
 
@@ -77,7 +76,7 @@ func (h *handler) ServeHTTP(w http.ResponseWriter, r *http.Request) {
 	}
 	}
 
 
 	if err := checkVersionCompability(r.Header.Get("X-Server-From"), serverVersion(r.Header), minClusterVersion(r.Header)); err != nil {
 	if err := checkVersionCompability(r.Header.Get("X-Server-From"), serverVersion(r.Header), minClusterVersion(r.Header)); err != nil {
-		log.Printf("rafthttp: request received was ignored (%v)", err)
+		plog.Errorf("request received was ignored (%v)", err)
 		http.Error(w, errIncompatibleVersion.Error(), http.StatusPreconditionFailed)
 		http.Error(w, errIncompatibleVersion.Error(), http.StatusPreconditionFailed)
 		return
 		return
 	}
 	}
@@ -87,7 +86,7 @@ func (h *handler) ServeHTTP(w http.ResponseWriter, r *http.Request) {
 
 
 	gcid := r.Header.Get("X-Etcd-Cluster-ID")
 	gcid := r.Header.Get("X-Etcd-Cluster-ID")
 	if gcid != wcid {
 	if gcid != wcid {
-		log.Printf("rafthttp: request ignored due to cluster ID mismatch got %s want %s", gcid, wcid)
+		plog.Errorf("request received was ignored (cluster ID mismatch got %s want %s)", gcid, wcid)
 		http.Error(w, errClusterIDMismatch.Error(), http.StatusPreconditionFailed)
 		http.Error(w, errClusterIDMismatch.Error(), http.StatusPreconditionFailed)
 		return
 		return
 	}
 	}
@@ -97,13 +96,13 @@ func (h *handler) ServeHTTP(w http.ResponseWriter, r *http.Request) {
 	limitedr := pioutil.NewLimitedBufferReader(r.Body, ConnReadLimitByte)
 	limitedr := pioutil.NewLimitedBufferReader(r.Body, ConnReadLimitByte)
 	b, err := ioutil.ReadAll(limitedr)
 	b, err := ioutil.ReadAll(limitedr)
 	if err != nil {
 	if err != nil {
-		log.Println("rafthttp: error reading raft message:", err)
+		plog.Errorf("failed to read raft message (%v)", err)
 		http.Error(w, "error reading raft message", http.StatusBadRequest)
 		http.Error(w, "error reading raft message", http.StatusBadRequest)
 		return
 		return
 	}
 	}
 	var m raftpb.Message
 	var m raftpb.Message
 	if err := m.Unmarshal(b); err != nil {
 	if err := m.Unmarshal(b); err != nil {
-		log.Println("rafthttp: error unmarshaling raft message:", err)
+		plog.Errorf("failed to unmarshal raft message (%v)", err)
 		http.Error(w, "error unmarshaling raft message", http.StatusBadRequest)
 		http.Error(w, "error unmarshaling raft message", http.StatusBadRequest)
 		return
 		return
 	}
 	}
@@ -112,7 +111,7 @@ func (h *handler) ServeHTTP(w http.ResponseWriter, r *http.Request) {
 		case writerToResponse:
 		case writerToResponse:
 			v.WriteTo(w)
 			v.WriteTo(w)
 		default:
 		default:
-			log.Printf("rafthttp: error processing raft message: %v", err)
+			plog.Warningf("failed to process raft message (%v)", err)
 			http.Error(w, "error processing raft message", http.StatusInternalServerError)
 			http.Error(w, "error processing raft message", http.StatusInternalServerError)
 		}
 		}
 		return
 		return
@@ -139,7 +138,7 @@ func (h *streamHandler) ServeHTTP(w http.ResponseWriter, r *http.Request) {
 	w.Header().Set("X-Server-Version", version.Version)
 	w.Header().Set("X-Server-Version", version.Version)
 
 
 	if err := checkVersionCompability(r.Header.Get("X-Server-From"), serverVersion(r.Header), minClusterVersion(r.Header)); err != nil {
 	if err := checkVersionCompability(r.Header.Get("X-Server-From"), serverVersion(r.Header), minClusterVersion(r.Header)); err != nil {
-		log.Printf("rafthttp: request received was ignored (%v)", err)
+		plog.Errorf("request received was ignored (%v)", err)
 		http.Error(w, errIncompatibleVersion.Error(), http.StatusPreconditionFailed)
 		http.Error(w, errIncompatibleVersion.Error(), http.StatusPreconditionFailed)
 		return
 		return
 	}
 	}
@@ -148,7 +147,7 @@ func (h *streamHandler) ServeHTTP(w http.ResponseWriter, r *http.Request) {
 	w.Header().Set("X-Etcd-Cluster-ID", wcid)
 	w.Header().Set("X-Etcd-Cluster-ID", wcid)
 
 
 	if gcid := r.Header.Get("X-Etcd-Cluster-ID"); gcid != wcid {
 	if gcid := r.Header.Get("X-Etcd-Cluster-ID"); gcid != wcid {
-		log.Printf("rafthttp: streaming request ignored due to cluster ID mismatch got %s want %s", gcid, wcid)
+		plog.Errorf("streaming request ignored (cluster ID mismatch got %s want %s)", gcid, wcid)
 		http.Error(w, errClusterIDMismatch.Error(), http.StatusPreconditionFailed)
 		http.Error(w, errClusterIDMismatch.Error(), http.StatusPreconditionFailed)
 		return
 		return
 	}
 	}
@@ -163,7 +162,7 @@ func (h *streamHandler) ServeHTTP(w http.ResponseWriter, r *http.Request) {
 	case path.Join(RaftStreamPrefix, string(streamTypeMessage)):
 	case path.Join(RaftStreamPrefix, string(streamTypeMessage)):
 		t = streamTypeMessage
 		t = streamTypeMessage
 	default:
 	default:
-		log.Printf("rafthttp: ignored unexpected streaming request path %s", r.URL.Path)
+		plog.Debugf("ignored unexpected streaming request path %s", r.URL.Path)
 		http.Error(w, "invalid path", http.StatusNotFound)
 		http.Error(w, "invalid path", http.StatusNotFound)
 		return
 		return
 	}
 	}
@@ -171,25 +170,30 @@ func (h *streamHandler) ServeHTTP(w http.ResponseWriter, r *http.Request) {
 	fromStr := path.Base(r.URL.Path)
 	fromStr := path.Base(r.URL.Path)
 	from, err := types.IDFromString(fromStr)
 	from, err := types.IDFromString(fromStr)
 	if err != nil {
 	if err != nil {
-		log.Printf("rafthttp: failed to parse from %s into ID", fromStr)
+		plog.Errorf("failed to parse from %s into ID (%v)", fromStr, err)
 		http.Error(w, "invalid from", http.StatusNotFound)
 		http.Error(w, "invalid from", http.StatusNotFound)
 		return
 		return
 	}
 	}
 	if h.r.IsIDRemoved(uint64(from)) {
 	if h.r.IsIDRemoved(uint64(from)) {
-		log.Printf("rafthttp: reject the stream from peer %s since it was removed", from)
+		plog.Warningf("rejected the stream from peer %s since it was removed", from)
 		http.Error(w, "removed member", http.StatusGone)
 		http.Error(w, "removed member", http.StatusGone)
 		return
 		return
 	}
 	}
 	p := h.peerGetter.Get(from)
 	p := h.peerGetter.Get(from)
 	if p == nil {
 	if p == nil {
-		log.Printf("rafthttp: fail to find sender %s", from)
+		// This may happen in following cases:
+		// 1. user starts a remote peer that belongs to a different cluster
+		// with the same cluster ID.
+		// 2. local etcd falls behind of the cluster, and cannot recognize
+		// the members that joined after its current progress.
+		plog.Errorf("failed to find member %s in cluster %s", from, wcid)
 		http.Error(w, "error sender not found", http.StatusNotFound)
 		http.Error(w, "error sender not found", http.StatusNotFound)
 		return
 		return
 	}
 	}
 
 
 	wto := h.id.String()
 	wto := h.id.String()
 	if gto := r.Header.Get("X-Raft-To"); gto != wto {
 	if gto := r.Header.Get("X-Raft-To"); gto != wto {
-		log.Printf("rafthttp: streaming request ignored due to ID mismatch got %s want %s", gto, wto)
+		plog.Errorf("streaming request ignored (ID mismatch got %s want %s)", gto, wto)
 		http.Error(w, "to field mismatch", http.StatusPreconditionFailed)
 		http.Error(w, "to field mismatch", http.StatusPreconditionFailed)
 		return
 		return
 	}
 	}

+ 5 - 5
rafthttp/peer.go

@@ -15,7 +15,6 @@
 package rafthttp
 package rafthttp
 
 
 import (
 import (
-	"log"
 	"net/http"
 	"net/http"
 	"time"
 	"time"
 
 
@@ -135,7 +134,7 @@ func startPeer(tr http.RoundTripper, urls types.URLs, local, to, cid types.ID, r
 			select {
 			select {
 			case mm := <-p.propc:
 			case mm := <-p.propc:
 				if err := r.Process(ctx, mm); err != nil {
 				if err := r.Process(ctx, mm); err != nil {
-					log.Printf("peer: process raft message error: %v", err)
+					plog.Warningf("failed to process raft message (%v)", err)
 				}
 				}
 			case <-p.stopc:
 			case <-p.stopc:
 				return
 				return
@@ -161,11 +160,12 @@ func startPeer(tr http.RoundTripper, urls types.URLs, local, to, cid types.ID, r
 					if isMsgSnap(m) {
 					if isMsgSnap(m) {
 						p.r.ReportSnapshot(m.To, raft.SnapshotFailure)
 						p.r.ReportSnapshot(m.To, raft.SnapshotFailure)
 					}
 					}
-					log.Printf("peer: dropping %s to %s since %s's sending buffer is full", m.Type, p.id, name)
+					// TODO: log start and end of message dropping
+					plog.Warningf("dropping %s to %s since %s's sending buffer is full", m.Type, p.id, name)
 				}
 				}
 			case mm := <-p.recvc:
 			case mm := <-p.recvc:
 				if err := r.Process(context.TODO(), mm); err != nil {
 				if err := r.Process(context.TODO(), mm); err != nil {
-					log.Printf("peer: process raft message error: %v", err)
+					plog.Warningf("failed to process raft message (%v)", err)
 				}
 				}
 			case urls := <-p.newURLsC:
 			case urls := <-p.newURLsC:
 				picker.update(urls)
 				picker.update(urls)
@@ -213,7 +213,7 @@ func (p *peer) attachOutgoingConn(conn *outgoingConn) {
 	case streamTypeMessage:
 	case streamTypeMessage:
 		ok = p.writer.attach(conn)
 		ok = p.writer.attach(conn)
 	default:
 	default:
-		log.Panicf("rafthttp: unhandled stream type %s", conn.t)
+		plog.Panicf("unhandled stream type %s", conn.t)
 	}
 	}
 	if !ok {
 	if !ok {
 		conn.Close()
 		conn.Close()

+ 5 - 6
rafthttp/pipeline.go

@@ -19,7 +19,6 @@ import (
 	"errors"
 	"errors"
 	"fmt"
 	"fmt"
 	"io/ioutil"
 	"io/ioutil"
-	"log"
 	"net/http"
 	"net/http"
 	"strings"
 	"strings"
 	"sync"
 	"sync"
@@ -111,11 +110,11 @@ func (p *pipeline) handle() {
 			reportSentFailure(pipelineMsg, m)
 			reportSentFailure(pipelineMsg, m)
 
 
 			if p.errored == nil || p.errored.Error() != err.Error() {
 			if p.errored == nil || p.errored.Error() != err.Error() {
-				log.Printf("pipeline: error posting to %s: %v", p.to, err)
+				plog.Errorf("failed to post to %s (%v)", p.to, err)
 				p.errored = err
 				p.errored = err
 			}
 			}
 			if p.active {
 			if p.active {
-				log.Printf("pipeline: the connection with %s became inactive", p.to)
+				plog.Infof("the connection with %s became inactive", p.to)
 				p.active = false
 				p.active = false
 			}
 			}
 			if m.Type == raftpb.MsgApp && p.fs != nil {
 			if m.Type == raftpb.MsgApp && p.fs != nil {
@@ -127,7 +126,7 @@ func (p *pipeline) handle() {
 			}
 			}
 		} else {
 		} else {
 			if !p.active {
 			if !p.active {
-				log.Printf("pipeline: the connection with %s became active", p.to)
+				plog.Infof("the connection with %s became active", p.to)
 				p.active = true
 				p.active = true
 				p.errored = nil
 				p.errored = nil
 			}
 			}
@@ -196,10 +195,10 @@ func (p *pipeline) post(data []byte) (err error) {
 	case http.StatusPreconditionFailed:
 	case http.StatusPreconditionFailed:
 		switch strings.TrimSuffix(string(b), "\n") {
 		switch strings.TrimSuffix(string(b), "\n") {
 		case errIncompatibleVersion.Error():
 		case errIncompatibleVersion.Error():
-			log.Printf("rafthttp: request sent was ignored by peer %s (server version incompatible)", p.to)
+			plog.Errorf("request sent was ignored by peer %s (server version incompatible)", p.to)
 			return errIncompatibleVersion
 			return errIncompatibleVersion
 		case errClusterIDMismatch.Error():
 		case errClusterIDMismatch.Error():
-			log.Printf("rafthttp: request sent was ignored (cluster ID mismatch: remote[%s]=%s, local=%s)",
+			plog.Errorf("request sent was ignored (cluster ID mismatch: remote[%s]=%s, local=%s)",
 				p.to, resp.Header.Get("X-Etcd-Cluster-ID"), p.cid)
 				p.to, resp.Header.Get("X-Etcd-Cluster-ID"), p.cid)
 			return errClusterIDMismatch
 			return errClusterIDMismatch
 		default:
 		default:

+ 2 - 2
rafthttp/remote.go

@@ -15,7 +15,6 @@
 package rafthttp
 package rafthttp
 
 
 import (
 import (
-	"log"
 	"net/http"
 	"net/http"
 
 
 	"github.com/coreos/etcd/pkg/types"
 	"github.com/coreos/etcd/pkg/types"
@@ -39,7 +38,8 @@ func (g *remote) Send(m raftpb.Message) {
 	select {
 	select {
 	case g.pipeline.msgc <- m:
 	case g.pipeline.msgc <- m:
 	default:
 	default:
-		log.Printf("remote: dropping %s to %s since sending buffer is full", m.Type, g.id)
+		// TODO: log start and end of message dropping
+		plog.Warningf("dropping %s to %s since sending buffer is full", m.Type, g.id)
 	}
 	}
 }
 }
 
 

+ 14 - 12
rafthttp/stream.go

@@ -18,7 +18,6 @@ import (
 	"fmt"
 	"fmt"
 	"io"
 	"io"
 	"io/ioutil"
 	"io/ioutil"
-	"log"
 	"net"
 	"net"
 	"net/http"
 	"net/http"
 	"path"
 	"path"
@@ -63,7 +62,7 @@ func (t streamType) endpoint() string {
 	case streamTypeMessage:
 	case streamTypeMessage:
 		return path.Join(RaftStreamPrefix, "message")
 		return path.Join(RaftStreamPrefix, "message")
 	default:
 	default:
-		log.Panicf("rafthttp: unhandled stream type %v", t)
+		plog.Panicf("unhandled stream type %v", t)
 		return ""
 		return ""
 	}
 	}
 }
 }
@@ -134,7 +133,7 @@ func (cw *streamWriter) run() {
 			if err := enc.encode(linkHeartbeatMessage); err != nil {
 			if err := enc.encode(linkHeartbeatMessage); err != nil {
 				reportSentFailure(string(t), linkHeartbeatMessage)
 				reportSentFailure(string(t), linkHeartbeatMessage)
 
 
-				log.Printf("rafthttp: failed to heartbeat on stream %s (%v)", t, err)
+				plog.Errorf("failed to heartbeat on stream %s (%v)", t, err)
 				cw.close()
 				cw.close()
 				heartbeatc, msgc = nil, nil
 				heartbeatc, msgc = nil, nil
 				continue
 				continue
@@ -156,7 +155,7 @@ func (cw *streamWriter) run() {
 			if err := enc.encode(m); err != nil {
 			if err := enc.encode(m); err != nil {
 				reportSentFailure(string(t), m)
 				reportSentFailure(string(t), m)
 
 
-				log.Printf("rafthttp: failed to send message on stream %s (%v)", t, err)
+				plog.Errorf("failed to send message on stream %s (%v)", t, err)
 				cw.close()
 				cw.close()
 				heartbeatc, msgc = nil, nil
 				heartbeatc, msgc = nil, nil
 				cw.r.ReportUnreachable(m.To)
 				cw.r.ReportUnreachable(m.To)
@@ -172,7 +171,7 @@ func (cw *streamWriter) run() {
 				var err error
 				var err error
 				msgAppTerm, err = strconv.ParseUint(conn.termStr, 10, 64)
 				msgAppTerm, err = strconv.ParseUint(conn.termStr, 10, 64)
 				if err != nil {
 				if err != nil {
-					log.Panicf("rafthttp: could not parse term %s to uint (%v)", conn.termStr, err)
+					plog.Panicf("could not parse term %s to uint (%v)", conn.termStr, err)
 				}
 				}
 				enc = &msgAppEncoder{w: conn.Writer, fs: cw.fs}
 				enc = &msgAppEncoder{w: conn.Writer, fs: cw.fs}
 			case streamTypeMsgAppV2:
 			case streamTypeMsgAppV2:
@@ -180,7 +179,7 @@ func (cw *streamWriter) run() {
 			case streamTypeMessage:
 			case streamTypeMessage:
 				enc = &messageEncoder{w: conn.Writer}
 				enc = &messageEncoder{w: conn.Writer}
 			default:
 			default:
-				log.Panicf("rafthttp: unhandled stream type %s", conn.t)
+				plog.Panicf("unhandled stream type %s", conn.t)
 			}
 			}
 			flusher = conn.Flusher
 			flusher = conn.Flusher
 			cw.mu.Lock()
 			cw.mu.Lock()
@@ -280,7 +279,9 @@ func (cr *streamReader) run() {
 		}
 		}
 		if err != nil {
 		if err != nil {
 			if err != errUnsupportedStreamType {
 			if err != errUnsupportedStreamType {
-				log.Printf("rafthttp: failed to dial stream %s (%v)", t, err)
+				// TODO: log start and end of the stream, and print
+				// error in backoff way
+				plog.Errorf("failed to dial stream %s (%v)", t, err)
 			}
 			}
 		} else {
 		} else {
 			err := cr.decodeLoop(rc, t)
 			err := cr.decodeLoop(rc, t)
@@ -293,7 +294,7 @@ func (cr *streamReader) run() {
 			// heartbeat on the idle stream, so it is expected to time out.
 			// heartbeat on the idle stream, so it is expected to time out.
 			case t == streamTypeMsgApp && isNetworkTimeoutError(err):
 			case t == streamTypeMsgApp && isNetworkTimeoutError(err):
 			default:
 			default:
-				log.Printf("rafthttp: failed to read message on stream %s (%v)", t, err)
+				plog.Errorf("failed to read message on stream %s (%v)", t, err)
 			}
 			}
 		}
 		}
 		select {
 		select {
@@ -318,7 +319,7 @@ func (cr *streamReader) decodeLoop(rc io.ReadCloser, t streamType) error {
 	case streamTypeMessage:
 	case streamTypeMessage:
 		dec = &messageDecoder{r: rc}
 		dec = &messageDecoder{r: rc}
 	default:
 	default:
-		log.Panicf("rafthttp: unhandled stream type %s", t)
+		plog.Panicf("unhandled stream type %s", t)
 	}
 	}
 	cr.closer = rc
 	cr.closer = rc
 	cr.mu.Unlock()
 	cr.mu.Unlock()
@@ -341,7 +342,8 @@ func (cr *streamReader) decodeLoop(rc io.ReadCloser, t streamType) error {
 			select {
 			select {
 			case recvc <- m:
 			case recvc <- m:
 			default:
 			default:
-				log.Printf("rafthttp: dropping %s from %x because receiving buffer is full",
+				// TODO: log start and end of message dropping
+				plog.Warningf("dropping %s from %x because receiving buffer is full",
 					m.Type, m.From)
 					m.Type, m.From)
 			}
 			}
 		}
 		}
@@ -442,10 +444,10 @@ func (cr *streamReader) dial(t streamType) (io.ReadCloser, error) {
 
 
 		switch strings.TrimSuffix(string(b), "\n") {
 		switch strings.TrimSuffix(string(b), "\n") {
 		case errIncompatibleVersion.Error():
 		case errIncompatibleVersion.Error():
-			log.Printf("rafthttp: request sent was ignored by peer %s (server version incompatible)", cr.to)
+			plog.Errorf("request sent was ignored by peer %s (server version incompatible)", cr.to)
 			return nil, errIncompatibleVersion
 			return nil, errIncompatibleVersion
 		case errClusterIDMismatch.Error():
 		case errClusterIDMismatch.Error():
-			log.Printf("rafthttp: request sent was ignored (cluster ID mismatch: remote[%s]=%s, local=%s)",
+			plog.Errorf("request sent was ignored (cluster ID mismatch: remote[%s]=%s, local=%s)",
 				cr.to, resp.Header.Get("X-Etcd-Cluster-ID"), cr.cid)
 				cr.to, resp.Header.Get("X-Etcd-Cluster-ID"), cr.cid)
 			return nil, errClusterIDMismatch
 			return nil, errClusterIDMismatch
 		default:
 		default:

+ 8 - 6
rafthttp/transport.go

@@ -15,10 +15,10 @@
 package rafthttp
 package rafthttp
 
 
 import (
 import (
-	"log"
 	"net/http"
 	"net/http"
 	"sync"
 	"sync"
 
 
+	"github.com/coreos/etcd/Godeps/_workspace/src/github.com/coreos/pkg/capnslog"
 	"github.com/coreos/etcd/Godeps/_workspace/src/golang.org/x/net/context"
 	"github.com/coreos/etcd/Godeps/_workspace/src/golang.org/x/net/context"
 	"github.com/coreos/etcd/etcdserver/stats"
 	"github.com/coreos/etcd/etcdserver/stats"
 	"github.com/coreos/etcd/pkg/types"
 	"github.com/coreos/etcd/pkg/types"
@@ -26,6 +26,8 @@ import (
 	"github.com/coreos/etcd/raft/raftpb"
 	"github.com/coreos/etcd/raft/raftpb"
 )
 )
 
 
+var plog = capnslog.NewPackageLogger("github.com/coreos/etcd", "snap")
+
 type Raft interface {
 type Raft interface {
 	Process(ctx context.Context, m raftpb.Message) error
 	Process(ctx context.Context, m raftpb.Message) error
 	IsIDRemoved(id uint64) bool
 	IsIDRemoved(id uint64) bool
@@ -150,7 +152,7 @@ func (t *transport) Send(msgs []raftpb.Message) {
 			continue
 			continue
 		}
 		}
 
 
-		log.Printf("rafthttp: ignored message %s (sent to unknown receiver %s)", m.Type, to)
+		plog.Debugf("ignored message %s (sent to unknown peer %s)", m.Type, to)
 	}
 	}
 }
 }
 
 
@@ -174,7 +176,7 @@ func (t *transport) AddRemote(id types.ID, us []string) {
 	}
 	}
 	urls, err := types.NewURLs(us)
 	urls, err := types.NewURLs(us)
 	if err != nil {
 	if err != nil {
-		log.Panicf("newURLs %+v should never fail: %+v", us, err)
+		plog.Panicf("newURLs %+v should never fail: %+v", us, err)
 	}
 	}
 	t.remotes[id] = startRemote(t.roundTripper, urls, t.id, id, t.clusterID, t.raft, t.errorc)
 	t.remotes[id] = startRemote(t.roundTripper, urls, t.id, id, t.clusterID, t.raft, t.errorc)
 }
 }
@@ -187,7 +189,7 @@ func (t *transport) AddPeer(id types.ID, us []string) {
 	}
 	}
 	urls, err := types.NewURLs(us)
 	urls, err := types.NewURLs(us)
 	if err != nil {
 	if err != nil {
-		log.Panicf("newURLs %+v should never fail: %+v", us, err)
+		plog.Panicf("newURLs %+v should never fail: %+v", us, err)
 	}
 	}
 	fs := t.leaderStats.Follower(id.String())
 	fs := t.leaderStats.Follower(id.String())
 	t.peers[id] = startPeer(t.roundTripper, urls, t.id, id, t.clusterID, t.raft, fs, t.errorc)
 	t.peers[id] = startPeer(t.roundTripper, urls, t.id, id, t.clusterID, t.raft, fs, t.errorc)
@@ -212,7 +214,7 @@ func (t *transport) removePeer(id types.ID) {
 	if peer, ok := t.peers[id]; ok {
 	if peer, ok := t.peers[id]; ok {
 		peer.Stop()
 		peer.Stop()
 	} else {
 	} else {
-		log.Panicf("rafthttp: unexpected removal of unknown peer '%d'", id)
+		plog.Panicf("unexpected removal of unknown peer '%d'", id)
 	}
 	}
 	delete(t.peers, id)
 	delete(t.peers, id)
 	delete(t.leaderStats.Followers, id.String())
 	delete(t.leaderStats.Followers, id.String())
@@ -227,7 +229,7 @@ func (t *transport) UpdatePeer(id types.ID, us []string) {
 	}
 	}
 	urls, err := types.NewURLs(us)
 	urls, err := types.NewURLs(us)
 	if err != nil {
 	if err != nil {
-		log.Panicf("newURLs %+v should never fail: %+v", us, err)
+		plog.Panicf("newURLs %+v should never fail: %+v", us, err)
 	}
 	}
 	t.peers[id].Update(urls)
 	t.peers[id].Update(urls)
 }
 }