成员变更在一致性协议里稍复杂一些,由于不同的成员不可能在同一时刻从旧成员组切换至新成员组,所以可能出现两个不相交的majority,从而导致同一个term出现两个leader,进而导致同一个index的日志不一致,违反一致性协议。下图是个例子:

raft作者提出了一种比较简单的方法,一次只增加或减少一个成员,这样能够保证任何时刻,都不可能出现两个不相交的majority,所以,可以从旧成员组直接切到新成员组。如下图:

切换的时机是把成员变更日志写盘的时候,不管是否commit。这个切换时机带来的问题是如果这条成员变更日志最终没有commit,在发生leader切换的时候,成员组就需要回滚到旧的成员组。

etcd raft为了实现简单,将切换成员组的实机选在apply成员变更日志的时候。

下面看看etcd raft library如何实现的:

应用调用

func (n *node) ProposeConfChange(ctx context.Context, cc pb.ConfChange) error {
data, err := cc.Marshal()
if err != nil {
return err
}
return n.Step(ctx, pb.Message{Type: pb.MsgProp, Entries: []pb.Entry{{Type: pb.EntryConfChange, Data: data}}})
}

可以看出,ConfChange是和普通的log entry一样封装在MsgProp消息中,进入propc,

跑raft算法的goroutine从propc中拿到消息后,会做如下判断:

for i, e := range m.Entries {
if e.Type == pb.EntryConfChange {
if r.pendingConf {
r.logger.Infof("propose conf %s ignored since pending unapplied configuration", e.String())
m.Entries[i] = pb.Entry{Type: pb.EntryNormal}
}
r.pendingConf = true
}
}

检查已经有成员变更正在做,就忽略新的成员变更。然后将pendingConf置为true,意味着目前有成员变更正在做了,从这里可以看出,多个成员变更不能同时进行。follower接收端的处理和普通log entry一样。

如果成员变更日志达成了一致,则会被封装在Ready中,应用拿到后,做如下处理:

if entry.Type == raftpb.EntryConfChange {
var cc raftpb.ConfChange
cc.Unmarshal(entry.Data)
s.Node.ApplyConfChange(cc)
}

ApplyConfChange:

func (n *node) ApplyConfChange(cc pb.ConfChange) *pb.ConfState {
var cs pb.ConfState
select {
case n.confc <- cc:
case <-n.done:
}
select {
case cs = <-n.confstatec:
case <-n.done:
}
return &cs
}

讲ConfChange放入confc,然后阻塞在confstatec上,跑raft协议的goroutine从confc中拿出ConfChange,做相应的增加/删除节点操作,然后将成员组放入confstatec。

switch cc.Type {
case pb.ConfChangeAddNode:
r.addNode(cc.NodeID)
case pb.ConfChangeRemoveNode:
// block incoming proposal when local node is
// removed
if cc.NodeID == r.id {
propc = nil
}
r.removeNode(cc.NodeID)
case pb.ConfChangeUpdateNode:
r.resetPendingConf()
default:
panic("unexpected conf type")
}
select {
case n.confstatec <- pb.ConfState{Nodes: r.nodes()}:
case <-n.done:
}

增加/删除节点操作都只是更新prs,map的每个元素保存一个peer的状态,其中最重要的状态莫过于

Match, Next uint64

看过raft小论文的人一看变量名就很明确意义,Match代表最大的已经落盘的log index,Next代表下一条需要发给这个peer的log index。然后将pendingConf置为false,代表成员变更结束。

重启如何恢复成员组:

hs, cs, err := c.Storage.InitialState()

Storage接口中:

// InitialState returns the saved HardState and ConfState information.
InitialState() (pb.HardState, pb.ConfState, error)

Storage是个接口,其中InitialState()用于恢复成员组,需要应用自己实现,通常将ConfState记在最后一次Snapshot的Metadata中:

message SnapshotMetadata {
optional ConfState conf_state = 1 [(gogoproto.nullable) = false];
optional uint64 index = 2 [(gogoproto.nullable) = false];
optional uint64 term = 3 [(gogoproto.nullable) = false];
}

ConfState:

message ConfState {
repeated uint64 nodes = 1;
}

拿到ConfState后就可以初始化上面提到的prs,snapshot后续的已经commit的log entry一样,通过Ready封装,应用进行apply,如果其中有ConfChange,则调用

s.Node.ApplyConfChange(cc)

etcd raft如何实现成员变更的更多相关文章

  1. 解读Raft(四 成员变更)

    将成员变更纳入到算法中是Raft易于应用到实践中的关键,相对于Paxos,它给出了明确的变更过程(实践的基础,任何现实的系统中都会遇到因为硬件故障等原因引起的节点变更的操作). 显然,我们可以通过sh ...

  2. 【翻译】Raft 共识算法:集群成员变更

    转载请注明出处:https://www.cnblogs.com/morningli/p/16770129.html 之前都在集群配置是固定的(参与共识算法的server集合)假设下讨论raft.在实践 ...

  3. etcd raft library设计原理和使用

    早在2013年11月份,在raft论文还只能在网上下载到草稿版时,我曾经写过一篇blog对其进行简要分析.4年过去了,各种raft协议的讲解铺天盖地,raft也确实得到了广泛的应用.其中最知名的应用莫 ...

  4. etcd raft library

    https://github.com/coreos/etcd/tree/master/raft import "github.com/coreos/etcd/raft" ----- ...

  5. 彻底搞懂etcd raft选举、数据同步

    etcd raft选举机制 etcd 是一个分布式的k/V存储系统.核心使用了RAFT分布式一致性协议.一致性这个概念,它是指多个服务器在状态达成一致,但是在一个分布式系统中,因为各种意外可能,有的服 ...

  6. etcd raft 处理流程图系列1-raftexample

    最近在看raft相关的代码和实现,发现etcd的raft模块在实现上还是比较灵活的,但缺点就是需要用户实现比较多的功能,如存储和网络等,同时带来的优点就是不会对用户的存储和传输作限制.网上对该模块的描 ...

  7. etcd raft如何实现leadership transfer

    leadership transfer可以把raft group中的leader身份转给其中一个follower.这个功能可以用来做负载均衡,比如可以把leader放在性能更好的机器或者离客户端更近的 ...

  8. etcd raft 处理流程图系列3-wal的存储和运行

    存储和节点的创建 raftexample中的存储其实有两种,一个是通过raft.NewMemoryStorage()进行创建的raft.raftStorage,关联到单个raft节点,另一个是通过ne ...

  9. etcd raft如何实现Linearizable Read

    Linearizable Read通俗来讲,就是读请求需要读到最新的已经commit的数据,不会读到老数据. 对于使用raft协议来保证多副本强一致的系统中,读写请求都可以通过走一次raft协议来满足 ...

随机推荐

  1. java基础(六章)

    一.for循环的使用场合 l  while循环--先判断,再循环 while(1.条件表达式){      //2.循环操作      //3.更改循环条件表达式 } l  do-while--先循环 ...

  2. typecho for SAE

    url:http://cloudbbs.org/forum.php?mod=viewthread&tid=22817 typecho和wordpress差不多,目前使用的用户非常之多.这里分享 ...

  3. pod trunk push --verbose 失败的原因总结

    用 pod trunk push --verbose  添加一个 pod 的时候,经常出现如下的错误 [!] The podspec does not validate. /Library/Ruby/ ...

  4. CSS 简单了解(二)

    我们第一天说了简单的HTML,第二天说了简单的CSS.那么今天.咱们就来说一说他们的结合如何使用吧! 首先说引用方式,和使用方法吧! 1.内部样式表.(放入<head>中) <hea ...

  5. webpack实用配置

    前面的话 上文介绍了webpack入门,本文将详细介绍webpack实用配置 版本号 以entry.js打包为bundle.js为例,出口的filename可以设置为[id].[name].[hash ...

  6. ASP.NET MVC5+EF6+EasyUI 后台管理系统(84)-Quartz 作业调度用法详解一

    前言 我从Quartz2.0开始使用,并对其进行了封装了界面,可以参考 http://www.cnblogs.com/ymnets/p/5065154.html 最近拿出来进行了优化,并升级到最新版, ...

  7. Web设计思想——渐进增强

    最近在拜读一本Web体验相关的书<渐进增强--跨平台用户体验设计 >,阅读后做些总结,消化一下书中的精髓. 在阅读本文前,可以先思考下面几个问题. 1. 浏览网页的目的是什么? 2. 浏览 ...

  8. An impassioned circulation of affection

    An impassioned circulation of affection time limit per test 2 seconds memory limit per test 256 mega ...

  9. 记一次使用搬瓦工VPS的经历

    自己因为有需求上Google,以前是通过修改hosts的方法实现访问Google,但是最近不知道为什么改hosts后还是无法访问Google,于是决定搭建VPS来实现科学上网,看了一下价格,作为穷逼学 ...

  10. Testlink安装步骤Checking if C:\inetpub\wwwroot\testlink-1.9.3\gui\templates_c directory is writable Failed !

    Testlink安装过程中问题现象: Checking if C:\inetpub\wwwroot\testlink-1.9.3\gui\templates_c directory is writab ...