mirror of
https://github.com/etcd-io/etcd.git
synced 2024-09-27 06:25:44 +00:00
etcdserver: rename "SnapshotCount", add "SnapshotCatchUpEntries"
Signed-off-by: Gyuho Lee <gyuhox@gmail.com>
This commit is contained in:
parent
374f4cef4d
commit
49d672ff9b
@ -41,9 +41,20 @@ type ServerConfig struct {
|
|||||||
// DedicatedWALDir config will make the etcd to write the WAL to the WALDir
|
// DedicatedWALDir config will make the etcd to write the WAL to the WALDir
|
||||||
// rather than the dataDir/member/wal.
|
// rather than the dataDir/member/wal.
|
||||||
DedicatedWALDir string
|
DedicatedWALDir string
|
||||||
SnapCount uint64
|
|
||||||
|
SnapshotCount uint64
|
||||||
|
|
||||||
|
// SnapshotCatchUpEntries is the number of entries for a slow follower
|
||||||
|
// to catch-up after compacting the raft storage entries.
|
||||||
|
// We expect the follower has a millisecond level latency with the leader.
|
||||||
|
// The max throughput is around 10K. Keep a 5K entries is enough for helping
|
||||||
|
// follower to catch up.
|
||||||
|
// WARNING: only change this for tests. Always use "DefaultSnapshotCatchUpEntries"
|
||||||
|
SnapshotCatchUpEntries uint64
|
||||||
|
|
||||||
MaxSnapFiles uint
|
MaxSnapFiles uint
|
||||||
MaxWALFiles uint
|
MaxWALFiles uint
|
||||||
|
|
||||||
InitialPeerURLsMap types.URLsMap
|
InitialPeerURLsMap types.URLsMap
|
||||||
InitialClusterToken string
|
InitialClusterToken string
|
||||||
NewCluster bool
|
NewCluster bool
|
||||||
@ -273,7 +284,7 @@ func (c *ServerConfig) print(initial bool) {
|
|||||||
}
|
}
|
||||||
plog.Infof("heartbeat = %dms", c.TickMs)
|
plog.Infof("heartbeat = %dms", c.TickMs)
|
||||||
plog.Infof("election = %dms", c.ElectionTicks*int(c.TickMs))
|
plog.Infof("election = %dms", c.ElectionTicks*int(c.TickMs))
|
||||||
plog.Infof("snapshot count = %d", c.SnapCount)
|
plog.Infof("snapshot count = %d", c.SnapshotCount)
|
||||||
if len(c.DiscoveryURL) != 0 {
|
if len(c.DiscoveryURL) != 0 {
|
||||||
plog.Infof("discovery URL= %s", c.DiscoveryURL)
|
plog.Infof("discovery URL= %s", c.DiscoveryURL)
|
||||||
if len(c.DiscoveryProxy) != 0 {
|
if len(c.DiscoveryProxy) != 0 {
|
||||||
@ -302,7 +313,8 @@ func (c *ServerConfig) print(initial bool) {
|
|||||||
zap.Int("election-tick-ms", c.ElectionTicks),
|
zap.Int("election-tick-ms", c.ElectionTicks),
|
||||||
zap.String("election-timeout", fmt.Sprintf("%v", time.Duration(c.ElectionTicks*int(c.TickMs))*time.Millisecond)),
|
zap.String("election-timeout", fmt.Sprintf("%v", time.Duration(c.ElectionTicks*int(c.TickMs))*time.Millisecond)),
|
||||||
zap.Bool("initial-election-tick-advance", c.InitialElectionTickAdvance),
|
zap.Bool("initial-election-tick-advance", c.InitialElectionTickAdvance),
|
||||||
zap.Uint64("snapshot-count", c.SnapCount),
|
zap.Uint64("snapshot-count", c.SnapshotCount),
|
||||||
|
zap.Uint64("snapshot-catchup-entries", c.SnapshotCatchUpEntries),
|
||||||
zap.Strings("advertise-client-urls", c.getACURLs()),
|
zap.Strings("advertise-client-urls", c.getACURLs()),
|
||||||
zap.Strings("initial-advertise-peer-urls", c.getAPURLs()),
|
zap.Strings("initial-advertise-peer-urls", c.getAPURLs()),
|
||||||
zap.Bool("initial", initial),
|
zap.Bool("initial", initial),
|
||||||
|
@ -39,13 +39,6 @@ import (
|
|||||||
)
|
)
|
||||||
|
|
||||||
const (
|
const (
|
||||||
// Number of entries for slow follower to catch-up after compacting
|
|
||||||
// the raft storage entries.
|
|
||||||
// We expect the follower has a millisecond level latency with the leader.
|
|
||||||
// The max throughput is around 10K. Keep a 5K entries is enough for helping
|
|
||||||
// follower to catch up.
|
|
||||||
numberOfCatchUpEntries = 5000
|
|
||||||
|
|
||||||
// The max throughput of etcd will not exceed 100MB/s (100K * 1KB value).
|
// The max throughput of etcd will not exceed 100MB/s (100K * 1KB value).
|
||||||
// Assuming the RTT is around 10ms, 1MB max size is large enough.
|
// Assuming the RTT is around 10ms, 1MB max size is large enough.
|
||||||
maxSizePerMsg = 1 * 1024 * 1024
|
maxSizePerMsg = 1 * 1024 * 1024
|
||||||
|
@ -64,7 +64,14 @@ import (
|
|||||||
)
|
)
|
||||||
|
|
||||||
const (
|
const (
|
||||||
DefaultSnapCount = 100000
|
DefaultSnapshotCount = 100000
|
||||||
|
|
||||||
|
// DefaultSnapshotCatchUpEntries is the number of entries for a slow follower
|
||||||
|
// to catch-up after compacting the raft storage entries.
|
||||||
|
// We expect the follower has a millisecond level latency with the leader.
|
||||||
|
// The max throughput is around 10K. Keep a 5K entries is enough for helping
|
||||||
|
// follower to catch up.
|
||||||
|
DefaultSnapshotCatchUpEntries uint64 = 5000
|
||||||
|
|
||||||
StoreClusterPrefix = "/0"
|
StoreClusterPrefix = "/0"
|
||||||
StoreKeysPrefix = "/1"
|
StoreKeysPrefix = "/1"
|
||||||
@ -703,14 +710,30 @@ func (s *EtcdServer) Start() {
|
|||||||
// This function is just used for testing.
|
// This function is just used for testing.
|
||||||
func (s *EtcdServer) start() {
|
func (s *EtcdServer) start() {
|
||||||
lg := s.getLogger()
|
lg := s.getLogger()
|
||||||
if s.Cfg.SnapCount == 0 {
|
|
||||||
if lg != nil {
|
|
||||||
|
|
||||||
|
if s.Cfg.SnapshotCount == 0 {
|
||||||
|
if lg != nil {
|
||||||
|
lg.Info(
|
||||||
|
"updating snapshot-count to default",
|
||||||
|
zap.Uint64("given-snapshot-count", s.Cfg.SnapshotCount),
|
||||||
|
zap.Uint64("updated-snapshot-count", DefaultSnapshotCount),
|
||||||
|
)
|
||||||
} else {
|
} else {
|
||||||
plog.Infof("set snapshot count to default %d", DefaultSnapCount)
|
plog.Infof("set snapshot count to default %d", DefaultSnapshotCount)
|
||||||
}
|
}
|
||||||
s.Cfg.SnapCount = DefaultSnapCount
|
s.Cfg.SnapshotCount = DefaultSnapshotCount
|
||||||
}
|
}
|
||||||
|
if s.Cfg.SnapshotCatchUpEntries == 0 {
|
||||||
|
if lg != nil {
|
||||||
|
lg.Info(
|
||||||
|
"updating snapshot catch-up entries to default",
|
||||||
|
zap.Uint64("given-snapshot-catchup-entries", s.Cfg.SnapshotCatchUpEntries),
|
||||||
|
zap.Uint64("updated-snapshot-catchup-entries", DefaultSnapshotCatchUpEntries),
|
||||||
|
)
|
||||||
|
}
|
||||||
|
s.Cfg.SnapshotCatchUpEntries = DefaultSnapshotCatchUpEntries
|
||||||
|
}
|
||||||
|
|
||||||
s.w = wait.New()
|
s.w = wait.New()
|
||||||
s.applyWait = wait.NewTimeList()
|
s.applyWait = wait.NewTimeList()
|
||||||
s.done = make(chan struct{})
|
s.done = make(chan struct{})
|
||||||
@ -743,6 +766,7 @@ func (s *EtcdServer) start() {
|
|||||||
plog.Infof("starting server... [version: %v, cluster version: to_be_decided]", version.Version)
|
plog.Infof("starting server... [version: %v, cluster version: to_be_decided]", version.Version)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
// TODO: if this is an empty log, writes all peer infos
|
// TODO: if this is an empty log, writes all peer infos
|
||||||
// into the first entry
|
// into the first entry
|
||||||
go s.run()
|
go s.run()
|
||||||
@ -1058,7 +1082,8 @@ func (s *EtcdServer) applySnapshot(ep *etcdProgress, apply *apply) {
|
|||||||
"applying snapshot",
|
"applying snapshot",
|
||||||
zap.Uint64("current-snapshot-index", ep.snapi),
|
zap.Uint64("current-snapshot-index", ep.snapi),
|
||||||
zap.Uint64("current-applied-index", ep.appliedi),
|
zap.Uint64("current-applied-index", ep.appliedi),
|
||||||
zap.Uint64("incoming-snapshot-index", apply.snapshot.Metadata.Index),
|
zap.Uint64("incoming-leader-snapshot-index", apply.snapshot.Metadata.Index),
|
||||||
|
zap.Uint64("incoming-leader-snapshot-term", apply.snapshot.Metadata.Term),
|
||||||
)
|
)
|
||||||
} else {
|
} else {
|
||||||
plog.Infof("applying snapshot at index %d...", ep.snapi)
|
plog.Infof("applying snapshot at index %d...", ep.snapi)
|
||||||
@ -1069,7 +1094,8 @@ func (s *EtcdServer) applySnapshot(ep *etcdProgress, apply *apply) {
|
|||||||
"applied snapshot",
|
"applied snapshot",
|
||||||
zap.Uint64("current-snapshot-index", ep.snapi),
|
zap.Uint64("current-snapshot-index", ep.snapi),
|
||||||
zap.Uint64("current-applied-index", ep.appliedi),
|
zap.Uint64("current-applied-index", ep.appliedi),
|
||||||
zap.Uint64("incoming-snapshot-index", apply.snapshot.Metadata.Index),
|
zap.Uint64("incoming-leader-snapshot-index", apply.snapshot.Metadata.Index),
|
||||||
|
zap.Uint64("incoming-leader-snapshot-term", apply.snapshot.Metadata.Term),
|
||||||
)
|
)
|
||||||
} else {
|
} else {
|
||||||
plog.Infof("finished applying incoming snapshot at index %d", ep.snapi)
|
plog.Infof("finished applying incoming snapshot at index %d", ep.snapi)
|
||||||
@ -1083,6 +1109,7 @@ func (s *EtcdServer) applySnapshot(ep *etcdProgress, apply *apply) {
|
|||||||
zap.Uint64("current-snapshot-index", ep.snapi),
|
zap.Uint64("current-snapshot-index", ep.snapi),
|
||||||
zap.Uint64("current-applied-index", ep.appliedi),
|
zap.Uint64("current-applied-index", ep.appliedi),
|
||||||
zap.Uint64("incoming-leader-snapshot-index", apply.snapshot.Metadata.Index),
|
zap.Uint64("incoming-leader-snapshot-index", apply.snapshot.Metadata.Index),
|
||||||
|
zap.Uint64("incoming-leader-snapshot-term", apply.snapshot.Metadata.Term),
|
||||||
)
|
)
|
||||||
} else {
|
} else {
|
||||||
plog.Panicf("snapshot index [%d] should > appliedi[%d] + 1",
|
plog.Panicf("snapshot index [%d] should > appliedi[%d] + 1",
|
||||||
@ -1304,7 +1331,7 @@ func (s *EtcdServer) applyEntries(ep *etcdProgress, apply *apply) {
|
|||||||
}
|
}
|
||||||
|
|
||||||
func (s *EtcdServer) triggerSnapshot(ep *etcdProgress) {
|
func (s *EtcdServer) triggerSnapshot(ep *etcdProgress) {
|
||||||
if ep.appliedi-ep.snapi <= s.Cfg.SnapCount {
|
if ep.appliedi-ep.snapi <= s.Cfg.SnapshotCount {
|
||||||
return
|
return
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -1314,7 +1341,7 @@ func (s *EtcdServer) triggerSnapshot(ep *etcdProgress) {
|
|||||||
zap.String("local-member-id", s.ID().String()),
|
zap.String("local-member-id", s.ID().String()),
|
||||||
zap.Uint64("local-member-applied-index", ep.appliedi),
|
zap.Uint64("local-member-applied-index", ep.appliedi),
|
||||||
zap.Uint64("local-member-snapshot-index", ep.snapi),
|
zap.Uint64("local-member-snapshot-index", ep.snapi),
|
||||||
zap.Uint64("local-member-snapshot-count", s.Cfg.SnapCount),
|
zap.Uint64("local-member-snapshot-count", s.Cfg.SnapshotCount),
|
||||||
)
|
)
|
||||||
} else {
|
} else {
|
||||||
plog.Infof("start to snapshot (applied: %d, lastsnap: %d)", ep.appliedi, ep.snapi)
|
plog.Infof("start to snapshot (applied: %d, lastsnap: %d)", ep.appliedi, ep.snapi)
|
||||||
@ -2132,9 +2159,10 @@ func (s *EtcdServer) snapshot(snapi uint64, confState raftpb.ConfState) {
|
|||||||
|
|
||||||
// keep some in memory log entries for slow followers.
|
// keep some in memory log entries for slow followers.
|
||||||
compacti := uint64(1)
|
compacti := uint64(1)
|
||||||
if snapi > numberOfCatchUpEntries {
|
if snapi > s.Cfg.SnapshotCatchUpEntries {
|
||||||
compacti = snapi - numberOfCatchUpEntries
|
compacti = snapi - s.Cfg.SnapshotCatchUpEntries
|
||||||
}
|
}
|
||||||
|
|
||||||
err = s.r.raftStorage.Compact(compacti)
|
err = s.r.raftStorage.Compact(compacti)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
// the compaction was done asynchronously with the progress of raft.
|
// the compaction was done asynchronously with the progress of raft.
|
||||||
|
@ -714,7 +714,7 @@ func TestDoProposal(t *testing.T) {
|
|||||||
srv := &EtcdServer{
|
srv := &EtcdServer{
|
||||||
lgMu: new(sync.RWMutex),
|
lgMu: new(sync.RWMutex),
|
||||||
lg: zap.NewExample(),
|
lg: zap.NewExample(),
|
||||||
Cfg: ServerConfig{Logger: zap.NewExample(), TickMs: 1},
|
Cfg: ServerConfig{Logger: zap.NewExample(), TickMs: 1, SnapshotCatchUpEntries: DefaultSnapshotCatchUpEntries},
|
||||||
r: *r,
|
r: *r,
|
||||||
v2store: st,
|
v2store: st,
|
||||||
reqIDGen: idutil.NewGenerator(0, time.Time{}),
|
reqIDGen: idutil.NewGenerator(0, time.Time{}),
|
||||||
@ -745,7 +745,7 @@ func TestDoProposalCancelled(t *testing.T) {
|
|||||||
srv := &EtcdServer{
|
srv := &EtcdServer{
|
||||||
lgMu: new(sync.RWMutex),
|
lgMu: new(sync.RWMutex),
|
||||||
lg: zap.NewExample(),
|
lg: zap.NewExample(),
|
||||||
Cfg: ServerConfig{Logger: zap.NewExample(), TickMs: 1},
|
Cfg: ServerConfig{Logger: zap.NewExample(), TickMs: 1, SnapshotCatchUpEntries: DefaultSnapshotCatchUpEntries},
|
||||||
r: *newRaftNode(raftNodeConfig{Node: newNodeNop()}),
|
r: *newRaftNode(raftNodeConfig{Node: newNodeNop()}),
|
||||||
w: wt,
|
w: wt,
|
||||||
reqIDGen: idutil.NewGenerator(0, time.Time{}),
|
reqIDGen: idutil.NewGenerator(0, time.Time{}),
|
||||||
@ -769,7 +769,7 @@ func TestDoProposalTimeout(t *testing.T) {
|
|||||||
srv := &EtcdServer{
|
srv := &EtcdServer{
|
||||||
lgMu: new(sync.RWMutex),
|
lgMu: new(sync.RWMutex),
|
||||||
lg: zap.NewExample(),
|
lg: zap.NewExample(),
|
||||||
Cfg: ServerConfig{Logger: zap.NewExample(), TickMs: 1},
|
Cfg: ServerConfig{Logger: zap.NewExample(), TickMs: 1, SnapshotCatchUpEntries: DefaultSnapshotCatchUpEntries},
|
||||||
r: *newRaftNode(raftNodeConfig{Node: newNodeNop()}),
|
r: *newRaftNode(raftNodeConfig{Node: newNodeNop()}),
|
||||||
w: mockwait.NewNop(),
|
w: mockwait.NewNop(),
|
||||||
reqIDGen: idutil.NewGenerator(0, time.Time{}),
|
reqIDGen: idutil.NewGenerator(0, time.Time{}),
|
||||||
@ -788,7 +788,7 @@ func TestDoProposalStopped(t *testing.T) {
|
|||||||
srv := &EtcdServer{
|
srv := &EtcdServer{
|
||||||
lgMu: new(sync.RWMutex),
|
lgMu: new(sync.RWMutex),
|
||||||
lg: zap.NewExample(),
|
lg: zap.NewExample(),
|
||||||
Cfg: ServerConfig{Logger: zap.NewExample(), TickMs: 1},
|
Cfg: ServerConfig{Logger: zap.NewExample(), TickMs: 1, SnapshotCatchUpEntries: DefaultSnapshotCatchUpEntries},
|
||||||
r: *newRaftNode(raftNodeConfig{lg: zap.NewExample(), Node: newNodeNop()}),
|
r: *newRaftNode(raftNodeConfig{lg: zap.NewExample(), Node: newNodeNop()}),
|
||||||
w: mockwait.NewNop(),
|
w: mockwait.NewNop(),
|
||||||
reqIDGen: idutil.NewGenerator(0, time.Time{}),
|
reqIDGen: idutil.NewGenerator(0, time.Time{}),
|
||||||
@ -899,7 +899,7 @@ func TestSyncTrigger(t *testing.T) {
|
|||||||
srv := &EtcdServer{
|
srv := &EtcdServer{
|
||||||
lgMu: new(sync.RWMutex),
|
lgMu: new(sync.RWMutex),
|
||||||
lg: zap.NewExample(),
|
lg: zap.NewExample(),
|
||||||
Cfg: ServerConfig{Logger: zap.NewExample(), TickMs: 1},
|
Cfg: ServerConfig{Logger: zap.NewExample(), TickMs: 1, SnapshotCatchUpEntries: DefaultSnapshotCatchUpEntries},
|
||||||
r: *r,
|
r: *r,
|
||||||
v2store: mockstore.NewNop(),
|
v2store: mockstore.NewNop(),
|
||||||
SyncTicker: tk,
|
SyncTicker: tk,
|
||||||
@ -1033,7 +1033,7 @@ func TestSnapshotOrdering(t *testing.T) {
|
|||||||
s := &EtcdServer{
|
s := &EtcdServer{
|
||||||
lgMu: new(sync.RWMutex),
|
lgMu: new(sync.RWMutex),
|
||||||
lg: zap.NewExample(),
|
lg: zap.NewExample(),
|
||||||
Cfg: ServerConfig{Logger: zap.NewExample(), DataDir: testdir},
|
Cfg: ServerConfig{Logger: zap.NewExample(), DataDir: testdir, SnapshotCatchUpEntries: DefaultSnapshotCatchUpEntries},
|
||||||
r: *r,
|
r: *r,
|
||||||
v2store: st,
|
v2store: st,
|
||||||
snapshotter: raftsnap.New(zap.NewExample(), snapdir),
|
snapshotter: raftsnap.New(zap.NewExample(), snapdir),
|
||||||
@ -1077,7 +1077,7 @@ func TestSnapshotOrdering(t *testing.T) {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
// Applied > SnapCount should trigger a SaveSnap event
|
// Applied > SnapshotCount should trigger a SaveSnap event
|
||||||
func TestTriggerSnap(t *testing.T) {
|
func TestTriggerSnap(t *testing.T) {
|
||||||
be, tmpPath := backend.NewDefaultTmpBackend()
|
be, tmpPath := backend.NewDefaultTmpBackend()
|
||||||
defer func() {
|
defer func() {
|
||||||
@ -1097,7 +1097,7 @@ func TestTriggerSnap(t *testing.T) {
|
|||||||
srv := &EtcdServer{
|
srv := &EtcdServer{
|
||||||
lgMu: new(sync.RWMutex),
|
lgMu: new(sync.RWMutex),
|
||||||
lg: zap.NewExample(),
|
lg: zap.NewExample(),
|
||||||
Cfg: ServerConfig{Logger: zap.NewExample(), TickMs: 1, SnapCount: uint64(snapc)},
|
Cfg: ServerConfig{Logger: zap.NewExample(), TickMs: 1, SnapshotCount: uint64(snapc), SnapshotCatchUpEntries: DefaultSnapshotCatchUpEntries},
|
||||||
r: *r,
|
r: *r,
|
||||||
v2store: st,
|
v2store: st,
|
||||||
reqIDGen: idutil.NewGenerator(0, time.Time{}),
|
reqIDGen: idutil.NewGenerator(0, time.Time{}),
|
||||||
@ -1116,7 +1116,7 @@ func TestTriggerSnap(t *testing.T) {
|
|||||||
gaction, _ := p.Wait(wcnt)
|
gaction, _ := p.Wait(wcnt)
|
||||||
|
|
||||||
// each operation is recorded as a Save
|
// each operation is recorded as a Save
|
||||||
// (SnapCount+1) * Puts + SaveSnap = (SnapCount+1) * Save + SaveSnap
|
// (SnapshotCount+1) * Puts + SaveSnap = (SnapshotCount+1) * Save + SaveSnap
|
||||||
if len(gaction) != wcnt {
|
if len(gaction) != wcnt {
|
||||||
t.Fatalf("len(action) = %d, want %d", len(gaction), wcnt)
|
t.Fatalf("len(action) = %d, want %d", len(gaction), wcnt)
|
||||||
}
|
}
|
||||||
@ -1164,7 +1164,7 @@ func TestConcurrentApplyAndSnapshotV3(t *testing.T) {
|
|||||||
s := &EtcdServer{
|
s := &EtcdServer{
|
||||||
lgMu: new(sync.RWMutex),
|
lgMu: new(sync.RWMutex),
|
||||||
lg: zap.NewExample(),
|
lg: zap.NewExample(),
|
||||||
Cfg: ServerConfig{Logger: zap.NewExample(), DataDir: testdir},
|
Cfg: ServerConfig{Logger: zap.NewExample(), DataDir: testdir, SnapshotCatchUpEntries: DefaultSnapshotCatchUpEntries},
|
||||||
r: *r,
|
r: *r,
|
||||||
v2store: st,
|
v2store: st,
|
||||||
snapshotter: raftsnap.New(zap.NewExample(), testdir),
|
snapshotter: raftsnap.New(zap.NewExample(), testdir),
|
||||||
@ -1375,7 +1375,7 @@ func TestPublish(t *testing.T) {
|
|||||||
lgMu: new(sync.RWMutex),
|
lgMu: new(sync.RWMutex),
|
||||||
lg: zap.NewExample(),
|
lg: zap.NewExample(),
|
||||||
readych: make(chan struct{}),
|
readych: make(chan struct{}),
|
||||||
Cfg: ServerConfig{Logger: zap.NewExample(), TickMs: 1},
|
Cfg: ServerConfig{Logger: zap.NewExample(), TickMs: 1, SnapshotCatchUpEntries: DefaultSnapshotCatchUpEntries},
|
||||||
id: 1,
|
id: 1,
|
||||||
r: *newRaftNode(raftNodeConfig{lg: zap.NewExample(), Node: n}),
|
r: *newRaftNode(raftNodeConfig{lg: zap.NewExample(), Node: n}),
|
||||||
attributes: membership.Attributes{Name: "node1", ClientURLs: []string{"http://a", "http://b"}},
|
attributes: membership.Attributes{Name: "node1", ClientURLs: []string{"http://a", "http://b"}},
|
||||||
@ -1428,7 +1428,7 @@ func TestPublishStopped(t *testing.T) {
|
|||||||
srv := &EtcdServer{
|
srv := &EtcdServer{
|
||||||
lgMu: new(sync.RWMutex),
|
lgMu: new(sync.RWMutex),
|
||||||
lg: zap.NewExample(),
|
lg: zap.NewExample(),
|
||||||
Cfg: ServerConfig{Logger: zap.NewExample(), TickMs: 1},
|
Cfg: ServerConfig{Logger: zap.NewExample(), TickMs: 1, SnapshotCatchUpEntries: DefaultSnapshotCatchUpEntries},
|
||||||
r: *r,
|
r: *r,
|
||||||
cluster: &membership.RaftCluster{},
|
cluster: &membership.RaftCluster{},
|
||||||
w: mockwait.NewNop(),
|
w: mockwait.NewNop(),
|
||||||
@ -1452,7 +1452,7 @@ func TestPublishRetry(t *testing.T) {
|
|||||||
srv := &EtcdServer{
|
srv := &EtcdServer{
|
||||||
lgMu: new(sync.RWMutex),
|
lgMu: new(sync.RWMutex),
|
||||||
lg: zap.NewExample(),
|
lg: zap.NewExample(),
|
||||||
Cfg: ServerConfig{Logger: zap.NewExample(), TickMs: 1},
|
Cfg: ServerConfig{Logger: zap.NewExample(), TickMs: 1, SnapshotCatchUpEntries: DefaultSnapshotCatchUpEntries},
|
||||||
r: *newRaftNode(raftNodeConfig{lg: zap.NewExample(), Node: n}),
|
r: *newRaftNode(raftNodeConfig{lg: zap.NewExample(), Node: n}),
|
||||||
w: mockwait.NewNop(),
|
w: mockwait.NewNop(),
|
||||||
stopping: make(chan struct{}),
|
stopping: make(chan struct{}),
|
||||||
@ -1495,7 +1495,7 @@ func TestUpdateVersion(t *testing.T) {
|
|||||||
lgMu: new(sync.RWMutex),
|
lgMu: new(sync.RWMutex),
|
||||||
lg: zap.NewExample(),
|
lg: zap.NewExample(),
|
||||||
id: 1,
|
id: 1,
|
||||||
Cfg: ServerConfig{Logger: zap.NewExample(), TickMs: 1},
|
Cfg: ServerConfig{Logger: zap.NewExample(), TickMs: 1, SnapshotCatchUpEntries: DefaultSnapshotCatchUpEntries},
|
||||||
r: *newRaftNode(raftNodeConfig{lg: zap.NewExample(), Node: n}),
|
r: *newRaftNode(raftNodeConfig{lg: zap.NewExample(), Node: n}),
|
||||||
attributes: membership.Attributes{Name: "node1", ClientURLs: []string{"http://node1.com"}},
|
attributes: membership.Attributes{Name: "node1", ClientURLs: []string{"http://node1.com"}},
|
||||||
cluster: &membership.RaftCluster{},
|
cluster: &membership.RaftCluster{},
|
||||||
|
Loading…
x
Reference in New Issue
Block a user