Versions Compared

Key

  • This line was added.
  • This line was removed.
  • Formatting was changed.
Comment: Fix dead link of "DISSECTING ZAB"; UPDATE -> UPTODATE

...

Zab at a high level is a leader based protocol similar to Paxos. Some of the aspects that distinguish Zab from Paxos is that Zab recovers histories rather than single instances of protocols; Zab has prefix ordering properties to allow primary/backup implementations to have multiple outstanding operations outstanding at a time; and Zab does special processing at leadership change to guarantee unique sequence numbers for values proposed by a leader.

The following structures are used for the initial handshake between client and server:

Code Block
    class LearnerInfo {
        int protocolVersion;
        int serverid;
    }

The actual proposals are communicated Apart from the state transfer, all communication between follower and leader is done using a single data structure:

Code Block

class QuorumPacket {
    int type; // Request, Ack, Commit, Ping, etc
    long zxid;
    buffer data;
    vector<org.apache.zookeeper.data.Id> authinfo; // only used for requests
}

The following structure is used the SERVERINFO packet:

Code Block

class Info {
    long sid; // server id
    int version; // protocol version
    int currentEpoch;
    int acceptedEpoch;
}

There are a couple of different types of packetsHere are the types of the QuorumPackets:

type

zxid

data

notation

meaning

SERVERINFO

FOLLOWERINFO(11)

aka FOLLOWERINFO

last zxid accepted

acceptedEpoch

Info

LearnerInfo

SERVERINFO

FOLLOWERINFO(

lastZxid, info

acceptedEpoch)

The follower has accepted

up to lastZxid.

NEWEPOCH(9)

newEpoch << 32

n/a

NEWEPOCH(newEpoch)

A proposal to move to a new epoch

epoch acceptedEpoch.

DIFF(13)

last committed zxid

n/a

DIFF(lastCommittedZxid)

lastCommittedZxid is the last zxid committed by the leader.

TRUNC(14)

truncZxid

n/a

TRUNC(truncZxid)

Truncate the history to truncZxid

SNAP(15)

n/a

lastZxid

n/a

SNAP

A state transfer (aka snapshot) will be sent to the follower. this will be a fuzzy state transfer that may include zxids being sent to the follower. The state transfer will immediately follow this packet.

OBSERVERINFO(16)

last zxid

accepted

learned

n/a

LearnerInfo

OBSERVERINFO(lastZxid)

The observer has accepted up to lastZxid.

LEADERINFO(17)

proposed epoch

protocol

LEADERINFO(e)

The new proposed epoch

ACKEPOCH(18)

lastZxid

currentEpoch

ACKEPOCH(lastZxid, currentEpoch)

Acknowledge the acceptance of the new epoch. If the follower has already acknowledged the given epoch, it passes -1 as the currentEpoch to signal that the new epoch is not acknowledged. (We still need to send lastZxid for syncing if necessary)

NEWLEADER(10)

lastZxid

e << 32

n/a

NEWLEADER(

lastZxid

e)

Accept this leader as the leader of the epoch

of lastZxid

e.

UPTODATE(12)

n/a

n/a

UPTODATE

The follower is now uptodate enough to begin serving clients.

PROPOSAL(2)

zxid of proposal

proposed message

PROPOSAL(zxid, data)

Propose a message. (Request that it be accepted into a followers history.)

ACK(3)

zxid of proposal to ack

n/a

ACK(zxid)

Everything sent to the follower by the leader up to zxid has been accepted into its history (logged to disk).

COMMIT(4)

zxid of proposal to commit

n/a

COMMIT(zxid)

Everything in the followers history up to zxid should be committed (aka delivered).

INFORM(8)

zxid of proposal

data of proposal

INFORM(zxid, data)

Deliver the data. (Only used with observers.)

Zab servers have the following state:

Name

Meaning

history

an on disk log of proposals accepted

lastZxid

zxid of the last proposal in the history

acceptedEpoch

the epoch number of the last NEWEPOCH packet accepted

currentEpoch

the epoch number of the last NEWLEADER packet accepted

Implementation assumptions

...

  1. l The leader starts accepting connections from followers.
  2. f Followers connect the the leader and send SERVERINFO(lastZxid, info)FOLLOWERINFO.
  3. l The leader sends SERVERINFO(lastZxid, info) to followers that connect.l Once the leader has quorum, it stops accepting connections, and sends NEWEPOCHLEADERINFO(e) to all followers, where e is greater than all f.acceptedEpoch in the quorum.
  4. f Followers send ACK(e<<32) When the follower receives LEADERINFO(e) it will do one of the following:
    • if e >
    f.acceptedEpoch. (If e <
    • f.acceptedEpoch, the follower
    will abandon the leader. If
    • sets f.acceptedEpoch = e and sends ACKEPOCH(e);
    • if e == f.acceptedEpoch, the follower
    will
    • does not send
    ACK(e<<32), but will still maintain
    • ACKEPOCH, but continues to next step;
    • if e < f.acceptedEpoch, the follower closes the connection to the
    leader.)
    • leader and goes back to leader election;
  5. l The leader waits for all a quorum of followers to ack NEWEPOCH(e)send ACKEPOCH.
  6. l If the following conditions are not met for all connected followers, the leader disconnects followers and goes back to leader election:
    • f.currentEpoch <= l.currentEpoch
    • if f.currentEpoch == l.currentEpoch, then f.lastZxid <= l.lastZxid

...

There are two noticeable differences between this description of phase 1 and the one found in http://research.yahoo.com/files/YL-2010-007.pdf:Image RemovedDISSECTING ZAB

  1. the leader does not sync with the most up-to-date follower
  2. the followers do not send their histories in the ACK of the NEWEPOCH message.

The leader election protocols that we use try to guarantee that the elected leader has the most up-to-date history from a quorum of processes. It They also tries try to guarantee that there is only one leader elected. It is possible for these guarantees not to hold due to race conditions, stale information, or perhaps even implementation error, so we make sure that the guarantees are satisfied in phase 1. In the tech report we do not assume that the elected leader has the most up-to-date history, so we must find the most up-to-date follower and sync with it. However since the elected leader should be the most up-to-date process, using the leader election protocols that we use, we just check the condition in step 7. If the leader meets the conditions of step 7 we are the most up-to-date, so we have implicitly synced with the most up-to-date process in the quorum. If it doesn't, we go back to leader election.In this description and in our implementation we use the notion of connections which map to TCP connections. A follower will only be connected to a single leader at a time. Also, a follower only changes its history under the direction of a leader. The phase 1 described in the protocol, sends its history in the ACK to NEWEPOCH, but we do not send the history in step 5. Since we just need to verify that the leader has a super set of history of the follower, we only need to know the lastZxid in the history. The lastZxid is sent in step 2 in the protocol described here rather than step 5. However if the follower executes step 5 and sends the ACK to the leader, it means that for the whole time period between step 2 and 5 it has been connected to the leader. Since the follow can only be connected to one leader at a time, and since it has been connect to the same leader from step 2 to 5, and since that leader has not asked the follower to change its history, then the lastZxid sent in step 2 is still the same lastZxid in step 5, so there is no reason to resend it.

Another difference from the protocol described in the tech report is the extra information sent in SERVERINFO LearnerInfo and the SERVERINFO sent by the leader in step 3FollowerInfo. The extra information is for implementation purposes: making sure that the protocol version is compatible between client and server, and information for debugging purposes.

...

  1. l The leader does the following with each follower connected to it:
    1. adds the follower to the list of connections to send new proposals, so while the server is performing the next steps, it is queuing up any new proposals sent to the follower.
    2. does one of the following:
      • SNAP if the follower is so far behind that it is better to do a state transfer than send missing transactions.
      • TRUNC(zxid) if the follower has transactions that the leader has chosen to skip. The leader sets zxid to the last zxid in its history for the epoch of the follower. The leader then sends the transactions that the follower is missing.
      • DIFF if the leader is sending transactions that the follower is missing. The leader sends missing messages to the follower.
    3. sends a NEWLEADER(e);
    4. The leader releases any queued messages to the follower.
  2. f The follower syncs with the leader, but doesn't modify its state until it receives the NEWLEADER(e) packet. Once it does receives NEWLEADER(e) it atomically applies any changes and the new state and sets f.currentEpoch = e. It then sends ACK(e << 32).
  3. l Once the leader has received an acknowledgements from a quorum of followers, it takes leadership of epoch e and queues UPTODATE to all followers.
  4. f When a follower receives the UPDATE UPTODATE message, it starts accepting client connections and serving new state.
  5. l Leader starts accepting connections from followers again. The variable nextZxid is set to (e << 32) + 1.

Phase 3: Broadcast

The leader and followers can have multiple proposals in process and at various stages in the pipeline. The leader will remain active as long as there is a quorum of followers acknowledging its proposals or pings within a timeout interval. The follower will continue to support a leader as long as it receives proposals or pings within a timeout interval. Any failures or timeouts will result in the server going back to leader election.

  1. l the leader queues a packet PROPOSE(zxid, data), where zxid = nextZxid, to all connected followers. It increments nextZxid.
  2. f a follower will log and sync proposals to disk and send ACK(zxid).
  3. l when the leader receives acknowledgements from a quorum of followers it queues COMMIT(zxid) to all followers.