rafthttp: pretty print connection error

1. print out the status change of connection with peer
2. only print the first error for repeated ones
release-2.1
Yicheng Qin 2015-06-05 16:34:36 -07:00
parent 97709b202d
commit 1dbe72bb74
7 changed files with 125 additions and 58 deletions

View File

@ -109,12 +109,13 @@ type peer struct {
func startPeer(tr http.RoundTripper, urls types.URLs, local, to, cid types.ID, r Raft, fs *stats.FollowerStats, errorc chan error) *peer { func startPeer(tr http.RoundTripper, urls types.URLs, local, to, cid types.ID, r Raft, fs *stats.FollowerStats, errorc chan error) *peer {
picker := newURLPicker(urls) picker := newURLPicker(urls)
status := newPeerStatus(to)
p := &peer{ p := &peer{
id: to, id: to,
r: r, r: r,
msgAppWriter: startStreamWriter(to, fs, r), msgAppWriter: startStreamWriter(to, status, fs, r),
writer: startStreamWriter(to, fs, r), writer: startStreamWriter(to, status, fs, r),
pipeline: newPipeline(tr, picker, local, to, cid, fs, r, errorc), pipeline: newPipeline(tr, picker, local, to, cid, status, fs, r, errorc),
sendc: make(chan raftpb.Message), sendc: make(chan raftpb.Message),
recvc: make(chan raftpb.Message, recvBufSize), recvc: make(chan raftpb.Message, recvBufSize),
propc: make(chan raftpb.Message, maxPendingProposals), propc: make(chan raftpb.Message, maxPendingProposals),
@ -144,8 +145,8 @@ func startPeer(tr http.RoundTripper, urls types.URLs, local, to, cid types.ID, r
go func() { go func() {
var paused bool var paused bool
p.msgAppReader = startStreamReader(tr, picker, streamTypeMsgAppV2, local, to, cid, p.recvc, p.propc, errorc) p.msgAppReader = startStreamReader(tr, picker, streamTypeMsgAppV2, local, to, cid, status, p.recvc, p.propc, errorc)
reader := startStreamReader(tr, picker, streamTypeMessage, local, to, cid, p.recvc, p.propc, errorc) reader := startStreamReader(tr, picker, streamTypeMessage, local, to, cid, status, p.recvc, p.propc, errorc)
for { for {
select { select {
case m := <-p.sendc: case m := <-p.sendc:

67
rafthttp/peer_status.go Normal file
View File

@ -0,0 +1,67 @@
// Copyright 2015 CoreOS, Inc.
//
// 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 rafthttp
import (
"fmt"
"sync"
"github.com/coreos/etcd/pkg/types"
)
type failureType struct {
source string
action string
}
type peerStatus struct {
id types.ID
mu sync.Mutex // protect active and failureMap
active bool
failureMap map[failureType]string
}
func newPeerStatus(id types.ID) *peerStatus {
return &peerStatus{
id: id,
failureMap: make(map[failureType]string),
}
}
func (s *peerStatus) activate() {
s.mu.Lock()
defer s.mu.Unlock()
if !s.active {
plog.Infof("the connection with %s became active", s.id)
s.active = true
s.failureMap = make(map[failureType]string)
}
}
func (s *peerStatus) deactivate(failure failureType, reason string) {
s.mu.Lock()
defer s.mu.Unlock()
if s.active {
plog.Infof("the connection with %s became inactive", s.id)
s.active = false
}
logline := fmt.Sprintf("failed to %s %s on %s (%s)", failure.action, s.id, failure.source, reason)
if r, ok := s.failureMap[failure]; ok && r == reason {
plog.Debugf(logline)
return
}
s.failureMap[failure] = reason
plog.Errorf(logline)
}

View File

@ -53,6 +53,7 @@ type pipeline struct {
tr http.RoundTripper tr http.RoundTripper
picker *urlPicker picker *urlPicker
status *peerStatus
fs *stats.FollowerStats fs *stats.FollowerStats
r Raft r Raft
errorc chan error errorc chan error
@ -61,26 +62,21 @@ type pipeline struct {
// wait for the handling routines // wait for the handling routines
wg sync.WaitGroup wg sync.WaitGroup
stopc chan struct{} stopc chan struct{}
sync.Mutex
// if the last send was successful, the pipeline is active.
// Or it is inactive
active bool
errored error
} }
func newPipeline(tr http.RoundTripper, picker *urlPicker, from, to, cid types.ID, fs *stats.FollowerStats, r Raft, errorc chan error) *pipeline { func newPipeline(tr http.RoundTripper, picker *urlPicker, from, to, cid types.ID, status *peerStatus, fs *stats.FollowerStats, r Raft, errorc chan error) *pipeline {
p := &pipeline{ p := &pipeline{
from: from, from: from,
to: to, to: to,
cid: cid, cid: cid,
tr: tr, tr: tr,
picker: picker, picker: picker,
status: status,
fs: fs, fs: fs,
r: r, r: r,
errorc: errorc, errorc: errorc,
stopc: make(chan struct{}), stopc: make(chan struct{}),
msgc: make(chan raftpb.Message, pipelineBufSize), msgc: make(chan raftpb.Message, pipelineBufSize),
active: true,
} }
p.wg.Add(connPerPipeline) p.wg.Add(connPerPipeline)
for i := 0; i < connPerPipeline; i++ { for i := 0; i < connPerPipeline; i++ {
@ -105,18 +101,9 @@ func (p *pipeline) handle() {
} }
end := time.Now() end := time.Now()
p.Lock()
if err != nil { if err != nil {
reportSentFailure(pipelineMsg, m) reportSentFailure(pipelineMsg, m)
p.status.deactivate(failureType{source: pipelineMsg, action: "write"}, err.Error())
if p.errored == nil || p.errored.Error() != err.Error() {
plog.Errorf("failed to post to %s (%v)", p.to, err)
p.errored = err
}
if p.active {
plog.Infof("the connection with %s became inactive", p.to)
p.active = false
}
if m.Type == raftpb.MsgApp && p.fs != nil { if m.Type == raftpb.MsgApp && p.fs != nil {
p.fs.Fail() p.fs.Fail()
} }
@ -125,11 +112,7 @@ func (p *pipeline) handle() {
p.r.ReportSnapshot(m.To, raft.SnapshotFailure) p.r.ReportSnapshot(m.To, raft.SnapshotFailure)
} }
} else { } else {
if !p.active { p.status.activate()
plog.Infof("the connection with %s became active", p.to)
p.active = true
p.errored = nil
}
if m.Type == raftpb.MsgApp && p.fs != nil { if m.Type == raftpb.MsgApp && p.fs != nil {
p.fs.Succ(end.Sub(start)) p.fs.Succ(end.Sub(start))
} }
@ -138,7 +121,6 @@ func (p *pipeline) handle() {
} }
reportSentDuration(pipelineMsg, m, time.Since(start)) reportSentDuration(pipelineMsg, m, time.Since(start))
} }
p.Unlock()
} }
} }

View File

@ -36,7 +36,7 @@ func TestPipelineSend(t *testing.T) {
tr := &roundTripperRecorder{} tr := &roundTripperRecorder{}
picker := mustNewURLPicker(t, []string{"http://localhost:2380"}) picker := mustNewURLPicker(t, []string{"http://localhost:2380"})
fs := &stats.FollowerStats{} fs := &stats.FollowerStats{}
p := newPipeline(tr, picker, types.ID(2), types.ID(1), types.ID(1), fs, &fakeRaft{}, nil) p := newPipeline(tr, picker, types.ID(2), types.ID(1), types.ID(1), newPeerStatus(types.ID(1)), fs, &fakeRaft{}, nil)
p.msgc <- raftpb.Message{Type: raftpb.MsgApp} p.msgc <- raftpb.Message{Type: raftpb.MsgApp}
testutil.ForceGosched() testutil.ForceGosched()
@ -56,7 +56,7 @@ func TestPipelineExceedMaximalServing(t *testing.T) {
tr := newRoundTripperBlocker() tr := newRoundTripperBlocker()
picker := mustNewURLPicker(t, []string{"http://localhost:2380"}) picker := mustNewURLPicker(t, []string{"http://localhost:2380"})
fs := &stats.FollowerStats{} fs := &stats.FollowerStats{}
p := newPipeline(tr, picker, types.ID(2), types.ID(1), types.ID(1), fs, &fakeRaft{}, nil) p := newPipeline(tr, picker, types.ID(2), types.ID(1), types.ID(1), newPeerStatus(types.ID(1)), fs, &fakeRaft{}, nil)
// keep the sender busy and make the buffer full // keep the sender busy and make the buffer full
// nothing can go out as we block the sender // nothing can go out as we block the sender
@ -96,7 +96,7 @@ func TestPipelineExceedMaximalServing(t *testing.T) {
func TestPipelineSendFailed(t *testing.T) { func TestPipelineSendFailed(t *testing.T) {
picker := mustNewURLPicker(t, []string{"http://localhost:2380"}) picker := mustNewURLPicker(t, []string{"http://localhost:2380"})
fs := &stats.FollowerStats{} fs := &stats.FollowerStats{}
p := newPipeline(newRespRoundTripper(0, errors.New("blah")), picker, types.ID(2), types.ID(1), types.ID(1), fs, &fakeRaft{}, nil) p := newPipeline(newRespRoundTripper(0, errors.New("blah")), picker, types.ID(2), types.ID(1), types.ID(1), newPeerStatus(types.ID(1)), fs, &fakeRaft{}, nil)
p.msgc <- raftpb.Message{Type: raftpb.MsgApp} p.msgc <- raftpb.Message{Type: raftpb.MsgApp}
testutil.ForceGosched() testutil.ForceGosched()
@ -112,7 +112,7 @@ func TestPipelineSendFailed(t *testing.T) {
func TestPipelinePost(t *testing.T) { func TestPipelinePost(t *testing.T) {
tr := &roundTripperRecorder{} tr := &roundTripperRecorder{}
picker := mustNewURLPicker(t, []string{"http://localhost:2380"}) picker := mustNewURLPicker(t, []string{"http://localhost:2380"})
p := newPipeline(tr, picker, types.ID(2), types.ID(1), types.ID(1), nil, &fakeRaft{}, nil) p := newPipeline(tr, picker, types.ID(2), types.ID(1), types.ID(1), newPeerStatus(types.ID(1)), nil, &fakeRaft{}, nil)
if err := p.post([]byte("some data")); err != nil { if err := p.post([]byte("some data")); err != nil {
t.Fatalf("unexpect post error: %v", err) t.Fatalf("unexpect post error: %v", err)
} }
@ -159,7 +159,7 @@ func TestPipelinePostBad(t *testing.T) {
} }
for i, tt := range tests { for i, tt := range tests {
picker := mustNewURLPicker(t, []string{tt.u}) picker := mustNewURLPicker(t, []string{tt.u})
p := newPipeline(newRespRoundTripper(tt.code, tt.err), picker, types.ID(2), types.ID(1), types.ID(1), nil, &fakeRaft{}, make(chan error)) p := newPipeline(newRespRoundTripper(tt.code, tt.err), picker, types.ID(2), types.ID(1), types.ID(1), newPeerStatus(types.ID(1)), nil, &fakeRaft{}, make(chan error))
err := p.post([]byte("some data")) err := p.post([]byte("some data"))
p.stop() p.stop()
@ -180,7 +180,7 @@ func TestPipelinePostErrorc(t *testing.T) {
for i, tt := range tests { for i, tt := range tests {
picker := mustNewURLPicker(t, []string{tt.u}) picker := mustNewURLPicker(t, []string{tt.u})
errorc := make(chan error, 1) errorc := make(chan error, 1)
p := newPipeline(newRespRoundTripper(tt.code, tt.err), picker, types.ID(2), types.ID(1), types.ID(1), nil, &fakeRaft{}, errorc) p := newPipeline(newRespRoundTripper(tt.code, tt.err), picker, types.ID(2), types.ID(1), types.ID(1), newPeerStatus(types.ID(1)), nil, &fakeRaft{}, errorc)
p.post([]byte("some data")) p.post([]byte("some data"))
p.stop() p.stop()
select { select {
@ -193,7 +193,7 @@ func TestPipelinePostErrorc(t *testing.T) {
func TestStopBlockedPipeline(t *testing.T) { func TestStopBlockedPipeline(t *testing.T) {
picker := mustNewURLPicker(t, []string{"http://localhost:2380"}) picker := mustNewURLPicker(t, []string{"http://localhost:2380"})
p := newPipeline(newRoundTripperBlocker(), picker, types.ID(2), types.ID(1), types.ID(1), nil, &fakeRaft{}, nil) p := newPipeline(newRoundTripperBlocker(), picker, types.ID(2), types.ID(1), types.ID(1), newPeerStatus(types.ID(1)), nil, &fakeRaft{}, nil)
// send many messages that most of them will be blocked in buffer // send many messages that most of them will be blocked in buffer
for i := 0; i < connPerPipeline*10; i++ { for i := 0; i < connPerPipeline*10; i++ {
p.msgc <- raftpb.Message{} p.msgc <- raftpb.Message{}

View File

@ -30,7 +30,7 @@ func startRemote(tr http.RoundTripper, urls types.URLs, local, to, cid types.ID,
picker := newURLPicker(urls) picker := newURLPicker(urls)
return &remote{ return &remote{
id: to, id: to,
pipeline: newPipeline(tr, picker, local, to, cid, nil, r, errorc), pipeline: newPipeline(tr, picker, local, to, cid, newPeerStatus(to), nil, r, errorc),
} }
} }

View File

@ -67,6 +67,19 @@ func (t streamType) endpoint() string {
} }
} }
func (t streamType) String() string {
switch t {
case streamTypeMsgApp:
return "stream MsgApp"
case streamTypeMsgAppV2:
return "stream MsgApp v2"
case streamTypeMessage:
return "stream Message"
default:
return "unknown stream"
}
}
var ( var (
// linkHeartbeatMessage is a special message used as heartbeat message in // linkHeartbeatMessage is a special message used as heartbeat message in
// link layer. It never conflicts with messages from raft because raft // link layer. It never conflicts with messages from raft because raft
@ -89,9 +102,10 @@ type outgoingConn struct {
// streamWriter is a long-running go-routine that writes messages into the // streamWriter is a long-running go-routine that writes messages into the
// attached outgoingConn. // attached outgoingConn.
type streamWriter struct { type streamWriter struct {
id types.ID id types.ID
fs *stats.FollowerStats status *peerStatus
r Raft fs *stats.FollowerStats
r Raft
mu sync.Mutex // guard field working and closer mu sync.Mutex // guard field working and closer
closer io.Closer closer io.Closer
@ -103,15 +117,16 @@ type streamWriter struct {
done chan struct{} done chan struct{}
} }
func startStreamWriter(id types.ID, fs *stats.FollowerStats, r Raft) *streamWriter { func startStreamWriter(id types.ID, status *peerStatus, fs *stats.FollowerStats, r Raft) *streamWriter {
w := &streamWriter{ w := &streamWriter{
id: id, id: id,
fs: fs, status: status,
r: r, fs: fs,
msgc: make(chan raftpb.Message, streamBufSize), r: r,
connc: make(chan *outgoingConn), msgc: make(chan raftpb.Message, streamBufSize),
stopc: make(chan struct{}), connc: make(chan *outgoingConn),
done: make(chan struct{}), stopc: make(chan struct{}),
done: make(chan struct{}),
} }
go w.run() go w.run()
return w return w
@ -133,7 +148,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)
plog.Errorf("failed to heartbeat on stream %s (%v)", t, err) cw.status.deactivate(failureType{source: t.String(), action: "heartbeat"}, err.Error())
cw.close() cw.close()
heartbeatc, msgc = nil, nil heartbeatc, msgc = nil, nil
continue continue
@ -155,7 +170,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)
plog.Errorf("failed to send message on stream %s (%v)", t, err) cw.status.deactivate(failureType{source: t.String(), action: "write"}, err.Error())
cw.close() cw.close()
heartbeatc, msgc = nil, nil heartbeatc, msgc = nil, nil
cw.r.ReportUnreachable(m.To) cw.r.ReportUnreachable(m.To)
@ -183,6 +198,7 @@ func (cw *streamWriter) run() {
} }
flusher = conn.Flusher flusher = conn.Flusher
cw.mu.Lock() cw.mu.Lock()
cw.status.activate()
cw.closer = conn.Closer cw.closer = conn.Closer
cw.working = true cw.working = true
cw.mu.Unlock() cw.mu.Unlock()
@ -237,6 +253,7 @@ type streamReader struct {
t streamType t streamType
from, to types.ID from, to types.ID
cid types.ID cid types.ID
status *peerStatus
recvc chan<- raftpb.Message recvc chan<- raftpb.Message
propc chan<- raftpb.Message propc chan<- raftpb.Message
errorc chan<- error errorc chan<- error
@ -249,7 +266,7 @@ type streamReader struct {
done chan struct{} done chan struct{}
} }
func startStreamReader(tr http.RoundTripper, picker *urlPicker, t streamType, from, to, cid types.ID, recvc chan<- raftpb.Message, propc chan<- raftpb.Message, errorc chan<- error) *streamReader { func startStreamReader(tr http.RoundTripper, picker *urlPicker, t streamType, from, to, cid types.ID, status *peerStatus, recvc chan<- raftpb.Message, propc chan<- raftpb.Message, errorc chan<- error) *streamReader {
r := &streamReader{ r := &streamReader{
tr: tr, tr: tr,
picker: picker, picker: picker,
@ -257,6 +274,7 @@ func startStreamReader(tr http.RoundTripper, picker *urlPicker, t streamType, fr
from: from, from: from,
to: to, to: to,
cid: cid, cid: cid,
status: status,
recvc: recvc, recvc: recvc,
propc: propc, propc: propc,
errorc: errorc, errorc: errorc,
@ -279,11 +297,10 @@ func (cr *streamReader) run() {
} }
if err != nil { if err != nil {
if err != errUnsupportedStreamType { if err != errUnsupportedStreamType {
// TODO: log start and end of the stream, and print cr.status.deactivate(failureType{source: t.String(), action: "dial"}, err.Error())
// error in backoff way
plog.Errorf("failed to dial stream %s (%v)", t, err)
} }
} else { } else {
cr.status.activate()
err := cr.decodeLoop(rc, t) err := cr.decodeLoop(rc, t)
switch { switch {
// all data is read out // all data is read out
@ -294,7 +311,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:
plog.Errorf("failed to read message on stream %s (%v)", t, err) cr.status.deactivate(failureType{source: t.String(), action: "read"}, err.Error())
} }
} }
select { select {

View File

@ -21,7 +21,7 @@ import (
// to streamWriter. After that, streamWriter can use it to send messages // to streamWriter. After that, streamWriter can use it to send messages
// continuously, and closes it when stopped. // continuously, and closes it when stopped.
func TestStreamWriterAttachOutgoingConn(t *testing.T) { func TestStreamWriterAttachOutgoingConn(t *testing.T) {
sw := startStreamWriter(types.ID(1), &stats.FollowerStats{}, &fakeRaft{}) sw := startStreamWriter(types.ID(1), newPeerStatus(types.ID(1)), &stats.FollowerStats{}, &fakeRaft{})
// the expected initial state of streamWrite is not working // the expected initial state of streamWrite is not working
if _, ok := sw.writec(); ok != false { if _, ok := sw.writec(); ok != false {
t.Errorf("initial working status = %v, want false", ok) t.Errorf("initial working status = %v, want false", ok)
@ -67,7 +67,7 @@ func TestStreamWriterAttachOutgoingConn(t *testing.T) {
// TestStreamWriterAttachBadOutgoingConn tests that streamWriter with bad // TestStreamWriterAttachBadOutgoingConn tests that streamWriter with bad
// outgoingConn will close the outgoingConn and fall back to non-working status. // outgoingConn will close the outgoingConn and fall back to non-working status.
func TestStreamWriterAttachBadOutgoingConn(t *testing.T) { func TestStreamWriterAttachBadOutgoingConn(t *testing.T) {
sw := startStreamWriter(types.ID(1), &stats.FollowerStats{}, &fakeRaft{}) sw := startStreamWriter(types.ID(1), newPeerStatus(types.ID(1)), &stats.FollowerStats{}, &fakeRaft{})
defer sw.stop() defer sw.stop()
wfc := &fakeWriteFlushCloser{err: errors.New("blah")} wfc := &fakeWriteFlushCloser{err: errors.New("blah")}
sw.attach(&outgoingConn{t: streamTypeMessage, Writer: wfc, Flusher: wfc, Closer: wfc}) sw.attach(&outgoingConn{t: streamTypeMessage, Writer: wfc, Flusher: wfc, Closer: wfc})
@ -269,12 +269,12 @@ func TestStream(t *testing.T) {
srv := httptest.NewServer(h) srv := httptest.NewServer(h)
defer srv.Close() defer srv.Close()
sw := startStreamWriter(types.ID(1), &stats.FollowerStats{}, &fakeRaft{}) sw := startStreamWriter(types.ID(1), newPeerStatus(types.ID(1)), &stats.FollowerStats{}, &fakeRaft{})
defer sw.stop() defer sw.stop()
h.sw = sw h.sw = sw
picker := mustNewURLPicker(t, []string{srv.URL}) picker := mustNewURLPicker(t, []string{srv.URL})
sr := startStreamReader(&http.Transport{}, picker, tt.t, types.ID(1), types.ID(2), types.ID(1), recvc, propc, nil) sr := startStreamReader(&http.Transport{}, picker, tt.t, types.ID(1), types.ID(2), types.ID(1), newPeerStatus(types.ID(1)), recvc, propc, nil)
defer sr.stop() defer sr.stop()
if tt.t == streamTypeMsgApp { if tt.t == streamTypeMsgApp {
sr.updateMsgAppTerm(tt.term) sr.updateMsgAppTerm(tt.term)