成员变更在一致性协议里稍复杂一些,由于不同的成员不可能在同一时刻从旧成员组切换至新成员组,所以可能出现两个不相交的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. Spark踩坑记——从RDD看集群调度

    [TOC] 前言 在Spark的使用中,性能的调优配置过程中,查阅了很多资料,之前自己总结过两篇小博文Spark踩坑记--初试和Spark踩坑记--数据库(Hbase+Mysql),第一篇概况的归纳了 ...

  2. collections模块方法详解

    collections是Python内建的一个集合模块,提供了许多有用的集合类. namedtuple 我们知道tuple可以表示不变集合,例如,一个点的二维坐标就可以表示成: >>> ...

  3. QT调用百度语音REST API实现语音合成

    QT调用百度语音REST API实现语音合成 1.首先点击点击链接http://yuyin.baidu.com/docs/tts 点击access_token,获取access_token,里面有详细 ...

  4. css清除浮动的八大方法

    清除浮动是每一个 web前台设计师必须掌握的机能.css清除浮动大全,共8种方法. 浮动会使当前标签产生向上浮的效果,同时会影响到前后标签.父级标签的位置及 width height 属性.而且同样的 ...

  5. iOS,Android,Jave后台AES加密解密

    AES256 在iOS和Android上的相关代码: http://www.tuicool.com/articles/RVFbmmU 里面可以下载相关的代码. 我们遇到的问题是: 把Android的代 ...

  6. 移动平台Unity3D 应用性能优化

    WeTest 导读 做了大概半年多VR应用了,VR由于双眼double渲染的原因,对性能的优化要求比较高,在项目的进展过程中,总结了一些关于移动平台上Unity3D的性能优化经验,供分享. 一.移动平 ...

  7. vue.js实现数据动态响应(Vue.set的应用)

    在vue里面,我们操作最多的就是各种数据,在jquery里面,我们习惯通过下标定向找到数据,然后重新赋值 比如var a[0]=111;(希望上家公司原谅菜鸟的我写了不少这样的代码

  8. python 标准库 -- subprocess

    subprocess 主要功能室执行外部的命令和程序 一个进程可 fork 一个子进程, 并让这个子进程 exec 另外一个程序. 在 python 中, 可以通过标准库中的 subprocess 包 ...

  9. java递归算法实现 数字人民币大写转换

    最近穷死了 ,没钱吃饭ing 写点钱给自己吧!public class Test{ public static String getChar(long a){ int b = (int)a; Map ...

  10. memcache基础

    一.Memcache是一种缓存技术(内存),你可以把它想像成一张巨大的内存表,形式如下[它就是一个服务] key value key值(字符串) 可以放(字符串[二进制数据[视频.音频.图片]],数值 ...