Merge pull request #3664 from yichengq/transport-more

rafthttp: build transport inside pkg instead of passed-in
This commit is contained in:
Xiang Li 2015-10-11 22:00:05 -07:00
commit df7074911e
10 changed files with 117 additions and 97 deletions

View File

@ -199,11 +199,6 @@ func startEtcd(cfg *config) (<-chan struct{}, error) {
return nil, fmt.Errorf("error setting up initial cluster: %v", err) return nil, fmt.Errorf("error setting up initial cluster: %v", err)
} }
pt, err := transport.NewTimeoutTransport(cfg.peerTLSInfo, peerDialTimeout(cfg.ElectionMs), rafthttp.ConnReadTimeout, rafthttp.ConnWriteTimeout)
if err != nil {
return nil, err
}
if !cfg.peerTLSInfo.Empty() { if !cfg.peerTLSInfo.Empty() {
plog.Infof("peerTLS: %s", cfg.peerTLSInfo) plog.Infof("peerTLS: %s", cfg.peerTLSInfo)
} }
@ -284,7 +279,7 @@ func startEtcd(cfg *config) (<-chan struct{}, error) {
DiscoveryProxy: cfg.dproxy, DiscoveryProxy: cfg.dproxy,
NewCluster: cfg.isNewCluster(), NewCluster: cfg.isNewCluster(),
ForceNewCluster: cfg.forceNewCluster, ForceNewCluster: cfg.forceNewCluster,
Transport: pt, PeerTLSInfo: cfg.peerTLSInfo,
TickMs: cfg.TickMs, TickMs: cfg.TickMs,
ElectionTicks: cfg.electionTicks(), ElectionTicks: cfg.electionTicks(),
V3demo: cfg.v3demo, V3demo: cfg.v3demo,
@ -534,9 +529,3 @@ func setupLogging(cfg *config) {
repoLog.SetLogLevel(settings) repoLog.SetLogLevel(settings)
} }
} }
func peerDialTimeout(electionMs uint) time.Duration {
// 1s for queue wait and system delay
// + one RTT, which is smaller than 1/5 election timeout
return time.Second + time.Duration(electionMs)*time.Millisecond/5
}

View File

@ -16,13 +16,13 @@ package etcdserver
import ( import (
"fmt" "fmt"
"net/http"
"path" "path"
"sort" "sort"
"strings" "strings"
"time" "time"
"github.com/coreos/etcd/pkg/netutil" "github.com/coreos/etcd/pkg/netutil"
"github.com/coreos/etcd/pkg/transport"
"github.com/coreos/etcd/pkg/types" "github.com/coreos/etcd/pkg/types"
) )
@ -44,7 +44,7 @@ type ServerConfig struct {
InitialClusterToken string InitialClusterToken string
NewCluster bool NewCluster bool
ForceNewCluster bool ForceNewCluster bool
Transport *http.Transport PeerTLSInfo transport.TLSInfo
TickMs uint TickMs uint
ElectionTicks int ElectionTicks int
@ -132,6 +132,12 @@ func (c *ServerConfig) ReqTimeout() time.Duration {
return 5*time.Second + 2*time.Duration(c.ElectionTicks)*time.Duration(c.TickMs)*time.Millisecond return 5*time.Second + 2*time.Duration(c.ElectionTicks)*time.Duration(c.TickMs)*time.Millisecond
} }
func (c *ServerConfig) peerDialTimeout() time.Duration {
// 1s for queue wait and system delay
// + one RTT, which is smaller than 1/5 election timeout
return time.Second + time.Duration(c.ElectionTicks)*time.Duration(c.TickMs)*time.Millisecond/5
}
func (c *ServerConfig) PrintWithInitial() { c.print(true) } func (c *ServerConfig) PrintWithInitial() { c.print(true) }
func (c *ServerConfig) Print() { c.print(false) } func (c *ServerConfig) Print() { c.print(false) }

View File

@ -40,6 +40,7 @@ import (
"github.com/coreos/etcd/pkg/pbutil" "github.com/coreos/etcd/pkg/pbutil"
"github.com/coreos/etcd/pkg/runtime" "github.com/coreos/etcd/pkg/runtime"
"github.com/coreos/etcd/pkg/timeutil" "github.com/coreos/etcd/pkg/timeutil"
"github.com/coreos/etcd/pkg/transport"
"github.com/coreos/etcd/pkg/types" "github.com/coreos/etcd/pkg/types"
"github.com/coreos/etcd/pkg/wait" "github.com/coreos/etcd/pkg/wait"
"github.com/coreos/etcd/raft" "github.com/coreos/etcd/raft"
@ -167,7 +168,9 @@ type EtcdServer struct {
SyncTicker <-chan time.Time SyncTicker <-chan time.Time
reqIDGen *idutil.Generator // versionTr used to send requests for peer version
versionTr *http.Transport
reqIDGen *idutil.Generator
// forceVersionC is used to force the version monitor loop // forceVersionC is used to force the version monitor loop
// to detect the cluster version immediately. // to detect the cluster version immediately.
@ -205,6 +208,10 @@ func NewServer(cfg *ServerConfig) (*EtcdServer, error) {
haveWAL := wal.Exist(cfg.WALDir()) haveWAL := wal.Exist(cfg.WALDir())
ss := snap.New(cfg.SnapDir()) ss := snap.New(cfg.SnapDir())
pt, err := transport.NewTransport(cfg.PeerTLSInfo, cfg.peerDialTimeout())
if err != nil {
return nil, err
}
var remotes []*Member var remotes []*Member
switch { switch {
case !haveWAL && !cfg.NewCluster: case !haveWAL && !cfg.NewCluster:
@ -215,14 +222,14 @@ func NewServer(cfg *ServerConfig) (*EtcdServer, error) {
if err != nil { if err != nil {
return nil, err return nil, err
} }
existingCluster, err := GetClusterFromRemotePeers(getRemotePeerURLs(cl, cfg.Name), cfg.Transport) existingCluster, err := GetClusterFromRemotePeers(getRemotePeerURLs(cl, cfg.Name), pt)
if err != nil { if err != nil {
return nil, fmt.Errorf("cannot fetch cluster info from peer urls: %v", err) return nil, fmt.Errorf("cannot fetch cluster info from peer urls: %v", err)
} }
if err := ValidateClusterAndAssignIDs(cl, existingCluster); err != nil { if err := ValidateClusterAndAssignIDs(cl, existingCluster); err != nil {
return nil, fmt.Errorf("error validating peerURLs %s: %v", existingCluster, err) return nil, fmt.Errorf("error validating peerURLs %s: %v", existingCluster, err)
} }
if !isCompatibleWithCluster(cl, cl.MemberByName(cfg.Name).ID, cfg.Transport) { if !isCompatibleWithCluster(cl, cl.MemberByName(cfg.Name).ID, pt) {
return nil, fmt.Errorf("incomptible with current running cluster") return nil, fmt.Errorf("incomptible with current running cluster")
} }
@ -240,7 +247,7 @@ func NewServer(cfg *ServerConfig) (*EtcdServer, error) {
return nil, err return nil, err
} }
m := cl.MemberByName(cfg.Name) m := cl.MemberByName(cfg.Name)
if isMemberBootstrapped(cl, cfg.Name, cfg.Transport) { if isMemberBootstrapped(cl, cfg.Name, pt) {
return nil, fmt.Errorf("member %s has already been bootstrapped", m.ID) return nil, fmt.Errorf("member %s has already been bootstrapped", m.ID)
} }
if cfg.ShouldDiscover() { if cfg.ShouldDiscover() {
@ -328,6 +335,7 @@ func NewServer(cfg *ServerConfig) (*EtcdServer, error) {
stats: sstats, stats: sstats,
lstats: lstats, lstats: lstats,
SyncTicker: time.Tick(500 * time.Millisecond), SyncTicker: time.Tick(500 * time.Millisecond),
versionTr: pt,
reqIDGen: idutil.NewGenerator(uint8(id), time.Now()), reqIDGen: idutil.NewGenerator(uint8(id), time.Now()),
forceVersionC: make(chan struct{}), forceVersionC: make(chan struct{}),
} }
@ -346,15 +354,18 @@ func NewServer(cfg *ServerConfig) (*EtcdServer, error) {
// TODO: move transport initialization near the definition of remote // TODO: move transport initialization near the definition of remote
tr := &rafthttp.Transport{ tr := &rafthttp.Transport{
RoundTripper: cfg.Transport, TLSInfo: cfg.PeerTLSInfo,
ID: id, DialTimeout: cfg.peerDialTimeout(),
ClusterID: cl.ID(), ID: id,
Raft: srv, ClusterID: cl.ID(),
ServerStats: sstats, Raft: srv,
LeaderStats: lstats, ServerStats: sstats,
ErrorC: srv.errorc, LeaderStats: lstats,
ErrorC: srv.errorc,
}
if err := tr.Start(); err != nil {
return nil, err
} }
tr.Start()
// add all remotes into transport // add all remotes into transport
for _, m := range remotes { for _, m := range remotes {
if m.ID != id { if m.ID != id {
@ -1032,7 +1043,7 @@ func (s *EtcdServer) monitorVersions() {
continue continue
} }
v := decideClusterVersion(getVersions(s.cluster, s.id, s.cfg.Transport)) v := decideClusterVersion(getVersions(s.cluster, s.id, s.versionTr))
if v != nil { if v != nil {
// only keep major.minor version for comparasion // only keep major.minor version for comparasion
v = &semver.Version{ v = &semver.Version{

View File

@ -1468,7 +1468,7 @@ func (n *readyNode) Ready() <-chan raft.Ready { return n.readyc }
type nopTransporter struct{} type nopTransporter struct{}
func (s *nopTransporter) Start() {} func (s *nopTransporter) Start() error { return nil }
func (s *nopTransporter) Handler() http.Handler { return nil } func (s *nopTransporter) Handler() http.Handler { return nil }
func (s *nopTransporter) Send(m []raftpb.Message) {} func (s *nopTransporter) Send(m []raftpb.Message) {}
func (s *nopTransporter) AddRemote(id types.ID, us []string) {} func (s *nopTransporter) AddRemote(id types.ID, us []string) {}

View File

@ -688,7 +688,7 @@ func mustNewMember(t *testing.T, name string, usePeerTLS bool) *member {
} }
m.InitialClusterToken = clusterName m.InitialClusterToken = clusterName
m.NewCluster = true m.NewCluster = true
m.Transport = mustNewTransport(t, m.PeerTLSInfo) m.ServerConfig.PeerTLSInfo = m.PeerTLSInfo
m.ElectionTicks = electionTicks m.ElectionTicks = electionTicks
m.TickMs = uint(tickDuration / time.Millisecond) m.TickMs = uint(tickDuration / time.Millisecond)
return m return m
@ -720,7 +720,6 @@ func (m *member) Clone(t *testing.T) *member {
panic(err) panic(err)
} }
mm.InitialClusterToken = m.InitialClusterToken mm.InitialClusterToken = m.InitialClusterToken
mm.Transport = mustNewTransport(t, m.PeerTLSInfo)
mm.ElectionTicks = m.ElectionTicks mm.ElectionTicks = m.ElectionTicks
mm.PeerTLSInfo = m.PeerTLSInfo mm.PeerTLSInfo = m.PeerTLSInfo
return mm return mm

View File

@ -15,7 +15,6 @@
package rafthttp package rafthttp
import ( import (
"net/http"
"net/http/httptest" "net/http/httptest"
"reflect" "reflect"
"testing" "testing"
@ -31,12 +30,11 @@ import (
func TestSendMessage(t *testing.T) { func TestSendMessage(t *testing.T) {
// member 1 // member 1
tr := &Transport{ tr := &Transport{
RoundTripper: &http.Transport{}, ID: types.ID(1),
ID: types.ID(1), ClusterID: types.ID(1),
ClusterID: types.ID(1), Raft: &fakeRaft{},
Raft: &fakeRaft{}, ServerStats: newServerStats(),
ServerStats: newServerStats(), LeaderStats: stats.NewLeaderStats("1"),
LeaderStats: stats.NewLeaderStats("1"),
} }
tr.Start() tr.Start()
srv := httptest.NewServer(tr.Handler()) srv := httptest.NewServer(tr.Handler())
@ -46,12 +44,11 @@ func TestSendMessage(t *testing.T) {
recvc := make(chan raftpb.Message, 1) recvc := make(chan raftpb.Message, 1)
p := &fakeRaft{recvc: recvc} p := &fakeRaft{recvc: recvc}
tr2 := &Transport{ tr2 := &Transport{
RoundTripper: &http.Transport{}, ID: types.ID(2),
ID: types.ID(2), ClusterID: types.ID(1),
ClusterID: types.ID(1), Raft: p,
Raft: p, ServerStats: newServerStats(),
ServerStats: newServerStats(), LeaderStats: stats.NewLeaderStats("2"),
LeaderStats: stats.NewLeaderStats("2"),
} }
tr2.Start() tr2.Start()
srv2 := httptest.NewServer(tr2.Handler()) srv2 := httptest.NewServer(tr2.Handler())
@ -92,12 +89,11 @@ func TestSendMessage(t *testing.T) {
func TestSendMessageWhenStreamIsBroken(t *testing.T) { func TestSendMessageWhenStreamIsBroken(t *testing.T) {
// member 1 // member 1
tr := &Transport{ tr := &Transport{
RoundTripper: &http.Transport{}, ID: types.ID(1),
ID: types.ID(1), ClusterID: types.ID(1),
ClusterID: types.ID(1), Raft: &fakeRaft{},
Raft: &fakeRaft{}, ServerStats: newServerStats(),
ServerStats: newServerStats(), LeaderStats: stats.NewLeaderStats("1"),
LeaderStats: stats.NewLeaderStats("1"),
} }
tr.Start() tr.Start()
srv := httptest.NewServer(tr.Handler()) srv := httptest.NewServer(tr.Handler())
@ -107,12 +103,11 @@ func TestSendMessageWhenStreamIsBroken(t *testing.T) {
recvc := make(chan raftpb.Message, 1) recvc := make(chan raftpb.Message, 1)
p := &fakeRaft{recvc: recvc} p := &fakeRaft{recvc: recvc}
tr2 := &Transport{ tr2 := &Transport{
RoundTripper: &http.Transport{}, ID: types.ID(2),
ID: types.ID(2), ClusterID: types.ID(1),
ClusterID: types.ID(1), Raft: p,
Raft: p, ServerStats: newServerStats(),
ServerStats: newServerStats(), LeaderStats: stats.NewLeaderStats("2"),
LeaderStats: stats.NewLeaderStats("2"),
} }
tr2.Start() tr2.Start()
srv2 := httptest.NewServer(tr2.Handler()) srv2 := httptest.NewServer(tr2.Handler())

View File

@ -111,7 +111,7 @@ type peer struct {
done chan struct{} done chan struct{}
} }
func startPeer(tr http.RoundTripper, urls types.URLs, local, to, cid types.ID, r Raft, fs *stats.FollowerStats, errorc chan error, term uint64) *peer { func startPeer(streamRt, pipelineRt http.RoundTripper, urls types.URLs, local, to, cid types.ID, r Raft, fs *stats.FollowerStats, errorc chan error, term uint64) *peer {
picker := newURLPicker(urls) picker := newURLPicker(urls)
status := newPeerStatus(to) status := newPeerStatus(to)
p := &peer{ p := &peer{
@ -120,7 +120,7 @@ func startPeer(tr http.RoundTripper, urls types.URLs, local, to, cid types.ID, r
status: status, status: status,
msgAppWriter: startStreamWriter(to, status, fs, r), msgAppWriter: startStreamWriter(to, status, fs, r),
writer: startStreamWriter(to, status, fs, r), writer: startStreamWriter(to, status, fs, r),
pipeline: newPipeline(tr, picker, local, to, cid, status, fs, r, errorc), pipeline: newPipeline(pipelineRt, 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),
@ -148,8 +148,8 @@ func startPeer(tr http.RoundTripper, urls types.URLs, local, to, cid types.ID, r
} }
}() }()
p.msgAppReader = startStreamReader(tr, picker, streamTypeMsgAppV2, local, to, cid, status, p.recvc, p.propc, errorc, term) p.msgAppReader = startStreamReader(streamRt, picker, streamTypeMsgAppV2, local, to, cid, status, p.recvc, p.propc, errorc, term)
reader := startStreamReader(tr, picker, streamTypeMessage, local, to, cid, status, p.recvc, p.propc, errorc, term) reader := startStreamReader(streamRt, picker, streamTypeMessage, local, to, cid, status, p.recvc, p.propc, errorc, term)
go func() { go func() {
var paused bool var paused bool
for { for {

View File

@ -23,6 +23,7 @@ import (
"github.com/coreos/etcd/Godeps/_workspace/src/github.com/xiang90/probing" "github.com/coreos/etcd/Godeps/_workspace/src/github.com/xiang90/probing"
"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/transport"
"github.com/coreos/etcd/pkg/types" "github.com/coreos/etcd/pkg/types"
"github.com/coreos/etcd/raft" "github.com/coreos/etcd/raft"
"github.com/coreos/etcd/raft/raftpb" "github.com/coreos/etcd/raft/raftpb"
@ -40,7 +41,7 @@ type Raft interface {
type Transporter interface { type Transporter interface {
// Start starts the given Transporter. // Start starts the given Transporter.
// Start MUST be called before calling other functions in the interface. // Start MUST be called before calling other functions in the interface.
Start() Start() error
// Handler returns the HTTP handler of the transporter. // Handler returns the HTTP handler of the transporter.
// A transporter HTTP handler handles the HTTP requests // A transporter HTTP handler handles the HTTP requests
// from remote peers. // from remote peers.
@ -88,11 +89,13 @@ type Transporter interface {
// User needs to call Start before calling other functions, and call // User needs to call Start before calling other functions, and call
// Stop when the Transport is no longer used. // Stop when the Transport is no longer used.
type Transport struct { type Transport struct {
RoundTripper http.RoundTripper // roundTripper to send requests DialTimeout time.Duration // maximum duration before timing out dial of the request
ID types.ID // local member ID TLSInfo transport.TLSInfo // TLS information used when creating connection
ClusterID types.ID // raft cluster ID for request validation
Raft Raft // raft state machine, to which the Transport forwards received messages and reports status ID types.ID // local member ID
ServerStats *stats.ServerStats // used to record general transportation statistics ClusterID types.ID // raft cluster ID for request validation
Raft Raft // raft state machine, to which the Transport forwards received messages and reports status
ServerStats *stats.ServerStats // used to record general transportation statistics
// used to record transportation statistics with followers when // used to record transportation statistics with followers when
// performing as leader in raft protocol // performing as leader in raft protocol
LeaderStats *stats.LeaderStats LeaderStats *stats.LeaderStats
@ -102,6 +105,9 @@ type Transport struct {
// machine and thus stop the Transport. // machine and thus stop the Transport.
ErrorC chan error ErrorC chan error
streamRt http.RoundTripper // roundTripper used by streams
pipelineRt http.RoundTripper // roundTripper used by pipelines
mu sync.RWMutex // protect the term, remote and peer map mu sync.RWMutex // protect the term, remote and peer map
term uint64 // the latest term that has been observed term uint64 // the latest term that has been observed
remotes map[types.ID]*remote // remotes map that helps newly joined member to catch up remotes map[types.ID]*remote // remotes map that helps newly joined member to catch up
@ -110,10 +116,23 @@ type Transport struct {
prober probing.Prober prober probing.Prober
} }
func (t *Transport) Start() { func (t *Transport) Start() error {
var err error
// Read/write timeout is set for stream roundTripper to promptly
// find out broken status, which minimizes the number of messages
// sent on broken connection.
t.streamRt, err = transport.NewTimeoutTransport(t.TLSInfo, t.DialTimeout, ConnReadTimeout, ConnWriteTimeout)
if err != nil {
return err
}
t.pipelineRt, err = transport.NewTransport(t.TLSInfo, t.DialTimeout)
if err != nil {
return err
}
t.remotes = make(map[types.ID]*remote) t.remotes = make(map[types.ID]*remote)
t.peers = make(map[types.ID]Peer) t.peers = make(map[types.ID]Peer)
t.prober = probing.NewProber(t.RoundTripper) t.prober = probing.NewProber(t.pipelineRt)
return nil
} }
func (t *Transport) Handler() http.Handler { func (t *Transport) Handler() http.Handler {
@ -183,7 +202,10 @@ func (t *Transport) Stop() {
p.Stop() p.Stop()
} }
t.prober.RemoveAll() t.prober.RemoveAll()
if tr, ok := t.RoundTripper.(*http.Transport); ok { if tr, ok := t.streamRt.(*http.Transport); ok {
tr.CloseIdleConnections()
}
if tr, ok := t.pipelineRt.(*http.Transport); ok {
tr.CloseIdleConnections() tr.CloseIdleConnections()
} }
} }
@ -198,7 +220,7 @@ func (t *Transport) AddRemote(id types.ID, us []string) {
if err != nil { if err != nil {
plog.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.pipelineRt, urls, t.ID, id, t.ClusterID, t.Raft, t.ErrorC)
} }
func (t *Transport) AddPeer(id types.ID, us []string) { func (t *Transport) AddPeer(id types.ID, us []string) {
@ -212,7 +234,7 @@ func (t *Transport) AddPeer(id types.ID, us []string) {
plog.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.term) t.peers[id] = startPeer(t.streamRt, t.pipelineRt, urls, t.ID, id, t.ClusterID, t.Raft, fs, t.ErrorC, t.term)
addPeerToProber(t.prober, id.String(), us) addPeerToProber(t.prober, id.String(), us)
} }

View File

@ -15,7 +15,6 @@
package rafthttp package rafthttp
import ( import (
"net/http"
"net/http/httptest" "net/http/httptest"
"sync" "sync"
"testing" "testing"
@ -31,12 +30,11 @@ import (
func BenchmarkSendingMsgApp(b *testing.B) { func BenchmarkSendingMsgApp(b *testing.B) {
// member 1 // member 1
tr := &Transport{ tr := &Transport{
RoundTripper: &http.Transport{}, ID: types.ID(1),
ID: types.ID(1), ClusterID: types.ID(1),
ClusterID: types.ID(1), Raft: &fakeRaft{},
Raft: &fakeRaft{}, ServerStats: newServerStats(),
ServerStats: newServerStats(), LeaderStats: stats.NewLeaderStats("1"),
LeaderStats: stats.NewLeaderStats("1"),
} }
tr.Start() tr.Start()
srv := httptest.NewServer(tr.Handler()) srv := httptest.NewServer(tr.Handler())
@ -45,12 +43,11 @@ func BenchmarkSendingMsgApp(b *testing.B) {
// member 2 // member 2
r := &countRaft{} r := &countRaft{}
tr2 := &Transport{ tr2 := &Transport{
RoundTripper: &http.Transport{}, ID: types.ID(2),
ID: types.ID(2), ClusterID: types.ID(1),
ClusterID: types.ID(1), Raft: r,
Raft: r, ServerStats: newServerStats(),
ServerStats: newServerStats(), LeaderStats: stats.NewLeaderStats("2"),
LeaderStats: stats.NewLeaderStats("2"),
} }
tr2.Start() tr2.Start()
srv2 := httptest.NewServer(tr2.Handler()) srv2 := httptest.NewServer(tr2.Handler())

View File

@ -70,11 +70,11 @@ func TestTransportAdd(t *testing.T) {
ls := stats.NewLeaderStats("") ls := stats.NewLeaderStats("")
term := uint64(10) term := uint64(10)
tr := &Transport{ tr := &Transport{
RoundTripper: &roundTripperRecorder{}, LeaderStats: ls,
LeaderStats: ls, streamRt: &roundTripperRecorder{},
term: term, term: term,
peers: make(map[types.ID]Peer), peers: make(map[types.ID]Peer),
prober: probing.NewProber(nil), prober: probing.NewProber(nil),
} }
tr.AddPeer(1, []string{"http://localhost:2380"}) tr.AddPeer(1, []string{"http://localhost:2380"})
@ -103,10 +103,10 @@ func TestTransportAdd(t *testing.T) {
func TestTransportRemove(t *testing.T) { func TestTransportRemove(t *testing.T) {
tr := &Transport{ tr := &Transport{
RoundTripper: &roundTripperRecorder{}, LeaderStats: stats.NewLeaderStats(""),
LeaderStats: stats.NewLeaderStats(""), streamRt: &roundTripperRecorder{},
peers: make(map[types.ID]Peer), peers: make(map[types.ID]Peer),
prober: probing.NewProber(nil), prober: probing.NewProber(nil),
} }
tr.AddPeer(1, []string{"http://localhost:2380"}) tr.AddPeer(1, []string{"http://localhost:2380"})
tr.RemovePeer(types.ID(1)) tr.RemovePeer(types.ID(1))
@ -134,11 +134,12 @@ func TestTransportUpdate(t *testing.T) {
func TestTransportErrorc(t *testing.T) { func TestTransportErrorc(t *testing.T) {
errorc := make(chan error, 1) errorc := make(chan error, 1)
tr := &Transport{ tr := &Transport{
RoundTripper: newRespRoundTripper(http.StatusForbidden, nil), LeaderStats: stats.NewLeaderStats(""),
LeaderStats: stats.NewLeaderStats(""), ErrorC: errorc,
ErrorC: errorc, streamRt: newRespRoundTripper(http.StatusForbidden, nil),
peers: make(map[types.ID]Peer), pipelineRt: newRespRoundTripper(http.StatusForbidden, nil),
prober: probing.NewProber(nil), peers: make(map[types.ID]Peer),
prober: probing.NewProber(nil),
} }
tr.AddPeer(1, []string{"http://localhost:2380"}) tr.AddPeer(1, []string{"http://localhost:2380"})
defer tr.Stop() defer tr.Stop()