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

Fix/fix stale read #34

Merged
merged 9 commits into from
Mar 13, 2019
Merged

Fix/fix stale read #34

merged 9 commits into from
Mar 13, 2019

Conversation

fengjiachun
Copy link
Contributor

@fengjiachun fengjiachun commented Mar 12, 2019

感谢阿里云的 静舟 提出以下这个 case:

leader的step down动作依赖timer,每隔0.5*ET检查一次;这个检查频率是不够的,会出双主问题;
举个实际例子,现有 A B C 三个节点,A是当前leader,ET为10秒,发生了以下事件:

时间戳 事件
0s 网络分区 (A B) (C)
1s A check step down 通过,lease剩余10s
6s A check step down 通过,lease剩余10s
7s 网络分区 (A) (B C)
9s C 选举超时,lease剩余1s,无动作
11s A check step down 通过,lease剩余6s
16s A check step down 通过,lease剩余1s
19s C 选举超时,此时B的lease已过期,因此prevote成功,vote成功,C升级为leader
21s A check step down 失败,降级为follower

从上面的时间线可以看到,19s~21s存在了两个leader,这段时间内的lease read有可能不满足线性一致性

解决思路:
思路一:
新增了一个 LeaderLeaseTimeout, 默认是 0.9 个 ElectionTimeout , 在 readLeader() 时检查 lease,如果 lease timeout 了的话 就回退到 readIndex 读

思路二:
考虑到每次 lease read 都要循环检查一次的 cpu 开销,还是采用另一个方案:
LeaderLeaseTimeout 默认为 0.8 个 ElectionTimeout(ET),每 0.1 个 ET 定时调度更新 lastLeaderTimestamp, , 在 readLeader() 时检查 lease,如果 lease timeout 了的话 就回退到 readIndex 读

最终方案:
类似方案一,新增了一个 LeaderLeaseTimeout, 默认是 0.9 个 ElectionTimeout , 在 readLeader() 时检查 lease,如果 leaseTimeout 了的话, 重新检查 majority lastTimestamp 并更新 startLease,再次检查 startLease 如果仍然是 leaseTimeout 就回退到 readIndex 读
相比方案一,在 read 路径上尽量减少了 ”循环遍历检查 majority lastTimestamp“ ,另外在 stepDown 定时其中也会顺便更新 lastTimestamp

想强调一下的是,不是很建议开启 lease read,
对于 lease read,所有努力终将在一个时钟漂移中白费,jraft 中默认不开启 lease read,因为 jraft 的 readIndex 实现,性能足够满足一般要求,内部测试结果看吞吐接近 rpc 的极限

@fengjiachun fengjiachun added this to In progress in v1.2.4 via automation Mar 12, 2019
@fengjiachun fengjiachun added this to the 1.2.4 milestone Mar 12, 2019
@fengjiachun
Copy link
Contributor Author

fengjiachun commented Mar 12, 2019

新增了一个 LeaderLeaseTimeout, 默认是 0.9 个 ElectionTimeout , 在 readLeader() 时检查 lease,如果 lease timeout 了的话 就回退到 readIndex 读

想强调一下的是,不是很建议开启 lease read,
对于 lease read,所有努力终将在一个时钟漂移中白费,jraft 中默认不开启 lease read,因为 jraft 的 readIndex 实现,性能足够满足一般要求,内部测试结果看吞吐接近 rpc 的极限

@killme2008
Copy link
Contributor

@fengjiachun 建议同时描述下修复的思路。这里只描述了问题。

@fengjiachun
Copy link
Contributor Author

@killme2008 刚补充上

@killme2008
Copy link
Contributor

@fengjiachun 放到主贴里吧,刚好是问题和答案。

@@ -1266,6 +1274,22 @@ private void readLeader(ReadIndexRequest request, final ReadIndexResponse.Builde
}
}

private boolean checkLeaderLease(final Configuration conf, final long monotonicNowMs) {
final List<PeerId> peers = conf.listPeers();
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

使用 getPeers 即可,这里不需要有序的

private boolean checkLeaderLease(final Configuration conf, final long monotonicNowMs) {
final List<PeerId> peers = conf.listPeers();
int aliveCount = 0;
for (final PeerId peer : peers) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

总体上我对这个改动持保留态度,因为这个 for 循环会消耗相当多的 cpu,考虑到读是非常频繁的操作。但是从安全角度,可以暂时采用这个方案,后面可以考虑下将 check step down 频率加快的方式来实现。

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

哈哈,说的我也纠结了,我来去掉这个 for 循环

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@killme2008 已去掉这个循环

@ujjboy ujjboy added the enhancement New feature or request label Mar 12, 2019
@pifuant
Copy link
Contributor

pifuant commented Mar 12, 2019

我认为case存在问题, leader更新lease时, follower应该也要一致更新, 如果只是更新leader lease, 那么lease已经失去了它的作用

参考raft, 当leader收到大多数后, lease的截止时间为start+lease(start为rpc开始时间), 其实这里必须有个保证, follower也要在lease里保证当前leader有效, 即不参与投票产生新leader

我觉得你们实现方式有点搞复杂了, 同时, 可能丢失了safety。

@fengjiachun
Copy link
Contributor Author

我认为case存在问题, leader更新lease时, follower应该也要一致更新, 如果只是更新leader lease, 那么lease已经失去了它的作用

参考raft, 当leader收到大多数后, lease的截止时间为start+lease(start为rpc开始时间), 其实这里必须有个保证, follower也要在lease里保证当前leader有效, 即不参与投票产生新leader

我觉得你们实现方式有点搞复杂了, 同时, 可能丢失了safety。

@pifuant 感谢分析,jraft 就是像你上面说的方式处理的,只是 leader 在检查 leaseTimeout 是依靠定时器,有一点延时, 所以这个 issue 中静舟描述的case (双主)是可能发生的

return Utils.monotonicMs() - this.lastLeaderTimestamp < this.options.getLeaderLeaseTimeoutMs();
}

private boolean isElectionTimeout() {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

这个名字似乎不合适,如果是 timeout,这里的条件需要改为 >=,并且上层调用都加一个否定。

// To minimize the effects of clock drift, we should make sure that:
// stepDownTimeoutMs + leaderLeaseTimeoutMs < electionTimeout
final int stepDownTimeoutMs = (this.options.getElectionTimeoutMs() * Math.max(
100 - this.options.getLeaderLeaseTimeRatio(), 0)) >> 1;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

这里要除以100

final List<PeerId> peers = conf.listPeers();
int aliveCount = 0;
final Configuration deadNodes = new Configuration();
long minLastRpcSendTimestamp = Integer.MAX_VALUE;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

这里应该是 Long.max_value 吧

@fengjiachun
Copy link
Contributor Author

    // in read_lock
    private boolean isLeaderLeaseValid() {
        final long monotonicNowMs = Utils.monotonicMs();
        final int leaderLeaseTimeoutMs = this.options.getLeaderLeaseTimeoutMs();
        final boolean isLeaseValid = monotonicNowMs - this.lastLeaderTimestamp < leaderLeaseTimeoutMs;
        if (isLeaseValid) {
            return true;
        }

        final List<PeerId> peers = this.conf.getConf().getPeers();
        int aliveCount = 0;
        long startLease = Long.MAX_VALUE;
        for (final PeerId peer : peers) {
            if (peer.equals(this.serverId)) {
                aliveCount++;
                continue;
            }
            final long lastRpcSendTimestamp = this.replicatorGroup.getLastRpcSendTimestamp(peer);
            if (monotonicNowMs - lastRpcSendTimestamp <= leaderLeaseTimeoutMs) {
                aliveCount++;
                if (startLease > lastRpcSendTimestamp) {
                    startLease = lastRpcSendTimestamp;
                }
            }
        }
        if (aliveCount >= peers.size() / 2 + 1) {
            this.updateLastLeaderTimestamp(startLease);
        }

        return monotonicNowMs - this.lastLeaderTimestamp < leaderLeaseTimeoutMs;
    }

我还是更倾向于在 readLeader() 中检查 lease,以上代码是一个 lazy compute 的模式,只有在检查失败的情况下才需要更新 lease 并重新检查 lease;另外在 checkStepDown 定时器里面也会顺手更新一下 startLease(这个行为会对避免多余的for循环计算有很大的帮助),理论上 isLeaderLeaseValid() 中的 for 循环计算逻辑应该只有很小概率被触发, 一步 monotonicNowMs - this.lastLeaderTimestamp < leaderLeaseTimeoutMs 检查多数情况下应该就直接返回 true 了,这样的好处是避免了定时器调度的不确定性,更严谨安全,性能上也有兼顾

@pifuant
Copy link
Contributor

pifuant commented Mar 12, 2019

@fengjiachun 是的, 由于gc或者其他什么原因, 都可能导致定时器出问题, leader去check self, 大多数情况也是不可信赖的(更直白点, 唯一可信的就是majority表决通过的)

lease在一定的时钟偏移范围内, 是可以依赖的, 我有个疑问, 在leader广播并受到majority确认时, 可以生成lease的截止时间, 当需要确认leader是否有效时(比如lease read), 直接check lease不就可以了吗?

我可能还没领悟你们一些实现细节的优化, 希望不吝赐教, ^_^

@fengjiachun
Copy link
Contributor Author

@pifuant 感谢提供了一个很好的思路,依靠广播回调生成一个 startLease 应该是可以实现的,并且也能省去 read 路径上的 循环检查计算 的开销,但是也有一个问题,这个检查并不仅仅是读取每个 follower 的 lastRpcSendTimestamp,还有一个隐含的动作是读取当前有效的所有节点 this .this.conf.getConf().getPeers() , 我觉得可能不会很好实现,也增加了相当大的实现复杂度,
@killme2008 讨论下来我们觉得 leaseRead 不应该给 jraft 带来不必要的高复杂度,leaseRead 在 jraft 中暂时不被推荐设置,毕竟还解决不了时钟漂移的问题

以上,我觉得而这个 pr 的目标应该是:
1)少动现有代码,不增加复杂的逻辑;
2)当用户关闭 leaseRead 时没有额外的运行时开销
3)如果用户开启 leaseRead ,也要尽量提供好的性能


final List<PeerId> peers = this.conf.getConf().getPeers();
int aliveCount = 0;
long startLease = Long.MAX_VALUE;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

这块逻辑和 checkDeadNodes 中的部分代码太相似了,抽取一个方法吧,传入参数 boolean checkReplicator,true 的情况下做下 check,这样就可以抽取了。

@killme2008 killme2008 merged commit 29b8009 into develop Mar 13, 2019
v1.2.4 automation moved this from In progress to Done Mar 13, 2019
@killme2008 killme2008 deleted the fix/fix_stale_read branch March 13, 2019 10:56
This was referenced Mar 20, 2019
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
enhancement New feature or request
Projects
No open projects
v1.2.4
  
Done
Development

Successfully merging this pull request may close these issues.

None yet

4 participants