etcd-io / raft

Raft library for maintaining a replicated state machine
Apache License 2.0
630 stars 160 forks source link

Question about LeaseRead #166

Open boringhello opened 6 months ago

boringhello commented 6 months ago

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.

boringhello commented 6 months ago
func TestLeaseRead(t *testing.T) {
    n1 := newTestRaft(1, 10, 1, newTestMemoryStorage(withPeers(1, 2, 3)))
    n2 := newTestRaft(2, 10, 1, newTestMemoryStorage(withPeers(1, 2, 3)))
    n3 := newTestRaft(3, 10, 1, newTestMemoryStorage(withPeers(1, 2, 3)))
    n1.checkQuorum = true
    n2.checkQuorum = true
    n3.checkQuorum = true
    n1.readOnly.option = ReadOnlyLeaseBased
    n2.readOnly.option = ReadOnlyLeaseBased
    n3.readOnly.option = ReadOnlyLeaseBased
    nt := newNetwork(n1, n2, n3)
    nt.send(pb.Message{From: 1, To: 1, Type: pb.MsgHup})
    if n1.state != StateLeader {
        t.Errorf("node 1 state: %s, want %s", n1.state, StateLeader)
    }
    if n2.state != StateFollower {
        t.Fatalf("node 2 state: %s, want %s", n2.state, StateFollower)
    }
    if n3.state != StateFollower {
        t.Fatalf("node 3 state: %s, want %s", n3.state, StateFollower)
    }
    nt.send(pb.Message{From: 1, To: 1, Type: pb.MsgProp, Entries: []pb.Entry{{Data: []byte("foo")}}})
    nt.cut(1, 2)
    nt.cut(1, 3)
    n1.Step(pb.Message{From: 1, Type: pb.MsgReadIndex, Entries: []pb.Entry{{Data: []byte("123")}}})
    msgs := n1.readStates
    if len(msgs) != 1 {
        t.Errorf("len(msgs) = %d, want 1", len(msgs))
    }
    for i := 0; i < 17; i++ {
        n1.tick()
    }
    // wait inLease
    for i := 0; i < n2.electionTimeout; i++ {
        n2.tick()
    }
    for i := 0; i < n3.electionTimeout; i++ {
        n3.tick()
    }
    nt.send(pb.Message{From: 2, To: 2, Type: pb.MsgHup})
    n1.Step(pb.Message{From: 1, Type: pb.MsgReadIndex, Entries: []pb.Entry{{Data: []byte("123")}}})
    msgs = n1.readStates
    if len(msgs) != 2 {
        t.Errorf("len(msgs) = %d, want 2", len(msgs))
    }
    if n2.state == StateLeader {
        t.Errorf("n2 become leader")
    }
    if n3.state == StateLeader {
        t.Errorf("n3 become leader")
    }
}
boringhello commented 6 months ago

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?

MrDXY commented 6 months ago

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. I believe the main issue is having two leaders serving at the same time. If we enable the Leader Lease and a network partition happens, it will take a maximum of two election timeout durations for the leader to realize it has lost the quorum and step down as a follower in the current etcd-raft.

image

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.

boringhello commented 6 months ago

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.

boringhello commented 6 months ago

@pav-kv can you have a look this? It seemed that the LeaseRead's implement have some issues.

pav-kv commented 6 months ago

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.

boringhello commented 6 months ago

You say that Leaseread was abandoned in this project?

pav-kv commented 6 months ago

I'm not aware who uses it. In CRDB, we have our own leases on top of raft based on hybrid logical clocks.

boringhello commented 6 months ago

So why not remove the LeaseRead feature if they are not useful/correct ?