先上之前的图:
现在主要介绍数据同步的流程。但在此之前需要介绍很多的概念。
zookeeper数据同步流程及概念
zookeeper有三种差异同步
DIFF:就是当前的zxid大于最新的快照,但又没到leader最新的zxid,所以做法就是从这个zxid开始一直同步到最新的。
TRUNC:就是当前的zxid比leader最新的zxid都大,证明follower已经越界了,必须回滚到leader最新的zxid那个地方上去。
SNAP:就是当前的zxid比最新的SNAP要小,说明最新的快照都比follower新,所以直接就把最新的快照给follower,然后再通过committedLog一直同步到最新。
zookeeper同步完成后会发出NEWLEADER
leader主动向所有learner发送同步数据消息,每个learner有自己的发送队列,互不干扰。同步结束时,leader会向learner发送NEWLEADER指令,同时learner会反馈一个ACK。当leader接收到来自learner的ACK消息后,就认为当前learner已经完成了数据同步,同时进入“过半策略”等待阶段。当leader统计到收到了一半已上的ACK时,会向所有已经完成数据同步的learner发送一个UPTODATE指令,用来通知learner集群已经完成了数据同步,可以对外服务了。
zookeeper源码中几个类的作用
learnerHandler:线程中会根据follower状态选择发不同的数据同步包.主要是处理follower发过来的数据和即将发给follower的数据。
learner:会把自身的信息发送给leader然后leader会根据获取到的相关信息向follower发送需要做的事情。
learnerHandler中重要的几个变量的含义:
peerLastZxid : follower 最后处理的 zxid
minCommittedLog : leader committedLog 中最小的 zxid
maxCommittedLog : leader committedLog 中最大的 zxid
4种同步的方式
1、DIFF同步
即peerLastZxid介于minCommittedLog和maxCommittedLog间。
首先leader会向这个learner发送一个DIFF指令,用于通知learner即将把一些proposal同步给自己。实际同步过程中,针对每个proposal,leader都会通过发送两个数据包来完成,分别是PROPOSAL内容数据包和COMMIT指令数据包——这和zookeeper运行时leader和follower间的事务请求的提交过程是一致的。
举例,某时刻leader的提议缓存队列对应的ZXID依次是:0x500000001,0x500000002,0x500000003,0x500000004,0x500000005。而learner最后处理的ZXID为0x500000003,于是leader依次将0x500000004和0x500000005两个提议同步给learner。
2、TRUNC+DIFF同步
A,B,C三台机器,某一时刻B是leader,此时leader_epoch为5,同时当前已被集群大部分机器都提交的ZXID包括:0x500000001,0x500000002。此时leader正处理ZXID:0x500000003,并且已经将事务写入到了leader本地的事务日志中去——就在leader恰好要将该proposal发给其他follower进行投票时,leader挂了,proposal没被同步出去。此时集群进行新一轮leader选举,假设此次选的leader为A,leader_epoch变更为6,之后A和C又提交了0x600000001,0x600000002两个事务。此时B再次启动并开始数据同步。
简单讲,上面场景就是leader在已经将事务记录到本地事务日志中,但没有成功发起proposal流程时就挂了。
当leader发现某个learner包含一条自己没的事务记录,就让该learner进行事务回滚——回滚到leader上存在的,最接近peerLastZxid的ZXID,上面例子中leader会让learner回滚到ZXID为0x500000002的事务记录。
3、TRUNC同步
即peerLastZxid大于maxCommittedLog。
这种场景就是上述先回滚再差异化同步的简化模式,leader会要求learner回滚到ZXID值为maxCommitedLog对应的事务操作。
4、SNAP同步
这个时候分为两种情况。
1、peerLastZxid小于minCommittedLog 。即leader中最小的log的zxid都比peerLastZxid大。
2、leader上没有提议缓存队列,peerLastZxid不等于lastProcessedZxid(leader服务器数据恢复后得到的最大ZXID)
这两种情况下,只能进行全量同步。leader首先向learner发送一个SNAP指令,通知learner进行全量同步,随后leader会从内存数据库中获取到全量的数据节点和会话超时时间记录器,将它们序列化后传输给learner,learner接收到后对其反序列化后载入内存数据库中。
代码实现
数据同步代码之也在之前的文章贴出来过。leader中定位到方法:org.apache.zookeeper.server.quorum.LearnerHandler#run
这里由于代码太多只截取部分代码:
try {
//数据同步代码
rl.lock();
final long maxCommittedLog = leader.zk.getZKDatabase().getmaxCommittedLog();
final long minCommittedLog = leader.zk.getZKDatabase().getminCommittedLog();
LOG.info("Synchronizing with Follower sid: " + sid
+" maxCommittedLog=0x"+Long.toHexString(maxCommittedLog)
+" minCommittedLog=0x"+Long.toHexString(minCommittedLog)
+" peerLastZxid=0x"+Long.toHexString(peerLastZxid));
LinkedList<Proposal> proposals = leader.zk.getZKDatabase().getCommittedLog();
//follower的zxid与leader一样,发空的DIFF包
if (peerLastZxid == leader.zk.getZKDatabase().getDataTreeLastProcessedZxid()) {
// Follower is already sync with us, send empty diff
LOG.info("leader and follower are in sync, zxid=0x{}",
Long.toHexString(peerLastZxid));
packetToSend = Leader.DIFF;
zxidToSend = peerLastZxid;
} else if (proposals.size() != 0) {//如果有提议请求
LOG.debug("proposal size is {}", proposals.size());
//如果peerLastZxid介于leader的[minCommittedLog ,maxCommittedLog ]之间
if ((maxCommittedLog >= peerLastZxid)
&& (minCommittedLog <= peerLastZxid)) {
LOG.debug("Sending proposals to follower");
// as we look through proposals, this variable keeps track of previous
// proposal Id.
long prevProposalZxid = minCommittedLog;
// Keep track of whether we are about to send the first packet.
// Before sending the first packet, we have to tell the learner
// whether to expect a trunc or a diff
//发送包之前,要先告诉对端是diff还是trunc
boolean firstPacket=true;
// If we are here, we can use committedLog to sync with
// follower. Then we only need to decide whether to
// send trunc or not
packetToSend = Leader.DIFF;
zxidToSend = maxCommittedLog;
for (Proposal propose: proposals) {
// skip the proposals the peer already has
//发起的propose小于对端的peerLastZxid,说明对端已执行,直接略过
if (propose.packet.getZxid() <= peerLastZxid) {
prevProposalZxid = propose.packet.getZxid();
continue;
} else {
// If we are sending the first packet, figure out whether to trunc
// in case the follower has some proposals that the leader doesn't
if (firstPacket) {//如果我们发送的第一个包
firstPacket = false;
// Does the peer have some proposals that the leader hasn't seen yet
if (prevProposalZxid < peerLastZxid) {
// send a trunc message before sending the diff
packetToSend = Leader.TRUNC;
zxidToSend = prevProposalZxid;
updates = zxidToSend;
}
}
//发送PROPOSAL数据包
queuePacket(propose.packet);
QuorumPacket qcommit = new QuorumPacket(Leader.COMMIT, propose.packet.getZxid(),
null, null);
//发送commit数据包
queuePacket(qcommit);
}
}
} else if (peerLastZxid > maxCommittedLog) {//如果对端的zxid大于leader的maxCommittedLog,发送TRUNC指令
LOG.debug("Sending TRUNC to follower zxidToSend=0x{} updates=0x{}",
Long.toHexString(maxCommittedLog),
Long.toHexString(updates));
packetToSend = Leader.TRUNC;
zxidToSend = maxCommittedLog;
updates = zxidToSend;
} else {
LOG.warn("Unhandled proposal scenario");
}
} else {
// just let the state transfer happen
LOG.debug("proposals is empty");
}
LOG.info("Sending " + Leader.getPacketType(packetToSend));
leaderLastZxid = leader.startForwarding(this, updates);
} finally {
rl.unlock();
}
上面代码主要是判断leader应该发送什么样子的数据包。并发送出去。
然后follower会接受到包,并开始做相应的处理。定位到org.apache.zookeeper.server.quorum.Learner#syncWithLeader,由于代码量太大,所以依然只截取关键部分。
下面代码作用是发送newLeader 包,如果是SNAP,序列化snapshot文件,并且发送给learner。
if (qp.getType() == Leader.DIFF) {//表明跟leader有差别
LOG.info("Getting a diff from the leader 0x{}", Long.toHexString(qp.getZxid()));
snapshotNeeded = false;
}
else if (qp.getType() == Leader.SNAP) {//表明follower需要从leader同步快照
LOG.info("Getting a snapshot from leader 0x" + Long.toHexString(qp.getZxid()));
// The leader is going to dump the database
// clear our own database and read
zk.getZKDatabase().clear();
zk.getZKDatabase().deserializeSnapshot(leaderIs);
String signature = leaderIs.readString("signature");
if (!signature.equals("BenWasHere")) {
LOG.error("Missing signature. Got " + signature);
throw new IOException("Missing signature");
}
zk.getZKDatabase().setlastProcessedZxid(qp.getZxid());
} else if (qp.getType() == Leader.TRUNC) {//事务包,代表需要回滚事务。
//we need to truncate the log to the lastzxid of the leader
LOG.warn("Truncating log to get in sync with the leader 0x"
+ Long.toHexString(qp.getZxid()));
boolean truncated=zk.getZKDatabase().truncateLog(qp.getZxid());
if (!truncated) {
// not able to truncate the log
LOG.error("Not able to truncate the log "
+ Long.toHexString(qp.getZxid()));
System.exit(13);
}
zk.getZKDatabase().setlastProcessedZxid(qp.getZxid());
}
else {
LOG.error("Got unexpected packet from leader "
+ qp.getType() + " exiting ... " );
System.exit(13);
}
现在数据同步也同步了。但是什么时候结束呢?follower怎么知道leader发完了呢?这就是NEWLEADER的作用了。
代码还是定位到org.apache.zookeeper.server.quorum.LearnerHandler#run中。截取NEWLEADER部分代码。
QuorumPacket newLeaderQP = new QuorumPacket(Leader.NEWLEADER,
ZxidUtils.makeZxid(newEpoch, 0), null, null);
if (getVersion() < 0x10000) {
oa.writeRecord(newLeaderQP, "packet");
} else {
queuedPackets.add(newLeaderQP);
}
bufferedOutput.flush();
//Need to set the zxidToSend to the latest zxid
if (packetToSend == Leader.SNAP) {
zxidToSend = leader.zk.getZKDatabase().getDataTreeLastProcessedZxid();
}
oa.writeRecord(new QuorumPacket(packetToSend, zxidToSend, null, null), "packet");
bufferedOutput.flush();
/* if we are not truncating or sending a diff just send a snapshot */
if (packetToSend == Leader.SNAP) {
LOG.info("Sending snapshot last zxid of peer is 0x"
+ Long.toHexString(peerLastZxid) + " "
+ " zxid of leader is 0x"
+ Long.toHexString(leaderLastZxid)
+ "sent zxid of db as 0x"
+ Long.toHexString(zxidToSend));
// Dump data to peer
leader.zk.getZKDatabase().serializeSnapshot(oa);
oa.writeString("BenWasHere", "signature");
}
bufferedOutput.flush();
启用一个线程去发送数据包
new Thread() {
public void run() {
Thread.currentThread().setName(
"Sender-" + sock.getRemoteSocketAddress());
try {
sendPackets();
} catch (InterruptedException e) {
LOG.warn("Unexpected interruption",e);
}
}
}.start();
等待learner的ACK
qp = new QuorumPacket();
ia.readRecord(qp, "packet");
if(qp.getType() != Leader.ACK){
LOG.error("Next packet was supposed to be an ACK");
return;
}
LOG.info("Received NEWLEADER-ACK message from " + getSid());
leader.waitForNewLeaderAck(getSid(), qp.getZxid());
而在learner中接到NEWLEADER指令也需要处理。依然定位到org.apache.zookeeper.server.quorum.Learner#syncWithLeader截取部分代码:
case Leader.NEWLEADER: // Getting NEWLEADER here instead of in discovery
// means this is Zab 1.0
// Create updatingEpoch file and remove it after current
// epoch is set. QuorumPeer.loadDataBase() uses this file to
// detect the case where the server was terminated after
// taking a snapshot but before setting the current epoch.
//创建一个updating文件名
File updating = new File(self.getTxnFactory().getSnapDir(),
QuorumPeer.UPDATING_EPOCH_FILENAME);
//非法判断
if (!updating.exists() && !updating.createNewFile()) {
throw new IOException("Failed to create " +
updating.toString());
}
//判断是否需要载入快照
if (snapshotNeeded) {
zk.takeSnapshot();
}
self.setCurrentEpoch(newEpoch);
//非法判断
if (!updating.delete()) {
throw new IOException("Failed to delete " +
updating.toString());
}
//在此之后的任何操作都需要转到事务日志,而不是直接应用到内存中,证明已经同步完成了
writeToTxnLog = true; //Anything after this needs to go to the transaction log, not applied directly in memory
isPreZAB1_0 = false;
//发送ack报文
writePacket(new QuorumPacket(Leader.ACK, newLeaderZxid, null, null), true);
break;
}
可能比较好奇的是LearnerHandler是从哪里加入启动的。代码定位到org.apache.zookeeper.server.quorum.Leader.LearnerCnxAcceptor#run中。
@Override
public void run() {
try {
while (!stop) {
try{
Socket s = ss.accept();
// start with the initLimit, once the ack is processed
// in LearnerHandler switch to the syncLimit
s.setSoTimeout(self.tickTime * self.initLimit);
s.setTcpNoDelay(nodelay);
BufferedInputStream is = new BufferedInputStream(
s.getInputStream());
//开启LearnerHandler线程
LearnerHandler fh = new LearnerHandler(s, is, Leader.this);
fh.start();
} catch (SocketException e) {
if (stop) {
LOG.info("exception while shutting down acceptor: "
+ e);
// When Leader.shutdown() calls ss.close(),
// the call to accept throws an exception.
// We catch and set stop to true.
stop = true;
} else {
throw e;
}
} catch (SaslException e){
LOG.error("Exception while connecting to quorum learner", e);
}
}
} catch (Exception e) {
LOG.warn("Exception while accepting follower", e);
}
}
而往前面定位会发现在org.apache.zookeeper.server.quorum.Leader#lead中有代码:
// Start thread that waits for connection requests from
// new followers.
//开个线程等follower连接进来.LearnerHandler线程就是在这里启动的
cnxAcceptor = new LearnerCnxAcceptor();
cnxAcceptor.start();
所以还是在leader.lead();方法中产生的。
当然还有很多其它的包定义。这里就去百度了。目前就把源码中的定义所有的类型复制出来。
/**
* This message is for follower to expect diff
*/
final static int DIFF = 13;
/**
* This is for follower to truncate its logs
*/
final static int TRUNC = 14;
/**
* This is for follower to download the snapshots
*/
final static int SNAP = 15;
/**
* This tells the leader that the connecting peer is actually an observer
*/
final static int OBSERVERINFO = 16;
/**
* This message type is sent by the leader to indicate it's zxid and if
* needed, its database.
*/
final static int NEWLEADER = 10;
/**
* This message type is sent by a follower to pass the last zxid. This is here
* for backward compatibility purposes.
*/
final static int FOLLOWERINFO = 11;
/**
* This message type is sent by the leader to indicate that the follower is
* now uptodate andt can start responding to clients.
*/
final static int UPTODATE = 12;
/**
* This message is the first that a follower receives from the leader.
* It has the protocol version and the epoch of the leader.
*/
public static final int LEADERINFO = 17;
/**
* This message is used by the follow to ack a proposed epoch.
*/
public static final int ACKEPOCH = 18;
/**
* This message type is sent to a leader to request and mutation operation.
* The payload will consist of a request header followed by a request.
*/
final static int REQUEST = 1;
/**
* This message type is sent by a leader to propose a mutation.
*/
public final static int PROPOSAL = 2;
/**
* This message type is sent by a follower after it has synced a proposal.
*/
final static int ACK = 3;
/**
* This message type is sent by a leader to commit a proposal and cause
* followers to start serving the corresponding data.
*/
final static int COMMIT = 4;
/**
* This message type is enchanged between follower and leader (initiated by
* follower) to determine liveliness.
*/
final static int PING = 5;
/**
* This message type is to validate a session that should be active.
*/
final static int REVALIDATE = 6;
/**
* This message is a reply to a synchronize command flushing the pipe
* between the leader and the follower.
*/
final static int SYNC = 7;
/**
* This message type informs observers of a committed proposal.
*/
final static int INFORM = 8;
至此,集群初始化中的数据同步就算完成了,总的来说概念比较多也比较杂,但是总的逻辑是没有什么问题的。建议还是调试源码,会理解得更加的深刻。