Skip to content
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

lease read safety #10082

Closed
pifuant opened this issue Sep 11, 2018 · 9 comments
Closed

lease read safety #10082

pifuant opened this issue Sep 11, 2018 · 9 comments

Comments

@pifuant
Copy link

pifuant commented Sep 11, 2018

5 nodes(1, 2, 3, 4, 5), node 1 is leader, connection of between node 1 and node 5 is break, others is ok.

In node 1(master) lease time, if node 5 triggers leader election and become leader, and then write some data. it would break lease read safety, how did etcd deal with this situation?

@hexfusion
Copy link
Contributor

@pifuant let's say we have 5 nodes on term 2 and node 1 is the leader. Although node 5 can communicate with the other nodes it will not receive heartbeat AppendEntries RPC's from leader node 1 and will timeout thus become a candidate and start the election process.

At this time node 5 will increment its term to 3 vote for itself and sends out RequestVote RPCs to the other nodes. Because it can still communicate with a majority of the nodes it could become the leader, which in this case we will assume. Once the remaining nodes increment to term 3 node 1 will remain in term 2 and all requests from it will be rejected by the nodes in term 3.

When the new leader is elected it will reset all lease TTLs to its original value plus extend which is basically the election timeout via lease.Promote.

etcd/lease/lessor.go

Lines 424 to 436 in e8b940f

func (le *lessor) Promote(extend time.Duration) {
le.mu.Lock()
defer le.mu.Unlock()
le.demotec = make(chan struct{})
// refresh the expiries of all leases.
for _, l := range le.leaseMap {
l.refresh(extend)
item := &LeaseWithTime{id: l.ID, time: l.expiry.UnixNano()}
heap.Push(&le.leaseHeap, item)
}

s.lessor.Promote(s.Cfg.electionTimeout())

In v3.4 we will begin to optimize this prevent indefinite auto-renewal (see ref below)

It is important to note that if entries were committed thus replicated to a majority of other nodes while node 1 was the leader and before node 5 was elected then node 5 will not contain all records of term 2 required to be elected in term 3. So, in this case, node 5 will not be voted for by the remaining nodes. You can read section 5.4 of the raft paper for more details on safety.

ref: #9924, #7320, op-guide: leader-failure, raft paper

I hope this helps shed a little light on how etcd would handle this situation. Lease data is still safe.

@pifuant
Copy link
Author

pifuant commented Sep 13, 2018

@hexfusion thanks

Important assumption: node1 is leader , node1 and node 5 has identical log, node 1 can't communicate with node 5

Node 1 don't know that node 5 has became leader and don't know TTL reset, because nobody can tell Node 1 all the thing. Unluckily, if node 1 receives a read request, node 1 just think it is leader and it will read local data in valid lease time.

@pifuant
Copy link
Author

pifuant commented Sep 13, 2018

I think raft lease read is unsafe, but lease read in "paxos made live" is safe.

@xiang90
Copy link
Contributor

xiang90 commented Sep 13, 2018

if node 1 receives a read request, node 1 just think it is leader and it will read local data in valid lease time.

no. l-read needs acks from the quorum.

@xiang90 xiang90 closed this as completed Sep 13, 2018
@pifuant
Copy link
Author

pifuant commented Sep 13, 2018

To use clocks instead of messages for read-only queries, the normal heartbeat mechanism would provide a form of lease [33]. Once the leader’s heartbeats were acknowledged by a majority of the cluster, the leader would assume that no other server will become leader for about an election timeout, and it could extend its lease accordingly (see Figure 6.3). The leader would then reply to read-only queries during that period without any additional communication. (The leadership transfer mechanism presented in Chapter 3 allows the leader to be replaced early; a leader would need to expire its lease before transferring leadership.)

@xiang90 no, node 1 don't need acks from the quorum

@pifuant
Copy link
Author

pifuant commented Sep 13, 2018

@hexfusion @xiang90 please take a look again, I really think raft lease read don't promise safety, unless etcd did extra thing for it.

@xiang90
Copy link
Contributor

xiang90 commented Sep 13, 2018

@pifuant If you have a question about raft's correctness, probably you should start the discussion in the raft mailing list: https://groups.google.com/forum/#!forum/raft-dev

@pifuant
Copy link
Author

pifuant commented Sep 14, 2018

@xiang90 thanks

@boringhello
Copy link

boringhello commented Mar 13, 2024

I have a question about the etcd raft lease Read's safety. etcd-io/raft#166.

craig bot pushed a commit to cockroachdb/cockroach that referenced this issue Apr 9, 2024
120613: raft: remove unused read-only requests r=nvanbenschoten a=pav-kv

This PR removes the read-only requests from `pkg/raft`. We don't use them in CRDB, and the leases implementation is known to be incorrect (e.g. see etcd-io/raft#166 and etcd-io/etcd#10082).

Epic: none
Release note: none

121956: sql: add accounting for entries in Txn Fingerprint ID cache r=yuzefovich a=yuzefovich

This commit fixes a bug that could previously result in the memory accounting leak that was exposed by 88ebd70. Namely, the problem is that we previously unconditionally grew the memory account in `Add` even though if the ID is already present in the cache, it wouldn't be inserted again. As a result, we'd only shrink the account once but might have grown it any number of times for a particular ID. Now we check whether the ID is present in the cache and only grow the account if not.

Epic: None

Release note: None

122011: backupccl: skip BackuprestoreJobDescription under stress race r=kev-cao a=msbutler

Informs #121927

Release note: none

Co-authored-by: Pavel Kalinnikov <pavel@cockroachlabs.com>
Co-authored-by: Yahor Yuzefovich <yahor@cockroachlabs.com>
Co-authored-by: Michael Butler <butler@cockroachlabs.com>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Development

No branches or pull requests

4 participants