-
Notifications
You must be signed in to change notification settings - Fork 9.8k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
etcdserver: adjust election timeout on restart #9415
Conversation
Codecov Report
@@ Coverage Diff @@
## master #9415 +/- ##
=========================================
Coverage ? 72.48%
=========================================
Files ? 362
Lines ? 30827
Branches ? 0
=========================================
Hits ? 22344
Misses ? 6854
Partials ? 1629
Continue to review full report at Codecov.
|
etcdserver/server.go
Outdated
// 1. all connections failed, or | ||
// 2. no active peers, or | ||
// 3. restarted single-node with no snapshot | ||
plog.Infof("%s waited %s but no active peer found (or restarted 1-node cluster); currently, %d member(s)", srv.ID(), rafthttp.ConnReadTimeout, len(cl.Members())) |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
just remove this logging? we only log if we do something special.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Removed. PTAL. Thanks!
rafthttp/peer_status.go
Outdated
|
||
if s.once != nil { | ||
s.once.Do(func() { | ||
plog.Infof("notifying of active peer %q", s.id) |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
this logging does not seem to be useful.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Removed.
rafthttp/transport.go
Outdated
// InitialPeerNotify returns a channel that closes when an initial | ||
// peer connection has been established. Use this to wait until the | ||
// first peer connection becomes active. | ||
InitialPeerNotify() <-chan struct{} |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
hmm... can you try to find another way to do this without introducing a new method to the interface? this interface is too heavy already.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Agree. Just removed.
fad0db1
to
60b3d7f
Compare
etcdserver/server.go
Outdated
// This can be used for fast-forwarding election | ||
// ticks in multi data-center deployments, thus | ||
// speeding up election process. | ||
advanceRaftTicks func(ticks int) |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
this should be a method on raft related struct.
etcdserver/server.go
Outdated
@@ -527,6 +539,32 @@ func NewServer(cfg ServerConfig) (srv *EtcdServer, err error) { | |||
} | |||
srv.r.transport = tr | |||
|
|||
srv.goAttach(func() { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
why this needs to be async? can we start the network first? then wait here? then decide to advance ticks or not. then start the raft routine?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
also if it works like what i described, we do not need the lock to protect the tick.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
It needs to be async, since we start peer handler in embed package, after we create etcd server here.
Addressed others. PTAL.
rafthttp/peer_status.go
Outdated
@@ -32,11 +32,16 @@ type peerStatus struct { | |||
mu sync.Mutex // protect variables below | |||
active bool | |||
since time.Time | |||
|
|||
once *sync.Once | |||
notify chan struct{} |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
activeNotify
rafthttp/peer_status.go
Outdated
|
||
if s.once != nil { | ||
s.once.Do(func() { | ||
close(s.notify) |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
probably just do
select {
case s.notify<- struct{}{}:
default:
}
we do not need the once struct.
rafthttp/transport.go
Outdated
// InitialPeerNotify returns a channel that closes when an initial | ||
// peer connection has been established. Use this to wait until the | ||
// first peer connection becomes active. | ||
func (t *Transport) InitialPeerNotify() <-chan struct{} { return t.initPeerNotifyCh } |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
well... actually, an easy way to solve this problem is to have a for loop to loop over the peer status and check if active is true.
for p := range peers {
if p.status.isActive() {
// send to chan
}
}
rafthttp/transport.go
Outdated
// InitialPeerNotify returns a channel that closes when an initial | ||
// peer connection has been established. Use this to wait until the | ||
// first peer connection becomes active. | ||
func (t *Transport) InitialPeerNotify() <-chan struct{} { return t.initPeerNotifyCh } |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Even better:
InitialPeerNotify() -> ActivePeers()
ActivePeers simply calculate how many peers are active. we move the for loop thing to the raft node side in etcdserver pkg to wait for the first active peer.
8c4a077
to
e42dd2a
Compare
etcdserver/server.go
Outdated
} | ||
} | ||
|
||
// 1. all connections failed, or |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
move this comment to line 544
etcdserver/server.go
Outdated
// retry up to "rafthttp.ConnReadTimeout", which is 5-sec | ||
for i := 0; i < 5; i++ { | ||
select { | ||
case <-time.After(time.Second): |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
reduce this to 50ms to be more responsive.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
define 50ms as wait time
etcdserver/server.go
Outdated
} | ||
|
||
// retry up to "rafthttp.ConnReadTimeout", which is 5-sec | ||
for i := 0; i < 5; i++ { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
define waitTime. then here can be 5second/waittime
e42dd2a
to
9f356c8
Compare
rafthttp/peer.go
Outdated
@@ -76,6 +76,9 @@ type Peer interface { | |||
// activeSince returns the time that the connection with the | |||
// peer becomes active. | |||
activeSince() time.Time | |||
// isActive returns true if the connection to this peer | |||
// has been established | |||
isActive() bool |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
we can reuse activeSince. if it is smaller than current, it it is active, right?
lgtm. |
Signed-off-by: Gyuho Lee <[email protected]>
Signed-off-by: Gyuho Lee <[email protected]>
9f356c8
to
33adce4
Compare
Signed-off-by: Gyuho Lee <[email protected]>
33adce4
to
9680b8a
Compare
When do we expect this fix to be released in 3.1 branch? |
@gyuho Yes, I'm available. Thursday (tomorrow) work? I'm free Friday as well. |
@jpbetz Tomorrow (Thursday) sounds good. Will ping you when the key is ready. Thanks. |
@@ -97,6 +97,7 @@ type raftNode struct { | |||
term uint64 | |||
lead uint64 | |||
|
|||
tickMu *sync.Mutex |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Out of curiosity, any reason for pointer here? My understanding is that pointers are not typically needed for sync.Mutex.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I'm planning to backport this to 3.1 without a pointer since that removes the need to initialize the mutex, which simplifies the backport: https://github.com/coreos/etcd/pull/9500/files#diff-8c6a0ae3bb0763acd9c96a35d89131feR99
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
@jpbetz govet would complain something like this
passes lock by value: github.com/coreos/etcd/clientv3.Client contains sync.Mutex
…elease-3.2 Automated cherry pick of #9415
…elease-3.1 Automated cherry pick of #9415
Address #9333 with simpler logic.
Single-node restart with no snapshot does not need special handling, because itself will be elected as leader, by the time peer connection report wait times out.
Fresh start 1-node cluster
Restart 1-node cluster from snapshot
Restart 1-node with no snapshot
Leader gets elected while waiting for peer connection report timeouts, so no side-effect.
Fresh start 3-node
node A:
node B:
No side-effect.
Rejoining to 3-node cluster with snapshot
Peer connection is notified and advance with adjusted ticks.
Previously, it advanced 9 ticks with only one tick left. Now, advances 8 ticks.
Rejoining to 3-node cluster with no snapshot
/cc @xiang90 @jpbetz