-
Notifications
You must be signed in to change notification settings - Fork 167
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
Question about LeaseRead #166
Comments
|
The test code provided above is a reproduction of the issue I described. However, the observed result where n2 became the leader while n1 still held the lease, allowing for a leaseRead operation. It is also possible that there is an issue with the test I wrote, as I am not familiar with this project. Can someone help me understand this question? |
Hey @boringhello, I found your question very interesting. I'm new to this topic, but I'll share my ideas since I'm also interested in it. Let's look at it from n1's perspective:
Between time2 and time3, there is a possibility of encountering two lease-read results. Even if you implement a new Lease Read algorithm that includes a clock drift bound and allows clients to keep track of the latest index corresponding to the results they have seen, the problem will still persist. This approach only optimizes the situation, it doesn't fix it. However, it can reduce the occurrence of the problem. I agree that having two ElectionTimeouts doesn't make much sense. And there is room for improvement. I'm not sure if my idea is right, but a possible solution is to track the follower's HeartBeatClock in ProgressTracker. The HeartBeatClock increases with the Leader's clock, and when we receive the follower's HeartBeatResponse, we reset the HeartBeatClock to 0. When the HeartBeatClock reaches the ElectionTimeout, we can mark the corresponding RecentActive as false. Before responding to a LeaseRead request, the leader needs to check the QuorumActive first. |
Thanks for your idea. But it seemed that if the responce is delayed in the network, it may be wrong. I am not sure why etcd raft desgin like this.Maybe this is a tradeoff. |
@pav-kv can you have a look this? It seemed that the LeaseRead's implement have some issues. |
Last time I looked at leases in this repo, they did not appear correct/usable and providing at-most-one guarantee. I don't have time to look at this in detail at the moment. For leases to work properly, there needs to be some "global" logical time notion shared by all the nodes. There must be a guarantee that any logical timestamp is owned by at most one lease. Ticks are a local heuristic, and ticks are not synchronized across nodes, so it's hard to tell if these heuristics give any guarantees. |
You say that Leaseread was abandoned in this project? |
I'm not aware who uses it. In CRDB, we have our own leases on top of raft based on hybrid logical clocks. |
So why not remove the LeaseRead feature if they are not useful/correct ? |
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 <[email protected]> Co-authored-by: Yahor Yuzefovich <[email protected]> Co-authored-by: Michael Butler <[email protected]>
In the Raft paper, there is a mentioned optimization method for Lease Read using clock + heartbeat. When the leader sends a heartbeat, it first records a timestamp called "start". If the majority of the nodes in the system respond with heartbeat responses, it is assumed that the leader's lease is valid until the time point "start + election timeout / clock drift bound".
But when I read etcd LeaseRead code, it seems that LeaseRead depend on the checkquorum. When the leader receive the major of heartbeat responses in the half of election_timeout, the MsgCheckQuorum will not step the leader down on this election_timeout round. The next round, the leader step down into follower because the major of nodes is not recentActive.This case seems that the leasetime last 1.5 election_timeout.This confused me.
The text was updated successfully, but these errors were encountered: