rafthttp: simplify streamReader initilization

This commit is contained in:
Xiang Li
2016-05-31 11:54:13 -07:00
parent 310ebdd3e1
commit 86269ab5bf
3 changed files with 69 additions and 52 deletions

View File

@ -244,46 +244,39 @@ func (cw *streamWriter) stop() {
// streamReader is a long-running go-routine that dials to the remote stream
// endpoint and reads messages from the response body returned.
type streamReader struct {
tr *Transport
picker *urlPicker
t streamType
local, remote types.ID
cid types.ID
status *peerStatus
recvc chan<- raftpb.Message
propc chan<- raftpb.Message
errorc chan<- error
typ streamType
tr *Transport
picker *urlPicker
to types.ID
status *peerStatus
recvc chan<- raftpb.Message
propc chan<- raftpb.Message
errorc chan<- error
mu sync.Mutex
paused bool
cancel func()
closer io.Closer
stopc chan struct{}
done chan struct{}
stopc chan struct{}
done chan struct{}
}
func startStreamReader(tr *Transport, picker *urlPicker, t streamType, local, remote, cid types.ID, status *peerStatus, recvc chan<- raftpb.Message, propc chan<- raftpb.Message, errorc chan<- error) *streamReader {
r := &streamReader{
tr: tr,
picker: picker,
t: t,
local: local,
remote: remote,
cid: cid,
status: status,
recvc: recvc,
propc: propc,
errorc: errorc,
stopc: make(chan struct{}),
done: make(chan struct{}),
func (r *streamReader) start() {
r.stopc = make(chan struct{})
r.done = make(chan struct{})
if r.errorc != nil {
r.errorc = r.tr.ErrorC
}
go r.run()
return r
}
func (cr *streamReader) run() {
for {
t := cr.t
t := cr.typ
rc, err := cr.dial(t)
if err != nil {
if err != errUnsupportedStreamType {
@ -317,7 +310,7 @@ func (cr *streamReader) decodeLoop(rc io.ReadCloser, t streamType) error {
cr.mu.Lock()
switch t {
case streamTypeMsgAppV2:
dec = newMsgAppV2Decoder(rc, cr.local, cr.remote)
dec = newMsgAppV2Decoder(rc, cr.tr.ID, cr.to)
case streamTypeMessage:
dec = &messageDecoder{r: rc}
default:
@ -382,18 +375,18 @@ func (cr *streamReader) stop() {
func (cr *streamReader) dial(t streamType) (io.ReadCloser, error) {
u := cr.picker.pick()
uu := u
uu.Path = path.Join(t.endpoint(), cr.local.String())
uu.Path = path.Join(t.endpoint(), cr.tr.ID.String())
req, err := http.NewRequest("GET", uu.String(), nil)
if err != nil {
cr.picker.unreachable(u)
return nil, fmt.Errorf("failed to make http request to %v (%v)", u, err)
}
req.Header.Set("X-Server-From", cr.local.String())
req.Header.Set("X-Server-From", cr.tr.ID.String())
req.Header.Set("X-Server-Version", version.Version)
req.Header.Set("X-Min-Cluster-Version", version.MinClusterVersion)
req.Header.Set("X-Etcd-Cluster-ID", cr.cid.String())
req.Header.Set("X-Raft-To", cr.remote.String())
req.Header.Set("X-Etcd-Cluster-ID", cr.tr.ClusterID.String())
req.Header.Set("X-Raft-To", cr.to.String())
setPeerURLsHeader(req, cr.tr.URLs)
@ -436,7 +429,7 @@ func (cr *streamReader) dial(t streamType) (io.ReadCloser, error) {
case http.StatusNotFound:
httputil.GracefulClose(resp)
cr.picker.unreachable(u)
return nil, fmt.Errorf("remote member %s could not recognize local member", cr.remote)
return nil, fmt.Errorf("remote member %s could not recognize local member", cr.to)
case http.StatusPreconditionFailed:
b, err := ioutil.ReadAll(resp.Body)
if err != nil {
@ -448,11 +441,11 @@ func (cr *streamReader) dial(t streamType) (io.ReadCloser, error) {
switch strings.TrimSuffix(string(b), "\n") {
case errIncompatibleVersion.Error():
plog.Errorf("request sent was ignored by peer %s (server version incompatible)", cr.remote)
plog.Errorf("request sent was ignored by peer %s (server version incompatible)", cr.to)
return nil, errIncompatibleVersion
case errClusterIDMismatch.Error():
plog.Errorf("request sent was ignored (cluster ID mismatch: remote[%s]=%s, local=%s)",
cr.remote, resp.Header.Get("X-Etcd-Cluster-ID"), cr.cid)
cr.to, resp.Header.Get("X-Etcd-Cluster-ID"), cr.tr.ClusterID)
return nil, errClusterIDMismatch
default:
return nil, fmt.Errorf("unhandled error %q when precondition failed", string(b))