forked from apache/zookeeper
/
Leader.java
1737 lines (1510 loc) · 61.9 KB
/
Leader.java
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.zookeeper.server.quorum;
import java.io.BufferedInputStream;
import java.io.ByteArrayInputStream;
import java.io.ByteArrayOutputStream;
import java.io.DataInputStream;
import java.io.DataOutputStream;
import java.io.IOException;
import java.net.BindException;
import java.net.InetSocketAddress;
import java.net.ServerSocket;
import java.net.Socket;
import java.net.SocketAddress;
import java.net.SocketException;
import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Iterator;
import java.util.LinkedList;
import java.util.List;
import java.util.Map;
import java.util.Objects;
import java.util.Set;
import java.util.concurrent.atomic.AtomicLong;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentLinkedQueue;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.stream.Collectors;
import javax.security.sasl.SaslException;
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.ZooDefs.OpCode;
import org.apache.zookeeper.common.Time;
import org.apache.zookeeper.jmx.MBeanRegistry;
import org.apache.zookeeper.server.FinalRequestProcessor;
import org.apache.zookeeper.server.Request;
import org.apache.zookeeper.server.RequestProcessor;
import org.apache.zookeeper.server.ServerMetrics;
import org.apache.zookeeper.server.ZooKeeperCriticalThread;
import org.apache.zookeeper.server.quorum.QuorumPeer.LearnerType;
import org.apache.zookeeper.server.ZKDatabase;
import org.apache.zookeeper.server.ZooTrace;
import org.apache.zookeeper.server.quorum.auth.QuorumAuthServer;
import org.apache.zookeeper.server.quorum.flexible.QuorumVerifier;
import org.apache.zookeeper.server.util.SerializeUtils;
import org.apache.zookeeper.server.util.ZxidUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* This class has the control logic for the Leader.
*/
public class Leader extends LearnerMaster {
private static final Logger LOG = LoggerFactory.getLogger(Leader.class);
static final private boolean nodelay = System.getProperty("leader.nodelay", "true").equals("true");
static {
LOG.info("TCP NoDelay set to: " + nodelay);
}
static public class Proposal extends SyncedLearnerTracker {
public QuorumPacket packet;
public Request request;
@Override
public String toString() {
return packet.getType() + ", " + packet.getZxid() + ", " + request;
}
}
// log ack latency if zxid is a multiple of ackLoggingFrequency. If <=0, disable logging.
private static final String ACK_LOGGING_FREQUENCY = "zookeeper.leader.ackLoggingFrequency";
private static int ackLoggingFrequency;
static {
ackLoggingFrequency = Integer.getInteger(ACK_LOGGING_FREQUENCY, 1000);
LOG.info(ACK_LOGGING_FREQUENCY + " = " + ackLoggingFrequency);
}
public static void setAckLoggingFrequency(int frequency) {
ackLoggingFrequency = frequency;
}
public static int getAckLoggingFrequency() {
return ackLoggingFrequency;
}
final LeaderZooKeeperServer zk;
final QuorumPeer self;
// VisibleForTesting
protected boolean quorumFormed = false;
// the follower acceptor thread
volatile LearnerCnxAcceptor cnxAcceptor = null;
// list of all the learners, including followers and observers
private final HashSet<LearnerHandler> learners =
new HashSet<LearnerHandler>();
private final BufferStats proposalStats;
public BufferStats getProposalStats() {
return proposalStats;
}
// beans for all learners
private final ConcurrentHashMap<LearnerHandler, LearnerHandlerBean> connectionBeans = new ConcurrentHashMap<>();
/**
* Returns a copy of the current learner snapshot
*/
public List<LearnerHandler> getLearners() {
synchronized (learners) {
return new ArrayList<LearnerHandler>(learners);
}
}
// list of followers that are ready to follow (i.e synced with the leader)
private final HashSet<LearnerHandler> forwardingFollowers =
new HashSet<LearnerHandler>();
/**
* Returns a copy of the current forwarding follower snapshot
*/
public List<LearnerHandler> getForwardingFollowers() {
synchronized (forwardingFollowers) {
return new ArrayList<LearnerHandler>(forwardingFollowers);
}
}
public List<LearnerHandler> getNonVotingFollowers() {
List<LearnerHandler> nonVotingFollowers = new ArrayList<LearnerHandler>();
synchronized (forwardingFollowers) {
for (LearnerHandler lh : forwardingFollowers) {
if (!isParticipant(lh.getSid())) {
nonVotingFollowers.add(lh);
}
}
}
return nonVotingFollowers;
}
void addForwardingFollower(LearnerHandler lh) {
synchronized (forwardingFollowers) {
forwardingFollowers.add(lh);
}
}
private final HashSet<LearnerHandler> observingLearners =
new HashSet<LearnerHandler>();
/**
* Returns a copy of the current observer snapshot
*/
public List<LearnerHandler> getObservingLearners() {
synchronized (observingLearners) {
return new ArrayList<LearnerHandler>(observingLearners);
}
}
private void addObserverLearnerHandler(LearnerHandler lh) {
synchronized (observingLearners) {
observingLearners.add(lh);
}
}
public Iterable<Map<String, Object>> getObservingLearnersInfo() {
Set<Map<String,Object>> info = new HashSet<>();
synchronized (observingLearners) {
for (LearnerHandler lh: observingLearners) {
info.add(lh.getLearnerHandlerInfo());
}
}
return info;
}
public void resetObserverConnectionStats() {
synchronized (observingLearners) {
for (LearnerHandler lh : observingLearners) {
lh.resetObserverConnectionStats();
}
}
}
// Pending sync requests. Must access under 'this' lock.
private final Map<Long,List<LearnerSyncRequest>> pendingSyncs =
new HashMap<Long,List<LearnerSyncRequest>>();
synchronized public int getNumPendingSyncs() {
return pendingSyncs.size();
}
//Follower counter
final AtomicLong followerCounter = new AtomicLong(-1);
/**
* Adds peer to the leader.
*
* @param learner
* instance of learner handle
*/
@Override
public void addLearnerHandler(LearnerHandler learner) {
synchronized (learners) {
learners.add(learner);
}
}
/**
* Remove the learner from the learner list
*
* @param peer
*/
@Override
public void removeLearnerHandler(LearnerHandler peer) {
synchronized (forwardingFollowers) {
forwardingFollowers.remove(peer);
}
synchronized (learners) {
learners.remove(peer);
}
synchronized (observingLearners) {
observingLearners.remove(peer);
}
}
boolean isLearnerSynced(LearnerHandler peer){
synchronized (forwardingFollowers) {
return forwardingFollowers.contains(peer);
}
}
/**
* 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) {
HashSet<Long> ids = new HashSet<Long>();
if (qv.getVotingMembers().containsKey(self.getId()))
ids.add(self.getId());
synchronized (forwardingFollowers) {
for (LearnerHandler learnerHandler: forwardingFollowers){
if (learnerHandler.synced() && qv.getVotingMembers().containsKey(learnerHandler.getSid())){
ids.add(learnerHandler.getSid());
}
}
}
return qv.containsQuorum(ids);
}
private final List<ServerSocket> serverSockets = new LinkedList<>();
Leader(QuorumPeer self,LeaderZooKeeperServer zk) throws IOException {
this.self = self;
this.proposalStats = new BufferStats();
Set<InetSocketAddress> addresses;
if (self.getQuorumListenOnAllIPs()) {
addresses = self.getQuorumAddress().getWildcardAddresses();
} else {
addresses = self.getQuorumAddress().getAllAddresses();
}
for (InetSocketAddress address : addresses) {
serverSockets.add(createServerSocket(address, self.shouldUsePortUnification(), self.isSslQuorum()));
}
this.zk = zk;
}
ServerSocket createServerSocket(InetSocketAddress address, boolean portUnification, boolean sslQuorum)
throws IOException {
ServerSocket serverSocket;
try {
if (portUnification || sslQuorum) {
serverSocket = new UnifiedServerSocket(self.getX509Util(), portUnification);
} else {
serverSocket = new ServerSocket();
}
serverSocket.setReuseAddress(true);
serverSocket.bind(address);
return serverSocket;
} catch (BindException e) {
LOG.error("Couldn't bind to " + address.toString(), e);
throw e;
}
}
/**
* 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;
/**
* 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>();
// VisibleForTesting
protected final Proposal newLeaderProposal = new Proposal();
class LearnerCnxAcceptor extends ZooKeeperCriticalThread {
private final AtomicBoolean stop = new AtomicBoolean(false);
private final AtomicBoolean fail = new AtomicBoolean(false);
LearnerCnxAcceptor() {
super("LearnerCnxAcceptor-" + serverSockets.stream()
.map(ServerSocket::getLocalSocketAddress)
.map(Objects::toString)
.collect(Collectors.joining(",")),
zk.getZooKeeperServerListener());
}
@Override
public void run() {
if (!stop.get() && !serverSockets.isEmpty()) {
ExecutorService executor = Executors.newFixedThreadPool(serverSockets.size());
CountDownLatch latch = new CountDownLatch(serverSockets.size());
serverSockets.forEach(serverSocket ->
executor.submit(new LearnerCnxAcceptorHandler(serverSocket, latch)));
try {
latch.await();
} catch (InterruptedException ie) {
LOG.error("Interrupted while sleeping. " +
"Ignoring exception", ie);
} finally {
closeSockets();
}
}
}
public void halt() {
stop.set(true);
closeSockets();
}
class LearnerCnxAcceptorHandler implements Runnable {
private ServerSocket serverSocket;
private CountDownLatch latch;
LearnerCnxAcceptorHandler(ServerSocket serverSocket, CountDownLatch latch) {
this.serverSocket = serverSocket;
this.latch = latch;
}
@Override
public void run() {
try {
Thread.currentThread().setName("LearnerCnxAcceptorHandler-" + serverSocket.getLocalSocketAddress());
while (!stop.get()) {
acceptConnections();
}
} catch (Exception e) {
LOG.warn("Exception while accepting follower", e);
if (fail.compareAndSet(false, true)) {
handleException(getName(), e);
halt();
}
} finally {
latch.countDown();
}
}
private void acceptConnections() throws IOException {
Socket socket = null;
boolean error = false;
try {
socket = serverSocket.accept();
// start with the initLimit, once the ack is processed
// in LearnerHandler switch to the syncLimit
socket.setSoTimeout(self.tickTime * self.initLimit);
socket.setTcpNoDelay(nodelay);
BufferedInputStream is = new BufferedInputStream(socket.getInputStream());
LearnerHandler fh = new LearnerHandler(socket, is, Leader.this);
fh.start();
} catch (SocketException e) {
error = true;
if (stop.get()) {
LOG.info("Exception while shutting down acceptor", e);
} else {
throw e;
}
} catch (SaslException e) {
LOG.error("Exception while connecting to quorum learner", e);
error = true;
} catch (Exception e) {
error = true;
throw e;
} finally {
// Don't leak sockets on errors
if (error && socket != null && !socket.isClosed()) {
try {
socket.close();
} catch (IOException e) {
LOG.warn("Error closing socket", e);
}
}
}
}
}
}
StateSummary leaderStateSummary;
long epoch = -1;
boolean waitingForNewEpoch = true;
// 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;
/**
* Timestamp when this leader started serving request (Quorum is running)
*/
private long leaderStartTime;
public long getUptime() {
if (leaderStartTime > 0) {
return Time.currentElapsedTime() - leaderStartTime;
}
return 0;
}
/**
* This method is main function that is called to lead
*
* @throws IOException
* @throws InterruptedException
*/
void lead() throws IOException, InterruptedException {
self.end_fle = Time.currentElapsedTime();
long electionTimeTaken = self.end_fle - self.start_fle;
self.setElectionTimeTaken(electionTimeTaken);
ServerMetrics.getMetrics().ELECTION_TIME.add(electionTimeTaken);
LOG.info("LEADING - LEADER ELECTION TOOK - {} {}", electionTimeTaken,
QuorumPeer.FLE_TIME_UNIT);
self.start_fle = 0;
self.end_fle = 0;
zk.registerJMX(new LeaderBean(this, zk), self.jmxLocalPeerBean);
try {
self.setZabState(QuorumPeer.ZabState.DISCOVERY);
self.tick.set(0);
zk.loadData();
leaderStateSummary = new StateSummary(self.getCurrentEpoch(), zk.getLastProcessedZxid());
// Start thread that waits for connection requests from
// new followers.
cnxAcceptor = new LearnerCnxAcceptor();
cnxAcceptor.start();
long epoch = getEpochToPropose(self.getId(), self.getAcceptedEpoch());
zk.setZxid(ZxidUtils.makeZxid(epoch, 0));
synchronized(this){
lastProposed = zk.getZxid();
}
newLeaderProposal.packet = new QuorumPacket(NEWLEADER, zk.getZxid(),
null, null);
if ((newLeaderProposal.packet.getZxid() & 0xffffffffL) != 0) {
LOG.info("NEWLEADER proposal has Zxid of "
+ Long.toHexString(newLeaderProposal.packet.getZxid()));
}
QuorumVerifier lastSeenQV = self.getLastSeenQuorumVerifier();
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).
// 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).
// 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
// 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
// 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
// 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
// the last-seen-quorumverifier of the leader, which we change below
try {
QuorumVerifier newQV = self.configFromString(curQV.toString());
newQV.setVersion(zk.getZxid());
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.setLeaderAddressAndId(self.getQuorumAddress(), self.getId());
self.setZabState(QuorumPeer.ZabState.SYNCHRONIZATION);
try {
waitForNewLeaderAck(self.getId(), zk.getZxid());
} catch (InterruptedException e) {
shutdown("Waiting for a quorum of followers, only synced with sids: [ "
+ newLeaderProposal.ackSetsToString() + " ]");
HashSet<Long> followerSet = new HashSet<Long>();
for(LearnerHandler f : getLearners()) {
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.");
}
return;
}
startZkServer();
/**
* WARNING: do not use this for anything other than QA testing
* on a real cluster. Specifically to enable verification that quorum
* can handle the lower 32bit roll-over issue identified in
* ZOOKEEPER-1277. Without this option it would take a very long
* time (on order of a month say) to see the 4 billion writes
* necessary to cause the roll-over to occur.
*
* This field allows you to override the zxid of the server. Typically
* you'll want to set it to something like 0xfffffff0 and then
* start the quorum, run some operations and see the re-election.
*/
String initialZxid = System.getProperty("zookeeper.testingonly.initialZxid");
if (initialZxid != null) {
long zxid = Long.parseLong(initialZxid);
zk.setZxid((zk.getZxid() & 0xffffffff00000000L) | zxid);
}
if (!System.getProperty("zookeeper.leaderServes", "yes").equals("no")) {
self.setZooKeeperServer(zk);
}
self.setZabState(QuorumPeer.ZabState.BROADCAST);
self.adminServer.setZooKeeperServer(zk);
// Everything is a go, simply start counting the ticks
// WARNING: I couldn't find any wait statement on a synchronized
// block that would be notified by this notifyAll() call, so
// I commented it out
//synchronized (this) {
// notifyAll();
//}
// We ping twice a tick, so we only update the tick every other
// iteration
boolean tickSkip = true;
// If not null then shutdown this leader
String shutdownMessage = null;
while (true) {
synchronized (this) {
long start = Time.currentElapsedTime();
long cur = start;
long end = start + self.tickTime / 2;
while (cur < end) {
wait(end - cur);
cur = Time.currentElapsedTime();
}
if (!tickSkip) {
self.tick.incrementAndGet();
}
// We use an instance of SyncedLearnerTracker to
// track synced learners to make sure we still have a
// quorum of current (and potentially next pending) view.
SyncedLearnerTracker syncedAckSet = new SyncedLearnerTracker();
syncedAckSet.addQuorumVerifier(self.getQuorumVerifier());
if (self.getLastSeenQuorumVerifier() != null
&& self.getLastSeenQuorumVerifier().getVersion() > self
.getQuorumVerifier().getVersion()) {
syncedAckSet.addQuorumVerifier(self
.getLastSeenQuorumVerifier());
}
syncedAckSet.addAck(self.getId());
for (LearnerHandler f : getLearners()) {
if (f.synced()) {
syncedAckSet.addAck(f.getSid());
}
}
// check leader running status
if (!this.isRunning()) {
// set shutdown flag
shutdownMessage = "Unexpected internal error";
break;
}
if (!tickSkip && !syncedAckSet.hasAllQuorums()) {
// Lost quorum of last committed and/or last proposed
// config, set shutdown flag
shutdownMessage = "Not sufficient followers synced, only synced with sids: [ "
+ syncedAckSet.ackSetsToString() + " ]";
break;
}
tickSkip = !tickSkip;
}
for (LearnerHandler f : getLearners()) {
f.ping();
}
}
if (shutdownMessage != null) {
shutdown(shutdownMessage);
// leader goes in looking state
}
} finally {
zk.unregisterJMX(this);
}
}
boolean isShutdown;
/**
* Close down all the LearnerHandlers
*/
void shutdown(String reason) {
LOG.info("Shutting down");
if (isShutdown) {
return;
}
LOG.info("Shutdown called",
new Exception("shutdown Leader! reason: " + reason));
if (cnxAcceptor != null) {
cnxAcceptor.halt();
} else {
closeSockets();
}
// NIO should not accept conenctions
self.setZooKeeperServer(null);
self.adminServer.setZooKeeperServer(null);
self.closeAllConnections();
// shutdown the previous zk
if (zk != null) {
zk.shutdown();
}
synchronized (learners) {
for (Iterator<LearnerHandler> it = learners.iterator(); it
.hasNext();) {
LearnerHandler f = it.next();
it.remove();
f.shutdown();
}
}
isShutdown = true;
}
synchronized void closeSockets() {
for (ServerSocket serverSocket : serverSockets) {
if (!serverSocket.isClosed()) {
try {
serverSocket.close();
} catch (IOException e) {
LOG.warn("Ignoring unexpected exception during close {}", serverSocket, e);
}
}
}
}
/** 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
* 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())){
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
// 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){
//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
if (candidates.isEmpty()) return curCandidate;
//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) {
// 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
// such that the reconfiguration and subsequent outstanding ops proposed while the reconfig is
// pending all wait for a quorum of old and new config, so it's not possible to get enough acks
// 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;
}
// commit proposals in order
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,
//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
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
// receive the commit message.
commitAndActivate(zxid, designatedLeader);
informAndActivate(p, designatedLeader);
//turnOffFollowers();
} else {
p.request.logLatency(ServerMetrics.getMetrics().QUORUM_ACK_LATENCY);
commit(zxid);
inform(p);
}
zk.commitProcessor.commit(p.request);
if(pendingSyncs.containsKey(zxid)){
for(LearnerSyncRequest r: pendingSyncs.remove(zxid)) {
sendSync(r);
}
}
return true;
}
/**
* Keep a count of acks that are received by the leader for a particular
* proposal
*
* @param zxid, the zxid of the proposal sent out
* @param sid, the id of the server that sent the ack
* @param followerAddr
*/
@Override
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()) {
long packetZxid = p.packet.getZxid();
LOG.trace("outstanding proposal: 0x{}",
Long.toHexString(packetZxid));
}
LOG.trace("outstanding proposals all");
}
if ((zxid & 0xffffffffL) == 0) {
/*
* We no longer process NEWLEADER ack with this method. However,
* the learner sends an ack back to the leader after it gets
* UPTODATE, so we just ignore the message.
*/
return;
}
if (outstandingProposals.size() == 0) {
LOG.debug("outstanding is 0");
return;
}
if (lastCommitted >= zxid) {
if (LOG.isDebugEnabled()) {
LOG.debug("proposal has already been committed, pzxid: 0x{} zxid: 0x{}",
Long.toHexString(lastCommitted), Long.toHexString(zxid));
}
// The proposal has already been committed
return;
}
Proposal p = outstandingProposals.get(zxid);