|
@@ -88,8 +88,7 @@ public class Leader {
|
|
|
LOG.info(MAX_CONCURRENT_SNAPSHOT_TIMEOUT + " = " + maxConcurrentSnapshotTimeout);
|
|
|
}
|
|
|
|
|
|
- private final LearnerSnapshotThrottler learnerSnapshotThrottler =
|
|
|
- new LearnerSnapshotThrottler(maxConcurrentSnapshots, maxConcurrentSnapshotTimeout);
|
|
|
+ private final LearnerSnapshotThrottler learnerSnapshotThrottler;
|
|
|
|
|
|
final LeaderZooKeeperServer zk;
|
|
|
|
|
@@ -111,6 +110,12 @@ public class Leader {
|
|
|
return proposalStats;
|
|
|
}
|
|
|
|
|
|
+ public LearnerSnapshotThrottler createLearnerSnapshotThrottler(
|
|
|
+ int maxConcurrentSnapshots, long maxConcurrentSnapshotTimeout) {
|
|
|
+ return new LearnerSnapshotThrottler(
|
|
|
+ maxConcurrentSnapshots, maxConcurrentSnapshotTimeout);
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
* Returns a copy of the current learner snapshot
|
|
|
*/
|
|
@@ -207,7 +212,7 @@ public class Leader {
|
|
|
/**
|
|
|
* Returns true if a quorum in qv is connected and synced with the leader
|
|
|
* and false otherwise
|
|
|
- *
|
|
|
+ *
|
|
|
* @param qv, a QuorumVerifier
|
|
|
*/
|
|
|
public boolean isQuorumSynced(QuorumVerifier qv) {
|
|
@@ -223,7 +228,7 @@ public class Leader {
|
|
|
}
|
|
|
return qv.containsQuorum(ids);
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
private final ServerSocket ss;
|
|
|
|
|
|
Leader(QuorumPeer self,LeaderZooKeeperServer zk) throws IOException {
|
|
@@ -248,6 +253,8 @@ public class Leader {
|
|
|
throw e;
|
|
|
}
|
|
|
this.zk = zk;
|
|
|
+ this.learnerSnapshotThrottler = createLearnerSnapshotThrottler(
|
|
|
+ maxConcurrentSnapshots, maxConcurrentSnapshotTimeout);
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -342,17 +349,17 @@ public class Leader {
|
|
|
* This message type informs observers of a committed proposal.
|
|
|
*/
|
|
|
final static int INFORM = 8;
|
|
|
-
|
|
|
+
|
|
|
/**
|
|
|
* Similar to COMMIT, only for a reconfig operation.
|
|
|
*/
|
|
|
final static int COMMITANDACTIVATE = 9;
|
|
|
-
|
|
|
+
|
|
|
/**
|
|
|
* Similar to INFORM, only for a reconfig operation.
|
|
|
*/
|
|
|
final static int INFORMANDACTIVATE = 19;
|
|
|
-
|
|
|
+
|
|
|
final ConcurrentMap<Long, Proposal> outstandingProposals = new ConcurrentHashMap<Long, Proposal>();
|
|
|
|
|
|
private final ConcurrentLinkedQueue<Proposal> toBeApplied = new ConcurrentLinkedQueue<Proposal>();
|
|
@@ -415,9 +422,9 @@ public class Leader {
|
|
|
long epoch = -1;
|
|
|
boolean waitingForNewEpoch = true;
|
|
|
|
|
|
- // when a reconfig occurs where the leader is removed or becomes an observer,
|
|
|
+ // when a reconfig occurs where the leader is removed or becomes an observer,
|
|
|
// it does not commit ops after committing the reconfig
|
|
|
- boolean allowedToCommit = true;
|
|
|
+ boolean allowedToCommit = true;
|
|
|
/**
|
|
|
* This method is main function that is called to lead
|
|
|
*
|
|
@@ -467,20 +474,20 @@ public class Leader {
|
|
|
QuorumVerifier curQV = self.getQuorumVerifier();
|
|
|
if (curQV.getVersion() == 0 && curQV.getVersion() == lastSeenQV.getVersion()) {
|
|
|
// This was added in ZOOKEEPER-1783. The initial config has version 0 (not explicitly
|
|
|
- // specified by the user; the lack of version in a config file is interpreted as version=0).
|
|
|
+ // specified by the user; the lack of version in a config file is interpreted as version=0).
|
|
|
// As soon as a config is established we would like to increase its version so that it
|
|
|
// takes presedence over other initial configs that were not established (such as a config
|
|
|
- // of a server trying to join the ensemble, which may be a partial view of the system, not the full config).
|
|
|
+ // of a server trying to join the ensemble, which may be a partial view of the system, not the full config).
|
|
|
// We chose to set the new version to the one of the NEWLEADER message. However, before we can do that
|
|
|
// there must be agreement on the new version, so we can only change the version when sending/receiving UPTODATE,
|
|
|
- // not when sending/receiving NEWLEADER. In other words, we can't change curQV here since its the committed quorum verifier,
|
|
|
- // and there's still no agreement on the new version that we'd like to use. Instead, we use
|
|
|
+ // not when sending/receiving NEWLEADER. In other words, we can't change curQV here since its the committed quorum verifier,
|
|
|
+ // and there's still no agreement on the new version that we'd like to use. Instead, we use
|
|
|
// lastSeenQuorumVerifier which is being sent with NEWLEADER message
|
|
|
- // so its a good way to let followers know about the new version. (The original reason for sending
|
|
|
+ // so its a good way to let followers know about the new version. (The original reason for sending
|
|
|
// lastSeenQuorumVerifier with NEWLEADER is so that the leader completes any potentially uncommitted reconfigs
|
|
|
- // that it finds before starting to propose operations. Here we're reusing the same code path for
|
|
|
+ // that it finds before starting to propose operations. Here we're reusing the same code path for
|
|
|
// reaching consensus on the new version number.)
|
|
|
-
|
|
|
+
|
|
|
// It is important that this is done before the leader executes waitForEpochAck,
|
|
|
// so before LearnerHandlers return from their waitForEpochAck
|
|
|
// hence before they construct the NEWLEADER message containing
|
|
@@ -488,24 +495,24 @@ public class Leader {
|
|
|
try {
|
|
|
QuorumVerifier newQV = self.configFromString(curQV.toString());
|
|
|
newQV.setVersion(zk.getZxid());
|
|
|
- self.setLastSeenQuorumVerifier(newQV, true);
|
|
|
+ self.setLastSeenQuorumVerifier(newQV, true);
|
|
|
} catch (Exception e) {
|
|
|
throw new IOException(e);
|
|
|
}
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
newLeaderProposal.addQuorumVerifier(self.getQuorumVerifier());
|
|
|
if (self.getLastSeenQuorumVerifier().getVersion() > self.getQuorumVerifier().getVersion()){
|
|
|
newLeaderProposal.addQuorumVerifier(self.getLastSeenQuorumVerifier());
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
// We have to get at least a majority of servers in sync with
|
|
|
// us. We do this by waiting for the NEWLEADER packet to get
|
|
|
// acknowledged
|
|
|
-
|
|
|
+
|
|
|
waitForEpochAck(self.getId(), leaderStateSummary);
|
|
|
- self.setCurrentEpoch(epoch);
|
|
|
-
|
|
|
+ self.setCurrentEpoch(epoch);
|
|
|
+
|
|
|
try {
|
|
|
waitForNewLeaderAck(self.getId(), zk.getZxid());
|
|
|
} catch (InterruptedException e) {
|
|
@@ -517,14 +524,14 @@ public class Leader {
|
|
|
if (self.getQuorumVerifier().getVotingMembers().containsKey(f.getSid())){
|
|
|
followerSet.add(f.getSid());
|
|
|
}
|
|
|
- }
|
|
|
+ }
|
|
|
boolean initTicksShouldBeIncreased = true;
|
|
|
for (Proposal.QuorumVerifierAcksetPair qvAckset:newLeaderProposal.qvAcksetPairs) {
|
|
|
if (!qvAckset.getQuorumVerifier().containsQuorum(followerSet)) {
|
|
|
initTicksShouldBeIncreased = false;
|
|
|
break;
|
|
|
}
|
|
|
- }
|
|
|
+ }
|
|
|
if (initTicksShouldBeIncreased) {
|
|
|
LOG.warn("Enough followers present. "+
|
|
|
"Perhaps the initTicks need to be increased.");
|
|
@@ -533,7 +540,7 @@ public class Leader {
|
|
|
}
|
|
|
|
|
|
startZkServer();
|
|
|
-
|
|
|
+
|
|
|
/**
|
|
|
* WARNING: do not use this for anything other than QA testing
|
|
|
* on a real cluster. Specifically to enable verification that quorum
|
|
@@ -679,39 +686,39 @@ public class Leader {
|
|
|
|
|
|
/** In a reconfig operation, this method attempts to find the best leader for next configuration.
|
|
|
* If the current leader is a voter in the next configuartion, then it remains the leader.
|
|
|
- * Otherwise, choose one of the new voters that acked the reconfiguartion, such that it is as
|
|
|
+ * Otherwise, choose one of the new voters that acked the reconfiguartion, such that it is as
|
|
|
* up-to-date as possible, i.e., acked as many outstanding proposals as possible.
|
|
|
- *
|
|
|
+ *
|
|
|
* @param reconfigProposal
|
|
|
* @param zxid of the reconfigProposal
|
|
|
* @return server if of the designated leader
|
|
|
*/
|
|
|
-
|
|
|
+
|
|
|
private long getDesignatedLeader(Proposal reconfigProposal, long zxid) {
|
|
|
//new configuration
|
|
|
- Proposal.QuorumVerifierAcksetPair newQVAcksetPair = reconfigProposal.qvAcksetPairs.get(reconfigProposal.qvAcksetPairs.size()-1);
|
|
|
-
|
|
|
- //check if I'm in the new configuration with the same quorum address -
|
|
|
- // if so, I'll remain the leader
|
|
|
- if (newQVAcksetPair.getQuorumVerifier().getVotingMembers().containsKey(self.getId()) &&
|
|
|
- newQVAcksetPair.getQuorumVerifier().getVotingMembers().get(self.getId()).addr.equals(self.getQuorumAddress())){
|
|
|
+ Proposal.QuorumVerifierAcksetPair newQVAcksetPair = reconfigProposal.qvAcksetPairs.get(reconfigProposal.qvAcksetPairs.size()-1);
|
|
|
+
|
|
|
+ //check if I'm in the new configuration with the same quorum address -
|
|
|
+ // if so, I'll remain the leader
|
|
|
+ if (newQVAcksetPair.getQuorumVerifier().getVotingMembers().containsKey(self.getId()) &&
|
|
|
+ newQVAcksetPair.getQuorumVerifier().getVotingMembers().get(self.getId()).addr.equals(self.getQuorumAddress())){
|
|
|
return self.getId();
|
|
|
}
|
|
|
- // start with an initial set of candidates that are voters from new config that
|
|
|
- // acknowledged the reconfig op (there must be a quorum). Choose one of them as
|
|
|
+ // start with an initial set of candidates that are voters from new config that
|
|
|
+ // acknowledged the reconfig op (there must be a quorum). Choose one of them as
|
|
|
// current leader candidate
|
|
|
HashSet<Long> candidates = new HashSet<Long>(newQVAcksetPair.getAckset());
|
|
|
candidates.remove(self.getId()); // if we're here, I shouldn't be the leader
|
|
|
long curCandidate = candidates.iterator().next();
|
|
|
-
|
|
|
+
|
|
|
//go over outstanding ops in order, and try to find a candidate that acked the most ops.
|
|
|
//this way it will be the most up-to-date and we'll minimize the number of ops that get dropped
|
|
|
-
|
|
|
+
|
|
|
long curZxid = zxid + 1;
|
|
|
Proposal p = outstandingProposals.get(curZxid);
|
|
|
-
|
|
|
- while (p!=null && !candidates.isEmpty()) {
|
|
|
- for (Proposal.QuorumVerifierAcksetPair qvAckset: p.qvAcksetPairs){
|
|
|
+
|
|
|
+ while (p!=null && !candidates.isEmpty()) {
|
|
|
+ for (Proposal.QuorumVerifierAcksetPair qvAckset: p.qvAcksetPairs){
|
|
|
//reduce the set of candidates to those that acknowledged p
|
|
|
candidates.retainAll(qvAckset.getAckset());
|
|
|
//no candidate acked p, return the best candidate found so far
|
|
@@ -719,18 +726,18 @@ public class Leader {
|
|
|
//update the current candidate, and if it is the only one remaining, return it
|
|
|
curCandidate = candidates.iterator().next();
|
|
|
if (candidates.size() == 1) return curCandidate;
|
|
|
- }
|
|
|
+ }
|
|
|
curZxid++;
|
|
|
p = outstandingProposals.get(curZxid);
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
return curCandidate;
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
* @return True if committed, otherwise false.
|
|
|
**/
|
|
|
- synchronized public boolean tryToCommit(Proposal p, long zxid, SocketAddress followerAddr) {
|
|
|
+ synchronized public boolean tryToCommit(Proposal p, long zxid, SocketAddress followerAddr) {
|
|
|
// make sure that ops are committed in order. With reconfigurations it is now possible
|
|
|
// that different operations wait for different sets of acks, and we still want to enforce
|
|
|
// that they are committed in order. Currently we only permit one outstanding reconfiguration
|
|
@@ -739,50 +746,50 @@ public class Leader {
|
|
|
// for an operation without getting enough acks for preceding ops. But in the future if multiple
|
|
|
// concurrent reconfigs are allowed, this can happen.
|
|
|
if (outstandingProposals.containsKey(zxid - 1)) return false;
|
|
|
-
|
|
|
+
|
|
|
// in order to be committed, a proposal must be accepted by a quorum.
|
|
|
//
|
|
|
// getting a quorum from all necessary configurations.
|
|
|
if (!p.hasAllQuorums()) {
|
|
|
- return false;
|
|
|
+ return false;
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
// commit proposals in order
|
|
|
- if (zxid != lastCommitted+1) {
|
|
|
+ if (zxid != lastCommitted+1) {
|
|
|
LOG.warn("Commiting zxid 0x" + Long.toHexString(zxid)
|
|
|
+ " from " + followerAddr + " not first!");
|
|
|
LOG.warn("First is "
|
|
|
+ (lastCommitted+1));
|
|
|
- }
|
|
|
-
|
|
|
+ }
|
|
|
+
|
|
|
outstandingProposals.remove(zxid);
|
|
|
-
|
|
|
+
|
|
|
if (p.request != null) {
|
|
|
toBeApplied.add(p);
|
|
|
}
|
|
|
|
|
|
if (p.request == null) {
|
|
|
LOG.warn("Going to commmit null: " + p);
|
|
|
- } else if (p.request.getHdr().getType() == OpCode.reconfig) {
|
|
|
- LOG.debug("Committing a reconfiguration! " + outstandingProposals.size());
|
|
|
-
|
|
|
- //if this server is voter in new config with the same quorum address,
|
|
|
+ } else if (p.request.getHdr().getType() == OpCode.reconfig) {
|
|
|
+ LOG.debug("Committing a reconfiguration! " + outstandingProposals.size());
|
|
|
+
|
|
|
+ //if this server is voter in new config with the same quorum address,
|
|
|
//then it will remain the leader
|
|
|
//otherwise an up-to-date follower will be designated as leader. This saves
|
|
|
- //leader election time, unless the designated leader fails
|
|
|
+ //leader election time, unless the designated leader fails
|
|
|
Long designatedLeader = getDesignatedLeader(p, zxid);
|
|
|
//LOG.warn("designated leader is: " + designatedLeader);
|
|
|
|
|
|
QuorumVerifier newQV = p.qvAcksetPairs.get(p.qvAcksetPairs.size()-1).getQuorumVerifier();
|
|
|
-
|
|
|
+
|
|
|
self.processReconfig(newQV, designatedLeader, zk.getZxid(), true);
|
|
|
|
|
|
if (designatedLeader != self.getId()) {
|
|
|
allowedToCommit = false;
|
|
|
}
|
|
|
-
|
|
|
- // we're sending the designated leader, and if the leader is changing the followers are
|
|
|
- // responsible for closing the connection - this way we are sure that at least a majority of them
|
|
|
+
|
|
|
+ // we're sending the designated leader, and if the leader is changing the followers are
|
|
|
+ // responsible for closing the connection - this way we are sure that at least a majority of them
|
|
|
// receive the commit message.
|
|
|
commitAndActivate(zxid, designatedLeader);
|
|
|
informAndActivate(p, designatedLeader);
|
|
@@ -795,12 +802,12 @@ public class Leader {
|
|
|
if(pendingSyncs.containsKey(zxid)){
|
|
|
for(LearnerSyncRequest r: pendingSyncs.remove(zxid)) {
|
|
|
sendSync(r);
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- return true;
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ return true;
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
/**
|
|
|
* Keep a count of acks that are received by the leader for a particular
|
|
|
* proposal
|
|
@@ -809,9 +816,9 @@ public class Leader {
|
|
|
* @param sid, the id of the server that sent the ack
|
|
|
* @param followerAddr
|
|
|
*/
|
|
|
- synchronized public void processAck(long sid, long zxid, SocketAddress followerAddr) {
|
|
|
- if (!allowedToCommit) return; // last op committed was a leader change - from now on
|
|
|
- // the new leader should commit
|
|
|
+ synchronized public void processAck(long sid, long zxid, SocketAddress followerAddr) {
|
|
|
+ if (!allowedToCommit) return; // last op committed was a leader change - from now on
|
|
|
+ // the new leader should commit
|
|
|
if (LOG.isTraceEnabled()) {
|
|
|
LOG.trace("Ack zxid: 0x{}", Long.toHexString(zxid));
|
|
|
for (Proposal p : outstandingProposals.values()) {
|
|
@@ -821,7 +828,7 @@ public class Leader {
|
|
|
}
|
|
|
LOG.trace("outstanding proposals all");
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
if ((zxid & 0xffffffffL) == 0) {
|
|
|
/*
|
|
|
* We no longer process NEWLEADER ack with this method. However,
|
|
@@ -830,8 +837,8 @@ public class Leader {
|
|
|
*/
|
|
|
return;
|
|
|
}
|
|
|
-
|
|
|
-
|
|
|
+
|
|
|
+
|
|
|
if (outstandingProposals.size() == 0) {
|
|
|
if (LOG.isDebugEnabled()) {
|
|
|
LOG.debug("outstanding is 0");
|
|
@@ -852,13 +859,13 @@ public class Leader {
|
|
|
Long.toHexString(zxid), followerAddr);
|
|
|
return;
|
|
|
}
|
|
|
-
|
|
|
- p.addAck(sid);
|
|
|
+
|
|
|
+ p.addAck(sid);
|
|
|
/*if (LOG.isDebugEnabled()) {
|
|
|
LOG.debug("Count for zxid: 0x{} is {}",
|
|
|
Long.toHexString(zxid), p.ackSet.size());
|
|
|
}*/
|
|
|
-
|
|
|
+
|
|
|
boolean hasCommitted = tryToCommit(p, zxid, followerAddr);
|
|
|
|
|
|
// If p is a reconfiguration, multiple other operations may be ready to be committed,
|
|
@@ -875,11 +882,11 @@ public class Leader {
|
|
|
while (allowedToCommit && hasCommitted && p!=null){
|
|
|
curZxid++;
|
|
|
p = outstandingProposals.get(curZxid);
|
|
|
- if (p !=null) hasCommitted = tryToCommit(p, curZxid, null);
|
|
|
+ if (p !=null) hasCommitted = tryToCommit(p, curZxid, null);
|
|
|
}
|
|
|
}
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
static class ToBeAppliedRequestProcessor implements RequestProcessor {
|
|
|
private final RequestProcessor next;
|
|
|
|
|
@@ -988,11 +995,11 @@ public class Leader {
|
|
|
synchronized(this){
|
|
|
lastCommitted = zxid;
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
byte data[] = new byte[8];
|
|
|
- ByteBuffer buffer = ByteBuffer.wrap(data);
|
|
|
+ ByteBuffer buffer = ByteBuffer.wrap(data);
|
|
|
buffer.putLong(designatedLeader);
|
|
|
-
|
|
|
+
|
|
|
QuorumPacket qp = new QuorumPacket(Leader.COMMITANDACTIVATE, zxid, data, null);
|
|
|
sendPacket(qp);
|
|
|
}
|
|
@@ -1006,17 +1013,17 @@ public class Leader {
|
|
|
sendObserverPacket(qp);
|
|
|
}
|
|
|
|
|
|
-
|
|
|
+
|
|
|
/**
|
|
|
* Create an inform&activate packet and send it to all observers.
|
|
|
*/
|
|
|
public void informAndActivate(Proposal proposal, long designatedLeader) {
|
|
|
byte[] proposalData = proposal.packet.getData();
|
|
|
byte[] data = new byte[proposalData.length + 8];
|
|
|
- ByteBuffer buffer = ByteBuffer.wrap(data);
|
|
|
+ ByteBuffer buffer = ByteBuffer.wrap(data);
|
|
|
buffer.putLong(designatedLeader);
|
|
|
buffer.put(proposalData);
|
|
|
-
|
|
|
+
|
|
|
QuorumPacket qp = new QuorumPacket(Leader.INFORMANDACTIVATE, proposal.request.zxid, data, null);
|
|
|
sendObserverPacket(qp);
|
|
|
}
|
|
@@ -1064,19 +1071,19 @@ public class Leader {
|
|
|
|
|
|
Proposal p = new Proposal();
|
|
|
p.packet = pp;
|
|
|
- p.request = request;
|
|
|
-
|
|
|
+ p.request = request;
|
|
|
+
|
|
|
synchronized(this) {
|
|
|
p.addQuorumVerifier(self.getQuorumVerifier());
|
|
|
-
|
|
|
+
|
|
|
if (request.getHdr().getType() == OpCode.reconfig){
|
|
|
- self.setLastSeenQuorumVerifier(request.qv, true);
|
|
|
+ self.setLastSeenQuorumVerifier(request.qv, true);
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
if (self.getQuorumVerifier().getVersion()<self.getLastSeenQuorumVerifier().getVersion()) {
|
|
|
p.addQuorumVerifier(self.getLastSeenQuorumVerifier());
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
if (LOG.isDebugEnabled()) {
|
|
|
LOG.debug("Proposing:: " + request);
|
|
|
}
|
|
@@ -1087,7 +1094,7 @@ public class Leader {
|
|
|
}
|
|
|
return p;
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
public LearnerSnapshotThrottler getLearnerSnapshotThrottler() {
|
|
|
return learnerSnapshotThrottler;
|
|
|
}
|
|
@@ -1125,7 +1132,7 @@ public class Leader {
|
|
|
*
|
|
|
* @param handler handler of the follower
|
|
|
* @return last proposed zxid
|
|
|
- * @throws InterruptedException
|
|
|
+ * @throws InterruptedException
|
|
|
*/
|
|
|
synchronized public long startForwarding(LearnerHandler handler,
|
|
|
long lastSeenZxid) {
|
|
@@ -1209,7 +1216,7 @@ public class Leader {
|
|
|
}
|
|
|
if (ss.getCurrentEpoch() != -1) {
|
|
|
if (ss.isMoreRecentThan(leaderStateSummary)) {
|
|
|
- throw new IOException("Follower is ahead of the leader, leader summary: "
|
|
|
+ throw new IOException("Follower is ahead of the leader, leader summary: "
|
|
|
+ leaderStateSummary.getCurrentEpoch()
|
|
|
+ " (current epoch), "
|
|
|
+ leaderStateSummary.getLastZxid()
|
|
@@ -1237,9 +1244,9 @@ public class Leader {
|
|
|
}
|
|
|
}
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
/**
|
|
|
- * Return a list of sid in set as string
|
|
|
+ * Return a list of sid in set as string
|
|
|
*/
|
|
|
private String getSidSetString(Set<Long> sidSet) {
|
|
|
StringBuilder sids = new StringBuilder();
|
|
@@ -1264,7 +1271,7 @@ public class Leader {
|
|
|
+ newLeaderProposal.ackSetsToString()
|
|
|
+ " ]; starting up and setting last processed zxid: 0x{}",
|
|
|
Long.toHexString(zk.getZxid()));
|
|
|
-
|
|
|
+
|
|
|
/*
|
|
|
* ZOOKEEPER-1324. the leader sends the new config it must complete
|
|
|
* to others inside a NEWLEADER message (see LearnerHandler where
|
|
@@ -1273,20 +1280,20 @@ public class Leader {
|
|
|
* config to itself.
|
|
|
*/
|
|
|
QuorumVerifier newQV = self.getLastSeenQuorumVerifier();
|
|
|
-
|
|
|
- Long designatedLeader = getDesignatedLeader(newLeaderProposal, zk.getZxid());
|
|
|
+
|
|
|
+ Long designatedLeader = getDesignatedLeader(newLeaderProposal, zk.getZxid());
|
|
|
|
|
|
self.processReconfig(newQV, designatedLeader, zk.getZxid(), true);
|
|
|
if (designatedLeader != self.getId()) {
|
|
|
allowedToCommit = false;
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
zk.startup();
|
|
|
/*
|
|
|
* Update the election vote here to ensure that all members of the
|
|
|
* ensemble report the same vote to new servers that start up and
|
|
|
* send leader election notifications to the ensemble.
|
|
|
- *
|
|
|
+ *
|
|
|
* @see https://issues.apache.org/jira/browse/ZOOKEEPER-1732
|
|
|
*/
|
|
|
self.updateElectionVote(getEpoch());
|
|
@@ -1378,7 +1385,7 @@ public class Leader {
|
|
|
case COMMIT:
|
|
|
return "COMMIT";
|
|
|
case COMMITANDACTIVATE:
|
|
|
- return "COMMITANDACTIVATE";
|
|
|
+ return "COMMITANDACTIVATE";
|
|
|
case PING:
|
|
|
return "PING";
|
|
|
case REVALIDATE:
|