etcd: All leases are revoked when the etcd leader is stuck in handling raft Ready due to slow fdatasync or high CPU.
What happened?
We have 3 etcd nodes which are running on k8s, the etcd pod use portworx storage for db saving, we’ve found when rebooting the portworx node, it will cause etcd to be stuck in fdatasync, if this etcd node is leader, then it will cause below 2 lease keep alive issues.
- the old leader still revokes lease after it steped to follower, this is the same issue as #12528
some clients connected to other etcd nodes and send keep alive to them, after the old leader steped to follower, the new leader will handle the lease keep alive, but the old leader’s lessor still considered itself is the primary and revoked all leases after their ttl expired in its view, after the old leader returned from fdatasync, these revoke request will be still sent out when handling the next Ready.
- the old leader still handle lease keep alive request from the connected client.
some clients connected to the old leader etcd node, when the old leader was stuck in fdatasync, the raft layer steped to follower, but the lessor was not demoted, the connected clients can still send keep alive request to the old leader, at the same time the other two etcd nodes elected a new leader and didn’t receive the lease keep alive message which should be sent to them but they were still sent to old leader, so these leases were revoked after the ttl expired.
Essentially the above 2 issues are because of the lessor primary is not synced up with the raft leader, there is big gap between lessor primary and raft leader when etcd server is stuck in processing raft Ready.
What did you expect to happen?
- The lease which is kept alive on old leader should return an error when its raft stepped to follower so that the clients can connect to other etcd nodes.
- The old leader should not revoke the other leases.
How can we reproduce it (as minimally and precisely as possible)?
- Add a piece of codes in func (w *WAL) sync() function to simulate fdatasync stuck
func (w *WAL) sync() error {
if w.encoder != nil {
if err := w.encoder.flush(); err != nil {
return err
}
}
start := time.Now()
err := fileutil.Fdatasync(w.tail().File)
//Simulate fdatasync is stuck
if fileutil.Exist("/run/etcd/fdatasync_block") {
f, err := ioutil.ReadFile("/run/etcd/fdatasync_block")
if err == nil {
str_num := strings.Trim(string(f), "\n")
block_duration, err := strconv.Atoi(str_num)
if err == nil {
time.Sleep(time.Duration(block_duration) * time.Second)
} else {
if w.lg != nil {
w.lg.Warn("Atoi error", zap.Error(err))
}
}
}
}
took := time.Since(start)
if took > warnSyncDuration {
if w.lg != nil {
w.lg.Warn(
"slow fdatasync",
zap.Duration("took", took),
zap.Duration("expected-duration", warnSyncDuration),
)
} else {
plog.Warningf("sync duration of %v, expected less than %v", took, warnSyncDuration)
}
}
walFsyncSec.Observe(took.Seconds())
return err
}
- Compile etcd with above changes.
- Apply the etcd patch in a cluster and restart them.
- Grant 3 leases with 10 seconds ttl.
- Keep different lease alive on different etcd node.
- Start to simulate fdatasync blocking on leader etcd with below commands:
mkdir /run/etcd; echo 60 > /run/etcd/fdatasync_block
- Put a kv to trigger fdatasync.
- Wait for etcd leader changing.
- Check the time to live of the lease on the new etcd leader which is kept alive on the old leader, you will see it’s expired but the keeping alive is on going on the old leader.
- Wait for about 60 seconds, the other two leases will be revoked after the old leader is recovered.
Anything else we need to know?
No response
Etcd version (please run commands below)
$ etcd --version
etcd Version: 3.4.3
Git SHA: 3cf2f69b5
Go Version: go1.14
Go OS/Arch: linux/amd64
$ etcdctl version
etcdctl version: 3.4.3
API version: 3.4
Etcd configuration (command line flags or environment variables)
paste your configuration here
Etcd debug information (please run commands below, feel free to obfuscate the IP address or FQDN in the output)
$ etcdctl member list -w table
# paste output here
$ etcdctl --endpoints=<member list> endpoint status -w table
# paste output here
Relevant log output
No response
About this issue
- Original URL
- State: open
- Created a year ago
- Reactions: 1
- Comments: 28 (24 by maintainers)
Commits related to this issue
- add e2e test cases to reproduce the lease revoke issue Refer to https://github.com/etcd-io/etcd/issues/15247 Signed-off-by: Benjamin Wang <benjamin.ahrtr@gmail.com> — committed to ahrtr/etcd by ahrtr 7 months ago
- Add e2e test cases to reproduce the lease revoke issue Refer to https://github.com/etcd-io/etcd/issues/15247 Signed-off-by: Benjamin Wang <benjamin.ahrtr@gmail.com> — committed to ahrtr/etcd by ahrtr 7 months ago
- Add e2e test cases to reproduce the lease revoke issue Refer to https://github.com/etcd-io/etcd/issues/15247 Signed-off-by: Benjamin Wang <benjamin.ahrtr@gmail.com> — committed to ahrtr/etcd by ahrtr 7 months ago
- Add e2e test cases to reproduce the lease revoke issue Refer to https://github.com/etcd-io/etcd/issues/15247 Signed-off-by: Benjamin Wang <benjamin.ahrtr@gmail.com> — committed to ahrtr/etcd by ahrtr 7 months ago
- Add e2e test cases to reproduce the lease revoke issue Refer to https://github.com/etcd-io/etcd/issues/15247 Signed-off-by: Benjamin Wang <benjamin.ahrtr@gmail.com> Signed-off-by: Raphael Skuza <rap... — committed to RaphSku/etcd by ahrtr 7 months ago
Thanks all for the feedback.
The first draft design doc: https://docs.google.com/document/d/1U9hAcZQp3Y36q_JFiw2VBJXVAo2dK2a-8Rsbqv3GgDo/edit#
https://docs.google.com/document/d/1peLDjwebnSuNR69ombuUiJ5fkSw7Rd3Slv_mUpOLsmg/edit
Root cause
When the leader gets blocked on the raft loop such as writing WAL entries, it can’t process any events or messages from the raft, such as leader changing, it can’t send messages to other members either. When the leader changes during this stalled writing, the older leader thinks it’s still the leader. There are two cases/consequences here,
Case 1: Leases are wrongly revoked by the older leader
If the client, which performs the lease keepAlive operation, is connected to another member, any lease renewal requests following the leader changing event will be forwarded to the new leader instead of the old leader. The old leader isn’t able to receive any checkpoint requests from the new leader either, because it’s being blocked on the stalled writing. When the old leader gets recovered from the stalled writing, it will revoke all expired leases, which are actually being kept renewed by the new leader.
Case 2: Leases are wrongly revoked by the new leader
If the client, which performs the lease keepAlive operation, is connected to the old leader, any lease renewal requests following the leader changing event will not be forwarded to the new leader. Any checkpoint requests are not able to be sent to other members either. So eventually the new leader will revoke all leases which passed the expiration time.
Thanks @tbg for the input.
When there is something wrong on the storage, etcd may get stuck on syncing data on the storage files. When running into this situation, the etcd member may not be able to process any requests, even become unresponsive when it is being removed from the cluster, e.g issues/14338. If the blocked member is the leader, it may cause two issues,
I am thinking to propose to terminate etcd process when it doesn’t finish syncing the data (WAL or boltdb) in a configured max duration (we can define two thresholds of duration, one for WAL and the other for boltDB). When the member starts again, it will not be elected as the leader anymore if the storage issue isn’t resolved, so the two issues mentioned above will not happen.
any immediate comments or concerns? @mitake @ptabor @serathius @spzala @tbg @liggitt @dims @lavalamp @wojtek-t @nikhita @neolit123 @chaochn47 @tjungblu @fuweid
If it looks overall good, then I will provide a detailed design doc.
In nature, this issue is an edge case needs a bug fix.
I am more inclined to a less risky change with e2e failure injection test that guarantees the issue is resolved without backward incompatible changes. This is still better than current state that exposing this potential bugs in production releases.
For example option 1 using ReadIndex or similar change proposed in https://github.com/etcd-io/etcd/pull/12531.
the other issue also need to be handled that the stuck leader should stop the grpc service so that the connected clients can switch the connection to other nodes util the disk issue is resolved.
I don’t see any data consistency issue. I will clearly clarify & explain the details in the design doc.
Note that it’s a critical issue if we see any data inconsistency in whatever situation, unless the storage hardware is broken.