/
QuorumPeer.java
1790 lines (1579 loc) · 61.7 KB
/
QuorumPeer.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.BufferedReader;
import java.io.File;
import java.io.FileNotFoundException;
import java.io.FileReader;
import java.io.IOException;
import java.io.StringReader;
import java.io.StringWriter;
import java.io.Writer;
import java.net.DatagramPacket;
import java.net.DatagramSocket;
import java.net.InetAddress;
import java.net.InetSocketAddress;
import java.net.SocketException;
import java.net.UnknownHostException;
import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
import java.util.Properties;
import java.util.Set;
import java.util.concurrent.atomic.AtomicInteger;
import org.apache.zookeeper.common.AtomicFileWritingIdiom;
import org.apache.zookeeper.common.AtomicFileWritingIdiom.WriterStatement;
import org.apache.zookeeper.common.Time;
import org.apache.zookeeper.jmx.MBeanRegistry;
import org.apache.zookeeper.jmx.ZKMBeanInfo;
import org.apache.zookeeper.server.ServerCnxnFactory;
import org.apache.zookeeper.server.ZKDatabase;
import org.apache.zookeeper.server.ZooKeeperServer;
import org.apache.zookeeper.server.ZooKeeperThread;
import org.apache.zookeeper.server.admin.AdminServer;
import org.apache.zookeeper.server.admin.AdminServer.AdminServerException;
import org.apache.zookeeper.server.admin.AdminServerFactory;
import org.apache.zookeeper.server.persistence.FileTxnSnapLog;
import org.apache.zookeeper.server.quorum.QuorumPeerConfig.ConfigException;
import org.apache.zookeeper.server.quorum.flexible.QuorumMaj;
import org.apache.zookeeper.server.quorum.flexible.QuorumVerifier;
import org.apache.zookeeper.server.util.ZxidUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* This class manages the quorum protocol. There are three states this server
* can be in:
* <ol>
* <li>Leader election - each server will elect a leader (proposing itself as a
* leader initially).</li>
* <li>Follower - the server will synchronize with the leader and replicate any
* transactions.</li>
* <li>Leader - the server will process requests and forward them to followers.
* A majority of followers must log the request before it can be accepted.
* </ol>
*
* This class will setup a datagram socket that will always respond with its
* view of the current leader. The response will take the form of:
*
* <pre>
* int xid;
*
* long myid;
*
* long leader_id;
*
* long leader_zxid;
* </pre>
*
* The request for the current leader will consist solely of an xid: int xid;
*/
public class QuorumPeer extends ZooKeeperThread implements QuorumStats.Provider {
private static final Logger LOG = LoggerFactory.getLogger(QuorumPeer.class);
private QuorumBean jmxQuorumBean;
LocalPeerBean jmxLocalPeerBean;
private Map<Long, RemotePeerBean> jmxRemotePeerBean;
LeaderElectionBean jmxLeaderElectionBean;
private QuorumCnxManager qcm;
/**
* ZKDatabase is a top level member of quorumpeer
* which will be used in all the zookeeperservers
* instantiated later. Also, it is created once on
* bootup and only thrown away in case of a truncate
* message from the leader
*/
private ZKDatabase zkDb;
public static class QuorumServer {
public InetSocketAddress addr = null;
public InetSocketAddress electionAddr = null;
public InetSocketAddress clientAddr = null;
public long id;
public LearnerType type = LearnerType.PARTICIPANT;
public QuorumServer(long id, InetSocketAddress addr,
InetSocketAddress electionAddr, InetSocketAddress clientAddr) {
this.id = id;
this.addr = addr;
this.electionAddr = electionAddr;
this.clientAddr = clientAddr;
}
public QuorumServer(long id, InetSocketAddress addr,
InetSocketAddress electionAddr) {
this.id = id;
this.addr = addr;
this.electionAddr = electionAddr;
this.clientAddr = null;
}
public QuorumServer(long id, InetSocketAddress addr) {
this.id = id;
this.addr = addr;
this.electionAddr = null;
this.clientAddr = null;
}
/**
* Performs a DNS lookup for server address and election address.
*
* If the DNS lookup fails, this.addr and electionAddr remain
* unmodified.
*/
public void recreateSocketAddresses() {
if (this.addr == null) {
LOG.warn("Server address has not been initialized");
return;
}
if (this.electionAddr == null) {
LOG.warn("Election address has not been initialized");
return;
}
String host = this.addr.getHostString();
InetAddress address = null;
try {
address = InetAddress.getByName(host);
} catch (UnknownHostException ex) {
LOG.warn("Failed to resolve address: {}", host, ex);
return;
}
LOG.debug("Resolved address for {}: {}", host, address);
int port = this.addr.getPort();
this.addr = new InetSocketAddress(address, port);
port = this.electionAddr.getPort();
this.electionAddr = new InetSocketAddress(address, port);
}
private void setType(String s) throws ConfigException {
if (s.toLowerCase().equals("observer")) {
type = LearnerType.OBSERVER;
} else if (s.toLowerCase().equals("participant")) {
type = LearnerType.PARTICIPANT;
} else {
throw new ConfigException("Unrecognised peertype: " + s);
}
}
private static final String wrongFormat = " does not have the form server_cofig or server_config;client_config"+
" where server_config is host:port:port or host:port:port:type and client_config is port or host:port";
public QuorumServer(long sid, String addressStr) throws ConfigException {
// LOG.warn("sid = " + sid + " addressStr = " + addressStr);
this.id = sid;
String serverClientParts[] = addressStr.split(";");
String serverParts[] = serverClientParts[0].split(":");
if ((serverClientParts.length > 2) || (serverParts.length < 3)
|| (serverParts.length > 4)) {
throw new ConfigException(addressStr + wrongFormat);
}
if (serverClientParts.length == 2) {
//LOG.warn("ClientParts: " + serverClientParts[1]);
String clientParts[] = serverClientParts[1].split(":");
if (clientParts.length > 2) {
throw new ConfigException(addressStr + wrongFormat);
}
// is client_config a host:port or just a port
String hostname = (clientParts.length == 2) ? clientParts[0] : "0.0.0.0";
try {
clientAddr = new InetSocketAddress(hostname,
Integer.parseInt(clientParts[clientParts.length - 1]));
//LOG.warn("Set clientAddr to " + clientAddr);
} catch (NumberFormatException e) {
throw new ConfigException("Address unresolved: " + hostname + ":" + clientParts[clientParts.length - 1]);
}
}
// server_config should be either host:port:port or host:port:port:type
try {
addr = new InetSocketAddress(serverParts[0],
Integer.parseInt(serverParts[1]));
} catch (NumberFormatException e) {
throw new ConfigException("Address unresolved: " + serverParts[0] + ":" + serverParts[1]);
}
try {
electionAddr = new InetSocketAddress(serverParts[0],
Integer.parseInt(serverParts[2]));
} catch (NumberFormatException e) {
throw new ConfigException("Address unresolved: " + serverParts[0] + ":" + serverParts[2]);
}
if (serverParts.length == 4) setType(serverParts[3]);
}
public QuorumServer(long id, InetSocketAddress addr,
InetSocketAddress electionAddr, LearnerType type) {
this.id = id;
this.addr = addr;
this.electionAddr = electionAddr;
this.type = type;
this.clientAddr = null;
}
public QuorumServer(long id, InetSocketAddress addr,
InetSocketAddress electionAddr, InetSocketAddress clientAddr, LearnerType type) {
this.id = id;
this.addr = addr;
this.electionAddr = electionAddr;
this.type = type;
this.clientAddr = clientAddr;
}
public String toString(){
StringWriter sw = new StringWriter();
//addr should never be null, but just to make sure
if (addr !=null) {
sw.append(addr.getHostString());
sw.append(":");
sw.append(String.valueOf(addr.getPort()));
}
if (electionAddr!=null){
sw.append(":");
sw.append(String.valueOf(electionAddr.getPort()));
}
if (type == LearnerType.OBSERVER) sw.append(":observer");
else if (type == LearnerType.PARTICIPANT) sw.append(":participant");
if (clientAddr!=null){
sw.append(";");
sw.append(clientAddr.getHostString());
sw.append(":");
sw.append(String.valueOf(clientAddr.getPort()));
}
return sw.toString();
}
public int hashCode() {
assert false : "hashCode not designed";
return 42; // any arbitrary constant will do
}
private boolean checkAddressesEqual(InetSocketAddress addr1, InetSocketAddress addr2){
if ((addr1 == null && addr2!=null) ||
(addr1!=null && addr2==null) ||
(addr1!=null && addr2!=null && !addr1.equals(addr2))) return false;
return true;
}
public boolean equals(Object o){
if (!(o instanceof QuorumServer)) return false;
QuorumServer qs = (QuorumServer)o;
if ((qs.id != id) || (qs.type != type)) return false;
if (!checkAddressesEqual(addr, qs.addr)) return false;
if (!checkAddressesEqual(electionAddr, qs.electionAddr)) return false;
if (!checkAddressesEqual(clientAddr, qs.clientAddr)) return false;
return true;
}
}
public enum ServerState {
LOOKING, FOLLOWING, LEADING, OBSERVING;
}
/*
* A peer can either be participating, which implies that it is willing to
* both vote in instances of consensus and to elect or become a Leader, or
* it may be observing in which case it isn't.
*
* We need this distinction to decide which ServerState to move to when
* conditions change (e.g. which state to become after LOOKING).
*/
public enum LearnerType {
PARTICIPANT, OBSERVER;
}
/*
* To enable observers to have no identifier, we need a generic identifier
* at least for QuorumCnxManager. We use the following constant to as the
* value of such a generic identifier.
*/
static final long OBSERVER_ID = Long.MAX_VALUE;
/*
* Record leader election time
*/
public long start_fle, end_fle;
/*
* Default value of peer is participant
*/
private LearnerType learnerType = LearnerType.PARTICIPANT;
public LearnerType getLearnerType() {
return learnerType;
}
/**
* Sets the LearnerType
*/
public void setLearnerType(LearnerType p) {
learnerType = p;
}
protected synchronized void setConfigFileName(String s) {
configFilename = s;
}
private String configFilename = null;
public int getQuorumSize(){
return getVotingView().size();
}
/**
* QuorumVerifier implementation; default (majority).
*/
//last committed quorum verifier
public QuorumVerifier quorumVerifier;
//last proposed quorum verifier
public QuorumVerifier lastSeenQuorumVerifier = null;
/**
* My id
*/
private long myid;
/**
* get the id of this quorum peer.
*/
public long getId() {
return myid;
}
/**
* This is who I think the leader currently is.
*/
volatile private Vote currentVote;
public synchronized Vote getCurrentVote(){
return currentVote;
}
public synchronized void setCurrentVote(Vote v){
currentVote = v;
}
volatile boolean running = true;
/**
* The number of milliseconds of each tick
*/
protected int tickTime;
/**
* Whether learners in this quorum should create new sessions as local.
* False by default to preserve existing behavior.
*/
protected boolean localSessionsEnabled = false;
/**
* Whether learners in this quorum should upgrade local sessions to
* global. Only matters if local sessions are enabled.
*/
protected boolean localSessionsUpgradingEnabled = true;
/**
* Minimum number of milliseconds to allow for session timeout.
* A value of -1 indicates unset, use default.
*/
protected int minSessionTimeout = -1;
/**
* Maximum number of milliseconds to allow for session timeout.
* A value of -1 indicates unset, use default.
*/
protected int maxSessionTimeout = -1;
/**
* The number of ticks that the initial synchronization phase can take
*/
protected int initLimit;
/**
* The number of ticks that can pass between sending a request and getting
* an acknowledgment
*/
protected int syncLimit;
/**
* Enables/Disables sync request processor. This option is enabled
* by default and is to be used with observers.
*/
protected boolean syncEnabled = true;
/**
* The current tick
*/
protected AtomicInteger tick = new AtomicInteger();
/**
* Whether or not to listen on all IPs for the two quorum ports
* (broadcast and fast leader election).
*/
protected boolean quorumListenOnAllIPs = false;
/**
* @deprecated As of release 3.4.0, this class has been deprecated, since
* it is used with one of the udp-based versions of leader election, which
* we are also deprecating.
*
* This class simply responds to requests for the current leader of this
* node.
* <p>
* The request contains just an xid generated by the requestor.
* <p>
* The response has the xid, the id of this server, the id of the leader,
* and the zxid of the leader.
*
*
*/
@Deprecated
class ResponderThread extends ZooKeeperThread {
ResponderThread() {
super("ResponderThread");
}
volatile boolean running = true;
@Override
public void run() {
try {
byte b[] = new byte[36];
ByteBuffer responseBuffer = ByteBuffer.wrap(b);
DatagramPacket packet = new DatagramPacket(b, b.length);
while (running) {
udpSocket.receive(packet);
if (packet.getLength() != 4) {
LOG.warn("Got more than just an xid! Len = "
+ packet.getLength());
} else {
responseBuffer.clear();
responseBuffer.getInt(); // Skip the xid
responseBuffer.putLong(myid);
Vote current = getCurrentVote();
switch (getPeerState()) {
case LOOKING:
responseBuffer.putLong(current.getId());
responseBuffer.putLong(current.getZxid());
break;
case LEADING:
responseBuffer.putLong(myid);
try {
long proposed;
synchronized(leader) {
proposed = leader.lastProposed;
}
responseBuffer.putLong(proposed);
} catch (NullPointerException npe) {
// This can happen in state transitions,
// just ignore the request
}
break;
case FOLLOWING:
responseBuffer.putLong(current.getId());
try {
responseBuffer.putLong(follower.getZxid());
} catch (NullPointerException npe) {
// This can happen in state transitions,
// just ignore the request
}
break;
case OBSERVING:
// Do nothing, Observers keep themselves to
// themselves.
break;
}
packet.setData(b);
udpSocket.send(packet);
}
packet.setLength(b.length);
}
} catch (RuntimeException e) {
LOG.warn("Unexpected runtime exception in ResponderThread",e);
} catch (IOException e) {
LOG.warn("Unexpected IO exception in ResponderThread",e);
} finally {
LOG.warn("QuorumPeer responder thread exited");
}
}
}
private ServerState state = ServerState.LOOKING;
private boolean reconfigFlag = false; // indicates that a reconfig just committed
public synchronized void setPeerState(ServerState newState){
state=newState;
}
public synchronized void reconfigFlagSet(){
reconfigFlag = true;
}
public synchronized void reconfigFlagClear(){
reconfigFlag = false;
}
public synchronized boolean isReconfigStateChange(){
return reconfigFlag;
}
public synchronized ServerState getPeerState(){
return state;
}
DatagramSocket udpSocket;
private InetSocketAddress myQuorumAddr;
private InetSocketAddress myElectionAddr = null;
private InetSocketAddress myClientAddr = null;
/**
* Resolves hostname for a given server ID.
*
* This method resolves hostname for a given server ID in both quorumVerifer
* and lastSeenQuorumVerifier. If the server ID matches the local server ID,
* it also updates myQuorumAddr and myElectionAddr.
*/
public void recreateSocketAddresses(long id) {
QuorumVerifier qv = getQuorumVerifier();
if (qv != null) {
QuorumServer qs = qv.getAllMembers().get(id);
if (qs != null) {
qs.recreateSocketAddresses();
if (id == getId()) {
setQuorumAddress(qs.addr);
setElectionAddress(qs.electionAddr);
}
}
}
qv = getLastSeenQuorumVerifier();
if (qv != null) {
QuorumServer qs = qv.getAllMembers().get(id);
if (qs != null) {
qs.recreateSocketAddresses();
}
}
}
public synchronized InetSocketAddress getQuorumAddress(){
return myQuorumAddr;
}
public synchronized void setQuorumAddress(InetSocketAddress addr){
myQuorumAddr = addr;
}
public InetSocketAddress getElectionAddress(){
return myElectionAddr;
}
public void setElectionAddress(InetSocketAddress addr){
myElectionAddr = addr;
}
public InetSocketAddress getClientAddress(){
return myClientAddr;
}
public void setClientAddress(InetSocketAddress addr){
myClientAddr = addr;
}
private int electionType;
Election electionAlg;
ServerCnxnFactory cnxnFactory;
ServerCnxnFactory secureCnxnFactory;
private FileTxnSnapLog logFactory = null;
private final QuorumStats quorumStats;
AdminServer adminServer;
public QuorumPeer() {
super("QuorumPeer");
quorumStats = new QuorumStats(this);
jmxRemotePeerBean = new HashMap<Long, RemotePeerBean>();
adminServer = AdminServerFactory.createAdminServer();
}
/**
* For backward compatibility purposes, we instantiate QuorumMaj by default.
*/
public QuorumPeer(Map<Long, QuorumServer> quorumPeers, File dataDir,
File dataLogDir, int electionType,
long myid, int tickTime, int initLimit, int syncLimit,
ServerCnxnFactory cnxnFactory) throws IOException {
this(quorumPeers, dataDir, dataLogDir, electionType, myid, tickTime,
initLimit, syncLimit, false, cnxnFactory,
new QuorumMaj(quorumPeers));
}
public QuorumPeer(Map<Long, QuorumServer> quorumPeers, File dataDir,
File dataLogDir, int electionType,
long myid, int tickTime, int initLimit, int syncLimit,
boolean quorumListenOnAllIPs,
ServerCnxnFactory cnxnFactory,
QuorumVerifier quorumConfig) throws IOException {
this();
this.cnxnFactory = cnxnFactory;
this.electionType = electionType;
this.myid = myid;
this.tickTime = tickTime;
this.initLimit = initLimit;
this.syncLimit = syncLimit;
this.quorumListenOnAllIPs = quorumListenOnAllIPs;
this.logFactory = new FileTxnSnapLog(dataLogDir, dataDir);
this.zkDb = new ZKDatabase(this.logFactory);
if(quorumConfig == null) quorumConfig = new QuorumMaj(quorumPeers);
setQuorumVerifier(quorumConfig, false);
adminServer = AdminServerFactory.createAdminServer();
}
QuorumStats quorumStats() {
return quorumStats;
}
@Override
public synchronized void start() {
if (!getView().containsKey(myid)) {
throw new RuntimeException("My id " + myid + " not in the peer list");
}
loadDataBase();
startServerCnxnFactory();
try {
adminServer.start();
} catch (AdminServerException e) {
LOG.warn("Problem starting AdminServer", e);
System.out.println(e);
}
startLeaderElection();
super.start();
}
private void loadDataBase() {
try {
zkDb.loadDataBase();
// load the epochs
long lastProcessedZxid = zkDb.getDataTree().lastProcessedZxid;
long epochOfZxid = ZxidUtils.getEpochFromZxid(lastProcessedZxid);
try {
currentEpoch = readLongFromFile(CURRENT_EPOCH_FILENAME);
} catch(FileNotFoundException e) {
// pick a reasonable epoch number
// this should only happen once when moving to a
// new code version
currentEpoch = epochOfZxid;
LOG.info(CURRENT_EPOCH_FILENAME
+ " not found! Creating with a reasonable default of {}. This should only happen when you are upgrading your installation",
currentEpoch);
writeLongToFile(CURRENT_EPOCH_FILENAME, currentEpoch);
}
if (epochOfZxid > currentEpoch) {
throw new IOException("The current epoch, " + ZxidUtils.zxidToString(currentEpoch) + ", is older than the last zxid, " + lastProcessedZxid);
}
try {
acceptedEpoch = readLongFromFile(ACCEPTED_EPOCH_FILENAME);
} catch(FileNotFoundException e) {
// pick a reasonable epoch number
// this should only happen once when moving to a
// new code version
acceptedEpoch = epochOfZxid;
LOG.info(ACCEPTED_EPOCH_FILENAME
+ " not found! Creating with a reasonable default of {}. This should only happen when you are upgrading your installation",
acceptedEpoch);
writeLongToFile(ACCEPTED_EPOCH_FILENAME, acceptedEpoch);
}
if (acceptedEpoch < currentEpoch) {
throw new IOException("The current epoch, " + ZxidUtils.zxidToString(currentEpoch) + " is less than the accepted epoch, " + ZxidUtils.zxidToString(acceptedEpoch));
}
} catch(IOException ie) {
LOG.error("Unable to load database on disk", ie);
throw new RuntimeException("Unable to run quorum server ", ie);
}
}
ResponderThread responder;
synchronized public void stopLeaderElection() {
responder.running = false;
responder.interrupt();
}
synchronized public void startLeaderElection() {
try {
if (getPeerState() == ServerState.LOOKING) {
currentVote = new Vote(myid, getLastLoggedZxid(), getCurrentEpoch());
}
} catch(IOException e) {
RuntimeException re = new RuntimeException(e.getMessage());
re.setStackTrace(e.getStackTrace());
throw re;
}
// if (!getView().containsKey(myid)) {
// throw new RuntimeException("My id " + myid + " not in the peer list");
//}
if (electionType == 0) {
try {
udpSocket = new DatagramSocket(myQuorumAddr.getPort());
responder = new ResponderThread();
responder.start();
} catch (SocketException e) {
throw new RuntimeException(e);
}
}
this.electionAlg = createElectionAlgorithm(electionType);
}
/**
* Count the number of nodes in the map that could be followers.
* @param peers
* @return The number of followers in the map
*/
protected static int countParticipants(Map<Long,QuorumServer> peers) {
int count = 0;
for (QuorumServer q : peers.values()) {
if (q.type == LearnerType.PARTICIPANT) {
count++;
}
}
return count;
}
/**
* This constructor is only used by the existing unit test code.
* It defaults to FileLogProvider persistence provider.
*/
public QuorumPeer(Map<Long,QuorumServer> quorumPeers, File snapDir,
File logDir, int clientPort, int electionAlg,
long myid, int tickTime, int initLimit, int syncLimit)
throws IOException
{
this(quorumPeers, snapDir, logDir, electionAlg,
myid,tickTime, initLimit,syncLimit, false,
ServerCnxnFactory.createFactory(new InetSocketAddress(clientPort), -1),
new QuorumMaj(quorumPeers));
}
/**
* This constructor is only used by the existing unit test code.
* It defaults to FileLogProvider persistence provider.
*/
public QuorumPeer(Map<Long,QuorumServer> quorumPeers, File snapDir,
File logDir, int clientPort, int electionAlg,
long myid, int tickTime, int initLimit, int syncLimit,
QuorumVerifier quorumConfig)
throws IOException
{
this(quorumPeers, snapDir, logDir, electionAlg,
myid,tickTime, initLimit,syncLimit, false,
ServerCnxnFactory.createFactory(new InetSocketAddress(clientPort), -1),
quorumConfig);
}
/**
* returns the highest zxid that this host has seen
*
* @return the highest zxid for this host
*/
public long getLastLoggedZxid() {
if (!zkDb.isInitialized()) {
loadDataBase();
}
return zkDb.getDataTreeLastProcessedZxid();
}
public Follower follower;
public Leader leader;
public Observer observer;
protected Follower makeFollower(FileTxnSnapLog logFactory) throws IOException {
return new Follower(this, new FollowerZooKeeperServer(logFactory, this, this.zkDb));
}
protected Leader makeLeader(FileTxnSnapLog logFactory) throws IOException {
return new Leader(this, new LeaderZooKeeperServer(logFactory, this, this.zkDb));
}
protected Observer makeObserver(FileTxnSnapLog logFactory) throws IOException {
return new Observer(this, new ObserverZooKeeperServer(logFactory, this, this.zkDb));
}
@SuppressWarnings("deprecation")
protected Election createElectionAlgorithm(int electionAlgorithm){
Election le=null;
//TODO: use a factory rather than a switch
switch (electionAlgorithm) {
case 0:
le = new LeaderElection(this);
break;
case 1:
le = new AuthFastLeaderElection(this);
break;
case 2:
le = new AuthFastLeaderElection(this, true);
break;
case 3:
qcm = new QuorumCnxManager(this);
QuorumCnxManager.Listener listener = qcm.listener;
if(listener != null){
listener.start();
FastLeaderElection fle = new FastLeaderElection(this, qcm);
fle.start();
le = fle;
} else {
LOG.error("Null listener when initializing cnx manager");
}
break;
default:
assert false;
}
return le;
}
@SuppressWarnings("deprecation")
protected Election makeLEStrategy(){
LOG.debug("Initializing leader election protocol...");
if (getElectionType() == 0) {
electionAlg = new LeaderElection(this);
}
return electionAlg;
}
synchronized protected void setLeader(Leader newLeader){
leader=newLeader;
}
synchronized protected void setFollower(Follower newFollower){
follower=newFollower;
}
synchronized protected void setObserver(Observer newObserver){
observer=newObserver;
}
synchronized public ZooKeeperServer getActiveServer(){
if(leader!=null)
return leader.zk;
else if(follower!=null)
return follower.zk;
else if (observer != null)
return observer.zk;
return null;
}
boolean shuttingDownLE = false;
@Override
public void run() {
updateThreadName();
LOG.debug("Starting quorum peer");
try {
jmxQuorumBean = new QuorumBean(this);
MBeanRegistry.getInstance().register(jmxQuorumBean, null);
for(QuorumServer s: getView().values()){
ZKMBeanInfo p;
if (getId() == s.id) {
p = jmxLocalPeerBean = new LocalPeerBean(this);
try {
MBeanRegistry.getInstance().register(p, jmxQuorumBean);
} catch (Exception e) {
LOG.warn("Failed to register with JMX", e);
jmxLocalPeerBean = null;
}
} else {
RemotePeerBean rBean = new RemotePeerBean(s);
try {
MBeanRegistry.getInstance().register(rBean, jmxQuorumBean);
jmxRemotePeerBean.put(s.id, rBean);
} catch (Exception e) {
LOG.warn("Failed to register with JMX", e);
}
}
}
} catch (Exception e) {
LOG.warn("Failed to register with JMX", e);
jmxQuorumBean = null;
}
try {
/*
* Main loop
*/
while (running) {
switch (getPeerState()) {
case LOOKING:
LOG.info("LOOKING");
if (Boolean.getBoolean("readonlymode.enabled")) {
LOG.info("Attempting to start ReadOnlyZooKeeperServer");
// Create read-only server but don't start it immediately
final ReadOnlyZooKeeperServer roZk =
new ReadOnlyZooKeeperServer(logFactory, this, this.zkDb);
// Instead of starting roZk immediately, wait some grace
// period before we decide we're partitioned.
//
// Thread is used here because otherwise it would require
// changes in each of election strategy classes which is
// unnecessary code coupling.
Thread roZkMgr = new Thread() {
public void run() {
try {
// lower-bound grace period to 2 secs
sleep(Math.max(2000, tickTime));
if (ServerState.LOOKING.equals(getPeerState())) {
roZk.startup();
}
} catch (InterruptedException e) {
LOG.info("Interrupted while attempting to start ReadOnlyZooKeeperServer, not started");
} catch (Exception e) {
LOG.error("FAILED to start ReadOnlyZooKeeperServer", e);
}
}
};
try {
roZkMgr.start();
reconfigFlagClear();
if (shuttingDownLE) {
shuttingDownLE = false;
startLeaderElection();
}
setCurrentVote(makeLEStrategy().lookForLeader());
} catch (Exception e) {
LOG.warn("Unexpected exception", e);
setPeerState(ServerState.LOOKING);
} finally {
// If the thread is in the the grace period, interrupt
// to come out of waiting.
roZkMgr.interrupt();
roZk.shutdown();
}
} else {
try {
reconfigFlagClear();
if (shuttingDownLE) {
shuttingDownLE = false;
startLeaderElection();
}
setCurrentVote(makeLEStrategy().lookForLeader());
} catch (Exception e) {