forked from apache/zookeeper
-
Notifications
You must be signed in to change notification settings - Fork 0
/
QuorumCnxManager.java
1478 lines (1301 loc) · 53.1 KB
/
QuorumCnxManager.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 static org.apache.zookeeper.common.NetUtils.formatInetAddr;
import java.io.BufferedInputStream;
import java.io.BufferedOutputStream;
import java.io.Closeable;
import java.io.DataInputStream;
import java.io.DataOutputStream;
import java.io.IOException;
import java.net.InetAddress;
import java.net.InetSocketAddress;
import java.net.NoRouteToHostException;
import java.net.ServerSocket;
import java.net.Socket;
import java.net.SocketException;
import java.net.SocketTimeoutException;
import java.nio.BufferUnderflowException;
import java.nio.ByteBuffer;
import java.nio.channels.UnresolvedAddressException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.Enumeration;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.NoSuchElementException;
import java.util.Set;
import java.util.concurrent.ArrayBlockingQueue;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.SynchronousQueue;
import java.util.concurrent.ThreadFactory;
import java.util.concurrent.ThreadPoolExecutor;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicLong;
import java.util.stream.Collectors;
import javax.net.ssl.SSLSocket;
import org.apache.zookeeper.common.NetUtils;
import org.apache.zookeeper.common.X509Exception;
import org.apache.zookeeper.server.ExitCode;
import org.apache.zookeeper.server.ZooKeeperThread;
import org.apache.zookeeper.server.quorum.QuorumPeerConfig.ConfigException;
import org.apache.zookeeper.server.quorum.auth.QuorumAuthLearner;
import org.apache.zookeeper.server.quorum.auth.QuorumAuthServer;
import org.apache.zookeeper.server.quorum.flexible.QuorumVerifier;
import org.apache.zookeeper.server.util.ConfigUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* This class implements a connection manager for leader election using TCP. It
* maintains one connection for every pair of servers. The tricky part is to
* guarantee that there is exactly one connection for every pair of servers that
* are operating correctly and that can communicate over the network.
*
* If two servers try to start a connection concurrently, then the connection
* manager uses a very simple tie-breaking mechanism to decide which connection
* to drop based on the IP addressed of the two parties.
*
* For every peer, the manager maintains a queue of messages to send. If the
* connection to any particular peer drops, then the sender thread puts the
* message back on the list. As this implementation currently uses a queue
* implementation to maintain messages to send to another peer, we add the
* message to the tail of the queue, thus changing the order of messages.
* Although this is not a problem for the leader election, it could be a problem
* when consolidating peer communication. This is to be verified, though.
*
*/
public class QuorumCnxManager {
private static final Logger LOG = LoggerFactory.getLogger(QuorumCnxManager.class);
/*
* Maximum capacity of thread queues
*/
static final int RECV_CAPACITY = 100;
// Initialized to 1 to prevent sending
// stale notifications to peers
static final int SEND_CAPACITY = 1;
static final int PACKETMAXSIZE = 1024 * 512;
/*
* Negative counter for observer server ids.
*/
private AtomicLong observerCounter = new AtomicLong(-1);
/*
* Protocol identifier used among peers
*/
public static final long PROTOCOL_VERSION = -65535L;
/*
* Max buffer size to be read from the network.
*/
public static final int maxBuffer = 2048;
/*
* Connection time out value in milliseconds
*/
private int cnxTO = 5000;
final QuorumPeer self;
/*
* Local IP address
*/
final long mySid;
final int socketTimeout;
final Map<Long, QuorumPeer.QuorumServer> view;
final boolean listenOnAllIPs;
private ThreadPoolExecutor connectionExecutor;
private final Set<Long> inprogressConnections = Collections.synchronizedSet(new HashSet<>());
private QuorumAuthServer authServer;
private QuorumAuthLearner authLearner;
private boolean quorumSaslAuthEnabled;
/*
* Counter to count connection processing threads.
*/
private AtomicInteger connectionThreadCnt = new AtomicInteger(0);
/*
* Mapping from Peer to Thread number
*/
final ConcurrentHashMap<Long, SendWorker> senderWorkerMap;
final ConcurrentHashMap<Long, ArrayBlockingQueue<ByteBuffer>> queueSendMap;
final ConcurrentHashMap<Long, ByteBuffer> lastMessageSent;
/*
* Reception queue
*/
public final ArrayBlockingQueue<Message> recvQueue;
/*
* Object to synchronize access to recvQueue
*/
private final Object recvQLock = new Object();
/*
* Shutdown flag
*/
volatile boolean shutdown = false;
/*
* Listener thread
*/
public final Listener listener;
/*
* Counter to count worker threads
*/
private AtomicInteger threadCnt = new AtomicInteger(0);
/*
* Socket options for TCP keepalive
*/
private final boolean tcpKeepAlive = Boolean.getBoolean("zookeeper.tcpKeepAlive");
public static class Message {
Message(ByteBuffer buffer, long sid) {
this.buffer = buffer;
this.sid = sid;
}
ByteBuffer buffer;
long sid;
}
/*
* This class parses the initial identification sent out by peers with their
* sid & hostname.
*/
public static class InitialMessage {
public Long sid;
public List<InetSocketAddress> electionAddr;
InitialMessage(Long sid, List<InetSocketAddress> addresses) {
this.sid = sid;
this.electionAddr = addresses;
}
@SuppressWarnings("serial")
public static class InitialMessageException extends Exception {
InitialMessageException(String message, Object... args) {
super(String.format(message, args));
}
}
public static InitialMessage parse(Long protocolVersion, DataInputStream din) throws InitialMessageException, IOException {
Long sid;
if (protocolVersion != PROTOCOL_VERSION) {
throw new InitialMessageException("Got unrecognized protocol version %s", protocolVersion);
}
sid = din.readLong();
int remaining = din.readInt();
if (remaining <= 0 || remaining > maxBuffer) {
throw new InitialMessageException("Unreasonable buffer length: %s", remaining);
}
byte[] b = new byte[remaining];
int num_read = din.read(b);
if (num_read != remaining) {
throw new InitialMessageException("Read only %s bytes out of %s sent by server %s", num_read, remaining, sid);
}
String[] addressStrings = new String(b).split("\\|");
List<InetSocketAddress> addresses = new ArrayList<>(addressStrings.length);
for (String addr : addressStrings) {
String[] host_port;
try {
host_port = ConfigUtils.getHostAndPort(addr);
} catch (ConfigException e) {
throw new InitialMessageException("Badly formed address: %s", addr);
}
if (host_port.length != 2) {
throw new InitialMessageException("Badly formed address: %s", addr);
}
int port;
try {
port = Integer.parseInt(host_port[1]);
} catch (NumberFormatException e) {
throw new InitialMessageException("Bad port number: %s", host_port[1]);
} catch (ArrayIndexOutOfBoundsException e) {
throw new InitialMessageException("No port number in: %s", addr);
}
addresses.add(new InetSocketAddress(host_port[0], port));
}
return new InitialMessage(sid, addresses);
}
}
public QuorumCnxManager(QuorumPeer self, final long mySid, Map<Long, QuorumPeer.QuorumServer> view,
QuorumAuthServer authServer, QuorumAuthLearner authLearner, int socketTimeout, boolean listenOnAllIPs,
int quorumCnxnThreadsSize, boolean quorumSaslAuthEnabled) {
this.recvQueue = new ArrayBlockingQueue<>(RECV_CAPACITY);
this.queueSendMap = new ConcurrentHashMap<>();
this.senderWorkerMap = new ConcurrentHashMap<>();
this.lastMessageSent = new ConcurrentHashMap<>();
String cnxToValue = System.getProperty("zookeeper.cnxTimeout");
if (cnxToValue != null) {
this.cnxTO = Integer.parseInt(cnxToValue);
}
this.self = self;
this.mySid = mySid;
this.socketTimeout = socketTimeout;
this.view = view;
this.listenOnAllIPs = listenOnAllIPs;
initializeAuth(mySid, authServer, authLearner, quorumCnxnThreadsSize, quorumSaslAuthEnabled);
// Starts listener thread that waits for connection requests
listener = new Listener();
listener.setName("QuorumPeerListener");
}
private void initializeAuth(final long mySid, final QuorumAuthServer authServer,
final QuorumAuthLearner authLearner, final int quorumCnxnThreadsSize, final boolean quorumSaslAuthEnabled) {
this.authServer = authServer;
this.authLearner = authLearner;
this.quorumSaslAuthEnabled = quorumSaslAuthEnabled;
if (!this.quorumSaslAuthEnabled) {
LOG.debug("Not initializing connection executor as quorum sasl auth is disabled");
return;
}
// init connection executors
final AtomicInteger threadIndex = new AtomicInteger(1);
SecurityManager s = System.getSecurityManager();
final ThreadGroup group = (s != null) ? s.getThreadGroup() : Thread.currentThread().getThreadGroup();
ThreadFactory daemonThFactory = new ThreadFactory() {
@Override
public Thread newThread(Runnable r) {
Thread t = new Thread(
group,
r,
"QuorumConnectionThread-[myid=" + mySid + "]-" + threadIndex.getAndIncrement());
return t;
}
};
this.connectionExecutor = new ThreadPoolExecutor(3, quorumCnxnThreadsSize, 60, TimeUnit.SECONDS, new SynchronousQueue<Runnable>(), daemonThFactory);
this.connectionExecutor.allowCoreThreadTimeOut(true);
}
/**
* Invokes initiateConnection for testing purposes
*
* @param sid
*/
public void testInitiateConnection(long sid) throws Exception {
LOG.debug("Opening channel to server {}", sid);
Socket sock = new Socket();
setSockOpts(sock);
InetSocketAddress address = self.getVotingView().get(sid).electionAddr.getReachableOrOne();
sock.connect(address, cnxTO);
initiateConnection(sock, sid);
}
/**
* If this server has initiated the connection, then it gives up on the
* connection if it loses challenge. Otherwise, it keeps the connection.
*/
public void initiateConnection(final Socket sock, final Long sid) {
try {
startConnection(sock, sid);
} catch (IOException e) {
LOG.error(
"Exception while connecting, id: {}, addr: {}, closing learner connection",
sid,
sock.getRemoteSocketAddress(),
e);
closeSocket(sock);
}
}
/**
* Server will initiate the connection request to its peer server
* asynchronously via separate connection thread.
*/
public void initiateConnectionAsync(final Socket sock, final Long sid) {
if (!inprogressConnections.add(sid)) {
// simply return as there is a connection request to
// server 'sid' already in progress.
LOG.debug("Connection request to server id: {} is already in progress, so skipping this request", sid);
closeSocket(sock);
return;
}
try {
connectionExecutor.execute(new QuorumConnectionReqThread(sock, sid));
connectionThreadCnt.incrementAndGet();
} catch (Throwable e) {
// Imp: Safer side catching all type of exceptions and remove 'sid'
// from inprogress connections. This is to avoid blocking further
// connection requests from this 'sid' in case of errors.
inprogressConnections.remove(sid);
LOG.error("Exception while submitting quorum connection request", e);
closeSocket(sock);
}
}
/**
* Thread to send connection request to peer server.
*/
private class QuorumConnectionReqThread extends ZooKeeperThread {
final Socket sock;
final Long sid;
QuorumConnectionReqThread(final Socket sock, final Long sid) {
super("QuorumConnectionReqThread-" + sid);
this.sock = sock;
this.sid = sid;
}
@Override
public void run() {
try {
initiateConnection(sock, sid);
} finally {
inprogressConnections.remove(sid);
}
}
}
private boolean startConnection(Socket sock, Long sid) throws IOException {
DataOutputStream dout = null;
DataInputStream din = null;
try {
// Use BufferedOutputStream to reduce the number of IP packets. This is
// important for x-DC scenarios.
BufferedOutputStream buf = new BufferedOutputStream(sock.getOutputStream());
dout = new DataOutputStream(buf);
// Sending id and challenge
// represents protocol version (in other words - message type)
dout.writeLong(PROTOCOL_VERSION);
dout.writeLong(self.getId());
String addr = self.getElectionAddress().getAllAddresses().stream()
.map(NetUtils::formatInetAddr).collect(Collectors.joining("|"));
byte[] addr_bytes = addr.getBytes();
dout.writeInt(addr_bytes.length);
dout.write(addr_bytes);
dout.flush();
din = new DataInputStream(new BufferedInputStream(sock.getInputStream()));
} catch (IOException e) {
LOG.warn("Ignoring exception reading or writing challenge: ", e);
closeSocket(sock);
return false;
}
// authenticate learner
QuorumPeer.QuorumServer qps = self.getVotingView().get(sid);
if (qps != null) {
// TODO - investigate why reconfig makes qps null.
authLearner.authenticate(sock, qps.hostname);
}
// If lost the challenge, then drop the new connection
if (sid > self.getId()) {
LOG.info(
"Have smaller server identifier, so dropping the connection: ({}, {})",
sid,
self.getId());
closeSocket(sock);
// Otherwise proceed with the connection
} else {
SendWorker sw = new SendWorker(sock, sid);
RecvWorker rw = new RecvWorker(sock, din, sid, sw);
sw.setRecv(rw);
SendWorker vsw = senderWorkerMap.get(sid);
if (vsw != null) {
vsw.finish();
}
senderWorkerMap.put(sid, sw);
queueSendMap.putIfAbsent(sid, new ArrayBlockingQueue<>(SEND_CAPACITY));
sw.start();
rw.start();
return true;
}
return false;
}
/**
* If this server receives a connection request, then it gives up on the new
* connection if it wins. Notice that it checks whether it has a connection
* to this server already or not. If it does, then it sends the smallest
* possible long value to lose the challenge.
*
*/
public void receiveConnection(final Socket sock) {
DataInputStream din = null;
try {
din = new DataInputStream(new BufferedInputStream(sock.getInputStream()));
handleConnection(sock, din);
} catch (IOException e) {
LOG.error("Exception handling connection, addr: {}, closing server connection", sock.getRemoteSocketAddress());
closeSocket(sock);
}
}
/**
* Server receives a connection request and handles it asynchronously via
* separate thread.
*/
public void receiveConnectionAsync(final Socket sock) {
try {
connectionExecutor.execute(new QuorumConnectionReceiverThread(sock));
connectionThreadCnt.incrementAndGet();
} catch (Throwable e) {
LOG.error("Exception handling connection, addr: {}, closing server connection", sock.getRemoteSocketAddress());
closeSocket(sock);
}
}
/**
* Thread to receive connection request from peer server.
*/
private class QuorumConnectionReceiverThread extends ZooKeeperThread {
private final Socket sock;
QuorumConnectionReceiverThread(final Socket sock) {
super("QuorumConnectionReceiverThread-" + sock.getRemoteSocketAddress());
this.sock = sock;
}
@Override
public void run() {
receiveConnection(sock);
}
}
private void handleConnection(Socket sock, DataInputStream din) throws IOException {
Long sid = null, protocolVersion = null;
MultipleAddresses electionAddr = null;
try {
protocolVersion = din.readLong();
if (protocolVersion >= 0) { // this is a server id and not a protocol version
sid = protocolVersion;
} else {
try {
InitialMessage init = InitialMessage.parse(protocolVersion, din);
sid = init.sid;
electionAddr = new MultipleAddresses(init.electionAddr);
} catch (InitialMessage.InitialMessageException ex) {
LOG.error(ex.toString());
closeSocket(sock);
return;
}
}
if (sid == QuorumPeer.OBSERVER_ID) {
/*
* Choose identifier at random. We need a value to identify
* the connection.
*/
sid = observerCounter.getAndDecrement();
LOG.info("Setting arbitrary identifier to observer: {}", sid);
}
} catch (IOException e) {
LOG.warn("Exception reading or writing challenge", e);
closeSocket(sock);
return;
}
// do authenticating learner
authServer.authenticate(sock, din);
//If wins the challenge, then close the new connection.
if (sid < self.getId()) {
/*
* This replica might still believe that the connection to sid is
* up, so we have to shut down the workers before trying to open a
* new connection.
*/
SendWorker sw = senderWorkerMap.get(sid);
if (sw != null) {
sw.finish();
}
/*
* Now we start a new connection
*/
LOG.debug("Create new connection to server: {}", sid);
closeSocket(sock);
if (electionAddr != null) {
connectOne(sid, electionAddr);
} else {
connectOne(sid);
}
} else { // Otherwise start worker threads to receive data.
SendWorker sw = new SendWorker(sock, sid);
RecvWorker rw = new RecvWorker(sock, din, sid, sw);
sw.setRecv(rw);
SendWorker vsw = senderWorkerMap.get(sid);
if (vsw != null) {
vsw.finish();
}
senderWorkerMap.put(sid, sw);
queueSendMap.putIfAbsent(sid, new ArrayBlockingQueue<>(SEND_CAPACITY));
sw.start();
rw.start();
}
}
/**
* Processes invoke this message to queue a message to send. Currently,
* only leader election uses it.
*/
public void toSend(Long sid, ByteBuffer b) {
/*
* If sending message to myself, then simply enqueue it (loopback).
*/
if (this.mySid == sid) {
b.position(0);
addToRecvQueue(new Message(b.duplicate(), sid));
/*
* Otherwise send to the corresponding thread to send.
*/
} else {
/*
* Start a new connection if doesn't have one already.
*/
ArrayBlockingQueue<ByteBuffer> bq = queueSendMap.computeIfAbsent(sid, serverId -> new ArrayBlockingQueue<>(SEND_CAPACITY));
addToSendQueue(bq, b);
connectOne(sid);
}
}
/**
* Try to establish a connection to server with id sid using its electionAddr.
*
* VisibleForTesting.
*
* @param sid server id
* @return boolean success indication
*/
synchronized boolean connectOne(long sid, MultipleAddresses electionAddr) {
if (senderWorkerMap.get(sid) != null) {
LOG.debug("There is a connection already for server {}", sid);
// since ZOOKEEPER-3188 we can use multiple election addresses to reach a server. It is possible, that the
// one we are using is already dead and we need to clean-up, so when we will create a new connection
// then we will choose an other one, which is actually reachable
senderWorkerMap.get(sid).asyncValidateIfSocketIsStillReachable();
return true;
}
Socket sock = null;
try {
LOG.debug("Opening channel to server {}", sid);
if (self.isSslQuorum()) {
sock = self.getX509Util().createSSLSocket();
} else {
sock = new Socket();
}
setSockOpts(sock);
sock.connect(electionAddr.getReachableAddress(), cnxTO);
if (sock instanceof SSLSocket) {
SSLSocket sslSock = (SSLSocket) sock;
sslSock.startHandshake();
LOG.info("SSL handshake complete with {} - {} - {}",
sslSock.getRemoteSocketAddress(),
sslSock.getSession().getProtocol(),
sslSock.getSession().getCipherSuite());
}
LOG.debug("Connected to server {} using election address: {}:{}",
sid, sock.getInetAddress(), sock.getPort());
// Sends connection request asynchronously if the quorum
// sasl authentication is enabled. This is required because
// sasl server authentication process may take few seconds to
// finish, this may delay next peer connection requests.
if (quorumSaslAuthEnabled) {
initiateConnectionAsync(sock, sid);
} else {
initiateConnection(sock, sid);
}
return true;
} catch (UnresolvedAddressException e) {
// Sun doesn't include the address that causes this
// exception to be thrown, also UAE cannot be wrapped cleanly
// so we log the exception in order to capture this critical
// detail.
LOG.warn("Cannot open channel to {} at election address {}", sid, electionAddr, e);
closeSocket(sock);
throw e;
} catch (X509Exception e) {
LOG.warn("Cannot open secure channel to {} at election address {}", sid, electionAddr, e);
closeSocket(sock);
return false;
} catch (NoRouteToHostException e) {
LOG.warn("None of the addresses ({}) are reachable for sid {}", electionAddr, sid, e);
closeSocket(sock);
return false;
} catch (IOException e) {
LOG.warn("Cannot open channel to {} at election address {}", sid, electionAddr, e);
closeSocket(sock);
return false;
}
}
/**
* Try to establish a connection to server with id sid.
*
* @param sid server id
*/
synchronized void connectOne(long sid) {
if (senderWorkerMap.get(sid) != null) {
LOG.debug("There is a connection already for server {}", sid);
// since ZOOKEEPER-3188 we can use multiple election addresses to reach a server. It is possible, that the
// one we are using is already dead and we need to clean-up, so when we will create a new connection
// then we will choose an other one, which is actually reachable
senderWorkerMap.get(sid).asyncValidateIfSocketIsStillReachable();
return;
}
synchronized (self.QV_LOCK) {
boolean knownId = false;
// Resolve hostname for the remote server before attempting to
// connect in case the underlying ip address has changed.
self.recreateSocketAddresses(sid);
Map<Long, QuorumPeer.QuorumServer> lastCommittedView = self.getView();
QuorumVerifier lastSeenQV = self.getLastSeenQuorumVerifier();
Map<Long, QuorumPeer.QuorumServer> lastProposedView = lastSeenQV.getAllMembers();
if (lastCommittedView.containsKey(sid)) {
knownId = true;
if (connectOne(sid, lastCommittedView.get(sid).electionAddr)) {
return;
}
}
if (lastSeenQV != null
&& lastProposedView.containsKey(sid)
&& (!knownId
|| (lastProposedView.get(sid).electionAddr != lastCommittedView.get(sid).electionAddr))) {
knownId = true;
if (connectOne(sid, lastProposedView.get(sid).electionAddr)) {
return;
}
}
if (!knownId) {
LOG.warn("Invalid server id: {} ", sid);
}
}
}
/**
* Try to establish a connection with each server if one
* doesn't exist.
*/
public void connectAll() {
long sid;
for (Enumeration<Long> en = queueSendMap.keys(); en.hasMoreElements(); ) {
sid = en.nextElement();
connectOne(sid);
}
}
/**
* Check if all queues are empty, indicating that all messages have been delivered.
*/
boolean haveDelivered() {
for (ArrayBlockingQueue<ByteBuffer> queue : queueSendMap.values()) {
LOG.debug("Queue size: {}", queue.size());
if (queue.size() == 0) {
return true;
}
}
return false;
}
/**
* Flag that it is time to wrap up all activities and interrupt the listener.
*/
public void halt() {
shutdown = true;
LOG.debug("Halting listener");
listener.halt();
// Wait for the listener to terminate.
try {
listener.join();
} catch (InterruptedException ex) {
LOG.warn("Got interrupted before joining the listener", ex);
}
softHalt();
// clear data structures used for auth
if (connectionExecutor != null) {
connectionExecutor.shutdown();
}
inprogressConnections.clear();
resetConnectionThreadCount();
}
/**
* A soft halt simply finishes workers.
*/
public void softHalt() {
for (SendWorker sw : senderWorkerMap.values()) {
LOG.debug("Halting sender: {}", sw);
sw.finish();
}
}
/**
* Helper method to set socket options.
*
* @param sock
* Reference to socket
*/
private void setSockOpts(Socket sock) throws SocketException {
sock.setTcpNoDelay(true);
sock.setKeepAlive(tcpKeepAlive);
sock.setSoTimeout(this.socketTimeout);
}
/**
* Helper method to close a socket.
*
* @param sock
* Reference to socket
*/
private void closeSocket(Socket sock) {
if (sock == null) {
return;
}
try {
sock.close();
} catch (IOException ie) {
LOG.error("Exception while closing", ie);
}
}
/**
* Return number of worker threads
*/
public long getThreadCount() {
return threadCnt.get();
}
/**
* Return number of connection processing threads.
*/
public long getConnectionThreadCount() {
return connectionThreadCnt.get();
}
/**
* Reset the value of connection processing threads count to zero.
*/
private void resetConnectionThreadCount() {
connectionThreadCnt.set(0);
}
/**
* Thread to listen on some ports
*/
public class Listener extends ZooKeeperThread {
private static final String ELECTION_PORT_BIND_RETRY = "zookeeper.electionPortBindRetry";
private static final int DEFAULT_PORT_BIND_MAX_RETRY = 3;
private final int portBindMaxRetry;
private Runnable socketBindErrorHandler = () -> System.exit(ExitCode.UNABLE_TO_BIND_QUORUM_PORT.getValue());
private List<ListenerHandler> listenerHandlers;
private final AtomicBoolean socketException;
public Listener() {
// During startup of thread, thread name will be overridden to
// specific election address
super("ListenerThread");
socketException = new AtomicBoolean(false);
// maximum retry count while trying to bind to election port
// see ZOOKEEPER-3320 for more details
final Integer maxRetry = Integer.getInteger(ELECTION_PORT_BIND_RETRY,
DEFAULT_PORT_BIND_MAX_RETRY);
if (maxRetry >= 0) {
LOG.info("Election port bind maximum retries is {}", maxRetry == 0 ? "infinite" : maxRetry);
portBindMaxRetry = maxRetry;
} else {
LOG.info(
"'{}' contains invalid value: {}(must be >= 0). Use default value of {} instead.",
ELECTION_PORT_BIND_RETRY,
maxRetry,
DEFAULT_PORT_BIND_MAX_RETRY);
portBindMaxRetry = DEFAULT_PORT_BIND_MAX_RETRY;
}
}
/**
* Change socket bind error handler. Used for testing.
*/
void setSocketBindErrorHandler(Runnable errorHandler) {
this.socketBindErrorHandler = errorHandler;
}
@Override
public void run() {
if (!shutdown) {
Set<InetSocketAddress> addresses;
if (self.getQuorumListenOnAllIPs()) {
addresses = self.getElectionAddress().getWildcardAddresses();
} else {
addresses = self.getElectionAddress().getAllAddresses();
}
CountDownLatch latch = new CountDownLatch(addresses.size());
listenerHandlers = addresses.stream().map(address ->
new ListenerHandler(address, self.shouldUsePortUnification(), self.isSslQuorum(), latch))
.collect(Collectors.toList());
ExecutorService executor = Executors.newFixedThreadPool(addresses.size());
listenerHandlers.forEach(executor::submit);
try {
latch.await();
} catch (InterruptedException ie) {
LOG.error("Interrupted while sleeping. Ignoring exception", ie);
} finally {
// Clean up for shutdown.
for (ListenerHandler handler : listenerHandlers) {
try {
handler.close();
} catch (IOException ie) {
// Don't log an error for shutdown.
LOG.debug("Error closing server socket", ie);
}
}
}
}
LOG.info("Leaving listener");
if (!shutdown) {
LOG.error(
"As I'm leaving the listener thread, I won't be able to participate in leader election any longer: {}",
self.getElectionAddress().getAllAddresses().stream()
.map(NetUtils::formatInetAddr)
.collect(Collectors.joining("|")));
if (socketException.get()) {
// After leaving listener thread, the host cannot join the quorum anymore,
// this is a severe error that we cannot recover from, so we need to exit
socketBindErrorHandler.run();
}
}
}
/**
* Halts this listener thread.
*/
void halt() {
LOG.debug("Trying to close listeners");
if (listenerHandlers != null) {
LOG.debug("Closing listener: {}", QuorumCnxManager.this.mySid);
for (ListenerHandler handler : listenerHandlers) {
try {
handler.close();
} catch (IOException e) {
LOG.warn("Exception when shutting down listener: ", e);
}
}
}
}
class ListenerHandler implements Runnable, Closeable {
private ServerSocket serverSocket;
private InetSocketAddress address;
private boolean portUnification;
private boolean sslQuorum;
private CountDownLatch latch;
ListenerHandler(InetSocketAddress address, boolean portUnification, boolean sslQuorum,
CountDownLatch latch) {
this.address = address;
this.portUnification = portUnification;
this.sslQuorum = sslQuorum;
this.latch = latch;
}
/**
* Sleeps on acceptConnections().
*/
@Override
public void run() {
try {
Thread.currentThread().setName("ListenerHandler-" + address);
acceptConnections();
try {
close();
} catch (IOException e) {
LOG.warn("Exception when shutting down listener: ", e);
}
} catch (Exception e) {