From 4d8caf9851f63073145996fd4a535278014d01be Mon Sep 17 00:00:00 2001 From: Mate Szalay-Beko Date: Mon, 23 Mar 2020 16:20:05 +0100 Subject: [PATCH 001/591] ZOOKEEPER-3756: Members slow to rejoin quorum using Kubernetes Whenever we close the current master ZooKeeper server, a new leader election is triggered. During the new election, a connection will be established between all the servers, by calling the synchronized 'connectOne' method in QuorumCnxManager. The method will open the socket and send a single small initial message to the other server, usually very quickly. If the destination host is unreachable, it should fail immediately. However, when we use Kubernetes, then the destination host is always reachable as it points to Kubernetes services. If the actual container / pod is not available then the 'socket.connect' method will timeout (by default after 5 sec) instead of failing immediately with NoRouteToHostException. As the 'connectOne' method is synchronized, this timeout will block the creation of other connections, so a single unreachable host can cause timeout in the leader election protocol. One workaround is to decrease the socket connection timeout with the '-Dzookeeper.cnxTimeout' stystem property, but the proper fix would be to make the connection initiation fully asynchronous, as using very low timeout can have its own side effect. Fortunately most of the initial message sending is already made async: the SASL authentication can take more time, so the second (authentication + initial message sending) part of the initiation protocol is already called in a separate thread, when Quorum SASL authentication is enabled. In the following patch I made the whole connection initiation async, by always using the async executor (not only when Quorum SASL is enabled) and also moving the socket.connect call into the async thread. I also created a unit test to verify my fix. I added a static socket factory that can be changed by the tests using a packet private setter method. My test failed (and produced the same error logs as we see in the original Jira ticket) before I applied my changes and a time-outed as no leader election succeeded after 15 seconds. After the changes the test runs very quickly, in 1-2 seconds. Note: due to the multiAddress changes, we will need different PRs to the branch 3.5 and to the 3.6+ branches. I will submit the other PR once this got reviewed. Author: Mate Szalay-Beko Author: Mate Szalay-Beko Reviewers: Enrico Olivelli , Norbert Kalmar Closes #1289 from symat/ZOOKEEPER-3756-master --- .../server/quorum/QuorumCnxManager.java | 170 ++++++++---------- ...CnxManagerSocketConnectionTimeoutTest.java | 112 ++++++++++++ 2 files changed, 188 insertions(+), 94 deletions(-) create mode 100644 zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/QuorumCnxManagerSocketConnectionTimeoutTest.java diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumCnxManager.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumCnxManager.java index 66f6883470a..08384920125 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumCnxManager.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumCnxManager.java @@ -27,7 +27,6 @@ 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; @@ -58,6 +57,7 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; +import java.util.function.Supplier; import java.util.stream.Collectors; import javax.net.ssl.SSLSocket; import org.apache.zookeeper.common.NetUtils; @@ -186,6 +186,17 @@ public class QuorumCnxManager { */ private final boolean tcpKeepAlive = Boolean.getBoolean("zookeeper.tcpKeepAlive"); + + /* + * Socket factory, allowing the injection of custom socket implementations for testing + */ + static final Supplier DEFAULT_SOCKET_FACTORY = () -> new Socket(); + private static Supplier SOCKET_FACTORY = DEFAULT_SOCKET_FACTORY; + static void setSocketFactory(Supplier factory) { + SOCKET_FACTORY = factory; + } + + public static class Message { Message(ByteBuffer buffer, long sid) { @@ -316,41 +327,30 @@ public QuorumCnxManager(QuorumPeer self, final long mySid, Map(), daemonThFactory); + final ThreadFactory daemonThFactory = runnable -> new Thread(group, runnable, + String.format("QuorumConnectionThread-[myid=%d]-%d", mySid, threadIndex.getAndIncrement())); + + this.connectionExecutor = new ThreadPoolExecutor(3, quorumCnxnThreadsSize, 60, TimeUnit.SECONDS, + new SynchronousQueue<>(), daemonThFactory); this.connectionExecutor.allowCoreThreadTimeOut(true); } @@ -359,20 +359,49 @@ public Thread newThread(Runnable r) { * * @param sid */ - public void testInitiateConnection(long sid) throws Exception { + public void testInitiateConnection(long sid) { 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); + initiateConnection(self.getVotingView().get(sid).electionAddr, sid); } /** + * First we create the socket, perform SSL handshake and authentication if needed. + * Then we perform the initiation protocol. * 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) { + public void initiateConnection(final MultipleAddresses electionAddr, final Long sid) { + Socket sock = null; + try { + LOG.debug("Opening channel to server {}", sid); + if (self.isSslQuorum()) { + sock = self.getX509Util().createSSLSocket(); + } else { + sock = SOCKET_FACTORY.get(); + } + setSockOpts(sock); + sock.connect(electionAddr.getReachableOrOne(), 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()); + } catch (X509Exception e) { + LOG.warn("Cannot open secure channel to {} at election address {}", sid, electionAddr, e); + closeSocket(sock); + return; + } catch (UnresolvedAddressException | IOException e) { + LOG.warn("Cannot open channel to {} at election address {}", sid, electionAddr, e); + closeSocket(sock); + return; + } + try { startConnection(sock, sid); } catch (IOException e) { @@ -389,16 +418,15 @@ public void initiateConnection(final Socket sock, final Long sid) { * Server will initiate the connection request to its peer server * asynchronously via separate connection thread. */ - public void initiateConnectionAsync(final Socket sock, final Long sid) { + public boolean initiateConnectionAsync(final MultipleAddresses electionAddr, 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; + return true; } try { - connectionExecutor.execute(new QuorumConnectionReqThread(sock, sid)); + connectionExecutor.execute(new QuorumConnectionReqThread(electionAddr, sid)); connectionThreadCnt.incrementAndGet(); } catch (Throwable e) { // Imp: Safer side catching all type of exceptions and remove 'sid' @@ -406,27 +434,27 @@ public void initiateConnectionAsync(final Socket sock, final Long sid) { // connection requests from this 'sid' in case of errors. inprogressConnections.remove(sid); LOG.error("Exception while submitting quorum connection request", e); - closeSocket(sock); + return false; } + return true; } /** * Thread to send connection request to peer server. */ private class QuorumConnectionReqThread extends ZooKeeperThread { - - final Socket sock; + final MultipleAddresses electionAddr; final Long sid; - QuorumConnectionReqThread(final Socket sock, final Long sid) { + QuorumConnectionReqThread(final MultipleAddresses electionAddr, final Long sid) { super("QuorumConnectionReqThread-" + sid); - this.sock = sock; + this.electionAddr = electionAddr; this.sid = sid; } @Override public void run() { try { - initiateConnection(sock, sid); + initiateConnection(electionAddr, sid); } finally { inprogressConnections.remove(sid); } @@ -679,6 +707,7 @@ public void toSend(Long sid, ByteBuffer b) { /** * Try to establish a connection to server with id sid using its electionAddr. + * The function will return quickly and the connection will be established asynchronously. * * VisibleForTesting. * @@ -697,62 +726,15 @@ synchronized boolean connectOne(long sid, MultipleAddresses electionAddr) { 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.getReachableOrOne(), 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; - } + // we are doing connection initiation always asynchronously, since it is possible that + // the socket connection timeouts or the SSL handshake takes too long and don't want + // to keep the rest of the connections to wait + return initiateConnectionAsync(electionAddr, sid); } /** * Try to establish a connection to server with id sid. + * The function will return quickly and the connection will be established asynchronously. * * @param sid server id */ diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/QuorumCnxManagerSocketConnectionTimeoutTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/QuorumCnxManagerSocketConnectionTimeoutTest.java new file mode 100644 index 00000000000..ab1c077d9bd --- /dev/null +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/QuorumCnxManagerSocketConnectionTimeoutTest.java @@ -0,0 +1,112 @@ +/** + * 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.junit.Assert.assertTrue; +import java.io.IOException; +import java.net.InetSocketAddress; +import java.net.Socket; +import java.net.SocketAddress; +import java.net.SocketTimeoutException; +import org.apache.zookeeper.ZKTestCase; +import org.apache.zookeeper.test.ClientBase; +import org.apache.zookeeper.test.QuorumUtil; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class QuorumCnxManagerSocketConnectionTimeoutTest extends ZKTestCase { + + private static final Logger LOG = LoggerFactory.getLogger(QuorumCnxManagerSocketConnectionTimeoutTest.class); + private QuorumUtil qu; + + @Before + public void setUp() throws Exception { + // starting a 3 node ensemble without observers + qu = new QuorumUtil(1, 2); + qu.startAll(); + } + + /** + * Testing an error case reported in ZOOKEEPER-3756: + * + * When a new leader election happens after a ZooKeeper server restarted, in Kubernetes + * the rest of the servers can not initiate connection to the restarted one. But they + * get SocketTimeoutException instead of immediate IOException. The Leader Election was + * time-outing quicker than the socket.connect call, so we ended up with cycles of broken + * leader elections. + * + * The fix was to make the connection initiation asynchronous, so one 'broken' connection + * doesn't make the whole leader election to be blocked, even in case of SocketTimeoutException. + * + * @throws Exception + */ + @Test + public void testSocketConnectionTimeoutDuringConnectingToElectionAddress() throws Exception { + + int leaderId = qu.getLeaderServer(); + + // use a custom socket factory that will cause timeout instead of connecting to the + // leader election port of the current leader + final InetSocketAddress leaderElectionAddress = + qu.getLeaderQuorumPeer().getElectionAddress().getOne(); + QuorumCnxManager.setSocketFactory(() -> new SocketStub(leaderElectionAddress)); + + qu.shutdown(leaderId); + + assertTrue("Timeout during waiting for current leader to go down", + ClientBase.waitForServerDown("127.0.0.1:" + qu.getPeer(leaderId).clientPort, + ClientBase.CONNECTION_TIMEOUT)); + + String errorMessage = "No new leader was elected"; + waitFor(errorMessage, () -> qu.leaderExists() && qu.getLeaderServer() != leaderId, 15); + } + + final class SocketStub extends Socket { + + private final InetSocketAddress addressToTimeout; + + SocketStub(InetSocketAddress addressToTimeout) { + this.addressToTimeout = addressToTimeout; + } + + @Override + public void connect(SocketAddress endpoint, int timeout) throws IOException { + if (addressToTimeout.equals(endpoint)) { + try { + Thread.sleep(timeout); + } catch (InterruptedException e) { + LOG.warn("interrupted SocketStub.connect", e); + } + throw new SocketTimeoutException("timeout reached in SocketStub.connect()"); + } + + super.connect(endpoint, timeout); + } + } + + @After + public void tearDown() throws Exception { + qu.shutdownAll(); + QuorumCnxManager.setSocketFactory(QuorumCnxManager.DEFAULT_SOCKET_FACTORY); + } + +} \ No newline at end of file From e87bad6774e7269ef21a156aff9dad089ef54794 Mon Sep 17 00:00:00 2001 From: Jie Huang Date: Tue, 24 Mar 2020 19:38:46 -0700 Subject: [PATCH 002/591] =?UTF-8?q?ZOOKEEPER-3683:=20Discard=20requests=20?= =?UTF-8?q?that=20are=20delayed=20longer=20than=20a=20confi=E2=80=A6?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit …gured threshold Author: Jie Huang Author: Ivailo Nedelchev Reviewers: Michael Han , Allan Lyu , Damien Diederen Closes #1211 from jhuan31/ZOOKEEPER-3683 --- .../zookeeper-client-c/include/zookeeper.h | 3 +- .../zookeeper-client-c/src/zookeeper.c | 2 + .../main/resources/markdown/zookeeperAdmin.md | 9 + .../org/apache/zookeeper/KeeperException.java | 18 +- .../server/FinalRequestProcessor.java | 39 ++- .../server/PrepRequestProcessor.java | 18 +- .../org/apache/zookeeper/server/Request.java | 18 ++ .../zookeeper/server/RequestThrottler.java | 16 +- .../zookeeper/server/ServerMetrics.java | 7 + .../server/SyncRequestProcessor.java | 7 +- .../zookeeper/server/ZooKeeperServer.java | 12 + .../zookeeper/server/ZooKeeperServerBean.java | 10 + .../server/ZooKeeperServerMXBean.java | 3 + .../server/quorum/CommitProcessor.java | 17 +- .../quorum/FollowerRequestProcessor.java | 3 + .../zookeeper/server/quorum/Leader.java | 6 + .../zookeeper/server/quorum/Learner.java | 4 + .../quorum/ObserverRequestProcessor.java | 4 + .../server/quorum/QuorumZooKeeperServer.java | 4 + .../org/apache/zookeeper/test/QuorumBase.java | 60 +++++ .../zookeeper/test/ThrottledOpHelper.java | 224 ++++++++++++++++++ .../test/ThrottledOpObserverTest.java | 73 ++++++ .../zookeeper/test/ThrottledOpQuorumTest.java | 98 ++++++++ .../test/ThrottledOpStandaloneTest.java | 64 +++++ 24 files changed, 695 insertions(+), 24 deletions(-) create mode 100644 zookeeper-server/src/test/java/org/apache/zookeeper/test/ThrottledOpHelper.java create mode 100644 zookeeper-server/src/test/java/org/apache/zookeeper/test/ThrottledOpObserverTest.java create mode 100644 zookeeper-server/src/test/java/org/apache/zookeeper/test/ThrottledOpQuorumTest.java create mode 100644 zookeeper-server/src/test/java/org/apache/zookeeper/test/ThrottledOpStandaloneTest.java diff --git a/zookeeper-client/zookeeper-client-c/include/zookeeper.h b/zookeeper-client/zookeeper-client-c/include/zookeeper.h index 243fac28430..d33a44695a7 100644 --- a/zookeeper-client/zookeeper-client-c/include/zookeeper.h +++ b/zookeeper-client/zookeeper-client-c/include/zookeeper.h @@ -140,7 +140,8 @@ enum ZOO_ERRORS { ZEPHEMERALONLOCALSESSION = -120, /*!< Attempt to create ephemeral node on a local session */ ZNOWATCHER = -121, /*!< The watcher couldn't be found */ ZRECONFIGDISABLED = -123, /*!< Attempts to perform a reconfiguration operation when reconfiguration feature is disabled */ - ZSESSIONCLOSEDREQUIRESASLAUTH = -124 /*!< The session has been closed by server because server requires client to do SASL authentication, but client is not configured with SASL authentication or configuted with SASL but failed (i.e. wrong credential used.). */ + ZSESSIONCLOSEDREQUIRESASLAUTH = -124, /*!< The session has been closed by server because server requires client to do SASL authentication, but client is not configured with SASL authentication or configuted with SASL but failed (i.e. wrong credential used.). */ + ZTHROTTLEDOP = -127 /*!< Operation was throttled and not executed at all. please, retry! */ }; #ifdef __cplusplus diff --git a/zookeeper-client/zookeeper-client-c/src/zookeeper.c b/zookeeper-client/zookeeper-client-c/src/zookeeper.c index 0c143e7bbe2..2b8053e5a40 100644 --- a/zookeeper-client/zookeeper-client-c/src/zookeeper.c +++ b/zookeeper-client/zookeeper-client-c/src/zookeeper.c @@ -4902,6 +4902,8 @@ const char* zerror(int c) return "the watcher couldn't be found"; case ZRECONFIGDISABLED: return "attempts to perform a reconfiguration operation when reconfiguration feature is disable"; + case ZTHROTTLEDOP: + return "Operation was throttled due to high load"; } if (c > 0) { return strerror(c); diff --git a/zookeeper-docs/src/main/resources/markdown/zookeeperAdmin.md b/zookeeper-docs/src/main/resources/markdown/zookeeperAdmin.md index 8d460462b1d..ca1a3f71f57 100644 --- a/zookeeper-docs/src/main/resources/markdown/zookeeperAdmin.md +++ b/zookeeper-docs/src/main/resources/markdown/zookeeperAdmin.md @@ -1133,6 +1133,15 @@ property, when available, is noted below. effect due to TLS handshake timeout when there are too many in-flight TLS handshakes. Set it to something like 250 is good enough to avoid herd effect. +* *throttledOpWaitTime* + (Java system property: **zookeeper.throttled_op_wait_time**) + The time in the RequestThrottler queue longer than which a request will be marked as throttled. + A throttled requests will not be processed other than being fed down the pipeline of the server it belongs to + to preserve the order of all requests. + The FinalProcessor will issue an error response (new error code: ZTHROTTLEDOP) for these undigested requests. + The intent is for the clients not to retry them immediately. + When set to 0, no requests will be throttled. The default is 0. + #### Cluster Options diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/KeeperException.java b/zookeeper-server/src/main/java/org/apache/zookeeper/KeeperException.java index 5cff6f3180c..c8b33b7fa7b 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/KeeperException.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/KeeperException.java @@ -148,6 +148,8 @@ public static KeeperException create(Code code) { return new SessionClosedRequireAuthException(); case REQUESTTIMEOUT: return new RequestTimeoutException(); + case THROTTLEDOP: + return new ThrottledOpException(); case OK: default: throw new IllegalArgumentException("Invalid exception code"); @@ -404,7 +406,11 @@ public enum Code implements CodeDeprecated { /** The session has been closed by server because server requires client to do SASL authentication, * but client is not configured with SASL authentication or configuted with SASL but failed * (i.e. wrong credential used.). */ - SESSIONCLOSEDREQUIRESASLAUTH(-124); + SESSIONCLOSEDREQUIRESASLAUTH(-124), + /** Operation was throttled and not executed at all. This error code indicates that zookeeper server + * is under heavy load and can't process incoming requests at full speed; please retry with back off. + */ + THROTTLEDOP (-127); private static final Map lookup = new HashMap(); @@ -495,6 +501,8 @@ static String getCodeMessage(Code code) { return "Reconfig is disabled"; case SESSIONCLOSEDREQUIRESASLAUTH: return "Session closed because client failed to authenticate"; + case THROTTLEDOP: + return "Op throttled due to high load"; default: return "Unknown error " + code; } @@ -940,4 +948,12 @@ public RequestTimeoutException() { } + /** + * @see Code#THROTTLEDOP + */ + public static class ThrottledOpException extends KeeperException { + public ThrottledOpException() { + super(Code.THROTTLEDOP); + } + } } diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/FinalRequestProcessor.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/FinalRequestProcessor.java index 9ffde55c10c..6ad61feddf0 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/FinalRequestProcessor.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/FinalRequestProcessor.java @@ -106,18 +106,7 @@ public FinalRequestProcessor(ZooKeeperServer zks) { this.requestPathMetricsCollector = zks.getRequestPathMetricsCollector(); } - public void processRequest(Request request) { - LOG.debug("Processing request:: {}", request); - - // request.addRQRec(">final"); - long traceMask = ZooTrace.CLIENT_REQUEST_TRACE_MASK; - if (request.type == OpCode.ping) { - traceMask = ZooTrace.SERVER_PING_TRACE_MASK; - } - if (LOG.isTraceEnabled()) { - ZooTrace.logRequest(LOG, traceMask, 'E', request, ""); - } - + private ProcessTxnResult applyRequest(Request request) { ProcessTxnResult rc = zks.processTxn(request); // ZOOKEEPER-558: @@ -131,7 +120,7 @@ public void processRequest(Request request) { // we are just playing diffs from the leader. if (closeSession(zks.serverCnxnFactory, request.sessionId) || closeSession(zks.secureServerCnxnFactory, request.sessionId)) { - return; + return rc; } } @@ -150,6 +139,24 @@ public void processRequest(Request request) { } } + return rc; + } + + public void processRequest(Request request) { + LOG.debug("Processing request:: {}", request); + + // request.addRQRec(">final"); + long traceMask = ZooTrace.CLIENT_REQUEST_TRACE_MASK; + if (request.type == OpCode.ping) { + traceMask = ZooTrace.SERVER_PING_TRACE_MASK; + } + if (LOG.isTraceEnabled()) { + ZooTrace.logRequest(LOG, traceMask, 'E', request, ""); + } + ProcessTxnResult rc = null; + if (!request.isThrottled()) { + rc = applyRequest(request); + } if (request.cnxn == null) { return; } @@ -195,7 +202,13 @@ public void processRequest(Request request) { if (request.isStale()) { ServerMetrics.getMetrics().STALE_REPLIES.add(1); } + + if (request.isThrottled()) { + throw KeeperException.create(Code.THROTTLEDOP); + } + AuditHelper.addAuditLog(request, rc); + switch (request.type) { case OpCode.ping: { lastOp = "PING"; diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/PrepRequestProcessor.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/PrepRequestProcessor.java index 74720ed4a1b..00ed555c759 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/PrepRequestProcessor.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/PrepRequestProcessor.java @@ -765,6 +765,21 @@ protected void pRequest(Request request) throws RequestProcessorException { request.setHdr(null); request.setTxn(null); + if (!request.isThrottled()) { + pRequestHelper(request); + } + + request.zxid = zks.getZxid(); + ServerMetrics.getMetrics().PREP_PROCESS_TIME.add(Time.currentElapsedTime() - request.prepStartTime); + nextProcessor.processRequest(request); + } + + /** + * This method is a helper to pRequest method + * + * @param request + */ + private void pRequestHelper(Request request) throws RequestProcessorException { try { switch (request.type) { case OpCode.createContainer: @@ -939,9 +954,6 @@ protected void pRequest(Request request) throws RequestProcessorException { request.setTxn(new ErrorTxn(Code.MARSHALLINGERROR.intValue())); } } - request.zxid = zks.getZxid(); - ServerMetrics.getMetrics().PREP_PROCESS_TIME.add(Time.currentElapsedTime() - request.prepStartTime); - nextProcessor.processRequest(request); } private static List removeDuplicates(final List acls) { diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/Request.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/Request.java index d0fb7da4d2e..43a68ac7aa2 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/Request.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/Request.java @@ -100,6 +100,8 @@ public Request(long sessionId, int xid, int type, TxnHeader hdr, Record txn, lon public long syncQueueStartTime; + public long requestThrottleQueueTime; + private Object owner; private KeeperException e; @@ -108,6 +110,22 @@ public Request(long sessionId, int xid, int type, TxnHeader hdr, Record txn, lon private TxnDigest txnDigest; + private boolean isThrottledFlag = false; + + public boolean isThrottled() { + return isThrottledFlag; + } + + public void setIsThrottled(boolean val) { + isThrottledFlag = val; + } + + public boolean isThrottlable() { + return this.type != OpCode.ping + && this.type != OpCode.closeSession + && this.type != OpCode.createSession; + } + /** * If this is a create or close request for a local-only session. */ diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/RequestThrottler.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/RequestThrottler.java index 16a1c6f1517..e9cdc5ec10b 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/RequestThrottler.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/RequestThrottler.java @@ -20,6 +20,7 @@ import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; import java.util.concurrent.LinkedBlockingQueue; +import org.apache.zookeeper.common.Time; import org.apache.zookeeper.util.ServiceUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -97,6 +98,13 @@ public class RequestThrottler extends ZooKeeperCriticalThread { */ private static volatile boolean dropStaleRequests = Boolean.parseBoolean(System.getProperty("zookeeper.request_throttle_drop_stale", "true")); + protected boolean shouldThrottleOp(Request request, long elapsedTime) { + return request.isThrottlable() + && zks.getThrottledOpWaitTime() > 0 + && elapsedTime > zks.getThrottledOpWaitTime(); + } + + public RequestThrottler(ZooKeeperServer zks) { super("RequestThrottler", zks.getZooKeeperServerListener()); this.zks = zks; @@ -171,6 +179,12 @@ public void run() { if (request.isStale()) { ServerMetrics.getMetrics().STALE_REQUESTS.add(1); } + final long elapsedTime = Time.currentElapsedTime() - request.requestThrottleQueueTime; + ServerMetrics.getMetrics().REQUEST_THROTTLE_QUEUE_TIME.add(elapsedTime); + if (shouldThrottleOp(request, elapsedTime)) { + request.setIsThrottled(true); + ServerMetrics.getMetrics().THROTTLED_OPS.add(1); + } zks.submitRequestNow(request); } } @@ -230,6 +244,7 @@ public void submitRequest(Request request) { LOG.debug("Shutdown in progress. Request cannot be processed"); dropRequest(request); } else { + request.requestThrottleQueueTime = Time.currentElapsedTime(); submittedRequests.add(request); } } @@ -238,7 +253,6 @@ public int getInflight() { return submittedRequests.size(); } - @SuppressFBWarnings("DM_EXIT") public void shutdown() { // Try to shutdown gracefully LOG.info("Shutting down"); diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/ServerMetrics.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/ServerMetrics.java index cbdb2347d6f..95214209c01 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/ServerMetrics.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/ServerMetrics.java @@ -154,6 +154,8 @@ private ServerMetrics(MetricsProvider metricsProvider) { READS_ISSUED_IN_COMMIT_PROC = metricsContext.getSummary("read_commit_proc_issued", DetailLevel.BASIC); WRITES_ISSUED_IN_COMMIT_PROC = metricsContext.getSummary("write_commit_proc_issued", DetailLevel.BASIC); + THROTTLED_OPS = metricsContext.getCounter("throttled_ops"); + /** * Time spent by a read request in the commit processor. */ @@ -223,6 +225,7 @@ private ServerMetrics(MetricsProvider metricsProvider) { STALE_REQUESTS = metricsContext.getCounter("stale_requests"); STALE_REQUESTS_DROPPED = metricsContext.getCounter("stale_requests_dropped"); STALE_REPLIES = metricsContext.getCounter("stale_replies"); + REQUEST_THROTTLE_QUEUE_TIME = metricsContext.getSummary("request_throttle_queue_time_ms", DetailLevel.ADVANCED); REQUEST_THROTTLE_WAIT_COUNT = metricsContext.getCounter("request_throttle_wait_count"); LARGE_REQUESTS_REJECTED = metricsContext.getCounter("large_requests_rejected"); @@ -381,6 +384,9 @@ private ServerMetrics(MetricsProvider metricsProvider) { public final Summary READS_ISSUED_IN_COMMIT_PROC; public final Summary WRITES_ISSUED_IN_COMMIT_PROC; + // Request op throttling related + public final Counter THROTTLED_OPS; + /** * Time spent by a read request in the commit processor. */ @@ -435,6 +441,7 @@ private ServerMetrics(MetricsProvider metricsProvider) { public final Counter STALE_REQUESTS; public final Counter STALE_REQUESTS_DROPPED; public final Counter STALE_REPLIES; + public final Summary REQUEST_THROTTLE_QUEUE_TIME; public final Counter REQUEST_THROTTLE_WAIT_COUNT; public final Counter LARGE_REQUESTS_REJECTED; diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/SyncRequestProcessor.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/SyncRequestProcessor.java index 66e85b14a2d..4df319f86a4 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/SyncRequestProcessor.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/SyncRequestProcessor.java @@ -178,7 +178,7 @@ public void run() { ServerMetrics.getMetrics().SYNC_PROCESSOR_QUEUE_TIME.add(startProcessTime - si.syncQueueStartTime); // track the number of records written to the log - if (zks.getZKDatabase().append(si)) { + if (!si.isThrottled() && zks.getZKDatabase().append(si)) { if (shouldSnapshot()) { resetSnapshotStats(); // roll the log @@ -202,9 +202,8 @@ public void run() { } } else if (toFlush.isEmpty()) { // optimization for read heavy workloads - // iff this is a read, and there are no pending - // flushes (writes), then just pass this to the next - // processor + // iff this is a read or a throttled request(which doesn't need to be written to the disk), + // and there are no pending flushes (writes), then just pass this to the next processor if (nextProcessor != null) { nextProcessor.processRequest(si); if (nextProcessor instanceof Flushable) { diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/ZooKeeperServer.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/ZooKeeperServer.java index 1a2d9a7e06a..8205f741499 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/ZooKeeperServer.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/ZooKeeperServer.java @@ -155,6 +155,9 @@ public static void setCloseSessionTxnEnabled(boolean enabled) { public static final int DEFAULT_TICK_TIME = 3000; protected int tickTime = DEFAULT_TICK_TIME; + public static final int DEFAULT_THROTTLED_OP_WAIT_TIME = 0; // disabled + protected static volatile int throttledOpWaitTime = + Integer.getInteger("zookeeper.throttled_op_wait_time", DEFAULT_THROTTLED_OP_WAIT_TIME); /** value of -1 indicates unset, use default */ protected int minSessionTimeout = -1; /** value of -1 indicates unset, use default */ @@ -1237,6 +1240,15 @@ public void setTickTime(int tickTime) { this.tickTime = tickTime; } + public static int getThrottledOpWaitTime() { + return throttledOpWaitTime; + } + + public static void setThrottledOpWaitTime(int time) { + LOG.info("throttledOpWaitTime set to {}", time); + throttledOpWaitTime = time; + } + public int getMinSessionTimeout() { return minSessionTimeout; } diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/ZooKeeperServerBean.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/ZooKeeperServerBean.java index bd9b6432440..17dd48b7505 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/ZooKeeperServerBean.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/ZooKeeperServerBean.java @@ -315,6 +315,16 @@ public void setFlushDelay(long delay) { // Request throttling settings /////////////////////////////////////////////////////////////////////////// + public int getThrottledOpWaitTime() { + return zks.getThrottledOpWaitTime(); + } + + public void setThrottledOpWaitTime(int val) { + zks.setThrottledOpWaitTime(val); + } + + /////////////////////////////////////////////////////////////////////////// + public int getRequestThrottleLimit() { return RequestThrottler.getMaxRequests(); } diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/ZooKeeperServerMXBean.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/ZooKeeperServerMXBean.java index 71a9d987e04..851fc564a09 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/ZooKeeperServerMXBean.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/ZooKeeperServerMXBean.java @@ -136,6 +136,9 @@ public interface ZooKeeperServerMXBean { boolean getRequestThrottleDropStale(); void setRequestThrottleDropStale(boolean drop); + int getThrottledOpWaitTime(); + void setThrottledOpWaitTime(int val); + boolean getRequestStaleLatencyCheck(); void setRequestStaleLatencyCheck(boolean check); diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/CommitProcessor.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/CommitProcessor.java index 01f9f0d2738..86dce2b7f85 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/CommitProcessor.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/CommitProcessor.java @@ -29,12 +29,14 @@ import java.util.concurrent.atomic.AtomicInteger; import org.apache.zookeeper.ZooDefs.OpCode; import org.apache.zookeeper.common.Time; +import org.apache.zookeeper.server.ExitCode; import org.apache.zookeeper.server.Request; import org.apache.zookeeper.server.RequestProcessor; import org.apache.zookeeper.server.ServerMetrics; import org.apache.zookeeper.server.WorkerService; import org.apache.zookeeper.server.ZooKeeperCriticalThread; import org.apache.zookeeper.server.ZooKeeperServerListener; +import org.apache.zookeeper.util.ServiceUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -165,6 +167,9 @@ private boolean isProcessingRequest() { } protected boolean needCommit(Request request) { + if (request.isThrottled()) { + return false; + } switch (request.type) { case OpCode.create: case OpCode.create2: @@ -306,6 +311,11 @@ public void run() { // Process committed head request = committedRequests.peek(); + if (request.isThrottled()) { + LOG.error("Throttled request in committed pool: {}. Exiting.", request); + ServiceUtils.requestSystemExit(ExitCode.UNEXPECTED_ERROR.getValue()); + } + /* * Check if this is a local write request is pending, * if so, update it with the committed info. If the commit matches @@ -349,6 +359,10 @@ public void run() { topPending.zxid = request.zxid; topPending.commitRecvTime = request.commitRecvTime; request = topPending; + if (request.isThrottled()) { + LOG.error("Throttled request in committed & pending pool: {}. Exiting.", request); + ServiceUtils.requestSystemExit(ExitCode.UNEXPECTED_ERROR.getValue()); + } // Only decrement if we take a request off the queue. numWriteQueuedRequests.decrementAndGet(); queuedWriteRequests.poll(); @@ -452,7 +466,8 @@ public void start() { */ private void sendToNextProcessor(Request request) { numRequestsProcessing.incrementAndGet(); - workerPool.schedule(new CommitWorkRequest(request), request.sessionId); + CommitWorkRequest workRequest = new CommitWorkRequest(request); + workerPool.schedule(workRequest, request.sessionId); } private void processWrite(Request request) throws RequestProcessorException { diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/FollowerRequestProcessor.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/FollowerRequestProcessor.java index 88144de3602..db51aee495d 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/FollowerRequestProcessor.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/FollowerRequestProcessor.java @@ -73,6 +73,9 @@ public void run() { // the request to the leader so that we are ready to receive // the response nextProcessor.processRequest(request); + if (request.isThrottled()) { + continue; + } // We now ship the request to the leader. As with all // other quorum operations, sync also follows this code diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Leader.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Leader.java index 5371bea91d9..0eb3722a47c 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Leader.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Leader.java @@ -56,6 +56,7 @@ import org.apache.zookeeper.ZooDefs.OpCode; import org.apache.zookeeper.common.Time; import org.apache.zookeeper.jmx.MBeanRegistry; +import org.apache.zookeeper.server.ExitCode; import org.apache.zookeeper.server.FinalRequestProcessor; import org.apache.zookeeper.server.Request; import org.apache.zookeeper.server.RequestProcessor; @@ -68,6 +69,7 @@ import org.apache.zookeeper.server.quorum.flexible.QuorumVerifier; import org.apache.zookeeper.server.util.SerializeUtils; import org.apache.zookeeper.server.util.ZxidUtils; +import org.apache.zookeeper.util.ServiceUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -1218,6 +1220,10 @@ public XidRolloverException(String message) { * @return the proposal that is queued to send to all the members */ public Proposal propose(Request request) throws XidRolloverException { + if (request.isThrottled()) { + LOG.error("Throttled request send as proposal: {}. Exiting.", request); + ServiceUtils.requestSystemExit(ExitCode.UNEXPECTED_ERROR.getValue()); + } /** * Address the rollover issue. All lower 32bits set indicate a new leader * election. Force a re-election instead. See ZOOKEEPER-1277 diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Learner.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Learner.java index 8a0cac17ad6..da5f1132ada 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Learner.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Learner.java @@ -234,6 +234,10 @@ void readPacket(QuorumPacket pp) throws IOException { * @throws IOException */ void request(Request request) throws IOException { + if (request.isThrottled()) { + LOG.error("Throttled request sent to leader: {}. Exiting", request); + ServiceUtils.requestSystemExit(ExitCode.UNEXPECTED_ERROR.getValue()); + } ByteArrayOutputStream baos = new ByteArrayOutputStream(); DataOutputStream oa = new DataOutputStream(baos); oa.writeLong(request.sessionId); diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/ObserverRequestProcessor.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/ObserverRequestProcessor.java index 46662032951..0075ce404a8 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/ObserverRequestProcessor.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/ObserverRequestProcessor.java @@ -82,6 +82,10 @@ public void run() { // the response nextProcessor.processRequest(request); + if (request.isThrottled()) { + continue; + } + // We now ship the request to the leader. As with all // other quorum operations, sync also follows this code // path, but different from others, we need to keep track diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumZooKeeperServer.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumZooKeeperServer.java index c6cd93b4d18..6fdea82751e 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumZooKeeperServer.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumZooKeeperServer.java @@ -60,6 +60,10 @@ protected void startSessionTracker() { } public Request checkUpgradeSession(Request request) throws IOException, KeeperException { + if (request.isThrottled()) { + return null; + } + // If this is a request for a local session and it is to // create an ephemeral node, then upgrade the session and return // a new session request for the leader. diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/test/QuorumBase.java b/zookeeper-server/src/test/java/org/apache/zookeeper/test/QuorumBase.java index b91aa19e915..638275a5415 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/test/QuorumBase.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/test/QuorumBase.java @@ -247,6 +247,66 @@ public int getLeaderIndex() { return -1; } + public int getLeaderClientPort() { + if (s1.getPeerState() == ServerState.LEADING) { + return portClient1; + } else if (s2.getPeerState() == ServerState.LEADING) { + return portClient2; + } else if (s3.getPeerState() == ServerState.LEADING) { + return portClient3; + } else if (s4.getPeerState() == ServerState.LEADING) { + return portClient4; + } else if (s5.getPeerState() == ServerState.LEADING) { + return portClient5; + } + return -1; + } + + public QuorumPeer getLeaderQuorumPeer() { + if (s1.getPeerState() == ServerState.LEADING) { + return s1; + } else if (s2.getPeerState() == ServerState.LEADING) { + return s2; + } else if (s3.getPeerState() == ServerState.LEADING) { + return s3; + } else if (s4.getPeerState() == ServerState.LEADING) { + return s4; + } else if (s5.getPeerState() == ServerState.LEADING) { + return s5; + } + return null; + } + + public QuorumPeer getFirstObserver() { + if (s1.getLearnerType() == LearnerType.OBSERVER) { + return s1; + } else if (s2.getLearnerType() == LearnerType.OBSERVER) { + return s2; + } else if (s3.getLearnerType() == LearnerType.OBSERVER) { + return s3; + } else if (s4.getLearnerType() == LearnerType.OBSERVER) { + return s4; + } else if (s5.getLearnerType() == LearnerType.OBSERVER) { + return s5; + } + return null; + } + + public int getFirstObserverClientPort() { + if (s1.getLearnerType() == LearnerType.OBSERVER) { + return portClient1; + } else if (s2.getLearnerType() == LearnerType.OBSERVER) { + return portClient2; + } else if (s3.getLearnerType() == LearnerType.OBSERVER) { + return portClient3; + } else if (s4.getLearnerType() == LearnerType.OBSERVER) { + return portClient4; + } else if (s5.getLearnerType() == LearnerType.OBSERVER) { + return portClient5; + } + return -1; + } + public String getPeersMatching(ServerState state) { StringBuilder hosts = new StringBuilder(); for (QuorumPeer p : getPeerList()) { diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/test/ThrottledOpHelper.java b/zookeeper-server/src/test/java/org/apache/zookeeper/test/ThrottledOpHelper.java new file mode 100644 index 00000000000..1d9e502257b --- /dev/null +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/test/ThrottledOpHelper.java @@ -0,0 +1,224 @@ +/** + * 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.test; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import mockit.Mock; +import mockit.MockUp; +import org.apache.zookeeper.CreateMode; +import org.apache.zookeeper.KeeperException; +import org.apache.zookeeper.ZooDefs; +import org.apache.zookeeper.ZooDefs.Ids; +import org.apache.zookeeper.ZooKeeper; +import org.apache.zookeeper.data.ACL; +import org.apache.zookeeper.data.Stat; +import org.apache.zookeeper.server.Request; +import org.apache.zookeeper.server.RequestThrottler; +import org.apache.zookeeper.server.ZooKeeperServer; +import org.junit.Assert; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class ThrottledOpHelper { + protected static final Logger LOG = LoggerFactory.getLogger(ThrottledOpHelper.class); + + public static final class RequestThrottleMock extends MockUp { + public static void throttleEveryNthOp(int n) { + everyNthOp = n; + opCounter = 0; + } + private static int everyNthOp = 0; + private static int opCounter = 0; + + @Mock + private boolean shouldThrottleOp(Request request, long elapsedTime) { + if (everyNthOp > 0 && request.isThrottlable() && (++opCounter % everyNthOp == 0)) { + opCounter %= everyNthOp; + return true; + } + return false; + } + } + + public static void applyMockUps() { + new RequestThrottleMock(); + } + + public void testThrottledOp(ZooKeeper zk, ZooKeeperServer zs) throws IOException, InterruptedException, KeeperException { + final int N = 5; // must be greater than 3 + final int COUNT = 100; + RequestThrottleMock.throttleEveryNthOp(N); + LOG.info("Before create /ivailo nodes"); + int opCount = 0; + for (int i = 0; i < COUNT; i++) { + String nodeName = "/ivailo" + i; + if (opCount % N == N - 1) { + try { + zk.create(nodeName, "".getBytes(), Ids.OPEN_ACL_UNSAFE, + (i % 2 == 0) ? CreateMode.PERSISTENT : CreateMode.EPHEMERAL); + Assert.fail("Should have gotten ThrottledOp exception"); + } catch (KeeperException.ThrottledOpException e) { + // anticipated outcome + Stat stat = zk.exists(nodeName, null); + Assert.assertNull(stat); + zk.create(nodeName, "".getBytes(), Ids.OPEN_ACL_UNSAFE, + (i % 2 == 0) ? CreateMode.PERSISTENT : CreateMode.EPHEMERAL); + } catch (KeeperException e) { + Assert.fail("Should have gotten ThrottledOp exception"); + } + opCount += 3; // three ops issued + } else { + zk.create(nodeName, "".getBytes(), Ids.OPEN_ACL_UNSAFE, + (i % 2 == 0) ? CreateMode.PERSISTENT : CreateMode.EPHEMERAL); + opCount++; // one op issued + } + if (opCount % N == N - 1) { + try { + zk.setData(nodeName, nodeName.getBytes(), -1); + Assert.fail("Should have gotten ThrottledOp exception"); + } catch (KeeperException.ThrottledOpException e) { + // anticipated outcome & retry + zk.setData(nodeName, nodeName.getBytes(), -1); + } catch (KeeperException e) { + Assert.fail("Should have gotten ThrottledOp exception"); + } + opCount += 2; // two ops issued, one for retry + } else { + zk.setData(nodeName, nodeName.getBytes(), -1); + opCount++; // one op issued + } + } + LOG.info("Before delete /ivailo nodes"); + for (int i = 0; i < COUNT; i++) { + String nodeName = "/ivailo" + i; + if (opCount % N == N - 1) { + try { + zk.exists(nodeName, null); + Assert.fail("Should have gotten ThrottledOp exception"); + } catch (KeeperException.ThrottledOpException e) { + // anticipated outcome & retry + Stat stat = zk.exists(nodeName, null); + Assert.assertNotNull(stat); + opCount += 2; // two ops issued, one is retry + } catch (KeeperException e) { + Assert.fail("Should have gotten ThrottledOp exception"); + } + } else { + Stat stat = zk.exists(nodeName, null); + Assert.assertNotNull(stat); + opCount++; + } + if (opCount % N == N - 1) { + try { + zk.getData(nodeName, null, null); + Assert.fail("Should have gotten ThrottledOp exception"); + } catch (KeeperException.ThrottledOpException e) { + // anticipated outcome & retry + byte[] data = zk.getData(nodeName, null, null); + Assert.assertEquals(nodeName, new String(data)); + opCount += 2; // two ops issued, one is retry + } catch (KeeperException e) { + Assert.fail("Should have gotten ThrottledOp exception"); + } + } else { + byte[] data = zk.getData(nodeName, null, null); + Assert.assertEquals(nodeName, new String(data)); + opCount++; + } + if (opCount % N == N - 1) { + try { + // version 0 should not trigger BadVersion exception + zk.delete(nodeName, 0); + Assert.fail("Should have gotten ThrottledOp exception"); + } catch (KeeperException.ThrottledOpException e) { + // anticipated outcome & retry + zk.delete(nodeName, -1); + } catch (KeeperException e) { + Assert.fail("Should have gotten ThrottledOp exception"); + } + opCount += 2; // two ops issues, one for retry + } else { + zk.delete(nodeName, -1); + opCount++; // one op only issued + } + if (opCount % N == N - 1) { + try { + zk.exists(nodeName, null); + Assert.fail("Should have gotten ThrottledOp exception"); + } catch (KeeperException.ThrottledOpException e) { + // anticipated outcome & retry + Stat stat = zk.exists(nodeName, null); + Assert.assertNull(stat); + opCount += 2; // two ops issued, one is retry + } catch (KeeperException e) { + Assert.fail("Should have gotten ThrottledOp exception"); + } + } else { + Stat stat = zk.exists(nodeName, null); + Assert.assertNull(stat); + opCount++; + } + } + LOG.info("After delete /ivailo"); + zk.close(); + } + + public void testThrottledAcl(ZooKeeper zk, ZooKeeperServer zs) throws Exception { + RequestThrottleMock.throttleEveryNthOp(0); + + final ArrayList ACL_PERMS = + new ArrayList() { { + add(new ACL(ZooDefs.Perms.READ, ZooDefs.Ids.ANYONE_ID_UNSAFE)); + add(new ACL(ZooDefs.Perms.WRITE, ZooDefs.Ids.ANYONE_ID_UNSAFE)); + add(new ACL(ZooDefs.Perms.ALL, ZooDefs.Ids.AUTH_IDS)); + }}; + String path = "/path1"; + zk.create(path, path.getBytes(), Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); + zk.addAuthInfo("digest", "pat:test".getBytes()); + List defaultAcls = zk.getACL(path, null); + Assert.assertEquals(1, defaultAcls.size()); + + RequestThrottleMock.throttleEveryNthOp(2); + + path = "/path2"; + zk.create(path, path.getBytes(), Ids.OPEN_ACL_UNSAFE, + CreateMode.PERSISTENT); + try { + zk.setACL(path, ACL_PERMS, -1); + Assert.fail("Should have gotten ThrottledOp exception"); + } catch (KeeperException.ThrottledOpException e) { + // expected + } catch (KeeperException e) { + Assert.fail("Should have gotten ThrottledOp exception"); + } + List acls = zk.getACL(path, null); + Assert.assertEquals(1, acls.size()); + + RequestThrottleMock.throttleEveryNthOp(0); + + path = "/path3"; + zk.create(path, path.getBytes(), Ids.OPEN_ACL_UNSAFE, + CreateMode.PERSISTENT); + zk.setACL(path, ACL_PERMS, -1); + acls = zk.getACL(path, null); + Assert.assertEquals(3, acls.size()); + } +} diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/test/ThrottledOpObserverTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/test/ThrottledOpObserverTest.java new file mode 100644 index 00000000000..dfd19642f08 --- /dev/null +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/test/ThrottledOpObserverTest.java @@ -0,0 +1,73 @@ +/** + * 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.test; + +import java.io.IOException; +import org.apache.zookeeper.KeeperException; +import org.apache.zookeeper.ZooKeeper; +import org.apache.zookeeper.server.ZooKeeperServer; +import org.junit.BeforeClass; +import org.junit.Test; + +public class ThrottledOpObserverTest extends QuorumBase { + @BeforeClass + public static void applyMockUps() { + ThrottledOpHelper.applyMockUps(); + } + + @Override + public void setUp() throws Exception { + super.setUp(true /* withObservers */); + } + + @Override + public void tearDown() throws Exception { + super.tearDown(); + } + + @Test + public void testThrottledOpObserver() throws IOException, InterruptedException, KeeperException { + ZooKeeper zk = null; + try { + zk = createClient("localhost:" + getFirstObserverClientPort()); + ZooKeeperServer zs = getFirstObserver().getActiveServer(); + ThrottledOpHelper test = new ThrottledOpHelper(); + test.testThrottledOp(zk, zs); + } finally { + if (zk != null) { + zk.close(); + } + } + } + + @Test + public void testThrottledAclObserver() throws Exception { + ZooKeeper zk = null; + try { + zk = createClient("localhost:" + getFirstObserverClientPort()); + ZooKeeperServer zs = getFirstObserver().getActiveServer(); + ThrottledOpHelper test = new ThrottledOpHelper(); + test.testThrottledAcl(zk, zs); + } finally { + if (zk != null) { + zk.close(); + } + } + } +} diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/test/ThrottledOpQuorumTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/test/ThrottledOpQuorumTest.java new file mode 100644 index 00000000000..27613657da4 --- /dev/null +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/test/ThrottledOpQuorumTest.java @@ -0,0 +1,98 @@ +/** + * 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.test; + +import java.io.IOException; +import org.apache.zookeeper.KeeperException; +import org.apache.zookeeper.ZooKeeper; +import org.apache.zookeeper.server.ZooKeeperServer; +import org.apache.zookeeper.server.quorum.QuorumPeer; +import org.junit.BeforeClass; +import org.junit.Test; + +public class ThrottledOpQuorumTest extends QuorumBase { + @BeforeClass + public static void applyMockUps() { + ThrottledOpHelper.applyMockUps(); + } + + @Test + public void testThrottledOpLeader() throws IOException, InterruptedException, KeeperException { + ZooKeeper zk = null; + try { + zk = createClient("localhost:" + getLeaderClientPort()); + ZooKeeperServer zs = getLeaderQuorumPeer().getActiveServer(); + ThrottledOpHelper test = new ThrottledOpHelper(); + test.testThrottledOp(zk, zs); + } finally { + if (zk != null) { + zk.close(); + } + } + } + + @Test + public void testThrottledAclLeader() throws Exception { + ZooKeeper zk = null; + try { + zk = createClient("localhost:" + getLeaderClientPort()); + ZooKeeperServer zs = getLeaderQuorumPeer().getActiveServer(); + ThrottledOpHelper test = new ThrottledOpHelper(); + test.testThrottledAcl(zk, zs); + } finally { + if (zk != null) { + zk.close(); + } + } + } + + @Test + public void testThrottledOpFollower() throws IOException, InterruptedException, KeeperException { + ZooKeeper zk = null; + try { + int clientPort = (getLeaderClientPort() == portClient1) ? portClient2 : portClient1; + zk = createClient("localhost:" + clientPort); + QuorumPeer qp = (getLeaderClientPort() == portClient1) ? s2 : s1; + ZooKeeperServer zs = qp.getActiveServer(); + ThrottledOpHelper test = new ThrottledOpHelper(); + test.testThrottledOp(zk, zs); + } finally { + if (zk != null) { + zk.close(); + } + } + } + + @Test + public void testThrottledAclFollower() throws Exception { + ZooKeeper zk = null; + try { + int clientPort = (getLeaderClientPort() == portClient1) ? portClient2 : portClient1; + zk = createClient("localhost:" + clientPort); + QuorumPeer qp = (getLeaderClientPort() == portClient1) ? s2 : s1; + ZooKeeperServer zs = qp.getActiveServer(); + ThrottledOpHelper test = new ThrottledOpHelper(); + test.testThrottledAcl(zk, zs); + } finally { + if (zk != null) { + zk.close(); + } + } + } +} diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/test/ThrottledOpStandaloneTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/test/ThrottledOpStandaloneTest.java new file mode 100644 index 00000000000..27ac6a6939c --- /dev/null +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/test/ThrottledOpStandaloneTest.java @@ -0,0 +1,64 @@ +/** + * 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.test; + +import java.io.IOException; +import org.apache.zookeeper.KeeperException; +import org.apache.zookeeper.ZooKeeper; +import org.apache.zookeeper.server.ZooKeeperServer; +import org.junit.BeforeClass; +import org.junit.Test; + +public class ThrottledOpStandaloneTest extends ClientBase { + + @BeforeClass + public static void applyMockUps() { + ThrottledOpHelper.applyMockUps(); + } + + @Test + public void testThrottledOp() throws IOException, InterruptedException, KeeperException { + ZooKeeper zk = null; + try { + zk = createClient(hostPort); + ZooKeeperServer zs = serverFactory.getZooKeeperServer(); + ThrottledOpHelper test = new ThrottledOpHelper(); + test.testThrottledOp(zk, zs); + } finally { + if (zk != null) { + zk.close(); + } + } + } + + @Test + public void testThrottledAcl() throws Exception { + ZooKeeper zk = null; + try { + zk = createClient(hostPort); + ZooKeeperServer zs = serverFactory.getZooKeeperServer(); + ThrottledOpHelper test = new ThrottledOpHelper(); + test.testThrottledAcl(zk, zs); + } finally { + if (zk != null) { + zk.close(); + } + } + } +} From 64863119bf92893f3c72350fd37005083b587327 Mon Sep 17 00:00:00 2001 From: Damien Diederen Date: Wed, 25 Mar 2020 12:08:54 +0100 Subject: [PATCH 003/591] ZOOKEEPER-3654: Incorrect *_CFLAGS handling in Automake MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit The `Makefile.am` distributed with the C client defines some per-target `*_CFLAGS` and `*_CXXFLAGS` variables. These however, do not reference `AM_CFLAGS` (resp. AM_CXXFLAGS`, which means that some options (notably `-Wall`) are missing when building subsets of the code. Dixit the [Automake docs](https://www.gnu.org/software/automake/manual/html_node/Program-and-Library-Variables.html): > In compilations with per-target flags, the ordinary ‘AM_’ form of > the flags variable is _not_ automatically included in the > compilation (however, the user form of the variable _is_ included). > So for instance, if you want the hypothetical ‘maude’ compilations > to also use the value of ‘AM_CFLAGS’, you would need to write: > > maude_CFLAGS = ... your flags ... $(AM_CFLAGS) Restoring the flags, however, causes compilation failures (in the library) and a slew of new warnings (in the tests) which had not been noticed because of the missing options. This series of patches (all "tagged" ZOOKEEPER-3654) fix these warnings and errors before re-enabling `-Wall` and friends for all targets. Author: Damien Diederen Reviewers: Enrico Olivelli , Andor Molnar Closes #1186 from ztzg/ZOOKEEPER-3654-incorrect-automake-flags --- .../zookeeper-client-c/Makefile.am | 10 ++-- .../zookeeper-client-c/src/load_gen.c | 2 - .../zookeeper-client-c/src/mt_adaptor.c | 3 +- .../zookeeper-client-c/src/zookeeper.c | 2 + .../zookeeper-client-c/tests/TestClient.cc | 25 ++++---- .../zookeeper-client-c/tests/TestMulti.cc | 3 - .../tests/TestReconfigServer.cc | 58 +++++++++---------- 7 files changed, 53 insertions(+), 50 deletions(-) diff --git a/zookeeper-client/zookeeper-client-c/Makefile.am b/zookeeper-client/zookeeper-client-c/Makefile.am index 9c794a5ef09..e42a3539b8a 100644 --- a/zookeeper-client/zookeeper-client-c/Makefile.am +++ b/zookeeper-client/zookeeper-client-c/Makefile.am @@ -60,7 +60,7 @@ libzookeeper_st_la_LDFLAGS = $(LIB_LDFLAGS) -export-symbols-regex $(EXPORT_SYMBO if WANT_SYNCAPI noinst_LTLIBRARIES += libzkmt.la libzkmt_la_SOURCES =$(COMMON_SRC) src/mt_adaptor.c -libzkmt_la_CFLAGS = -DTHREADED +libzkmt_la_CFLAGS = $(AM_CFLAGS) -DTHREADED libzkmt_la_LIBADD = -lm $(CLOCK_GETTIME_LIBS) lib_LTLIBRARIES += libzookeeper_mt.la @@ -80,11 +80,11 @@ bin_PROGRAMS += cli_mt load_gen cli_mt_SOURCES = src/cli.c cli_mt_LDADD = libzookeeper_mt.la $(SASL_LIB_LDFLAGS) -cli_mt_CFLAGS = -DTHREADED +cli_mt_CFLAGS = $(AM_CFLAGS) -DTHREADED load_gen_SOURCES = src/load_gen.c load_gen_LDADD = libzookeeper_mt.la -load_gen_CFLAGS = -DTHREADED +load_gen_CFLAGS = $(AM_CFLAGS) -DTHREADED endif @@ -134,14 +134,14 @@ TESTS_ENVIRONMENT = ZKROOT=${srcdir}/../.. \ CLASSPATH=$$CLASSPATH:$$CLOVER_HOME/lib/clover*.jar nodist_zktest_st_SOURCES = $(TEST_SOURCES) zktest_st_LDADD = libzkst.la libhashtable.la $(CPPUNIT_LIBS) $(OPENSSL_LIB_LDFLAGS) $(SASL_LIB_LDFLAGS) -ldl -zktest_st_CXXFLAGS = -DUSE_STATIC_LIB $(CPPUNIT_CFLAGS) $(USEIPV6) $(SOLARIS_CPPFLAGS) +zktest_st_CXXFLAGS = $(AM_CXXFLAGS) -DUSE_STATIC_LIB $(CPPUNIT_CFLAGS) $(SOLARIS_CPPFLAGS) zktest_st_LDFLAGS = -shared $(SYMBOL_WRAPPERS) $(SOLARIS_LIB_LDFLAGS) if WANT_SYNCAPI check_PROGRAMS += zktest-mt nodist_zktest_mt_SOURCES = $(TEST_SOURCES) tests/PthreadMocks.cc zktest_mt_LDADD = libzkmt.la libhashtable.la -lpthread $(CPPUNIT_LIBS) $(OPENSSL_LIB_LDFLAGS) $(SASL_LIB_LDFLAGS) -ldl - zktest_mt_CXXFLAGS = -DUSE_STATIC_LIB -DTHREADED $(CPPUNIT_CFLAGS) $(USEIPV6) + zktest_mt_CXXFLAGS = $(AM_CXXFLAGS) -DUSE_STATIC_LIB -DTHREADED $(CPPUNIT_CFLAGS) $(USEIPV6) if SOLARIS SHELL_SYMBOL_WRAPPERS_MT = cat ${srcdir}/tests/wrappers-mt.opt SYMBOL_WRAPPERS_MT=$(SYMBOL_WRAPPERS) $(SHELL_SYMBOL_WRAPPERS_MT:sh) diff --git a/zookeeper-client/zookeeper-client-c/src/load_gen.c b/zookeeper-client/zookeeper-client-c/src/load_gen.c index 886fe1b3e3c..f25edcbe928 100644 --- a/zookeeper-client/zookeeper-client-c/src/load_gen.c +++ b/zookeeper-client/zookeeper-client-c/src/load_gen.c @@ -27,8 +27,6 @@ static zhandle_t *zh; -static int shutdownThisThing=0; - // ***************************************************************************** // static pthread_cond_t cond=PTHREAD_COND_INITIALIZER; diff --git a/zookeeper-client/zookeeper-client-c/src/mt_adaptor.c b/zookeeper-client/zookeeper-client-c/src/mt_adaptor.c index 38cced425ba..73b64c391ff 100644 --- a/zookeeper-client/zookeeper-client-c/src/mt_adaptor.c +++ b/zookeeper-client/zookeeper-client-c/src/mt_adaptor.c @@ -369,13 +369,14 @@ void *do_io(void *v) fds[0].fd=adaptor_threads->self_pipe[0]; fds[0].events=POLLIN; while(!zh->close_requested) { - zh->io_count++; struct timeval tv; int fd; int interest; int timeout; int maxfd=1; + zh->io_count++; + zookeeper_interest(zh, &fd, &interest, &tv); if (fd != -1) { fds[1].fd=fd; diff --git a/zookeeper-client/zookeeper-client-c/src/zookeeper.c b/zookeeper-client/zookeeper-client-c/src/zookeeper.c index 2b8053e5a40..a273225ff69 100644 --- a/zookeeper-client/zookeeper-client-c/src/zookeeper.c +++ b/zookeeper-client/zookeeper-client-c/src/zookeeper.c @@ -342,6 +342,7 @@ static int is_sasl_auth_in_progress(zhandle_t* zh) #endif /* HAVE_CYRUS_SASL_H */ } +#ifndef THREADED /* * abort due to the use of a sync api in a singlethreaded environment */ @@ -350,6 +351,7 @@ static void abort_singlethreaded(zhandle_t *zh) LOG_ERROR(LOGCALLBACK(zh), "Sync completion used without threads"); abort(); } +#endif /* THREADED */ static ssize_t zookeeper_send(zsock_t *fd, const void* buf, size_t len) { diff --git a/zookeeper-client/zookeeper-client-c/tests/TestClient.cc b/zookeeper-client/zookeeper-client-c/tests/TestClient.cc index a7d055fbe47..2a6e9925470 100644 --- a/zookeeper-client/zookeeper-client-c/tests/TestClient.cc +++ b/zookeeper-client/zookeeper-client-c/tests/TestClient.cc @@ -325,9 +325,10 @@ class Zookeeper_simpleSystem : public CPPUNIT_NS::TestFixture /** have a callback in the default watcher **/ static void default_zoo_watcher(zhandle_t *zzh, int type, int state, const char *path, void *context){ - int zrc = 0; + int zrc; struct String_vector str_vec = {0, NULL}; zrc = zoo_wget_children(zzh, "/mytest", default_zoo_watcher, NULL, &str_vec); + CPPUNIT_ASSERT(zrc == ZOK || zrc == ZCLOSING); } /** ZOOKEEPER-1057 This checks that the client connects to the second server when the first is not reachable **/ @@ -353,24 +354,28 @@ class Zookeeper_simpleSystem : public CPPUNIT_NS::TestFixture /** this checks for a deadlock in calling zookeeper_close and calls from a default watcher that might get triggered just when zookeeper_close() is in progress **/ void testHangingClient() { - int zrc = 0; + int zrc; char buff[10] = "testall"; char path[512]; - watchctx_t *ctx; + watchctx_t *ctx = NULL; struct String_vector str_vec = {0, NULL}; zhandle_t *zh = zookeeper_init(hostPorts, NULL, 10000, 0, ctx, 0); sleep(1); zrc = zoo_create(zh, "/mytest", buff, 10, &ZOO_OPEN_ACL_UNSAFE, 0, path, 512); + CPPUNIT_ASSERT_EQUAL((int)ZOK, zrc); zrc = zoo_wget_children(zh, "/mytest", default_zoo_watcher, NULL, &str_vec); + CPPUNIT_ASSERT_EQUAL((int)ZOK, zrc); zrc = zoo_create(zh, "/mytest/test1", buff, 10, &ZOO_OPEN_ACL_UNSAFE, 0, path, 512); + CPPUNIT_ASSERT_EQUAL((int)ZOK, zrc); zrc = zoo_wget_children(zh, "/mytest", default_zoo_watcher, NULL, &str_vec); + CPPUNIT_ASSERT_EQUAL((int)ZOK, zrc); zrc = zoo_delete(zh, "/mytest/test1", -1); + CPPUNIT_ASSERT_EQUAL((int)ZOK, zrc); zookeeper_close(zh); } void testBadDescriptor() { - int zrc = 0; - watchctx_t *ctx; + watchctx_t *ctx = NULL; zhandle_t *zh = zookeeper_init(hostPorts, NULL, 10000, 0, ctx, 0); sleep(1); zh->io_count = 0; @@ -1039,8 +1044,8 @@ class Zookeeper_simpleSystem : public CPPUNIT_NS::TestFixture CPPUNIT_ASSERT_EQUAL(zoo_get_log_callback(zk), &logMessageHandler); // Log 10 messages and ensure all go to callback - int expected = 10; - for (int i = 0; i < expected; i++) + const std::size_t expected = 10; + for (std::size_t i = 0; i < expected; i++) { LOG_INFO(LOGCALLBACK(zk), "%s #%d", __FUNCTION__, i); } @@ -1057,12 +1062,12 @@ class Zookeeper_simpleSystem : public CPPUNIT_NS::TestFixture // All the connection messages should have gone to the callback -- don't // want this to be a maintenance issue so we're not asserting exact count - int numBefore = logMessages.size(); + std::size_t numBefore = logMessages.size(); CPPUNIT_ASSERT(numBefore != 0); // Log 10 messages and ensure all go to callback - int expected = 10; - for (int i = 0; i < expected; i++) + const std::size_t expected = 10; + for (std::size_t i = 0; i < expected; i++) { LOG_INFO(LOGCALLBACK(zk), "%s #%d", __FUNCTION__, i); } diff --git a/zookeeper-client/zookeeper-client-c/tests/TestMulti.cc b/zookeeper-client/zookeeper-client-c/tests/TestMulti.cc index 226e4708202..6dfeb96e0c0 100644 --- a/zookeeper-client/zookeeper-client-c/tests/TestMulti.cc +++ b/zookeeper-client/zookeeper-client-c/tests/TestMulti.cc @@ -486,8 +486,6 @@ class Zookeeper_multi : public CPPUNIT_NS::TestFixture watchctx_t ctx; zhandle_t *zk = createClient(&ctx); int sz = 512; - char buf[sz]; - int blen; char p1[sz]; p1[0] = '\0'; struct Stat stat; @@ -573,7 +571,6 @@ class Zookeeper_multi : public CPPUNIT_NS::TestFixture int sz = 512; char p1[sz]; p1[0] = '\0'; - struct Stat s1; rc = zoo_create(zk, "/multi0", "", 0, &ZOO_OPEN_ACL_UNSAFE, 0, p1, sz); CPPUNIT_ASSERT_EQUAL((int)ZOK, rc); diff --git a/zookeeper-client/zookeeper-client-c/tests/TestReconfigServer.cc b/zookeeper-client/zookeeper-client-c/tests/TestReconfigServer.cc index c15774e9e30..54810b60c29 100644 --- a/zookeeper-client/zookeeper-client-c/tests/TestReconfigServer.cc +++ b/zookeeper-client/zookeeper-client-c/tests/TestReconfigServer.cc @@ -49,12 +49,12 @@ class TestReconfigServer : public CPPUNIT_NS::TestFixture { static const uint32_t NUM_SERVERS; FILE* logfile_; std::vector cluster_; - int32_t getLeader(); - std::vector getFollowers(); + std::size_t getLeader(); + std::vector getFollowers(); void parseConfig(char* buf, int len, std::vector& servers, std::string& version); bool waitForConnected(zhandle_t* zh, uint32_t timeout_sec); - zhandle_t* connectFollowers(std::vector &followers); + zhandle_t* connectFollowers(std::vector &followers); }; const uint32_t TestReconfigServer::NUM_SERVERS = 3; @@ -84,15 +84,15 @@ setUp() { void TestReconfigServer:: tearDown() { - for (int i = 0; i < cluster_.size(); i++) { + for (std::size_t i = 0; i < cluster_.size(); i++) { delete cluster_[i]; } cluster_.clear(); } -int32_t TestReconfigServer:: +std::size_t TestReconfigServer:: getLeader() { - for (int32_t i = 0; i < cluster_.size(); i++) { + for (std::size_t i = 0; i < cluster_.size(); i++) { if (cluster_[i]->isLeader()) { return i; } @@ -100,10 +100,10 @@ getLeader() { return -1; } -std::vector TestReconfigServer:: +std::vector TestReconfigServer:: getFollowers() { - std::vector followers; - for (int32_t i = 0; i < cluster_.size(); i++) { + std::vector followers; + for (std::size_t i = 0; i < cluster_.size(); i++) { if (cluster_[i]->isFollower()) { followers.push_back(i); } @@ -154,7 +154,7 @@ testRemoveFollower() { char buf[len]; // get config from leader. - int32_t leader = getLeader(); + std::size_t leader = getLeader(); CPPUNIT_ASSERT(leader >= 0); std::string host = cluster_[leader]->getHostPort(); zhandle_t* zk = zookeeper_init(host.c_str(), NULL, 10000, NULL, NULL, 0); @@ -167,13 +167,13 @@ testRemoveFollower() { // of the first NEWLEADER message, used as the initial version CPPUNIT_ASSERT_EQUAL(std::string("100000000"), version); CPPUNIT_ASSERT_EQUAL(NUM_SERVERS, (uint32_t)(servers.size())); - for (int i = 0; i < cluster_.size(); i++) { + for (std::size_t i = 0; i < cluster_.size(); i++) { CPPUNIT_ASSERT(std::find(servers.begin(), servers.end(), cluster_[i]->getServerString()) != servers.end()); } // remove a follower. - std::vector followers = getFollowers(); + std::vector followers = getFollowers(); len = 1024; CPPUNIT_ASSERT_EQUAL(NUM_SERVERS - 1, (uint32_t)(followers.size())); @@ -185,7 +185,7 @@ testRemoveFollower() { parseConfig(buf, len, servers, version); CPPUNIT_ASSERT_EQUAL(std::string("100000002"), version); CPPUNIT_ASSERT_EQUAL(NUM_SERVERS - 1, (uint32_t)(servers.size())); - for (int i = 0; i < cluster_.size(); i++) { + for (std::size_t i = 0; i < cluster_.size(); i++) { if (i == followers[0]) { continue; } @@ -201,7 +201,7 @@ testRemoveFollower() { CPPUNIT_ASSERT_EQUAL((int)ZOK, rc); parseConfig(buf, len, servers, version); CPPUNIT_ASSERT_EQUAL(NUM_SERVERS, (uint32_t)(servers.size())); - for (int i = 0; i < cluster_.size(); i++) { + for (std::size_t i = 0; i < cluster_.size(); i++) { CPPUNIT_ASSERT(std::find(servers.begin(), servers.end(), cluster_[i]->getServerString()) != servers.end()); } @@ -222,7 +222,7 @@ testNonIncremental() { char buf[len]; // get config from leader. - int32_t leader = getLeader(); + std::size_t leader = getLeader(); CPPUNIT_ASSERT(leader >= 0); std::string host = cluster_[leader]->getHostPort(); zhandle_t* zk = zookeeper_init(host.c_str(), NULL, 10000, NULL, NULL, 0); @@ -236,18 +236,18 @@ testNonIncremental() { // of the first NEWLEADER message, used as the initial version CPPUNIT_ASSERT_EQUAL(std::string("100000000"), version); CPPUNIT_ASSERT_EQUAL(NUM_SERVERS, (uint32_t)(servers.size())); - for (int i = 0; i < cluster_.size(); i++) { + for (std::size_t i = 0; i < cluster_.size(); i++) { CPPUNIT_ASSERT(std::find(servers.begin(), servers.end(), cluster_[i]->getServerString()) != servers.end()); } // remove a follower. - std::vector followers = getFollowers(); + std::vector followers = getFollowers(); len = 1024; CPPUNIT_ASSERT_EQUAL(NUM_SERVERS - 1, (uint32_t)(followers.size())); std::stringstream ss; - for (int i = 1; i < followers.size(); i++) { + for (std::size_t i = 1; i < followers.size(); i++) { ss << cluster_[followers[i]]->getServerString() << ","; } ss << cluster_[leader]->getServerString(); @@ -258,7 +258,7 @@ testNonIncremental() { parseConfig(buf, len, servers, version); CPPUNIT_ASSERT_EQUAL(std::string("100000002"), version); CPPUNIT_ASSERT_EQUAL(NUM_SERVERS - 1, (uint32_t)(servers.size())); - for (int i = 0; i < cluster_.size(); i++) { + for (std::size_t i = 0; i < cluster_.size(); i++) { if (i == followers[0]) { continue; } @@ -269,7 +269,7 @@ testNonIncremental() { // add the follower back. len = 1024; ss.str(""); - for (int i = 0; i < cluster_.size(); i++) { + for (std::size_t i = 0; i < cluster_.size(); i++) { ss << cluster_[i]->getServerString() << ","; } rc = zoo_reconfig(zk, NULL, NULL, ss.str().c_str(), -1, buf, &len, @@ -277,7 +277,7 @@ testNonIncremental() { CPPUNIT_ASSERT_EQUAL((int)ZOK, rc); parseConfig(buf, len, servers, version); CPPUNIT_ASSERT_EQUAL(NUM_SERVERS, (uint32_t)(servers.size())); - for (int i = 0; i < cluster_.size(); i++) { + for (std::size_t i = 0; i < cluster_.size(); i++) { CPPUNIT_ASSERT(std::find(servers.begin(), servers.end(), cluster_[i]->getServerString()) != servers.end()); } @@ -285,12 +285,12 @@ testNonIncremental() { } zhandle_t* TestReconfigServer:: -connectFollowers(std::vector &followers) { +connectFollowers(std::vector &followers) { std::stringstream ss; - int32_t leader = getLeader(); + std::size_t leader = getLeader(); CPPUNIT_ASSERT(leader >= 0); CPPUNIT_ASSERT_EQUAL(NUM_SERVERS - 1, (uint32_t)(followers.size())); - for (int i = 0; i < followers.size(); i++) { + for (std::size_t i = 0; i < followers.size(); i++) { ss << cluster_[followers[i]]->getHostPort() << ","; } ss << cluster_[leader]->getHostPort(); @@ -321,7 +321,7 @@ testRemoveConnectedFollower() { // connect to a follower. std::stringstream ss; - std::vector followers = getFollowers(); + std::vector followers = getFollowers(); zhandle_t* zk = connectFollowers(followers); CPPUNIT_ASSERT_EQUAL((int)ZOK, zoo_add_auth(zk, "digest", "super:test", 10, NULL,(void*)ZOK)); @@ -333,7 +333,7 @@ testRemoveConnectedFollower() { CPPUNIT_ASSERT_EQUAL((int)ZOK, zoo_getconfig(zk, 0, buf, &len, &stat)); parseConfig(buf, len, servers, version); CPPUNIT_ASSERT_EQUAL(NUM_SERVERS - 1, (uint32_t)(servers.size())); - for (int i = 0; i < cluster_.size(); i++) { + for (std::size_t i = 0; i < cluster_.size(); i++) { if (i == followers[0]) { continue; } @@ -356,7 +356,7 @@ testReconfigFailureWithoutAuth() { // connect to a follower. std::stringstream ss; - std::vector followers = getFollowers(); + std::vector followers = getFollowers(); zhandle_t* zk = connectFollowers(followers); // remove the follower. @@ -374,7 +374,7 @@ testReconfigFailureWithoutAuth() { CPPUNIT_ASSERT_EQUAL((int)ZOK, zoo_getconfig(zk, 0, buf, &len, &stat)); parseConfig(buf, len, servers, version); CPPUNIT_ASSERT_EQUAL(NUM_SERVERS - 1, (uint32_t)(servers.size())); - for (int i = 0; i < cluster_.size(); i++) { + for (std::size_t i = 0; i < cluster_.size(); i++) { if (i == followers[0]) { continue; } @@ -400,7 +400,7 @@ testReconfigFailureWithoutServerSuperuserPasswordConfigured() { // connect to a follower. std::stringstream ss; - std::vector followers = getFollowers(); + std::vector followers = getFollowers(); zhandle_t* zk = connectFollowers(followers); // remove the follower. From 2581f71dd97cfda2cf6768c3457b81c7a93b9ef7 Mon Sep 17 00:00:00 2001 From: lingjinjiang Date: Thu, 26 Mar 2020 18:41:57 +0100 Subject: [PATCH 004/591] ZOOKEEPER-3760: remove a useless throwing CliException https://issues.apache.org/jira/browse/ZOOKEEPER-3760 When I upgrade zookeeper from 3.4.13 to 3.5.7 in my application, I find the function processCmd in ZooKeeperMain.java throws a CliException which has been caught in the function. So I think it can be removed Author: lingjinjiang Reviewers: Enrico Olivelli , Norbert Kalmar Closes #1286 from lingjinjiang/master --- .../src/main/java/org/apache/zookeeper/ZooKeeperMain.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/ZooKeeperMain.java b/zookeeper-server/src/main/java/org/apache/zookeeper/ZooKeeperMain.java index dad878cf87c..8d8a419401f 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/ZooKeeperMain.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/ZooKeeperMain.java @@ -289,7 +289,7 @@ protected void connectToZK(String newHost) throws InterruptedException, IOExcept zk = new ZooKeeperAdmin(host, Integer.parseInt(cl.getOption("timeout")), new MyWatcher(), readOnly); } - public static void main(String[] args) throws CliException, IOException, InterruptedException { + public static void main(String[] args) throws IOException, InterruptedException { ZooKeeperMain main = new ZooKeeperMain(args); main.run(); } @@ -304,7 +304,7 @@ public ZooKeeperMain(ZooKeeper zk) { this.zk = zk; } - void run() throws CliException, IOException, InterruptedException { + void run() throws IOException, InterruptedException { if (cl.getCommand() == null) { System.out.println("Welcome to ZooKeeper!"); @@ -353,7 +353,7 @@ void run() throws CliException, IOException, InterruptedException { ServiceUtils.requestSystemExit(exitCode); } - public void executeLine(String line) throws CliException, InterruptedException, IOException { + public void executeLine(String line) throws InterruptedException, IOException { if (!line.equals("")) { cl.parseCommand(line); addToHistory(commandCount, line); @@ -362,7 +362,7 @@ public void executeLine(String line) throws CliException, InterruptedException, } } - protected boolean processCmd(MyCommandOptions co) throws CliException, IOException, InterruptedException { + protected boolean processCmd(MyCommandOptions co) throws IOException, InterruptedException { boolean watch = false; try { watch = processZKCmd(co); From 6c7a273bc1c553be34dc7b9e658c6c16b8a023ef Mon Sep 17 00:00:00 2001 From: Brittany Barnes <60981978+blb93@users.noreply.github.com> Date: Thu, 26 Mar 2020 22:12:18 +0100 Subject: [PATCH 005/591] =?UTF-8?q?ZOOKEEPER-3728:=20move=20traceMask=20ca?= =?UTF-8?q?lculation=20logic=20into=20the=20trace=20log=20i=E2=80=A6?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit …n the FinalRequestProcessor#processRequest Author: Brittany Barnes <60981978+blb93@users.noreply.github.com> Author: Brittany Barnes Reviewers: Enrico Olivelli , Justin Ling Mao , Luciano Resende Closes #1271 from blb93/ZOOKEEPER-3728 --- .../apache/zookeeper/server/FinalRequestProcessor.java | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/FinalRequestProcessor.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/FinalRequestProcessor.java index 6ad61feddf0..a487626dee6 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/FinalRequestProcessor.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/FinalRequestProcessor.java @@ -145,12 +145,11 @@ private ProcessTxnResult applyRequest(Request request) { public void processRequest(Request request) { LOG.debug("Processing request:: {}", request); - // request.addRQRec(">final"); - long traceMask = ZooTrace.CLIENT_REQUEST_TRACE_MASK; - if (request.type == OpCode.ping) { - traceMask = ZooTrace.SERVER_PING_TRACE_MASK; - } if (LOG.isTraceEnabled()) { + long traceMask = ZooTrace.CLIENT_REQUEST_TRACE_MASK; + if (request.type == OpCode.ping) { + traceMask = ZooTrace.SERVER_PING_TRACE_MASK; + } ZooTrace.logRequest(LOG, traceMask, 'E', request, ""); } ProcessTxnResult rc = null; From 2bc0a932400e97c5cdb70b4eb1da4b2e0ad3375f Mon Sep 17 00:00:00 2001 From: tison Date: Thu, 26 Mar 2020 22:16:27 +0100 Subject: [PATCH 006/591] ZOOKEEPER-3767: Fix several document invalid syntax Straightforward. The question might be at which level we'd like to maintain jute document. I'm not quite familiar with this section and suspect it is far more outdated. Author: tison Reviewers: Enrico Olivelli , Norbert Kalmar , Christopher Tubbs Closes #1291 from TisonKun/ZOOKEEPER-3767 --- .../main/java/org/apache/jute/package.html | 72 +++++++++---------- .../zookeeper/server/SessionTrackerImpl.java | 13 ++-- 2 files changed, 45 insertions(+), 40 deletions(-) diff --git a/zookeeper-jute/src/main/java/org/apache/jute/package.html b/zookeeper-jute/src/main/java/org/apache/jute/package.html index 64a030a6d7d..f769b74c111 100644 --- a/zookeeper-jute/src/main/java/org/apache/jute/package.html +++ b/zookeeper-jute/src/main/java/org/apache/jute/package.html @@ -16,7 +16,7 @@ limitations under the License. --> - + Hadoop Record I/O @@ -197,8 +197,8 @@

Hadoop DDL Syntax

ptype := ("byte" / "boolean" / "int" | "long" / "float" / "double" "ustring" / "buffer") -ctype := (("vector" "<" type ">") / - ("map" "<" type "," type ">" ) ) / name) +ctype := (("vector" "<" type ">") / + ("map" "<" type "," type ">" ) ) / name) A DDL file describes one or more record types. It begins with zero or @@ -248,7 +248,7 @@

Examples

module outlinks { class OutLinks { ustring baseURL; - vector outLinks; + vector<links.Link> outLinks; }; } @@ -262,7 +262,7 @@

Code Generation

optional language argument (the default is Java) --language or -l. Thus a typical invocation would look like:

-$ rcc -l C++  ...
+$ rcc -l C++ <filename> ...
 
@@ -299,7 +299,7 @@

C++

class IOError : public runtime_error { public: - explicit IOError(const std::string& msg); + explicit IOError(const std::string& msg); }; class IArchive; @@ -307,18 +307,18 @@

C++

class RecordReader { public: - RecordReader(InStream& in, RecFormat fmt); + RecordReader(InStream& in, RecFormat fmt); virtual ~RecordReader(void); - virtual void read(Record& rec); + virtual void read(Record& rec); }; class RecordWriter { public: - RecordWriter(OutStream& out, RecFormat fmt); + RecordWriter(OutStream& out, RecFormat fmt); virtual ~RecordWriter(void); - virtual void write(Record& rec); + virtual void write(Record& rec); }; @@ -330,10 +330,10 @@

C++

virtual bool validate(void) const = 0; virtual void - serialize(OArchive& oa, const std::string& tag) const = 0; + serialize(OArchive& oa, const std::string& tag) const = 0; virtual void - deserialize(IArchive& ia, const std::string& tag) = 0; + deserialize(IArchive& ia, const std::string& tag) = 0; }; } @@ -408,7 +408,7 @@

C++

either the member or a reference to the member. For members that are returned by value, a setter method is also generated. This is true for primitive data members of the types byte, int, long, boolean, float and -double. For example, for a int field called MyField the folowing +double. For example, for a int field called MyField the following code is generated.

@@ -438,11 +438,11 @@ 

C++

... public: - std::string& getMyBuf() { + std::string& getMyBuf() { return mMyBuf; }; - const std::string& getMyBuf() const { + const std::string& getMyBuf() const { return mMyBuf; }; ... @@ -467,7 +467,7 @@

Examples

include "inclrec.jr" module testrec { class R { - vector VF; + vector<float> VF; RI Rec; buffer Buf; }; @@ -504,8 +504,8 @@

Examples

RI(void); virtual ~RI(void); - virtual bool operator==(const RI& peer) const; - virtual bool operator<(const RI& peer) const; + virtual bool operator==(const RI& peer) const; + virtual bool operator<(const RI& peer) const; virtual int32_t getI32(void) const { return mI32; } virtual void setI32(int32_t v) { mI32 = v; } @@ -513,16 +513,16 @@

Examples

virtual double getD(void) const { return mD; } virtual void setD(double v) { mD = v; } - virtual std::string& getS(void) const { return mS; } - virtual const std::string& getS(void) const { return mS; } + virtual std::string& getS(void) const { return mS; } + virtual const std::string& getS(void) const { return mS; } virtual std::string type(void) const; virtual std::string signature(void) const; protected: - virtual void serialize(hadoop::OArchive& a) const; - virtual void deserialize(hadoop::IArchive& a); + virtual void serialize(hadoop::OArchive& a) const; + virtual void deserialize(hadoop::IArchive& a); virtual bool validate(void); }; @@ -547,7 +547,7 @@

Examples

private: - std::vector mVF; + std::vector<float> mVF; inclrec::RI mRec; std::string mBuf; @@ -556,20 +556,20 @@

Examples

R(void); virtual ~R(void); - virtual bool operator==(const R& peer) const; - virtual bool operator<(const R& peer) const; + virtual bool operator==(const R& peer) const; + virtual bool operator<(const R& peer) const; - virtual std::vector& getVF(void) const; - virtual const std::vector& getVF(void) const; + virtual std::vector<float>& getVF(void) const; + virtual const std::vector<float>& getVF(void) const; - virtual std::string& getBuf(void) const ; - virtual const std::string& getBuf(void) const; + virtual std::string& getBuf(void) const ; + virtual const std::string& getBuf(void) const; - virtual inclrec::RI& getRec(void) const; - virtual const inclrec::RI& getRec(void) const; + virtual inclrec::RI& getRec(void) const; + virtual const inclrec::RI& getRec(void) const; - virtual bool serialize(hadoop::OutArchive& a) const; - virtual bool deserialize(hadoop::InArchive& a); + virtual bool serialize(hadoop::OutArchive& a) const; + virtual bool deserialize(hadoop::InArchive& a); virtual std::string type(void) const; virtual std::string signature(void) const; @@ -614,8 +614,8 @@

Mapping Summary

ustring std::string Text buffer std::string java.io.ByteArrayOutputStream class type class type class type -vector std::vector java.util.ArrayList -map std::map java.util.TreeMap +vector<type> std::vector<type> java.util.ArrayList +map<type,type> std::map<type,type> java.util.TreeMap

Data encodings

@@ -646,7 +646,7 @@

Binary Serialization Format

  • byte: Represented by 1 byte, as is.
  • boolean: Represented by 1-byte (0 or 1)
  • int/long: Integers and longs are serialized zero compressed. -Represented as 1-byte if -120 <= value < 128. Otherwise, serialized as a +Represented as 1-byte if -120 <= value < 128. Otherwise, serialized as a sequence of 2-5 bytes for ints, 2-9 bytes for longs. The first byte represents the number of trailing bytes, N, as the negative number (-120-N). For example, the number 1024 (0x400) is represented by the byte sequence 'x86 x04 x00'. diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/SessionTrackerImpl.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/SessionTrackerImpl.java index 488fb662339..cedd0c0c76c 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/SessionTrackerImpl.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/SessionTrackerImpl.java @@ -83,11 +83,16 @@ public String toString() { } /** - * Generates an initial sessionId. High order 1 byte is serverId, next - * 5 bytes are from timestamp, and low order 2 bytes are 0s. - * Use ">>> 8", not ">> 8" to make sure that the high order 1 byte is entirely up to the server Id(@see ZOOKEEPER-1622). + * Generates an initial sessionId. + * + *

    High order 1 byte is serverId, next 5 bytes are from timestamp, and low order 2 bytes are 0s. + * Use {@literal ">>> 8"}, not {@literal ">> 8"} to make sure that the high order 1 byte is entirely + * up to the server Id. + * + *

    See also http://jira.apache.org/jira/browse/ZOOKEEPER-1622 + * * @param id server Id - * @return the Session Id + * @return the session Id */ public static long initializeNextSessionId(long id) { long nextSid; From b4a3238d1d0cebe1c97ed23427bbcc69c8477c8a Mon Sep 17 00:00:00 2001 From: Sankalp Date: Thu, 26 Mar 2020 22:19:08 +0100 Subject: [PATCH 007/591] ZOOKEEPER-3689: zkCli/ZooKeeperMain relies on system properties for TLS config The current zkCli uses system properties to set confidential information like keystore location, password etc. This is not secure as these properties need to be passed on the command line as "-D" arguments. Currently, there is no way to create a ZookeeperAdmin does not have a constructor which takes both canBeReadOnly and ZKClientConfig as parameters. I am introducing a new constructor in ZookeeperAdmin which takes an additional ZKClientConfig parameter. This ZKClientConfig is created by an optional command line argument ``client-configuration``. If no argument is passed, a ZookeeperAdmin object with null client config is created, just like before. Author: Sankalp Reviewers: Enrico Olivelli , Norbert Kalmar , Justin Ling Mao Closes #1285 from sankalpbhatia/ZOOKEEPER-3689 --- .../main/resources/markdown/zookeeperCLI.md | 2 + .../org/apache/zookeeper/ZooKeeperMain.java | 19 +++++++- .../zookeeper/admin/ZooKeeperAdmin.java | 48 +++++++++++++++++++ 3 files changed, 67 insertions(+), 2 deletions(-) diff --git a/zookeeper-docs/src/main/resources/markdown/zookeeperCLI.md b/zookeeper-docs/src/main/resources/markdown/zookeeperCLI.md index 637e6b4596a..c9ebd2b06cd 100644 --- a/zookeeper-docs/src/main/resources/markdown/zookeeperCLI.md +++ b/zookeeper-docs/src/main/resources/markdown/zookeeperCLI.md @@ -24,6 +24,8 @@ Enter into the ZooKeeper-cli bin/zkCli.sh # connect to the remote host with timeout:3s bin/zkCli.sh -timeout 3000 -server remoteIP:2181 +# connect with a custom client configuration properties file +bin/zkCli.sh -client-configuration /path/to/client.properties ``` ## help Showing helps about ZooKeeper commands diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/ZooKeeperMain.java b/zookeeper-server/src/main/java/org/apache/zookeeper/ZooKeeperMain.java index 8d8a419401f..243aca33c39 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/ZooKeeperMain.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/ZooKeeperMain.java @@ -65,6 +65,7 @@ import org.apache.zookeeper.cli.VersionCommand; import org.apache.zookeeper.client.ZKClientConfig; import org.apache.zookeeper.server.ExitCode; +import org.apache.zookeeper.server.quorum.QuorumPeerConfig; import org.apache.zookeeper.util.ServiceUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -130,7 +131,7 @@ public boolean getPrintWatches() { } static void usage() { - System.err.println("ZooKeeper -server host:port cmd args"); + System.err.println("ZooKeeper -server host:port -client-configuration properties-file cmd args"); List cmdList = new ArrayList(commandMap.keySet()); Collections.sort(cmdList); for (String cmd : cmdList) { @@ -205,6 +206,8 @@ public boolean parseOptions(String[] args) { options.put("timeout", it.next()); } else if (opt.equals("-r")) { options.put("readonly", "true"); + } else if (opt.equals("-client-configuration")) { + options.put("client-configuration", it.next()); } } catch (NoSuchElementException e) { System.err.println("Error: no argument found for option " + opt); @@ -286,7 +289,19 @@ protected void connectToZK(String newHost) throws InterruptedException, IOExcept System.setProperty(ZKClientConfig.SECURE_CLIENT, "true"); System.out.println("Secure connection is enabled"); } - zk = new ZooKeeperAdmin(host, Integer.parseInt(cl.getOption("timeout")), new MyWatcher(), readOnly); + + ZKClientConfig clientConfig = null; + + if (cl.getOption("client-configuration") != null) { + try { + clientConfig = new ZKClientConfig(cl.getOption("client-configuration")); + } catch (QuorumPeerConfig.ConfigException e) { + e.printStackTrace(); + ServiceUtils.requestSystemExit(ExitCode.INVALID_INVOCATION.getValue()); + } + } + + zk = new ZooKeeperAdmin(host, Integer.parseInt(cl.getOption("timeout")), new MyWatcher(), readOnly, clientConfig); } public static void main(String[] args) throws IOException, InterruptedException { diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/admin/ZooKeeperAdmin.java b/zookeeper-server/src/main/java/org/apache/zookeeper/admin/ZooKeeperAdmin.java index 05c992899d7..8240526a513 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/admin/ZooKeeperAdmin.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/admin/ZooKeeperAdmin.java @@ -124,6 +124,54 @@ public ZooKeeperAdmin( super(connectString, sessionTimeout, watcher, conf); } + /** + * Create a ZooKeeperAdmin object which is used to perform dynamic reconfiguration + * operations. + * + * @param connectString + * comma separated host:port pairs, each corresponding to a zk + * server. e.g. "127.0.0.1:3000,127.0.0.1:3001,127.0.0.1:3002" If + * the optional chroot suffix is used the example would look + * like: "127.0.0.1:3000,127.0.0.1:3001,127.0.0.1:3002/app/a" + * where the client would be rooted at "/app/a" and all paths + * would be relative to this root - ie getting/setting/etc... + * "/foo/bar" would result in operations being run on + * "/app/a/foo/bar" (from the server perspective). + * @param sessionTimeout + * session timeout in milliseconds + * @param watcher + * a watcher object which will be notified of state changes, may + * also be notified for node events + * @param canBeReadOnly + * whether the created client is allowed to go to + * read-only mode in case of partitioning. Read-only mode + * basically means that if the client can't find any majority + * servers but there's partitioned server it could reach, it + * connects to one in read-only mode, i.e. read requests are + * allowed while write requests are not. It continues seeking for + * majority in the background. + * @param conf + * passing this conf object gives each client the flexibility of + * configuring properties differently compared to other instances + * + * @throws IOException + * in cases of network failure + * @throws IllegalArgumentException + * if an invalid chroot path is specified + * + * @see ZooKeeper#ZooKeeper(String, int, Watcher, boolean, ZKClientConfig) + * + * @since 3.6.1 + */ + public ZooKeeperAdmin( + String connectString, + int sessionTimeout, + Watcher watcher, + boolean canBeReadOnly, + ZKClientConfig conf) throws IOException { + super(connectString, sessionTimeout, watcher, canBeReadOnly, conf); + } + /** * Create a ZooKeeperAdmin object which is used to perform dynamic reconfiguration * operations. From 1ff1b779b70dd794c9e45f35f8ca305229df10e0 Mon Sep 17 00:00:00 2001 From: Sushant Mane Date: Thu, 26 Mar 2020 22:20:51 +0100 Subject: [PATCH 008/591] ZOOKEEPER-3755: Use maven to create fatjar - replace ant build.xml with maven pom.xml for zookeeper-contrib-fatjar module - create maven profile "fatjar" to build a fatjar file. - update readme files Author: Sushant Mane Reviewers: Enrico Olivelli , Benjamin Reed Closes #1284 from sushantmane/master --- pom.xml | 7 + zookeeper-contrib/pom.xml | 9 ++ .../zookeeper-contrib-fatjar/README.md | 21 +++ .../zookeeper-contrib-fatjar/README.txt | 2 - .../zookeeper-contrib-fatjar/build.xml | 74 --------- .../zookeeper-contrib-fatjar/pom.xml | 145 ++++++++++++++++++ .../src/main/resources/mainClasses | 2 - zookeeper-it/README.txt | 9 +- zookeeper-it/pom.xml | 77 ++++++++++ .../zookeeper/server/watch/WatchBench.java | 4 +- .../zookeeper/test/system/BaseSysTest.java | 13 +- .../test/system/InstanceContainer.java | 2 +- 12 files changed, 274 insertions(+), 91 deletions(-) create mode 100644 zookeeper-contrib/zookeeper-contrib-fatjar/README.md delete mode 100644 zookeeper-contrib/zookeeper-contrib-fatjar/README.txt delete mode 100644 zookeeper-contrib/zookeeper-contrib-fatjar/build.xml create mode 100755 zookeeper-contrib/zookeeper-contrib-fatjar/pom.xml create mode 100755 zookeeper-it/pom.xml diff --git a/pom.xml b/pom.xml index 041b41571f8..6fad123737f 100755 --- a/pom.xml +++ b/pom.xml @@ -261,6 +261,13 @@ zookeeper-contrib + + fatjar + + zookeeper-it + zookeeper-contrib/zookeeper-contrib-fatjar + + java-build diff --git a/zookeeper-contrib/pom.xml b/zookeeper-contrib/pom.xml index edc3cb351fd..228a520ed4b 100755 --- a/zookeeper-contrib/pom.xml +++ b/zookeeper-contrib/pom.xml @@ -40,6 +40,15 @@ zookeeper-contrib-zooinspector + + + fatjar + + zookeeper-contrib-fatjar + + + + diff --git a/zookeeper-contrib/zookeeper-contrib-fatjar/README.md b/zookeeper-contrib/zookeeper-contrib-fatjar/README.md new file mode 100644 index 00000000000..baf2f95f91f --- /dev/null +++ b/zookeeper-contrib/zookeeper-contrib-fatjar/README.md @@ -0,0 +1,21 @@ +ZooKeeper Fatjar +================ + +This package contains build to create a fat zookeeper jar. Fatjar can be used to run: +- zookeeper server +- zookeeper client +- distributed load generator for testing (generateLoad) +- container that will instantiate classes as directed by an instance manager (ic) +- system test (systest) +- jmh micro benchmarks (jmh) + + +Use following command to build fatjar +``` +mvn clean install -P fatjar -DskipTests +``` + +To run the fatjar use: +``` +java -jar zoookeeper--fatjar.jar +``` diff --git a/zookeeper-contrib/zookeeper-contrib-fatjar/README.txt b/zookeeper-contrib/zookeeper-contrib-fatjar/README.txt deleted file mode 100644 index f8027ae8c9e..00000000000 --- a/zookeeper-contrib/zookeeper-contrib-fatjar/README.txt +++ /dev/null @@ -1,2 +0,0 @@ -This package contains build to create a fat zookeeper jar. You need to run ant to create the fat jar. -To run the fatjar you can use. java -jar zoookeeper-*fatjar.jar diff --git a/zookeeper-contrib/zookeeper-contrib-fatjar/build.xml b/zookeeper-contrib/zookeeper-contrib-fatjar/build.xml deleted file mode 100644 index 3a1dd958e4e..00000000000 --- a/zookeeper-contrib/zookeeper-contrib-fatjar/build.xml +++ /dev/null @@ -1,74 +0,0 @@ - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - diff --git a/zookeeper-contrib/zookeeper-contrib-fatjar/pom.xml b/zookeeper-contrib/zookeeper-contrib-fatjar/pom.xml new file mode 100755 index 00000000000..9a90dd7fc3c --- /dev/null +++ b/zookeeper-contrib/zookeeper-contrib-fatjar/pom.xml @@ -0,0 +1,145 @@ + + + + 4.0.0 + + org.apache.zookeeper + zookeeper-contrib + 3.7.0-SNAPSHOT + + + org.apache.zookeeper + zookeeper-contrib-fatjar + jar + Apache ZooKeeper - Contrib - Fatjar + + + true + true + + + + + org.apache.zookeeper + zookeeper-jute + ${project.version} + + + org.apache.zookeeper + zookeeper + ${project.version} + + + org.apache.zookeeper + zookeeper + ${project.version} + test-jar + + + org.apache.zookeeper + zookeeper-it + ${project.version} + + + org.slf4j + slf4j-api + + + commons-cli + commons-cli + + + org.eclipse.jetty + jetty-server + + + org.eclipse.jetty + jetty-servlet + + + com.fasterxml.jackson.core + jackson-databind + + + com.googlecode.json-simple + json-simple + + + jline + jline + + + io.dropwizard.metrics + metrics-core + + + org.xerial.snappy + snappy-java + + + log4j + log4j + + + + + + + ${project.basedir}/src/main/resources + + + ${project.basedir}/../../conf + + log4j.properties + + + + + + org.apache.maven.plugins + maven-assembly-plugin + + + jar-with-dependencies + + + + org.apache.zookeeper.util.FatJarMain + + + zookeeper-${project.version}-fatjar + false + false + + + + make-assembly + package + + single + + + + + + + + diff --git a/zookeeper-contrib/zookeeper-contrib-fatjar/src/main/resources/mainClasses b/zookeeper-contrib/zookeeper-contrib-fatjar/src/main/resources/mainClasses index ba29e891973..c7b27a1325b 100644 --- a/zookeeper-contrib/zookeeper-contrib-fatjar/src/main/resources/mainClasses +++ b/zookeeper-contrib/zookeeper-contrib-fatjar/src/main/resources/mainClasses @@ -4,8 +4,6 @@ client:org.apache.zookeeper.ZooKeeperMain:Client shell to ZooKeeper server:org.apache.zookeeper.server.quorum.QuorumPeerMain:Start ZooKeeper server ::Test Commands generateLoad:org.apache.zookeeper.test.system.GenerateLoad:A distributed load generator for testing -quorumBench:org.apache.zookeeper.server.QuorumBenchmark:A benchmark of just the quorum protocol -abBench:org.apache.zookeeper.server.quorum.AtomicBroadcastBenchmark:A benchmark of just the atomic broadcast ic:org.apache.zookeeper.test.system.InstanceContainer:A container that will instantiate classes as directed by an instance manager systest:org.apache.zookeeper.test.system.BaseSysTest:Start system test jmh:org.apache.zookeeper.BenchMain:Run jmh micro benchmarks diff --git a/zookeeper-it/README.txt b/zookeeper-it/README.txt index 8a2985c2eda..2c4b1cc4865 100644 --- a/zookeeper-it/README.txt +++ b/zookeeper-it/README.txt @@ -7,11 +7,12 @@ The easiest way to do all of this is to use the zookeeper fat jar. Steps to run system test ------------------------ -1) transfer the fatjar from the release directory to all systems - participating in the test. fatjar is in contrib/fatjar directory. +1) build and transfer the fatjar from the `zookeeper-contrib/zookeeper-contrib-fatjar/target` + directory to all systems participating in the test - (developers can generate by running "ant jar compile-test" - targets in trunk, then compiling using "ant jar" in src/contrib/jarjar) + Command to build fatjar without executing the tests: + + `mvn clean install -P fatjar -DskipTests` 2) run a zookeeper standalone instance (cluster is ok too) diff --git a/zookeeper-it/pom.xml b/zookeeper-it/pom.xml new file mode 100755 index 00000000000..163930fa3c5 --- /dev/null +++ b/zookeeper-it/pom.xml @@ -0,0 +1,77 @@ + + + + 4.0.0 + + org.apache.zookeeper + parent + 3.7.0-SNAPSHOT + + + zookeeper-it + jar + Apache ZooKeeper - Tests + + ZooKeeper system tests + + + + + org.apache.zookeeper + zookeeper + ${project.version} + + + org.apache.zookeeper + zookeeper + tests + test-jar + ${project.version} + + + junit + junit + + + org.openjdk.jmh + jmh-core + 1.23 + + + org.openjdk.jmh + jmh-generator-annprocess + 1.23 + + + + + src + + + org.apache.maven.plugins + maven-javadoc-plugin + + src + + + + + diff --git a/zookeeper-it/src/test/bench/org/apache/zookeeper/server/watch/WatchBench.java b/zookeeper-it/src/test/bench/org/apache/zookeeper/server/watch/WatchBench.java index 0510df76ca6..aee5b2f18ab 100644 --- a/zookeeper-it/src/test/bench/org/apache/zookeeper/server/watch/WatchBench.java +++ b/zookeeper-it/src/test/bench/org/apache/zookeeper/server/watch/WatchBench.java @@ -35,9 +35,9 @@ public class WatchBench { static final EventType event = EventType.NodeDataChanged; static IWatchManager createWatchManager(String className) throws Exception { - Class clazz = Class.forName( + Class clazz = Class.forName( "org.apache.zookeeper.server.watch." + className); - return (IWatchManager) clazz.newInstance(); + return (IWatchManager) clazz.getConstructor().newInstance(); } static void forceGC() { diff --git a/zookeeper-it/src/test/java/org/apache/zookeeper/test/system/BaseSysTest.java b/zookeeper-it/src/test/java/org/apache/zookeeper/test/system/BaseSysTest.java index 8856282e6dc..6816f2d80cf 100644 --- a/zookeeper-it/src/test/java/org/apache/zookeeper/test/system/BaseSysTest.java +++ b/zookeeper-it/src/test/java/org/apache/zookeeper/test/system/BaseSysTest.java @@ -20,6 +20,7 @@ import java.io.File; import java.io.IOException; +import java.lang.reflect.InvocationTargetException; import java.net.InetAddress; import java.net.InetSocketAddress; import java.net.UnknownHostException; @@ -223,15 +224,15 @@ private void distributedConfigureClients(int count, Class cl } private Instance fakeBaseClients[]; - private void fakeConfigureClients(int count, Class clazz, String params) throws IOException, ClassNotFoundException { + private void fakeConfigureClients(int count, Class clazz, String params) { fakeBaseClients = new Instance[count]; for(int i = 0; i < count; i++) { try { - fakeBaseClients[i] = clazz.newInstance(); - } catch (InstantiationException e) { - e.printStackTrace(); - return; - } catch (IllegalAccessException e) { + fakeBaseClients[i] = clazz.getConstructor().newInstance(); + } catch (InstantiationException + | IllegalAccessException + | NoSuchMethodException + | InvocationTargetException e) { e.printStackTrace(); return; } diff --git a/zookeeper-it/src/test/java/org/apache/zookeeper/test/system/InstanceContainer.java b/zookeeper-it/src/test/java/org/apache/zookeeper/test/system/InstanceContainer.java index 1cb7bc02edb..44f6531b94f 100644 --- a/zookeeper-it/src/test/java/org/apache/zookeeper/test/system/InstanceContainer.java +++ b/zookeeper-it/src/test/java/org/apache/zookeeper/test/system/InstanceContainer.java @@ -271,7 +271,7 @@ public void processResult(int rc, String path, Object ctx, List children } try { Class c = Class.forName(clazz); - i = (Instance)c.newInstance(); + i = (Instance) c.getConstructor().newInstance(); Reporter reporter = new MyReporter(child); i.setReporter(reporter); i.configure(conf); From a4c97d255ad7bb86c61a35d01415de1caebd3dac Mon Sep 17 00:00:00 2001 From: tison Date: Tue, 31 Mar 2020 13:39:49 +0200 Subject: [PATCH 009/591] ZOOKEEPER-3771: Update zk-merge-pr script to Python3 eolivelli generally I use `2to3` util and check the codepath that I can arrive, manually fix some lines. But it seems we can verify this patch totally when merging this patch :) Author: tison Reviewers: Enrico Olivelli , Norbert Kalmar Closes #1295 from TisonKun/patch-1 --- zk-merge-pr.py | 116 ++++++++++++++++++++++++------------------------- 1 file changed, 58 insertions(+), 58 deletions(-) diff --git a/zk-merge-pr.py b/zk-merge-pr.py index 5768a94d74f..dc5549d9dfa 100644 --- a/zk-merge-pr.py +++ b/zk-merge-pr.py @@ -32,7 +32,7 @@ import re import subprocess import sys -import urllib2 +import urllib.request, urllib.error, urllib.parse import getpass try: @@ -75,48 +75,48 @@ def get_json(url): try: - request = urllib2.Request(url) + request = urllib.request.Request(url) if GITHUB_OAUTH_KEY: request.add_header('Authorization', 'token %s' % GITHUB_OAUTH_KEY) - return json.load(urllib2.urlopen(request)) - except urllib2.HTTPError as e: + return json.load(urllib.request.urlopen(request)) + except urllib.error.HTTPError as e: if "X-RateLimit-Remaining" in e.headers and e.headers["X-RateLimit-Remaining"] == '0': - print "Exceeded the GitHub API rate limit; see the instructions in " + \ + print("Exceeded the GitHub API rate limit; see the instructions in " + \ "zk-merge-pr.py to configure an OAuth token for making authenticated " + \ - "GitHub requests." + "GitHub requests.") else: - print "Unable to fetch URL, exiting: %s" % url + print("Unable to fetch URL, exiting: %s" % url) sys.exit(-1) def fail(msg): - print msg + print(msg) clean_up() sys.exit(-1) def run_cmd(cmd): - print cmd + print(cmd) if isinstance(cmd, list): - return subprocess.check_output(cmd) + return subprocess.check_output(cmd, encoding='utf8') else: - return subprocess.check_output(cmd.split(" ")) + return subprocess.check_output(cmd.split(" "), encoding='utf8') def continue_maybe(prompt): - result = raw_input("\n%s (y/n): " % prompt) + result = input("\n%s (y/n): " % prompt) if result.lower().strip() != "y": fail("Okay, exiting") def clean_up(): if original_head != get_current_branch(): - print "Restoring head pointer to %s" % original_head + print("Restoring head pointer to %s" % original_head) run_cmd("git checkout %s" % original_head) branches = run_cmd("git branch").replace(" ", "").split("\n") - for branch in filter(lambda x: x.startswith(TEMP_BRANCH_PREFIX), branches): - print "Deleting local branch %s" % branch + for branch in [x for x in branches if x.startswith(TEMP_BRANCH_PREFIX)]: + print("Deleting local branch %s" % branch) run_cmd("git branch -D %s" % branch) def get_current_branch(): @@ -144,20 +144,20 @@ def merge_pr(pr_num, target_ref, title, body, pr_repo_desc): '--pretty=format:%an <%ae>']).split("\n") distinct_authors = sorted(set(commit_authors), key=lambda x: commit_authors.count(x), reverse=True) - primary_author = raw_input( + primary_author = input( "Enter primary author in the format of \"name \" [%s]: " % distinct_authors[0]) if primary_author == "": primary_author = distinct_authors[0] - reviewers = raw_input( + reviewers = input( "Enter reviewers in the format of \"name1 , name2 \": ").strip() commits = run_cmd(['git', 'log', 'HEAD..%s' % pr_branch_name, '--pretty=format:%h [%an] %s']).split("\n") if len(commits) > 1: - result = raw_input("List pull request commits in squashed commit message? (y/n): ") + result = input("List pull request commits in squashed commit message? (y/n): ") if result.lower().strip() == "y": should_list_commits = True else: @@ -209,13 +209,13 @@ def merge_pr(pr_num, target_ref, title, body, pr_repo_desc): merge_hash = run_cmd("git rev-parse %s" % target_branch_name)[:8] clean_up() - print("Pull request #%s merged!" % pr_num) - print("Merge hash: %s" % merge_hash) + print(("Pull request #%s merged!" % pr_num)) + print(("Merge hash: %s" % merge_hash)) return merge_hash def cherry_pick(pr_num, merge_hash, default_branch): - pick_ref = raw_input("Enter a branch name [%s]: " % default_branch) + pick_ref = input("Enter a branch name [%s]: " % default_branch) if pick_ref == "": pick_ref = default_branch @@ -244,21 +244,21 @@ def cherry_pick(pr_num, merge_hash, default_branch): pick_hash = run_cmd("git rev-parse %s" % pick_branch_name)[:8] clean_up() - print("Pull request #%s picked into %s!" % (pr_num, pick_ref)) - print("Pick hash: %s" % pick_hash) + print(("Pull request #%s picked into %s!" % (pr_num, pick_ref))) + print(("Pick hash: %s" % pick_hash)) return pick_ref def fix_version_from_branch(branch, versions): # Note: Assumes this is a sorted (newest->oldest) list of un-released versions if branch == DEV_BRANCH_NAME: - versions = filter(lambda x: x == DEFAULT_FIX_VERSION, versions) + versions = [x for x in versions if x == DEFAULT_FIX_VERSION] if len(versions) > 0: return versions[0] else: return None else: - versions = filter(lambda x: x.startswith(branch), versions) + versions = [x for x in versions if x.startswith(branch)] if len(versions) > 0: return versions[-1] else: @@ -269,7 +269,7 @@ def resolve_jira_issue(merge_branches, comment, default_jira_id=""): asf_jira = jira.client.JIRA({'server': JIRA_API_BASE}, basic_auth=(JIRA_USERNAME, JIRA_PASSWORD)) - jira_id = raw_input("Enter a JIRA id [%s]: " % default_jira_id) + jira_id = input("Enter a JIRA id [%s]: " % default_jira_id) if jira_id == "": jira_id = default_jira_id @@ -288,20 +288,20 @@ def resolve_jira_issue(merge_branches, comment, default_jira_id=""): if cur_status == "Resolved" or cur_status == "Closed": fail("JIRA issue %s already has status '%s'" % (jira_id, cur_status)) - print ("=== JIRA %s ===" % jira_id) - print ("summary\t\t%s\nassignee\t%s\nstatus\t\t%s\nurl\t\t%s/%s\n" % ( - cur_summary, cur_assignee, cur_status, JIRA_BASE, jira_id)) + print(("=== JIRA %s ===" % jira_id)) + print(("summary\t\t%s\nassignee\t%s\nstatus\t\t%s\nurl\t\t%s/%s\n" % ( + cur_summary, cur_assignee, cur_status, JIRA_BASE, jira_id))) versions = asf_jira.project_versions(CAPITALIZED_PROJECT_NAME) versions = sorted(versions, key=lambda x: x.name, reverse=True) - versions = filter(lambda x: x.raw['released'] is False, versions) + versions = [x for x in versions if x.raw['released'] is False] - version_names = map(lambda x: x.name, versions) - default_fix_versions = map(lambda x: fix_version_from_branch(x, version_names), merge_branches) - default_fix_versions = filter(lambda x: x != None, default_fix_versions) + version_names = [x.name for x in versions] + default_fix_versions = [fix_version_from_branch(x, version_names) for x in merge_branches] + default_fix_versions = [x for x in default_fix_versions if x != None] default_fix_versions = ",".join(default_fix_versions) - fix_versions = raw_input("Enter comma-separated fix version(s) [%s]: " % default_fix_versions) + fix_versions = input("Enter comma-separated fix version(s) [%s]: " % default_fix_versions) if fix_versions == "": fix_versions = default_fix_versions fix_versions = fix_versions.replace(" ", "").split(",") @@ -309,7 +309,7 @@ def resolve_jira_issue(merge_branches, comment, default_jira_id=""): def get_version_json(version_str): return filter(lambda v: v.name == version_str, versions)[0].raw - jira_fix_versions = map(lambda v: get_version_json(v), fix_versions) + jira_fix_versions = [get_version_json(v) for v in fix_versions] resolve = filter(lambda a: a['name'] == "Resolve Issue", asf_jira.transitions(jira_id))[0] resolution = filter(lambda r: r.raw['name'] == "Fixed", asf_jira.resolutions())[0] @@ -317,7 +317,7 @@ def get_version_json(version_str): jira_id, resolve["id"], fixVersions = jira_fix_versions, comment = comment, resolution = {'id': resolution.raw['id']}) - print "Successfully resolved %s with fixVersions=%s!" % (jira_id, fix_versions) + print("Successfully resolved %s with fixVersions=%s!" % (jira_id, fix_versions)) def resolve_jira_issues(title, merge_branches, comment): @@ -401,7 +401,7 @@ def check_git_remote(): # check if all remote endpoints' URLs point to project git repo name = PROJECT_NAME + ".git" - for url in repos.values(): + for url in list(repos.values()): if not url.endswith(name): fail("Error: not a %s git repo or at least one remote is invalid" % PROJECT_NAME) @@ -419,7 +419,7 @@ def check_jira_env(): if JIRA_IMPORTED: if JIRA_USERNAME.strip() != "" and JIRA_PASSWORD.strip() == "": - inform_pwd = raw_input("JIRA_USERNAME set but JIRA_PASSWORD is not. Want to inform it? ") + inform_pwd = input("JIRA_USERNAME set but JIRA_PASSWORD is not. Want to inform it? ") if inform_pwd.strip() == "y": JIRA_PASSWORD = getpass.getpass('JIRA PASSWORD: ') @@ -440,34 +440,34 @@ def main(): check_git_remote() branches = get_json("%s/branches" % GITHUB_API_BASE) - branch_names = filter(lambda x: x.startswith(RELEASE_BRANCH_PREFIX), [x['name'] for x in branches]) + branch_names = [x for x in [x['name'] for x in branches] if x.startswith(RELEASE_BRANCH_PREFIX)] # Assumes branch names can be sorted lexicographically latest_branch = sorted(branch_names, reverse=True)[0] - pr_num = raw_input("Which pull request would you like to merge? (e.g. 34): ") + pr_num = input("Which pull request would you like to merge? (e.g. 34): ") pr = get_json("%s/pulls/%s" % (GITHUB_API_BASE, pr_num)) pr_events = get_json("%s/issues/%s/events" % (GITHUB_API_BASE, pr_num)) url = pr["url"] pr_title = pr["title"] - commit_title = raw_input("Commit title [%s]: " % pr_title.encode("utf-8")).decode("utf-8") + commit_title = input("Commit title [%s]: " % pr_title) if commit_title == "": commit_title = pr_title # Decide whether to use the modified title or not modified_title = standardize_jira_ref(commit_title) if modified_title != commit_title: - print "I've re-written the title as follows to match the standard format:" - print "Original: %s" % commit_title - print "Modified: %s" % modified_title - result = raw_input("Would you like to use the modified title? (y/n): ") + print("I've re-written the title as follows to match the standard format:") + print("Original: %s" % commit_title) + print("Modified: %s" % modified_title) + result = input("Would you like to use the modified title? (y/n): ") if result.lower().strip() == "y": commit_title = modified_title - print "Using modified title:" + print("Using modified title:") else: - print "Using original title:" - print commit_title + print("Using original title:") + print(commit_title) body = pr["body"] target_ref = pr["base"]["ref"] @@ -484,13 +484,13 @@ def main(): merge_hash = merge_commits[0]["commit_id"] message = get_json("%s/commits/%s" % (GITHUB_API_BASE, merge_hash))["commit"]["message"] - print "Pull request %s has already been merged, assuming you want to backport" % pr_num + print("Pull request %s has already been merged, assuming you want to backport" % pr_num) commit_is_downloaded = run_cmd(['git', 'rev-parse', '--quiet', '--verify', "%s^{commit}" % merge_hash]).strip() != "" if not commit_is_downloaded: fail("Couldn't find any merge commit for #%s, you may need to update HEAD." % pr_num) - print "Found commit %s:\n%s" % (merge_hash, message) + print("Found commit %s:\n%s" % (merge_hash, message)) cherry_pick(pr_num, merge_hash, latest_branch) sys.exit(0) @@ -499,9 +499,9 @@ def main(): "Continue? (experts only!)" continue_maybe(msg) - print ("\n=== Pull Request #%s ===" % pr_num) - print ("PR title\t%s\nCommit title\t%s\nSource\t\t%s\nTarget\t\t%s\nURL\t\t%s" % ( - pr_title, commit_title, pr_repo_desc, target_ref, url)) + print(("\n=== Pull Request #%s ===" % pr_num)) + print(("PR title\t%s\nCommit title\t%s\nSource\t\t%s\nTarget\t\t%s\nURL\t\t%s" % ( + pr_title, commit_title, pr_repo_desc, target_ref, url))) continue_maybe("Proceed with merging pull request #%s?" % pr_num) merged_refs = [target_ref] @@ -509,7 +509,7 @@ def main(): merge_hash = merge_pr(pr_num, target_ref, commit_title, body, pr_repo_desc) pick_prompt = "Would you like to pick %s into another branch?" % merge_hash - while raw_input("\n%s (y/n): " % pick_prompt).lower().strip() == "y": + while input("\n%s (y/n): " % pick_prompt).lower().strip() == "y": merged_refs = merged_refs + [cherry_pick(pr_num, merge_hash, latest_branch)] if JIRA_IMPORTED: @@ -518,11 +518,11 @@ def main(): jira_comment = "Issue resolved by pull request %s\n[%s/%s]" % (pr_num, GITHUB_BASE, pr_num) resolve_jira_issues(commit_title, merged_refs, jira_comment) else: - print "JIRA_USERNAME and JIRA_PASSWORD not set" - print "Exiting without trying to close the associated JIRA." + print("JIRA_USERNAME and JIRA_PASSWORD not set") + print("Exiting without trying to close the associated JIRA.") else: - print "Could not find jira-python library. Run 'sudo pip install jira' to install." - print "Exiting without trying to close the associated JIRA." + print("Could not find jira-python library. Run 'sudo pip install jira' to install.") + print("Exiting without trying to close the associated JIRA.") if __name__ == "__main__": import doctest From 7e6386aa9b404dd236b6bdc3b5c714116f3986db Mon Sep 17 00:00:00 2001 From: Jono Date: Wed, 1 Apr 2020 17:02:20 +0200 Subject: [PATCH 010/591] ZOOKEEPER-3581: Use factory design pattern to refactor ZooKeeperMain Using a factory design pattern to refactor ZooKeeperMain, making the code more elegant. Author: Jono Reviewers: andor@apache.org Closes #1255 from jono-morris/ZOOKEEPER-3581 and squashes the following commits: 556e1763e [Jono] Fix style errors with import statements. e10dfb872 [Jono] Add licence text. 4957d89b5 [Jono] 1. Reinstate missing imports to ZooKeeperMain. ed76f14da [Jono] Minor Javadoc updates. cc8de4920 [Jono] Add Factory to create CliCommand classes. Initial Commit. --- .../org/apache/zookeeper/ZooKeeperMain.java | 62 +++------------- .../apache/zookeeper/cli/CommandFactory.java | 74 +++++++++++++++++++ .../zookeeper/cli/CommandFactoryTest.java | 38 ++++++++++ 3 files changed, 123 insertions(+), 51 deletions(-) create mode 100644 zookeeper-server/src/main/java/org/apache/zookeeper/cli/CommandFactory.java create mode 100644 zookeeper-server/src/test/java/org/apache/zookeeper/cli/CommandFactoryTest.java diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/ZooKeeperMain.java b/zookeeper-server/src/main/java/org/apache/zookeeper/ZooKeeperMain.java index 243aca33c39..ff8ce623286 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/ZooKeeperMain.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/ZooKeeperMain.java @@ -31,38 +31,17 @@ import java.util.LinkedList; import java.util.List; import java.util.Map; -import java.util.Map.Entry; import java.util.NoSuchElementException; import java.util.regex.Matcher; import java.util.regex.Pattern; +import java.util.stream.Stream; import org.apache.yetus.audience.InterfaceAudience; import org.apache.zookeeper.admin.ZooKeeperAdmin; -import org.apache.zookeeper.cli.AddAuthCommand; -import org.apache.zookeeper.cli.AddWatchCommand; import org.apache.zookeeper.cli.CliCommand; import org.apache.zookeeper.cli.CliException; -import org.apache.zookeeper.cli.CloseCommand; +import org.apache.zookeeper.cli.CommandFactory; import org.apache.zookeeper.cli.CommandNotFoundException; -import org.apache.zookeeper.cli.CreateCommand; -import org.apache.zookeeper.cli.DelQuotaCommand; -import org.apache.zookeeper.cli.DeleteAllCommand; -import org.apache.zookeeper.cli.DeleteCommand; -import org.apache.zookeeper.cli.GetAclCommand; -import org.apache.zookeeper.cli.GetAllChildrenNumberCommand; -import org.apache.zookeeper.cli.GetCommand; -import org.apache.zookeeper.cli.GetConfigCommand; -import org.apache.zookeeper.cli.GetEphemeralsCommand; -import org.apache.zookeeper.cli.ListQuotaCommand; -import org.apache.zookeeper.cli.LsCommand; import org.apache.zookeeper.cli.MalformedCommandException; -import org.apache.zookeeper.cli.ReconfigCommand; -import org.apache.zookeeper.cli.RemoveWatchesCommand; -import org.apache.zookeeper.cli.SetAclCommand; -import org.apache.zookeeper.cli.SetCommand; -import org.apache.zookeeper.cli.SetQuotaCommand; -import org.apache.zookeeper.cli.StatCommand; -import org.apache.zookeeper.cli.SyncCommand; -import org.apache.zookeeper.cli.VersionCommand; import org.apache.zookeeper.client.ZKClientConfig; import org.apache.zookeeper.server.ExitCode; import org.apache.zookeeper.server.quorum.QuorumPeerConfig; @@ -100,34 +79,15 @@ public boolean getPrintWatches() { commandMap.put("redo", "cmdno"); commandMap.put("printwatches", "on|off"); commandMap.put("quit", ""); - - new CloseCommand().addToMap(commandMapCli); - new CreateCommand().addToMap(commandMapCli); - new DeleteCommand().addToMap(commandMapCli); - new DeleteAllCommand().addToMap(commandMapCli); - new SetCommand().addToMap(commandMapCli); - new GetCommand().addToMap(commandMapCli); - new LsCommand().addToMap(commandMapCli); - new GetAclCommand().addToMap(commandMapCli); - new SetAclCommand().addToMap(commandMapCli); - new StatCommand().addToMap(commandMapCli); - new SyncCommand().addToMap(commandMapCli); - new SetQuotaCommand().addToMap(commandMapCli); - new ListQuotaCommand().addToMap(commandMapCli); - new DelQuotaCommand().addToMap(commandMapCli); - new AddAuthCommand().addToMap(commandMapCli); - new ReconfigCommand().addToMap(commandMapCli); - new GetConfigCommand().addToMap(commandMapCli); - new RemoveWatchesCommand().addToMap(commandMapCli); - new GetEphemeralsCommand().addToMap(commandMapCli); - new GetAllChildrenNumberCommand().addToMap(commandMapCli); - new VersionCommand().addToMap(commandMapCli); - new AddWatchCommand().addToMap(commandMapCli); - - // add all to commandMap - for (Entry entry : commandMapCli.entrySet()) { - commandMap.put(entry.getKey(), entry.getValue().getOptionStr()); - } + Stream.of(CommandFactory.Command.values()) + .map(command -> CommandFactory.getInstance(command)) + // add all commands to commandMapCli and commandMap + .forEach(cliCommand ->{ + cliCommand.addToMap(commandMapCli); + commandMap.put( + cliCommand.getCmdStr(), + cliCommand.getOptionStr()); + }); } static void usage() { diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/cli/CommandFactory.java b/zookeeper-server/src/main/java/org/apache/zookeeper/cli/CommandFactory.java new file mode 100644 index 00000000000..87f016398d7 --- /dev/null +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/cli/CommandFactory.java @@ -0,0 +1,74 @@ +/* + * 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.cli; + +import java.util.function.Supplier; + +/** + * Factory class for creating instances of {@link CliCommand}. + */ +public class CommandFactory { + + /** + * All Cli Commands. + */ + public enum Command { + CLOSE(CloseCommand::new), + CREATE(CreateCommand::new), + DELETE(DeleteCommand::new), + DELETE_ALL(DeleteAllCommand::new), + SET(SetCommand::new), + GET(GetCommand::new), + LS(LsCommand::new), + GET_ACL(GetAclCommand::new), + SET_ACL(SetAclCommand::new), + STAT(StatCommand::new), + SYNC(SyncCommand::new), + SET_QUOTA(SetQuotaCommand::new), + LIST_QUOTA(ListQuotaCommand::new), + DEL_QUOTA(DelQuotaCommand::new), + ADD_AUTH(AddAuthCommand::new), + RECONFIG(ReconfigCommand::new), + GET_CONFIG(GetConfigCommand::new), + REMOVE_WATCHES(RemoveWatchesCommand::new), + GET_EPHEMERALS(GetEphemeralsCommand::new), + GET_ALL_CHILDREN_NUMBER(GetAllChildrenNumberCommand::new), + VERSION(VersionCommand::new), + ADD_WATCH(AddWatchCommand::new); + + private Supplier instantiator; + + private CliCommand getInstance() { + return instantiator.get(); + } + + Command(Supplier instantiator) { + this.instantiator = instantiator; + } + } + + /** + * Creates a new {@link CliCommand} instance. + * @param command the {@link Command} to create a new instance of + * @return the new {@code CliCommand} instance + */ + public static CliCommand getInstance (Command command) { + return command.getInstance(); + } +} diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/cli/CommandFactoryTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/cli/CommandFactoryTest.java new file mode 100644 index 00000000000..a118399d520 --- /dev/null +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/cli/CommandFactoryTest.java @@ -0,0 +1,38 @@ +/* + * 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.cli; + +import static org.junit.Assert.assertTrue; +import org.junit.Test; + +/** + * Unit test for {@link CommandFactory}. + */ +public class CommandFactoryTest { + + /** + * Verify that the {@code CommandFactory} can create a command instance. + */ + @Test + public void testCommandCreation() { + CliCommand cliCommand = + CommandFactory.getInstance(CommandFactory.Command.CREATE); + assertTrue(cliCommand instanceof CreateCommand); + } +} From a548253408384d9e943b151c942d795017dca70c Mon Sep 17 00:00:00 2001 From: Mate Szalay-Beko Date: Tue, 7 Apr 2020 09:07:50 +0200 Subject: [PATCH 011/591] ZOOKEEPER-3769: handling malformed Leader Election notification messages Using ZooKeeper with JDK 12.0.2 on CentOS 7 when the current leader is killed, we saw a few times that some partial Leader Election notification (vote) messages were delivered to the other ZooKeeper servers. The malformed / partial messages are causing different exceptions in the WorkerReceiver thread of FastLeaderElection which were not handled before. This was leading to the death of the WorkerReceiver thread, which caused that the given ZooKeeper Server was unable to receive leader election messages anymore and was not able to re-join to any quorum until it got restarted. In the proposed fix I created unit tests to simulate certain error cases with regards to partial leader election messages, and fixed the error handling in FastLeaderElection. Author: Mate Szalay-Beko Reviewers: Enrico Olivelli , Norbert Kalmar Closes #1300 from symat/ZOOKEEPER-3769-master --- .../server/quorum/FastLeaderElection.java | 115 ++++---- .../FLEMalformedNotificationMessageTest.java | 249 ++++++++++++++++++ 2 files changed, 313 insertions(+), 51 deletions(-) create mode 100644 zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/FLEMalformedNotificationMessageTest.java diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/FastLeaderElection.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/FastLeaderElection.java index 3ab10073f4d..967adf05a19 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/FastLeaderElection.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/FastLeaderElection.java @@ -19,6 +19,7 @@ package org.apache.zookeeper.server.quorum; import java.io.IOException; +import java.nio.BufferUnderflowException; import java.nio.ByteBuffer; import java.util.HashMap; import java.util.Map; @@ -237,19 +238,21 @@ public void run() { continue; } + final int capacity = response.buffer.capacity(); + // The current protocol and two previous generations all send at least 28 bytes - if (response.buffer.capacity() < 28) { - LOG.error("Got a short response: {}", response.buffer.capacity()); + if (capacity < 28) { + LOG.error("Got a short response from server {}: {}", response.sid, capacity); continue; } // this is the backwardCompatibility mode in place before ZK-107 // It is for a version of the protocol in which we didn't send peer epoch // With peer epoch and version the message became 40 bytes - boolean backCompatibility28 = (response.buffer.capacity() == 28); + boolean backCompatibility28 = (capacity == 28); // this is the backwardCompatibility mode for no version information - boolean backCompatibility40 = (response.buffer.capacity() == 40); + boolean backCompatibility40 = (capacity == 40); response.buffer.clear(); @@ -263,64 +266,74 @@ public void run() { long rpeerepoch; int version = 0x0; - if (!backCompatibility28) { - rpeerepoch = response.buffer.getLong(); - if (!backCompatibility40) { - /* - * Version added in 3.4.6 - */ + QuorumVerifier rqv = null; - version = response.buffer.getInt(); + try { + if (!backCompatibility28) { + rpeerepoch = response.buffer.getLong(); + if (!backCompatibility40) { + /* + * Version added in 3.4.6 + */ + + version = response.buffer.getInt(); + } else { + LOG.info("Backward compatibility mode (36 bits), server id: {}", response.sid); + } } else { - LOG.info("Backward compatibility mode (36 bits), server id: {}", response.sid); + LOG.info("Backward compatibility mode (28 bits), server id: {}", response.sid); + rpeerepoch = ZxidUtils.getEpochFromZxid(rzxid); } - } else { - LOG.info("Backward compatibility mode (28 bits), server id: {}", response.sid); - rpeerepoch = ZxidUtils.getEpochFromZxid(rzxid); - } - QuorumVerifier rqv = null; + // check if we have a version that includes config. If so extract config info from message. + if (version > 0x1) { + int configLength = response.buffer.getInt(); + + // we want to avoid errors caused by the allocation of a byte array with negative length + // (causing NegativeArraySizeException) or huge length (causing e.g. OutOfMemoryError) + if (configLength < 0 || configLength > capacity) { + throw new IOException(String.format("Invalid configLength in notification message! sid=%d, capacity=%d, version=%d, configLength=%d", + response.sid, capacity, version, configLength)); + } - // check if we have a version that includes config. If so extract config info from message. - if (version > 0x1) { - int configLength = response.buffer.getInt(); - byte[] b = new byte[configLength]; - - response.buffer.get(b); - - synchronized (self) { - try { - rqv = self.configFromString(new String(b)); - QuorumVerifier curQV = self.getQuorumVerifier(); - if (rqv.getVersion() > curQV.getVersion()) { - LOG.info("{} Received version: {} my version: {}", - self.getId(), - Long.toHexString(rqv.getVersion()), - Long.toHexString(self.getQuorumVerifier().getVersion())); - if (self.getPeerState() == ServerState.LOOKING) { - LOG.debug("Invoking processReconfig(), state: {}", self.getServerState()); - self.processReconfig(rqv, null, null, false); - if (!rqv.equals(curQV)) { - LOG.info("restarting leader election"); - self.shuttingDownLE = true; - self.getElectionAlg().shutdown(); - - break; + byte[] b = new byte[configLength]; + response.buffer.get(b); + + synchronized (self) { + try { + rqv = self.configFromString(new String(b)); + QuorumVerifier curQV = self.getQuorumVerifier(); + if (rqv.getVersion() > curQV.getVersion()) { + LOG.info("{} Received version: {} my version: {}", + self.getId(), + Long.toHexString(rqv.getVersion()), + Long.toHexString(self.getQuorumVerifier().getVersion())); + if (self.getPeerState() == ServerState.LOOKING) { + LOG.debug("Invoking processReconfig(), state: {}", self.getServerState()); + self.processReconfig(rqv, null, null, false); + if (!rqv.equals(curQV)) { + LOG.info("restarting leader election"); + self.shuttingDownLE = true; + self.getElectionAlg().shutdown(); + + break; + } + } else { + LOG.debug("Skip processReconfig(), state: {}", self.getServerState()); } - } else { - LOG.debug("Skip processReconfig(), state: {}", self.getServerState()); } + } catch (IOException | ConfigException e) { + LOG.error("Something went wrong while processing config received from {}", response.sid); } - } catch (IOException e) { - LOG.error("Something went wrong while processing config received from {}", response.sid); - } catch (ConfigException e) { - LOG.error("Something went wrong while processing config received from {}", response.sid); } + } else { + LOG.info("Backward compatibility mode (before reconfig), server id: {}", response.sid); } - } else { - LOG.info("Backward compatibility mode (before reconfig), server id: {}", response.sid); + } catch (BufferUnderflowException | IOException e) { + LOG.warn("Skipping the processing of a partial / malformed response message sent by sid={} (message length: {})", + response.sid, capacity, e); + continue; } - /* * If it is from a non-voting server (such as an observer or * a non-voting follower), respond right away. diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/FLEMalformedNotificationMessageTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/FLEMalformedNotificationMessageTest.java new file mode 100644 index 00000000000..8465c9ee55d --- /dev/null +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/FLEMalformedNotificationMessageTest.java @@ -0,0 +1,249 @@ +/** + * 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.File; +import java.io.IOException; +import java.net.InetSocketAddress; +import java.nio.ByteBuffer; +import java.util.HashMap; +import org.apache.zookeeper.PortAssignment; +import org.apache.zookeeper.ZKTestCase; +import org.apache.zookeeper.server.quorum.QuorumPeer.QuorumServer; +import org.apache.zookeeper.server.quorum.QuorumPeer.ServerState; +import org.apache.zookeeper.test.ClientBase; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +public class FLEMalformedNotificationMessageTest extends ZKTestCase { + private static final Logger LOG = LoggerFactory.getLogger(FLEMalformedNotificationMessageTest.class); + private static final byte[] CONFIG_BYTES = "my very invalid config string".getBytes(); + private static final int CONFIG_BYTES_LENGTH = CONFIG_BYTES.length; + + int count; + HashMap peers; + File tmpdir[]; + int port[]; + + QuorumCnxManager mockCnxManager; + FLETestUtils.LEThread leaderElectionThread; + QuorumPeer peerRunningLeaderElection; + + + @Before + public void setUp() throws Exception { + count = 3; + + peers = new HashMap<>(count); + tmpdir = new File[count]; + port = new int[count]; + + LOG.info("FLEMalformedNotificationMessageTest: {}, {}", getTestName(), count); + for (int i = 0; i < count; i++) { + int clientport = PortAssignment.unique(); + peers.put((long) i, + new QuorumServer(i, + new InetSocketAddress(clientport), + new InetSocketAddress(PortAssignment.unique()))); + tmpdir[i] = ClientBase.createTmpDir(); + port[i] = clientport; + } + + /* + * Start server 0 + */ + peerRunningLeaderElection = new QuorumPeer(peers, tmpdir[0], tmpdir[0], port[0], 3, 0, 1000, 2, 2, 2); + peerRunningLeaderElection.startLeaderElection(); + leaderElectionThread = new FLETestUtils.LEThread(peerRunningLeaderElection, 0); + leaderElectionThread.start(); + } + + + @After + public void tearDown() throws Exception { + peerRunningLeaderElection.shutdown(); + mockCnxManager.halt(); + } + + + @Test + public void testTooShortPartialNotificationMessage() throws Exception { + + /* + * Start mock server 1, send a message too short to be compatible with any protocol version + * This simulates the case when only some parts of the whole message is received. + */ + startMockServer(1); + byte requestBytes[] = new byte[12]; + ByteBuffer requestBuffer = ByteBuffer.wrap(requestBytes); + requestBuffer.clear(); + requestBuffer.putInt(ServerState.LOOKING.ordinal()); // state + requestBuffer.putLong(0); // leader + mockCnxManager.toSend(0L, requestBuffer); + + /* + * Assert that the message receiver thread in leader election is still healthy: + * we are sending valid votes and waiting for the leader election to be finished. + */ + sendValidNotifications(1, 0); + leaderElectionThread.join(5000); + if (leaderElectionThread.isAlive()) { + Assert.fail("Leader election thread didn't join, something went wrong."); + } + } + + + @Test + public void testNotificationMessageWithNegativeConfigLength() throws Exception { + + /* + * Start mock server 1, send a message with negative configLength field + */ + startMockServer(1); + byte requestBytes[] = new byte[48]; + ByteBuffer requestBuffer = ByteBuffer.wrap(requestBytes); + requestBuffer.clear(); + requestBuffer.putInt(ServerState.LOOKING.ordinal()); // state + requestBuffer.putLong(0); // leader + requestBuffer.putLong(0); // zxid + requestBuffer.putLong(0); // electionEpoch + requestBuffer.putLong(0); // epoch + requestBuffer.putInt(FastLeaderElection.Notification.CURRENTVERSION); // version + requestBuffer.putInt(-123); // configData.length + mockCnxManager.toSend(0L, requestBuffer); + + /* + * Assert that the message receiver thread in leader election is still healthy: + * we are sending valid votes and waiting for the leader election to be finished. + */ + sendValidNotifications(1, 0); + leaderElectionThread.join(5000); + if (leaderElectionThread.isAlive()) { + Assert.fail("Leader election thread didn't join, something went wrong."); + } + } + + + @Test + public void testNotificationMessageWithInvalidConfigLength() throws Exception { + + /* + * Start mock server 1, send a message with an invalid configLength field + * (instead of sending CONFIG_BYTES_LENGTH, we send 10000) + */ + startMockServer(1); + byte requestBytes[] = new byte[48 + CONFIG_BYTES_LENGTH]; + ByteBuffer requestBuffer = ByteBuffer.wrap(requestBytes); + requestBuffer.clear(); + requestBuffer.putInt(ServerState.LOOKING.ordinal()); // state + requestBuffer.putLong(0); // leader + requestBuffer.putLong(0); // zxid + requestBuffer.putLong(0); // electionEpoch + requestBuffer.putLong(0); // epoch + requestBuffer.putInt(FastLeaderElection.Notification.CURRENTVERSION); // version + requestBuffer.putInt(10000); // configData.length + requestBuffer.put(CONFIG_BYTES); // configData + mockCnxManager.toSend(0L, requestBuffer); + + /* + * Assert that the message receiver thread in leader election is still healthy: + * we are sending valid votes and waiting for the leader election to be finished. + */ + sendValidNotifications(1, 0); + leaderElectionThread.join(5000); + if (leaderElectionThread.isAlive()) { + Assert.fail("Leader election thread didn't join, something went wrong."); + } + } + + + @Test + public void testNotificationMessageWithInvalidConfig() throws Exception { + + /* + * Start mock server 1, send a message with an invalid config field + * (the receiver should not be able to parse the config part of the message) + */ + startMockServer(1); + ByteBuffer requestBuffer = FastLeaderElection.buildMsg(ServerState.LOOKING.ordinal(), 1, 0, 0, 0, CONFIG_BYTES); + mockCnxManager.toSend(0L, requestBuffer); + + /* + * Assert that the message receiver thread in leader election is still healthy: + * we are sending valid votes and waiting for the leader election to be finished. + */ + sendValidNotifications(1, 0); + leaderElectionThread.join(5000); + if (leaderElectionThread.isAlive()) { + Assert.fail("Leader election thread didn't join, something went wrong."); + } + } + + + @Test + public void testNotificationMessageWithBadProtocol() throws Exception { + + /* + * Start mock server 1, send an invalid 30 bytes long message + * (the receiver should not be able to parse the message and should skip it) + * This simulates the case when only some parts of the whole message is received. + */ + startMockServer(1); + byte requestBytes[] = new byte[30]; + ByteBuffer requestBuffer = ByteBuffer.wrap(requestBytes); + requestBuffer.clear(); + requestBuffer.putInt(ServerState.LOOKING.ordinal()); // state + requestBuffer.putLong(1); // leader + requestBuffer.putLong(0); // zxid + requestBuffer.putLong(0); // electionEpoch + requestBuffer.putShort((short) 0); // this is the first two bytes of a proper + // 8 bytes Long we should send here + mockCnxManager.toSend(0L, requestBuffer); + + /* + * Assert that the message receiver thread in leader election is still healthy: + * we are sending valid votes and waiting for the leader election to be finished. + */ + sendValidNotifications(1, 0); + leaderElectionThread.join(5000); + if (leaderElectionThread.isAlive()) { + Assert.fail("Leader election thread didn't join, something went wrong."); + } + } + + + void startMockServer(int sid) throws IOException { + QuorumPeer peer = new QuorumPeer(peers, tmpdir[sid], tmpdir[sid], port[sid], 3, sid, 1000, 2, 2, 2); + mockCnxManager = peer.createCnxnManager(); + mockCnxManager.listener.start(); + } + + + void sendValidNotifications(int fromSid, int toSid) throws InterruptedException { + mockCnxManager.toSend((long) toSid, FLETestUtils.createMsg(ServerState.LOOKING.ordinal(), fromSid, 0, 0)); + mockCnxManager.recvQueue.take(); + mockCnxManager.toSend((long) toSid, FLETestUtils.createMsg(ServerState.FOLLOWING.ordinal(), toSid, 0, 0)); + } + +} From d142e5a86993971a56bb2081020c085ed9ca68d7 Mon Sep 17 00:00:00 2001 From: Norbert Kalmar Date: Tue, 7 Apr 2020 15:40:39 +0200 Subject: [PATCH 012/591] ZOOKEEPER-3780: restore Version.getRevision() to be bacward compatible Added a warning in https://cwiki.apache.org/confluence/display/ZOOKEEPER/Upgrade+FAQ Author: Norbert Kalmar Reviewers: Enrico Olivelli Closes #1304 from nkalmar/branch-3.6 (cherry picked from commit 5062c39cb028115a9b79f9b28d6837065e70bf40) Signed-off-by: Enrico Olivelli --- .../src/main/java/org/apache/zookeeper/Version.java | 12 ++++++++++++ .../org/apache/zookeeper/version/util/VerGen.java | 1 + 2 files changed, 13 insertions(+) diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/Version.java b/zookeeper-server/src/main/java/org/apache/zookeeper/Version.java index 4029c60ede3..4fc0507a2fc 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/Version.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/Version.java @@ -24,6 +24,18 @@ public class Version implements org.apache.zookeeper.version.Info { + /* + * Since the SVN to Git port this field doesn't return the revision anymore + * In version 3.5.6, 3.5.7 and 3.6.0 this function is removed by accident. + * From version 3.5.8+ and 3.6.1+ it is restored for backward compatibility, but will be removed later + * @deprecated deprecated in 3.5.5, use @see {@link #getRevisionHash()} instead + * @return the default value -1 + */ + @Deprecated + public static int getRevision() { + return REVISION; + } + public static String getRevisionHash() { return REVISION_HASH; } diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/version/util/VerGen.java b/zookeeper-server/src/main/java/org/apache/zookeeper/version/util/VerGen.java index 25bffd72866..9bb699f090d 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/version/util/VerGen.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/version/util/VerGen.java @@ -84,6 +84,7 @@ public static void generateFile(File outputDir, Version version, String rev, Str if (rev.equals("-1")) { System.out.println("Unknown REVISION number, using " + rev); } + w.write(" int REVISION=-1; //@deprecated, please use REVISION_HASH\n"); w.write(" String REVISION_HASH=\"" + rev + "\";\n"); w.write(" String BUILD_DATE=\"" + buildDate + "\";\n"); w.write("}\n"); From a8fb880218410ec3b31374d957e5a0bd6c8cdf28 Mon Sep 17 00:00:00 2001 From: Enrico Olivelli Date: Tue, 7 Apr 2020 17:00:35 +0200 Subject: [PATCH 013/591] ZOOKEEPER-3785: Make sources buildable with JDK14 Author: Enrico Olivelli Reviewers: Mate Szalay-Beko , TisunKun Closes #1308 from eolivelli/fix/jdk14 --- .../src/main/java/org/apache/jute/compiler/JRecord.java | 1 + 1 file changed, 1 insertion(+) diff --git a/zookeeper-jute/src/main/java/org/apache/jute/compiler/JRecord.java b/zookeeper-jute/src/main/java/org/apache/jute/compiler/JRecord.java index f53399add02..7c73dedf9c9 100644 --- a/zookeeper-jute/src/main/java/org/apache/jute/compiler/JRecord.java +++ b/zookeeper-jute/src/main/java/org/apache/jute/compiler/JRecord.java @@ -434,6 +434,7 @@ public void genJavaCode(File outputDirectory) throws IOException { jj.write("\n"); jj.write("package " + getJavaPackage() + ";\n\n"); jj.write("import org.apache.jute.*;\n"); + jj.write("import org.apache.jute.Record; // JDK14 needs explicit import due to clash with java.lang.Record\n"); jj.write("import org.apache.yetus.audience.InterfaceAudience;\n"); jj.write("@InterfaceAudience.Public\n"); jj.write("public class " + getName() + " implements Record {\n"); From 09cb43554fb41eb651ebfb948f9a4230c2c4dfa2 Mon Sep 17 00:00:00 2001 From: tison Date: Wed, 8 Apr 2020 14:04:51 +0200 Subject: [PATCH 014/591] ZOOKEEPER-3782: Replace filter() with list comprehension for returning list After Py3, `filter` return a `filter object` instead of `list object`, which causes ``` Traceback (most recent call last): File "zk-merge-pr.py", line 533, in main() File "zk-merge-pr.py", line 519, in main resolve_jira_issues(commit_title, merged_refs, jira_comment) File "zk-merge-pr.py", line 329, in resolve_jira_issues resolve_jira_issue(merge_branches, comment, jira_id) File "zk-merge-pr.py", line 312, in resolve_jira_issue jira_fix_versions = [get_version_json(v) for v in fix_versions] File "zk-merge-pr.py", line 312, in jira_fix_versions = [get_version_json(v) for v in fix_versions] File "zk-merge-pr.py", line 310, in get_version_json return filter(lambda v: v.name == version_str, versions)[0].raw TypeError: 'filter' object is not subscriptable ``` We can replace filter with list comprehension to fix it. Author: tison Reviewers: Enrico Olivelli Closes #1303 from TisonKun/ZOOKEEPER-3782 --- zk-merge-pr.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/zk-merge-pr.py b/zk-merge-pr.py index dc5549d9dfa..0118cba2c64 100644 --- a/zk-merge-pr.py +++ b/zk-merge-pr.py @@ -307,12 +307,12 @@ def resolve_jira_issue(merge_branches, comment, default_jira_id=""): fix_versions = fix_versions.replace(" ", "").split(",") def get_version_json(version_str): - return filter(lambda v: v.name == version_str, versions)[0].raw + return [v for v in versions if v.name == version_str][0].raw jira_fix_versions = [get_version_json(v) for v in fix_versions] - resolve = filter(lambda a: a['name'] == "Resolve Issue", asf_jira.transitions(jira_id))[0] - resolution = filter(lambda r: r.raw['name'] == "Fixed", asf_jira.resolutions())[0] + resolve = [a for a in asf_jira.transitions(jira_id) if a['name'] == "Resolve Issue"][0] + resolution = [r for r in asf_jira.resolutions() if r.raw['name'] == "Fixed"][0] asf_jira.transition_issue( jira_id, resolve["id"], fixVersions = jira_fix_versions, comment = comment, resolution = {'id': resolution.raw['id']}) From f3c969718f9f8a590474388636510cf5c097b2aa Mon Sep 17 00:00:00 2001 From: Damien Diederen Date: Wed, 8 Apr 2020 19:18:32 +0200 Subject: [PATCH 015/591] ZOOKEEPER-3714: zkperl: Add (Cyrus) SASL authentication support to Perl client This patch allows one to access the C client Cyrus SASL support (ZOOKEEPER-1112) from the Perl binding by passing a `--with-sasl2` flag (and, optionally, header and lib locations): perl Makefile.PL \ --with-sasl2 \ --sasl2-include=/path/to/sasl2/include \ --sasl2-lib=/path/to/sasl2/lib When enabled, `Net::ZooKeeper->new(...)` admits a new key, `sasl_options`, which can be used to automatically authenticate with the server during connections (including reconnects). Some of the preparatory work for this contribution, which is available in the other commits, may also fix [ZOOKEEPER-3303](https://issues.apache.org/jira/browse/ZOOKEEPER-3303), which is about compilation issues. The resulting patch builds with GCC 8.3.0 in a (moderately) "hardened" environment; it also passes all enabled tests. Author: Damien Diederen Reviewers: Norbert Kalmar Closes #1243 from ztzg/ZOOKEEPER-3714-zkperl-sasl-support --- .../zookeeper-contrib-zkperl/Makefile.PL | 36 ++++-- .../zookeeper-contrib-zkperl/README | 21 ++++ .../zookeeper-contrib-zkperl/ZooKeeper.pm | 30 ++++- .../zookeeper-contrib-zkperl/ZooKeeper.xs | 70 ++++++++++- .../zookeeper-contrib-zkperl/t/10_invalid.t | 4 +- .../zookeeper-contrib-zkperl/t/30_connect.t | 8 ++ .../zookeeper-contrib-zkperl/t/35_log.t | 7 +- .../zookeeper-contrib-zkperl/t/50_access.t | 10 +- .../zookeeper-contrib-zkperl/t/70_sasl.t | 110 ++++++++++++++++++ 9 files changed, 272 insertions(+), 24 deletions(-) create mode 100644 zookeeper-contrib/zookeeper-contrib-zkperl/t/70_sasl.t diff --git a/zookeeper-contrib/zookeeper-contrib-zkperl/Makefile.PL b/zookeeper-contrib/zookeeper-contrib-zkperl/Makefile.PL index 9a0996dddcf..0c7486f9e29 100644 --- a/zookeeper-contrib/zookeeper-contrib-zkperl/Makefile.PL +++ b/zookeeper-contrib/zookeeper-contrib-zkperl/Makefile.PL @@ -28,21 +28,25 @@ my $ZOO_REQUIRED_VERSION = qr{^$ZOO_MAJOR_VERSION\.\d+.\d+$}ismx; my @zk_inc_paths; my @zk_lib_paths; +my $with_sasl2 = 0; +my @sasl2_inc_paths; +my @sasl2_lib_paths; + GetOptions( 'zookeeper-include=s' => \@zk_inc_paths, - 'zookeeper-lib=s' => \@zk_lib_paths + 'zookeeper-lib=s' => \@zk_lib_paths, + 'with-sasl2!' => \$with_sasl2, + 'sasl2-include=s' => \@sasl2_inc_paths, + 'sasl2-lib=s' => \@sasl2_lib_paths ); -my $zk_inc_paths = join(' ', map("-I$_", @zk_inc_paths)); -my $zk_lib_paths = join(' ', map("-L$_", @zk_lib_paths)); - -$zk_inc_paths .= ' ' unless ($zk_inc_paths eq ''); -$zk_lib_paths .= ' ' unless ($zk_lib_paths eq ''); +my $zk_inc = (join(' ', map("-I$_", @zk_inc_paths)) . ' -I.'); +my $zk_libs = (join(' ', map("-L$_", @zk_lib_paths)) . ' -lzookeeper_mt'); my $cc = $Config{'cc'}; my $check_file = 'build/check_zk_version'; -my $check_out = qx($cc $zk_inc_paths $zk_lib_paths -I. -o $check_file $check_file.c 2>&1); +my $check_out = qx($cc $zk_inc -o $check_file $check_file.c $zk_libs 2>&1); if ($?) { if ($check_out =~ /zookeeper_version\.h/) { @@ -63,11 +67,21 @@ elsif ($zk_ver !~ $ZOO_REQUIRED_VERSION) { warn "Net::ZooKeeper requires ZooKeeper 3.x, found $zk_ver!"; } +my @inc = ($zk_inc); +my @libs = ($zk_libs); +my %mmopt = (); + +if ($with_sasl2) { + push(@inc, join(' ', map("-I$_", @sasl2_inc_paths))); + push(@libs, join(' ', map("-L$_", @sasl2_lib_paths)) . ' -lsasl2'); + $mmopt{DEFINE} = '-DHAVE_CYRUS_SASL_H'; +} + WriteMakefile( - 'INC' => "$zk_inc_paths-I.", - 'LIBS' => [ "$zk_lib_paths-lzookeeper_mt" ], + 'INC' => join(' ', @inc), + 'LIBS' => \@libs, 'NAME' => 'Net::ZooKeeper', 'VERSION_FROM' => 'ZooKeeper.pm', - 'clean' => { 'FILES' => 'build/check_zk_version.o' } + 'clean' => { 'FILES' => 'build/check_zk_version.o' }, + %mmopt, ); - diff --git a/zookeeper-contrib/zookeeper-contrib-zkperl/README b/zookeeper-contrib/zookeeper-contrib-zkperl/README index bbe2a0d8f3b..ac13481529c 100644 --- a/zookeeper-contrib/zookeeper-contrib-zkperl/README +++ b/zookeeper-contrib/zookeeper-contrib-zkperl/README @@ -32,6 +32,15 @@ ZooKeeper C include files. The path supplied to the --zookeeper-lib option should identify the directory that contains the libzookeeper_mt library. +If the C client supports Cyrus SASL (ZOOKEEPER-1112), it can also be +enabled in the Perl binding by passing a --with-sasl2 flag (and, +optionally, non-standard locations): + + perl Makefile.PL \ + --with-sasl2 \ + --sasl2-include=/path/to/sasl2/include \ + --sasl2-lib=/path/to/sasl2/lib + When running "make test", if no ZK_TEST_HOSTS environment variable is set, many tests will be skipped because no connection to a ZooKeeper server is available. To execute these tests, @@ -44,6 +53,18 @@ The tests expect to have full read/write/create/delete/admin ZooKeeper permissions under this path. If no ZK_TEST_PATH variable is defined, the root ZooKeeper path ("/") is used. +The ZK_TEST_SASL_OPTIONS environment variable, if defined, provides a +JSON-encoded map of SASL authentication options, enabling SASL tests. +E.g., + + { + "host": "zk-sasl-md5", + "mechlist": "DIGEST-MD5", + "service": "zookeeper", + "user": "bob", + "password_file": "bob.secret" + } + DEPENDENCIES Version 3.1.1 of ZooKeeper is required at a minimum. diff --git a/zookeeper-contrib/zookeeper-contrib-zkperl/ZooKeeper.pm b/zookeeper-contrib/zookeeper-contrib-zkperl/ZooKeeper.pm index 507f0298d16..b17c9704c17 100644 --- a/zookeeper-contrib/zookeeper-contrib-zkperl/ZooKeeper.pm +++ b/zookeeper-contrib/zookeeper-contrib-zkperl/ZooKeeper.pm @@ -688,7 +688,8 @@ The following methods are defined for the Net::ZooKeeper class. $zkh = Net::ZooKeeper->new('host1:7000,host2:7000'); $zkh = Net::ZooKeeper->new('host1:7000,host2:7000', 'session_timeout' => $session_timeout, - 'session_id' => $session_id); + 'session_id' => $session_id, + 'sasl_options' => $sasl_options); Creates a new Net::ZooKeeper handle object and attempts to connect to the one of the servers of the given ZooKeeper @@ -725,6 +726,33 @@ initial connection request; again, the actual timeout period to which the server agrees will be available subsequently as the value of the C attribute. +If a C<'sasl_options'> option is provided, it is used to automatically +SASL-authenticate with the server during connections (including +reconnects). Here is a brief description of the recognized keys; +please refer to the C client documentation for details: + +=over 5 + +=item service => VALUE + +=item host => VALUE + +=item mechlist => VALUE + +These map to the corresponding fields of C from the +library. + +=item user => VALUE + +=item realm => VALUE + +=item password_file => VALUE + +These map to the corresponding parameters of +C from the library. + +=back + Upon successful connection (i.e., after the success of a method which requires communication with the server), the C attribute will hold a short binary string which represents the diff --git a/zookeeper-contrib/zookeeper-contrib-zkperl/ZooKeeper.xs b/zookeeper-contrib/zookeeper-contrib-zkperl/ZooKeeper.xs index 4b6067b1024..1fd178aba60 100644 --- a/zookeeper-contrib/zookeeper-contrib-zkperl/ZooKeeper.xs +++ b/zookeeper-contrib/zookeeper-contrib-zkperl/ZooKeeper.xs @@ -28,7 +28,9 @@ #include /* CHAR_BIT */ #include /* gettimeofday() */ +#define THREADED #include +#undef THREADED #include "build/check_zk_version.h" @@ -758,6 +760,13 @@ zk_new(package, hosts, ...) char *hosts PREINIT: int recv_timeout = DEFAULT_RECV_TIMEOUT_MSEC; +#ifdef HAVE_CYRUS_SASL_H + zoo_sasl_params_t sasl_params = { 0 }; + const char *sasl_user = NULL; + const char *sasl_realm = NULL; + const char *sasl_password_file = NULL; + int use_sasl = 0; +#endif /* HAVE_CYRUS_SASL_H */ const clientid_t *client_id = NULL; zk_t *zk; zk_handle_t *handle; @@ -792,12 +801,61 @@ zk_new(package, hosts, ...) Perl_croak(aTHX_ "invalid session ID"); } } +#ifdef HAVE_CYRUS_SASL_H + else if (strcaseEQ(key, "sasl_options")) { + SV *hash_sv = ST(i + 1); + HV *hash; + char *key; + I32 key_length; + SV *value; + + if (!SvROK(hash_sv) || SvTYPE(SvRV(hash_sv)) != SVt_PVHV) { + Perl_croak(aTHX_ "sasl_options requires a hash reference"); + } + + hash = (HV *)SvRV(hash_sv); + hv_iterinit(hash); + while ((value = hv_iternextsv(hash, &key, &key_length))) { + if (strcaseEQ(key, "service")) { + sasl_params.service = SvPV_nolen(value); + } + else if (strcaseEQ(key, "host")) { + sasl_params.host = SvPV_nolen(value); + } + else if (strcaseEQ(key, "mechlist")) { + sasl_params.mechlist = SvPV_nolen(value); + } + else if (strcaseEQ(key, "user")) { + sasl_user = SvPV_nolen(value); + } + else if (strcaseEQ(key, "realm")) { + sasl_realm = SvPV_nolen(value); + } + else if (strcaseEQ(key, "password_file")) { + sasl_password_file = SvPV_nolen(value); + } + } + use_sasl = 1; + } +#endif /* HAVE_CYRUS_SASL_H */ } Newxz(zk, 1, zk_t); +#ifdef HAVE_CYRUS_SASL_H + if (use_sasl) { + /* KLUDGE: Leaks a reference count. Authen::SASL::XS does + the same, though. TODO(ddiederen): Fix. */ + sasl_client_init(NULL); + sasl_params.callbacks = zoo_sasl_make_basic_callbacks(sasl_user, + sasl_realm, sasl_password_file); + } + zk->handle = zookeeper_init_sasl(hosts, NULL, recv_timeout, + client_id, NULL, 0, NULL, use_sasl ? &sasl_params : NULL); +#else zk->handle = zookeeper_init(hosts, NULL, recv_timeout, - client_id, NULL, 0); + client_id, NULL, 0); +#endif /* HAVE_CYRUS_SASL_H */ if (!zk->handle) { Safefree(zk); @@ -1203,7 +1261,7 @@ zk_add_auth(zkh, scheme, cert) zk->last_errno = 0; if (cert_len > PERL_INT_MAX) { - Perl_croak(aTHX_ "invalid certificate length: %u", cert_len); + Perl_croak(aTHX_ "invalid certificate length: %zu", cert_len); } watch = _zk_create_watch(aTHX); @@ -1283,7 +1341,7 @@ zk_create(zkh, path, buf, ...) } if (buf_len > PERL_INT_MAX) { - Perl_croak(aTHX_ "invalid data length: %u", buf_len); + Perl_croak(aTHX_ "invalid data length: %zu", buf_len); } path_buf_len = zk->path_buf_len; @@ -1318,7 +1376,7 @@ zk_create(zkh, path, buf, ...) err = _zk_fill_acl(aTHX_ acl_arr, &acl); if (err) { - Perl_croak(aTHX_ err); + Perl_croak(aTHX_ "%s", err); } } } @@ -1757,7 +1815,7 @@ zk_set(zkh, path, buf, ...) } if (buf_len > PERL_INT_MAX) { - Perl_croak(aTHX_ "invalid data length: %u", buf_len); + Perl_croak(aTHX_ "invalid data length: %zu", buf_len); } for (i = 3; i < items; i += 2) { @@ -1920,7 +1978,7 @@ zk_set_acl(zkh, path, acl_arr, ...) err = _zk_fill_acl(aTHX_ acl_arr, &acl); if (err) { - Perl_croak(aTHX_ err); + Perl_croak(aTHX_ "%s", err); } for (i = 3; i < items; i += 2) { diff --git a/zookeeper-contrib/zookeeper-contrib-zkperl/t/10_invalid.t b/zookeeper-contrib/zookeeper-contrib-zkperl/t/10_invalid.t index 5e080b64c7d..c1a118e008c 100644 --- a/zookeeper-contrib/zookeeper-contrib-zkperl/t/10_invalid.t +++ b/zookeeper-contrib/zookeeper-contrib-zkperl/t/10_invalid.t @@ -527,13 +527,13 @@ like($@, qr/Usage: Net::ZooKeeper::set_acl\(zkh, path, acl_arr, \.\.\.\)/, eval { $zkh->set_acl($node_path, 'foo'); }; -like($@, qr/acl_arr is not an array reference/, +like($@, qr/acl_arr is not an array reference/i, 'set_acl(): invalid ACL array reference'); eval { $zkh->set_acl($node_path, {}); }; -like($@, qr/acl_arr is not an array reference/, +like($@, qr/acl_arr is not an array reference/i, 'set_acl(): invalid ACL array reference to hash'); eval { diff --git a/zookeeper-contrib/zookeeper-contrib-zkperl/t/30_connect.t b/zookeeper-contrib/zookeeper-contrib-zkperl/t/30_connect.t index c2b68bb4e5f..4745e427d14 100644 --- a/zookeeper-contrib/zookeeper-contrib-zkperl/t/30_connect.t +++ b/zookeeper-contrib/zookeeper-contrib-zkperl/t/30_connect.t @@ -176,6 +176,13 @@ SKIP: { ## NOTE: to test re-connections with saved session IDs we create a second ## connection with the same ID while the first is still active; ## this is bad practice in normal usage + ## + ## Test disabled because it breaks with current ZooKeeper servers: + ## $zkh1's connection gets closed as soon as $zkh2 connects, which + ## causes it to reconnect, which kills $zkh2's connection, etc. + ## TODO: figure out a way to test this. + SKIP: { + skip 'does not work with current ZK servers', 4; my $zkh2 = Net::ZooKeeper->new($hosts, 'session_id' => $session_id1, @@ -198,5 +205,6 @@ SKIP: { and $session_id2 eq $session_id1), 'FETCH(): reconnect with session ID'); } + } } diff --git a/zookeeper-contrib/zookeeper-contrib-zkperl/t/35_log.t b/zookeeper-contrib/zookeeper-contrib-zkperl/t/35_log.t index 92821afc1f3..cb7f7dbc538 100644 --- a/zookeeper-contrib/zookeeper-contrib-zkperl/t/35_log.t +++ b/zookeeper-contrib/zookeeper-contrib-zkperl/t/35_log.t @@ -31,7 +31,7 @@ my($hosts, $root_path, $node_path) = zk_test_setup(0); my $zkh = Net::ZooKeeper->new($hosts); -Net::ZooKeeper::set_log_level(ZOO_LOG_LEVEL_INFO); +Net::ZooKeeper::set_log_level(ZOO_LOG_LEVEL_DEBUG); SKIP: { skip 'no valid handle', 2 unless (defined($zkh)); @@ -45,6 +45,7 @@ SKIP: { my $old_select = select(STDERR); $| = 1; + $/ = undef; # slurp mode. select($old_select); } else { @@ -63,7 +64,7 @@ SKIP: { skip 'no seek on stderr', 1 unless (seek(STDERR, 0, 0)); my $log = ; - like($log, qr/ZOO_/, + like($log, qr/ZOO_.*exists/, 'exists(): generated log message'); } @@ -75,7 +76,7 @@ SKIP: { skip 'no seek on stderr', 1 unless (seek(STDERR, 0, 0)); my $log = ; - like($log, qr/ZOO_/, + like($log, qr/ZOO_.*close/, 'DESTROY(): generated log message'); } diff --git a/zookeeper-contrib/zookeeper-contrib-zkperl/t/50_access.t b/zookeeper-contrib/zookeeper-contrib-zkperl/t/50_access.t index ef61ed6688f..71eb17c883b 100644 --- a/zookeeper-contrib/zookeeper-contrib-zkperl/t/50_access.t +++ b/zookeeper-contrib/zookeeper-contrib-zkperl/t/50_access.t @@ -18,6 +18,7 @@ use File::Spec; use Test::More tests => 40; +use Storable qw(dclone); BEGIN { use_ok('Net::ZooKeeper', qw(:all)) }; @@ -163,9 +164,16 @@ SKIP: { $! eq ''), 'get_acl(): undef returned for non-extant node'); + # The test is not running as ADMIN, which means that the server + # returns "redacted" ACLs (see ZOOKEEPER-1392 and OpCode.getACL in + # FinalRequestProcessor). We must do the same for the comparison + # to succeed. + my $redacted_digest_acl = dclone($digest_acl); + $redacted_digest_acl->[1]->{id} =~ s/:.*/:x/; + @acl = ('abc'); @acl = $zkh->get_acl($acl_node_path); - is_deeply(\@acl, $digest_acl, + is_deeply(\@acl, $redacted_digest_acl, 'get_acl(): retrieved digest ACL'); my $stat = $zkh->stat(); diff --git a/zookeeper-contrib/zookeeper-contrib-zkperl/t/70_sasl.t b/zookeeper-contrib/zookeeper-contrib-zkperl/t/70_sasl.t new file mode 100644 index 00000000000..9de379a7a4d --- /dev/null +++ b/zookeeper-contrib/zookeeper-contrib-zkperl/t/70_sasl.t @@ -0,0 +1,110 @@ +# Net::ZooKeeper - Perl extension for Apache ZooKeeper +# +# 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. + +use File::Spec; +use Test::More tests => 7; +use JSON::PP qw(decode_json); + +BEGIN { use_ok('Net::ZooKeeper', qw(:all)) }; + + +my $test_dir; +(undef, $test_dir, undef) = File::Spec->splitpath($0); +require File::Spec->catfile($test_dir, 'util.pl'); + +my($hosts, $root_path, $node_path) = zk_test_setup(0); + +my $sasl_options = $ENV{'ZK_TEST_SASL_OPTIONS'}; +if (defined($sasl_options)) { + $sasl_options = decode_json($sasl_options); +} + +SKIP: { + skip 'no sasl_options', 6 unless defined($sasl_options); + + my $zkh = Net::ZooKeeper->new($hosts, + 'sasl_options' => $sasl_options); + + my $path = $zkh->create($node_path, 'foo', + 'acl' => ZOO_OPEN_ACL_UNSAFE) if (defined($zkh)); + + skip 'no connection to ZooKeeper', 36 unless + (defined($path) and $path eq $node_path); + + ## _zk_acl_constant() + + my $acl_node_path = "$node_path/a1"; + + my $sasl_acl = [ + { + 'perms' => ZOO_PERM_READ, + 'scheme' => 'world', + 'id' => 'anyone' + }, + { + 'perms' => ZOO_PERM_ALL, + 'scheme' => 'sasl', + 'id' => $sasl_options->{user} + } + ]; + + $path = $zkh->create($acl_node_path, 'foo', 'acl' => $sasl_acl); + is($path, $acl_node_path, + 'create(): created node with SASL ACL'); + + + ## get_acl() + + @acl = ('abc'); + @acl = $zkh->get_acl($acl_node_path); + is_deeply(\@acl, $sasl_acl, + 'get_acl(): retrieved SASL ACL'); + + SKIP: { + my $zkh2 = Net::ZooKeeper->new($hosts); + + my $ret = $zkh->exists($root_path) if (defined($zkh)); + + skip 'no connection to ZooKeeper', 1 unless + (defined($ret) and $ret); + + my $node = $zkh2->get($acl_node_path); + is($node, 'foo', + 'get(): retrieved node value with world ACL'); + + $ret = $zkh2->set($acl_node_path, 'bar'); + ok((!$ret and $zkh2->get_error() == ZNOAUTH and $! eq ''), + 'set(): node value unchanged if no auth'); + } + + my $ret = $zkh->set($acl_node_path, 'bar'); + ok($ret, + 'set(): set node with SASL ACL'); + + my $node = $zkh->get($acl_node_path); + is($node, 'bar', + 'get(): retrieved new node value with SASL ACL'); + + $ret = $zkh->delete($acl_node_path); + diag(sprintf('unable to delete node %s: %d, %s', + $acl_node_path, $zkh->get_error(), $!)) unless ($ret); + + $ret = $zkh->delete($node_path); + diag(sprintf('unable to delete node %s: %d, %s', + $node_path, $zkh->get_error(), $!)) unless ($ret); +} From f9a08032c5cdf6ca4dc4649992436207576fbe9e Mon Sep 17 00:00:00 2001 From: Christopher Tubbs Date: Thu, 9 Apr 2020 10:59:02 +0000 Subject: [PATCH 016/591] ZOOKEEPER-3786: Simplify version generation Simplify generation of VersionInfoMain.java and Info.java by using maven-resource-plugin's built-in resource filtering at build time. This eliminates the need to use VerGen to generate java source files during the build. Also make other slight pom improvements: 1. Remove trailing tab character in Ted's name in pom.xml 2. Simplify spotbugs skipping in contrib pom.xml 3. Add m2e configuration for build plugin executions to be ignored by Eclipse, for developers (like me) using Eclipse IDE 4. Format build time in a more international-friendly and less ambiguous way (year first, then month, then day, using UTC instead of GMT) Link to issue: https://issues.apache.org/jira/browse/ZOOKEEPER-3786 Author: Christopher Tubbs Reviewers: Enrico Olivelli , Mate Szalay-Beko Closes #1310 from ctubbsii/use-resource-filtering-for-version-info --- pom.xml | 2 +- zookeeper-contrib/pom.xml | 13 +- zookeeper-server/pom.xml | 55 ++--- .../org/apache/zookeeper/version/Info.java | 29 +++ .../zookeeper/version/VersionInfoMain.java | 25 +++ .../apache/zookeeper/version/util/VerGen.java | 202 ------------------ .../java/org/apache/zookeeper/VerGenTest.java | 71 ------ 7 files changed, 79 insertions(+), 318 deletions(-) create mode 100644 zookeeper-server/src/main/java-filtered/org/apache/zookeeper/version/Info.java create mode 100644 zookeeper-server/src/main/java-filtered/org/apache/zookeeper/version/VersionInfoMain.java delete mode 100644 zookeeper-server/src/main/java/org/apache/zookeeper/version/util/VerGen.java delete mode 100644 zookeeper-server/src/test/java/org/apache/zookeeper/VerGenTest.java diff --git a/pom.xml b/pom.xml index 6fad123737f..79e88df6ca9 100755 --- a/pom.xml +++ b/pom.xml @@ -117,7 +117,7 @@ tdunning - Ted Dunning + Ted Dunning tdunning@apache.org -8 diff --git a/zookeeper-contrib/pom.xml b/zookeeper-contrib/pom.xml index 228a520ed4b..0e2959d3be0 100755 --- a/zookeeper-contrib/pom.xml +++ b/zookeeper-contrib/pom.xml @@ -49,18 +49,13 @@ + + true + + - - com.github.spotbugs - spotbugs-maven-plugin - 3.1.9 - - - true - - org.apache.maven.plugins maven-compiler-plugin diff --git a/zookeeper-server/pom.xml b/zookeeper-server/pom.xml index 21c30eae87f..117a77b8ec7 100755 --- a/zookeeper-server/pom.xml +++ b/zookeeper-server/pom.xml @@ -166,22 +166,29 @@ - + org.codehaus.mojo build-helper-maven-plugin + tbuild-time timestamp-property build.time - MM/dd/yyyy HH:mm zz + yyyy-MM-dd HH:mm zz en_US - GMT + UTC + + parse-version + + parse-version + + generate-sources @@ -197,44 +204,22 @@ org.apache.maven.plugins - maven-compiler-plugin + maven-resources-plugin - pre-compile-vergen - generate-sources - - - org/apache/zookeeper/version/**/*.java - - + prepare-filtered-java-source - compile + copy-resources - - - - - org.codehaus.mojo - exec-maven-plugin - - - generate-version-info generate-sources - - exec - - ${project.basedir}/src/main/java/ - java - - -classpath - - org.apache.zookeeper.version.util.VerGen - ${project.version} - ${mvngit.commit.id} - ${build.time} - ${project.basedir}/target/generated-sources/java - + ${project.build.directory}/generated-sources/java + + + src/main/java-filtered + true + + diff --git a/zookeeper-server/src/main/java-filtered/org/apache/zookeeper/version/Info.java b/zookeeper-server/src/main/java-filtered/org/apache/zookeeper/version/Info.java new file mode 100644 index 00000000000..5570eccf800 --- /dev/null +++ b/zookeeper-server/src/main/java-filtered/org/apache/zookeeper/version/Info.java @@ -0,0 +1,29 @@ +/* + * 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.version; + +public interface Info { + int MAJOR=${parsedVersion.majorVersion}; + int MINOR=${parsedVersion.minorVersion}; + int MICRO=${parsedVersion.incrementalVersion}; + String QUALIFIER="${parsedVersion.qualifier}".isEmpty() ? null : "${parsedVersion.qualifier}"; + int REVISION=-1; //@deprecated, please use REVISION_HASH + String REVISION_HASH="${mvngit.commit.id}"; + String BUILD_DATE="${build.time}"; +} diff --git a/zookeeper-server/src/main/java-filtered/org/apache/zookeeper/version/VersionInfoMain.java b/zookeeper-server/src/main/java-filtered/org/apache/zookeeper/version/VersionInfoMain.java new file mode 100644 index 00000000000..b7d0054bd72 --- /dev/null +++ b/zookeeper-server/src/main/java-filtered/org/apache/zookeeper/version/VersionInfoMain.java @@ -0,0 +1,25 @@ +/* + * 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.version; + +public class VersionInfoMain implements org.apache.zookeeper.version.Info { + public static void main(String[] args) { + System.out.println("Apache ZooKeeper, version ${project.version} ${build.time}"); + } +} diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/version/util/VerGen.java b/zookeeper-server/src/main/java/org/apache/zookeeper/version/util/VerGen.java deleted file mode 100644 index 9bb699f090d..00000000000 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/version/util/VerGen.java +++ /dev/null @@ -1,202 +0,0 @@ -/* - * 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.version.util; - -import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; -import java.io.File; -import java.io.FileWriter; -import java.io.IOException; -import java.util.regex.Matcher; -import java.util.regex.Pattern; -import org.apache.zookeeper.server.ExitCode; - -@SuppressFBWarnings("DM_EXIT") -public class VerGen { - - private static final String PACKAGE_NAME = "org.apache.zookeeper.version"; - private static final String VERSION_CLASS_NAME = "VersionInfoMain"; - private static final String VERSION_INTERFACE_NAME = "Info"; - - static void printUsage() { - System.out.print("Usage:\tjava -cp org.apache.zookeeper." - + "version.util.VerGen maj.min.micro[-qualifier] rev buildDate outputDirectory"); - System.exit(ExitCode.UNEXPECTED_ERROR.getValue()); - } - - public static void generateFile(File outputDir, Version version, String rev, String buildDate) { - String path = PACKAGE_NAME.replaceAll("\\.", "/"); - File pkgdir = new File(outputDir, path); - if (!pkgdir.exists()) { - // create the pkg directory - boolean ret = pkgdir.mkdirs(); - if (!ret) { - System.out.println("Cannnot create directory: " + path); - System.exit(ExitCode.UNEXPECTED_ERROR.getValue()); - } - } else if (!pkgdir.isDirectory()) { - // not a directory - System.out.println(path + " is not a directory."); - System.exit(ExitCode.UNEXPECTED_ERROR.getValue()); - } - - try (FileWriter w = new FileWriter(new File(pkgdir, VERSION_INTERFACE_NAME + ".java"))) { - w.write("// Do not edit!\n// File generated by org.apache.zookeeper" + ".version.util.VerGen.\n"); - w.write("/**\n"); - w.write("* Licensed to the Apache Software Foundation (ASF) under one\n"); - w.write("* or more contributor license agreements. See the NOTICE file\n"); - w.write("* distributed with this work for additional information\n"); - w.write("* regarding copyright ownership. The ASF licenses this file\n"); - w.write("* to you under the Apache License, Version 2.0 (the\n"); - w.write("* \"License\"); you may not use this file except in compliance\n"); - w.write("* with the License. You may obtain a copy of the License at\n"); - w.write("*\n"); - w.write("* http://www.apache.org/licenses/LICENSE-2.0\n"); - w.write("*\n"); - w.write("* Unless required by applicable law or agreed to in writing, software\n"); - w.write("* distributed under the License is distributed on an \"AS IS\" BASIS,\n"); - w.write("* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n"); - w.write("* See the License for the specific language governing permissions and\n"); - w.write("* limitations under the License.\n"); - w.write("*/\n"); - w.write("\n"); - w.write("package " + PACKAGE_NAME + ";\n\n"); - w.write("public interface " + VERSION_INTERFACE_NAME + " {\n"); - w.write(" int MAJOR=" + version.maj + ";\n"); - w.write(" int MINOR=" + version.min + ";\n"); - w.write(" int MICRO=" + version.micro + ";\n"); - w.write(" String QUALIFIER=" + (version.qualifier == null ? "\"\"" : "\"" + version.qualifier + "\"") + ";\n"); - if (rev.equals("-1")) { - System.out.println("Unknown REVISION number, using " + rev); - } - w.write(" int REVISION=-1; //@deprecated, please use REVISION_HASH\n"); - w.write(" String REVISION_HASH=\"" + rev + "\";\n"); - w.write(" String BUILD_DATE=\"" + buildDate + "\";\n"); - w.write("}\n"); - } catch (IOException e) { - System.out.println("Unable to generate version.Info file: " + e.getMessage()); - System.exit(ExitCode.UNEXPECTED_ERROR.getValue()); - } - - // Generate a main class to display version data - // that can be exec'd in zkServer.sh - try (FileWriter w = new FileWriter(new File(pkgdir, VERSION_CLASS_NAME + ".java"))) { - w.write("// Do not edit!\n// File generated by org.apache.zookeeper" + ".version.util.VerGen.\n"); - w.write("/**\n"); - w.write("* Licensed to the Apache Software Foundation (ASF) under one\n"); - w.write("* or more contributor license agreements. See the NOTICE file\n"); - w.write("* distributed with this work for additional information\n"); - w.write("* regarding copyright ownership. The ASF licenses this file\n"); - w.write("* to you under the Apache License, Version 2.0 (the\n"); - w.write("* \"License\"); you may not use this file except in compliance\n"); - w.write("* with the License. You may obtain a copy of the License at\n"); - w.write("*\n"); - w.write("* http://www.apache.org/licenses/LICENSE-2.0\n"); - w.write("*\n"); - w.write("* Unless required by applicable law or agreed to in writing, software\n"); - w.write("* distributed under the License is distributed on an \"AS IS\" BASIS,\n"); - w.write("* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n"); - w.write("* See the License for the specific language governing permissions and\n"); - w.write("* limitations under the License.\n"); - w.write("*/\n"); - w.write("\n"); - w.write("package " + PACKAGE_NAME + ";\n\n"); - w.write("public class " + VERSION_CLASS_NAME + " implements " + PACKAGE_NAME + ".Info {\n"); - w.write(" public static void main(String[] args) {\n"); - w.write(" final String VER_STRING = MAJOR + \".\" + MINOR + \".\" + MICRO +"); - w.write(" (QUALIFIER == null ? \"\" : \"-\" + QUALIFIER) + \" \" +"); - w.write(" BUILD_DATE;" + "\n"); - w.write(" System.out.println(\"Apache ZooKeeper, version \" + VER_STRING);\n"); - w.write(" }\n"); - w.write("}\n"); - } catch (IOException e) { - System.out.println("Unable to generate version.VersionInfoMain file: " + e.getMessage()); - System.exit(ExitCode.UNEXPECTED_ERROR.getValue()); - } - } - - public static class Version { - - public int maj; - public int min; - public int micro; - public String qualifier; - - } - - public static Version parseVersionString(String input) { - Version result = new Version(); - - Pattern p = Pattern.compile("^(\\d+)\\.(\\d+)\\.(\\d+)((\\.\\d+)*)(-(.+))?$"); - Matcher m = p.matcher(input); - - if (!m.matches()) { - return null; - } - result.maj = Integer.parseInt(m.group(1)); - result.min = Integer.parseInt(m.group(2)); - result.micro = Integer.parseInt(m.group(3)); - if (m.groupCount() == 7) { - result.qualifier = m.group(7); - } else { - result.qualifier = null; - } - return result; - } - - /** - * Emits a org.apache.zookeeper.version.Info interface file with version and - * revision information constants set to the values passed in as command - * line parameters. The file is created in the current directory.
    - * Usage: java org.apache.zookeeper.version.util.VerGen maj.min.micro[-qualifier] - * rev buildDate - * - * @param args - *

      - *
    • maj - major version number - *
    • min - minor version number - *
    • micro - minor minor version number - *
    • qualifier - optional qualifier (dash followed by qualifier text) - *
    • rev - current Git revision number - *
    • buildDate - date the build - *
    - */ - public static void main(String[] args) { - if (args.length != 4) { - printUsage(); - } - try { - Version version = parseVersionString(args[0]); - if (version == null) { - System.err.println("Invalid version number format, must be \"x.y.z(-.*)?\""); - System.exit(ExitCode.UNEXPECTED_ERROR.getValue()); - } - String rev = args[1]; - if (rev == null || rev.trim().isEmpty()) { - rev = "-1"; - } else { - rev = rev.trim(); - } - generateFile(new File(args[3]), version, rev, args[2]); - } catch (NumberFormatException e) { - System.err.println("All version-related parameters must be valid integers!"); - throw e; - } - } - -} diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/VerGenTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/VerGenTest.java deleted file mode 100644 index 1542df0caa5..00000000000 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/VerGenTest.java +++ /dev/null @@ -1,71 +0,0 @@ -/* - * 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; - -import static org.junit.Assert.assertEquals; -import java.io.File; -import java.util.Arrays; -import java.util.Collection; -import org.apache.zookeeper.test.ClientBase; -import org.apache.zookeeper.version.util.VerGen; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; -import org.junit.runners.Parameterized.Parameters; - -/** - * Test VerGen, used during the build. - * - */ -@RunWith(Parameterized.class) -@Parameterized.UseParametersRunnerFactory(ZKParameterized.RunnerFactory.class) -public class VerGenTest extends ZKTestCase { - - @Parameters - public static Collection data() { - return Arrays.asList(new Object[][]{{"1.2.3", new Object[]{1, 2, 3, null}}, {"1.2.3-dev", new Object[]{1, 2, 3, "dev"}}, {"1.2.3-SNAPSHOT", new Object[]{1, 2, 3, "SNAPSHOT"}}, {"1.2.3-SNAPSHOT", new Object[]{1, 2, 3, "SNAPSHOT"}}, {"1.2.3-foo-bar+123", new Object[]{1, 2, 3, "foo-bar+123"}}, {"1.2.3.4.5-SNAPSHOT", new Object[]{1, 2, 3, "SNAPSHOT"}}, {"1.2.3.4.5-foo-bar+123", new Object[]{1, 2, 3, "foo-bar+123"}}}); - } - - private String input; - - private Object[] expected; - - public VerGenTest(String input, Object[] expected) { - this.input = input; - this.expected = expected; - } - - @Test - public void testParser() { - VerGen.Version v = VerGen.parseVersionString(input); - assertEquals(expected[0], v.maj); - assertEquals(expected[1], v.min); - assertEquals(expected[2], v.micro); - assertEquals(expected[3], v.qualifier); - } - - @Test - public void testGenFile() throws Exception { - VerGen.Version v = VerGen.parseVersionString(input); - File outputDir = ClientBase.createTmpDir(); - VerGen.generateFile(outputDir, v, "1", "Nov1"); - ClientBase.recursiveDelete(outputDir); - } - -} From 726f6843eba75bb3673284b52d8ca3801c5431d3 Mon Sep 17 00:00:00 2001 From: Vladislav Tyulbashev Date: Sat, 11 Apr 2020 06:19:04 +0000 Subject: [PATCH 017/591] ZOOKEEPER-3726: invalid ipv6 address comparison in C client Fix for https://issues.apache.org/jira/browse/ZOOKEEPER-3726 sockaddr_storage can contain ipv4 or ipv6 address. If address is ipv6, then we need to compare more bytes. In this PR correct comparison of sockaddr_storage was added. Author: Vladislav Tyulbashev Reviewers: Norbert Kalmar , Mate Szalay-Beko Closes #1252 from vtyulb/ZOOKEEPER-3726 --- .../zookeeper-client-c/src/addrvec.c | 22 ++++- .../zookeeper-client-c/tests/TestReconfig.cc | 83 +++++++++++++++++++ 2 files changed, 103 insertions(+), 2 deletions(-) diff --git a/zookeeper-client/zookeeper-client-c/src/addrvec.c b/zookeeper-client/zookeeper-client-c/src/addrvec.c index fdfb68d34fd..b7f244e7afb 100644 --- a/zookeeper-client/zookeeper-client-c/src/addrvec.c +++ b/zookeeper-client/zookeeper-client-c/src/addrvec.c @@ -126,8 +126,26 @@ int addrvec_contains(const addrvec_t *avec, const struct sockaddr_storage *addr) for (i = 0; i < avec->count; i++) { - if(memcmp(&avec->data[i], addr, INET_ADDRSTRLEN) == 0) - return 1; + if (avec->data[i].ss_family != addr->ss_family) + continue; + switch (addr->ss_family) { + case AF_INET: + if (memcmp(&((struct sockaddr_in*)&avec->data[i])->sin_addr, + &((struct sockaddr_in*)addr)->sin_addr, + sizeof(struct in_addr)) == 0) + return 1; + break; +#ifdef AF_INET6 + case AF_INET6: + if (memcmp(&((struct sockaddr_in6*)&avec->data[i])->sin6_addr, + &((struct sockaddr_in6*)addr)->sin6_addr, + sizeof(struct in6_addr)) == 0) + return 1; + break; +#endif + default: + break; + } } return 0; diff --git a/zookeeper-client/zookeeper-client-c/tests/TestReconfig.cc b/zookeeper-client/zookeeper-client-c/tests/TestReconfig.cc index 317ffcddfed..22f87e94491 100644 --- a/zookeeper-client/zookeeper-client-c/tests/TestReconfig.cc +++ b/zookeeper-client/zookeeper-client-c/tests/TestReconfig.cc @@ -26,6 +26,10 @@ #include #include +extern "C" { +#include +} + #include "Util.h" #include "LibCMocks.h" #include "ZKMocks.h" @@ -218,6 +222,10 @@ class Zookeeper_reconfig : public CPPUNIT_NS::TestFixture CPPUNIT_TEST(testcycleNextServer); CPPUNIT_TEST(testMigrateOrNot); CPPUNIT_TEST(testMigrationCycle); + CPPUNIT_TEST(testAddrVecContainsIPv4); +#ifdef AF_INET6 + CPPUNIT_TEST(testAddrVecContainsIPv6); +#endif // In threaded mode each 'create' is a thread -- it's not practical to create // 10,000 threads to test load balancing. The load balancing code can easily @@ -609,6 +617,81 @@ class Zookeeper_reconfig : public CPPUNIT_NS::TestFixture numServers = 9; updateAllClientsAndServers(numServers); } + + /** + * This tests that client can detect server's ipv4 address change. + * + * (1) We generate some address and put in addr, which saddr point to + * (2) Add all addresses that differ by one bit from the source + * (3) Add same address, but set ipv6 protocol + * (4) Ensure, that our address is not equal to any of generated, + * and that it equals to itself + */ + void testAddrVecContainsIPv4() { + addrvec_t vec; + addrvec_init(&vec); + + sockaddr_storage addr; + sockaddr_in* saddr = (sockaddr_in*)&addr; + saddr->sin_family = AF_INET; + saddr->sin_port = htons((u_short)1234); + saddr->sin_addr.s_addr = INADDR_ANY; + + CPPUNIT_ASSERT(sizeof(saddr->sin_addr.s_addr) == 4); + + for (int i = 0; i < 32; i++) { + saddr->sin_addr.s_addr ^= (1 << i); + addrvec_append(&vec, &addr); + saddr->sin_addr.s_addr ^= (1 << i); + } + + saddr->sin_family = AF_INET6; + addrvec_append(&vec, &addr); + saddr->sin_family = AF_INET; + + CPPUNIT_ASSERT(!addrvec_contains(&vec, &addr)); + addrvec_append(&vec, &addr); + CPPUNIT_ASSERT(addrvec_contains(&vec, &addr)); + addrvec_free(&vec); + } + + /** + * This tests that client can detect server's ipv6 address change. + * + * Same logic as in previous testAddrVecContainsIPv4 method, + * but we keep in mind, that ipv6 is 128-bit long. + */ +#ifdef AF_INET6 + void testAddrVecContainsIPv6() { + addrvec_t vec; + addrvec_init(&vec); + + sockaddr_storage addr; + sockaddr_in6* saddr = (sockaddr_in6*)&addr; + saddr->sin6_family = AF_INET6; + saddr->sin6_port = htons((u_short)1234); + saddr->sin6_addr = in6addr_any; + + CPPUNIT_ASSERT(sizeof(saddr->sin6_addr.s6_addr) == 16); + + for (int i = 0; i < 16; i++) { + for (int j = 0; j < 8; j++) { + saddr->sin6_addr.s6_addr[i] ^= (1 << j); + addrvec_append(&vec, &addr); + saddr->sin6_addr.s6_addr[i] ^= (1 << j); + } + } + + saddr->sin6_family = AF_INET; + addrvec_append(&vec, &addr); + saddr->sin6_family = AF_INET6; + + CPPUNIT_ASSERT(!addrvec_contains(&vec, &addr)); + addrvec_append(&vec, &addr); + CPPUNIT_ASSERT(addrvec_contains(&vec, &addr)); + addrvec_free(&vec); + } +#endif }; CPPUNIT_TEST_SUITE_REGISTRATION(Zookeeper_reconfig); From 4d32f6cf39f76d606b436bd4c04a8d1bc9c60148 Mon Sep 17 00:00:00 2001 From: Michael Han Date: Sun, 12 Apr 2020 11:14:45 +0200 Subject: [PATCH 018/591] ZOOKEEPER-3793: Request throttling is broken when RequestThrottler is disabled or configured incorrectly.. When RequestThrottler is not enabled or is enabled but configured incorrectly, ZooKeeper server will stop throttling. This is a serious bug as without request throttling, it's fairly easy to overwhelm ZooKeeper which leads to all sorts of issues. This is a regression introduced in ZOOKEEPER-3243, where the total number of queued requests in request processing pipeline is not taking into consideration when deciding whether to throttle or not, or only taken into consideration conditionally based on RequestThrottler's configurations. We should make sure always taking into account the number of queued requests in request processing pipeline before making throttling decisions. Author: Michael Han Reviewers: Enrico Olivelli Closes #1316 from hanm/ZOOKEEPER-3793 --- .../zookeeper/server/ZooKeeperServer.java | 3 +- .../server/RequestThrottlerTest.java | 39 +++++++++++++++++++ 2 files changed, 41 insertions(+), 1 deletion(-) diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/ZooKeeperServer.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/ZooKeeperServer.java index 8205f741499..f37225edbf0 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/ZooKeeperServer.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/ZooKeeperServer.java @@ -1431,7 +1431,8 @@ public void processConnectRequest(ServerCnxn cnxn, ByteBuffer incomingBuffer) } public boolean shouldThrottle(long outStandingCount) { - if (getGlobalOutstandingLimit() < getInflight()) { + int globalOutstandingLimit = getGlobalOutstandingLimit(); + if (globalOutstandingLimit < getInflight() || globalOutstandingLimit < getInProcess()) { return outStandingCount > 0; } return false; diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/RequestThrottlerTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/RequestThrottlerTest.java index 3afe81c6029..3e1de555f0b 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/RequestThrottlerTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/RequestThrottlerTest.java @@ -47,6 +47,7 @@ public class RequestThrottlerTest extends ZKTestCase { private static final Logger LOG = LoggerFactory.getLogger(RequestThrottlerTest.class); private static String HOSTPORT = "127.0.0.1:" + PortAssignment.unique(); + private static String GLOBAL_OUTSTANDING_LIMIT = "1"; private static final int TOTAL_REQUESTS = 5; private static final int STALL_TIME = 5000; @@ -307,4 +308,42 @@ public void testLargeRequestThrottling() throws Exception { metrics = MetricsUtils.currentServerMetrics(); Assert.assertEquals(2, (long) metrics.get("stale_replies")); } + + @Test + public void testGlobalOutstandingRequestThrottlingWithRequestThrottlerDisabled() throws Exception { + try { + System.setProperty(ZooKeeperServer.GLOBAL_OUTSTANDING_LIMIT, GLOBAL_OUTSTANDING_LIMIT); + + ServerMetrics.getMetrics().resetAll(); + + // Here we disable RequestThrottler and let incoming requests queued at first request processor. + RequestThrottler.setMaxRequests(0); + resumeProcess = new CountDownLatch(1); + int totalRequests = 10; + submitted = new CountDownLatch(totalRequests); + + for (int i = 0; i < totalRequests; i++) { + zk.create("/request_throttle_test- " + i, ("/request_throttle_test- " + + i).getBytes(), ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT, (rc, path, ctx, name) -> { + }, null); + } + + submitted.await(5, TimeUnit.SECONDS); + + resumeProcess.countDown(); + + // We should start throttling instead of queuing more requests. + // + // We always allow up to GLOBAL_OUTSTANDING_LIMIT + 1 number of requests coming in request processing pipeline + // before throttling. For the next request, we will throttle by disabling receiving future requests but we still + // allow this single request coming in. So the total number of queued requests in processing pipeline would + // be GLOBAL_OUTSTANDING_LIMIT + 2. + assertEquals(Integer.parseInt(GLOBAL_OUTSTANDING_LIMIT) + 2, + (long) MetricsUtils.currentServerMetrics().get("prep_processor_request_queued")); + } catch (Exception e) { + throw e; + } finally { + System.clearProperty(ZooKeeperServer.GLOBAL_OUTSTANDING_LIMIT); + } + } } From 8a1e0b3643d85ffa362207880095e08f1dbc47da Mon Sep 17 00:00:00 2001 From: Patrick Hunt Date: Wed, 15 Apr 2020 11:44:17 +0200 Subject: [PATCH 019/591] ZOOKEEPER-3794: upgrade netty to address CVE-2020-11612 Latest owasp runs are flagging this. Simple upgrade to the current latest version of netty 4.1.x codeline. Change-Id: I3defe8a731a2bf22abbd154e7d51054cb62aaa64 Author: Patrick Hunt Reviewers: Enrico Olivelli Closes #1319 from phunt/ZOOKEEPER-3794 --- pom.xml | 2 +- ....Final.LICENSE.txt => netty-buffer-4.1.48.Final.LICENSE.txt} | 0 ...5.Final.LICENSE.txt => netty-codec-4.1.49.Final.LICENSE.txt} | 0 ....Final.LICENSE.txt => netty-common-4.1.48.Final.LICENSE.txt} | 0 ...Final.LICENSE.txt => netty-handler-4.1.48.Final.LICENSE.txt} | 0 ...inal.LICENSE.txt => netty-resolver-4.1.48.Final.LICENSE.txt} | 0 ...nal.LICENSE.txt => netty-transport-4.1.48.Final.LICENSE.txt} | 0 ...xt => netty-transport-native-epoll-4.1.48.Final.LICENSE.txt} | 0 ...netty-transport-native-unix-common-4.1.48.Final.LICENSE.txt} | 0 9 files changed, 1 insertion(+), 1 deletion(-) rename zookeeper-server/src/main/resources/lib/{netty-buffer-4.1.45.Final.LICENSE.txt => netty-buffer-4.1.48.Final.LICENSE.txt} (100%) rename zookeeper-server/src/main/resources/lib/{netty-codec-4.1.45.Final.LICENSE.txt => netty-codec-4.1.49.Final.LICENSE.txt} (100%) rename zookeeper-server/src/main/resources/lib/{netty-common-4.1.45.Final.LICENSE.txt => netty-common-4.1.48.Final.LICENSE.txt} (100%) rename zookeeper-server/src/main/resources/lib/{netty-handler-4.1.45.Final.LICENSE.txt => netty-handler-4.1.48.Final.LICENSE.txt} (100%) rename zookeeper-server/src/main/resources/lib/{netty-resolver-4.1.45.Final.LICENSE.txt => netty-resolver-4.1.48.Final.LICENSE.txt} (100%) rename zookeeper-server/src/main/resources/lib/{netty-transport-4.1.45.Final.LICENSE.txt => netty-transport-4.1.48.Final.LICENSE.txt} (100%) rename zookeeper-server/src/main/resources/lib/{netty-transport-native-epoll-4.1.45.Final.LICENSE.txt => netty-transport-native-epoll-4.1.48.Final.LICENSE.txt} (100%) rename zookeeper-server/src/main/resources/lib/{netty-transport-native-unix-common-4.1.45.Final.LICENSE.txt => netty-transport-native-unix-common-4.1.48.Final.LICENSE.txt} (100%) diff --git a/pom.xml b/pom.xml index 79e88df6ca9..e9e973facfe 100755 --- a/pom.xml +++ b/pom.xml @@ -353,7 +353,7 @@ 2.27.0 1.3 1.2 - 4.1.45.Final + 4.1.48.Final 9.4.24.v20191120 2.10.3 1.1.1 diff --git a/zookeeper-server/src/main/resources/lib/netty-buffer-4.1.45.Final.LICENSE.txt b/zookeeper-server/src/main/resources/lib/netty-buffer-4.1.48.Final.LICENSE.txt similarity index 100% rename from zookeeper-server/src/main/resources/lib/netty-buffer-4.1.45.Final.LICENSE.txt rename to zookeeper-server/src/main/resources/lib/netty-buffer-4.1.48.Final.LICENSE.txt diff --git a/zookeeper-server/src/main/resources/lib/netty-codec-4.1.45.Final.LICENSE.txt b/zookeeper-server/src/main/resources/lib/netty-codec-4.1.49.Final.LICENSE.txt similarity index 100% rename from zookeeper-server/src/main/resources/lib/netty-codec-4.1.45.Final.LICENSE.txt rename to zookeeper-server/src/main/resources/lib/netty-codec-4.1.49.Final.LICENSE.txt diff --git a/zookeeper-server/src/main/resources/lib/netty-common-4.1.45.Final.LICENSE.txt b/zookeeper-server/src/main/resources/lib/netty-common-4.1.48.Final.LICENSE.txt similarity index 100% rename from zookeeper-server/src/main/resources/lib/netty-common-4.1.45.Final.LICENSE.txt rename to zookeeper-server/src/main/resources/lib/netty-common-4.1.48.Final.LICENSE.txt diff --git a/zookeeper-server/src/main/resources/lib/netty-handler-4.1.45.Final.LICENSE.txt b/zookeeper-server/src/main/resources/lib/netty-handler-4.1.48.Final.LICENSE.txt similarity index 100% rename from zookeeper-server/src/main/resources/lib/netty-handler-4.1.45.Final.LICENSE.txt rename to zookeeper-server/src/main/resources/lib/netty-handler-4.1.48.Final.LICENSE.txt diff --git a/zookeeper-server/src/main/resources/lib/netty-resolver-4.1.45.Final.LICENSE.txt b/zookeeper-server/src/main/resources/lib/netty-resolver-4.1.48.Final.LICENSE.txt similarity index 100% rename from zookeeper-server/src/main/resources/lib/netty-resolver-4.1.45.Final.LICENSE.txt rename to zookeeper-server/src/main/resources/lib/netty-resolver-4.1.48.Final.LICENSE.txt diff --git a/zookeeper-server/src/main/resources/lib/netty-transport-4.1.45.Final.LICENSE.txt b/zookeeper-server/src/main/resources/lib/netty-transport-4.1.48.Final.LICENSE.txt similarity index 100% rename from zookeeper-server/src/main/resources/lib/netty-transport-4.1.45.Final.LICENSE.txt rename to zookeeper-server/src/main/resources/lib/netty-transport-4.1.48.Final.LICENSE.txt diff --git a/zookeeper-server/src/main/resources/lib/netty-transport-native-epoll-4.1.45.Final.LICENSE.txt b/zookeeper-server/src/main/resources/lib/netty-transport-native-epoll-4.1.48.Final.LICENSE.txt similarity index 100% rename from zookeeper-server/src/main/resources/lib/netty-transport-native-epoll-4.1.45.Final.LICENSE.txt rename to zookeeper-server/src/main/resources/lib/netty-transport-native-epoll-4.1.48.Final.LICENSE.txt diff --git a/zookeeper-server/src/main/resources/lib/netty-transport-native-unix-common-4.1.45.Final.LICENSE.txt b/zookeeper-server/src/main/resources/lib/netty-transport-native-unix-common-4.1.48.Final.LICENSE.txt similarity index 100% rename from zookeeper-server/src/main/resources/lib/netty-transport-native-unix-common-4.1.45.Final.LICENSE.txt rename to zookeeper-server/src/main/resources/lib/netty-transport-native-unix-common-4.1.48.Final.LICENSE.txt From 7812399f2c34033d56267d3cd2189abefbf35172 Mon Sep 17 00:00:00 2001 From: tison Date: Thu, 16 Apr 2020 10:50:00 +0000 Subject: [PATCH 020/591] ZOOKEEPER-3579: Handle null default watcher gracefully See also https://issues.apache.org/jira/browse/ZOOKEEPER-3579 Prevent error logs noise like >2019-10-14 18:41:49 ERROR ClientCnxn:537 - Error while calling watcher2019-10-14 18:41:49 ERROR ClientCnxn:537 - Error while calling watcherjava.lang.NullPointerException at org.apache.zookeeper.ClientCnxn$EventThread.processEvent(ClientCnxn.java:535) at org.apache.zookeeper.ClientCnxn$EventThread.run(ClientCnxn.java:510)2019-10-14 18:41:50 ERROR ClientCnxn:537 - Error while calling watcherjava.lang.NullPointerException at org.apache.zookeeper.ClientCnxn$EventThread.processEvent(ClientCnxn.java:535) at org.apache.zookeeper.ClientCnxn$EventThread.run(ClientCnxn.java:510) Author: tison Reviewers: Christopher Tubbs , Enrico Olivelli , maoling , Mate Szalay-Beko Closes #1120 from TisonKun/ZOOKEEPER-3579 --- .../java/org/apache/zookeeper/ClientCnxn.java | 2 +- .../java/org/apache/zookeeper/ZooKeeper.java | 82 +++++++++++++------ 2 files changed, 60 insertions(+), 24 deletions(-) diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/ClientCnxn.java b/zookeeper-server/src/main/java/org/apache/zookeeper/ClientCnxn.java index b3591461675..c87d3cba862 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/ClientCnxn.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/ClientCnxn.java @@ -576,7 +576,7 @@ private void processEvent(Object event) { try { watcher.process(pair.event); } catch (Throwable t) { - LOG.error("Error while calling watcher ", t); + LOG.error("Error while calling watcher.", t); } } } else if (event instanceof LocalCallback) { diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/ZooKeeper.java b/zookeeper-server/src/main/java/org/apache/zookeeper/ZooKeeper.java index f6f165deaf4..0210493efc0 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/ZooKeeper.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/ZooKeeper.java @@ -525,12 +525,16 @@ protected boolean removeWatches( public Set materialize( Watcher.Event.KeeperState state, Watcher.Event.EventType type, - String clientPath) { - Set result = new HashSet(); + String clientPath + ) { + final Set result = new HashSet<>(); switch (type) { case None: - result.add(defaultWatcher); + if (defaultWatcher != null) { + result.add(defaultWatcher); + } + boolean clear = disableAutoWatchReset && state != Watcher.Event.KeeperState.SyncConnected; synchronized (dataWatches) { for (Set ws : dataWatches.values()) { @@ -2252,23 +2256,22 @@ public Stat exists(final String path, Watcher watcher) throws KeeperException, I /** * Return the stat of the node of the given path. Return null if no such a * node exists. - *

    - * If the watch is true and the call is successful (no exception is thrown), + * + *

    If the watch is true and the call is successful (no exception is thrown), * a watch will be left on the node with the given path. The watch will be * triggered by a successful operation that creates/delete the node or sets * the data on the node. * - * @param path - * the node path - * @param watch - * whether need to watch this node + * @param path the node path + * @param watch whether need to watch this node * @return the stat of the node of the given path; return null if no such a * node exists. * @throws KeeperException If the server signals an error + * @throws IllegalStateException if watch this node with a null default watcher * @throws InterruptedException If the server transaction is interrupted. */ public Stat exists(String path, boolean watch) throws KeeperException, InterruptedException { - return exists(path, watch ? watchManager.defaultWatcher : null); + return exists(path, getDefaultWatcher(watch)); } /** @@ -2300,10 +2303,12 @@ public void exists(final String path, Watcher watcher, StatCallback cb, Object c /** * The asynchronous version of exists. * + * @throws IllegalStateException if watch this node with a null default watcher + * * @see #exists(String, boolean) */ public void exists(String path, boolean watch, StatCallback cb, Object ctx) { - exists(path, watch ? watchManager.defaultWatcher : null, cb, ctx); + exists(path, getDefaultWatcher(watch), cb, ctx); } /** @@ -2369,10 +2374,11 @@ public byte[] getData(final String path, Watcher watcher, Stat stat) throws Keep * @param stat the stat of the node * @return the data of the node * @throws KeeperException If the server signals an error with a non-zero error code + * @throws IllegalStateException if watch this node with a null default watcher * @throws InterruptedException If the server transaction is interrupted. */ public byte[] getData(String path, boolean watch, Stat stat) throws KeeperException, InterruptedException { - return getData(path, watch ? watchManager.defaultWatcher : null, stat); + return getData(path, getDefaultWatcher(watch), stat); } /** @@ -2404,10 +2410,12 @@ public void getData(final String path, Watcher watcher, DataCallback cb, Object /** * The asynchronous version of getData. * + * @throws IllegalStateException if watch this node with a null default watcher + * * @see #getData(String, boolean, Stat) */ public void getData(String path, boolean watch, DataCallback cb, Object ctx) { - getData(path, watch ? watchManager.defaultWatcher : null, cb, ctx); + getData(path, getDefaultWatcher(watch), cb, ctx); } /** @@ -2490,19 +2498,22 @@ public void getConfig(Watcher watcher, DataCallback cb, Object ctx) { * @param stat the stat of the configuration node ZooDefs.CONFIG_NODE * @return configuration data stored in ZooDefs.CONFIG_NODE * @throws KeeperException If the server signals an error with a non-zero error code + * @throws IllegalStateException if watch this node with a null default watcher * @throws InterruptedException If the server transaction is interrupted. */ public byte[] getConfig(boolean watch, Stat stat) throws KeeperException, InterruptedException { - return getConfig(watch ? watchManager.defaultWatcher : null, stat); + return getConfig(getDefaultWatcher(watch), stat); } /** * The Asynchronous version of getConfig. * + * @throws IllegalStateException if watch this node with a null default watcher + * * @see #getData(String, boolean, Stat) */ public void getConfig(boolean watch, DataCallback cb, Object ctx) { - getConfig(watch ? watchManager.defaultWatcher : null, cb, ctx); + getConfig(getDefaultWatcher(watch), cb, ctx); } /** @@ -2752,14 +2763,15 @@ public List getChildren(final String path, Watcher watcher) throws Keepe * A KeeperException with error code KeeperException.NoNode will be thrown * if no node with the given path exists. * - * @param path - * @param watch + * @param path the node path + * @param watch whether need to watch this node * @return an unordered array of children of the node with the given path + * @throws IllegalStateException if watch this node with a null default watcher * @throws InterruptedException If the server transaction is interrupted. * @throws KeeperException If the server signals an error with a non-zero error code. */ public List getChildren(String path, boolean watch) throws KeeperException, InterruptedException { - return getChildren(path, watch ? watchManager.defaultWatcher : null); + return getChildren(path, getDefaultWatcher(watch)); } /** @@ -2791,10 +2803,12 @@ public void getChildren(final String path, Watcher watcher, ChildrenCallback cb, /** * The asynchronous version of getChildren. * + * @throws IllegalStateException if watch this node with a null default watcher + * * @see #getChildren(String, boolean) */ public void getChildren(String path, boolean watch, ChildrenCallback cb, Object ctx) { - getChildren(path, watch ? watchManager.defaultWatcher : null, cb, ctx); + getChildren(path, getDefaultWatcher(watch), cb, ctx); } /** @@ -2868,10 +2882,11 @@ public List getChildren( * * @since 3.3.0 * - * @param path - * @param watch + * @param path the node path + * @param watch whether need to watch this node * @param stat stat of the znode designated by path * @return an unordered array of children of the node with the given path + * @throws IllegalStateException if watch this node with a null default watcher * @throws InterruptedException If the server transaction is interrupted. * @throws KeeperException If the server signals an error with a non-zero * error code. @@ -2880,7 +2895,7 @@ public List getChildren( String path, boolean watch, Stat stat) throws KeeperException, InterruptedException { - return getChildren(path, watch ? watchManager.defaultWatcher : null, stat); + return getChildren(path, getDefaultWatcher(watch), stat); } /** @@ -2916,10 +2931,12 @@ public void getChildren(final String path, Watcher watcher, Children2Callback cb * * @since 3.3.0 * + * @throws IllegalStateException if watch this node with a null default watcher + * * @see #getChildren(String, boolean, Stat) */ public void getChildren(String path, boolean watch, Children2Callback cb, Object ctx) { - getChildren(path, watch ? watchManager.defaultWatcher : null, cb, ctx); + getChildren(path, getDefaultWatcher(watch), cb, ctx); } /** @@ -3398,6 +3415,25 @@ private ClientCnxnSocket getClientCnxnSocket() throws IOException { } } + /** + * Return the default watcher of this instance if required. + * + * @param required if the default watcher required + * @return the default watcher if required, otherwise {@code null}. + * @throws IllegalStateException if a null default watcher is required + */ + private Watcher getDefaultWatcher(boolean required) { + if (required) { + if (watchManager.defaultWatcher != null) { + return watchManager.defaultWatcher; + } else { + throw new IllegalStateException("Default watcher is required, but it is null."); + } + } + + return null; + } + /** * Validates the provided ACL list for null, empty or null value in it. * From 5e592d303b596961c03bff9731cfe2e202cea5f2 Mon Sep 17 00:00:00 2001 From: Christopher Tubbs Date: Thu, 16 Apr 2020 11:20:25 -0700 Subject: [PATCH 021/591] ZOOKEEPER-3788: Add m2e config for Eclipse devs Add information to the pom.xml about plugin executions when importing the project into Eclipse using M2E. This information is added to a profile that is automatically activated when using Eclipse, but otherwise ignored. This change enables developers who use Eclipse to more easily develop and contribute to ZooKeeper without the project generating a bunch of errors when importing into the Eclipse IDE. As plugins add their own m2e lifecycle metadata, the section added in this commit can be removed. For more, see: https://www.eclipse.org/m2e/documentation/m2e-execution-not-covered.html Author: Christopher Tubbs Reviewers: ben reed Closes #1313 from ctubbsii/eclipse-m2e-config --- pom.xml | 82 +++++++++++++++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 82 insertions(+) diff --git a/pom.xml b/pom.xml index e9e973facfe..7361dbae7c7 100755 --- a/pom.xml +++ b/pom.xml @@ -324,6 +324,88 @@ 8 + + + + + org.eclipse.m2e + lifecycle-mapping + 1.0.0 + + + + + + org.codehaus.mojo + exec-maven-plugin + [0,) + + exec + + + + + + + + + org.apache.maven.plugins + maven-antrun-plugin + [0,) + + run + + + + + + + + + com.ruleoftech + markdown-page-generator-plugin + [0,) + + generate + + + + + + + + + org.codehaus.mojo + javacc-maven-plugin + [0,) + + javacc + + + + + + + + + com.github.koraktor + mavanagaiata + [0,) + + commit + + + + + + + + + + + + + jdk-release-flag From f44937ec6a23aa5cb98e3121b4de85d3d42093ed Mon Sep 17 00:00:00 2001 From: Damien Diederen Date: Mon, 27 Apr 2020 10:58:25 +0200 Subject: [PATCH 022/591] ZOOKEEPER-3790: zkpython compilation and testing issues This series makes the zkpython "contrib" compile cleanly, and makes the tests runnable out of the box with Python 3: * Defined `THREADED`, as zkpython uses the sync API Without this, compilation produces a number of warnings about undefined functions (and misleading suggestions!) as it only sees the async API: src/c/zookeeper.c:1080:13: warning: implicit declaration of function 'zoo_delete'; did you mean 'zoo_adelete'? [-Wimplicit-function-declaration] int err = zoo_delete(zh, path, version); ^~~~~~~~~~ zoo_adelete * Define `HAVE_OPENSSL_H`, as the extension calls zookeeper_init_ssl The flag is unconditionally defined for now, as the function is unconditionally called. src/c/zookeeper.c:646:10: warning: implicit declaration of function 'zookeeper_init_ssl'; did you mean 'zookeeper_init2'? [-Wimplicit-function-declaration] zh = zookeeper_init_ssl( host, cert_str, watcherfn != Py_None ? watcher_dispatch : NULL, ^~~~~~~~~~~~~~~~~~ zookeeper_init2 * Make SSL support optional (but on by default) * Raise `MemoryError` if module initialization fails * Allow for version/ABI information in shared object name In some versions of the Python framework, native extensions encode the interpreter version and some ABI information in the filename, giving e.g. `zookeeper.cpython-37m-x86_64-linux-gnu.so` instead of `zookeeper.so`. Take this into account when setting up test runs. * Make sure test failures are detected * `async` is a keyword in Python 3.5+ Author: Damien Diederen Reviewers: Mate Szalay-Beko , Norbert Kalmar Closes #1312 from ztzg/ZOOKEEPER-3790-zkpython-compilation-and-testing-issues (cherry picked from commit 21221ac692405962bdda3e9ceb63186ef0f180ef) Signed-off-by: Norbert Kalmar --- .../zookeeper-contrib-zkpython/README | 10 +++++--- .../src/c/zookeeper.c | 25 +++++++++++++++---- .../src/python/setup.py | 9 +++++++ .../src/test/async_test.py | 2 +- .../src/test/callback_test.py | 6 ++--- .../src/test/connection_test.py | 3 ++- .../src/test/run_tests.sh | 4 ++- 7 files changed, 44 insertions(+), 15 deletions(-) diff --git a/zookeeper-contrib/zookeeper-contrib-zkpython/README b/zookeeper-contrib/zookeeper-contrib-zkpython/README index 56154618356..ffad255f06f 100644 --- a/zookeeper-contrib/zookeeper-contrib-zkpython/README +++ b/zookeeper-contrib/zookeeper-contrib-zkpython/README @@ -5,13 +5,15 @@ Please do not rely on APIs staying constant in the short term. The handling of e DEPENDENCIES: ------------- -This has only been tested against SVN (i.e. 3.2.0 in development) but should work against 3.1.1. +This has only been tested against SVN/Git (i.e. 3.2.0 in development) but should work against 3.1.1. You will need the Python development headers installed to build the module - on many package-management systems, these can be found in python-devel. (On ubuntu 18.4, install python2.7 and python2.7-dev.) -Python >= 2.6 is required. We have tested against 2.6. We have not tested against 3.x. +Python >= 2.6 is required. We have tested against 2.6 and 3.5+. -E.g. setting up tpyhon and python devel on ubuntu 18.4: +By default, the extension assumes that the C client library was compiled with OpenSSL enabled (--with-openssl). You can disable OpenSSL support in the Python binding by setting the ZKPYTHON_NO_SSL environment variable to a non-empty string before executing Ant or setup.py. + +E.g. setting up python and python devel on ubuntu 18.4: sudo apt-get install python2.7 python2.7-dev sudo update-alternatives --install /usr/bin/python python /usr/bin/python2.7 1 @@ -22,7 +24,7 @@ To install, make sure that the C client has been built (use `mvn clean install - ant install -from zookeeper/src/contrib/zkpython/. +from zookeeper-contrib/zookeeper-contrib-zkpython/. To test, run ant test from the same directory. diff --git a/zookeeper-contrib/zookeeper-contrib-zkpython/src/c/zookeeper.c b/zookeeper-contrib/zookeeper-contrib-zkpython/src/c/zookeeper.c index e84c2b70700..ee8a75a03f7 100644 --- a/zookeeper-contrib/zookeeper-contrib-zkpython/src/c/zookeeper.c +++ b/zookeeper-contrib/zookeeper-contrib-zkpython/src/c/zookeeper.c @@ -600,7 +600,7 @@ void acl_completion_dispatch(int rc, struct ACL_vector *acl, struct Stat *stat, /* -------------------------------------------------------------------------- */ -static PyObject *pyzookeeper_init_optional_ssl(PyObject *self, PyObject *args, int ssl) { +static PyObject *pyzookeeper_init_common(PyObject *self, PyObject *args, int ssl) { const char *host; const char *cert_str; PyObject *watcherfn = Py_None; @@ -643,8 +643,13 @@ static PyObject *pyzookeeper_init_optional_ssl(PyObject *self, PyObject *args, i watchers[handle] = pyw; if (ssl) { +#ifdef HAVE_OPENSSL_H zh = zookeeper_init_ssl( host, cert_str, watcherfn != Py_None ? watcher_dispatch : NULL, recv_timeout, cid.client_id == -1 ? 0 : &cid, pyw, 0 ); +#else + fprintf(stderr, "SSL support not compiled in (called with ssl=%d).\n", ssl); + abort(); +#endif } else { zh = zookeeper_init( host, watcherfn != Py_None ? watcher_dispatch : NULL, recv_timeout, cid.client_id == -1 ? 0 : &cid, pyw, 0 ); @@ -652,7 +657,7 @@ static PyObject *pyzookeeper_init_optional_ssl(PyObject *self, PyObject *args, i if (zh == NULL) { - PyErr_SetString( ZooKeeperException, "Could not internally obtain SSL zookeeper handle" ); + PyErr_Format( ZooKeeperException, "Could not internally obtain%s zookeeper handle", ssl ? " SSL" : "" ); return NULL; } @@ -662,14 +667,16 @@ static PyObject *pyzookeeper_init_optional_ssl(PyObject *self, PyObject *args, i static PyObject *pyzookeeper_init(PyObject *self, PyObject *args) { - return pyzookeeper_init_optional_ssl(self, args, 0); + return pyzookeeper_init_common(self, args, /*ssl*/0); } +#ifdef HAVE_OPENSSL_H static PyObject *pyzookeeper_init_ssl(PyObject *self, PyObject *args) { - return pyzookeeper_init_optional_ssl(self, args, 1); + return pyzookeeper_init_common(self, args, /*ssl*/1); } +#endif /* -------------------------------------------------------------------------- */ @@ -1518,7 +1525,9 @@ PyObject *pyzoo_deterministic_conn_order(PyObject *self, PyObject *args) static PyMethodDef ZooKeeperMethods[] = { {"init", pyzookeeper_init, METH_VARARGS, pyzk_init_doc }, +#ifdef HAVE_OPENSSL_H {"init_ssl", pyzookeeper_init_ssl, METH_VARARGS, pyzk_init_ssl_doc }, +#endif {"create",pyzoo_create, METH_VARARGS, pyzk_create_doc }, {"delete",pyzoo_delete, METH_VARARGS, pyzk_delete_doc }, {"get_children", pyzoo_get_children, METH_VARARGS, pyzk_get_children_doc }, @@ -1589,8 +1598,14 @@ PyMODINIT_FUNC initzookeeper(void) { #else PyObject *module = Py_InitModule("zookeeper", ZooKeeperMethods); #endif + if (init_zhandles(32) == 0) { - return; // TODO: Is there any way to raise an exception here? +#if PY_MAJOR_VERSION >= 3 + Py_DECREF(module); + return PyErr_NoMemory(); +#else + return; +#endif } ZooKeeperException = PyErr_NewException("zookeeper.ZooKeeperException", diff --git a/zookeeper-contrib/zookeeper-contrib-zkpython/src/python/setup.py b/zookeeper-contrib/zookeeper-contrib-zkpython/src/python/setup.py index 313c020569c..b225a317400 100755 --- a/zookeeper-contrib/zookeeper-contrib-zkpython/src/python/setup.py +++ b/zookeeper-contrib/zookeeper-contrib-zkpython/src/python/setup.py @@ -15,11 +15,20 @@ # limitations under the License. from distutils.core import setup, Extension +import os zookeeper_basedir = "../../" +zookeeper_macros = [("THREADED", None)] + +# Assume the C extension includes OpenSSL support unless told +# otherwise. +if not os.environ.get("ZKPYTHON_NO_SSL"): + zookeeper_macros.append(("HAVE_OPENSSL_H", True)) + zookeepermodule = Extension("zookeeper", sources=["src/c/zookeeper.c"], + define_macros=zookeeper_macros, include_dirs=[zookeeper_basedir + "/zookeeper-client/zookeeper-client-c/include", zookeeper_basedir + "/zookeeper-client/zookeeper-client-c/target/c", zookeeper_basedir + "/zookeeper-client/zookeeper-client-c/generated"], diff --git a/zookeeper-contrib/zookeeper-contrib-zkpython/src/test/async_test.py b/zookeeper-contrib/zookeeper-contrib-zkpython/src/test/async_test.py index e81343570ea..61740ae433f 100644 --- a/zookeeper-contrib/zookeeper-contrib-zkpython/src/test/async_test.py +++ b/zookeeper-contrib/zookeeper-contrib-zkpython/src/test/async_test.py @@ -26,7 +26,7 @@ def setUp( self ): def test_async(self): self.assertEqual(self.connected, True) - ret = zookeeper.async(self.handle, "/") + ret = getattr(zookeeper, 'async')(self.handle, "/") self.assertEqual(ret, zookeeper.OK, "async failed") if __name__ == '__main__': diff --git a/zookeeper-contrib/zookeeper-contrib-zkpython/src/test/callback_test.py b/zookeeper-contrib/zookeeper-contrib-zkpython/src/test/callback_test.py index 55d7fe17866..95e20b4dea1 100644 --- a/zookeeper-contrib/zookeeper-contrib-zkpython/src/test/callback_test.py +++ b/zookeeper-contrib/zookeeper-contrib-zkpython/src/test/callback_test.py @@ -91,9 +91,9 @@ def dispatch_callback(*args, **kwargs): self.create_callback( dispatch_callback )), lambda: self.assertEqual(True, self.callback_flag, "Strings dispatch not fired")) - self.callback_harness( lambda: zookeeper.async(self.handle, - "/", - self.create_callback( dispatch_callback )), + self.callback_harness( lambda: getattr(zookeeper, 'async')(self.handle, + "/", + self.create_callback( dispatch_callback )), lambda: self.assertEqual(True, self.callback_flag, "String dispatch not fired")) self.callback_harness( lambda: zookeeper.aget_acl(self.handle, diff --git a/zookeeper-contrib/zookeeper-contrib-zkpython/src/test/connection_test.py b/zookeeper-contrib/zookeeper-contrib-zkpython/src/test/connection_test.py index 2661e6ecaf8..3fbbd4bf85b 100755 --- a/zookeeper-contrib/zookeeper-contrib-zkpython/src/test/connection_test.py +++ b/zookeeper-contrib/zookeeper-contrib-zkpython/src/test/connection_test.py @@ -58,7 +58,8 @@ def connection_watcher(handle, type, state, path): self.handle, "/") - + @unittest.skipUnless(hasattr(zookeeper, 'init_ssl'), + "SSL support not compiled in.") def testsslconnection(self): cv = threading.Condition() self.connected = False diff --git a/zookeeper-contrib/zookeeper-contrib-zkpython/src/test/run_tests.sh b/zookeeper-contrib/zookeeper-contrib-zkpython/src/test/run_tests.sh index a3cf4d87969..232359b16e0 100755 --- a/zookeeper-contrib/zookeeper-contrib-zkpython/src/test/run_tests.sh +++ b/zookeeper-contrib/zookeeper-contrib-zkpython/src/test/run_tests.sh @@ -19,6 +19,8 @@ # Usage: run_tests.sh testdir [logdir] # logdir is optional, defaults to cwd +set -e + # get the number of command-line arguments given ARGC=$# @@ -30,7 +32,7 @@ else fi # Find the build directory containing zookeeper.so -SO_PATH=`find ./target/ -name "zookeeper.so" | head -1` +SO_PATH=`find ./target/ -name 'zookeeper*.so' | head -1` PYTHONPATH=`dirname $SO_PATH` LIB_PATH=../../zookeeper-client/zookeeper-client-c/target/c/.libs for test in `ls $1/*_test.py`; From 2e14a29cc6e58d9561e80b737a3168fbb1f752b4 Mon Sep 17 00:00:00 2001 From: xavier Date: Thu, 30 Apr 2020 16:53:16 +0000 Subject: [PATCH 023/591] ZOOKEEPER-3811: cleaning up code,static field be directly referred by its class name optimized code,A static field should be directly referred by its class name Author: xavier <1217353758@qq.com> Reviewers: Christopher Tubbs , Fangmin Lyu , Mate Szalay-Beko Closes #1337 from XavierChengZW/master --- .../java/org/apache/zookeeper/server/RequestThrottler.java | 4 ++-- .../java/org/apache/zookeeper/server/ZooKeeperServerBean.java | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/RequestThrottler.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/RequestThrottler.java index e9cdc5ec10b..32863d92a61 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/RequestThrottler.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/RequestThrottler.java @@ -100,8 +100,8 @@ public class RequestThrottler extends ZooKeeperCriticalThread { protected boolean shouldThrottleOp(Request request, long elapsedTime) { return request.isThrottlable() - && zks.getThrottledOpWaitTime() > 0 - && elapsedTime > zks.getThrottledOpWaitTime(); + && ZooKeeperServer.getThrottledOpWaitTime() > 0 + && elapsedTime > ZooKeeperServer.getThrottledOpWaitTime(); } diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/ZooKeeperServerBean.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/ZooKeeperServerBean.java index 17dd48b7505..5eac7edf8c5 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/ZooKeeperServerBean.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/ZooKeeperServerBean.java @@ -316,11 +316,11 @@ public void setFlushDelay(long delay) { /////////////////////////////////////////////////////////////////////////// public int getThrottledOpWaitTime() { - return zks.getThrottledOpWaitTime(); + return ZooKeeperServer.getThrottledOpWaitTime(); } public void setThrottledOpWaitTime(int val) { - zks.setThrottledOpWaitTime(val); + ZooKeeperServer.setThrottledOpWaitTime(val); } /////////////////////////////////////////////////////////////////////////// From b4f6e82de10e6f87836f8221970110dad2b3825c Mon Sep 17 00:00:00 2001 From: Jie Huang Date: Sun, 3 May 2020 19:43:27 +0200 Subject: [PATCH 024/591] ZOOKEEPER-3574: Close quorum socket asynchronously to avoid server shutdown stalled by long socket closing time MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit …utdown stalled by long socket closing time Author: Jie Huang Reviewers: Enrico Olivelli , Fangmin Lyu Closes #1115 from jhuan31/ZOOKEEPER-3574 --- .../main/resources/markdown/zookeeperAdmin.md | 173 +++++++++--------- .../zookeeper/server/ServerMetrics.java | 5 + .../zookeeper/server/quorum/Learner.java | 26 +++ 3 files changed, 120 insertions(+), 84 deletions(-) diff --git a/zookeeper-docs/src/main/resources/markdown/zookeeperAdmin.md b/zookeeper-docs/src/main/resources/markdown/zookeeperAdmin.md index ca1a3f71f57..6c616d1af68 100644 --- a/zookeeper-docs/src/main/resources/markdown/zookeeperAdmin.md +++ b/zookeeper-docs/src/main/resources/markdown/zookeeperAdmin.md @@ -202,11 +202,11 @@ ensemble: though about a few here: Every machine that is part of the ZooKeeper ensemble should know about every other machine in the ensemble. You accomplish this with - the series of lines of the form **server.id=host:port:port**. - (The parameters **host** and **port** are straightforward, for each server + the series of lines of the form **server.id=host:port:port**. + (The parameters **host** and **port** are straightforward, for each server you need to specify first a Quorum port then a dedicated port for ZooKeeper leader - election). Since ZooKeeper 3.6.0 you can also [specify multiple addresses](#id_multi_address) - for each ZooKeeper server instance (this can increase availability when multiple physical + election). Since ZooKeeper 3.6.0 you can also [specify multiple addresses](#id_multi_address) + for each ZooKeeper server instance (this can increase availability when multiple physical network interfaces can be used parallel in the cluster). You attribute the server id to each machine by creating a file named @@ -233,7 +233,7 @@ ensemble: ensemble. 7. If your configuration file is set up, you can start a - ZooKeeper server: + ZooKeeper server: $ java -cp zookeeper.jar:lib/*:conf org.apache.zookeeper.server.quorum.QuorumPeerMain zoo.conf @@ -1004,7 +1004,7 @@ property, when available, is noted below. * *zookeeper.request_throttler.shutdownTimeout* : (Java system property only) **New in 3.6.0:** - The time (in milliseconds) the RequestThrottler waits for the request queue to drain during shutdown before it shuts down forcefully. The default is 10000. + The time (in milliseconds) the RequestThrottler waits for the request queue to drain during shutdown before it shuts down forcefully. The default is 10000. * *advancedFlowControlEnabled* : (Java system property: **zookeeper.netty.advancedFlowControl.enabled**) @@ -1032,59 +1032,59 @@ property, when available, is noted below. **New in 3.6.0:** The digest feature is added to detect the data inconsistency inside ZooKeeper when loading database from disk, catching up and following - leader, its doing incrementally hash check for the DataTree based on + leader, its doing incrementally hash check for the DataTree based on the adHash paper mentioned in https://cseweb.ucsd.edu/~daniele/papers/IncHash.pdf - The idea is simple, the hash value of DataTree will be updated incrementally - based on the changes to the set of data. When the leader is preparing the txn, - it will pre-calculate the hash of the tree based on the changes happened with + The idea is simple, the hash value of DataTree will be updated incrementally + based on the changes to the set of data. When the leader is preparing the txn, + it will pre-calculate the hash of the tree based on the changes happened with formula: current_hash = current_hash + hash(new node data) - hash(old node data) - If it’s creating a new node, the hash(old node data) will be 0, and if it’s a + If it’s creating a new node, the hash(old node data) will be 0, and if it’s a delete node op, the hash(new node data) will be 0. - This hash will be associated with each txn to represent the expected hash value - after applying the txn to the data tree, it will be sent to followers with - original proposals. Learner will compare the actual hash value with the one in - the txn after applying the txn to the data tree, and report mismatch if it’s not + This hash will be associated with each txn to represent the expected hash value + after applying the txn to the data tree, it will be sent to followers with + original proposals. Learner will compare the actual hash value with the one in + the txn after applying the txn to the data tree, and report mismatch if it’s not the same. - These digest value will also be persisted with each txn and snapshot on the disk, - so when servers restarted and load data from disk, it will compare and see if + These digest value will also be persisted with each txn and snapshot on the disk, + so when servers restarted and load data from disk, it will compare and see if there is hash mismatch, which will help detect data loss issue on disk. - For the actual hash function, we’re using CRC internally, it’s not a collisionless - hash function, but it’s more efficient compared to collisionless hash, and the + For the actual hash function, we’re using CRC internally, it’s not a collisionless + hash function, but it’s more efficient compared to collisionless hash, and the collision possibility is really really rare and can already meet our needs here. - This feature is backward and forward compatible, so it can safely rolling upgrade, - downgrade, enabled and later disabled without any compatible issue. Here are the + This feature is backward and forward compatible, so it can safely rolling upgrade, + downgrade, enabled and later disabled without any compatible issue. Here are the scenarios have been covered and tested: - 1. When leader runs with new code while follower runs with old one, the digest will - be append to the end of each txn, follower will only read header and txn data, - digest value in the txn will be ignored. It won't affect the follower reads and + 1. When leader runs with new code while follower runs with old one, the digest will + be append to the end of each txn, follower will only read header and txn data, + digest value in the txn will be ignored. It won't affect the follower reads and processes the next txn. 2. When leader runs with old code while follower runs with new one, the digest won't - be sent with txn, when follower tries to read the digest, it will throw EOF which + be sent with txn, when follower tries to read the digest, it will throw EOF which is caught and handled gracefully with digest value set to null. 3. When loading old snapshot with new code, it will throw IOException when trying to read the non-exist digest value, and the exception will be caught and digest will - be set to null, which means we won't compare digest when loading this snapshot, + be set to null, which means we won't compare digest when loading this snapshot, which is expected to happen during rolling upgrade - 4. When loading new snapshot with old code, it will finish successfully after deserialzing + 4. When loading new snapshot with old code, it will finish successfully after deserialzing the data tree, the digest value at the end of snapshot file will be ignored - 5. The scenarios of rolling restart with flags change are similar to the 1st and 2nd + 5. The scenarios of rolling restart with flags change are similar to the 1st and 2nd scenarios discussed above, if the leader enabled but follower not, digest value will be ignored, and follower won't compare the digest during runtime; if leader disabled but follower enabled, follower will get EOF exception which is handled gracefully. - Note: the current digest calculation excluded nodes under /zookeeper - due to the potential inconsistency in the /zookeeper/quota stat node, + Note: the current digest calculation excluded nodes under /zookeeper + due to the potential inconsistency in the /zookeeper/quota stat node, we can include that after that issue is fixed. By default, this feautre is disabled, set "true" to enable it. @@ -1107,7 +1107,7 @@ property, when available, is noted below. **New in 3.6.0:** By default audit logs are disabled. Set to "true" to enable it. Default value is "false". See the [ZooKeeper audit logs](zookeeperAuditLogs.html) for more information. - + * *audit.impl.class* : (Java system property: **zookeeper.audit.impl.class**) **New in 3.6.0:** @@ -1125,12 +1125,12 @@ property, when available, is noted below. **New in 3.6.0:** The size threshold after which a request is considered a large request. If it is -1, then all requests are considered small, effectively turning off large request throttling. The default is -1. -* *outstandingHandshake.limit* +* *outstandingHandshake.limit* (Jave system property only: **zookeeper.netty.server.outstandingHandshake.limit**) - The maximum in-flight TLS handshake connections could have in ZooKeeper, - the connections exceed this limit will be rejected before starting handshake. - This setting doesn't limit the max TLS concurrency, but helps avoid herd - effect due to TLS handshake timeout when there are too many in-flight TLS + The maximum in-flight TLS handshake connections could have in ZooKeeper, + the connections exceed this limit will be rejected before starting handshake. + This setting doesn't limit the max TLS concurrency, but helps avoid herd + effect due to TLS handshake timeout when there are too many in-flight TLS handshakes. Set it to something like 250 is good enough to avoid herd effect. * *throttledOpWaitTime* @@ -1142,6 +1142,11 @@ property, when available, is noted below. The intent is for the clients not to retry them immediately. When set to 0, no requests will be throttled. The default is 0. +* *learner.closeSocketAsync* + (Jave system property only: **learner.closeSocketAsync**) + When enabled, a learner will close the quorum socket asynchronously. This is useful for TLS connections where closing a socket might take a long time, block the shutdown process, potentially delay a new leader election, and leave the quorum unavailabe. Closing the socket asynchronously avoids blocking the shutdown process despite the long socket closing time and a new leader election can be started while the socket being closed. The default is false. + + #### Cluster Options @@ -1155,13 +1160,13 @@ of servers -- that is, when deploying clusters of servers. non-authenticated UDP-based version of fast leader election, "2" corresponds to the authenticated UDP-based version of fast leader election, and "3" corresponds to TCP-based version of - fast leader election. Algorithm 3 was made default in 3.2.0 and + fast leader election. Algorithm 3 was made default in 3.2.0 and prior versions (3.0.0 and 3.1.0) were using algorithm 1 and 2 as well. ###### Note - >The implementations of leader election 1, and 2 were - **deprecated** in 3.4.0. Since 3.6.0 only FastLeaderElection is available, - in case of upgrade you have to shutdown all of your servers and - restart them with electionAlg=3 (or by removing the line from the configuration file). > + >The implementations of leader election 1, and 2 were + **deprecated** in 3.4.0. Since 3.6.0 only FastLeaderElection is available, + in case of upgrade you have to shutdown all of your servers and + restart them with electionAlg=3 (or by removing the line from the configuration file). > * *maxTimeToWaitForEpoch* : (Java system property: **zookeeper.leader.maxTimeToWaitForEpoch**) @@ -1215,15 +1220,15 @@ of servers -- that is, when deploying clusters of servers. The first followers use to connect to the leader, and the second is for leader election. If you want to test multiple servers on a single machine, then different ports can be used for each server. - + Since ZooKeeper 3.6.0 it is possible to specify **multiple addresses** for each ZooKeeper server (see [ZOOKEEPER-3188](https://issues.apache.org/jira/projects/ZOOKEEPER/issues/ZOOKEEPER-3188)). To enable this feature, you must set the *multiAddress.enabled* configuration property - to *true*. This helps to increase availability and adds network level - resiliency to ZooKeeper. When multiple physical network interfaces are used - for the servers, ZooKeeper is able to bind on all interfaces and runtime switching + to *true*. This helps to increase availability and adds network level + resiliency to ZooKeeper. When multiple physical network interfaces are used + for the servers, ZooKeeper is able to bind on all interfaces and runtime switching to a working interface in case a network error. The different addresses can be specified in the config using a pipe ('|') character. A valid configuration using multiple addresses looks like: @@ -1375,16 +1380,16 @@ As an example, this will enable all four letter word commands: * *electionPortBindRetry* : (Java system property only: **zookeeper.electionPortBindRetry**) - Property set max retry count when Zookeeper server fails to bind - leader election port. Such errors can be temporary and recoverable, + Property set max retry count when Zookeeper server fails to bind + leader election port. Such errors can be temporary and recoverable, such as DNS issue described in [ZOOKEEPER-3320](https://issues.apache.org/jira/projects/ZOOKEEPER/issues/ZOOKEEPER-3320), - or non-retryable, such as port already in use. - In case of transient errors, this property can improve availability - of Zookeeper server and help it to self recover. - Default value 3. In container environment, especially in Kubernetes, - this value should be increased or set to 0(infinite retry) to overcome issues + or non-retryable, such as port already in use. + In case of transient errors, this property can improve availability + of Zookeeper server and help it to self recover. + Default value 3. In container environment, especially in Kubernetes, + this value should be increased or set to 0(infinite retry) to overcome issues related to DNS name resolving. - + * *observer.reconnectDelayMs* : (Java system property: **zookeeper.observer.reconnectDelayMs**) @@ -1414,9 +1419,9 @@ As an example, this will enable all four letter word commands: The options in this section allow control over encryption/authentication/authorization performed by the service. -Beside this page, you can also find useful information about client side configuration in the -[Programmers Guide](zookeeperProgrammers.html#sc_java_client_configuration). -The ZooKeeper Wiki also has useful pages about [ZooKeeper SSL support](https://cwiki.apache.org/confluence/display/ZOOKEEPER/ZooKeeper+SSL+User+Guide), +Beside this page, you can also find useful information about client side configuration in the +[Programmers Guide](zookeeperProgrammers.html#sc_java_client_configuration). +The ZooKeeper Wiki also has useful pages about [ZooKeeper SSL support](https://cwiki.apache.org/confluence/display/ZOOKEEPER/ZooKeeper+SSL+User+Guide), and [SASL authentication for ZooKeeper](https://cwiki.apache.org/confluence/display/ZOOKEEPER/ZooKeeper+and+SASL). * *DigestAuthenticationProvider.superDigest* : @@ -1509,8 +1514,8 @@ and [SASL authentication for ZooKeeper](https://cwiki.apache.org/confluence/disp * *ssl.keyStore.type* and *ssl.quorum.keyStore.type* : (Java system properties: **zookeeper.ssl.keyStore.type** and **zookeeper.ssl.quorum.keyStore.type**) **New in 3.5.5:** - Specifies the file format of client and quorum keystores. Values: JKS, PEM, PKCS12 or null (detect by filename). - Default: null + Specifies the file format of client and quorum keystores. Values: JKS, PEM, PKCS12 or null (detect by filename). + Default: null * *ssl.trustStore.location* and *ssl.trustStore.password* and *ssl.quorum.trustStore.location* and *ssl.quorum.trustStore.password* : (Java system properties: **zookeeper.ssl.trustStore.location** and **zookeeper.ssl.trustStore.password** and **zookeeper.ssl.quorum.trustStore.location** and **zookeeper.ssl.quorum.trustStore.password**) @@ -1522,8 +1527,8 @@ and [SASL authentication for ZooKeeper](https://cwiki.apache.org/confluence/disp * *ssl.trustStore.type* and *ssl.quorum.trustStore.type* : (Java system properties: **zookeeper.ssl.trustStore.type** and **zookeeper.ssl.quorum.trustStore.type**) **New in 3.5.5:** - Specifies the file format of client and quorum trustStores. Values: JKS, PEM, PKCS12 or null (detect by filename). - Default: null + Specifies the file format of client and quorum trustStores. Values: JKS, PEM, PKCS12 or null (detect by filename). + Default: null * *ssl.protocol* and *ssl.quorum.protocol* : (Java system properties: **zookeeper.ssl.protocol** and **zookeeper.ssl.quorum.protocol**) @@ -1541,7 +1546,7 @@ and [SASL authentication for ZooKeeper](https://cwiki.apache.org/confluence/disp (Java system properties: **zookeeper.ssl.ciphersuites** and **zookeeper.ssl.quorum.ciphersuites**) **New in 3.5.5:** Specifies the enabled cipher suites to be used in client and quorum TLS negotiation. - Default: Enabled cipher suites depend on the Java runtime version being used. + Default: Enabled cipher suites depend on the Java runtime version being used. * *ssl.context.supplier.class* and *ssl.quorum.context.supplier.class* : (Java system properties: **zookeeper.ssl.context.supplier.class** and **zookeeper.ssl.quorum.context.supplier.class**) @@ -1586,19 +1591,19 @@ and [SASL authentication for ZooKeeper](https://cwiki.apache.org/confluence/disp Specifies that the client port should accept SSL connections (using the same configuration as the secure client port). Default: false - + * *authProvider*: (Java system property: **zookeeper.authProvider**) You can specify multiple authentication provider classes for ZooKeeper. Usually you use this parameter to specify the SASL authentication provider like: `authProvider.1=org.apache.zookeeper.server.auth.SASLAuthenticationProvider` - + * *kerberos.removeHostFromPrincipal* (Java system property: **zookeeper.kerberos.removeHostFromPrincipal**) You can instruct ZooKeeper to remove the host from the client principal name during authentication. (e.g. the zk/myhost@EXAMPLE.COM client principal will be authenticated in ZooKeeper as zk@EXAMPLE.COM) Default: false - + * *kerberos.removeRealmFromPrincipal* (Java system property: **zookeeper.kerberos.removeRealmFromPrincipal**) You can instruct ZooKeeper to remove the realm from the client principal name during authentication. @@ -1608,9 +1613,9 @@ and [SASL authentication for ZooKeeper](https://cwiki.apache.org/confluence/disp * *multiAddress.enabled* : (Java system property: **zookeeper.multiAddress.enabled**) **New in 3.6.0:** - Since ZooKeeper 3.6.0 you can also [specify multiple addresses](#id_multi_address) - for each ZooKeeper server instance (this can increase availability when multiple physical - network interfaces can be used parallel in the cluster). Setting this parameter to + Since ZooKeeper 3.6.0 you can also [specify multiple addresses](#id_multi_address) + for each ZooKeeper server instance (this can increase availability when multiple physical + network interfaces can be used parallel in the cluster). Setting this parameter to **true** will enable this feature. Please note, that you can not enable this feature during a rolling upgrade if the version of the old ZooKeeper cluster is prior to 3.6.0. The default value is **false**. @@ -1618,17 +1623,17 @@ and [SASL authentication for ZooKeeper](https://cwiki.apache.org/confluence/disp * *multiAddress.reachabilityCheckTimeoutMs* : (Java system property: **zookeeper.multiAddress.reachabilityCheckTimeoutMs**) **New in 3.6.0:** - Since ZooKeeper 3.6.0 you can also [specify multiple addresses](#id_multi_address) - for each ZooKeeper server instance (this can increase availability when multiple physical + Since ZooKeeper 3.6.0 you can also [specify multiple addresses](#id_multi_address) + for each ZooKeeper server instance (this can increase availability when multiple physical network interfaces can be used parallel in the cluster). ZooKeeper will perform ICMP ECHO requests - or try to establish a TCP connection on port 7 (Echo) of the destination host in order to find + or try to establish a TCP connection on port 7 (Echo) of the destination host in order to find the reachable addresses. This happens only if you provide multiple addresses in the configuration. - In this property you can set the timeout in millisecs for the reachability check. The check happens + In this property you can set the timeout in millisecs for the reachability check. The check happens in parallel for the different addresses, so the timeout you set here is the maximum time will be taken by checking the reachability of all addresses. The default value is **1000**. - This parameter has no effect, unless you enable the MultiAddress feature by setting *multiAddress.enabled=true*. + This parameter has no effect, unless you enable the MultiAddress feature by setting *multiAddress.enabled=true*. @@ -1684,11 +1689,11 @@ the variable does. * *jute.maxbuffer.extrasize*: (Java system property: **zookeeper.jute.maxbuffer.extrasize**) **New in 3.5.7:** - While processing client requests ZooKeeper server adds some additional information into - the requests before persisting it as a transaction. Earlier this additional information size + While processing client requests ZooKeeper server adds some additional information into + the requests before persisting it as a transaction. Earlier this additional information size was fixed to 1024 bytes. For many scenarios, specially scenarios where jute.maxbuffer value is more than 1 MB and request type is multi, this fixed size was insufficient. - To handle all the scenarios additional information size is increased from 1024 byte + To handle all the scenarios additional information size is increased from 1024 byte to same as jute.maxbuffer size and also it is made configurable through jute.maxbuffer.extrasize. Generally this property is not required to be configured as default value is the most optimal value. @@ -1708,17 +1713,17 @@ the variable does. * *multiAddress.reachabilityCheckEnabled* : (Java system property: **zookeeper.multiAddress.reachabilityCheckEnabled**) **New in 3.6.0:** - Since ZooKeeper 3.6.0 you can also [specify multiple addresses](#id_multi_address) - for each ZooKeeper server instance (this can increase availability when multiple physical + Since ZooKeeper 3.6.0 you can also [specify multiple addresses](#id_multi_address) + for each ZooKeeper server instance (this can increase availability when multiple physical network interfaces can be used parallel in the cluster). ZooKeeper will perform ICMP ECHO requests - or try to establish a TCP connection on port 7 (Echo) of the destination host in order to find + or try to establish a TCP connection on port 7 (Echo) of the destination host in order to find the reachable addresses. This happens only if you provide multiple addresses in the configuration. - The reachable check can fail if you hit some ICMP rate-limitation, (e.g. on MacOS) when you try to - start a large (e.g. 11+) ensemble members cluster on a single machine for testing. - - Default value is **true**. By setting this parameter to 'false' you can disable the reachability checks. - Please note, disabling the reachability check will cause the cluster not to be able to reconfigure - itself properly during network problems, so the disabling is advised only during testing. + The reachable check can fail if you hit some ICMP rate-limitation, (e.g. on MacOS) when you try to + start a large (e.g. 11+) ensemble members cluster on a single machine for testing. + + Default value is **true**. By setting this parameter to 'false' you can disable the reachability checks. + Please note, disabling the reachability check will cause the cluster not to be able to reconfigure + itself properly during network problems, so the disabling is advised only during testing. This parameter has no effect, unless you enable the MultiAddress feature by setting *multiAddress.enabled=true*. diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/ServerMetrics.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/ServerMetrics.java index 95214209c01..7ea701081e3 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/ServerMetrics.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/ServerMetrics.java @@ -232,9 +232,12 @@ private ServerMetrics(MetricsProvider metricsProvider) { NETTY_QUEUED_BUFFER = metricsContext.getSummary("netty_queued_buffer_capacity", DetailLevel.BASIC); DIGEST_MISMATCHES_COUNT = metricsContext.getCounter("digest_mismatches_count"); + TLS_HANDSHAKE_EXCEEDED = metricsContext.getCounter("tls_handshake_exceeded"); CNXN_CLOSED_WITHOUT_ZK_SERVER_RUNNING = metricsContext.getCounter("cnxn_closed_without_zk_server_running"); + + SOCKET_CLOSING_TIME = metricsContext.getSummary("socket_closing_time", DetailLevel.BASIC); } /** @@ -455,6 +458,8 @@ private ServerMetrics(MetricsProvider metricsProvider) { public final Counter CNXN_CLOSED_WITHOUT_ZK_SERVER_RUNNING; + public final Summary SOCKET_CLOSING_TIME; + private final MetricsProvider metricsProvider; public void resetAll() { diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Learner.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Learner.java index da5f1132ada..0b807ef799e 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Learner.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Learner.java @@ -38,6 +38,7 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicReference; import javax.net.ssl.SSLSocket; import org.apache.jute.BinaryInputArchive; @@ -46,10 +47,12 @@ import org.apache.jute.OutputArchive; import org.apache.jute.Record; import org.apache.zookeeper.ZooDefs.OpCode; +import org.apache.zookeeper.common.Time; import org.apache.zookeeper.common.X509Exception; import org.apache.zookeeper.server.ExitCode; import org.apache.zookeeper.server.Request; import org.apache.zookeeper.server.ServerCnxn; +import org.apache.zookeeper.server.ServerMetrics; import org.apache.zookeeper.server.TxnLogEntry; import org.apache.zookeeper.server.ZooTrace; import org.apache.zookeeper.server.quorum.QuorumPeer.QuorumServer; @@ -86,6 +89,7 @@ static class PacketInFlight { protected Socket sock; protected MultipleAddresses leaderAddr; + protected AtomicBoolean sockBeingClosed = new AtomicBoolean(false); /** * Socket getter @@ -116,10 +120,14 @@ public Socket getSocket() { public static final String LEARNER_ASYNC_SENDING = "learner.asyncSending"; private static boolean asyncSending = Boolean.getBoolean(LEARNER_ASYNC_SENDING); + public static final String LEARNER_CLOSE_SOCKET_ASYNC = "learner.closeSocketAsync"; + public static final boolean closeSocketAsync = Boolean.getBoolean(LEARNER_CLOSE_SOCKET_ASYNC); + static { LOG.info("leaderConnectDelayDuringRetryMs: {}", leaderConnectDelayDuringRetryMs); LOG.info("TCP NoDelay set to: {}", nodelay); LOG.info("{} = {}", LEARNER_ASYNC_SENDING, asyncSending); + LOG.info("{} = {}", LEARNER_CLOSE_SOCKET_ASYNC, closeSocketAsync); } final ConcurrentHashMap pendingRevalidations = new ConcurrentHashMap(); @@ -337,6 +345,7 @@ protected void connectToLeader(MultipleAddresses multiAddr, String hostname) thr throw new IOException("Failed connect to " + multiAddr); } else { sock = socket.get(); + sockBeingClosed.set(false); } self.authLearner.authenticate(sock, hostname); @@ -847,10 +856,27 @@ boolean isRunning() { } void closeSocket() { + if (sock != null) { + if (sockBeingClosed.compareAndSet(false, true)) { + if (closeSocketAsync) { + final Thread closingThread = new Thread(() -> closeSockSync(), "CloseSocketThread(sid:" + zk.getServerId()); + closingThread.setDaemon(true); + closingThread.start(); + } else { + closeSockSync(); + } + } + } + } + + void closeSockSync() { try { + long startTime = Time.currentElapsedTime(); if (sock != null) { sock.close(); + sock = null; } + ServerMetrics.getMetrics().SOCKET_CLOSING_TIME.add(Time.currentElapsedTime() - startTime); } catch (IOException e) { LOG.warn("Ignoring error closing connection to leader", e); } From ac4da3ebed34c536c4ea873503b0a6c036fa3910 Mon Sep 17 00:00:00 2001 From: Patrick Hunt Date: Mon, 4 May 2020 06:49:35 +0000 Subject: [PATCH 025/591] ZOOKEEPER-3794: upgrade netty to address CVE-2020-11612 - fixed file rename typo I borked the rename of the netty-codec file on the original submit. This fixes by renaming the one file appropriately. Change-Id: Ifff1e7bb49b3d0a5816ee97161a4043be7cdaaa6 Author: Patrick Hunt Reviewers: Mate Szalay-Beko Closes #1333 from phunt/ZOOKEEPER-3794_rename --- ....49.Final.LICENSE.txt => netty-codec-4.1.48.Final.LICENSE.txt} | 0 1 file changed, 0 insertions(+), 0 deletions(-) rename zookeeper-server/src/main/resources/lib/{netty-codec-4.1.49.Final.LICENSE.txt => netty-codec-4.1.48.Final.LICENSE.txt} (100%) diff --git a/zookeeper-server/src/main/resources/lib/netty-codec-4.1.49.Final.LICENSE.txt b/zookeeper-server/src/main/resources/lib/netty-codec-4.1.48.Final.LICENSE.txt similarity index 100% rename from zookeeper-server/src/main/resources/lib/netty-codec-4.1.49.Final.LICENSE.txt rename to zookeeper-server/src/main/resources/lib/netty-codec-4.1.48.Final.LICENSE.txt From f6b54a6cd227ac37f28803f45d7287c7fd3a8142 Mon Sep 17 00:00:00 2001 From: Mate Szalay-Beko Date: Mon, 4 May 2020 14:13:41 +0200 Subject: [PATCH 026/591] ZOOKEEPER-3817: suppress log4j SmtpAppender related CVE-2020-9488 Author: Mate Szalay-Beko Reviewers: Enrico Olivelli Closes #1346 from symat/ZOOKEEPER-3817 --- owaspSuppressions.xml | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/owaspSuppressions.xml b/owaspSuppressions.xml index ae94db494b9..2565f0db148 100644 --- a/owaspSuppressions.xml +++ b/owaspSuppressions.xml @@ -46,4 +46,9 @@ ZOOKEEPER-3677 --> CVE-2019-17571 + + + CVE-2020-9488 + From 391cb4aa6b54e19a028215e1340232a114c23ed3 Mon Sep 17 00:00:00 2001 From: Andor Molnar Date: Mon, 4 May 2020 14:17:33 +0200 Subject: [PATCH 027/591] ZOOKEEPER-3813: FileChangeWatcherTest is broken on Mac This patch will increase the FS wait timeout to give Mac OSX longer time to detect file modifications. Author: Andor Molnar Reviewers: Enrico Olivelli , Christopher Tubbs, Mate Szalay-Beko Closes #1345 from anmolnar/ZOOKEEPER-3813 --- .../zookeeper/common/FileChangeWatcherTest.java | 14 ++++++++------ 1 file changed, 8 insertions(+), 6 deletions(-) diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/common/FileChangeWatcherTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/common/FileChangeWatcherTest.java index bbac072833f..619ab1d9767 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/common/FileChangeWatcherTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/common/FileChangeWatcherTest.java @@ -45,6 +45,8 @@ public class FileChangeWatcherTest extends ZKTestCase { private static final Logger LOG = LoggerFactory.getLogger(FileChangeWatcherTest.class); + private static final long FS_TIMEOUT = 30000L; + @BeforeClass public static void createTempFile() throws IOException { tempDir = ClientBase.createEmptyTestDir(); @@ -87,7 +89,7 @@ public void testCallbackWorksOnFileChanges() throws IOException, InterruptedExce FileUtils.writeStringToFile(tempFile, "Hello world " + i + "\n", StandardCharsets.UTF_8, true); synchronized (events) { if (events.size() < i + 1) { - events.wait(3000L); + events.wait(FS_TIMEOUT); } assertEquals("Wrong number of events", i + 1, events.size()); WatchEvent event = events.get(i); @@ -128,7 +130,7 @@ public void testCallbackWorksOnFileTouched() throws IOException, InterruptedExce FileUtils.touch(tempFile); synchronized (events) { if (events.isEmpty()) { - events.wait(3000L); + events.wait(FS_TIMEOUT); } assertFalse(events.isEmpty()); WatchEvent event = events.get(0); @@ -162,7 +164,7 @@ public void testCallbackWorksOnFileAdded() throws IOException, InterruptedExcept tempFile2.deleteOnExit(); synchronized (events) { if (events.isEmpty()) { - events.wait(3000L); + events.wait(FS_TIMEOUT); } assertFalse(events.isEmpty()); WatchEvent event = events.get(0); @@ -201,7 +203,7 @@ public void testCallbackWorksOnFileDeleted() throws IOException, InterruptedExce tempFile.delete(); synchronized (events) { if (events.isEmpty()) { - events.wait(3000L); + events.wait(FS_TIMEOUT); } assertFalse(events.isEmpty()); WatchEvent event = events.get(0); @@ -239,14 +241,14 @@ public void testCallbackErrorDoesNotCrashWatcherThread() throws IOException, Int FileUtils.writeStringToFile(tempFile, "Hello world\n", StandardCharsets.UTF_8, true); synchronized (callCount) { while (callCount.get() == 0) { - callCount.wait(3000L); + callCount.wait(FS_TIMEOUT); } } LOG.info("Modifying file again"); FileUtils.writeStringToFile(tempFile, "Hello world again\n", StandardCharsets.UTF_8, true); synchronized (callCount) { if (callCount.get() == 1) { - callCount.wait(3000L); + callCount.wait(FS_TIMEOUT); } } // The value of callCount can exceed 1 only if the callback thread From bbe30290c2dd70bea20d3b08a0d705b10564c3b1 Mon Sep 17 00:00:00 2001 From: Christopher Tubbs Date: Mon, 4 May 2020 14:19:04 +0200 Subject: [PATCH 028/591] ZOOKEEPER-3801: Fix Jenkins link Author: Christopher Tubbs Reviewers: Enrico Olivelli , Mate Szalay-Beko Closes #1324 from ctubbsii/zk3801-fix-jenkins-link --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 7361dbae7c7..39e04805e8b 100755 --- a/pom.xml +++ b/pom.xml @@ -78,7 +78,7 @@ jenkins - https://builds.apache.org/view/S-Z/view/ZooKeeper/ + https://builds.apache.org/view/Z/view/ZooKeeper/ From 92f23ecd8159fd8d9c2fb8f34dd94f680ee6f712 Mon Sep 17 00:00:00 2001 From: Christopher Tubbs Date: Mon, 4 May 2020 14:20:27 +0200 Subject: [PATCH 029/591] ZOOKEEPER-3791: Miscellaneous Maven fixes * Fix fatjar and full-build profile problems with redundant module activation * Remove "activeByDefault" profiles that don't do anything useful * Make javadoc a bit quieter * Fix assembly plugin warning about `/` * Fix antrun warnings about tasks config renamed to target * Fix docs module packaging (doesn't need to be jar) * Fix zookeeper-it module by moving classes into standarad Maven directory layout and configuring surefire plugin to find the tests Author: Christopher Tubbs Reviewers: Enrico Olivelli , Norbert Kalmar Closes #1314 from ctubbsii/misc-maven-fixes --- .travis.yml | 2 +- pom.xml | 56 ++++++++++--------- zookeeper-assembly/pom.xml | 2 +- .../src/main/assembly/lib-package.xml | 2 +- zookeeper-client/pom.xml | 7 +-- zookeeper-client/zookeeper-client-c/pom.xml | 28 ++++------ zookeeper-contrib/pom.xml | 21 ++++--- zookeeper-docs/pom.xml | 9 +-- zookeeper-it/pom.xml | 29 ++++++---- .../java}/org/apache/zookeeper/BenchMain.java | 0 .../zookeeper/server/watch/WatchBench.java | 0 .../zookeeper/test/system/BaseSysTest.java | 0 .../test/system/DuplicateNameException.java | 0 .../zookeeper/test/system/GenerateLoad.java | 0 .../zookeeper/test/system/Instance.java | 0 .../test/system/InstanceContainer.java | 0 .../test/system/InstanceManager.java | 0 .../test/system/NoAssignmentException.java | 0 .../test/system/NoAvailableContainers.java | 0 .../test/system/QuorumPeerInstance.java | 0 .../zookeeper/test/system/SimpleClient.java | 0 .../zookeeper/test/system/SimpleSysTest.java | 0 zookeeper-jute/pom.xml | 1 - zookeeper-metrics-providers/pom.xml | 1 - .../zookeeper-prometheus-metrics/pom.xml | 1 - zookeeper-server/pom.xml | 1 - 26 files changed, 76 insertions(+), 84 deletions(-) rename zookeeper-it/src/{test/bench => main/java}/org/apache/zookeeper/BenchMain.java (100%) rename zookeeper-it/src/{test/bench => main/java}/org/apache/zookeeper/server/watch/WatchBench.java (100%) rename zookeeper-it/src/{test => main}/java/org/apache/zookeeper/test/system/BaseSysTest.java (100%) rename zookeeper-it/src/{test => main}/java/org/apache/zookeeper/test/system/DuplicateNameException.java (100%) rename zookeeper-it/src/{test => main}/java/org/apache/zookeeper/test/system/GenerateLoad.java (100%) rename zookeeper-it/src/{test => main}/java/org/apache/zookeeper/test/system/Instance.java (100%) rename zookeeper-it/src/{test => main}/java/org/apache/zookeeper/test/system/InstanceContainer.java (100%) rename zookeeper-it/src/{test => main}/java/org/apache/zookeeper/test/system/InstanceManager.java (100%) rename zookeeper-it/src/{test => main}/java/org/apache/zookeeper/test/system/NoAssignmentException.java (100%) rename zookeeper-it/src/{test => main}/java/org/apache/zookeeper/test/system/NoAvailableContainers.java (100%) rename zookeeper-it/src/{test => main}/java/org/apache/zookeeper/test/system/QuorumPeerInstance.java (100%) rename zookeeper-it/src/{test => main}/java/org/apache/zookeeper/test/system/SimpleClient.java (100%) rename zookeeper-it/src/{test => main}/java/org/apache/zookeeper/test/system/SimpleSysTest.java (100%) diff --git a/.travis.yml b/.travis.yml index fe2e952a1d9..7f320b9e005 100644 --- a/.travis.yml +++ b/.travis.yml @@ -26,7 +26,7 @@ addons: packages: - libcppunit-dev -script: mvn clean apache-rat:check install -DskipTests spotbugs:check checkstyle:check -Pfull-build +script: mvn clean apache-rat:check verify -DskipTests spotbugs:check checkstyle:check -Pfull-build branches: only: diff --git a/pom.xml b/pom.xml index 39e04805e8b..a3cb59d32ee 100755 --- a/pom.xml +++ b/pom.xml @@ -259,21 +259,19 @@ full-build zookeeper-contrib + + zookeeper-it + fatjar + zookeeper-contrib + zookeeper-it - zookeeper-contrib/zookeeper-contrib-fatjar - - java-build - - true - - apache-release @@ -446,7 +444,7 @@ 3.2.2 2.6 3.2.5 - 3.1.9 + 4.0.2 8.17 @@ -672,8 +670,9 @@ org.apache.maven.plugins maven-javadoc-plugin - 3.0.1 + 3.1.1 + true none @@ -685,7 +684,7 @@ org.apache.maven.plugins maven-release-plugin - 2.5.3 + 3.0.0-M1 org.apache.maven.plugins @@ -738,7 +737,7 @@ com.github.spotbugs spotbugs-maven-plugin - 3.1.9 + 4.0.0 excludeFindBugsFilter.xml @@ -958,23 +957,22 @@ src/main/resources/markdown/html/* src/main/resources/markdown/images/* - zookeeper-contrib-monitoring/JMX-RESOURCES - zookeeper-contrib-fatjar/src/main/resources/mainClasses - zookeeper-contrib-zkperl/Changes - zookeeper-contrib-zkperl/MANIFEST - zookeeper-contrib-zkpython/src/test/zoo.cfg - zookeeper-contrib-loggraph/src/main/resources/webapp/org/apache/zookeeper/graph/resources/* - src/main/resources/webapp/org/apache/zookeeper/graph/resources/* - src/main/java/com/nitido/utils/toaster/Toaster.java - TODO + **/JMX-RESOURCES + **/src/main/resources/mainClasses + **/Changes + **/MANIFEST + **/src/test/zoo.cfg + **/src/main/resources/webapp/org/apache/zookeeper/graph/resources/* + **/src/main/java/com/nitido/utils/toaster/Toaster.java + **/TODO **/acinclude.m4 **/aminclude.am - src/hashtable/* - include/winconfig.h - tests/wrappers.opt - tests/zoo.cfg - tests/wrappers-mt.opt + **/src/hashtable/* + **/include/winconfig.h + **/tests/wrappers.opt + **/tests/zoo.cfg + **/tests/wrappers-mt.opt **/c-doc.Doxyfile true @@ -986,8 +984,12 @@ false - clean install -DskipTests antrun:run@replace-cclient-files-during-release scm:add@add-cclient-files-during-release scm:checkin@commit-cclient-files-during-release - clean install -DskipTests antrun:run@replace-cclient-files-during-release scm:add@add-cclient-files-during-release scm:checkin@commit-cclient-files-during-release + clean verify -DskipTests antrun:run@replace-cclient-files-during-release scm:add@add-cclient-files-during-release scm:checkin@commit-cclient-files-during-release + clean verify -DskipTests antrun:run@replace-cclient-files-during-release scm:add@add-cclient-files-during-release scm:checkin@commit-cclient-files-during-release + false + true + true + release-@{project.version} diff --git a/zookeeper-assembly/pom.xml b/zookeeper-assembly/pom.xml index c2b97c0e6a3..8a2967fe616 100755 --- a/zookeeper-assembly/pom.xml +++ b/zookeeper-assembly/pom.xml @@ -24,7 +24,6 @@ org.apache.zookeeper parent 3.7.0-SNAPSHOT - .. zookeeper-assembly @@ -53,6 +52,7 @@ org.apache.zookeeper zookeeper-docs ${project.version} + pom org.apache.zookeeper diff --git a/zookeeper-assembly/src/main/assembly/lib-package.xml b/zookeeper-assembly/src/main/assembly/lib-package.xml index 81194f9e64d..40336cee8fd 100644 --- a/zookeeper-assembly/src/main/assembly/lib-package.xml +++ b/zookeeper-assembly/src/main/assembly/lib-package.xml @@ -54,7 +54,7 @@ LICENSE - / + . ${rw.file.permission} ${rwx.file.permission} diff --git a/zookeeper-client/pom.xml b/zookeeper-client/pom.xml index 307d60583fe..d42256c6155 100755 --- a/zookeeper-client/pom.xml +++ b/zookeeper-client/pom.xml @@ -24,7 +24,6 @@ org.apache.zookeeper parent 3.7.0-SNAPSHOT - .. zookeeper-client @@ -39,10 +38,6 @@ zookeeper-client-c - - java-build - - @@ -55,4 +50,4 @@ - \ No newline at end of file + diff --git a/zookeeper-client/zookeeper-client-c/pom.xml b/zookeeper-client/zookeeper-client-c/pom.xml index e8ea1aa7a97..b66712dcd74 100755 --- a/zookeeper-client/zookeeper-client-c/pom.xml +++ b/zookeeper-client/zookeeper-client-c/pom.xml @@ -24,14 +24,17 @@ org.apache.zookeeper zookeeper-client 3.7.0-SNAPSHOT - .. zookeeper-client-c - jar + pom Apache ZooKeeper - Client - C ZooKeeper c client + + + + c-test-coverage @@ -39,15 +42,6 @@ --enable-gcov - - no-c-test-coverage - - true - - - - - @@ -104,9 +98,9 @@ generate-sources generate-sources - + - + run @@ -116,7 +110,7 @@ build-c-client compile - + @@ -125,7 +119,7 @@ - + run @@ -137,7 +131,7 @@ ${skipTests} - + @@ -146,7 +140,7 @@ - + run diff --git a/zookeeper-contrib/pom.xml b/zookeeper-contrib/pom.xml index 0e2959d3be0..1d4cf74eca9 100755 --- a/zookeeper-contrib/pom.xml +++ b/zookeeper-contrib/pom.xml @@ -34,13 +34,20 @@ Contrib projects to Apache ZooKeeper - - zookeeper-contrib-loggraph - zookeeper-contrib-rest - zookeeper-contrib-zooinspector - + + true + + + full-build + + zookeeper-contrib-fatjar + zookeeper-contrib-loggraph + zookeeper-contrib-rest + zookeeper-contrib-zooinspector + + fatjar @@ -49,10 +56,6 @@ - - true - - diff --git a/zookeeper-docs/pom.xml b/zookeeper-docs/pom.xml index a46b880dbc6..75fd65a3f94 100644 --- a/zookeeper-docs/pom.xml +++ b/zookeeper-docs/pom.xml @@ -24,12 +24,12 @@ org.apache.zookeeper parent 3.7.0-SNAPSHOT - .. zookeeper-docs Apache ZooKeeper - Documentation Documentation + pom @@ -58,13 +58,6 @@ true - - org.apache.maven.plugins - maven-javadoc-plugin - - true - - diff --git a/zookeeper-it/pom.xml b/zookeeper-it/pom.xml index 163930fa3c5..10f8b9d14b0 100755 --- a/zookeeper-it/pom.xml +++ b/zookeeper-it/pom.xml @@ -33,6 +33,13 @@ ZooKeeper system tests + + + true + true + true + + org.apache.zookeeper @@ -63,15 +70,17 @@ - src - - - org.apache.maven.plugins - maven-javadoc-plugin - - src - - - + + + + org.apache.maven.plugins + maven-surefire-plugin + + ${project.basedir}/src/main/java/ + ${project.build.directory}/classes/ + + + + diff --git a/zookeeper-it/src/test/bench/org/apache/zookeeper/BenchMain.java b/zookeeper-it/src/main/java/org/apache/zookeeper/BenchMain.java similarity index 100% rename from zookeeper-it/src/test/bench/org/apache/zookeeper/BenchMain.java rename to zookeeper-it/src/main/java/org/apache/zookeeper/BenchMain.java diff --git a/zookeeper-it/src/test/bench/org/apache/zookeeper/server/watch/WatchBench.java b/zookeeper-it/src/main/java/org/apache/zookeeper/server/watch/WatchBench.java similarity index 100% rename from zookeeper-it/src/test/bench/org/apache/zookeeper/server/watch/WatchBench.java rename to zookeeper-it/src/main/java/org/apache/zookeeper/server/watch/WatchBench.java diff --git a/zookeeper-it/src/test/java/org/apache/zookeeper/test/system/BaseSysTest.java b/zookeeper-it/src/main/java/org/apache/zookeeper/test/system/BaseSysTest.java similarity index 100% rename from zookeeper-it/src/test/java/org/apache/zookeeper/test/system/BaseSysTest.java rename to zookeeper-it/src/main/java/org/apache/zookeeper/test/system/BaseSysTest.java diff --git a/zookeeper-it/src/test/java/org/apache/zookeeper/test/system/DuplicateNameException.java b/zookeeper-it/src/main/java/org/apache/zookeeper/test/system/DuplicateNameException.java similarity index 100% rename from zookeeper-it/src/test/java/org/apache/zookeeper/test/system/DuplicateNameException.java rename to zookeeper-it/src/main/java/org/apache/zookeeper/test/system/DuplicateNameException.java diff --git a/zookeeper-it/src/test/java/org/apache/zookeeper/test/system/GenerateLoad.java b/zookeeper-it/src/main/java/org/apache/zookeeper/test/system/GenerateLoad.java similarity index 100% rename from zookeeper-it/src/test/java/org/apache/zookeeper/test/system/GenerateLoad.java rename to zookeeper-it/src/main/java/org/apache/zookeeper/test/system/GenerateLoad.java diff --git a/zookeeper-it/src/test/java/org/apache/zookeeper/test/system/Instance.java b/zookeeper-it/src/main/java/org/apache/zookeeper/test/system/Instance.java similarity index 100% rename from zookeeper-it/src/test/java/org/apache/zookeeper/test/system/Instance.java rename to zookeeper-it/src/main/java/org/apache/zookeeper/test/system/Instance.java diff --git a/zookeeper-it/src/test/java/org/apache/zookeeper/test/system/InstanceContainer.java b/zookeeper-it/src/main/java/org/apache/zookeeper/test/system/InstanceContainer.java similarity index 100% rename from zookeeper-it/src/test/java/org/apache/zookeeper/test/system/InstanceContainer.java rename to zookeeper-it/src/main/java/org/apache/zookeeper/test/system/InstanceContainer.java diff --git a/zookeeper-it/src/test/java/org/apache/zookeeper/test/system/InstanceManager.java b/zookeeper-it/src/main/java/org/apache/zookeeper/test/system/InstanceManager.java similarity index 100% rename from zookeeper-it/src/test/java/org/apache/zookeeper/test/system/InstanceManager.java rename to zookeeper-it/src/main/java/org/apache/zookeeper/test/system/InstanceManager.java diff --git a/zookeeper-it/src/test/java/org/apache/zookeeper/test/system/NoAssignmentException.java b/zookeeper-it/src/main/java/org/apache/zookeeper/test/system/NoAssignmentException.java similarity index 100% rename from zookeeper-it/src/test/java/org/apache/zookeeper/test/system/NoAssignmentException.java rename to zookeeper-it/src/main/java/org/apache/zookeeper/test/system/NoAssignmentException.java diff --git a/zookeeper-it/src/test/java/org/apache/zookeeper/test/system/NoAvailableContainers.java b/zookeeper-it/src/main/java/org/apache/zookeeper/test/system/NoAvailableContainers.java similarity index 100% rename from zookeeper-it/src/test/java/org/apache/zookeeper/test/system/NoAvailableContainers.java rename to zookeeper-it/src/main/java/org/apache/zookeeper/test/system/NoAvailableContainers.java diff --git a/zookeeper-it/src/test/java/org/apache/zookeeper/test/system/QuorumPeerInstance.java b/zookeeper-it/src/main/java/org/apache/zookeeper/test/system/QuorumPeerInstance.java similarity index 100% rename from zookeeper-it/src/test/java/org/apache/zookeeper/test/system/QuorumPeerInstance.java rename to zookeeper-it/src/main/java/org/apache/zookeeper/test/system/QuorumPeerInstance.java diff --git a/zookeeper-it/src/test/java/org/apache/zookeeper/test/system/SimpleClient.java b/zookeeper-it/src/main/java/org/apache/zookeeper/test/system/SimpleClient.java similarity index 100% rename from zookeeper-it/src/test/java/org/apache/zookeeper/test/system/SimpleClient.java rename to zookeeper-it/src/main/java/org/apache/zookeeper/test/system/SimpleClient.java diff --git a/zookeeper-it/src/test/java/org/apache/zookeeper/test/system/SimpleSysTest.java b/zookeeper-it/src/main/java/org/apache/zookeeper/test/system/SimpleSysTest.java similarity index 100% rename from zookeeper-it/src/test/java/org/apache/zookeeper/test/system/SimpleSysTest.java rename to zookeeper-it/src/main/java/org/apache/zookeeper/test/system/SimpleSysTest.java diff --git a/zookeeper-jute/pom.xml b/zookeeper-jute/pom.xml index 13fe7c80681..129ba12ce08 100755 --- a/zookeeper-jute/pom.xml +++ b/zookeeper-jute/pom.xml @@ -24,7 +24,6 @@ org.apache.zookeeper parent 3.7.0-SNAPSHOT - .. zookeeper-jute diff --git a/zookeeper-metrics-providers/pom.xml b/zookeeper-metrics-providers/pom.xml index f07551bd50b..a9b18d98999 100755 --- a/zookeeper-metrics-providers/pom.xml +++ b/zookeeper-metrics-providers/pom.xml @@ -24,7 +24,6 @@ org.apache.zookeeper parent 3.7.0-SNAPSHOT - .. zookeeper-metrics-providers diff --git a/zookeeper-metrics-providers/zookeeper-prometheus-metrics/pom.xml b/zookeeper-metrics-providers/zookeeper-prometheus-metrics/pom.xml index c6b5f133a34..68e7eb62cbd 100755 --- a/zookeeper-metrics-providers/zookeeper-prometheus-metrics/pom.xml +++ b/zookeeper-metrics-providers/zookeeper-prometheus-metrics/pom.xml @@ -24,7 +24,6 @@ org.apache.zookeeper zookeeper-metrics-providers 3.7.0-SNAPSHOT - .. zookeeper-prometheus-metrics diff --git a/zookeeper-server/pom.xml b/zookeeper-server/pom.xml index 117a77b8ec7..90f6151584e 100755 --- a/zookeeper-server/pom.xml +++ b/zookeeper-server/pom.xml @@ -24,7 +24,6 @@ org.apache.zookeeper parent 3.7.0-SNAPSHOT - .. zookeeper From 44de549657a539dcedba58d56e67b8c906c89580 Mon Sep 17 00:00:00 2001 From: lingjinjiang Date: Tue, 5 May 2020 09:18:18 +0200 Subject: [PATCH 030/591] ZOOKEEPER-3772: JettyAdminServer should not allow Http TRACE method A nessus scan pinged the cluster after I upgrade my zookeeper cluster, because the 8080 port of JettyAdminServer allows Http TRACE method. Author: lingjinjiang Reviewers: Mate Szalay-Beko , Andor Molnar , Norbert Kalmar Closes #1296 from lingjinjiang/master --- .../server/admin/JettyAdminServer.java | 22 +++++++++++++++++++ .../server/admin/JettyAdminServerTest.java | 20 +++++++++++++++++ 2 files changed, 42 insertions(+) diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/admin/JettyAdminServer.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/admin/JettyAdminServer.java index 27242436cff..6845f233e24 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/admin/JettyAdminServer.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/admin/JettyAdminServer.java @@ -34,6 +34,8 @@ import org.apache.zookeeper.common.X509Util; import org.apache.zookeeper.server.ZooKeeperServer; import org.eclipse.jetty.http.HttpVersion; +import org.eclipse.jetty.security.ConstraintMapping; +import org.eclipse.jetty.security.ConstraintSecurityHandler; import org.eclipse.jetty.server.HttpConfiguration; import org.eclipse.jetty.server.HttpConnectionFactory; import org.eclipse.jetty.server.SecureRequestCustomizer; @@ -41,6 +43,7 @@ import org.eclipse.jetty.server.ServerConnector; import org.eclipse.jetty.servlet.ServletContextHandler; import org.eclipse.jetty.servlet.ServletHolder; +import org.eclipse.jetty.util.security.Constraint; import org.eclipse.jetty.util.ssl.SslContextFactory; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -153,6 +156,7 @@ public JettyAdminServer( ServletContextHandler context = new ServletContextHandler(ServletContextHandler.SESSIONS); context.setContextPath("/*"); + constrainTraceMethod(context); server.setHandler(context); context.addServlet(new ServletHolder(new CommandServlet()), commandUrl + "/*"); @@ -266,4 +270,22 @@ private List commandLinks() { return links; } + /** + * Add constraint to a given context to disallow TRACE method + * @param ctxHandler the context to modify + */ + private void constrainTraceMethod(ServletContextHandler ctxHandler) { + Constraint c = new Constraint(); + c.setAuthenticate(true); + + ConstraintMapping cmt = new ConstraintMapping(); + cmt.setConstraint(c); + cmt.setMethod("TRACE"); + cmt.setPathSpec("/*"); + + ConstraintSecurityHandler securityHandler = new ConstraintSecurityHandler(); + securityHandler.setConstraintMappings(new ConstraintMapping[] {cmt}); + + ctxHandler.setSecurityHandler(securityHandler); + } } diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/admin/JettyAdminServerTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/admin/JettyAdminServerTest.java index 525c1c971f9..4e77a6bb9c7 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/admin/JettyAdminServerTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/admin/JettyAdminServerTest.java @@ -18,11 +18,13 @@ package org.apache.zookeeper.server.admin; +import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; import java.io.BufferedReader; import java.io.File; import java.io.IOException; import java.io.InputStreamReader; +import java.net.HttpURLConnection; import java.net.URL; import java.security.GeneralSecurityException; import java.security.Security; @@ -152,6 +154,7 @@ public void testJettyAdminServer() throws AdminServerException, IOException, SSL try { server.start(); queryAdminServer(jettyAdminPort); + traceAdminServer(jettyAdminPort); } finally { server.shutdown(); } @@ -262,4 +265,21 @@ private void queryAdminServer(String urlStr, boolean encrypted) throws IOExcepti assertTrue(line.length() > 0); } + /** + * Using TRACE method to visit admin server + */ + private void traceAdminServer(int port) throws IOException { + traceAdminServer(String.format(URL_FORMAT, port)); + traceAdminServer(String.format(HTTPS_URL_FORMAT, port)); + } + + /** + * Using TRACE method to visit admin server, the response should be 403 forbidden + */ + private void traceAdminServer(String urlStr) throws IOException { + HttpURLConnection conn = (HttpURLConnection) new URL(urlStr).openConnection(); + conn.setRequestMethod("TRACE"); + conn.connect(); + assertEquals(HttpURLConnection.HTTP_FORBIDDEN, conn.getResponseCode()); + } } From 05cd214a0cc9c870de373b54cfeb47a2a75efd28 Mon Sep 17 00:00:00 2001 From: Mayank Tuteja Date: Tue, 5 May 2020 19:20:35 +0000 Subject: [PATCH 031/591] ZOOKEEPER-3796: Skip Learner Request made to ObserverMaster from going ... to next processor Author: mayank99 Author: mayank99fb <63824268+mayank99fb@users.noreply.github.com> Reviewers: Andor Molnar , Enrico Olivelli , Mate Szalay-Beko Closes #1322 from mayank99fb/ZOOKEEPER-3796 --- .../main/resources/markdown/zookeeperAdmin.md | 9 + .../org/apache/zookeeper/server/Request.java | 5 + .../zookeeper/server/ServerMetrics.java | 6 + .../quorum/FollowerRequestProcessor.java | 19 +- .../server/FollowerRequestProcessorTest.java | 89 ++++++++ .../zookeeper/test/ObserverMasterTest.java | 196 +----------------- .../test/ObserverMasterTestBase.java | 120 +++++++++++ 7 files changed, 249 insertions(+), 195 deletions(-) create mode 100644 zookeeper-server/src/test/java/org/apache/zookeeper/server/FollowerRequestProcessorTest.java create mode 100644 zookeeper-server/src/test/java/org/apache/zookeeper/test/ObserverMasterTestBase.java diff --git a/zookeeper-docs/src/main/resources/markdown/zookeeperAdmin.md b/zookeeper-docs/src/main/resources/markdown/zookeeperAdmin.md index 6c616d1af68..2a9d7d3c731 100644 --- a/zookeeper-docs/src/main/resources/markdown/zookeeperAdmin.md +++ b/zookeeper-docs/src/main/resources/markdown/zookeeperAdmin.md @@ -1653,6 +1653,15 @@ New features that are currently considered experimental. values and see changes from other clients. See ZOOKEEPER-784 for more details. +* *zookeeper.follower.skipLearnerRequestToNextProcessor* : + (Java system property: **zookeeper.follower.skipLearnerRequestToNextProcessor**) + When our cluster has observers which are connected with ObserverMaster, then turning on this flag might help + you reduce some memory pressure on the Observer Master. If your cluster doesn't have any observers or + they are not connected with ObserverMaster or your Observer's don't make much writes, then using this flag + won't help you. + Currently the change here is guarded behind the flag to help us get more confidence around the memory gains. + In Long run, we might want to remove this flag and set its behavior as the default codepath. + #### Unsafe Options diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/Request.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/Request.java index 43a68ac7aa2..42964712bb0 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/Request.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/Request.java @@ -27,6 +27,7 @@ import org.apache.zookeeper.data.Id; import org.apache.zookeeper.metrics.Summary; import org.apache.zookeeper.metrics.SummarySet; +import org.apache.zookeeper.server.quorum.LearnerHandler; import org.apache.zookeeper.server.quorum.flexible.QuorumVerifier; import org.apache.zookeeper.server.util.AuthUtil; import org.apache.zookeeper.txn.TxnDigest; @@ -490,4 +491,8 @@ public TxnDigest getTxnDigest() { public void setTxnDigest(TxnDigest txnDigest) { this.txnDigest = txnDigest; } + + public boolean isFromLearner() { + return owner instanceof LearnerHandler; + } } diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/ServerMetrics.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/ServerMetrics.java index 7ea701081e3..36a65df801f 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/ServerMetrics.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/ServerMetrics.java @@ -237,7 +237,10 @@ private ServerMetrics(MetricsProvider metricsProvider) { CNXN_CLOSED_WITHOUT_ZK_SERVER_RUNNING = metricsContext.getCounter("cnxn_closed_without_zk_server_running"); + SKIP_LEARNER_REQUEST_TO_NEXT_PROCESSOR_COUNT = metricsContext.getCounter("skip_learner_request_to_next_processor_count"); + SOCKET_CLOSING_TIME = metricsContext.getSummary("socket_closing_time", DetailLevel.BASIC); + } /** @@ -458,8 +461,11 @@ private ServerMetrics(MetricsProvider metricsProvider) { public final Counter CNXN_CLOSED_WITHOUT_ZK_SERVER_RUNNING; + public final Counter SKIP_LEARNER_REQUEST_TO_NEXT_PROCESSOR_COUNT; + public final Summary SOCKET_CLOSING_TIME; + private final MetricsProvider metricsProvider; public void resetAll() { diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/FollowerRequestProcessor.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/FollowerRequestProcessor.java index db51aee495d..90c4d490344 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/FollowerRequestProcessor.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/FollowerRequestProcessor.java @@ -24,6 +24,7 @@ import org.apache.zookeeper.ZooDefs.OpCode; 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.ZooTrace; import org.apache.zookeeper.txn.ErrorTxn; @@ -38,6 +39,10 @@ public class FollowerRequestProcessor extends ZooKeeperCriticalThread implements private static final Logger LOG = LoggerFactory.getLogger(FollowerRequestProcessor.class); + public static final String SKIP_LEARNER_REQUEST_TO_NEXT_PROCESSOR = "zookeeper.follower.skipLearnerRequestToNextProcessor"; + + private final boolean skipLearnerRequestToNextProcessor; + FollowerZooKeeperServer zks; RequestProcessor nextProcessor; @@ -50,6 +55,9 @@ public FollowerRequestProcessor(FollowerZooKeeperServer zks, RequestProcessor ne super("FollowerRequestProcessor:" + zks.getServerId(), zks.getZooKeeperServerListener()); this.zks = zks; this.nextProcessor = nextProcessor; + this.skipLearnerRequestToNextProcessor = Boolean.getBoolean(SKIP_LEARNER_REQUEST_TO_NEXT_PROCESSOR); + LOG.info("Initialized FollowerRequestProcessor with {} as {}", SKIP_LEARNER_REQUEST_TO_NEXT_PROCESSOR, + skipLearnerRequestToNextProcessor); } @Override @@ -72,7 +80,8 @@ public void run() { // We want to queue the request to be processed before we submit // the request to the leader so that we are ready to receive // the response - nextProcessor.processRequest(request); + maybeSendRequestToNextProcessor(request); + if (request.isThrottled()) { continue; } @@ -115,6 +124,14 @@ public void run() { LOG.info("FollowerRequestProcessor exited loop!"); } + private void maybeSendRequestToNextProcessor(Request request) throws RequestProcessorException { + if (skipLearnerRequestToNextProcessor && request.isFromLearner()) { + ServerMetrics.getMetrics().SKIP_LEARNER_REQUEST_TO_NEXT_PROCESSOR_COUNT.add(1); + } else { + nextProcessor.processRequest(request); + } + } + public void processRequest(Request request) { processRequest(request, true); } diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/FollowerRequestProcessorTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/FollowerRequestProcessorTest.java new file mode 100644 index 00000000000..dd81c77eec5 --- /dev/null +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/FollowerRequestProcessorTest.java @@ -0,0 +1,89 @@ +/* + * 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.test.ClientBase.CONNECTION_TIMEOUT; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; +import org.apache.zookeeper.CreateMode; +import org.apache.zookeeper.PortAssignment; +import org.apache.zookeeper.ZooDefs.Ids; +import org.apache.zookeeper.ZooKeeper; +import org.apache.zookeeper.ZooKeeper.States; +import org.apache.zookeeper.server.ServerMetrics; +import org.apache.zookeeper.server.util.PortForwarder; +import org.apache.zookeeper.test.ClientBase; +import org.apache.zookeeper.test.ObserverMasterTestBase; +import org.junit.After; +import org.junit.Test; + +public class FollowerRequestProcessorTest extends ObserverMasterTestBase { + + private PortForwarder forwarder; + + @Test + public void testFollowerRequestProcessorSkipsLearnerRequestToNextProcessor() throws Exception { + setupTestObserverServer("true"); + + zk.create("/testFollowerSkipNextAProcessor", "test".getBytes(), Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); + + assertEquals("test", new String(zk.getData("/testFollowerSkipNextAProcessor", null, null))); + assertEquals(1L, ServerMetrics.getMetrics().SKIP_LEARNER_REQUEST_TO_NEXT_PROCESSOR_COUNT.get()); + } + + @Test + public void testFollowerRequestProcessorSendsLearnerRequestToNextProcessor() throws Exception { + setupTestObserverServer("false"); + + zk.create("/testFollowerSkipNextAProcessor", "test".getBytes(), Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); + + assertEquals("test", new String(zk.getData("/testFollowerSkipNextAProcessor", null, null))); + assertEquals(0L, ServerMetrics.getMetrics().SKIP_LEARNER_REQUEST_TO_NEXT_PROCESSOR_COUNT.get()); + } + + private void setupTestObserverServer(String skipLearnerRequestToNextProcessor) throws Exception { + System.setProperty(FollowerRequestProcessor.SKIP_LEARNER_REQUEST_TO_NEXT_PROCESSOR, skipLearnerRequestToNextProcessor); + + // Setup Ensemble with observer master port so that observer connects with Observer master and not the leader + final int OM_PROXY_PORT = PortAssignment.unique(); + forwarder = setUp(OM_PROXY_PORT, true); + + q3.start(); + assertTrue( + "waiting for server 3 being up", + ClientBase.waitForServerUp("127.0.0.1:" + CLIENT_PORT_OBS, CONNECTION_TIMEOUT)); + + // Connect with observer zookeeper + zk = new ZooKeeper("127.0.0.1:" + CLIENT_PORT_OBS, ClientBase.CONNECTION_TIMEOUT, this); + waitForOne(zk, States.CONNECTED); + + // Clear all service metrics collected so far + ServerMetrics.getMetrics().resetAll(); + } + + @After + public void cleanup() throws Exception { + System.setProperty(FollowerRequestProcessor.SKIP_LEARNER_REQUEST_TO_NEXT_PROCESSOR, "false"); + + shutdown(); + if (forwarder != null) { + forwarder.shutdown(); + } + } +} diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/test/ObserverMasterTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/test/ObserverMasterTest.java index 620953a2db5..cfac042b7ac 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/test/ObserverMasterTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/test/ObserverMasterTest.java @@ -50,8 +50,6 @@ import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.KeeperException.ConnectionLossException; import org.apache.zookeeper.PortAssignment; -import org.apache.zookeeper.WatchedEvent; -import org.apache.zookeeper.Watcher; import org.apache.zookeeper.Watcher.Event.KeeperState; import org.apache.zookeeper.ZooDefs.Ids; import org.apache.zookeeper.ZooKeeper; @@ -60,10 +58,7 @@ import org.apache.zookeeper.jmx.MBeanRegistry; import org.apache.zookeeper.jmx.ZKMBeanInfo; import org.apache.zookeeper.server.admin.Commands; -import org.apache.zookeeper.server.quorum.DelayRequestProcessor; -import org.apache.zookeeper.server.quorum.FollowerZooKeeperServer; import org.apache.zookeeper.server.quorum.QuorumPeerConfig; -import org.apache.zookeeper.server.quorum.QuorumPeerTestBase; import org.apache.zookeeper.server.util.PortForwarder; import org.junit.Test; import org.junit.runner.RunWith; @@ -72,7 +67,7 @@ import org.slf4j.LoggerFactory; @RunWith(Parameterized.class) -public class ObserverMasterTest extends QuorumPeerTestBase implements Watcher { +public class ObserverMasterTest extends ObserverMasterTestBase { protected static final Logger LOG = LoggerFactory.getLogger(ObserverMasterTest.class); @@ -87,184 +82,8 @@ public static List data() { private Boolean testObserverMaster; - private CountDownLatch latch; - ZooKeeper zk; - private WatchedEvent lastEvent = null; - - private int CLIENT_PORT_QP1; - private int CLIENT_PORT_QP2; - private int CLIENT_PORT_OBS; - private int OM_PORT; - private MainThread q1; - private MainThread q2; - private MainThread q3; - private PortForwarder setUp(final int omProxyPort) throws IOException { - ClientBase.setupTestEnv(); - - final int PORT_QP1 = PortAssignment.unique(); - final int PORT_QP2 = PortAssignment.unique(); - final int PORT_OBS = PortAssignment.unique(); - final int PORT_QP_LE1 = PortAssignment.unique(); - final int PORT_QP_LE2 = PortAssignment.unique(); - final int PORT_OBS_LE = PortAssignment.unique(); - - CLIENT_PORT_QP1 = PortAssignment.unique(); - CLIENT_PORT_QP2 = PortAssignment.unique(); - CLIENT_PORT_OBS = PortAssignment.unique(); - - OM_PORT = PortAssignment.unique(); - - String quorumCfgSection = "server.1=127.0.0.1:" + (PORT_QP1) + ":" + (PORT_QP_LE1) + ";" + CLIENT_PORT_QP1 - + "\nserver.2=127.0.0.1:" + (PORT_QP2) + ":" + (PORT_QP_LE2) + ";" + CLIENT_PORT_QP2 - + "\nserver.3=127.0.0.1:" + (PORT_OBS) + ":" + (PORT_OBS_LE) + ":observer" + ";" + CLIENT_PORT_OBS; - String extraCfgs = testObserverMaster - ? String.format("observerMasterPort=%d%n", OM_PORT) - : ""; - String extraCfgsObs = testObserverMaster - ? String.format("observerMasterPort=%d%n", omProxyPort <= 0 ? OM_PORT : omProxyPort) - : ""; - - PortForwarder forwarder = null; - if (testObserverMaster && omProxyPort >= 0) { - forwarder = new PortForwarder(omProxyPort, OM_PORT); - } - - q1 = new MainThread(1, CLIENT_PORT_QP1, quorumCfgSection, extraCfgs); - q2 = new MainThread(2, CLIENT_PORT_QP2, quorumCfgSection, extraCfgs); - q3 = new MainThread(3, CLIENT_PORT_OBS, quorumCfgSection, extraCfgsObs); - q1.start(); - q2.start(); - assertTrue( - "waiting for server 1 being up", - ClientBase.waitForServerUp("127.0.0.1:" + CLIENT_PORT_QP1, CONNECTION_TIMEOUT)); - assertTrue( - "waiting for server 2 being up", - ClientBase.waitForServerUp("127.0.0.1:" + CLIENT_PORT_QP2, CONNECTION_TIMEOUT)); - return forwarder; - } - - private void shutdown() throws InterruptedException { - LOG.info("Shutting down all servers"); - zk.close(); - - q1.shutdown(); - q2.shutdown(); - q3.shutdown(); - - assertTrue( - "Waiting for server 1 to shut down", - ClientBase.waitForServerDown("127.0.0.1:" + CLIENT_PORT_QP1, ClientBase.CONNECTION_TIMEOUT)); - assertTrue( - "Waiting for server 2 to shut down", - ClientBase.waitForServerDown("127.0.0.1:" + CLIENT_PORT_QP2, ClientBase.CONNECTION_TIMEOUT)); - assertTrue( - "Waiting for server 3 to shut down", - ClientBase.waitForServerDown("127.0.0.1:" + CLIENT_PORT_OBS, ClientBase.CONNECTION_TIMEOUT)); - } - - @Test - public void testLaggingObserverMaster() throws Exception { - final int OM_PROXY_PORT = PortAssignment.unique(); - PortForwarder forwarder = setUp(OM_PROXY_PORT); - - // find the leader and observer master - int leaderPort; - MainThread leader; - MainThread follower; - if (q1.getQuorumPeer().leader != null) { - leaderPort = CLIENT_PORT_QP1; - leader = q1; - follower = q2; - } else if (q2.getQuorumPeer().leader != null) { - leaderPort = CLIENT_PORT_QP2; - leader = q2; - follower = q1; - } else { - throw new RuntimeException("No leader"); - } - - // ensure the observer master has commits in the queue before observer sync - zk = new ZooKeeper("127.0.0.1:" + leaderPort, ClientBase.CONNECTION_TIMEOUT, this); - for (int i = 0; i < 10; i++) { - zk.create("/bulk" + i, ("initial data of some size").getBytes(), Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); - } - zk.close(); - - q3.start(); - assertTrue( - "waiting for server 3 being up", - ClientBase.waitForServerUp("127.0.0.1:" + CLIENT_PORT_OBS, CONNECTION_TIMEOUT)); - - latch = new CountDownLatch(1); - zk = new ZooKeeper("127.0.0.1:" + leaderPort, ClientBase.CONNECTION_TIMEOUT, this); - latch.await(); - assertEquals(zk.getState(), States.CONNECTED); - - zk.create("/init", "first".getBytes(), Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); - final long lastLoggedZxid = leader.getQuorumPeer().getLastLoggedZxid(); - - // wait for change to propagate - waitFor("Timeout waiting for observer sync", new WaitForCondition() { - public boolean evaluate() { - return lastLoggedZxid == q3.getQuorumPeer().getLastLoggedZxid(); - } - }, 30); - - // simulate network fault - if (forwarder != null) { - forwarder.shutdown(); - } - - for (int i = 0; i < 10; i++) { - zk.create("/basic" + i, "second".getBytes(), Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); - } - - DelayRequestProcessor delayRequestProcessor = null; - if (testObserverMaster) { - FollowerZooKeeperServer followerZooKeeperServer = (FollowerZooKeeperServer) follower.getQuorumPeer().getActiveServer(); - delayRequestProcessor = DelayRequestProcessor.injectDelayRequestProcessor(followerZooKeeperServer); - } - - zk.create("/target1", "third".getBytes(), Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); - zk.create("/target2", "third".getBytes(), Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); - - LOG.info( - "observer zxid {}{} leader zxid {}", - Long.toHexString(q3.getQuorumPeer().getLastLoggedZxid()), - (testObserverMaster ? "" : " observer master zxid " + Long.toHexString(follower.getQuorumPeer().getLastLoggedZxid())), - Long.toHexString(leader.getQuorumPeer().getLastLoggedZxid())); - - // restore network - forwarder = testObserverMaster ? new PortForwarder(OM_PROXY_PORT, OM_PORT) : null; - - assertTrue( - "waiting for server 3 being up", - ClientBase.waitForServerUp("127.0.0.1:" + CLIENT_PORT_OBS, CONNECTION_TIMEOUT)); - assertNotNull("Leader switched", leader.getQuorumPeer().leader); - - if (delayRequestProcessor != null) { - delayRequestProcessor.unblockQueue(); - } - - latch = new CountDownLatch(1); - ZooKeeper obsZk = new ZooKeeper("127.0.0.1:" + CLIENT_PORT_OBS, ClientBase.CONNECTION_TIMEOUT, this); - latch.await(); - zk.create("/finalop", "fourth".getBytes(), Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); - - assertEquals("first", new String(obsZk.getData("/init", null, null))); - assertEquals("third", new String(obsZk.getData("/target1", null, null))); - - obsZk.close(); - shutdown(); - - try { - if (forwarder != null) { - forwarder.shutdown(); - } - } catch (Exception e) { - // ignore - } + return setUp(omProxyPort, testObserverMaster); } /** @@ -678,17 +497,6 @@ public void testDynamicReconfig() throws InterruptedException, IOException, Keep s1.shutdown(); } - /** - * Implementation of watcher interface. - */ - public void process(WatchedEvent event) { - lastEvent = event; - if (latch != null) { - latch.countDown(); - } - LOG.info("Latch got event :: {}", event); - } - class AsyncWriter implements Runnable { private final ZooKeeper client; diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/test/ObserverMasterTestBase.java b/zookeeper-server/src/test/java/org/apache/zookeeper/test/ObserverMasterTestBase.java new file mode 100644 index 00000000000..505378a5f1e --- /dev/null +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/test/ObserverMasterTestBase.java @@ -0,0 +1,120 @@ +/* + * 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.test; + +import static org.apache.zookeeper.test.ClientBase.CONNECTION_TIMEOUT; +import static org.junit.Assert.assertTrue; +import java.io.IOException; +import java.util.concurrent.CountDownLatch; +import org.apache.zookeeper.PortAssignment; +import org.apache.zookeeper.WatchedEvent; +import org.apache.zookeeper.Watcher; +import org.apache.zookeeper.ZooKeeper; +import org.apache.zookeeper.server.quorum.QuorumPeerTestBase; +import org.apache.zookeeper.server.util.PortForwarder; + +public class ObserverMasterTestBase extends QuorumPeerTestBase implements Watcher { + + protected CountDownLatch latch; + protected ZooKeeper zk; + protected int CLIENT_PORT_QP1; + protected int CLIENT_PORT_QP2; + protected int CLIENT_PORT_OBS; + protected int OM_PORT; + protected MainThread q1; + protected MainThread q2; + protected MainThread q3; + protected WatchedEvent lastEvent = null; + + protected PortForwarder setUp(final int omProxyPort, final Boolean testObserverMaster) throws IOException { + ClientBase.setupTestEnv(); + final int PORT_QP1 = PortAssignment.unique(); + final int PORT_QP2 = PortAssignment.unique(); + final int PORT_OBS = PortAssignment.unique(); + final int PORT_QP_LE1 = PortAssignment.unique(); + final int PORT_QP_LE2 = PortAssignment.unique(); + final int PORT_OBS_LE = PortAssignment.unique(); + + CLIENT_PORT_QP1 = PortAssignment.unique(); + CLIENT_PORT_QP2 = PortAssignment.unique(); + CLIENT_PORT_OBS = PortAssignment.unique(); + + OM_PORT = PortAssignment.unique(); + + String quorumCfgSection = + "server.1=127.0.0.1:" + (PORT_QP1) + + ":" + (PORT_QP_LE1) + ";" + CLIENT_PORT_QP1 + + "\nserver.2=127.0.0.1:" + (PORT_QP2) + + ":" + (PORT_QP_LE2) + ";" + CLIENT_PORT_QP2 + + "\nserver.3=127.0.0.1:" + (PORT_OBS) + + ":" + (PORT_OBS_LE) + ":observer" + ";" + CLIENT_PORT_OBS; + + String extraCfgs = testObserverMaster ? String.format("observerMasterPort=%d%n", OM_PORT) : ""; + String extraCfgsObs = testObserverMaster ? String.format("observerMasterPort=%d%n", omProxyPort <= 0 ? OM_PORT : omProxyPort) : ""; + + PortForwarder forwarder = null; + if (testObserverMaster && omProxyPort >= 0) { + forwarder = new PortForwarder(omProxyPort, OM_PORT); + } + + q1 = new MainThread(1, CLIENT_PORT_QP1, quorumCfgSection, extraCfgs); + q2 = new MainThread(2, CLIENT_PORT_QP2, quorumCfgSection, extraCfgs); + q3 = new MainThread(3, CLIENT_PORT_OBS, quorumCfgSection, extraCfgsObs); + q1.start(); + q2.start(); + assertTrue( + "waiting for server 1 being up", + ClientBase.waitForServerUp("127.0.0.1:" + CLIENT_PORT_QP1, CONNECTION_TIMEOUT)); + assertTrue( + "waiting for server 2 being up", + ClientBase.waitForServerUp("127.0.0.1:" + CLIENT_PORT_QP2, CONNECTION_TIMEOUT)); + return forwarder; + } + + protected void shutdown() throws InterruptedException { + LOG.info("Shutting down all servers"); + + zk.close(); + + q1.shutdown(); + q2.shutdown(); + q3.shutdown(); + + assertTrue( + "Waiting for server 1 to shut down", + ClientBase.waitForServerDown("127.0.0.1:" + CLIENT_PORT_QP1, ClientBase.CONNECTION_TIMEOUT)); + assertTrue( + "Waiting for server 2 to shut down", + ClientBase.waitForServerDown("127.0.0.1:" + CLIENT_PORT_QP2, ClientBase.CONNECTION_TIMEOUT)); + assertTrue( + "Waiting for server 3 to shut down", + ClientBase.waitForServerDown("127.0.0.1:" + CLIENT_PORT_OBS, ClientBase.CONNECTION_TIMEOUT)); + } + + /** + * Implementation of watcher interface. + */ + public void process(WatchedEvent event) { + lastEvent = event; + if (latch != null) { + latch.countDown(); + } + LOG.info("Latch got event :: {}", event); + } +} From ae7f72d31e4dc9c6e307b1c14271e74950b022dc Mon Sep 17 00:00:00 2001 From: javhu <13621135641@163.com> Date: Thu, 7 May 2020 14:54:59 -0700 Subject: [PATCH 032/591] ZOOKEEPER-3805: NIOServerCnxnFactory static block has no used code the jvm bug: JDK-6427854 : (se) NullPointerException in Selector.open() has been fixed in JDK7 b08 Author: javhu <13621135641@163.com> Reviewers: ctubbsii@apache.org, symat@apache.org, eolivelli@apache.org, fangmin@apache.org Closes #1338 from Javhu/ZOOKEEPER-3805 --- .../apache/zookeeper/server/NIOServerCnxnFactory.java | 10 ---------- 1 file changed, 10 deletions(-) diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/NIOServerCnxnFactory.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/NIOServerCnxnFactory.java index de29fa2424d..558b93be59b 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/NIOServerCnxnFactory.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/NIOServerCnxnFactory.java @@ -87,16 +87,6 @@ public void uncaughtException(Thread t, Throwable e) { LOG.error("Thread {} died", t, e); } }); - /** - * this is to avoid the jvm bug: - * NullPointerException in Selector.open() - * http://bugs.sun.com/view_bug.do?bug_id=6427854 - */ - try { - Selector.open().close(); - } catch (IOException ie) { - LOG.error("Selector failed to open", ie); - } /** * Value of 0 disables use of direct buffers and instead uses From c56506162bfde339e801743396361862cc8c5b41 Mon Sep 17 00:00:00 2001 From: asutosh936 Date: Tue, 12 May 2020 09:39:58 +0000 Subject: [PATCH 033/591] ZOOKEEPER-3761: Upgrade JLine version to 2.14.6 Author: asutosh936 Reviewers: Enrico Olivelli , Mate Szalay-Beko Closes #1292 from asutosh936/ZOOKEEPER-3761 --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index a3cb59d32ee..c0dbbee7a97 100755 --- a/pom.xml +++ b/pom.xml @@ -437,7 +437,7 @@ 9.4.24.v20191120 2.10.3 1.1.1 - 2.11 + 2.14.6 1.1.7 2.0.0 1.60 From 236e3d9183606512f0e03a1f828ad0d392eb6091 Mon Sep 17 00:00:00 2001 From: Mate Szalay-Beko Date: Tue, 12 May 2020 09:53:40 +0000 Subject: [PATCH 034/591] ZOOKEEPER-3818: client SSL support for zkServer.sh status command When the ZooKeeper cluster is started in client SSL-only mode (omitting the clientPort from the zoo.cfg), then the current `zkServer.sh status` command fails to connect to the server. **This patch contains:** - a fix for the zkServer.sh to fall-back to SSL connetion if no unsecure port is defined - documenting the necessary system properties one needs to define in this case - some formatting fixes in the `zookeeperTools.md` file to get proper code blocks generated **Hints for testing:** you can generate SSL certificate files e.g. by: ``` mkdir -p /tmp/ssl mkdir -p /tmp/zkdata cp ./zookeeper-client/zookeeper-client-c/ssl/gencerts.sh /tmp/ssl/ cd /tmp/ssl/ ./gencerts.sh localhost ``` then you can change your zoo.cfg: ``` tickTime=3000 initLimit=10 syncLimit=5 dataDir=/tmp/zkdata secureClientPort=22281 # clientPort=2181 serverCnxnFactory=org.apache.zookeeper.server.NettyServerCnxnFactory ssl.keyStore.location=/tmp/ssl/server.jks ssl.keyStore.password=password ssl.trustStore.location=/tmp/ssl/servertrust.jks ssl.trustStore.password=password ``` then start ZooKeeper: `./bin/zkServer.sh start-foreground` then you can run `zkServer.sh status` like: ``` CLIENT_JVMFLAGS="-Dzookeeper.clientCnxnSocket=org.apache.zookeeper.ClientCnxnSocketNetty -Dzookeeper.ssl.trustStore.location=/tmp/ssl/clienttrust.jks -Dzookeeper.ssl.trustStore.password=password -Dzookeeper.ssl.keyStore.location=/tmp/ssl/client.jks -Dzookeeper.ssl.keyStore.password=password -Dzookeeper.client.secure=true " ./bin/zkServer.sh status ``` Author: Mate Szalay-Beko Reviewers: Aishwarya Soni , Norbert Kalmar Closes #1348 from symat/ZOOKEEPER-3818 --- bin/zkServer.sh | 83 ++++++++++++------- .../main/resources/markdown/zookeeperAdmin.md | 8 +- .../main/resources/markdown/zookeeperTools.md | 8 ++ 3 files changed, 64 insertions(+), 35 deletions(-) diff --git a/bin/zkServer.sh b/bin/zkServer.sh index 10ead606b2d..f5113a9a3a0 100755 --- a/bin/zkServer.sh +++ b/bin/zkServer.sh @@ -235,51 +235,70 @@ restart) ;; status) # -q is necessary on some versions of linux where nc returns too quickly, and no stat result is output + isSSL="false" clientPortAddress=`$GREP "^[[:space:]]*clientPortAddress[^[:alpha:]]" "$ZOOCFG" | sed -e 's/.*=//'` if ! [ $clientPortAddress ] then - clientPortAddress="localhost" + clientPortAddress="localhost" fi clientPort=`$GREP "^[[:space:]]*clientPort[^[:alpha:]]" "$ZOOCFG" | sed -e 's/.*=//'` if ! [[ "$clientPort" =~ ^[0-9]+$ ]] then - dataDir=`$GREP "^[[:space:]]*dataDir" "$ZOOCFG" | sed -e 's/.*=//'` - myid=`cat "$dataDir/myid"` - if ! [[ "$myid" =~ ^[0-9]+$ ]] ; then - echo "clientPort not found and myid could not be determined. Terminating." - exit 1 - fi - clientPortAndAddress=`$GREP "^[[:space:]]*server.$myid=.*;.*" "$ZOOCFG" | sed -e 's/.*=//' | sed -e 's/.*;//'` - if [ ! "$clientPortAndAddress" ] ; then - echo "Client port not found in static config file. Looking in dynamic config file." - dynamicConfigFile=`$GREP "^[[:space:]]*dynamicConfigFile" "$ZOOCFG" | sed -e 's/.*=//'` - clientPortAndAddress=`$GREP "^[[:space:]]*server.$myid=.*;.*" "$dynamicConfigFile" | sed -e 's/.*=//' | sed -e 's/.*;//'` - fi - if [ ! "$clientPortAndAddress" ] ; then - echo "Client port not found. Terminating." - exit 1 - fi - if [[ "$clientPortAndAddress" =~ ^.*:[0-9]+ ]] ; then - clientPortAddress=`echo "$clientPortAndAddress" | sed -e 's/:.*//'` - fi - clientPort=`echo "$clientPortAndAddress" | sed -e 's/.*://'` - if [ ! "$clientPort" ] ; then - echo "Client port not found. Terminating." - exit 1 - fi + dataDir=`$GREP "^[[:space:]]*dataDir" "$ZOOCFG" | sed -e 's/.*=//'` + myid=`cat "$dataDir/myid" 2> /dev/null` + if ! [[ "$myid" =~ ^[0-9]+$ ]] ; then + echo "myid could not be determined, will not able to locate clientPort in the server configs." + else + clientPortAndAddress=`$GREP "^[[:space:]]*server.$myid=.*;.*" "$ZOOCFG" | sed -e 's/.*=//' | sed -e 's/.*;//'` + if [ ! "$clientPortAndAddress" ] ; then + echo "Client port not found in static config file. Looking in dynamic config file." + dynamicConfigFile=`$GREP "^[[:space:]]*dynamicConfigFile" "$ZOOCFG" | sed -e 's/.*=//'` + clientPortAndAddress=`$GREP "^[[:space:]]*server.$myid=.*;.*" "$dynamicConfigFile" | sed -e 's/.*=//' | sed -e 's/.*;//'` + fi + if [ ! "$clientPortAndAddress" ] ; then + echo "Client port not found in the server configs" + else + if [[ "$clientPortAndAddress" =~ ^.*:[0-9]+ ]] ; then + clientPortAddress=`echo "$clientPortAndAddress" | sed -e 's/:.*//'` + fi + clientPort=`echo "$clientPortAndAddress" | sed -e 's/.*://'` + fi + fi fi - echo "Client port found: $clientPort. Client address: $clientPortAddress." + if [ ! "$clientPort" ] ; then + echo "Client port not found. Looking for secureClientPort in the static config." + secureClientPort=`$GREP "^[[:space:]]*secureClientPort[^[:alpha:]]" "$ZOOCFG" | sed -e 's/.*=//'` + if [ "$secureClientPort" ] ; then + isSSL="true" + clientPort=$secureClientPort + else + echo "Unable to find either secure or unsecure client port in any configs. Terminating." + exit 1 + fi + fi + echo "Client port found: $clientPort. Client address: $clientPortAddress. Client SSL: $isSSL." STAT=`"$JAVA" "-Dzookeeper.log.dir=${ZOO_LOG_DIR}" "-Dzookeeper.root.logger=${ZOO_LOG4J_PROP}" "-Dzookeeper.log.file=${ZOO_LOG_FILE}" \ - -cp "$CLASSPATH" $JVMFLAGS org.apache.zookeeper.client.FourLetterWordMain \ - $clientPortAddress $clientPort srvr 2> /dev/null \ + -cp "$CLASSPATH" $CLIENT_JVMFLAGS $JVMFLAGS org.apache.zookeeper.client.FourLetterWordMain \ + $clientPortAddress $clientPort srvr $isSSL 2> /dev/null \ | $GREP Mode` if [ "x$STAT" = "x" ] then - echo "Error contacting service. It is probably not running." - exit 1 + if [ "$isSSL" = "true" ] ; then + echo " " + echo "Note: We used secureClientPort ($secureClientPort) to establish connection, but we failed. The 'status'" + echo " command establishes a client connection to the server to execute diagnostic commands. Please make sure you" + echo " provided all the Client SSL connection related parameters in the CLIENT_JVMFLAGS environment variable! E.g.:" + echo " CLIENT_JVMFLAGS=\"-Dzookeeper.clientCnxnSocket=org.apache.zookeeper.ClientCnxnSocketNetty" + echo " -Dzookeeper.ssl.trustStore.location=/tmp/clienttrust.jks -Dzookeeper.ssl.trustStore.password=password" + echo " -Dzookeeper.ssl.keyStore.location=/tmp/client.jks -Dzookeeper.ssl.keyStore.password=password" + echo " -Dzookeeper.client.secure=true\" ./zkServer.sh status" + echo " " + fi + echo "Error contacting service. It is probably not running." + exit 1 else - echo $STAT - exit 0 + echo $STAT + exit 0 fi ;; *) diff --git a/zookeeper-docs/src/main/resources/markdown/zookeeperAdmin.md b/zookeeper-docs/src/main/resources/markdown/zookeeperAdmin.md index 2a9d7d3c731..8d5f11b5516 100644 --- a/zookeeper-docs/src/main/resources/markdown/zookeeperAdmin.md +++ b/zookeeper-docs/src/main/resources/markdown/zookeeperAdmin.md @@ -480,9 +480,11 @@ these options. ### Monitoring -The ZooKeeper service can be monitored in one of two -primary ways; 1) the command port through the use of [4 letter words](#sc_zkCommands) and 2) [JMX](zookeeperJMX.html). See the appropriate section for -your environment/requirements. +The ZooKeeper service can be monitored in one of three primary ways: + +* the command port through the use of [4 letter words](#sc_zkCommands) +* with [JMX](zookeeperJMX.html) +* using the [`zkServer.sh status` command](zookeeperTools.html#zkServer) diff --git a/zookeeper-docs/src/main/resources/markdown/zookeeperTools.md b/zookeeper-docs/src/main/resources/markdown/zookeeperTools.md index 09865f99d2a..46b3ca5c878 100644 --- a/zookeeper-docs/src/main/resources/markdown/zookeeperTools.md +++ b/zookeeper-docs/src/main/resources/markdown/zookeeperTools.md @@ -69,6 +69,14 @@ Apache ZooKeeper, version 3.6.0-SNAPSHOT 06/11/2019 05:39 GMT ``` +The `status` command establishes a client connection to the server to execute diagnostic commands. +When the ZooKeeper cluster is started in client SSL only mode (by omitting the clientPort +from the zoo.cfg), then additional SSL related configuration has to be provided before using +the `./zkServer.sh status` command to find out if the ZooKeeper server is running. An example: + + CLIENT_JVMFLAGS="-Dzookeeper.clientCnxnSocket=org.apache.zookeeper.ClientCnxnSocketNetty -Dzookeeper.ssl.trustStore.location=/tmp/clienttrust.jks -Dzookeeper.ssl.trustStore.password=password -Dzookeeper.ssl.keyStore.location=/tmp/client.jks -Dzookeeper.ssl.keyStore.password=password -Dzookeeper.client.secure=true" ./zkServer.sh status + + ### zkCli.sh From 45bc3ece8964a24feb808926f18f6e53319aa683 Mon Sep 17 00:00:00 2001 From: ctest-team Date: Thu, 14 May 2020 18:18:28 +0200 Subject: [PATCH 035/591] ZOOKEEPER-3721: Making the boolean configuration parameters consistent Author: ctest-team Reviewers: Enrico Olivelli , Mate Szalay-Beko , Justin Ling Mao Closes #1266 from ctest-team/ZOOKEEPER-3721 --- .../server/quorum/QuorumPeerConfig.java | 59 +++++++++---------- .../server/quorum/QuorumPeerConfigTest.java | 33 +++++++++++ 2 files changed, 61 insertions(+), 31 deletions(-) diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumPeerConfig.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumPeerConfig.java index ae73ef35e0c..dbebef3734e 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumPeerConfig.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumPeerConfig.java @@ -292,9 +292,9 @@ public void parseProperties(Properties zkProp) throws IOException, ConfigExcepti } else if (key.equals("clientPort")) { clientPort = Integer.parseInt(value); } else if (key.equals("localSessionsEnabled")) { - localSessionsEnabled = Boolean.parseBoolean(value); + localSessionsEnabled = parseBoolean(key, value); } else if (key.equals("localSessionsUpgradingEnabled")) { - localSessionsUpgradingEnabled = Boolean.parseBoolean(value); + localSessionsUpgradingEnabled = parseBoolean(key, value); } else if (key.equals("clientPortAddress")) { clientPortAddress = value.trim(); } else if (key.equals("secureClientPort")) { @@ -325,7 +325,7 @@ public void parseProperties(Properties zkProp) throws IOException, ConfigExcepti throw new ConfigException("Invalid electionAlg value. Only 3 is supported."); } } else if (key.equals("quorumListenOnAllIPs")) { - quorumListenOnAllIPs = Boolean.parseBoolean(value); + quorumListenOnAllIPs = parseBoolean(key, value); } else if (key.equals("peerType")) { if (value.toLowerCase().equals("observer")) { peerType = LearnerType.OBSERVER; @@ -335,7 +335,7 @@ public void parseProperties(Properties zkProp) throws IOException, ConfigExcepti throw new ConfigException("Unrecognised peertype: " + value); } } else if (key.equals("syncEnabled")) { - syncEnabled = Boolean.parseBoolean(value); + syncEnabled = parseBoolean(key, value); } else if (key.equals("dynamicConfigFile")) { dynamicConfigFileStr = value; } else if (key.equals("autopurge.snapRetainCount")) { @@ -343,40 +343,24 @@ public void parseProperties(Properties zkProp) throws IOException, ConfigExcepti } else if (key.equals("autopurge.purgeInterval")) { purgeInterval = Integer.parseInt(value); } else if (key.equals("standaloneEnabled")) { - if (value.toLowerCase().equals("true")) { - setStandaloneEnabled(true); - } else if (value.toLowerCase().equals("false")) { - setStandaloneEnabled(false); - } else { - throw new ConfigException("Invalid option " - + value - + " for standalone mode. Choose 'true' or 'false.'"); - } + setStandaloneEnabled(parseBoolean(key, value)); } else if (key.equals("reconfigEnabled")) { - if (value.toLowerCase().equals("true")) { - setReconfigEnabled(true); - } else if (value.toLowerCase().equals("false")) { - setReconfigEnabled(false); - } else { - throw new ConfigException("Invalid option " - + value - + " for reconfigEnabled flag. Choose 'true' or 'false.'"); - } + setReconfigEnabled(parseBoolean(key, value)); } else if (key.equals("sslQuorum")) { - sslQuorum = Boolean.parseBoolean(value); + sslQuorum = parseBoolean(key, value); } else if (key.equals("portUnification")) { - shouldUsePortUnification = Boolean.parseBoolean(value); + shouldUsePortUnification = parseBoolean(key, value); } else if (key.equals("sslQuorumReloadCertFiles")) { - sslQuorumReloadCertFiles = Boolean.parseBoolean(value); + sslQuorumReloadCertFiles = parseBoolean(key, value); } else if ((key.startsWith("server.") || key.startsWith("group") || key.startsWith("weight")) && zkProp.containsKey("dynamicConfigFile")) { throw new ConfigException("parameter: " + key + " must be in a separate dynamic config file"); } else if (key.equals(QuorumAuth.QUORUM_SASL_AUTH_ENABLED)) { - quorumEnableSasl = Boolean.parseBoolean(value); + quorumEnableSasl = parseBoolean(key, value); } else if (key.equals(QuorumAuth.QUORUM_SERVER_SASL_AUTH_REQUIRED)) { - quorumServerRequireSasl = Boolean.parseBoolean(value); + quorumServerRequireSasl = parseBoolean(key, value); } else if (key.equals(QuorumAuth.QUORUM_LEARNER_SASL_AUTH_REQUIRED)) { - quorumLearnerRequireSasl = Boolean.parseBoolean(value); + quorumLearnerRequireSasl = parseBoolean(key, value); } else if (key.equals(QuorumAuth.QUORUM_LEARNER_SASL_LOGIN_CONTEXT)) { quorumLearnerLoginContext = value; } else if (key.equals(QuorumAuth.QUORUM_SERVER_SASL_LOGIN_CONTEXT)) { @@ -392,18 +376,18 @@ public void parseProperties(Properties zkProp) throws IOException, ConfigExcepti } else if (key.equals(JvmPauseMonitor.SLEEP_TIME_MS_KEY)) { jvmPauseSleepTimeMs = Long.parseLong(value); } else if (key.equals(JvmPauseMonitor.JVM_PAUSE_MONITOR_FEATURE_SWITCH_KEY)) { - jvmPauseMonitorToRun = Boolean.parseBoolean(value); + jvmPauseMonitorToRun = parseBoolean(key, value); } else if (key.equals("metricsProvider.className")) { metricsProviderClassName = value; } else if (key.startsWith("metricsProvider.")) { String keyForMetricsProvider = key.substring(16); metricsProviderConfiguration.put(keyForMetricsProvider, value); } else if (key.equals("multiAddress.enabled")) { - multiAddressEnabled = Boolean.parseBoolean(value); + multiAddressEnabled = parseBoolean(key, value); } else if (key.equals("multiAddress.reachabilityCheckTimeoutMs")) { multiAddressReachabilityCheckTimeoutMs = Integer.parseInt(value); } else if (key.equals("multiAddress.reachabilityCheckEnabled")) { - multiAddressReachabilityCheckEnabled = Boolean.parseBoolean(value); + multiAddressReachabilityCheckEnabled = parseBoolean(key, value); } else { System.setProperty("zookeeper." + key, value); } @@ -969,4 +953,17 @@ public static void setReconfigEnabled(boolean enabled) { reconfigEnabled = enabled; } + private boolean parseBoolean(String key, String value) throws ConfigException { + if (value.equalsIgnoreCase("true")) { + return true; + } else if (value.equalsIgnoreCase("false")) { + return false; + } else { + throw new ConfigException("Invalid option " + + value + + " for " + + key + + ". Choose 'true' or 'false.'"); + } + } } diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/QuorumPeerConfigTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/QuorumPeerConfigTest.java index 0d7e7cce015..da4bedf7c1d 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/QuorumPeerConfigTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/QuorumPeerConfigTest.java @@ -164,6 +164,39 @@ public void testJvmPauseMonitorConfigured() throws IOException, ConfigException assertTrue(quorumPeerConfig.isJvmPauseMonitorToRun()); } + /** + * Test case for https://issues.apache.org/jira/browse/ZOOKEEPER-3721 + */ + @Test + public void testParseBoolean() throws IOException, ConfigException { + QuorumPeerConfig quorumPeerConfig = new QuorumPeerConfig(); + Properties zkProp = getDefaultZKProperties(); + + zkProp.setProperty("localSessionsEnabled", "true"); + quorumPeerConfig.parseProperties(zkProp); + assertEquals(true, quorumPeerConfig.areLocalSessionsEnabled()); + + zkProp.setProperty("localSessionsEnabled", "false"); + quorumPeerConfig.parseProperties(zkProp); + assertEquals(false, quorumPeerConfig.areLocalSessionsEnabled()); + + zkProp.setProperty("localSessionsEnabled", "True"); + quorumPeerConfig.parseProperties(zkProp); + assertEquals(true, quorumPeerConfig.areLocalSessionsEnabled()); + + zkProp.setProperty("localSessionsEnabled", "False"); + quorumPeerConfig.parseProperties(zkProp); + assertEquals(false, quorumPeerConfig.areLocalSessionsEnabled()); + + zkProp.setProperty("localSessionsEnabled", "yes"); + try { + quorumPeerConfig.parseProperties(zkProp); + fail("Must throw exception as 'yes' is not accpetable for parseBoolean!"); + } catch (ConfigException e) { + // expected + } + } + private Properties getDefaultZKProperties() { Properties zkProp = new Properties(); zkProp.setProperty("dataDir", new File("myDataDir").getAbsolutePath()); From a14bd54f7112df96622c04fe6951bb25d6eddcf0 Mon Sep 17 00:00:00 2001 From: Fangmin Lyu Date: Fri, 15 May 2020 09:10:25 +0200 Subject: [PATCH 036/591] [ZOOKEEPER-3690] Improving leader efficiency via not processing learner forwarded requests in commit processor Author: Fangmin Lyu Reviewers: Enrico Olivelli , Michael Han Closes #1223 from lvfangmin/ZOOKEEPER-3690 --- .../main/resources/markdown/zookeeperAdmin.md | 8 +++++ .../zookeeper/server/ServerMetrics.java | 3 ++ .../quorum/ProposalRequestProcessor.java | 26 +++++++++++++- .../server/quorum/QuorumPeerMainTest.java | 35 +++++++++++++++++++ 4 files changed, 71 insertions(+), 1 deletion(-) diff --git a/zookeeper-docs/src/main/resources/markdown/zookeeperAdmin.md b/zookeeper-docs/src/main/resources/markdown/zookeeperAdmin.md index 8d5f11b5516..6dbfa277cc7 100644 --- a/zookeeper-docs/src/main/resources/markdown/zookeeperAdmin.md +++ b/zookeeper-docs/src/main/resources/markdown/zookeeperAdmin.md @@ -1148,6 +1148,14 @@ property, when available, is noted below. (Jave system property only: **learner.closeSocketAsync**) When enabled, a learner will close the quorum socket asynchronously. This is useful for TLS connections where closing a socket might take a long time, block the shutdown process, potentially delay a new leader election, and leave the quorum unavailabe. Closing the socket asynchronously avoids blocking the shutdown process despite the long socket closing time and a new leader election can be started while the socket being closed. The default is false. +* *forward_learner_requests_to_commit_processor_disabled* + (Jave system property: **zookeeper.forward_learner_requests_to_commit_processor_disabled**) + When this property is set, the requests from learners won't be enqueued to + CommitProcessor queue, which will help save the resources and GC time on + leader. + + The default value is false. + diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/ServerMetrics.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/ServerMetrics.java index 36a65df801f..3962bb974f1 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/ServerMetrics.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/ServerMetrics.java @@ -241,6 +241,8 @@ private ServerMetrics(MetricsProvider metricsProvider) { SOCKET_CLOSING_TIME = metricsContext.getSummary("socket_closing_time", DetailLevel.BASIC); + REQUESTS_NOT_FORWARDED_TO_COMMIT_PROCESSOR = metricsContext.getCounter( + "requests_not_forwarded_to_commit_processor"); } /** @@ -465,6 +467,7 @@ private ServerMetrics(MetricsProvider metricsProvider) { public final Summary SOCKET_CLOSING_TIME; + public final Counter REQUESTS_NOT_FORWARDED_TO_COMMIT_PROCESSOR; private final MetricsProvider metricsProvider; diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/ProposalRequestProcessor.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/ProposalRequestProcessor.java index c4b5a6e30b3..a3f1b29bdb6 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/ProposalRequestProcessor.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/ProposalRequestProcessor.java @@ -20,6 +20,7 @@ import org.apache.zookeeper.server.Request; import org.apache.zookeeper.server.RequestProcessor; +import org.apache.zookeeper.server.ServerMetrics; import org.apache.zookeeper.server.SyncRequestProcessor; import org.apache.zookeeper.server.quorum.Leader.XidRolloverException; import org.slf4j.Logger; @@ -39,11 +40,22 @@ public class ProposalRequestProcessor implements RequestProcessor { SyncRequestProcessor syncProcessor; + // If this property is set, requests from Learners won't be forwarded + // to the CommitProcessor in order to save resources + public static final String FORWARD_LEARNER_REQUESTS_TO_COMMIT_PROCESSOR_DISABLED = + "zookeeper.forward_learner_requests_to_commit_processor_disabled"; + private final boolean forwardLearnerRequestsToCommitProcessorDisabled; + public ProposalRequestProcessor(LeaderZooKeeperServer zks, RequestProcessor nextProcessor) { this.zks = zks; this.nextProcessor = nextProcessor; AckRequestProcessor ackProcessor = new AckRequestProcessor(zks.getLeader()); syncProcessor = new SyncRequestProcessor(zks, ackProcessor); + + forwardLearnerRequestsToCommitProcessorDisabled = Boolean.getBoolean( + FORWARD_LEARNER_REQUESTS_TO_COMMIT_PROCESSOR_DISABLED); + LOG.info("{} = {}", FORWARD_LEARNER_REQUESTS_TO_COMMIT_PROCESSOR_DISABLED, + forwardLearnerRequestsToCommitProcessorDisabled); } /** @@ -70,7 +82,9 @@ public void processRequest(Request request) throws RequestProcessorException { if (request instanceof LearnerSyncRequest) { zks.getLeader().processSync((LearnerSyncRequest) request); } else { - nextProcessor.processRequest(request); + if (shouldForwardToNextProcessor(request)) { + nextProcessor.processRequest(request); + } if (request.getHdr() != null) { // We need to sync and get consensus on any transactions try { @@ -89,4 +103,14 @@ public void shutdown() { syncProcessor.shutdown(); } + private boolean shouldForwardToNextProcessor(Request request) { + if (!forwardLearnerRequestsToCommitProcessorDisabled) { + return true; + } + if (request.getOwner() instanceof LearnerHandler) { + ServerMetrics.getMetrics().REQUESTS_NOT_FORWARDED_TO_COMMIT_PROCESSOR.add(1); + return false; + } + return true; + } } diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/QuorumPeerMainTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/QuorumPeerMainTest.java index beb3bb9c069..487507bfc53 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/QuorumPeerMainTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/QuorumPeerMainTest.java @@ -62,6 +62,7 @@ import org.apache.zookeeper.common.X509Exception; import org.apache.zookeeper.metrics.BaseTestMetricsProvider; import org.apache.zookeeper.metrics.impl.NullMetricsProvider; +import org.apache.zookeeper.server.ServerMetrics; import org.apache.zookeeper.server.persistence.FileTxnSnapLog; import org.apache.zookeeper.server.quorum.Leader.Proposal; import org.apache.zookeeper.test.ClientBase; @@ -1620,6 +1621,40 @@ public void testFaultyMetricsProviderOnConfigure() throws Exception { assertTrue("complains about metrics provider MetricsProviderLifeCycleException", found); } + /** + * Test the behavior to skip processing the learner forwarded requests in + * Leader's CommitProcessor. + */ + @Test + public void testLearnerRequestForwardBehavior() throws Exception { + System.setProperty(ProposalRequestProcessor.FORWARD_LEARNER_REQUESTS_TO_COMMIT_PROCESSOR_DISABLED, "true"); + + try { + // 1. set up an ensemble with 3 servers + final int numServers = 3; + servers = LaunchServers(numServers); + int leaderId = servers.findLeader(); + + int followerA = (leaderId + 1) % numServers; + waitForOne(servers.zk[followerA], States.CONNECTED); + + // 2. reset all metrics + ServerMetrics.getMetrics().resetAll(); + + // 3. issue a request + final String node = "/testLearnerRequestForwardBehavior"; + servers.zk[followerA].create(node, new byte[0], Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); + + assertNotNull("node " + node + " should exist", + servers.zk[followerA].exists("/testLearnerRequestForwardBehavior", false)); + + assertEquals(1L, ServerMetrics.getMetrics().REQUESTS_NOT_FORWARDED_TO_COMMIT_PROCESSOR.get()); + } finally { + //clean up + System.setProperty(ProposalRequestProcessor.FORWARD_LEARNER_REQUESTS_TO_COMMIT_PROCESSOR_DISABLED, "false"); + } + } + static class Context { boolean quitFollowing = false; From d99c9e8b701ebdbd3f0098d4d81696df97432262 Mon Sep 17 00:00:00 2001 From: tison Date: Sun, 17 May 2020 15:13:37 +0200 Subject: [PATCH 037/591] ZOOKEEPER-837: Eliminate cycle dependency between ClientCnxn and ZooKeeper 1. Extract ZKWatchManager to single file 2. Move ZKWatchManager instance to ClientCnxn in order to eliminate cycle dependency 3. let `ZooKeeper` syncs a copy of default watcher, in order to reduce dependencies to `getWatchManager()` Author: tison Reviewers: Enrico Olivelli , Andor Molnar Closes #1095 from TisonKun/ZOOKEEPER-837 --- .../java/org/apache/zookeeper/ClientCnxn.java | 110 ++-- .../apache/zookeeper/WatchDeregistration.java | 1 - .../org/apache/zookeeper/ZKWatchManager.java | 455 ++++++++++++++++ .../java/org/apache/zookeeper/ZooKeeper.java | 504 ++---------------- .../ClientCnxnSocketFragilityTest.java | 35 +- .../apache/zookeeper/ClientReconnectTest.java | 10 +- .../zookeeper/ClientRequestTimeoutTest.java | 37 +- .../apache/zookeeper/RemoveWatchesTest.java | 70 +-- .../apache/zookeeper/TestableZooKeeper.java | 70 +-- 9 files changed, 651 insertions(+), 641 deletions(-) create mode 100644 zookeeper-server/src/main/java/org/apache/zookeeper/ZKWatchManager.java diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/ClientCnxn.java b/zookeeper-server/src/main/java/org/apache/zookeeper/ClientCnxn.java index c87d3cba862..87094b8b1b9 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/ClientCnxn.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/ClientCnxn.java @@ -165,13 +165,11 @@ static class AuthData { private final int sessionTimeout; - private final ZooKeeper zooKeeper; - - private final ClientWatchManager watcher; + private final ZKWatchManager watchManager; private long sessionId; - private byte[] sessionPasswd = new byte[16]; + private byte[] sessionPasswd; /** * If true, the connection is allowed to go to r-o mode. This field's value @@ -224,6 +222,10 @@ static class AuthData { */ private long requestTimeout; + ZKWatchManager getWatcherManager() { + return watchManager; + } + public long getSessionId() { return sessionId; } @@ -362,35 +364,29 @@ public String toString() { * established until needed. The start() instance method must be called * subsequent to construction. * - * @param chrootPath - the chroot of this client. Should be removed from this Class in ZOOKEEPER-838 - * @param hostProvider - * the list of ZooKeeper servers to connect to - * @param sessionTimeout - * the timeout for connections. - * @param zooKeeper - * the zookeeper object that this connection is related to. - * @param watcher watcher for this connection - * @param clientCnxnSocket - * the socket implementation used (e.g. NIO/Netty) - * @param canBeReadOnly - * whether the connection is allowed to go to read-only - * mode in case of partitioning - * @throws IOException + * @param chrootPath the chroot of this client. Should be removed from this Class in ZOOKEEPER-838 + * @param hostProvider the list of ZooKeeper servers to connect to + * @param sessionTimeout the timeout for connections. + * @param clientConfig the client configuration. + * @param defaultWatcher default watcher for this connection + * @param clientCnxnSocket the socket implementation used (e.g. NIO/Netty) + * @param canBeReadOnly whether the connection is allowed to go to read-only mode in case of partitioning */ public ClientCnxn( String chrootPath, HostProvider hostProvider, int sessionTimeout, - ZooKeeper zooKeeper, - ClientWatchManager watcher, + ZKClientConfig clientConfig, + Watcher defaultWatcher, ClientCnxnSocket clientCnxnSocket, - boolean canBeReadOnly) throws IOException { + boolean canBeReadOnly + ) throws IOException { this( chrootPath, hostProvider, sessionTimeout, - zooKeeper, - watcher, + clientConfig, + defaultWatcher, clientCnxnSocket, 0, new byte[16], @@ -402,48 +398,45 @@ public ClientCnxn( * established until needed. The start() instance method must be called * subsequent to construction. * - * @param chrootPath - the chroot of this client. Should be removed from this Class in ZOOKEEPER-838 - * @param hostProvider - * the list of ZooKeeper servers to connect to - * @param sessionTimeout - * the timeout for connections. - * @param zooKeeper - * the zookeeper object that this connection is related to. - * @param watcher watcher for this connection - * @param clientCnxnSocket - * the socket implementation used (e.g. NIO/Netty) + * @param chrootPath the chroot of this client. Should be removed from this Class in ZOOKEEPER-838 + * @param hostProvider the list of ZooKeeper servers to connect to + * @param sessionTimeout the timeout for connections. + * @param clientConfig the client configuration. + * @param defaultWatcher default watcher for this connection + * @param clientCnxnSocket the socket implementation used (e.g. NIO/Netty) * @param sessionId session id if re-establishing session * @param sessionPasswd session passwd if re-establishing session - * @param canBeReadOnly - * whether the connection is allowed to go to read-only - * mode in case of partitioning + * @param canBeReadOnly whether the connection is allowed to go to read-only mode in case of partitioning * @throws IOException in cases of broken network */ public ClientCnxn( String chrootPath, HostProvider hostProvider, int sessionTimeout, - ZooKeeper zooKeeper, - ClientWatchManager watcher, + ZKClientConfig clientConfig, + Watcher defaultWatcher, ClientCnxnSocket clientCnxnSocket, long sessionId, byte[] sessionPasswd, - boolean canBeReadOnly) throws IOException { - this.zooKeeper = zooKeeper; - this.watcher = watcher; + boolean canBeReadOnly + ) throws IOException { + this.chrootPath = chrootPath; + this.hostProvider = hostProvider; + this.sessionTimeout = sessionTimeout; + this.clientConfig = clientConfig; this.sessionId = sessionId; this.sessionPasswd = sessionPasswd; - this.sessionTimeout = sessionTimeout; - this.hostProvider = hostProvider; - this.chrootPath = chrootPath; + this.readOnly = canBeReadOnly; + + this.watchManager = new ZKWatchManager( + clientConfig.getBoolean(ZKClientConfig.DISABLE_AUTO_WATCH_RESET), + defaultWatcher); - connectTimeout = sessionTimeout / hostProvider.size(); - readTimeout = sessionTimeout * 2 / 3; - readOnly = canBeReadOnly; + this.connectTimeout = sessionTimeout / hostProvider.size(); + this.readTimeout = sessionTimeout * 2 / 3; - sendThread = new SendThread(clientCnxnSocket); - eventThread = new EventThread(); - this.clientConfig = zooKeeper.getClientConfig(); + this.sendThread = new SendThread(clientCnxnSocket); + this.eventThread = new EventThread(); initRequestTimeout(); } @@ -506,10 +499,9 @@ private void queueEvent(WatchedEvent event, Set materializedWatchers) { final Set watchers; if (materializedWatchers == null) { // materialize the watchers based on the event - watchers = watcher.materialize(event.getState(), event.getType(), event.getPath()); + watchers = watchManager.materialize(event.getState(), event.getType(), event.getPath()); } else { - watchers = new HashSet(); - watchers.addAll(materializedWatchers); + watchers = new HashSet<>(materializedWatchers); } WatcherSetEventPair pair = new WatcherSetEventPair(watchers, event); // queue the pair (watch set & event) for later processing @@ -1007,14 +999,12 @@ void primeConnection() throws IOException { ConnectRequest conReq = new ConnectRequest(0, lastZxid, sessionTimeout, sessId, sessionPasswd); // We add backwards since we are pushing into the front // Only send if there's a pending watch - // TODO: here we have the only remaining use of zooKeeper in - // this class. It's to be eliminated! if (!clientConfig.getBoolean(ZKClientConfig.DISABLE_AUTO_WATCH_RESET)) { - List dataWatches = zooKeeper.getDataWatches(); - List existWatches = zooKeeper.getExistWatches(); - List childWatches = zooKeeper.getChildWatches(); - List persistentWatches = zooKeeper.getPersistentWatches(); - List persistentRecursiveWatches = zooKeeper.getPersistentRecursiveWatches(); + List dataWatches = watchManager.getDataWatchList(); + List existWatches = watchManager.getExistWatchList(); + List childWatches = watchManager.getChildWatchList(); + List persistentWatches = watchManager.getPersistentWatchList(); + List persistentRecursiveWatches = watchManager.getPersistentRecursiveWatchList(); if (!dataWatches.isEmpty() || !existWatches.isEmpty() || !childWatches.isEmpty() || !persistentWatches.isEmpty() || !persistentRecursiveWatches.isEmpty()) { Iterator dataWatchesIter = prependChroot(dataWatches).iterator(); diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/WatchDeregistration.java b/zookeeper-server/src/main/java/org/apache/zookeeper/WatchDeregistration.java index 16c7f843473..710f47b34c3 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/WatchDeregistration.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/WatchDeregistration.java @@ -22,7 +22,6 @@ import java.util.Set; import org.apache.zookeeper.Watcher.Event.EventType; import org.apache.zookeeper.Watcher.WatcherType; -import org.apache.zookeeper.ZooKeeper.ZKWatchManager; /** * Handles the special case of removing watches which has registered for a diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/ZKWatchManager.java b/zookeeper-server/src/main/java/org/apache/zookeeper/ZKWatchManager.java new file mode 100644 index 00000000000..95a07f0e7be --- /dev/null +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/ZKWatchManager.java @@ -0,0 +1,455 @@ +/* + * 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; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import org.apache.zookeeper.server.watch.PathParentIterator; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Manage watchers and handle events generated by the {@link ClientCnxn} object. + * + * This class is intended to be packaged-private so that it doesn't serve + * as part of ZooKeeper client API. + */ +class ZKWatchManager implements ClientWatchManager { + + private static final Logger LOG = LoggerFactory.getLogger(ZKWatchManager.class); + + private final Map> dataWatches = new HashMap<>(); + private final Map> existWatches = new HashMap<>(); + private final Map> childWatches = new HashMap<>(); + private final Map> persistentWatches = new HashMap<>(); + private final Map> persistentRecursiveWatches = new HashMap<>(); + private final boolean disableAutoWatchReset; + + private volatile Watcher defaultWatcher; + + ZKWatchManager(boolean disableAutoWatchReset, Watcher defaultWatcher) { + this.disableAutoWatchReset = disableAutoWatchReset; + this.defaultWatcher = defaultWatcher; + } + + void setDefaultWatcher(Watcher defaultWatcher) { + this.defaultWatcher = defaultWatcher; + } + + Watcher getDefaultWatcher() { + return defaultWatcher; + } + + List getDataWatchList() { + synchronized (dataWatches) { + return new ArrayList<>(dataWatches.keySet()); + } + } + + List getChildWatchList() { + synchronized (childWatches) { + return new ArrayList<>(childWatches.keySet()); + } + } + + List getExistWatchList() { + synchronized (existWatches) { + return new ArrayList<>(existWatches.keySet()); + } + } + + List getPersistentWatchList() { + synchronized (persistentWatches) { + return new ArrayList<>(persistentWatches.keySet()); + } + } + + List getPersistentRecursiveWatchList() { + synchronized (persistentRecursiveWatches) { + return new ArrayList<>(persistentRecursiveWatches.keySet()); + } + } + + Map> getDataWatches() { + return dataWatches; + } + + Map> getExistWatches() { + return existWatches; + } + + Map> getChildWatches() { + return childWatches; + } + + Map> getPersistentWatches() { + return persistentWatches; + } + + Map> getPersistentRecursiveWatches() { + return persistentRecursiveWatches; + } + + private void addTo(Set from, Set to) { + if (from != null) { + to.addAll(from); + } + } + + public Map> removeWatcher( + String clientPath, + Watcher watcher, + Watcher.WatcherType watcherType, + boolean local, + int rc + ) throws KeeperException { + // Validate the provided znode path contains the given watcher of + // watcherType + containsWatcher(clientPath, watcher, watcherType); + + Map> removedWatchers = new HashMap<>(); + HashSet childWatchersToRem = new HashSet<>(); + removedWatchers.put(Watcher.Event.EventType.ChildWatchRemoved, childWatchersToRem); + HashSet dataWatchersToRem = new HashSet<>(); + removedWatchers.put(Watcher.Event.EventType.DataWatchRemoved, dataWatchersToRem); + HashSet persistentWatchersToRem = new HashSet<>(); + removedWatchers.put(Watcher.Event.EventType.PersistentWatchRemoved, persistentWatchersToRem); + boolean removedWatcher = false; + switch (watcherType) { + case Children: { + synchronized (childWatches) { + removedWatcher = removeWatches(childWatches, watcher, clientPath, local, rc, childWatchersToRem); + } + break; + } + case Data: { + synchronized (dataWatches) { + removedWatcher = removeWatches(dataWatches, watcher, clientPath, local, rc, dataWatchersToRem); + } + + synchronized (existWatches) { + boolean removedDataWatcher = removeWatches(existWatches, watcher, clientPath, local, rc, dataWatchersToRem); + removedWatcher |= removedDataWatcher; + } + break; + } + case Any: { + synchronized (childWatches) { + removedWatcher = removeWatches(childWatches, watcher, clientPath, local, rc, childWatchersToRem); + } + + synchronized (dataWatches) { + boolean removedDataWatcher = removeWatches(dataWatches, watcher, clientPath, local, rc, dataWatchersToRem); + removedWatcher |= removedDataWatcher; + } + + synchronized (existWatches) { + boolean removedDataWatcher = removeWatches(existWatches, watcher, clientPath, local, rc, dataWatchersToRem); + removedWatcher |= removedDataWatcher; + } + + synchronized (persistentWatches) { + boolean removedPersistentWatcher = removeWatches(persistentWatches, + watcher, clientPath, local, rc, persistentWatchersToRem); + removedWatcher |= removedPersistentWatcher; + } + + synchronized (persistentRecursiveWatches) { + boolean removedPersistentRecursiveWatcher = removeWatches(persistentRecursiveWatches, + watcher, clientPath, local, rc, persistentWatchersToRem); + removedWatcher |= removedPersistentRecursiveWatcher; + } + } + } + // Watcher function doesn't exists for the specified params + if (!removedWatcher) { + throw new KeeperException.NoWatcherException(clientPath); + } + return removedWatchers; + } + + private boolean contains(String path, Watcher watcherObj, Map> pathVsWatchers) { + boolean watcherExists = true; + if (pathVsWatchers == null || pathVsWatchers.size() == 0) { + watcherExists = false; + } else { + Set watchers = pathVsWatchers.get(path); + if (watchers == null) { + watcherExists = false; + } else if (watcherObj == null) { + watcherExists = watchers.size() > 0; + } else { + watcherExists = watchers.contains(watcherObj); + } + } + return watcherExists; + } + + /** + * Validate the provided znode path contains the given watcher and + * watcherType + * + * @param path + * - client path + * @param watcher + * - watcher object reference + * @param watcherType + * - type of the watcher + * @throws KeeperException.NoWatcherException + */ + void containsWatcher(String path, Watcher watcher, Watcher.WatcherType watcherType) throws + KeeperException.NoWatcherException { + boolean containsWatcher = false; + switch (watcherType) { + case Children: { + synchronized (childWatches) { + containsWatcher = contains(path, watcher, childWatches); + } + + synchronized (persistentWatches) { + boolean contains_temp = contains(path, watcher, + persistentWatches); + containsWatcher |= contains_temp; + } + + synchronized (persistentRecursiveWatches) { + boolean contains_temp = contains(path, watcher, + persistentRecursiveWatches); + containsWatcher |= contains_temp; + } + break; + } + case Data: { + synchronized (dataWatches) { + containsWatcher = contains(path, watcher, dataWatches); + } + + synchronized (existWatches) { + boolean contains_temp = contains(path, watcher, existWatches); + containsWatcher |= contains_temp; + } + + synchronized (persistentWatches) { + boolean contains_temp = contains(path, watcher, + persistentWatches); + containsWatcher |= contains_temp; + } + + synchronized (persistentRecursiveWatches) { + boolean contains_temp = contains(path, watcher, + persistentRecursiveWatches); + containsWatcher |= contains_temp; + } + break; + } + case Any: { + synchronized (childWatches) { + containsWatcher = contains(path, watcher, childWatches); + } + + synchronized (dataWatches) { + boolean contains_temp = contains(path, watcher, dataWatches); + containsWatcher |= contains_temp; + } + + synchronized (existWatches) { + boolean contains_temp = contains(path, watcher, existWatches); + containsWatcher |= contains_temp; + } + + synchronized (persistentWatches) { + boolean contains_temp = contains(path, watcher, + persistentWatches); + containsWatcher |= contains_temp; + } + + synchronized (persistentRecursiveWatches) { + boolean contains_temp = contains(path, watcher, + persistentRecursiveWatches); + containsWatcher |= contains_temp; + } + } + } + // Watcher function doesn't exists for the specified params + if (!containsWatcher) { + throw new KeeperException.NoWatcherException(path); + } + } + + protected boolean removeWatches( + Map> pathVsWatcher, + Watcher watcher, + String path, + boolean local, + int rc, + Set removedWatchers) throws KeeperException { + if (!local && rc != KeeperException.Code.OK.intValue()) { + throw KeeperException.create(KeeperException.Code.get(rc), path); + } + boolean success = false; + // When local flag is true, remove watchers for the given path + // irrespective of rc. Otherwise shouldn't remove watchers locally + // when sees failure from server. + if (rc == KeeperException.Code.OK.intValue() || (local && rc != KeeperException.Code.OK.intValue())) { + // Remove all the watchers for the given path + if (watcher == null) { + Set pathWatchers = pathVsWatcher.remove(path); + if (pathWatchers != null) { + // found path watchers + removedWatchers.addAll(pathWatchers); + success = true; + } + } else { + Set watchers = pathVsWatcher.get(path); + if (watchers != null) { + if (watchers.remove(watcher)) { + // found path watcher + removedWatchers.add(watcher); + // cleanup + if (watchers.size() <= 0) { + pathVsWatcher.remove(path); + } + success = true; + } + } + } + } + return success; + } + + /* (non-Javadoc) + * @see org.apache.zookeeper.ClientWatchManager#materialize(Event.KeeperState, + * Event.EventType, java.lang.String) + */ + @Override + public Set materialize( + Watcher.Event.KeeperState state, + Watcher.Event.EventType type, + String clientPath + ) { + final Set result = new HashSet<>(); + + switch (type) { + case None: + if (defaultWatcher != null) { + result.add(defaultWatcher); + } + + boolean clear = disableAutoWatchReset && state != Watcher.Event.KeeperState.SyncConnected; + synchronized (dataWatches) { + for (Set ws : dataWatches.values()) { + result.addAll(ws); + } + if (clear) { + dataWatches.clear(); + } + } + + synchronized (existWatches) { + for (Set ws : existWatches.values()) { + result.addAll(ws); + } + if (clear) { + existWatches.clear(); + } + } + + synchronized (childWatches) { + for (Set ws : childWatches.values()) { + result.addAll(ws); + } + if (clear) { + childWatches.clear(); + } + } + + synchronized (persistentWatches) { + for (Set ws: persistentWatches.values()) { + result.addAll(ws); + } + } + + synchronized (persistentRecursiveWatches) { + for (Set ws: persistentRecursiveWatches.values()) { + result.addAll(ws); + } + } + + return result; + case NodeDataChanged: + case NodeCreated: + synchronized (dataWatches) { + addTo(dataWatches.remove(clientPath), result); + } + synchronized (existWatches) { + addTo(existWatches.remove(clientPath), result); + } + addPersistentWatches(clientPath, result); + break; + case NodeChildrenChanged: + synchronized (childWatches) { + addTo(childWatches.remove(clientPath), result); + } + addPersistentWatches(clientPath, result); + break; + case NodeDeleted: + synchronized (dataWatches) { + addTo(dataWatches.remove(clientPath), result); + } + // TODO This shouldn't be needed, but just in case + synchronized (existWatches) { + Set list = existWatches.remove(clientPath); + if (list != null) { + addTo(list, result); + LOG.warn("We are triggering an exists watch for delete! Shouldn't happen!"); + } + } + synchronized (childWatches) { + addTo(childWatches.remove(clientPath), result); + } + addPersistentWatches(clientPath, result); + break; + default: + String errorMsg = String.format( + "Unhandled watch event type %s with state %s on path %s", + type, + state, + clientPath); + LOG.error(errorMsg); + throw new RuntimeException(errorMsg); + } + + return result; + } + + private void addPersistentWatches(String clientPath, Set result) { + synchronized (persistentWatches) { + addTo(persistentWatches.get(clientPath), result); + } + synchronized (persistentRecursiveWatches) { + for (String path : PathParentIterator.forAll(clientPath).asIterable()) { + addTo(persistentRecursiveWatches.get(path), result); + } + } + } +} diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/ZooKeeper.java b/zookeeper-server/src/main/java/org/apache/zookeeper/ZooKeeper.java index 0210493efc0..201cf20f203 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/ZooKeeper.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/ZooKeeper.java @@ -24,7 +24,6 @@ import java.net.SocketAddress; import java.util.ArrayList; import java.util.Collection; -import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; @@ -40,10 +39,7 @@ import org.apache.zookeeper.AsyncCallback.StatCallback; import org.apache.zookeeper.AsyncCallback.StringCallback; import org.apache.zookeeper.AsyncCallback.VoidCallback; -import org.apache.zookeeper.KeeperException.Code; -import org.apache.zookeeper.KeeperException.NoWatcherException; import org.apache.zookeeper.OpResult.ErrorResult; -import org.apache.zookeeper.Watcher.Event.EventType; import org.apache.zookeeper.Watcher.WatcherType; import org.apache.zookeeper.client.ConnectStringParser; import org.apache.zookeeper.client.HostProvider; @@ -85,7 +81,6 @@ import org.apache.zookeeper.proto.SyncResponse; import org.apache.zookeeper.server.DataTree; import org.apache.zookeeper.server.EphemeralType; -import org.apache.zookeeper.server.watch.PathParentIterator; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -231,8 +226,6 @@ public ZooKeeperSaslClient getSaslClient() { return cnxn.zooKeeperSaslClient; } - protected final ZKWatchManager watchManager; - private final ZKClientConfig clientConfig; public ZKClientConfig getClientConfig() { @@ -240,407 +233,37 @@ public ZKClientConfig getClientConfig() { } protected List getDataWatches() { - synchronized (watchManager.dataWatches) { - List rc = new ArrayList(watchManager.dataWatches.keySet()); - return rc; - } + return getWatchManager().getDataWatchList(); } + protected List getExistWatches() { - synchronized (watchManager.existWatches) { - List rc = new ArrayList(watchManager.existWatches.keySet()); - return rc; - } + return getWatchManager().getExistWatchList(); } + protected List getChildWatches() { - synchronized (watchManager.childWatches) { - List rc = new ArrayList(watchManager.childWatches.keySet()); - return rc; - } + return getWatchManager().getChildWatchList(); } + protected List getPersistentWatches() { - synchronized (watchManager.persistentWatches) { - List rc = new ArrayList(watchManager.persistentWatches.keySet()); - return rc; - } + return getWatchManager().getPersistentWatchList(); } + protected List getPersistentRecursiveWatches() { - synchronized (watchManager.persistentRecursiveWatches) { - List rc = new ArrayList(watchManager.persistentRecursiveWatches.keySet()); - return rc; - } + return getWatchManager().getPersistentRecursiveWatchList(); } - /** - * Manage watchers and handle events generated by the ClientCnxn object. - * - * We are implementing this as a nested class of ZooKeeper so that - * the public methods will not be exposed as part of the ZooKeeper client - * API. - */ - static class ZKWatchManager implements ClientWatchManager { - - private final Map> dataWatches = new HashMap>(); - private final Map> existWatches = new HashMap>(); - private final Map> childWatches = new HashMap>(); - private final Map> persistentWatches = new HashMap>(); - private final Map> persistentRecursiveWatches = new HashMap>(); - private boolean disableAutoWatchReset; - - ZKWatchManager(boolean disableAutoWatchReset) { - this.disableAutoWatchReset = disableAutoWatchReset; - } - - protected volatile Watcher defaultWatcher; - - private void addTo(Set from, Set to) { - if (from != null) { - to.addAll(from); - } - } - - public Map> removeWatcher( - String clientPath, - Watcher watcher, - WatcherType watcherType, - boolean local, - int rc) throws KeeperException { - // Validate the provided znode path contains the given watcher of - // watcherType - containsWatcher(clientPath, watcher, watcherType); - - Map> removedWatchers = new HashMap<>(); - HashSet childWatchersToRem = new HashSet<>(); - removedWatchers.put(EventType.ChildWatchRemoved, childWatchersToRem); - HashSet dataWatchersToRem = new HashSet<>(); - removedWatchers.put(EventType.DataWatchRemoved, dataWatchersToRem); - HashSet persistentWatchersToRem = new HashSet<>(); - removedWatchers.put(EventType.PersistentWatchRemoved, persistentWatchersToRem); - boolean removedWatcher = false; - switch (watcherType) { - case Children: { - synchronized (childWatches) { - removedWatcher = removeWatches(childWatches, watcher, clientPath, local, rc, childWatchersToRem); - } - break; - } - case Data: { - synchronized (dataWatches) { - removedWatcher = removeWatches(dataWatches, watcher, clientPath, local, rc, dataWatchersToRem); - } - - synchronized (existWatches) { - boolean removedDataWatcher = removeWatches(existWatches, watcher, clientPath, local, rc, dataWatchersToRem); - removedWatcher |= removedDataWatcher; - } - break; - } - case Any: { - synchronized (childWatches) { - removedWatcher = removeWatches(childWatches, watcher, clientPath, local, rc, childWatchersToRem); - } - - synchronized (dataWatches) { - boolean removedDataWatcher = removeWatches(dataWatches, watcher, clientPath, local, rc, dataWatchersToRem); - removedWatcher |= removedDataWatcher; - } - - synchronized (existWatches) { - boolean removedDataWatcher = removeWatches(existWatches, watcher, clientPath, local, rc, dataWatchersToRem); - removedWatcher |= removedDataWatcher; - } - - synchronized (persistentWatches) { - boolean removedPersistentWatcher = removeWatches(persistentWatches, - watcher, clientPath, local, rc, persistentWatchersToRem); - removedWatcher |= removedPersistentWatcher; - } - - synchronized (persistentRecursiveWatches) { - boolean removedPersistentRecursiveWatcher = removeWatches(persistentRecursiveWatches, - watcher, clientPath, local, rc, persistentWatchersToRem); - removedWatcher |= removedPersistentRecursiveWatcher; - } - } - } - // Watcher function doesn't exists for the specified params - if (!removedWatcher) { - throw new KeeperException.NoWatcherException(clientPath); - } - return removedWatchers; - } - - private boolean contains(String path, Watcher watcherObj, Map> pathVsWatchers) { - boolean watcherExists = true; - if (pathVsWatchers == null || pathVsWatchers.size() == 0) { - watcherExists = false; - } else { - Set watchers = pathVsWatchers.get(path); - if (watchers == null) { - watcherExists = false; - } else if (watcherObj == null) { - watcherExists = watchers.size() > 0; - } else { - watcherExists = watchers.contains(watcherObj); - } - } - return watcherExists; - } - - /** - * Validate the provided znode path contains the given watcher and - * watcherType - * - * @param path - * - client path - * @param watcher - * - watcher object reference - * @param watcherType - * - type of the watcher - * @throws NoWatcherException - */ - void containsWatcher(String path, Watcher watcher, WatcherType watcherType) throws NoWatcherException { - boolean containsWatcher = false; - switch (watcherType) { - case Children: { - synchronized (childWatches) { - containsWatcher = contains(path, watcher, childWatches); - } - - synchronized (persistentWatches) { - boolean contains_temp = contains(path, watcher, - persistentWatches); - containsWatcher |= contains_temp; - } - - synchronized (persistentRecursiveWatches) { - boolean contains_temp = contains(path, watcher, - persistentRecursiveWatches); - containsWatcher |= contains_temp; - } - break; - } - case Data: { - synchronized (dataWatches) { - containsWatcher = contains(path, watcher, dataWatches); - } - - synchronized (existWatches) { - boolean contains_temp = contains(path, watcher, existWatches); - containsWatcher |= contains_temp; - } - - synchronized (persistentWatches) { - boolean contains_temp = contains(path, watcher, - persistentWatches); - containsWatcher |= contains_temp; - } - - synchronized (persistentRecursiveWatches) { - boolean contains_temp = contains(path, watcher, - persistentRecursiveWatches); - containsWatcher |= contains_temp; - } - break; - } - case Any: { - synchronized (childWatches) { - containsWatcher = contains(path, watcher, childWatches); - } - - synchronized (dataWatches) { - boolean contains_temp = contains(path, watcher, dataWatches); - containsWatcher |= contains_temp; - } - - synchronized (existWatches) { - boolean contains_temp = contains(path, watcher, existWatches); - containsWatcher |= contains_temp; - } - - synchronized (persistentWatches) { - boolean contains_temp = contains(path, watcher, - persistentWatches); - containsWatcher |= contains_temp; - } - - synchronized (persistentRecursiveWatches) { - boolean contains_temp = contains(path, watcher, - persistentRecursiveWatches); - containsWatcher |= contains_temp; - } - } - } - // Watcher function doesn't exists for the specified params - if (!containsWatcher) { - throw new KeeperException.NoWatcherException(path); - } - } - - protected boolean removeWatches( - Map> pathVsWatcher, - Watcher watcher, - String path, - boolean local, - int rc, - Set removedWatchers) throws KeeperException { - if (!local && rc != Code.OK.intValue()) { - throw KeeperException.create(KeeperException.Code.get(rc), path); - } - boolean success = false; - // When local flag is true, remove watchers for the given path - // irrespective of rc. Otherwise shouldn't remove watchers locally - // when sees failure from server. - if (rc == Code.OK.intValue() || (local && rc != Code.OK.intValue())) { - // Remove all the watchers for the given path - if (watcher == null) { - Set pathWatchers = pathVsWatcher.remove(path); - if (pathWatchers != null) { - // found path watchers - removedWatchers.addAll(pathWatchers); - success = true; - } - } else { - Set watchers = pathVsWatcher.get(path); - if (watchers != null) { - if (watchers.remove(watcher)) { - // found path watcher - removedWatchers.add(watcher); - // cleanup - if (watchers.size() <= 0) { - pathVsWatcher.remove(path); - } - success = true; - } - } - } - } - return success; - } - - /* (non-Javadoc) - * @see org.apache.zookeeper.ClientWatchManager#materialize(Event.KeeperState, - * Event.EventType, java.lang.String) - */ - @Override - public Set materialize( - Watcher.Event.KeeperState state, - Watcher.Event.EventType type, - String clientPath - ) { - final Set result = new HashSet<>(); - - switch (type) { - case None: - if (defaultWatcher != null) { - result.add(defaultWatcher); - } - - boolean clear = disableAutoWatchReset && state != Watcher.Event.KeeperState.SyncConnected; - synchronized (dataWatches) { - for (Set ws : dataWatches.values()) { - result.addAll(ws); - } - if (clear) { - dataWatches.clear(); - } - } - - synchronized (existWatches) { - for (Set ws : existWatches.values()) { - result.addAll(ws); - } - if (clear) { - existWatches.clear(); - } - } - - synchronized (childWatches) { - for (Set ws : childWatches.values()) { - result.addAll(ws); - } - if (clear) { - childWatches.clear(); - } - } - - synchronized (persistentWatches) { - for (Set ws: persistentWatches.values()) { - result.addAll(ws); - } - } - - synchronized (persistentRecursiveWatches) { - for (Set ws: persistentRecursiveWatches.values()) { - result.addAll(ws); - } - } - - return result; - case NodeDataChanged: - case NodeCreated: - synchronized (dataWatches) { - addTo(dataWatches.remove(clientPath), result); - } - synchronized (existWatches) { - addTo(existWatches.remove(clientPath), result); - } - addPersistentWatches(clientPath, result); - break; - case NodeChildrenChanged: - synchronized (childWatches) { - addTo(childWatches.remove(clientPath), result); - } - addPersistentWatches(clientPath, result); - break; - case NodeDeleted: - synchronized (dataWatches) { - addTo(dataWatches.remove(clientPath), result); - } - // TODO This shouldn't be needed, but just in case - synchronized (existWatches) { - Set list = existWatches.remove(clientPath); - if (list != null) { - addTo(list, result); - LOG.warn("We are triggering an exists watch for delete! Shouldn't happen!"); - } - } - synchronized (childWatches) { - addTo(childWatches.remove(clientPath), result); - } - addPersistentWatches(clientPath, result); - break; - default: - String errorMsg = String.format( - "Unhandled watch event type %s with state %s on path %s", - type, - state, - clientPath); - LOG.error(errorMsg); - throw new RuntimeException(errorMsg); - } - - return result; - } - - private void addPersistentWatches(String clientPath, Set result) { - synchronized (persistentWatches) { - addTo(persistentWatches.get(clientPath), result); - } - synchronized (persistentRecursiveWatches) { - for (String path : PathParentIterator.forAll(clientPath).asIterable()) { - addTo(persistentRecursiveWatches.get(path), result); - } - } - } + ZKWatchManager getWatchManager() { + return cnxn.getWatcherManager(); } /** * Register a watcher for a particular path. */ - public abstract class WatchRegistration { + public abstract static class WatchRegistration { private Watcher watcher; private String clientPath; + public WatchRegistration(Watcher watcher, String clientPath) { this.watcher = watcher; this.clientPath = clientPath; @@ -689,7 +312,7 @@ public ExistsWatchRegistration(Watcher watcher, String clientPath) { @Override protected Map> getWatches(int rc) { - return rc == 0 ? watchManager.dataWatches : watchManager.existWatches; + return rc == 0 ? getWatchManager().getDataWatches() : getWatchManager().getExistWatches(); } @Override @@ -707,7 +330,7 @@ public DataWatchRegistration(Watcher watcher, String clientPath) { @Override protected Map> getWatches(int rc) { - return watchManager.dataWatches; + return getWatchManager().getDataWatches(); } } @@ -720,7 +343,7 @@ public ChildWatchRegistration(Watcher watcher, String clientPath) { @Override protected Map> getWatches(int rc) { - return watchManager.childWatches; + return getWatchManager().getChildWatches(); } } @@ -737,9 +360,9 @@ public AddWatchRegistration(Watcher watcher, String clientPath, AddWatchMode mod protected Map> getWatches(int rc) { switch (mode) { case PERSISTENT: - return watchManager.persistentWatches; + return getWatchManager().getPersistentWatches(); case PERSISTENT_RECURSIVE: - return watchManager.persistentRecursiveWatches; + return getWatchManager().getPersistentRecursiveWatches(); } throw new IllegalArgumentException("Mode not supported: " + mode); } @@ -989,7 +612,7 @@ public ZooKeeper( * connects to one in read-only mode, i.e. read requests are * allowed while write requests are not. It continues seeking for * majority in the background. - * @param aHostProvider + * @param hostProvider * use this as HostProvider to enable custom behaviour. * @param clientConfig * (added in 3.5.2) passing this conf object gives each client the flexibility of @@ -1004,49 +627,45 @@ public ZooKeeper( int sessionTimeout, Watcher watcher, boolean canBeReadOnly, - HostProvider aHostProvider, - ZKClientConfig clientConfig) throws IOException { + HostProvider hostProvider, + ZKClientConfig clientConfig + ) throws IOException { LOG.info( "Initiating client connection, connectString={} sessionTimeout={} watcher={}", connectString, sessionTimeout, watcher); - if (clientConfig == null) { - clientConfig = new ZKClientConfig(); - } - this.clientConfig = clientConfig; - watchManager = defaultWatchManager(); - watchManager.defaultWatcher = watcher; + this.clientConfig = clientConfig != null ? clientConfig : new ZKClientConfig(); + this.hostProvider = hostProvider; ConnectStringParser connectStringParser = new ConnectStringParser(connectString); - hostProvider = aHostProvider; cnxn = createConnection( connectStringParser.getChrootPath(), hostProvider, sessionTimeout, - this, - watchManager, + this.clientConfig, + watcher, getClientCnxnSocket(), canBeReadOnly); cnxn.start(); } - // @VisibleForTesting - protected ClientCnxn createConnection( + ClientCnxn createConnection( String chrootPath, HostProvider hostProvider, int sessionTimeout, - ZooKeeper zooKeeper, - ClientWatchManager watcher, + ZKClientConfig clientConfig, + Watcher defaultWatcher, ClientCnxnSocket clientCnxnSocket, - boolean canBeReadOnly) throws IOException { + boolean canBeReadOnly + ) throws IOException { return new ClientCnxn( chrootPath, hostProvider, sessionTimeout, - this, - watchManager, + clientConfig, + defaultWatcher, clientCnxnSocket, canBeReadOnly); } @@ -1383,7 +1002,7 @@ public ZooKeeper( * connects to one in read-only mode, i.e. read requests are * allowed while write requests are not. It continues seeking for * majority in the background. - * @param aHostProvider + * @param hostProvider * use this as HostProvider to enable custom behaviour. * @param clientConfig * (added in 3.5.2) passing this conf object gives each client the flexibility of @@ -1400,7 +1019,7 @@ public ZooKeeper( long sessionId, byte[] sessionPasswd, boolean canBeReadOnly, - HostProvider aHostProvider, + HostProvider hostProvider, ZKClientConfig clientConfig) throws IOException { LOG.info( "Initiating client connection, connectString={} " @@ -1411,22 +1030,16 @@ public ZooKeeper( Long.toHexString(sessionId), (sessionPasswd == null ? "" : "")); - if (clientConfig == null) { - clientConfig = new ZKClientConfig(); - } - this.clientConfig = clientConfig; - watchManager = defaultWatchManager(); - watchManager.defaultWatcher = watcher; - + this.clientConfig = clientConfig != null ? clientConfig : new ZKClientConfig(); ConnectStringParser connectStringParser = new ConnectStringParser(connectString); - hostProvider = aHostProvider; + this.hostProvider = hostProvider; cnxn = new ClientCnxn( connectStringParser.getChrootPath(), hostProvider, sessionTimeout, - this, - watchManager, + this.clientConfig, + watcher, getClientCnxnSocket(), sessionId, sessionPasswd, @@ -1523,11 +1136,6 @@ public Testable getTestable() { return new ZooKeeperTestable(cnxn); } - /* Useful for testing watch handling behavior */ - protected ZKWatchManager defaultWatchManager() { - return new ZKWatchManager(getClientConfig().getBoolean(ZKClientConfig.DISABLE_AUTO_WATCH_RESET)); - } - /** * The session id for this ZooKeeper client instance. The value returned is * not valid until the client connects to a server and may change after a @@ -1582,11 +1190,9 @@ public void addAuthInfo(String scheme, byte[] auth) { /** * Specify the default watcher for the connection (overrides the one * specified during construction). - * - * @param watcher */ public synchronized void register(Watcher watcher) { - watchManager.defaultWatcher = watcher; + getWatchManager().setDefaultWatcher(watcher); } /** @@ -3213,9 +2819,11 @@ public void addWatch(String basePath, Watcher watcher, AddWatchMode mode) * error code. * @since 3.6.0 */ - public void addWatch(String basePath, AddWatchMode mode) - throws KeeperException, InterruptedException { - addWatch(basePath, watchManager.defaultWatcher, mode); + public void addWatch( + String basePath, + AddWatchMode mode + ) throws KeeperException, InterruptedException { + addWatch(basePath, getWatchManager().getDefaultWatcher(), mode); } /** @@ -3229,8 +2837,12 @@ public void addWatch(String basePath, AddWatchMode mode) * @throws IllegalArgumentException if an invalid path is specified * @since 3.6.0 */ - public void addWatch(String basePath, Watcher watcher, AddWatchMode mode, - VoidCallback cb, Object ctx) { + public void addWatch( + String basePath, + Watcher watcher, AddWatchMode mode, + VoidCallback cb, + Object ctx + ) { PathUtils.validatePath(basePath); String serverPath = prependChroot(basePath); @@ -3251,9 +2863,8 @@ public void addWatch(String basePath, Watcher watcher, AddWatchMode mode, * @throws IllegalArgumentException if an invalid path is specified * @since 3.6.0 */ - public void addWatch(String basePath, AddWatchMode mode, - VoidCallback cb, Object ctx) { - addWatch(basePath, watchManager.defaultWatcher, mode, cb, ctx); + public void addWatch(String basePath, AddWatchMode mode, VoidCallback cb, Object ctx) { + addWatch(basePath, getWatchManager().getDefaultWatcher(), mode, cb, ctx); } private void validateWatcher(Watcher watcher) { @@ -3271,7 +2882,7 @@ private void removeWatches( PathUtils.validatePath(path); final String clientPath = path; final String serverPath = prependChroot(clientPath); - WatchDeregistration wcb = new WatchDeregistration(clientPath, watcher, watcherType, local, watchManager); + WatchDeregistration wcb = new WatchDeregistration(clientPath, watcher, watcherType, local, getWatchManager()); RequestHeader h = new RequestHeader(); h.setType(opCode); @@ -3294,7 +2905,7 @@ private void removeWatches( PathUtils.validatePath(path); final String clientPath = path; final String serverPath = prependChroot(clientPath); - WatchDeregistration wcb = new WatchDeregistration(clientPath, watcher, watcherType, local, watchManager); + WatchDeregistration wcb = new WatchDeregistration(clientPath, watcher, watcherType, local, getWatchManager()); RequestHeader h = new RequestHeader(); h.setType(opCode); @@ -3424,8 +3035,9 @@ private ClientCnxnSocket getClientCnxnSocket() throws IOException { */ private Watcher getDefaultWatcher(boolean required) { if (required) { - if (watchManager.defaultWatcher != null) { - return watchManager.defaultWatcher; + final Watcher defaultWatcher = getWatchManager().getDefaultWatcher(); + if (defaultWatcher != null) { + return defaultWatcher; } else { throw new IllegalStateException("Default watcher is required, but it is null."); } diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/ClientCnxnSocketFragilityTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/ClientCnxnSocketFragilityTest.java index 2c7f6bd0547..5ed43fca72e 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/ClientCnxnSocketFragilityTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/ClientCnxnSocketFragilityTest.java @@ -281,11 +281,19 @@ public CustomClientCnxn( String chrootPath, HostProvider hostProvider, int sessionTimeout, - ZooKeeper zooKeeper, - ClientWatchManager watcher, + ZKClientConfig zkClientConfig, + Watcher defaultWatcher, ClientCnxnSocket clientCnxnSocket, - boolean canBeReadOnly) throws IOException { - super(chrootPath, hostProvider, sessionTimeout, zooKeeper, watcher, clientCnxnSocket, canBeReadOnly); + boolean canBeReadOnly + ) throws IOException { + super( + chrootPath, + hostProvider, + sessionTimeout, + zkClientConfig, + defaultWatcher, + clientCnxnSocket, + canBeReadOnly); } void attemptClose() { @@ -344,18 +352,25 @@ public boolean isAlive() { return cnxn.getState().isAlive(); } - @Override - protected ClientCnxn createConnection( + ClientCnxn createConnection( String chrootPath, HostProvider hostProvider, int sessionTimeout, - ZooKeeper zooKeeper, - ClientWatchManager watcher, + ZKClientConfig clientConfig, + Watcher defaultWatcher, ClientCnxnSocket clientCnxnSocket, - boolean canBeReadOnly) throws IOException { + boolean canBeReadOnly + ) throws IOException { Assert.assertTrue(clientCnxnSocket instanceof FragileClientCnxnSocketNIO); socket = (FragileClientCnxnSocketNIO) clientCnxnSocket; - ClientCnxnSocketFragilityTest.this.cnxn = new CustomClientCnxn(chrootPath, hostProvider, sessionTimeout, zooKeeper, watcher, clientCnxnSocket, canBeReadOnly); + ClientCnxnSocketFragilityTest.this.cnxn = new CustomClientCnxn( + chrootPath, + hostProvider, + sessionTimeout, + clientConfig, + defaultWatcher, + clientCnxnSocket, + canBeReadOnly); return ClientCnxnSocketFragilityTest.this.cnxn; } } diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/ClientReconnectTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/ClientReconnectTest.java index e77283c84e5..41f52ea697d 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/ClientReconnectTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/ClientReconnectTest.java @@ -66,8 +66,14 @@ public void testClientReconnect() throws IOException, InterruptedException { sc = SocketChannel.open(); ClientCnxnSocketNIO nioCnxn = new MockCnxn(); - ClientWatchManager watcher = mock(ClientWatchManager.class); - ClientCnxn clientCnxn = new ClientCnxn("tmp", hostProvider, 5000, zk, watcher, nioCnxn, false); + ClientCnxn clientCnxn = new ClientCnxn( + "tmp", + hostProvider, + 5000, + zk.getClientConfig(), + DummyWatcher.INSTANCE, + nioCnxn, + false); clientCnxn.start(); countDownLatch.await(5000, TimeUnit.MILLISECONDS); assertTrue(countDownLatch.getCount() == 0); diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/ClientRequestTimeoutTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/ClientRequestTimeoutTest.java index 7c3bf5198f4..d4f95aeb50d 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/ClientRequestTimeoutTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/ClientRequestTimeoutTest.java @@ -25,6 +25,7 @@ import java.io.IOException; import org.apache.zookeeper.ZooDefs.Ids; import org.apache.zookeeper.client.HostProvider; +import org.apache.zookeeper.client.ZKClientConfig; import org.apache.zookeeper.server.quorum.QuorumPeerTestBase; import org.apache.zookeeper.test.ClientBase; import org.apache.zookeeper.test.ClientBase.CountdownWatcher; @@ -110,15 +111,23 @@ private String getCxnString(int[] clientPorts) { class CustomClientCnxn extends ClientCnxn { - public CustomClientCnxn( + CustomClientCnxn( String chrootPath, HostProvider hostProvider, int sessionTimeout, - ZooKeeper zooKeeper, - ClientWatchManager watcher, + ZKClientConfig clientConfig, + Watcher defaultWatcher, ClientCnxnSocket clientCnxnSocket, - boolean canBeReadOnly) throws IOException { - super(chrootPath, hostProvider, sessionTimeout, zooKeeper, watcher, clientCnxnSocket, canBeReadOnly); + boolean canBeReadOnly + ) throws IOException { + super( + chrootPath, + hostProvider, + sessionTimeout, + clientConfig, + defaultWatcher, + clientCnxnSocket, + canBeReadOnly); } @Override @@ -140,15 +149,23 @@ public CustomZooKeeper(String connectString, int sessionTimeout, Watcher watcher } @Override - protected ClientCnxn createConnection( + ClientCnxn createConnection( String chrootPath, HostProvider hostProvider, int sessionTimeout, - ZooKeeper zooKeeper, - ClientWatchManager watcher, + ZKClientConfig clientConfig, + Watcher defaultWatcher, ClientCnxnSocket clientCnxnSocket, - boolean canBeReadOnly) throws IOException { - return new CustomClientCnxn(chrootPath, hostProvider, sessionTimeout, zooKeeper, watcher, clientCnxnSocket, canBeReadOnly); + boolean canBeReadOnly + ) throws IOException { + return new CustomClientCnxn( + chrootPath, + hostProvider, + sessionTimeout, + clientConfig, + defaultWatcher, + clientCnxnSocket, + canBeReadOnly); } } diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/RemoveWatchesTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/RemoveWatchesTest.java index 30f1558fd01..8bb91e338bd 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/RemoveWatchesTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/RemoveWatchesTest.java @@ -18,12 +18,16 @@ package org.apache.zookeeper; +import static org.hamcrest.CoreMatchers.is; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.spy; import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; @@ -37,11 +41,9 @@ import java.util.concurrent.TimeoutException; import org.apache.commons.collections.CollectionUtils; import org.apache.zookeeper.KeeperException.Code; -import org.apache.zookeeper.KeeperException.NoWatcherException; import org.apache.zookeeper.Watcher.Event.EventType; import org.apache.zookeeper.Watcher.WatcherType; import org.apache.zookeeper.ZooDefs.Ids; -import org.apache.zookeeper.client.ZKClientConfig; import org.apache.zookeeper.server.ServerCnxn; import org.apache.zookeeper.test.ClientBase; import org.junit.Test; @@ -657,7 +659,9 @@ public void testChRootRemoveWatcher() throws Exception { @Test(timeout = 90000) public void testNoWatcherServerException() throws InterruptedException, IOException, TimeoutException { CountdownWatcher watcher = new CountdownWatcher(); - MyZooKeeper zk = new MyZooKeeper(hostPort, CONNECTION_TIMEOUT, watcher); + ZooKeeper zk = spy(new ZooKeeper(hostPort, CONNECTION_TIMEOUT, watcher)); + MyWatchManager watchManager = new MyWatchManager(false, watcher); + doReturn(watchManager).when(zk).getWatchManager(); boolean nw = false; watcher.waitForConnected(CONNECTION_TIMEOUT); @@ -670,8 +674,8 @@ public void testNoWatcherServerException() throws InterruptedException, IOExcept } } - assertTrue("Server didn't return NOWATCHER", zk.getRemoveWatchesRC() == Code.NOWATCHER.intValue()); - assertTrue("NoWatcherException didn't happen", nw); + assertThat("Server didn't return NOWATCHER", watchManager.lastReturnCode, is(Code.NOWATCHER.intValue())); + assertThat("NoWatcherException didn't happen", nw, is(true)); } /** @@ -920,54 +924,32 @@ public void testRemoveAllWatchesOnAPath() throws Exception { assertEquals("Received watch notification after removal!", 2, watchCount.getCount()); } - /* a mocked ZK class that doesn't do client-side verification - * before/after calling removeWatches */ - private class MyZooKeeper extends ZooKeeper { + private static class MyWatchManager extends ZKWatchManager { - class MyWatchManager extends ZKWatchManager { + int lastReturnCode; - public MyWatchManager(boolean disableAutoWatchReset) { - super(disableAutoWatchReset); - } - - public int lastrc; - - /* Pretend that any watcher exists */ - void containsWatcher(String path, Watcher watcher, WatcherType watcherType) throws NoWatcherException { - } - - /* save the return error code by the server */ - protected boolean removeWatches( - Map> pathVsWatcher, - Watcher watcher, - String path, - boolean local, - int rc, - Set removedWatchers) throws KeeperException { - lastrc = rc; - return false; - } - - } - - public MyZooKeeper(String hp, int timeout, Watcher watcher) throws IOException { - super(hp, timeout, watcher, false); + MyWatchManager(boolean disableAutoWatchReset, Watcher defaultWatcher) { + super(disableAutoWatchReset, defaultWatcher); } - private MyWatchManager myWatchManager; - - protected ZKWatchManager defaultWatchManager() { - myWatchManager = new MyWatchManager(getClientConfig().getBoolean(ZKClientConfig.DISABLE_AUTO_WATCH_RESET)); - return myWatchManager; + void containsWatcher(String path, Watcher watcher, WatcherType watcherType) { + // prevent contains watcher } - public int getRemoveWatchesRC() { - return myWatchManager.lastrc; + @Override + protected boolean removeWatches( + Map> pathVsWatcher, + Watcher watcher, + String path, + boolean local, + int rc, + Set removedWatchers) { + lastReturnCode = rc; + return false; } - } - private class MyWatcher implements Watcher { + private static class MyWatcher implements Watcher { private final String path; private String eventPath; diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/TestableZooKeeper.java b/zookeeper-server/src/test/java/org/apache/zookeeper/TestableZooKeeper.java index a02127aae07..7f9e41e3380 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/TestableZooKeeper.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/TestableZooKeeper.java @@ -20,12 +20,10 @@ import java.io.IOException; import java.net.SocketAddress; -import java.util.List; import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; import org.apache.jute.Record; import org.apache.zookeeper.admin.ZooKeeperAdmin; -import org.apache.zookeeper.client.HostProvider; import org.apache.zookeeper.proto.ReplyHeader; import org.apache.zookeeper.proto.RequestHeader; @@ -35,76 +33,12 @@ public TestableZooKeeper(String host, int sessionTimeout, Watcher watcher) throw super(host, sessionTimeout, watcher); } - class TestableClientCnxn extends ClientCnxn { - - TestableClientCnxn( - String chrootPath, - HostProvider hostProvider, - int sessionTimeout, - ZooKeeper zooKeeper, - ClientWatchManager watcher, - ClientCnxnSocket clientCnxnSocket, - boolean canBeReadOnly) throws IOException { - super(chrootPath, - hostProvider, - sessionTimeout, - zooKeeper, - watcher, - clientCnxnSocket, - 0, - new byte[16], - canBeReadOnly); - } - - void setXid(int newXid) { - xid = newXid; - } - - int checkXid() { - return xid; - } - - } - - protected ClientCnxn createConnection( - String chrootPath, - HostProvider hostProvider, - int sessionTimeout, - ZooKeeper zooKeeper, - ClientWatchManager watcher, - ClientCnxnSocket clientCnxnSocket, - boolean canBeReadOnly) throws IOException { - return new TestableClientCnxn( - chrootPath, - hostProvider, - sessionTimeout, - this, - watcher, - clientCnxnSocket, - canBeReadOnly); - } - public void setXid(int xid) { - ((TestableClientCnxn) cnxn).setXid(xid); + cnxn.xid = xid; } public int checkXid() { - return ((TestableClientCnxn) cnxn).checkXid(); - } - - @Override - public List getChildWatches() { - return super.getChildWatches(); - } - - @Override - public List getDataWatches() { - return super.getDataWatches(); - } - - @Override - public List getExistWatches() { - return super.getExistWatches(); + return cnxn.xid; } /** From a908001be9641d78040b1954acb0cd3a8e9e42c2 Mon Sep 17 00:00:00 2001 From: Damien Diederen Date: Sun, 17 May 2020 15:15:54 +0200 Subject: [PATCH 038/591] ZOOKEEPER-1998: Allow C client to throttle host name resolutions Some environments experience high DNS load because of the name resolutions introduced by [ZOOKEEPER-1355](https://issues.apache.org/jira/browse/ZOOKEEPER-1355). This patch allows clients to set a minimum delay to observe between "routine" resolutions using a `zoo_set_servers_resolution_delay` API function. An application can influence the rate of polling via its `delay_ms` parameter: when set to a value greater than zero, the client skips most "routine" resolutions which would have happened in a window of that many milliseconds since the last successful one. Setting `delay_ms` to `0` disables the new logic, reverting to the default behavior. Setting it to `-1` disables network resolutions during normal operation (but not, e.g., on connection loss). Author: Damien Diederen Reviewers: Enrico Olivelli , Mate Szalay-Beko , Suhas Dantkale Closes #1068 from ztzg/ZOOKEEPER-1998-throttle-dns-resolutions --- .../zookeeper-client-c/include/zookeeper.h | 26 ++++++ .../zookeeper-client-c/src/zk_adaptor.h | 3 + .../zookeeper-client-c/src/zookeeper.c | 84 +++++++++++++++---- .../zookeeper-client-c/tests/TestClient.cc | 74 +++++++++++++++- 4 files changed, 171 insertions(+), 16 deletions(-) diff --git a/zookeeper-client/zookeeper-client-c/include/zookeeper.h b/zookeeper-client/zookeeper-client-c/include/zookeeper.h index d33a44695a7..95959af56fe 100644 --- a/zookeeper-client/zookeeper-client-c/include/zookeeper.h +++ b/zookeeper-client/zookeeper-client-c/include/zookeeper.h @@ -699,6 +699,32 @@ ZOOAPI sasl_callback_t *zoo_sasl_make_basic_callbacks(const char *user, */ ZOOAPI int zoo_set_servers(zhandle_t *zh, const char *hosts); +/** + * \brief sets a minimum delay to observe between "routine" host name + * resolutions. + * + * The client performs regular resolutions of the list of servers + * passed to \ref zookeeper_init or set with \ref zoo_set_servers in + * order to detect changes at the DNS level. + * + * By default, it does so every time it checks for socket readiness. + * This results in low latency in the detection of changes, but can + * lead to heavy DNS traffic when the local cache is not effective. + * + * This method allows an application to influence the rate of polling. + * When delay_ms is set to a value greater than zero, the client skips + * most "routine" resolutions which would have happened in a window of + * that many milliseconds since the last succesful one. + * + * Setting delay_ms to 0 disables this logic, reverting to the default + * behavior. Setting it to -1 disables network resolutions during + * normal operation (but not, e.g., on connection loss). + * + * \param delay_ms 0, -1, or the window size in milliseconds + * \return ZOK on success or ZBADARGUMENTS for invalid input parameters + */ +ZOOAPI int zoo_set_servers_resolution_delay(zhandle_t *zh, int delay_ms); + /** * \brief cycle to the next server on the next connection attempt. * diff --git a/zookeeper-client/zookeeper-client-c/src/zk_adaptor.h b/zookeeper-client/zookeeper-client-c/src/zk_adaptor.h index 57696d4b203..46a7e7ed6e3 100644 --- a/zookeeper-client/zookeeper-client-c/src/zk_adaptor.h +++ b/zookeeper-client/zookeeper-client-c/src/zk_adaptor.h @@ -200,6 +200,9 @@ struct _zhandle { addrvec_t addrs_old; // old list of addresses that we are no longer connected to addrvec_t addrs_new; // new list of addresses to connect to if we're reconfiguring + struct timeval last_resolve; // time of last hostname resolution + int resolve_delay_ms; // see zoo_set_servers_resolution_delay + int reconfig; // Are we in the process of reconfiguring cluster's ensemble double pOld, pNew; // Probability for selecting between 'addrs_old' and 'addrs_new' int delay; diff --git a/zookeeper-client/zookeeper-client-c/src/zookeeper.c b/zookeeper-client/zookeeper-client-c/src/zookeeper.c index a273225ff69..8df4b29dcc5 100644 --- a/zookeeper-client/zookeeper-client-c/src/zookeeper.c +++ b/zookeeper-client/zookeeper-client-c/src/zookeeper.c @@ -244,6 +244,8 @@ typedef struct _completion_list { } completion_list_t; const char*err2string(int err); +static inline int calculate_interval(const struct timeval *start, + const struct timeval *end); static int queue_session_event(zhandle_t *zh, int state); static const char* format_endpoint_info(const struct sockaddr_storage* ep); @@ -984,10 +986,14 @@ static int resolve_hosts(const zhandle_t *zh, const char *hosts_in, addrvec_t *a * * See zoo_cycle_next_server for the selection logic. * + * \param ref_time an optional "reference time," used to determine if + * resolution can be skipped in accordance to the delay set by \ref + * zoo_set_servers_resolution_delay. Passing NULL prevents skipping. + * * See {@link https://issues.apache.org/jira/browse/ZOOKEEPER-1355} for the * protocol and its evaluation, */ -int update_addrs(zhandle_t *zh) +int update_addrs(zhandle_t *zh, const struct timeval *ref_time) { int rc = ZOK; char *hosts = NULL; @@ -1008,26 +1014,54 @@ int update_addrs(zhandle_t *zh) return ZSYSTEMERROR; } - // NOTE: guard access to {hostname, addr_cur, addrs, addrs_old, addrs_new} + // NOTE: guard access to {hostname, addr_cur, addrs, addrs_old, addrs_new, last_resolve, resolve_delay_ms} lock_reconfig(zh); + // Check if we are due for a host name resolution. (See + // zoo_set_servers_resolution_delay. The answer is always "yes" + // if no reference is provided or the file descriptor is invalid.) + if (ref_time && zh->fd->sock != -1) { + int do_resolve; + + if (zh->resolve_delay_ms <= 0) { + // -1 disables, 0 means unconditional. Fail safe. + do_resolve = zh->resolve_delay_ms != -1; + } else { + int elapsed_ms = calculate_interval(&zh->last_resolve, ref_time); + // Include < 0 in case of overflow, or if we are not + // backed by a monotonic clock. + do_resolve = elapsed_ms > zh->resolve_delay_ms || elapsed_ms < 0; + } + + if (!do_resolve) { + goto finish; + } + } + // Copy zh->hostname for local use hosts = strdup(zh->hostname); if (hosts == NULL) { rc = ZSYSTEMERROR; - goto fail; + goto finish; } rc = resolve_hosts(zh, hosts, &resolved); if (rc != ZOK) { - goto fail; + goto finish; + } + + // Unconditionally note last resolution time. + if (ref_time) { + zh->last_resolve = *ref_time; + } else { + get_system_time(&zh->last_resolve); } // If the addrvec list is identical to last time we ran don't do anything if (addrvec_eq(&zh->addrs, &resolved)) { - goto fail; + goto finish; } // Is the server we're connected to in the new resolved list? @@ -1047,14 +1081,14 @@ int update_addrs(zhandle_t *zh) rc = addrvec_append(&zh->addrs_old, resolved_address); if (rc != ZOK) { - goto fail; + goto finish; } } else { rc = addrvec_append(&zh->addrs_new, resolved_address); if (rc != ZOK) { - goto fail; + goto finish; } } } @@ -1107,13 +1141,13 @@ int update_addrs(zhandle_t *zh) zh->state = ZOO_NOTCONNECTED_STATE; } -fail: +finish: unlock_reconfig(zh); // If we short-circuited out and never assigned resolved to zh->addrs then we // need to free resolved to avoid a memleak. - if (zh->addrs.data != resolved.data) + if (resolved.data && zh->addrs.data != resolved.data) { addrvec_free(&resolved); } @@ -1310,7 +1344,7 @@ static zhandle_t *zookeeper_init_internal(const char *host, watcher_fn watcher, if (zh->hostname == 0) { goto abort; } - if(update_addrs(zh) != 0) { + if(update_addrs(zh, NULL) != 0) { goto abort; } @@ -1404,7 +1438,7 @@ int zoo_set_servers(zhandle_t *zh, const char *hosts) return ZBADARGUMENTS; } - // NOTE: guard access to {hostname, addr_cur, addrs, addrs_old, addrs_new} + // NOTE: guard access to {hostname, addr_cur, addrs, addrs_old, addrs_new, last_resolve, resolve_delay_ms} lock_reconfig(zh); // Reset hostname to new set of hosts to connect to @@ -1416,7 +1450,27 @@ int zoo_set_servers(zhandle_t *zh, const char *hosts) unlock_reconfig(zh); - return update_addrs(zh); + return update_addrs(zh, NULL); +} + +/* + * Sets a minimum delay to observe between "routine" host name + * resolutions. See prototype for full documentation. + */ +int zoo_set_servers_resolution_delay(zhandle_t *zh, int delay_ms) { + if (delay_ms < -1) { + LOG_ERROR(LOGCALLBACK(zh), "Resolution delay cannot be %d", delay_ms); + return ZBADARGUMENTS; + } + + // NOTE: guard access to {hostname, addr_cur, addrs, addrs_old, addrs_new, last_resolve, resolve_delay_ms} + lock_reconfig(zh); + + zh->resolve_delay_ms = delay_ms; + + unlock_reconfig(zh); + + return ZOK; } /** @@ -1481,7 +1535,7 @@ static int get_next_server_in_reconfig(zhandle_t *zh) */ void zoo_cycle_next_server(zhandle_t *zh) { - // NOTE: guard access to {hostname, addr_cur, addrs, addrs_old, addrs_new} + // NOTE: guard access to {hostname, addr_cur, addrs, addrs_old, addrs_new, last_resolve, resolve_delay_ms} lock_reconfig(zh); memset(&zh->addr_cur, 0, sizeof(zh->addr_cur)); @@ -1513,7 +1567,7 @@ const char* zoo_get_current_server(zhandle_t* zh) { const char *endpoint_info = NULL; - // NOTE: guard access to {hostname, addr_cur, addrs, addrs_old, addrs_new} + // NOTE: guard access to {hostname, addr_cur, addrs, addrs_old, addrs_new, last_resolve, resolve_delay_ms} // Need the lock here as it is changed in update_addrs() lock_reconfig(zh); @@ -2415,7 +2469,7 @@ int zookeeper_interest(zhandle_t *zh, socket_t *fd, int *interest, } api_prolog(zh); - rc = update_addrs(zh); + rc = update_addrs(zh, &now); if (rc != ZOK) { return api_epilog(zh, rc); } diff --git a/zookeeper-client/zookeeper-client-c/tests/TestClient.cc b/zookeeper-client/zookeeper-client-c/tests/TestClient.cc index 2a6e9925470..80e4c928a51 100644 --- a/zookeeper-client/zookeeper-client-c/tests/TestClient.cc +++ b/zookeeper-client/zookeeper-client-c/tests/TestClient.cc @@ -227,6 +227,7 @@ class Zookeeper_simpleSystem : public CPPUNIT_NS::TestFixture CPPUNIT_TEST(testWatcherAutoResetWithLocal); CPPUNIT_TEST(testGetChildren2); CPPUNIT_TEST(testLastZxid); + CPPUNIT_TEST(testServersResolutionDelay); CPPUNIT_TEST(testRemoveWatchers); #endif CPPUNIT_TEST_SUITE_END(); @@ -386,7 +387,78 @@ class Zookeeper_simpleSystem : public CPPUNIT_NS::TestFixture CPPUNIT_ASSERT(zh->io_count < 2); zookeeper_close(zh); } - + + /* Checks the zoo_set_servers_resolution_delay default and operation */ + void testServersResolutionDelay() { + watchctx_t ctx; + zhandle_t *zk = createClient(&ctx); + int rc; + struct timeval tv; + struct Stat stat; + + CPPUNIT_ASSERT(zk); + CPPUNIT_ASSERT(zk->resolve_delay_ms == 0); + + // a) Default/0 case: resolve at each request. + + tv = zk->last_resolve; + usleep(10000); // 10ms + + rc = zoo_exists(zk, "/", 0, &stat); + CPPUNIT_ASSERT_EQUAL((int)ZOK, rc); + + // Must have changed because of the request. + CPPUNIT_ASSERT(zk->last_resolve.tv_sec != tv.tv_sec || + zk->last_resolve.tv_usec != tv.tv_usec); + + // b) Disabled/-1 case: never perform "routine" resolutions. + + rc = zoo_set_servers_resolution_delay(zk, -1); // Disabled + CPPUNIT_ASSERT_EQUAL((int)ZOK, rc); + + tv = zk->last_resolve; + usleep(10000); // 10ms + + rc = zoo_exists(zk, "/", 0, &stat); + CPPUNIT_ASSERT_EQUAL((int)ZOK, rc); + + // Must not have changed as auto-resolution is disabled. + CPPUNIT_ASSERT(zk->last_resolve.tv_sec == tv.tv_sec && + zk->last_resolve.tv_usec == tv.tv_usec); + + // c) Invalid delay is rejected. + + rc = zoo_set_servers_resolution_delay(zk, -1000); // Bad + CPPUNIT_ASSERT_EQUAL((int)ZBADARGUMENTS, rc); + + // d) Valid delay, no resolution within window. + + rc = zoo_set_servers_resolution_delay(zk, 500); // 0.5s + CPPUNIT_ASSERT_EQUAL((int)ZOK, rc); + + tv = zk->last_resolve; + usleep(10000); // 10ms + + rc = zoo_exists(zk, "/", 0, &stat); + CPPUNIT_ASSERT_EQUAL((int)ZOK, rc); + + // Must not have changed because the request (hopefully!) + // executed in less than 0.5s. + CPPUNIT_ASSERT(zk->last_resolve.tv_sec == tv.tv_sec && + zk->last_resolve.tv_usec == tv.tv_usec); + + // e) Valid delay, at least one resolution after delay. + + usleep(500 * 1000); // 0.5s + + rc = zoo_exists(zk, "/", 0, &stat); + CPPUNIT_ASSERT_EQUAL((int)ZOK, rc); + + // Must have changed because we waited 0.5s between the + // capture and the last request. + CPPUNIT_ASSERT(zk->last_resolve.tv_sec != tv.tv_sec || + zk->last_resolve.tv_usec != tv.tv_usec); + } void testPing() { From 5e88b541dcc2fd30e60bb437efa7704bbec8f6e6 Mon Sep 17 00:00:00 2001 From: David Mollitor Date: Wed, 20 May 2020 19:56:26 +0200 Subject: [PATCH 039/591] ZOOKEEPER-3834: Do Not Set Explicit Test Includes in POM Author: David Mollitor Reviewers: Enrico Olivelli , Norbert Kalmar Closes #1355 from belugabehr/ZOOKEEPER-3834 --- zookeeper-recipes/zookeeper-recipes-election/pom.xml | 3 --- zookeeper-recipes/zookeeper-recipes-lock/pom.xml | 3 --- zookeeper-recipes/zookeeper-recipes-queue/pom.xml | 3 --- 3 files changed, 9 deletions(-) diff --git a/zookeeper-recipes/zookeeper-recipes-election/pom.xml b/zookeeper-recipes/zookeeper-recipes-election/pom.xml index a4deb42eb2e..cc9728c0fb1 100755 --- a/zookeeper-recipes/zookeeper-recipes-election/pom.xml +++ b/zookeeper-recipes/zookeeper-recipes-election/pom.xml @@ -39,9 +39,6 @@ org.apache.maven.plugins maven-surefire-plugin - - **/*Test.java - ${surefire-forkcount} false -Xmx512m diff --git a/zookeeper-recipes/zookeeper-recipes-lock/pom.xml b/zookeeper-recipes/zookeeper-recipes-lock/pom.xml index 4a5b134cb0f..68dbec53485 100755 --- a/zookeeper-recipes/zookeeper-recipes-lock/pom.xml +++ b/zookeeper-recipes/zookeeper-recipes-lock/pom.xml @@ -39,9 +39,6 @@ org.apache.maven.plugins maven-surefire-plugin - - **/*Test.java - ${surefire-forkcount} false -Xmx512m diff --git a/zookeeper-recipes/zookeeper-recipes-queue/pom.xml b/zookeeper-recipes/zookeeper-recipes-queue/pom.xml index 3fa024694df..918fb647132 100755 --- a/zookeeper-recipes/zookeeper-recipes-queue/pom.xml +++ b/zookeeper-recipes/zookeeper-recipes-queue/pom.xml @@ -44,9 +44,6 @@ org.apache.maven.plugins maven-surefire-plugin - - **/*Test.java - ${surefire-forkcount} false -Xmx512m From 9e50853da2da5aa982a4b0da790f2da51575b6f1 Mon Sep 17 00:00:00 2001 From: Norbert Kalmar Date: Thu, 21 May 2020 11:19:39 +0200 Subject: [PATCH 040/591] ZOOKEEPER-3792: fix apidocs path Fix path that leads to the apidocs. I will manually correct the already generated paths in asf-site after this has been merged. This PR needs to land on master, 3.6 and 3.5 Author: Norbert Kalmar Reviewers: Mate Szalay-Beko Closes #1362 from nkalmar/master --- zookeeper-docs/src/main/resources/markdown/html/header.html | 2 +- zookeeper-docs/src/main/resources/markdown/index.md | 2 +- zookeeper-docs/src/main/resources/markdown/releasenotes.md | 2 +- zookeeper-docs/src/main/resources/markdown/skin/init.js | 2 +- .../src/main/resources/markdown/zookeeperProgrammers.md | 2 +- 5 files changed, 5 insertions(+), 5 deletions(-) diff --git a/zookeeper-docs/src/main/resources/markdown/html/header.html b/zookeeper-docs/src/main/resources/markdown/html/header.html index ef07e99b325..c529f880747 100644 --- a/zookeeper-docs/src/main/resources/markdown/html/header.html +++ b/zookeeper-docs/src/main/resources/markdown/html/header.html @@ -70,7 +70,7 @@