Skip to content

Commit a5d6bcb

Browse files
author
Mate Szalay-Beko
committed
ZOOKEEPER-3188: support for dynamic reconfig + add more unit tests
1 parent ed31d2c commit a5d6bcb

13 files changed

Lines changed: 635 additions & 33 deletions

File tree

zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Leader.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -434,7 +434,7 @@ class LearnerCnxAcceptor extends ZooKeeperCriticalThread {
434434
super("LearnerCnxAcceptor-" + serverSockets.stream()
435435
.map(ServerSocket::getLocalSocketAddress)
436436
.map(Objects::toString)
437-
.collect(Collectors.joining(",")),
437+
.collect(Collectors.joining("|")),
438438
zk.getZooKeeperServerListener());
439439
}
440440

zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Learner.java

Lines changed: 5 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -381,7 +381,11 @@ private Socket connectToLeader() throws IOException, X509Exception, InterruptedE
381381
}
382382
}
383383

384-
private Socket createSocket() throws X509Exception, IOException {
384+
/**
385+
* Creating a simple or and SSL socket.
386+
* This can be overridden in tests to fake already connected sockets for connectToLeader.
387+
*/
388+
protected Socket createSocket() throws X509Exception, IOException {
385389
Socket sock;
386390
if (self.isSslQuorum()) {
387391
sock = self.getX509Util().createSSLSocket();

zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/LocalPeerBean.java

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -83,7 +83,7 @@ public String getState() {
8383

8484
public String getQuorumAddress() {
8585
return peer.getQuorumAddress().getAllAddresses().stream().map(NetUtils::formatInetAddr)
86-
.collect(Collectors.joining(","));
86+
.collect(Collectors.joining("|"));
8787
}
8888

8989
public int getElectionType() {
@@ -92,7 +92,7 @@ public int getElectionType() {
9292

9393
public String getElectionAddress() {
9494
return peer.getElectionAddress().getAllAddresses().stream().map(NetUtils::formatInetAddr)
95-
.collect(Collectors.joining(","));
95+
.collect(Collectors.joining("|"));
9696
}
9797

9898
public String getClientAddress() {

zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/MultipleAddresses.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -208,6 +208,6 @@ public int hashCode() {
208208

209209
@Override
210210
public String toString() {
211-
return addresses.stream().map(InetSocketAddress::toString).collect(Collectors.joining(","));
211+
return addresses.stream().map(InetSocketAddress::toString).collect(Collectors.joining("|"));
212212
}
213213
}

zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumCnxManager.java

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -238,7 +238,7 @@ public static InitialMessage parse(Long protocolVersion, DataInputStream din) th
238238
throw new InitialMessageException("Read only %s bytes out of %s sent by server %s", num_read, remaining, sid);
239239
}
240240

241-
String[] addressStrings = new String(b).split(",");
241+
String[] addressStrings = new String(b).split("\\|");
242242
List<InetSocketAddress> addresses = new ArrayList<>(addressStrings.length);
243243
for (String addr : addressStrings) {
244244

@@ -420,7 +420,7 @@ private boolean startConnection(Socket sock, Long sid) throws IOException {
420420
dout.writeLong(PROTOCOL_VERSION);
421421
dout.writeLong(self.getId());
422422
String addr = self.getElectionAddress().getAllAddresses().stream()
423-
.map(NetUtils::formatInetAddr).collect(Collectors.joining(","));
423+
.map(NetUtils::formatInetAddr).collect(Collectors.joining("|"));
424424
byte[] addr_bytes = addr.getBytes();
425425
dout.writeInt(addr_bytes.length);
426426
dout.write(addr_bytes);
@@ -935,7 +935,7 @@ public void run() {
935935
+ "I won't be able to participate in leader "
936936
+ "election any longer: {}"
937937
, self.getElectionAddress().getAllAddresses().stream().map(NetUtils::formatInetAddr)
938-
.collect(Collectors.joining(",")));
938+
.collect(Collectors.joining("|")));
939939
if (socketException.get()) {
940940
// After leaving listener thread, the host cannot join the quorum anymore,
941941
// this is a severe error that we cannot recover from, so we need to exit

zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumPeer.java

Lines changed: 5 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -228,13 +228,14 @@ private LearnerType getType(String s) throws ConfigException {
228228

229229
private static final String wrongFormat =
230230
" does not have the form server_config or server_config;client_config"
231-
+ " where server_config is host:port:port or host:port:port:type and client_config is port or host:port";
231+
+ " where server_config is the pipe separated list of host:port:port or host:port:port:type"
232+
+ " and client_config is port or host:port";
232233

233234
public QuorumServer(long sid, String addressStr) throws ConfigException {
234235
this.id = sid;
235236
LearnerType newType = null;
236237
String[] serverClientParts = addressStr.split(";");
237-
String[] serverAddresses = serverClientParts[0].split(",");
238+
String[] serverAddresses = serverClientParts[0].split("\\|");
238239

239240
if (serverClientParts.length == 2) {
240241
String[] clientParts = ConfigUtils.getHostAndPort(serverClientParts[1]);
@@ -346,7 +347,7 @@ public String toString() {
346347
electionAddrList.sort(Comparator.comparing(InetSocketAddress::getHostString));
347348
sw.append(IntStream.range(0, addrList.size()).mapToObj(i -> String.format("%s:%d:%d",
348349
delimitedHostString(addrList.get(i)), addrList.get(i).getPort(), electionAddrList.get(i).getPort()))
349-
.collect(Collectors.joining(",")));
350+
.collect(Collectors.joining("|")));
350351
}
351352

352353
if (type == LearnerType.OBSERVER) {
@@ -833,7 +834,7 @@ public SyncMode getSyncMode() {
833834

834835
public void setLeaderAddressAndId(MultipleAddresses addr, long newId) {
835836
if (addr != null) {
836-
leaderAddress.set(String.join(",", addr.getAllHostStrings()));
837+
leaderAddress.set(String.join("|", addr.getAllHostStrings()));
837838
} else {
838839
leaderAddress.set(null);
839840
}

zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumZooKeeperServer.java

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -176,10 +176,10 @@ public void dumpConf(PrintWriter pwriter) {
176176
pwriter.println(self.getElectionType());
177177
pwriter.print("electionPort=");
178178
pwriter.println(self.getElectionAddress().getAllPorts()
179-
.stream().map(Objects::toString).collect(Collectors.joining(",")));
179+
.stream().map(Objects::toString).collect(Collectors.joining("|")));
180180
pwriter.print("quorumPort=");
181181
pwriter.println(self.getQuorumAddress().getAllPorts()
182-
.stream().map(Objects::toString).collect(Collectors.joining(",")));
182+
.stream().map(Objects::toString).collect(Collectors.joining("|")));
183183
pwriter.print("peerType=");
184184
pwriter.println(self.getLearnerType().ordinal());
185185
pwriter.println("membership: ");

zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/ReadOnlyZooKeeperServer.java

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -173,10 +173,10 @@ public void dumpConf(PrintWriter pwriter) {
173173
pwriter.println(self.getElectionType());
174174
pwriter.print("electionPort=");
175175
pwriter.println(self.getElectionAddress().getAllPorts()
176-
.stream().map(Objects::toString).collect(Collectors.joining(",")));
176+
.stream().map(Objects::toString).collect(Collectors.joining("|")));
177177
pwriter.print("quorumPort=");
178178
pwriter.println(self.getQuorumAddress().getAllPorts()
179-
.stream().map(Objects::toString).collect(Collectors.joining(",")));
179+
.stream().map(Objects::toString).collect(Collectors.joining("|")));
180180
pwriter.print("peerType=");
181181
pwriter.println(self.getLearnerType().ordinal());
182182
}

zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/RemotePeerBean.java

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -49,13 +49,13 @@ public boolean isHidden() {
4949
public String getQuorumAddress() {
5050
return peer.addr.getAllAddresses().stream()
5151
.map(address -> String.format("%s:%d", address.getHostString(), address.getPort()))
52-
.collect(Collectors.joining(","));
52+
.collect(Collectors.joining("|"));
5353
}
5454

5555
public String getElectionAddress() {
5656
return peer.electionAddr.getAllAddresses().stream()
5757
.map(address -> String.format("%s:%d", address.getHostString(), address.getPort()))
58-
.collect(Collectors.joining(","));
58+
.collect(Collectors.joining("|"));
5959
}
6060

6161
public String getClientAddress() {

zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/CnxManagerTest.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -647,7 +647,7 @@ public void testInitialMessage() throws Exception {
647647
// good message, multiple election addresses (ZOOKEEPER-3188)
648648
try {
649649

650-
hostport = "1.1.1.1:9999,2.2.2.2:8888,3.3.3.3:7777";
650+
hostport = "1.1.1.1:9999|2.2.2.2:8888|3.3.3.3:7777";
651651
bos = new ByteArrayOutputStream();
652652
dout = new DataOutputStream(bos);
653653
dout.writeLong(5L); // sid

0 commit comments

Comments
 (0)