From 66e9eda68539c7eda5c67af4f12003382aba8529 Mon Sep 17 00:00:00 2001 From: Mate Szalay-Beko Date: Mon, 16 Dec 2019 22:09:36 +0100 Subject: [PATCH 001/118] ZOOKEEPER-3651: try to fix flaky NettyServerCnxnFactoryTest The testOutstandingHandshakeLimit is flaky, I tried to fix it in this commit. - I added extra comments and did some restructuring in the code. - Avoiding to start unnecessary ZooKeeper servers for tests don't require it - Decreasing the number of client connections the test tries to initiate - Increasing the timeout to make sure the connections get established - Filtering the 'SyncConnected' events in the client watcher to make sure the given connection is really established before counting it I think the last two points above should fix the flakiness. I tried to run the test in docker, and before the fix it failed for me once in every 4-5 execution. After applying these changes I re-executed it 100 times without failure. If these fixes are not enough, then we can introduce some only-visible-by-test method to add sleep in the SSLHandshake process in the production code to force to have handshakes in parallel. However, it would be nice to avoid that. Let's hope that these fixes will be enough. Author: Mate Szalay-Beko Reviewers: Enrico Olivelli , Norbert Kalmar Closes #1184 from symat/ZOOKEEPER-3651 (cherry picked from commit 20daae7d5fa934629e7825ed72e66ad76a94d6aa) Signed-off-by: Enrico Olivelli --- .../server/NettyServerCnxnFactoryTest.java | 150 +++++++++++++----- 1 file changed, 114 insertions(+), 36 deletions(-) diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/NettyServerCnxnFactoryTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/NettyServerCnxnFactoryTest.java index afb97b14ab3..76136c4ee4f 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/NettyServerCnxnFactoryTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/NettyServerCnxnFactoryTest.java @@ -43,24 +43,33 @@ public class NettyServerCnxnFactoryTest extends ClientBase { private static final Logger LOG = LoggerFactory .getLogger(NettyServerCnxnFactoryTest.class); - final LinkedBlockingQueue zks = new LinkedBlockingQueue(); + ClientX509Util x509Util; + final LinkedBlockingQueue zooKeeperClients = new LinkedBlockingQueue<>(); + @Override public void setUp() throws Exception { System.setProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN_FACTORY, "org.apache.zookeeper.server.NettyServerCnxnFactory"); - super.setUp(); + + // by default, we don't start any ZooKeeper server, as not all the tests are needing it. } @Override public void tearDown() throws Exception { - System.clearProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN_FACTORY); - // clean up - for (ZooKeeper zk : zks) { + System.clearProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN_FACTORY); + if (x509Util != null) { + SSLAuthTest.clearSecureSetting(x509Util); + } + for (ZooKeeper zk : zooKeeperClients) { zk.close(); } - super.tearDown(); + + //stopping the server only if it was started + if (serverFactory != null) { + super.tearDown(); + } } @Test @@ -96,63 +105,132 @@ public void testRebindIPv4IPv6() throws Exception { Assert.assertTrue(factory.getParentChannel().isActive()); } + /* + * In this test we are flooding the server with SSL connections, and expecting that not + * all the connection will succeed at once. Some of the connections should be closed, + * as there is a maximum number of parallel SSL handshake the server is willing to do + * for security reasons. + */ @Test public void testOutstandingHandshakeLimit() throws Exception { + // setting up SSL params, but disable some debug logs + x509Util = SSLAuthTest.setUpSecure(); + System.clearProperty("javax.net.debug"); + + // starting a single server (it will be closed in the tearDown) + setUpWithServerId(1); + + // initializing the statistics SimpleCounter tlsHandshakeExceeded = (SimpleCounter) ServerMetrics.getMetrics().TLS_HANDSHAKE_EXCEEDED; tlsHandshakeExceeded.reset(); Assert.assertEquals(tlsHandshakeExceeded.get(), 0); - ClientX509Util x509Util = SSLAuthTest.setUpSecure(); + // setting the HandshakeLimit to 3, so only 3 SSL handshakes can happen in parallel NettyServerCnxnFactory factory = (NettyServerCnxnFactory) serverFactory; factory.setSecure(true); - factory.setOutstandingHandshakeLimit(10); + factory.setOutstandingHandshakeLimit(3); + // starting the threads that will try to connect to the server + // we will have 3 threads, each of them establishing 3 connections int threadNum = 3; - int cnxnPerThread = 10; - Thread[] cnxnWorker = new Thread[threadNum]; - + int cnxnPerThread = 3; + int cnxnLimit = threadNum * cnxnPerThread; AtomicInteger cnxnCreated = new AtomicInteger(0); CountDownLatch latch = new CountDownLatch(1); - + Thread[] cnxnWorker = new Thread[threadNum]; for (int i = 0; i < cnxnWorker.length; i++) { - cnxnWorker[i] = new Thread() { - @Override - public void run() { - for (int i = 0; i < cnxnPerThread; i++) { - try { - zks.add(new ZooKeeper(hostPort, 3000, new Watcher() { - @Override - public void process(WatchedEvent event) { - int created = cnxnCreated.addAndGet(1); - if (created == threadNum * cnxnPerThread) { - latch.countDown(); - } - } - })); - } catch (Exception e) { - LOG.info("Error while creating zk client", e); - } - } - } - }; + cnxnWorker[i] = new ClientConnectionGenerator(i, cnxnPerThread, cnxnCreated, cnxnLimit, latch, zooKeeperClients); cnxnWorker[i].start(); } - Assert.assertThat(latch.await(3, TimeUnit.SECONDS), Matchers.is(true)); - LOG.info("created {} connections", threadNum * cnxnPerThread); + // we might need to wait potentially for a longer time for all the connection to get established, + // as the ZooKeeper Server will close some of the connections and the clients will have to re-try + boolean allConnectionsCreatedInTime = latch.await(30, TimeUnit.SECONDS); + int actualConnections = cnxnCreated.get(); + LOG.info("created {} connections", actualConnections); + if (!allConnectionsCreatedInTime) { + Assert.fail(String.format("Only %d out of %d connections created!", actualConnections, cnxnLimit)); + } - // Assert throttling not 0 + // Assert the server refused some of the connections because the handshake limit was reached + // (throttling should be greater than 0) long handshakeThrottledNum = tlsHandshakeExceeded.get(); LOG.info("TLS_HANDSHAKE_EXCEEDED: {}", handshakeThrottledNum); Assert.assertThat("The number of handshake throttled should be " + "greater than 0", handshakeThrottledNum, Matchers.greaterThan(0L)); - // Assert there is no outstanding handshake anymore + // Assert there is no outstanding handshake anymore, all the clients connected in the end int outstandingHandshakeNum = factory.getOutstandingHandshakeNum(); LOG.info("outstanding handshake is {}", outstandingHandshakeNum); Assert.assertThat("The outstanding handshake number should be 0 " + "after all cnxns established", outstandingHandshakeNum, Matchers.is(0)); + } + + private final class ClientConnectionWatcher implements Watcher { + + private final AtomicInteger cnxnCreated; + private final int cnxnLimit; + private final int cnxnThreadId; + private final int cnxnId; + private final CountDownLatch latch; + + public ClientConnectionWatcher(AtomicInteger cnxnCreated, int cnxnLimit, int cnxnThreadId, + int cnxnId, CountDownLatch latch) { + this.cnxnCreated = cnxnCreated; + this.cnxnLimit = cnxnLimit; + this.cnxnThreadId = cnxnThreadId; + this.cnxnId = cnxnId; + this.latch = latch; + } + + @Override + public void process(WatchedEvent event) { + LOG.info(String.format("WATCHER [thread: %d, cnx:%d] - new event: %s", cnxnThreadId, cnxnId, event.toString())); + if (event.getState() == Event.KeeperState.SyncConnected) { + int created = cnxnCreated.addAndGet(1); + if (created == cnxnLimit) { + latch.countDown(); + } + } + } } + + + private final class ClientConnectionGenerator extends Thread { + + private final int cnxnThreadId; + private final int cnxnPerThread; + private final AtomicInteger cnxnCreated; + private final int cnxnLimit; + private final CountDownLatch latch; + private final LinkedBlockingQueue zks; + + private ClientConnectionGenerator(int cnxnThreadId, int cnxnPerThread, + AtomicInteger cnxnCreated, int cnxnLimit, + CountDownLatch latch, + LinkedBlockingQueue zks) { + this.cnxnThreadId = cnxnThreadId; + this.cnxnPerThread = cnxnPerThread; + this.cnxnCreated = cnxnCreated; + this.cnxnLimit = cnxnLimit; + this.latch = latch; + this.zks = zks; + } + + @Override + public void run() { + + for (int j = 0; j < cnxnPerThread; j++) { + try { + zks.add(new ZooKeeper(hostPort, 30000, + new ClientConnectionWatcher(cnxnCreated, cnxnLimit, cnxnThreadId, j, latch))); + } catch (Exception e) { + LOG.info("Error while creating zk client", e); + } + } + } + } + } From be57cc255918b6e015d08bfd1203a6f04b20212e Mon Sep 17 00:00:00 2001 From: Sujith Simon Date: Thu, 19 Dec 2019 14:21:41 +0530 Subject: [PATCH 002/118] ZOOKEEPER-3653: Audit Log feature fails in a stand alone zookeeper setup Author: sujithsimon22 Reviewers: Enrico Olivelli ,Mohammad Arshad Closes #1185 from sujithsimon22/master (cherry picked from commit 7c9a1e4e9ae9e3a17c4766b34787e39ca1e42794) Signed-off-by: Mohammad Arshad --- .../apache/zookeeper/audit/AuditHelper.java | 10 +-- .../audit/StandaloneServerAuditTest.java | 90 +++++++++++++++++++ 2 files changed, 92 insertions(+), 8 deletions(-) create mode 100644 zookeeper-server/src/test/java/org/apache/zookeeper/audit/StandaloneServerAuditTest.java diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/audit/AuditHelper.java b/zookeeper-server/src/main/java/org/apache/zookeeper/audit/AuditHelper.java index c39980133a2..ce0d58a87ba 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/audit/AuditHelper.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/audit/AuditHelper.java @@ -18,7 +18,6 @@ package org.apache.zookeeper.audit; import java.io.IOException; -import java.nio.ByteBuffer; import java.util.HashMap; import java.util.Map; import org.apache.jute.Record; @@ -133,13 +132,8 @@ public static void addAuditLog(Request request, ProcessTxnResult txnResult, bool } private static void deserialize(Request request, Record record) throws IOException { - ByteBufferInputStream.byteBuffer2Record(getRequestData(request), record); - } - - private static ByteBuffer getRequestData(Request request) { - ByteBuffer reqData = request.request.slice(); - reqData.rewind(); - return reqData; + request.request.rewind(); + ByteBufferInputStream.byteBuffer2Record(request.request.slice(), record); } private static Result getResult(ProcessTxnResult rc, boolean failedTxn) { diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/audit/StandaloneServerAuditTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/audit/StandaloneServerAuditTest.java new file mode 100644 index 00000000000..7aed4c6ff42 --- /dev/null +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/audit/StandaloneServerAuditTest.java @@ -0,0 +1,90 @@ +/* + * 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.audit; + + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.io.LineNumberReader; +import java.io.StringReader; +import java.util.ArrayList; +import java.util.List; +import org.apache.log4j.Layout; +import org.apache.log4j.Level; +import org.apache.log4j.Logger; +import org.apache.log4j.SimpleLayout; +import org.apache.log4j.WriterAppender; +import org.apache.zookeeper.CreateMode; +import org.apache.zookeeper.KeeperException; +import org.apache.zookeeper.ZooDefs; +import org.apache.zookeeper.ZooKeeper; +import org.apache.zookeeper.test.ClientBase; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Test; + + + +public class StandaloneServerAuditTest extends ClientBase { + private static ByteArrayOutputStream os; + + @BeforeClass + public static void setup() { + System.setProperty(ZKAuditProvider.AUDIT_ENABLE, "true"); + // setup the logger to capture all the logs + Layout layout = new SimpleLayout(); + os = new ByteArrayOutputStream(); + WriterAppender appender = new WriterAppender(layout, os); + appender.setImmediateFlush(true); + appender.setThreshold(Level.INFO); + Logger zLogger = Logger.getLogger(Log4jAuditLogger.class); + zLogger.addAppender(appender); + } + + @AfterClass + public static void teardown() { + System.clearProperty(ZKAuditProvider.AUDIT_ENABLE); + } + + @Test + public void testCreateAuditLog() throws KeeperException, InterruptedException, IOException { + final ZooKeeper zk = createClient(); + String path = "/createPath"; + zk.create(path, "".getBytes(), ZooDefs.Ids.OPEN_ACL_UNSAFE, + CreateMode.PERSISTENT); + List logs = readAuditLog(os); + assertEquals(1, logs.size()); + assertTrue(logs.get(0).endsWith("operation=create\tznode=/createPath\tznode_type=persistent\tresult=success")); + } + + private static List readAuditLog(ByteArrayOutputStream os) throws IOException { + List logs = new ArrayList<>(); + LineNumberReader r = new LineNumberReader( + new StringReader(os.toString())); + String line; + while ((line = r.readLine()) != null) { + logs.add(line); + } + os.reset(); + return logs; + } +} + From a21632bf3b38fbf41fe3b0aba9b801dc31f8c12e Mon Sep 17 00:00:00 2001 From: Enrico Olivelli Date: Fri, 20 Dec 2019 00:14:03 +0100 Subject: [PATCH 003/118] ZooKeeper 3.6.0 release notes draft --- .../main/resources/markdown/releasenotes.md | 964 +++++++++++++----- 1 file changed, 723 insertions(+), 241 deletions(-) diff --git a/zookeeper-docs/src/main/resources/markdown/releasenotes.md b/zookeeper-docs/src/main/resources/markdown/releasenotes.md index 67dd47ab1a6..522e8bab780 100644 --- a/zookeeper-docs/src/main/resources/markdown/releasenotes.md +++ b/zookeeper-docs/src/main/resources/markdown/releasenotes.md @@ -14,254 +14,736 @@ See the License for the specific language governing permissions and limitations under the License. //--> -# ZooKeeper 3.0.0 Release Notes -* [Migration Instructions when Upgrading to 3.0.0](#migration) - * [Migrating Client Code](#migration_code) - * [Watch Management](#Watch+Management) - * [Java API](#Java+API) - * [C API](#C+API) - * [Migrating Server Data](#migration_data) - * [Migrating Server Configuration](#migration_config) -* [Changes Since ZooKeeper 2.2.1](#changes) +# Release Notes - ZooKeeper - Version 3.6.0 + +## New Feature + +* [ZOOKEEPER-27](https://issues.apache.org/jira/browse/ZOOKEEPER-27) - Unique DB identifiers for servers and clients +* [ZOOKEEPER-1112](https://issues.apache.org/jira/browse/ZOOKEEPER-1112) - Add support for C client for SASL authentication +* [ZOOKEEPER-1260](https://issues.apache.org/jira/browse/ZOOKEEPER-1260) - Audit logging in ZooKeeper servers. +* [ZOOKEEPER-1634](https://issues.apache.org/jira/browse/ZOOKEEPER-1634) - A new feature proposal to ZooKeeper: authentication enforcement +* [ZOOKEEPER-1703](https://issues.apache.org/jira/browse/ZOOKEEPER-1703) - Please add instructions for running the tutorial +* [ZOOKEEPER-1962](https://issues.apache.org/jira/browse/ZOOKEEPER-1962) - Add a CLI command to recursively list a znode and children +* [ZOOKEEPER-2163](https://issues.apache.org/jira/browse/ZOOKEEPER-2163) - Introduce new ZNode type: container +* [ZOOKEEPER-2169](https://issues.apache.org/jira/browse/ZOOKEEPER-2169) - Enable creation of nodes with TTLs +* [ZOOKEEPER-2875](https://issues.apache.org/jira/browse/ZOOKEEPER-2875) - Add ant task for running OWASP dependency report +* [ZOOKEEPER-2933](https://issues.apache.org/jira/browse/ZOOKEEPER-2933) - Ability to monitor the jute.maxBuffer usage in real-time +* [ZOOKEEPER-2994](https://issues.apache.org/jira/browse/ZOOKEEPER-2994) - Tool required to recover log and snapshot entries with CRC errors +* [ZOOKEEPER-3066](https://issues.apache.org/jira/browse/ZOOKEEPER-3066) - Expose on JMX of Followers the id of the current leader +* [ZOOKEEPER-3091](https://issues.apache.org/jira/browse/ZOOKEEPER-3091) - Prometheus.io integration +* [ZOOKEEPER-3092](https://issues.apache.org/jira/browse/ZOOKEEPER-3092) - Pluggable metrics system for ZooKeeper +* [ZOOKEEPER-3114](https://issues.apache.org/jira/browse/ZOOKEEPER-3114) - Built-in data consistency check inside ZooKeeper +* [ZOOKEEPER-3137](https://issues.apache.org/jira/browse/ZOOKEEPER-3137) - add a utility to truncate logs to a zxid +* [ZOOKEEPER-3140](https://issues.apache.org/jira/browse/ZOOKEEPER-3140) - Allow Followers to host Observers +* [ZOOKEEPER-3160](https://issues.apache.org/jira/browse/ZOOKEEPER-3160) - Custom User SSLContext +* [ZOOKEEPER-3167](https://issues.apache.org/jira/browse/ZOOKEEPER-3167) - add an API and the corresponding CLI to get total count of recursive sub nodes under a specific path +* [ZOOKEEPER-3209](https://issues.apache.org/jira/browse/ZOOKEEPER-3209) - New `getEphemerals` api to get all the ephemeral nodes created by the session +* [ZOOKEEPER-3244](https://issues.apache.org/jira/browse/ZOOKEEPER-3244) - Add option to snapshot based on log size +* [ZOOKEEPER-3269](https://issues.apache.org/jira/browse/ZOOKEEPER-3269) - Testable facade would benefit from a queueEvent() method +* [ZOOKEEPER-3311](https://issues.apache.org/jira/browse/ZOOKEEPER-3311) - Allow a delay to the transaction log flush +* [ZOOKEEPER-3331](https://issues.apache.org/jira/browse/ZOOKEEPER-3331) - Automatically add IP authorization for Netty connections +* [ZOOKEEPER-3343](https://issues.apache.org/jira/browse/ZOOKEEPER-3343) - Add a new doc: zookeeperTools.md +* [ZOOKEEPER-3344](https://issues.apache.org/jira/browse/ZOOKEEPER-3344) - write a new script:zkSnapShotToolkit.sh to encapsulate SnapshotFormatter and doc the usage +* [ZOOKEEPER-3371](https://issues.apache.org/jira/browse/ZOOKEEPER-3371) - Port unification for admin server +* [ZOOKEEPER-3447](https://issues.apache.org/jira/browse/ZOOKEEPER-3447) - add a doc: zookeeperMonitor.md +* [ZOOKEEPER-3657](https://issues.apache.org/jira/browse/ZOOKEEPER-3657) - Implementing snapshot schedule to avoid high latency issue due to disk contention + +## Improvement + +* [ZOOKEEPER-261](https://issues.apache.org/jira/browse/ZOOKEEPER-261) - Reinitialized servers should not participate in leader election +* [ZOOKEEPER-761](https://issues.apache.org/jira/browse/ZOOKEEPER-761) - Remove *synchronous* calls from the *single-threaded* C clieant API, since they are documented not to work +* [ZOOKEEPER-974](https://issues.apache.org/jira/browse/ZOOKEEPER-974) - Configurable listen socket backlog for the client port +* [ZOOKEEPER-1177](https://issues.apache.org/jira/browse/ZOOKEEPER-1177) - Enabling a large number of watches for a large number of clients +* [ZOOKEEPER-1416](https://issues.apache.org/jira/browse/ZOOKEEPER-1416) - Persistent Recursive Watch +* [ZOOKEEPER-1423](https://issues.apache.org/jira/browse/ZOOKEEPER-1423) - 4lw and jmx should expose the size of the datadir/datalogdir +* [ZOOKEEPER-1425](https://issues.apache.org/jira/browse/ZOOKEEPER-1425) - add version command to the zookeeper client shell +* [ZOOKEEPER-1426](https://issues.apache.org/jira/browse/ZOOKEEPER-1426) - add version command to the zookeeper server +* [ZOOKEEPER-1467](https://issues.apache.org/jira/browse/ZOOKEEPER-1467) - Make server principal configurable at client side. +* [ZOOKEEPER-1504](https://issues.apache.org/jira/browse/ZOOKEEPER-1504) - Multi-thread NIOServerCnxn +* [ZOOKEEPER-1506](https://issues.apache.org/jira/browse/ZOOKEEPER-1506) - Re-try DNS hostname -> IP resolution if node connection fails +* [ZOOKEEPER-1525](https://issues.apache.org/jira/browse/ZOOKEEPER-1525) - Plumb ZooKeeperServer object into auth plugins +* [ZOOKEEPER-1651](https://issues.apache.org/jira/browse/ZOOKEEPER-1651) - Add support for compressed snapshot +* [ZOOKEEPER-1748](https://issues.apache.org/jira/browse/ZOOKEEPER-1748) - TCP keepalive for leader election connections +* [ZOOKEEPER-1907](https://issues.apache.org/jira/browse/ZOOKEEPER-1907) - Improve Thread handling +* [ZOOKEEPER-1908](https://issues.apache.org/jira/browse/ZOOKEEPER-1908) - setAcl should be have a recursive function +* [ZOOKEEPER-1948](https://issues.apache.org/jira/browse/ZOOKEEPER-1948) - Enable JMX remote monitoring +* [ZOOKEEPER-1963](https://issues.apache.org/jira/browse/ZOOKEEPER-1963) - Make JDK 7 the minimum requirement for Zookeeper +* [ZOOKEEPER-1994](https://issues.apache.org/jira/browse/ZOOKEEPER-1994) - Backup config files. +* [ZOOKEEPER-2024](https://issues.apache.org/jira/browse/ZOOKEEPER-2024) - Major throughput improvement with mixed workloads +* [ZOOKEEPER-2040](https://issues.apache.org/jira/browse/ZOOKEEPER-2040) - Server to log underlying cause of SASL connection problems +* [ZOOKEEPER-2079](https://issues.apache.org/jira/browse/ZOOKEEPER-2079) - Stop daemon with "kill" rather than "kill -9" +* [ZOOKEEPER-2084](https://issues.apache.org/jira/browse/ZOOKEEPER-2084) - Document local session parameters +* [ZOOKEEPER-2087](https://issues.apache.org/jira/browse/ZOOKEEPER-2087) - Few UX improvements in ZooInspector +* [ZOOKEEPER-2098](https://issues.apache.org/jira/browse/ZOOKEEPER-2098) - QuorumCnxManager: use BufferedOutputStream for initial msg +* [ZOOKEEPER-2107](https://issues.apache.org/jira/browse/ZOOKEEPER-2107) - zookeeper client should support custom HostProviders +* [ZOOKEEPER-2110](https://issues.apache.org/jira/browse/ZOOKEEPER-2110) - Typo fixes in the ZK documentation +* [ZOOKEEPER-2126](https://issues.apache.org/jira/browse/ZOOKEEPER-2126) - Improve exit log messsage of EventThread and SendThread by adding SessionId +* [ZOOKEEPER-2139](https://issues.apache.org/jira/browse/ZOOKEEPER-2139) - Support multiple ZooKeeper client, with different configurations, in a single JVM +* [ZOOKEEPER-2140](https://issues.apache.org/jira/browse/ZOOKEEPER-2140) - NettyServerCnxn and NIOServerCnxn code should be improved +* [ZOOKEEPER-2149](https://issues.apache.org/jira/browse/ZOOKEEPER-2149) - Logging of client address when socket connection established +* [ZOOKEEPER-2176](https://issues.apache.org/jira/browse/ZOOKEEPER-2176) - Unclear error message should be info not error +* [ZOOKEEPER-2179](https://issues.apache.org/jira/browse/ZOOKEEPER-2179) - Typo in Watcher.java +* [ZOOKEEPER-2183](https://issues.apache.org/jira/browse/ZOOKEEPER-2183) - Concurrent Testing Processes and Port Assignments +* [ZOOKEEPER-2185](https://issues.apache.org/jira/browse/ZOOKEEPER-2185) - Run server with -XX:+HeapDumpOnOutOfMemoryError and -XX:OnOutOfMemoryError='kill %p'. +* [ZOOKEEPER-2191](https://issues.apache.org/jira/browse/ZOOKEEPER-2191) - Continue supporting prior Ant versions that don't implement the threads attribute for the JUnit task. +* [ZOOKEEPER-2194](https://issues.apache.org/jira/browse/ZOOKEEPER-2194) - Let DataNode.getChildren() return an unmodifiable view of its children set +* [ZOOKEEPER-2205](https://issues.apache.org/jira/browse/ZOOKEEPER-2205) - Log type of unexpected quorum packet in learner handler loop +* [ZOOKEEPER-2206](https://issues.apache.org/jira/browse/ZOOKEEPER-2206) - Add missing packet types to LearnerHandler.packetToString() +* [ZOOKEEPER-2207](https://issues.apache.org/jira/browse/ZOOKEEPER-2207) - Enhance error logs with LearnerHandler.packetToString() +* [ZOOKEEPER-2208](https://issues.apache.org/jira/browse/ZOOKEEPER-2208) - Log type of unexpected quorum packet in observer loop +* [ZOOKEEPER-2214](https://issues.apache.org/jira/browse/ZOOKEEPER-2214) - Findbugs warning: LearnerHandler.packetToString Dead store to local variable +* [ZOOKEEPER-2223](https://issues.apache.org/jira/browse/ZOOKEEPER-2223) - support method-level JUnit testcase +* [ZOOKEEPER-2238](https://issues.apache.org/jira/browse/ZOOKEEPER-2238) - Support limiting the maximum number of connections/clients to a zookeeper server. +* [ZOOKEEPER-2240](https://issues.apache.org/jira/browse/ZOOKEEPER-2240) - Make the three-node minimum more explicit in documentation and on website +* [ZOOKEEPER-2270](https://issues.apache.org/jira/browse/ZOOKEEPER-2270) - Allow MBeanRegistry to be overridden for better unit tests +* [ZOOKEEPER-2306](https://issues.apache.org/jira/browse/ZOOKEEPER-2306) - Remove file delete duplicate code from test code +* [ZOOKEEPER-2315](https://issues.apache.org/jira/browse/ZOOKEEPER-2315) - Change client connect zk service timeout log level from Info to Warn level +* [ZOOKEEPER-2326](https://issues.apache.org/jira/browse/ZOOKEEPER-2326) - Include connected server address:port in log +* [ZOOKEEPER-2359](https://issues.apache.org/jira/browse/ZOOKEEPER-2359) - ZooKeeper client has unnecessary logs for watcher removal errors +* [ZOOKEEPER-2368](https://issues.apache.org/jira/browse/ZOOKEEPER-2368) - Client watches are not disconnected on close +* [ZOOKEEPER-2373](https://issues.apache.org/jira/browse/ZOOKEEPER-2373) - Licenses section missing from pom file +* [ZOOKEEPER-2378](https://issues.apache.org/jira/browse/ZOOKEEPER-2378) - upgrade ivy to recent version +* [ZOOKEEPER-2392](https://issues.apache.org/jira/browse/ZOOKEEPER-2392) - Update netty to 3.7.1.Final +* [ZOOKEEPER-2402](https://issues.apache.org/jira/browse/ZOOKEEPER-2402) - Document client side properties +* [ZOOKEEPER-2410](https://issues.apache.org/jira/browse/ZOOKEEPER-2410) - add time unit to 'ELECTION TOOK' log.info message +* [ZOOKEEPER-2433](https://issues.apache.org/jira/browse/ZOOKEEPER-2433) - ZooKeeperSaslServer: allow user principals in subject +* [ZOOKEEPER-2479](https://issues.apache.org/jira/browse/ZOOKEEPER-2479) - Add 'electionTimeTaken' value in LeaderMXBean and FollowerMXBean +* [ZOOKEEPER-2489](https://issues.apache.org/jira/browse/ZOOKEEPER-2489) - Upgrade Jetty dependency to a recent stable release version. +* [ZOOKEEPER-2505](https://issues.apache.org/jira/browse/ZOOKEEPER-2505) - Use shared library instead of static library in C client unit test +* [ZOOKEEPER-2507](https://issues.apache.org/jira/browse/ZOOKEEPER-2507) - C unit test improvement: line break between 'ZooKeeper server started' and 'Running' +* [ZOOKEEPER-2511](https://issues.apache.org/jira/browse/ZOOKEEPER-2511) - Implement AutoCloseable in ZooKeeper.java +* [ZOOKEEPER-2557](https://issues.apache.org/jira/browse/ZOOKEEPER-2557) - Update gitignore to account for other file extensions +* [ZOOKEEPER-2594](https://issues.apache.org/jira/browse/ZOOKEEPER-2594) - Use TLS for downloading artifacts during build +* [ZOOKEEPER-2620](https://issues.apache.org/jira/browse/ZOOKEEPER-2620) - Add comments to testReadOnlySnapshotDir and testReadOnlyTxnLogDir indicating that the tests will fail when run as root +* [ZOOKEEPER-2630](https://issues.apache.org/jira/browse/ZOOKEEPER-2630) - Use interface type instead of implementation type when appropriate. +* [ZOOKEEPER-2632](https://issues.apache.org/jira/browse/ZOOKEEPER-2632) - Add option to inform JIRA_PASSWORD at CLI prompt +* [ZOOKEEPER-2638](https://issues.apache.org/jira/browse/ZOOKEEPER-2638) - ZooKeeper should log which serverCnxnFactory is used during startup +* [ZOOKEEPER-2641](https://issues.apache.org/jira/browse/ZOOKEEPER-2641) - AvgRequestLatency metric improves to be more accurate +* [ZOOKEEPER-2655](https://issues.apache.org/jira/browse/ZOOKEEPER-2655) - Improve NIOServerCnxn#isZKServerRunning to reflect the semantics correctly +* [ZOOKEEPER-2662](https://issues.apache.org/jira/browse/ZOOKEEPER-2662) - Export a metric for txn log sync times +* [ZOOKEEPER-2672](https://issues.apache.org/jira/browse/ZOOKEEPER-2672) - Remove CHANGE.txt +* [ZOOKEEPER-2682](https://issues.apache.org/jira/browse/ZOOKEEPER-2682) - Make it optional to fail build on test failure +* [ZOOKEEPER-2697](https://issues.apache.org/jira/browse/ZOOKEEPER-2697) - Handle graceful stop of ZookKeeper client +* [ZOOKEEPER-2744](https://issues.apache.org/jira/browse/ZOOKEEPER-2744) - Typos in the comments of ZooKeeper class +* [ZOOKEEPER-2767](https://issues.apache.org/jira/browse/ZOOKEEPER-2767) - Correct the exception messages in X509Util if truststore location or password is not configured +* [ZOOKEEPER-2788](https://issues.apache.org/jira/browse/ZOOKEEPER-2788) - The define of MAX_CONNECTION_ATTEMPTS in QuorumCnxManager.java seems useless, should it be removed? +* [ZOOKEEPER-2815](https://issues.apache.org/jira/browse/ZOOKEEPER-2815) - 1. Using try clause to close resource; 2. Others code refactoring for PERSISTENCE module +* [ZOOKEEPER-2816](https://issues.apache.org/jira/browse/ZOOKEEPER-2816) - Code refactoring for `ZK_SERVER` module +* [ZOOKEEPER-2824](https://issues.apache.org/jira/browse/ZOOKEEPER-2824) - `FileChannel#size` info should be added to `FileTxnLog#commit` to solve the confuse that reason is too large log or too busy disk I/O +* [ZOOKEEPER-2825](https://issues.apache.org/jira/browse/ZOOKEEPER-2825) - 1. Remove unnecessary import; 2. `contains` instead of `indexOf > -1` for more readable; 3. Standardize `StringBuilder#append` usage for CLIENT module +* [ZOOKEEPER-2826](https://issues.apache.org/jira/browse/ZOOKEEPER-2826) - Code refactoring for `CLI` module +* [ZOOKEEPER-2829](https://issues.apache.org/jira/browse/ZOOKEEPER-2829) - Interface usability / compatibility improvements through Java annotation. +* [ZOOKEEPER-2856](https://issues.apache.org/jira/browse/ZOOKEEPER-2856) - ZooKeeperSaslClient#respondToServer should log exception message of SaslException +* [ZOOKEEPER-2864](https://issues.apache.org/jira/browse/ZOOKEEPER-2864) - Add script to run a java api compatibility tool +* [ZOOKEEPER-2865](https://issues.apache.org/jira/browse/ZOOKEEPER-2865) - Reconfig Causes Inconsistent Configuration file among the nodes +* [ZOOKEEPER-2870](https://issues.apache.org/jira/browse/ZOOKEEPER-2870) - Improve the efficiency of AtomicFileOutputStream +* [ZOOKEEPER-2880](https://issues.apache.org/jira/browse/ZOOKEEPER-2880) - Rename README.txt to README.md +* [ZOOKEEPER-2887](https://issues.apache.org/jira/browse/ZOOKEEPER-2887) - define dependency versions in build.xml to be easily overridden in build.properties +* [ZOOKEEPER-2892](https://issues.apache.org/jira/browse/ZOOKEEPER-2892) - Improve lazy initialize and close stream for `PrepRequestProcessor` +* [ZOOKEEPER-2896](https://issues.apache.org/jira/browse/ZOOKEEPER-2896) - Remove unused imports from org.apache.zookeeper.test.CreateTest.java +* [ZOOKEEPER-2904](https://issues.apache.org/jira/browse/ZOOKEEPER-2904) - Remove unused imports from org.apache.zookeeper.server.quorum.WatchLeakTest +* [ZOOKEEPER-2915](https://issues.apache.org/jira/browse/ZOOKEEPER-2915) - Use "strict" conflict management in ivy +* [ZOOKEEPER-2950](https://issues.apache.org/jira/browse/ZOOKEEPER-2950) - Add keys for the Zxid from the stat command to check_zookeeper.py +* [ZOOKEEPER-2952](https://issues.apache.org/jira/browse/ZOOKEEPER-2952) - Upgrade third party libraries to address vulnerabilities +* [ZOOKEEPER-2967](https://issues.apache.org/jira/browse/ZOOKEEPER-2967) - Add check to validate dataDir and dataLogDir parameters at startup +* [ZOOKEEPER-2999](https://issues.apache.org/jira/browse/ZOOKEEPER-2999) - CMake build should use target-level commands +* [ZOOKEEPER-3012](https://issues.apache.org/jira/browse/ZOOKEEPER-3012) - Fix unit test: testDataDirAndDataLogDir should not use hardcode test folders +* [ZOOKEEPER-3019](https://issues.apache.org/jira/browse/ZOOKEEPER-3019) - Add a metric to track number of slow fsyncs +* [ZOOKEEPER-3020](https://issues.apache.org/jira/browse/ZOOKEEPER-3020) - Review of SyncRequestProcessor +* [ZOOKEEPER-3037](https://issues.apache.org/jira/browse/ZOOKEEPER-3037) - Add JvmPauseMonitor to ZooKeeper +* [ZOOKEEPER-3043](https://issues.apache.org/jira/browse/ZOOKEEPER-3043) - QuorumKerberosHostBasedAuthTest fails on Linux box: Unable to parse:includedir /etc/krb5.conf.d/ +* [ZOOKEEPER-3044](https://issues.apache.org/jira/browse/ZOOKEEPER-3044) - OutOfMemoryError exceptions in Jenkins when running tests +* [ZOOKEEPER-3063](https://issues.apache.org/jira/browse/ZOOKEEPER-3063) - Track outstanding changes with ArrayDeque +* [ZOOKEEPER-3068](https://issues.apache.org/jira/browse/ZOOKEEPER-3068) - Improve C client logging of IPv6 hosts +* [ZOOKEEPER-3071](https://issues.apache.org/jira/browse/ZOOKEEPER-3071) - Add a config parameter to control transaction log size +* [ZOOKEEPER-3077](https://issues.apache.org/jira/browse/ZOOKEEPER-3077) - Build native C library outside of source directory +* [ZOOKEEPER-3078](https://issues.apache.org/jira/browse/ZOOKEEPER-3078) - Remove unused print_completion_queue function +* [ZOOKEEPER-3083](https://issues.apache.org/jira/browse/ZOOKEEPER-3083) - Remove some redundant and noisy log lines +* [ZOOKEEPER-3084](https://issues.apache.org/jira/browse/ZOOKEEPER-3084) - Exit when ZooKeeper cannot bind to the leader election port +* [ZOOKEEPER-3085](https://issues.apache.org/jira/browse/ZOOKEEPER-3085) - Define constant exit code and add documents +* [ZOOKEEPER-3094](https://issues.apache.org/jira/browse/ZOOKEEPER-3094) - Make BufferSizeTest reliable +* [ZOOKEEPER-3095](https://issues.apache.org/jira/browse/ZOOKEEPER-3095) - Connect string fix for non-existent hosts +* [ZOOKEEPER-3097](https://issues.apache.org/jira/browse/ZOOKEEPER-3097) - Use Runnable instead of Thread for working items in WorkerService to improve the throughput of CommitProcessor +* [ZOOKEEPER-3098](https://issues.apache.org/jira/browse/ZOOKEEPER-3098) - Add additional server metrics +* [ZOOKEEPER-3109](https://issues.apache.org/jira/browse/ZOOKEEPER-3109) - Avoid long unavailable time due to voter changed mind when activating the leader during election +* [ZOOKEEPER-3110](https://issues.apache.org/jira/browse/ZOOKEEPER-3110) - Improve the closeSession throughput in PrepRequestProcessor +* [ZOOKEEPER-3116](https://issues.apache.org/jira/browse/ZOOKEEPER-3116) - Make the DataTree.approximateDataSize more efficient +* [ZOOKEEPER-3124](https://issues.apache.org/jira/browse/ZOOKEEPER-3124) - Add the correct comment to show why we need the special logic to handle cversion and pzxid +* [ZOOKEEPER-3142](https://issues.apache.org/jira/browse/ZOOKEEPER-3142) - Extend SnapshotFormatter to dump data in json format +* [ZOOKEEPER-3146](https://issues.apache.org/jira/browse/ZOOKEEPER-3146) - Limit the maximum client connections per IP in NettyServerCnxnFactory +* [ZOOKEEPER-3152](https://issues.apache.org/jira/browse/ZOOKEEPER-3152) - Port ZK netty stack to netty 4 +* [ZOOKEEPER-3159](https://issues.apache.org/jira/browse/ZOOKEEPER-3159) - Flaky: ClientRequestTimeoutTest.testClientRequestTimeout +* [ZOOKEEPER-3161](https://issues.apache.org/jira/browse/ZOOKEEPER-3161) - Refactor QuorumPeerMainTest.java: move commonly used functions to base class +* [ZOOKEEPER-3163](https://issues.apache.org/jira/browse/ZOOKEEPER-3163) - Use session map to improve the performance when closing session in Netty +* [ZOOKEEPER-3177](https://issues.apache.org/jira/browse/ZOOKEEPER-3177) - Refactor request throttle logic in NIO and Netty to keep the same behavior and make the code easier to maintain +* [ZOOKEEPER-3179](https://issues.apache.org/jira/browse/ZOOKEEPER-3179) - Add snapshot compression to reduce the disk IO +* [ZOOKEEPER-3180](https://issues.apache.org/jira/browse/ZOOKEEPER-3180) - Add response cache to improve the throughput of read heavy traffic +* [ZOOKEEPER-3183](https://issues.apache.org/jira/browse/ZOOKEEPER-3183) - Interrupting or notifying the WatcherCleaner thread during shutdown if it is waiting for dead watchers get certain number(watcherCleanThreshold) and also stop adding incoming deadWatcher to deadWatchersList when shutdown is initiated. +* [ZOOKEEPER-3188](https://issues.apache.org/jira/browse/ZOOKEEPER-3188) - Improve resilience to network +* [ZOOKEEPER-3190](https://issues.apache.org/jira/browse/ZOOKEEPER-3190) - Spell check on the Zookeeper server files +* [ZOOKEEPER-3195](https://issues.apache.org/jira/browse/ZOOKEEPER-3195) - TLS - disable client-initiated renegotiation +* [ZOOKEEPER-3203](https://issues.apache.org/jira/browse/ZOOKEEPER-3203) - Tracking and exposing the non voting followers in ZK +* [ZOOKEEPER-3208](https://issues.apache.org/jira/browse/ZOOKEEPER-3208) - Remove the SSLTest.java.orig introduced in ZOOKEEPER-3032 +* [ZOOKEEPER-3216](https://issues.apache.org/jira/browse/ZOOKEEPER-3216) - Make init/sync limit tunable via JMX +* [ZOOKEEPER-3219](https://issues.apache.org/jira/browse/ZOOKEEPER-3219) - Fix flaky FileChangeWatcherTest +* [ZOOKEEPER-3228](https://issues.apache.org/jira/browse/ZOOKEEPER-3228) - [TLS] Fix key usage extension in test certs +* [ZOOKEEPER-3232](https://issues.apache.org/jira/browse/ZOOKEEPER-3232) - make the log of notification about LE more readable +* [ZOOKEEPER-3234](https://issues.apache.org/jira/browse/ZOOKEEPER-3234) - Add Travis-CI configuration file +* [ZOOKEEPER-3235](https://issues.apache.org/jira/browse/ZOOKEEPER-3235) - Enable secure processing and disallow DTDs in the SAXParserFactory +* [ZOOKEEPER-3236](https://issues.apache.org/jira/browse/ZOOKEEPER-3236) - Upgrade BouncyCastle +* [ZOOKEEPER-3237](https://issues.apache.org/jira/browse/ZOOKEEPER-3237) - Allow IPv6 wildcard address in peer config +* [ZOOKEEPER-3238](https://issues.apache.org/jira/browse/ZOOKEEPER-3238) - Add rel="noopener noreferrer" to target blank link in zookeeper-contrib-huebrowser +* [ZOOKEEPER-3239](https://issues.apache.org/jira/browse/ZOOKEEPER-3239) - Adding EnsembleAuthProvider to verify the ensemble name +* [ZOOKEEPER-3240](https://issues.apache.org/jira/browse/ZOOKEEPER-3240) - Close socket on Learner shutdown to avoid dangling socket +* [ZOOKEEPER-3242](https://issues.apache.org/jira/browse/ZOOKEEPER-3242) - Add server side connecting throttling +* [ZOOKEEPER-3243](https://issues.apache.org/jira/browse/ZOOKEEPER-3243) - Add server side request throttling +* [ZOOKEEPER-3245](https://issues.apache.org/jira/browse/ZOOKEEPER-3245) - Add useful metrics for ZK pipeline and request/server states +* [ZOOKEEPER-3249](https://issues.apache.org/jira/browse/ZOOKEEPER-3249) - Avoid reverting the cversion and pzxid during replaying txns with fuzzy snapshot +* [ZOOKEEPER-3250](https://issues.apache.org/jira/browse/ZOOKEEPER-3250) - typo in doc - zookeeperInternals +* [ZOOKEEPER-3255](https://issues.apache.org/jira/browse/ZOOKEEPER-3255) - add a banner to make the startup of zk server more cool +* [ZOOKEEPER-3257](https://issues.apache.org/jira/browse/ZOOKEEPER-3257) - Merge count and byte update of Stat +* [ZOOKEEPER-3262](https://issues.apache.org/jira/browse/ZOOKEEPER-3262) - Update dependencies flagged by OWASP report +* [ZOOKEEPER-3263](https://issues.apache.org/jira/browse/ZOOKEEPER-3263) - Illegal reflective access in zookeer's kerberosUtil +* [ZOOKEEPER-3272](https://issues.apache.org/jira/browse/ZOOKEEPER-3272) - Clean up netty4 code per Norman Maurer's review comments +* [ZOOKEEPER-3273](https://issues.apache.org/jira/browse/ZOOKEEPER-3273) - Sync BouncyCastle version in Maven build and Ant build +* [ZOOKEEPER-3274](https://issues.apache.org/jira/browse/ZOOKEEPER-3274) - Use CompositeByteBuf to queue data in NettyServerCnxn +* [ZOOKEEPER-3276](https://issues.apache.org/jira/browse/ZOOKEEPER-3276) - Make X509UtilTest.testCreateSSLServerSocketWithPort less flaky +* [ZOOKEEPER-3277](https://issues.apache.org/jira/browse/ZOOKEEPER-3277) - Add trace listener in NettyServerCnxnFactory only if trace logging is enabled +* [ZOOKEEPER-3291](https://issues.apache.org/jira/browse/ZOOKEEPER-3291) - improve error message when JAVA_HOME is set to the wrong value +* [ZOOKEEPER-3312](https://issues.apache.org/jira/browse/ZOOKEEPER-3312) - Upgrade Jetty to 9.4.15.v20190215 +* [ZOOKEEPER-3314](https://issues.apache.org/jira/browse/ZOOKEEPER-3314) - Document the possibility of MultiCallback receiving a null pointer +* [ZOOKEEPER-3332](https://issues.apache.org/jira/browse/ZOOKEEPER-3332) - TxnLogToolkit should print multi transactions readably +* [ZOOKEEPER-3335](https://issues.apache.org/jira/browse/ZOOKEEPER-3335) - Improve the usage of Collections +* [ZOOKEEPER-3339](https://issues.apache.org/jira/browse/ZOOKEEPER-3339) - Improve Debug and Trace Log Statements +* [ZOOKEEPER-3340](https://issues.apache.org/jira/browse/ZOOKEEPER-3340) - Introduce CircularBlockingQueue in QuorumCnxManager.java +* [ZOOKEEPER-3341](https://issues.apache.org/jira/browse/ZOOKEEPER-3341) - Remove Superfluous ByteBuffer Duplicate +* [ZOOKEEPER-3347](https://issues.apache.org/jira/browse/ZOOKEEPER-3347) - Improve PathTrie Consistency +* [ZOOKEEPER-3348](https://issues.apache.org/jira/browse/ZOOKEEPER-3348) - Make TxnLog and TxnLog Iterator Closable +* [ZOOKEEPER-3350](https://issues.apache.org/jira/browse/ZOOKEEPER-3350) - Get rid of CommonNames +* [ZOOKEEPER-3351](https://issues.apache.org/jira/browse/ZOOKEEPER-3351) - Migrate qa-test-pullrequest ant task to maven +* [ZOOKEEPER-3353](https://issues.apache.org/jira/browse/ZOOKEEPER-3353) - Admin commands for showing initial settings +* [ZOOKEEPER-3354](https://issues.apache.org/jira/browse/ZOOKEEPER-3354) - Improve efficiency of DeleteAllCommand +* [ZOOKEEPER-3359](https://issues.apache.org/jira/browse/ZOOKEEPER-3359) - Batch commits in the CommitProcessor +* [ZOOKEEPER-3360](https://issues.apache.org/jira/browse/ZOOKEEPER-3360) - Misprint in WriteLock javadoc +* [ZOOKEEPER-3364](https://issues.apache.org/jira/browse/ZOOKEEPER-3364) - Compile with strict options in order to check code quality +* [ZOOKEEPER-3365](https://issues.apache.org/jira/browse/ZOOKEEPER-3365) - Use Concurrent HashMap in NettyServerCnxnFactory +* [ZOOKEEPER-3369](https://issues.apache.org/jira/browse/ZOOKEEPER-3369) - Maven release artifacts cleanup +* [ZOOKEEPER-3370](https://issues.apache.org/jira/browse/ZOOKEEPER-3370) - Remove SVN specific revision generation +* [ZOOKEEPER-3372](https://issues.apache.org/jira/browse/ZOOKEEPER-3372) - Cleanup pom.xml in order to let Maven clients import as few dependencies as possible +* [ZOOKEEPER-3378](https://issues.apache.org/jira/browse/ZOOKEEPER-3378) - Set the quorum cnxn timeout independently from syncLimit +* [ZOOKEEPER-3382](https://issues.apache.org/jira/browse/ZOOKEEPER-3382) - Update Documentation: If you only have one storage device +* [ZOOKEEPER-3385](https://issues.apache.org/jira/browse/ZOOKEEPER-3385) - Add admin command to display leader +* [ZOOKEEPER-3386](https://issues.apache.org/jira/browse/ZOOKEEPER-3386) - Add admin command to display voting view +* [ZOOKEEPER-3388](https://issues.apache.org/jira/browse/ZOOKEEPER-3388) - Allow client port to support plaintext and encrypted connections simultaneously +* [ZOOKEEPER-3391](https://issues.apache.org/jira/browse/ZOOKEEPER-3391) - Drop unused CSVInputArchive and XMLInputArchive +* [ZOOKEEPER-3392](https://issues.apache.org/jira/browse/ZOOKEEPER-3392) - Add admin command to display last snapshot information +* [ZOOKEEPER-3394](https://issues.apache.org/jira/browse/ZOOKEEPER-3394) - Delay observer reconnect when all learner masters have been tried +* [ZOOKEEPER-3395](https://issues.apache.org/jira/browse/ZOOKEEPER-3395) - Document individual admin commands in markdown +* [ZOOKEEPER-3396](https://issues.apache.org/jira/browse/ZOOKEEPER-3396) - Flaky test in RestoreCommittedLogTest +* [ZOOKEEPER-3398](https://issues.apache.org/jira/browse/ZOOKEEPER-3398) - Learner.connectToLeader() may take too long to time-out +* [ZOOKEEPER-3400](https://issues.apache.org/jira/browse/ZOOKEEPER-3400) - Add documentation on local sessions +* [ZOOKEEPER-3402](https://issues.apache.org/jira/browse/ZOOKEEPER-3402) - Add a multiRead operation +* [ZOOKEEPER-3416](https://issues.apache.org/jira/browse/ZOOKEEPER-3416) - Remove redundant ServerCnxnFactoryAccessor +* [ZOOKEEPER-3418](https://issues.apache.org/jira/browse/ZOOKEEPER-3418) - Improve quorum throughput through eager ACL checks of requests on local servers +* [ZOOKEEPER-3423](https://issues.apache.org/jira/browse/ZOOKEEPER-3423) - use the maven-like way to ignore the generated version java files and doc the cmd:'./zkServer.sh version' +* [ZOOKEEPER-3430](https://issues.apache.org/jira/browse/ZOOKEEPER-3430) - Observability improvement: provide top N read / write path queries +* [ZOOKEEPER-3436](https://issues.apache.org/jira/browse/ZOOKEEPER-3436) - Enhance Mavenized Make C client +* [ZOOKEEPER-3437](https://issues.apache.org/jira/browse/ZOOKEEPER-3437) - Improve sync throttling on a learner master +* [ZOOKEEPER-3439](https://issues.apache.org/jira/browse/ZOOKEEPER-3439) - Observability improvements on client / server connection close +* [ZOOKEEPER-3448](https://issues.apache.org/jira/browse/ZOOKEEPER-3448) - Introduce MessageTracker to assist debug leader and leaner connectivity issues +* [ZOOKEEPER-3453](https://issues.apache.org/jira/browse/ZOOKEEPER-3453) - missing 'SET' in zkCli on windows +* [ZOOKEEPER-3457](https://issues.apache.org/jira/browse/ZOOKEEPER-3457) - Code optimization in QuorumCnxManager +* [ZOOKEEPER-3459](https://issues.apache.org/jira/browse/ZOOKEEPER-3459) - Add admin command to display synced state of peer +* [ZOOKEEPER-3472](https://issues.apache.org/jira/browse/ZOOKEEPER-3472) - Treat check request as a write request which needs to wait for the check txn commit from leader +* [ZOOKEEPER-3473](https://issues.apache.org/jira/browse/ZOOKEEPER-3473) - Improving successful TLS handshake throughput with concurrent control +* [ZOOKEEPER-3484](https://issues.apache.org/jira/browse/ZOOKEEPER-3484) - Improve the throughput by optimizing the synchronization around outstandingChanges +* [ZOOKEEPER-3491](https://issues.apache.org/jira/browse/ZOOKEEPER-3491) - Specify commitLogCount value using a system property +* [ZOOKEEPER-3492](https://issues.apache.org/jira/browse/ZOOKEEPER-3492) - Add weights to server side connection throttling +* [ZOOKEEPER-3494](https://issues.apache.org/jira/browse/ZOOKEEPER-3494) - No need to depend on netty-all (SSL) +* [ZOOKEEPER-3501](https://issues.apache.org/jira/browse/ZOOKEEPER-3501) - unify the method:op2String() +* [ZOOKEEPER-3502](https://issues.apache.org/jira/browse/ZOOKEEPER-3502) - improve the server command: zabstate to have a better observation on the process of leader election +* [ZOOKEEPER-3503](https://issues.apache.org/jira/browse/ZOOKEEPER-3503) - Add server side large request throttling +* [ZOOKEEPER-3506](https://issues.apache.org/jira/browse/ZOOKEEPER-3506) - correct the SessionTrackerImpl#initializeNextSession's javaDoc about how to generate the sessionId +* [ZOOKEEPER-3509](https://issues.apache.org/jira/browse/ZOOKEEPER-3509) - Revisit log format +* [ZOOKEEPER-3519](https://issues.apache.org/jira/browse/ZOOKEEPER-3519) - upgrade dependency-check to 5.2.1 +* [ZOOKEEPER-3522](https://issues.apache.org/jira/browse/ZOOKEEPER-3522) - Consistency guarantees discussion. +* [ZOOKEEPER-3523](https://issues.apache.org/jira/browse/ZOOKEEPER-3523) - Replace dummy watcher with a unified singleton +* [ZOOKEEPER-3525](https://issues.apache.org/jira/browse/ZOOKEEPER-3525) - Add project status badges to README +* [ZOOKEEPER-3530](https://issues.apache.org/jira/browse/ZOOKEEPER-3530) - Include compiled C-client in the binary tarball +* [ZOOKEEPER-3532](https://issues.apache.org/jira/browse/ZOOKEEPER-3532) - Provide a docker-based environment to work on a known OS +* [ZOOKEEPER-3537](https://issues.apache.org/jira/browse/ZOOKEEPER-3537) - Leader election - Use of out of election messages +* [ZOOKEEPER-3548](https://issues.apache.org/jira/browse/ZOOKEEPER-3548) - Redundant zxid check in SnapStream.isValidSnapshot +* [ZOOKEEPER-3560](https://issues.apache.org/jira/browse/ZOOKEEPER-3560) - Add response cache to serve get children (2) requests. +* [ZOOKEEPER-3570](https://issues.apache.org/jira/browse/ZOOKEEPER-3570) - make the special client xid constant +* [ZOOKEEPER-3571](https://issues.apache.org/jira/browse/ZOOKEEPER-3571) - Create test base directory on test started +* [ZOOKEEPER-3593](https://issues.apache.org/jira/browse/ZOOKEEPER-3593) - fix the default value of jute.maxbuffer in client side and an optimization for the documentation +* [ZOOKEEPER-3595](https://issues.apache.org/jira/browse/ZOOKEEPER-3595) - Fsync parameter for serialize method is ingnored +* [ZOOKEEPER-3599](https://issues.apache.org/jira/browse/ZOOKEEPER-3599) - cli.c: Resuscitate "old-style" argument parsing +* [ZOOKEEPER-3606](https://issues.apache.org/jira/browse/ZOOKEEPER-3606) - add JMXHOSTNAME to zkServer.sh to enable user to change the exposed hostname of jmx service +* [ZOOKEEPER-3620](https://issues.apache.org/jira/browse/ZOOKEEPER-3620) - Allow to override calls to System.exit in server side code +* [ZOOKEEPER-3630](https://issues.apache.org/jira/browse/ZOOKEEPER-3630) - Autodetection of SSL library during Zookeeper C client build +* [ZOOKEEPER-3636](https://issues.apache.org/jira/browse/ZOOKEEPER-3636) - find back the missing configuration property in the zookeeperAdmin page when moving from xml to markdown + +## Bug + +* [ZOOKEEPER-706](https://issues.apache.org/jira/browse/ZOOKEEPER-706) - large numbers of watches can cause session re-establishment to fail +* [ZOOKEEPER-1029](https://issues.apache.org/jira/browse/ZOOKEEPER-1029) - C client bug in zookeeper_init (if bad hostname is given) +* [ZOOKEEPER-1077](https://issues.apache.org/jira/browse/ZOOKEEPER-1077) - C client lib doesn't build on Solaris +* [ZOOKEEPER-1256](https://issues.apache.org/jira/browse/ZOOKEEPER-1256) - ClientPortBindTest is failing on Mac OS X +* [ZOOKEEPER-1366](https://issues.apache.org/jira/browse/ZOOKEEPER-1366) - Zookeeper should be tolerant of clock adjustments +* [ZOOKEEPER-1371](https://issues.apache.org/jira/browse/ZOOKEEPER-1371) - Remove dependency on log4j in the source code. +* [ZOOKEEPER-1392](https://issues.apache.org/jira/browse/ZOOKEEPER-1392) - Should not allow to read ACL when not authorized to read node +* [ZOOKEEPER-1460](https://issues.apache.org/jira/browse/ZOOKEEPER-1460) - IPv6 literal address not supported for quorum members +* [ZOOKEEPER-1580](https://issues.apache.org/jira/browse/ZOOKEEPER-1580) - QuorumPeer.setRunning is not used +* [ZOOKEEPER-1636](https://issues.apache.org/jira/browse/ZOOKEEPER-1636) - c-client crash when zoo_amulti failed +* [ZOOKEEPER-1782](https://issues.apache.org/jira/browse/ZOOKEEPER-1782) - zookeeper.superUser is not as super as superDigest +* [ZOOKEEPER-1803](https://issues.apache.org/jira/browse/ZOOKEEPER-1803) - Add description for pzxid in programmer's guide. +* [ZOOKEEPER-1807](https://issues.apache.org/jira/browse/ZOOKEEPER-1807) - Observers spam each other creating connections to the election addr +* [ZOOKEEPER-1818](https://issues.apache.org/jira/browse/ZOOKEEPER-1818) - Fix don't care for trunk +* [ZOOKEEPER-1823](https://issues.apache.org/jira/browse/ZOOKEEPER-1823) - zkTxnLogToolkit -dump should support printing transaction data as a string +* [ZOOKEEPER-1853](https://issues.apache.org/jira/browse/ZOOKEEPER-1853) - zkCli.sh can't issue a CREATE command containing spaces in the data +* [ZOOKEEPER-1893](https://issues.apache.org/jira/browse/ZOOKEEPER-1893) - automake: use serial-tests option +* [ZOOKEEPER-1898](https://issues.apache.org/jira/browse/ZOOKEEPER-1898) - ZooKeeper Java cli shell always returns "0" as exit code +* [ZOOKEEPER-1917](https://issues.apache.org/jira/browse/ZOOKEEPER-1917) - Apache Zookeeper logs cleartext admin passwords +* [ZOOKEEPER-1919](https://issues.apache.org/jira/browse/ZOOKEEPER-1919) - Update the C implementation of removeWatches to have it match ZOOKEEPER-1910 +* [ZOOKEEPER-1927](https://issues.apache.org/jira/browse/ZOOKEEPER-1927) - zkServer.sh fails to read dataDir (and others) from zoo.cfg on Solaris 10 (grep issue, manifests as FAILED TO WRITE PID). +* [ZOOKEEPER-1932](https://issues.apache.org/jira/browse/ZOOKEEPER-1932) - Remove deprecated LeaderElection class +* [ZOOKEEPER-1949](https://issues.apache.org/jira/browse/ZOOKEEPER-1949) - recipes jar not included in the distribution package +* [ZOOKEEPER-1952](https://issues.apache.org/jira/browse/ZOOKEEPER-1952) - Default log directory and file name can be changed +* [ZOOKEEPER-1990](https://issues.apache.org/jira/browse/ZOOKEEPER-1990) - suspicious instantiation of java Random instances +* [ZOOKEEPER-1991](https://issues.apache.org/jira/browse/ZOOKEEPER-1991) - zkServer.sh returns with a zero exit status when a ZooKeeper process is already running +* [ZOOKEEPER-2006](https://issues.apache.org/jira/browse/ZOOKEEPER-2006) - Standalone mode won't take client port from dynamic config +* [ZOOKEEPER-2008](https://issues.apache.org/jira/browse/ZOOKEEPER-2008) - System test fails due to missing leader election port +* [ZOOKEEPER-2013](https://issues.apache.org/jira/browse/ZOOKEEPER-2013) - typos in zookeeperProgrammers +* [ZOOKEEPER-2014](https://issues.apache.org/jira/browse/ZOOKEEPER-2014) - Only admin should be allowed to reconfig a cluster +* [ZOOKEEPER-2026](https://issues.apache.org/jira/browse/ZOOKEEPER-2026) - Startup order in ServerCnxnFactory-ies is wrong +* [ZOOKEEPER-2029](https://issues.apache.org/jira/browse/ZOOKEEPER-2029) - Leader.LearnerCnxAcceptor should handle exceptions in run() +* [ZOOKEEPER-2030](https://issues.apache.org/jira/browse/ZOOKEEPER-2030) - dynamicConfigFile should have an absolute path, not a relative path, to the dynamic configuration file +* [ZOOKEEPER-2049](https://issues.apache.org/jira/browse/ZOOKEEPER-2049) - Yosemite build failure: htonll conflict +* [ZOOKEEPER-2052](https://issues.apache.org/jira/browse/ZOOKEEPER-2052) - Unable to delete a node when the node has no children +* [ZOOKEEPER-2054](https://issues.apache.org/jira/browse/ZOOKEEPER-2054) - test-patch.sh: don't set ulimit -n +* [ZOOKEEPER-2056](https://issues.apache.org/jira/browse/ZOOKEEPER-2056) - Zookeeper 3.4.x and 3.5.0-alpha is not OSGi compliant +* [ZOOKEEPER-2058](https://issues.apache.org/jira/browse/ZOOKEEPER-2058) - rat: exclude *.cer files +* [ZOOKEEPER-2060](https://issues.apache.org/jira/browse/ZOOKEEPER-2060) - Trace bug in NettyServerCnxnFactory +* [ZOOKEEPER-2062](https://issues.apache.org/jira/browse/ZOOKEEPER-2062) - RemoveWatchesTest takes forever to run +* [ZOOKEEPER-2064](https://issues.apache.org/jira/browse/ZOOKEEPER-2064) - Prevent resource leak in various classes +* [ZOOKEEPER-2072](https://issues.apache.org/jira/browse/ZOOKEEPER-2072) - Netty Server Should Configure Child Channel Pipeline By Specifying ChannelPipelineFactory +* [ZOOKEEPER-2073](https://issues.apache.org/jira/browse/ZOOKEEPER-2073) - Memory leak on zookeeper_close +* [ZOOKEEPER-2074](https://issues.apache.org/jira/browse/ZOOKEEPER-2074) - Incorrect exit codes for "./zkCli.sh cmd arg" +* [ZOOKEEPER-2096](https://issues.apache.org/jira/browse/ZOOKEEPER-2096) - C client builds with incorrect error codes in VisualStudio 2010+ +* [ZOOKEEPER-2109](https://issues.apache.org/jira/browse/ZOOKEEPER-2109) - Typo in src/c/src/load_gen.c +* [ZOOKEEPER-2111](https://issues.apache.org/jira/browse/ZOOKEEPER-2111) - Not isAlive states should be synchronized in ClientCnxn +* [ZOOKEEPER-2114](https://issues.apache.org/jira/browse/ZOOKEEPER-2114) - jute generated allocate_* functions are not externally visible +* [ZOOKEEPER-2116](https://issues.apache.org/jira/browse/ZOOKEEPER-2116) - zkCli.sh doesn't honor host:port parameter +* [ZOOKEEPER-2124](https://issues.apache.org/jira/browse/ZOOKEEPER-2124) - Allow Zookeeper version string to have underscore '_' +* [ZOOKEEPER-2133](https://issues.apache.org/jira/browse/ZOOKEEPER-2133) - zkperl: Segmentation fault if getting a node with null value +* [ZOOKEEPER-2142](https://issues.apache.org/jira/browse/ZOOKEEPER-2142) - JMX ObjectName is incorrect for observers +* [ZOOKEEPER-2146](https://issues.apache.org/jira/browse/ZOOKEEPER-2146) - BinaryInputArchive readString should check length before allocating memory +* [ZOOKEEPER-2156](https://issues.apache.org/jira/browse/ZOOKEEPER-2156) - If JAVA_HOME is not set zk startup and fetching status command execution result misleads user. +* [ZOOKEEPER-2157](https://issues.apache.org/jira/browse/ZOOKEEPER-2157) - Upgrade option should be removed from zkServer.sh usage +* [ZOOKEEPER-2171](https://issues.apache.org/jira/browse/ZOOKEEPER-2171) - avoid reverse lookups in QuorumCnxManager +* [ZOOKEEPER-2172](https://issues.apache.org/jira/browse/ZOOKEEPER-2172) - Cluster crashes when reconfig a new node as a participant +* [ZOOKEEPER-2173](https://issues.apache.org/jira/browse/ZOOKEEPER-2173) - ZK startup failure should be handled with proper error message +* [ZOOKEEPER-2174](https://issues.apache.org/jira/browse/ZOOKEEPER-2174) - JUnit4ZKTestRunner logs test failure for all exceptions even if the test method is annotated with an expected exception. +* [ZOOKEEPER-2178](https://issues.apache.org/jira/browse/ZOOKEEPER-2178) - Native client fails compilation on Windows. +* [ZOOKEEPER-2182](https://issues.apache.org/jira/browse/ZOOKEEPER-2182) - Several test suites are not running during pre-commit, because their names do not end with "Test". +* [ZOOKEEPER-2184](https://issues.apache.org/jira/browse/ZOOKEEPER-2184) - Zookeeper Client should re-resolve hosts when connection attempts fail +* [ZOOKEEPER-2186](https://issues.apache.org/jira/browse/ZOOKEEPER-2186) - QuorumCnxManager#receiveConnection may crash with random input +* [ZOOKEEPER-2187](https://issues.apache.org/jira/browse/ZOOKEEPER-2187) - remove duplicated code between CreateRequest{,2} +* [ZOOKEEPER-2190](https://issues.apache.org/jira/browse/ZOOKEEPER-2190) - In StandaloneDisabledTest, testReconfig() shouldn't take leaving servers as joining servers +* [ZOOKEEPER-2193](https://issues.apache.org/jira/browse/ZOOKEEPER-2193) - reconfig command completes even if parameter is wrong obviously +* [ZOOKEEPER-2195](https://issues.apache.org/jira/browse/ZOOKEEPER-2195) - fsync.warningthresholdms in zoo.cfg not working +* [ZOOKEEPER-2197](https://issues.apache.org/jira/browse/ZOOKEEPER-2197) - non-ascii character in FinalRequestProcessor.java +* [ZOOKEEPER-2198](https://issues.apache.org/jira/browse/ZOOKEEPER-2198) - Set default test.junit.threads to 1. +* [ZOOKEEPER-2201](https://issues.apache.org/jira/browse/ZOOKEEPER-2201) - Network issues can cause cluster to hang due to near-deadlock +* [ZOOKEEPER-2210](https://issues.apache.org/jira/browse/ZOOKEEPER-2210) - clock_gettime is not available in os x +* [ZOOKEEPER-2211](https://issues.apache.org/jira/browse/ZOOKEEPER-2211) - PurgeTxnLog does not correctly purge when snapshots and logs are at different locations +* [ZOOKEEPER-2212](https://issues.apache.org/jira/browse/ZOOKEEPER-2212) - distributed race condition related to QV version +* [ZOOKEEPER-2213](https://issues.apache.org/jira/browse/ZOOKEEPER-2213) - Empty path in Set crashes server and prevents restart +* [ZOOKEEPER-2221](https://issues.apache.org/jira/browse/ZOOKEEPER-2221) - Zookeeper JettyAdminServer server should start on configured IP. +* [ZOOKEEPER-2224](https://issues.apache.org/jira/browse/ZOOKEEPER-2224) - Four letter command hangs when network is slow +* [ZOOKEEPER-2227](https://issues.apache.org/jira/browse/ZOOKEEPER-2227) - stmk four-letter word fails execution at server while reading trace mask argument. +* [ZOOKEEPER-2229](https://issues.apache.org/jira/browse/ZOOKEEPER-2229) - Several four-letter words are undocumented. +* [ZOOKEEPER-2235](https://issues.apache.org/jira/browse/ZOOKEEPER-2235) - License update +* [ZOOKEEPER-2239](https://issues.apache.org/jira/browse/ZOOKEEPER-2239) - JMX State from LocalPeerBean incorrect +* [ZOOKEEPER-2243](https://issues.apache.org/jira/browse/ZOOKEEPER-2243) - Supported platforms is completely out of date +* [ZOOKEEPER-2244](https://issues.apache.org/jira/browse/ZOOKEEPER-2244) - On Windows zookeeper fails to restart +* [ZOOKEEPER-2245](https://issues.apache.org/jira/browse/ZOOKEEPER-2245) - SimpleSysTest test cases fails +* [ZOOKEEPER-2247](https://issues.apache.org/jira/browse/ZOOKEEPER-2247) - Zookeeper service becomes unavailable when leader fails to write transaction log +* [ZOOKEEPER-2249](https://issues.apache.org/jira/browse/ZOOKEEPER-2249) - CRC check failed when preAllocSize smaller than node data +* [ZOOKEEPER-2251](https://issues.apache.org/jira/browse/ZOOKEEPER-2251) - Add Client side packet response timeout to avoid infinite wait. +* [ZOOKEEPER-2252](https://issues.apache.org/jira/browse/ZOOKEEPER-2252) - Random test case failure in org.apache.zookeeper.test.StaticHostProviderTest +* [ZOOKEEPER-2256](https://issues.apache.org/jira/browse/ZOOKEEPER-2256) - Zookeeper is not using specified JMX port in zkEnv.sh +* [ZOOKEEPER-2261](https://issues.apache.org/jira/browse/ZOOKEEPER-2261) - When only secureClientPort is configured connections, configuration, connection_stat_reset, and stats admin commands throw NullPointerException +* [ZOOKEEPER-2264](https://issues.apache.org/jira/browse/ZOOKEEPER-2264) - Wrong error message when secureClientPortAddress is configured but secureClientPort is not configured +* [ZOOKEEPER-2269](https://issues.apache.org/jira/browse/ZOOKEEPER-2269) - NullPointerException in RemotePeerBean +* [ZOOKEEPER-2279](https://issues.apache.org/jira/browse/ZOOKEEPER-2279) - QuorumPeer loadDataBase() error message is incorrect +* [ZOOKEEPER-2281](https://issues.apache.org/jira/browse/ZOOKEEPER-2281) - ZK Server startup fails if there are spaces in the JAVA_HOME path +* [ZOOKEEPER-2282](https://issues.apache.org/jira/browse/ZOOKEEPER-2282) - chroot not stripped from path in asynchronous callbacks +* [ZOOKEEPER-2283](https://issues.apache.org/jira/browse/ZOOKEEPER-2283) - traceFile property is not used in the ZooKeeper, it should be removed from documentation +* [ZOOKEEPER-2284](https://issues.apache.org/jira/browse/ZOOKEEPER-2284) - LogFormatter and SnapshotFormatter does not handle FileNotFoundException gracefully +* [ZOOKEEPER-2294](https://issues.apache.org/jira/browse/ZOOKEEPER-2294) - Ant target generate-clover-reports is broken +* [ZOOKEEPER-2295](https://issues.apache.org/jira/browse/ZOOKEEPER-2295) - TGT refresh time logic is wrong +* [ZOOKEEPER-2297](https://issues.apache.org/jira/browse/ZOOKEEPER-2297) - NPE is thrown while creating "key manager" and "trust manager" +* [ZOOKEEPER-2299](https://issues.apache.org/jira/browse/ZOOKEEPER-2299) - NullPointerException in LocalPeerBean for ClientAddress +* [ZOOKEEPER-2302](https://issues.apache.org/jira/browse/ZOOKEEPER-2302) - Some test cases are not running because wrongly named +* [ZOOKEEPER-2307](https://issues.apache.org/jira/browse/ZOOKEEPER-2307) - ZooKeeper not starting because acceptedEpoch is less than the currentEpoch +* [ZOOKEEPER-2311](https://issues.apache.org/jira/browse/ZOOKEEPER-2311) - assert in setup_random +* [ZOOKEEPER-2316](https://issues.apache.org/jira/browse/ZOOKEEPER-2316) - comment does not match code logic +* [ZOOKEEPER-2317](https://issues.apache.org/jira/browse/ZOOKEEPER-2317) - Non-OSGi compatible version +* [ZOOKEEPER-2319](https://issues.apache.org/jira/browse/ZOOKEEPER-2319) - UnresolvedAddressException cause the QuorumCnxManager.Listener exit +* [ZOOKEEPER-2325](https://issues.apache.org/jira/browse/ZOOKEEPER-2325) - Data inconsistency if all snapshots empty or missing +* [ZOOKEEPER-2330](https://issues.apache.org/jira/browse/ZOOKEEPER-2330) - ZooKeeper close API does not close Login thread. +* [ZOOKEEPER-2335](https://issues.apache.org/jira/browse/ZOOKEEPER-2335) - Java Compilation Error in ClientCnxn.java +* [ZOOKEEPER-2338](https://issues.apache.org/jira/browse/ZOOKEEPER-2338) - c bindings should create socket's with SOCK_CLOEXEC to avoid fd leaks on fork/exec +* [ZOOKEEPER-2340](https://issues.apache.org/jira/browse/ZOOKEEPER-2340) - JMX is disabled even if JMXDISABLE is false +* [ZOOKEEPER-2349](https://issues.apache.org/jira/browse/ZOOKEEPER-2349) - Update documentation for snapCount +* [ZOOKEEPER-2355](https://issues.apache.org/jira/browse/ZOOKEEPER-2355) - Ephemeral node is never deleted if follower fails while reading the proposal packet +* [ZOOKEEPER-2364](https://issues.apache.org/jira/browse/ZOOKEEPER-2364) - "ant docs" fails on branch-3.5 due to missing releasenotes.xml. +* [ZOOKEEPER-2366](https://issues.apache.org/jira/browse/ZOOKEEPER-2366) - Reconfiguration of client port causes a socket leak +* [ZOOKEEPER-2375](https://issues.apache.org/jira/browse/ZOOKEEPER-2375) - Prevent multiple initialization of login object in each ZooKeeperSaslClient instance +* [ZOOKEEPER-2379](https://issues.apache.org/jira/browse/ZOOKEEPER-2379) - recent commit broke findbugs qabot check +* [ZOOKEEPER-2380](https://issues.apache.org/jira/browse/ZOOKEEPER-2380) - Deadlock between leader shutdown and forwarding ACK to the leader +* [ZOOKEEPER-2383](https://issues.apache.org/jira/browse/ZOOKEEPER-2383) - Startup race in ZooKeeperServer +* [ZOOKEEPER-2385](https://issues.apache.org/jira/browse/ZOOKEEPER-2385) - Zookeeper trunk build is failing on windows +* [ZOOKEEPER-2388](https://issues.apache.org/jira/browse/ZOOKEEPER-2388) - Unit tests failing on Solaris +* [ZOOKEEPER-2393](https://issues.apache.org/jira/browse/ZOOKEEPER-2393) - Revert run-time dependency on log4j and slf4j-log4j12 +* [ZOOKEEPER-2405](https://issues.apache.org/jira/browse/ZOOKEEPER-2405) - getTGT() in Login.java mishandles confidential information +* [ZOOKEEPER-2413](https://issues.apache.org/jira/browse/ZOOKEEPER-2413) - ContainerManager doesn't close the Timer it creates when stop() is called +* [ZOOKEEPER-2418](https://issues.apache.org/jira/browse/ZOOKEEPER-2418) - txnlog diff sync can skip sending some transactions to followers +* [ZOOKEEPER-2442](https://issues.apache.org/jira/browse/ZOOKEEPER-2442) - Socket leak in QuorumCnxManager connectOne +* [ZOOKEEPER-2450](https://issues.apache.org/jira/browse/ZOOKEEPER-2450) - Upgrade Netty version due to security vulnerability (CVE-2014-3488) +* [ZOOKEEPER-2458](https://issues.apache.org/jira/browse/ZOOKEEPER-2458) - Remove license file for servlet-api dependency +* [ZOOKEEPER-2459](https://issues.apache.org/jira/browse/ZOOKEEPER-2459) - Update NOTICE file with Netty notice +* [ZOOKEEPER-2460](https://issues.apache.org/jira/browse/ZOOKEEPER-2460) - Remove javacc dependency from public Maven pom +* [ZOOKEEPER-2464](https://issues.apache.org/jira/browse/ZOOKEEPER-2464) - NullPointerException on ContainerManager +* [ZOOKEEPER-2465](https://issues.apache.org/jira/browse/ZOOKEEPER-2465) - Documentation copyright notice is out of date. +* [ZOOKEEPER-2467](https://issues.apache.org/jira/browse/ZOOKEEPER-2467) - NullPointerException when redo Command is passed negative value +* [ZOOKEEPER-2470](https://issues.apache.org/jira/browse/ZOOKEEPER-2470) - ServerConfig#parse(String[]) ignores tickTime +* [ZOOKEEPER-2474](https://issues.apache.org/jira/browse/ZOOKEEPER-2474) - add a way for client to reattach to a session when using ZKClientConfig +* [ZOOKEEPER-2477](https://issues.apache.org/jira/browse/ZOOKEEPER-2477) - documentation should refer to Java cli shell and not C cli shell +* [ZOOKEEPER-2500](https://issues.apache.org/jira/browse/ZOOKEEPER-2500) - Fix compilation warnings for CliException classes +* [ZOOKEEPER-2517](https://issues.apache.org/jira/browse/ZOOKEEPER-2517) - jute.maxbuffer is ignored +* [ZOOKEEPER-2536](https://issues.apache.org/jira/browse/ZOOKEEPER-2536) - When provide path for "dataDir" with trailing space, it is taking correct path (by trucating space) for snapshot but creating temporary file with some junk folder name for zookeeper_server.pid +* [ZOOKEEPER-2539](https://issues.apache.org/jira/browse/ZOOKEEPER-2539) - Throwing nullpointerException when run the command "config -c" when client port is mentioned as separate and not like new style +* [ZOOKEEPER-2548](https://issues.apache.org/jira/browse/ZOOKEEPER-2548) - zooInspector does not start on Windows +* [ZOOKEEPER-2558](https://issues.apache.org/jira/browse/ZOOKEEPER-2558) - Potential memory leak in recordio.c +* [ZOOKEEPER-2563](https://issues.apache.org/jira/browse/ZOOKEEPER-2563) - A revisit to setquota +* [ZOOKEEPER-2573](https://issues.apache.org/jira/browse/ZOOKEEPER-2573) - Modify Info.REVISION to adapt git repo +* [ZOOKEEPER-2574](https://issues.apache.org/jira/browse/ZOOKEEPER-2574) - PurgeTxnLog can inadvertently delete required txn log files +* [ZOOKEEPER-2576](https://issues.apache.org/jira/browse/ZOOKEEPER-2576) - After svn to git migration ZooKeeper Precommit jenkins job is failing. +* [ZOOKEEPER-2579](https://issues.apache.org/jira/browse/ZOOKEEPER-2579) - ZooKeeper server should verify that dataDir and snapDir are writeable before starting +* [ZOOKEEPER-2581](https://issues.apache.org/jira/browse/ZOOKEEPER-2581) - Not handled NullPointerException while creating key manager and trustManager +* [ZOOKEEPER-2606](https://issues.apache.org/jira/browse/ZOOKEEPER-2606) - SaslServerCallbackHandler#handleAuthorizeCallback() should log the exception +* [ZOOKEEPER-2611](https://issues.apache.org/jira/browse/ZOOKEEPER-2611) - zoo_remove_watchers - can remove the wrong watch +* [ZOOKEEPER-2617](https://issues.apache.org/jira/browse/ZOOKEEPER-2617) - correct a few spelling typos +* [ZOOKEEPER-2621](https://issues.apache.org/jira/browse/ZOOKEEPER-2621) - ZooKeeper doesn't start on MINGW32 (Windows) +* [ZOOKEEPER-2622](https://issues.apache.org/jira/browse/ZOOKEEPER-2622) - ZooTrace.logQuorumPacket does nothing +* [ZOOKEEPER-2628](https://issues.apache.org/jira/browse/ZOOKEEPER-2628) - Investigate and fix findbug warnings +* [ZOOKEEPER-2633](https://issues.apache.org/jira/browse/ZOOKEEPER-2633) - Build failure in contrib/zkfuse with gcc 6.x +* [ZOOKEEPER-2635](https://issues.apache.org/jira/browse/ZOOKEEPER-2635) - Regenerate documentation +* [ZOOKEEPER-2636](https://issues.apache.org/jira/browse/ZOOKEEPER-2636) - Fix C build break. +* [ZOOKEEPER-2642](https://issues.apache.org/jira/browse/ZOOKEEPER-2642) - ZooKeeper reconfig API backward compatibility fix +* [ZOOKEEPER-2647](https://issues.apache.org/jira/browse/ZOOKEEPER-2647) - Fix TestReconfigServer.cc +* [ZOOKEEPER-2651](https://issues.apache.org/jira/browse/ZOOKEEPER-2651) - Missing src/pom.template in release +* [ZOOKEEPER-2678](https://issues.apache.org/jira/browse/ZOOKEEPER-2678) - Large databases take a long time to regain a quorum +* [ZOOKEEPER-2680](https://issues.apache.org/jira/browse/ZOOKEEPER-2680) - Correct DataNode.getChildren() inconsistent behaviour. +* [ZOOKEEPER-2683](https://issues.apache.org/jira/browse/ZOOKEEPER-2683) - RaceConditionTest is flaky +* [ZOOKEEPER-2684](https://issues.apache.org/jira/browse/ZOOKEEPER-2684) - Fix a crashing bug in the mixed workloads commit processor +* [ZOOKEEPER-2687](https://issues.apache.org/jira/browse/ZOOKEEPER-2687) - Deadlock while shutting down the Leader server. +* [ZOOKEEPER-2690](https://issues.apache.org/jira/browse/ZOOKEEPER-2690) - Update documentation source for ZOOKEEPER-2574 +* [ZOOKEEPER-2693](https://issues.apache.org/jira/browse/ZOOKEEPER-2693) - DOS attack on wchp/wchc four letter words (4lw) +* [ZOOKEEPER-2694](https://issues.apache.org/jira/browse/ZOOKEEPER-2694) - sync CLI command does not wait for result from server +* [ZOOKEEPER-2722](https://issues.apache.org/jira/browse/ZOOKEEPER-2722) - Flaky Test: org.apache.zookeeper.test.ReadOnlyModeTest.testSessionEstablishment +* [ZOOKEEPER-2725](https://issues.apache.org/jira/browse/ZOOKEEPER-2725) - Upgrading to a global session fails with a multiop +* [ZOOKEEPER-2726](https://issues.apache.org/jira/browse/ZOOKEEPER-2726) - Patch for ZOOKEEPER-2693 introduces potential race condition +* [ZOOKEEPER-2735](https://issues.apache.org/jira/browse/ZOOKEEPER-2735) - Typo fixes in some scripts +* [ZOOKEEPER-2737](https://issues.apache.org/jira/browse/ZOOKEEPER-2737) - NettyServerCnxFactory leaks connection if exception happens while writing to a channel. +* [ZOOKEEPER-2743](https://issues.apache.org/jira/browse/ZOOKEEPER-2743) - Netty connection leaks JMX connection bean upon connection close in certain race conditions. +* [ZOOKEEPER-2747](https://issues.apache.org/jira/browse/ZOOKEEPER-2747) - Fix ZooKeeperAdmin Compilation Warning +* [ZOOKEEPER-2757](https://issues.apache.org/jira/browse/ZOOKEEPER-2757) - Incorrect path crashes zkCli +* [ZOOKEEPER-2758](https://issues.apache.org/jira/browse/ZOOKEEPER-2758) - Typo: transasction --> transaction +* [ZOOKEEPER-2775](https://issues.apache.org/jira/browse/ZOOKEEPER-2775) - ZK Client not able to connect with Xid out of order error +* [ZOOKEEPER-2777](https://issues.apache.org/jira/browse/ZOOKEEPER-2777) - There is a typo in zk.py which prevents from using/compiling it. +* [ZOOKEEPER-2778](https://issues.apache.org/jira/browse/ZOOKEEPER-2778) - Potential server deadlock between follower sync with leader and follower receiving external connection requests. +* [ZOOKEEPER-2785](https://issues.apache.org/jira/browse/ZOOKEEPER-2785) - Server inappropriately throttles connections under load before SASL completes +* [ZOOKEEPER-2786](https://issues.apache.org/jira/browse/ZOOKEEPER-2786) - Flaky test: org.apache.zookeeper.test.ClientTest.testNonExistingOpCode +* [ZOOKEEPER-2797](https://issues.apache.org/jira/browse/ZOOKEEPER-2797) - Invalid TTL from misbehaving client nukes zookeeper +* [ZOOKEEPER-2798](https://issues.apache.org/jira/browse/ZOOKEEPER-2798) - Fix flaky test: org.apache.zookeeper.test.ReadOnlyModeTest.testConnectionEvents +* [ZOOKEEPER-2804](https://issues.apache.org/jira/browse/ZOOKEEPER-2804) - Node creation fails with NPE if ACLs are null +* [ZOOKEEPER-2806](https://issues.apache.org/jira/browse/ZOOKEEPER-2806) - Flaky test: org.apache.zookeeper.server.quorum.FLEBackwardElectionRoundTest.testBackwardElectionRound +* [ZOOKEEPER-2808](https://issues.apache.org/jira/browse/ZOOKEEPER-2808) - ACL with index 1 might be removed if it's only being used once +* [ZOOKEEPER-2818](https://issues.apache.org/jira/browse/ZOOKEEPER-2818) - Improve the ZooKeeper#setACL java doc +* [ZOOKEEPER-2819](https://issues.apache.org/jira/browse/ZOOKEEPER-2819) - Changing membership configuration via rolling restart does not work on 3.5.x. +* [ZOOKEEPER-2822](https://issues.apache.org/jira/browse/ZOOKEEPER-2822) - Wrong `ObjectName` about `MBeanServer` in JMX module +* [ZOOKEEPER-2841](https://issues.apache.org/jira/browse/ZOOKEEPER-2841) - ZooKeeper public include files leak porting changes +* [ZOOKEEPER-2845](https://issues.apache.org/jira/browse/ZOOKEEPER-2845) - Data inconsistency issue due to retain database in leader election +* [ZOOKEEPER-2847](https://issues.apache.org/jira/browse/ZOOKEEPER-2847) - Cannot bind to client port when reconfig based on old static config +* [ZOOKEEPER-2852](https://issues.apache.org/jira/browse/ZOOKEEPER-2852) - Snapshot size factor is not read from system property +* [ZOOKEEPER-2853](https://issues.apache.org/jira/browse/ZOOKEEPER-2853) - The lastZxidSeen in FileTxnLog.java is never being assigned +* [ZOOKEEPER-2859](https://issues.apache.org/jira/browse/ZOOKEEPER-2859) - CMake build doesn't support OS X +* [ZOOKEEPER-2861](https://issues.apache.org/jira/browse/ZOOKEEPER-2861) - Main-Class JAR manifest attribute is incorrect +* [ZOOKEEPER-2862](https://issues.apache.org/jira/browse/ZOOKEEPER-2862) - Incorrect javadoc syntax for web links in StaticHostProvider.java +* [ZOOKEEPER-2874](https://issues.apache.org/jira/browse/ZOOKEEPER-2874) - Windows Debug builds don't link with `/MTd` +* [ZOOKEEPER-2886](https://issues.apache.org/jira/browse/ZOOKEEPER-2886) - Permanent session moved error in multi-op only connections +* [ZOOKEEPER-2890](https://issues.apache.org/jira/browse/ZOOKEEPER-2890) - Local automatic variable is left uninitialized and then freed. +* [ZOOKEEPER-2891](https://issues.apache.org/jira/browse/ZOOKEEPER-2891) - Invalid processing of zookeeper_close for mutli-request +* [ZOOKEEPER-2893](https://issues.apache.org/jira/browse/ZOOKEEPER-2893) - very poor choice of logging if client fails to connect to server +* [ZOOKEEPER-2894](https://issues.apache.org/jira/browse/ZOOKEEPER-2894) - Memory and completions leak on zookeeper_close +* [ZOOKEEPER-2901](https://issues.apache.org/jira/browse/ZOOKEEPER-2901) - Session ID that is negative causes mis-calculation of Ephemeral Type +* [ZOOKEEPER-2905](https://issues.apache.org/jira/browse/ZOOKEEPER-2905) - Don't include `config.h` in `zookeeper.h` +* [ZOOKEEPER-2906](https://issues.apache.org/jira/browse/ZOOKEEPER-2906) - The OWASP dependency check jar should not be included in the default classpath +* [ZOOKEEPER-2909](https://issues.apache.org/jira/browse/ZOOKEEPER-2909) - Create ant task to generate ivy dependency reports +* [ZOOKEEPER-2913](https://issues.apache.org/jira/browse/ZOOKEEPER-2913) - testEphemeralNodeDeletion is flaky +* [ZOOKEEPER-2914](https://issues.apache.org/jira/browse/ZOOKEEPER-2914) - compiler warning using java 9 +* [ZOOKEEPER-2920](https://issues.apache.org/jira/browse/ZOOKEEPER-2920) - Upgrade OWASP Dependency Check to 3.2.1 +* [ZOOKEEPER-2923](https://issues.apache.org/jira/browse/ZOOKEEPER-2923) - The comment of the variable matchSyncs in class CommitProcessor has a mistake. +* [ZOOKEEPER-2924](https://issues.apache.org/jira/browse/ZOOKEEPER-2924) - Flaky Test: org.apache.zookeeper.test.LoadFromLogTest.testRestoreWithTransactionErrors +* [ZOOKEEPER-2926](https://issues.apache.org/jira/browse/ZOOKEEPER-2926) - Data inconsistency issue due to the flaw in the session management +* [ZOOKEEPER-2931](https://issues.apache.org/jira/browse/ZOOKEEPER-2931) - WriteLock recipe: incorrect znode ordering when the sessionId is part of the znode name +* [ZOOKEEPER-2934](https://issues.apache.org/jira/browse/ZOOKEEPER-2934) - c versions of election and queue recipes do not compile +* [ZOOKEEPER-2936](https://issues.apache.org/jira/browse/ZOOKEEPER-2936) - Duplicate Keys in log4j.properties config files +* [ZOOKEEPER-2944](https://issues.apache.org/jira/browse/ZOOKEEPER-2944) - Specify correct overflow value +* [ZOOKEEPER-2948](https://issues.apache.org/jira/browse/ZOOKEEPER-2948) - Failing c unit tests on apache jenkins +* [ZOOKEEPER-2949](https://issues.apache.org/jira/browse/ZOOKEEPER-2949) - SSL ServerName not set when using hostname, some proxies may failed to proxy the request. +* [ZOOKEEPER-2951](https://issues.apache.org/jira/browse/ZOOKEEPER-2951) - zkServer.cmd does not start when JAVA_HOME ends with a \ +* [ZOOKEEPER-2953](https://issues.apache.org/jira/browse/ZOOKEEPER-2953) - Flaky Test: testNoLogBeforeLeaderEstablishment +* [ZOOKEEPER-2959](https://issues.apache.org/jira/browse/ZOOKEEPER-2959) - ignore accepted epoch and LEADERINFO ack from observers when a newly elected leader computes new epoch +* [ZOOKEEPER-2961](https://issues.apache.org/jira/browse/ZOOKEEPER-2961) - Fix testElectionFraud Flakyness +* [ZOOKEEPER-2964](https://issues.apache.org/jira/browse/ZOOKEEPER-2964) - "Conf" command returns dataDir and dataLogDir opposingly +* [ZOOKEEPER-2978](https://issues.apache.org/jira/browse/ZOOKEEPER-2978) - fix potential null pointer exception when deleting node +* [ZOOKEEPER-2982](https://issues.apache.org/jira/browse/ZOOKEEPER-2982) - Re-try DNS hostname -> IP resolution +* [ZOOKEEPER-2988](https://issues.apache.org/jira/browse/ZOOKEEPER-2988) - NPE triggered if server receives a vote for a server id not in their voting view +* [ZOOKEEPER-2992](https://issues.apache.org/jira/browse/ZOOKEEPER-2992) - The eclipse build target fails due to protocol redirection: http->https +* [ZOOKEEPER-2993](https://issues.apache.org/jira/browse/ZOOKEEPER-2993) - .ignore file prevents adding src/java/main/org/apache/jute/compiler/generated dir to git repo +* [ZOOKEEPER-2997](https://issues.apache.org/jira/browse/ZOOKEEPER-2997) - CMake should not force static CRT linking +* [ZOOKEEPER-2998](https://issues.apache.org/jira/browse/ZOOKEEPER-2998) - CMake declares incorrect ZooKeeper version +* [ZOOKEEPER-3001](https://issues.apache.org/jira/browse/ZOOKEEPER-3001) - Incorrect log message when try to delete container node +* [ZOOKEEPER-3006](https://issues.apache.org/jira/browse/ZOOKEEPER-3006) - Potential NPE in ZKDatabase#calculateTxnLogSizeLimit +* [ZOOKEEPER-3007](https://issues.apache.org/jira/browse/ZOOKEEPER-3007) - Potential NPE in ReferenceCountedACLCache#deserialize +* [ZOOKEEPER-3009](https://issues.apache.org/jira/browse/ZOOKEEPER-3009) - Potential NPE in NIOServerCnxnFactory +* [ZOOKEEPER-3025](https://issues.apache.org/jira/browse/ZOOKEEPER-3025) - cmake windows build is broken on jenkins +* [ZOOKEEPER-3027](https://issues.apache.org/jira/browse/ZOOKEEPER-3027) - Accidently removed public API of FileTxnLog.setPreallocSize() +* [ZOOKEEPER-3034](https://issues.apache.org/jira/browse/ZOOKEEPER-3034) - Facing issues while building from source +* [ZOOKEEPER-3038](https://issues.apache.org/jira/browse/ZOOKEEPER-3038) - Cleanup some nitpicks in TTL implementation +* [ZOOKEEPER-3039](https://issues.apache.org/jira/browse/ZOOKEEPER-3039) - TxnLogToolkit uses Scanner badly +* [ZOOKEEPER-3041](https://issues.apache.org/jira/browse/ZOOKEEPER-3041) - Typo in error message, affects log analysis +* [ZOOKEEPER-3050](https://issues.apache.org/jira/browse/ZOOKEEPER-3050) - owasp ant target is highlighting jetty version needs to be updated +* [ZOOKEEPER-3051](https://issues.apache.org/jira/browse/ZOOKEEPER-3051) - owasp complaining about jackson version used +* [ZOOKEEPER-3056](https://issues.apache.org/jira/browse/ZOOKEEPER-3056) - Fails to load database with missing snapshot file but valid transaction log file +* [ZOOKEEPER-3057](https://issues.apache.org/jira/browse/ZOOKEEPER-3057) - Fix IPv6 literal usage +* [ZOOKEEPER-3059](https://issues.apache.org/jira/browse/ZOOKEEPER-3059) - EventThread leak in case of Sasl AuthFailed +* [ZOOKEEPER-3072](https://issues.apache.org/jira/browse/ZOOKEEPER-3072) - Race condition in throttling +* [ZOOKEEPER-3079](https://issues.apache.org/jira/browse/ZOOKEEPER-3079) - Fix unsafe use of sprintf(3) for creating IP address strings +* [ZOOKEEPER-3082](https://issues.apache.org/jira/browse/ZOOKEEPER-3082) - Fix server snapshot behavior when out of disk space +* [ZOOKEEPER-3093](https://issues.apache.org/jira/browse/ZOOKEEPER-3093) - sync zerror(int rc) with newest error definitions +* [ZOOKEEPER-3104](https://issues.apache.org/jira/browse/ZOOKEEPER-3104) - Potential data inconsistency due to NEWLEADER packet being sent too early during SNAP sync +* [ZOOKEEPER-3105](https://issues.apache.org/jira/browse/ZOOKEEPER-3105) - Character coding problem occur when create a node using python3 +* [ZOOKEEPER-3113](https://issues.apache.org/jira/browse/ZOOKEEPER-3113) - EphemeralType.get() fails to verify ephemeralOwner when currentElapsedTime() is small enough +* [ZOOKEEPER-3117](https://issues.apache.org/jira/browse/ZOOKEEPER-3117) - Correct the LeaderBean.followerInfo to only return the followers list +* [ZOOKEEPER-3125](https://issues.apache.org/jira/browse/ZOOKEEPER-3125) - Pzxid inconsistent issue when replaying a txn for a deleted node +* [ZOOKEEPER-3127](https://issues.apache.org/jira/browse/ZOOKEEPER-3127) - Fixing potential data inconsistency due to update last processed zxid with partial multi-op txn +* [ZOOKEEPER-3131](https://issues.apache.org/jira/browse/ZOOKEEPER-3131) - org.apache.zookeeper.server.WatchManager resource leak +* [ZOOKEEPER-3144](https://issues.apache.org/jira/browse/ZOOKEEPER-3144) - Potential ephemeral nodes inconsistent due to global session inconsistent with fuzzy snapshot +* [ZOOKEEPER-3145](https://issues.apache.org/jira/browse/ZOOKEEPER-3145) - Potential watch missing issue due to stale pzxid when replaying CloseSession txn with fuzzy snapshot +* [ZOOKEEPER-3156](https://issues.apache.org/jira/browse/ZOOKEEPER-3156) - ZOOKEEPER-2184 causes kerberos principal to not have resolved host name +* [ZOOKEEPER-3162](https://issues.apache.org/jira/browse/ZOOKEEPER-3162) - Broken lock semantics in C client lock-recipe +* [ZOOKEEPER-3210](https://issues.apache.org/jira/browse/ZOOKEEPER-3210) - Typo in zookeeperInternals doc +* [ZOOKEEPER-3212](https://issues.apache.org/jira/browse/ZOOKEEPER-3212) - Fix website with adding doap.rdf back +* [ZOOKEEPER-3217](https://issues.apache.org/jira/browse/ZOOKEEPER-3217) - owasp job flagging slf4j on trunk +* [ZOOKEEPER-3218](https://issues.apache.org/jira/browse/ZOOKEEPER-3218) - zk server reopened,the interval for observer connect to the new leader is too long,then session expired +* [ZOOKEEPER-3253](https://issues.apache.org/jira/browse/ZOOKEEPER-3253) - client should not send requests with cxid=-4, -2, or -1 +* [ZOOKEEPER-3265](https://issues.apache.org/jira/browse/ZOOKEEPER-3265) - Build failure on branch-3.4 +* [ZOOKEEPER-3296](https://issues.apache.org/jira/browse/ZOOKEEPER-3296) - Cannot join quorum due to Quorum SSLSocket connection not closed explicitly when there is handshake issue +* [ZOOKEEPER-3306](https://issues.apache.org/jira/browse/ZOOKEEPER-3306) - Node may not accessible due the the inconsistent ACL reference map after SNAP sync +* [ZOOKEEPER-3320](https://issues.apache.org/jira/browse/ZOOKEEPER-3320) - Leader election port stop listen when hostname unresolvable for some time +* [ZOOKEEPER-3356](https://issues.apache.org/jira/browse/ZOOKEEPER-3356) - Request throttling in Netty is not working as expected and could cause direct buffer OOM issue +* [ZOOKEEPER-3373](https://issues.apache.org/jira/browse/ZOOKEEPER-3373) - need change description for "Single System Image" guarantee in document +* [ZOOKEEPER-3399](https://issues.apache.org/jira/browse/ZOOKEEPER-3399) - Remove logging in getGlobalOutstandingLimit for optimal performance. +* [ZOOKEEPER-3404](https://issues.apache.org/jira/browse/ZOOKEEPER-3404) - BouncyCastle upgrade to 1.61 might cause flaky test issues +* [ZOOKEEPER-3405](https://issues.apache.org/jira/browse/ZOOKEEPER-3405) - owasp flagging jackson-databind +* [ZOOKEEPER-3433](https://issues.apache.org/jira/browse/ZOOKEEPER-3433) - zkpython build broken after maven migration +* [ZOOKEEPER-3440](https://issues.apache.org/jira/browse/ZOOKEEPER-3440) - Fix Apache RAT check by excluding binary files (images) +* [ZOOKEEPER-3471](https://issues.apache.org/jira/browse/ZOOKEEPER-3471) - Potential lock unavailable due to dangling ephemeral nodes left during local session upgrading +* [ZOOKEEPER-3479](https://issues.apache.org/jira/browse/ZOOKEEPER-3479) - Logging false leader election times +* [ZOOKEEPER-3496](https://issues.apache.org/jira/browse/ZOOKEEPER-3496) - Transaction larger than jute.maxbuffer makes ZooKeeper unavailable +* [ZOOKEEPER-3498](https://issues.apache.org/jira/browse/ZOOKEEPER-3498) - In zookeeper-jute project generated source should not be in target\classes folder +* [ZOOKEEPER-3510](https://issues.apache.org/jira/browse/ZOOKEEPER-3510) - Frequent 'zkServer.sh stop' failures when running C test suite +* [ZOOKEEPER-3518](https://issues.apache.org/jira/browse/ZOOKEEPER-3518) - owasp check flagging jackson-databind 2.9.9.1 +* [ZOOKEEPER-3531](https://issues.apache.org/jira/browse/ZOOKEEPER-3531) - Synchronization on ACLCache cause cluster to hang when network/disk issues happen during datatree serialization +* [ZOOKEEPER-3540](https://issues.apache.org/jira/browse/ZOOKEEPER-3540) - Client port unavailable after binding the same client port during reconfig +* [ZOOKEEPER-3546](https://issues.apache.org/jira/browse/ZOOKEEPER-3546) - Containers that never have children stay forever +* [ZOOKEEPER-3559](https://issues.apache.org/jira/browse/ZOOKEEPER-3559) - Update Jackson to 2.9.10 +* [ZOOKEEPER-3563](https://issues.apache.org/jira/browse/ZOOKEEPER-3563) - dependency check failing on 3.4 and 3.5 branches - CVE-2019-16869 on Netty +* [ZOOKEEPER-3590](https://issues.apache.org/jira/browse/ZOOKEEPER-3590) - Zookeeper is unable to set the zookeeper.sasl.client.canonicalize.hostname using system variable +* [ZOOKEEPER-3605](https://issues.apache.org/jira/browse/ZOOKEEPER-3605) - ZOOKEEPER-3242 add a connection throttle. Default constructor needs to set it +* [ZOOKEEPER-3633](https://issues.apache.org/jira/browse/ZOOKEEPER-3633) - AdminServer commands throw NPE when only secure client port is used +* [ZOOKEEPER-3641](https://issues.apache.org/jira/browse/ZOOKEEPER-3641) - New ZOO_VERSION define breaks Perl & Python contribs +* [ZOOKEEPER-3651](https://issues.apache.org/jira/browse/ZOOKEEPER-3651) - NettyServerCnxnFactoryTest is flaky +* [ZOOKEEPER-3653](https://issues.apache.org/jira/browse/ZOOKEEPER-3653) - Audit Log feature fails in a stand alone zookeeper setup + +## Test + +* [ZOOKEEPER-1441](https://issues.apache.org/jira/browse/ZOOKEEPER-1441) - Some test cases are failing because Port bind issue. +* [ZOOKEEPER-2017](https://issues.apache.org/jira/browse/ZOOKEEPER-2017) - New tests for reconfig failure cases +* [ZOOKEEPER-2204](https://issues.apache.org/jira/browse/ZOOKEEPER-2204) - LearnerSnapshotThrottlerTest.testHighContentionWithTimeout fails occasionally +* [ZOOKEEPER-2415](https://issues.apache.org/jira/browse/ZOOKEEPER-2415) - SessionTest is using Thread deprecated API. +* [ZOOKEEPER-2484](https://issues.apache.org/jira/browse/ZOOKEEPER-2484) - Flaky Test: org.apache.zookeeper.test.LoadFromLogTest.testLoadFailure +* [ZOOKEEPER-2508](https://issues.apache.org/jira/browse/ZOOKEEPER-2508) - Many ZooKeeper tests are flaky because they proceed with zk operation without connecting to ZooKeeper server. +* [ZOOKEEPER-2577](https://issues.apache.org/jira/browse/ZOOKEEPER-2577) - Flaky Test: org.apache.zookeeper.server.quorum.ReconfigDuringLeaderSyncTest.testDuringLeaderSync +* [ZOOKEEPER-2656](https://issues.apache.org/jira/browse/ZOOKEEPER-2656) - Fix ServerConfigTest#testValidArguments test case failures +* [ZOOKEEPER-2664](https://issues.apache.org/jira/browse/ZOOKEEPER-2664) - ClientPortBindTest#testBindByAddress may fail due to "No such device" exception +* [ZOOKEEPER-2686](https://issues.apache.org/jira/browse/ZOOKEEPER-2686) - Flaky Test: org.apache.zookeeper.test.WatcherTest. +* [ZOOKEEPER-2716](https://issues.apache.org/jira/browse/ZOOKEEPER-2716) - Flaky Test: org.apache.zookeeper.server.SessionTrackerTest.testAddSessionAfterSessionExpiry +* [ZOOKEEPER-2718](https://issues.apache.org/jira/browse/ZOOKEEPER-2718) - org.apache.zookeeper.server.quorum.StandaloneDisabledTest fails intermittently +* [ZOOKEEPER-2742](https://issues.apache.org/jira/browse/ZOOKEEPER-2742) - Few test cases of org.apache.zookeeper.ZooKeeperTest fails in Windows +* [ZOOKEEPER-2746](https://issues.apache.org/jira/browse/ZOOKEEPER-2746) - Leader hand-off during dynamic reconfig is best effort, while test always expects it +* [ZOOKEEPER-2796](https://issues.apache.org/jira/browse/ZOOKEEPER-2796) - Test org.apache.zookeeper.ZooKeeperTest.testCreateNodeWithoutData is broken by ZOOKEEPER-2757 +* [ZOOKEEPER-2955](https://issues.apache.org/jira/browse/ZOOKEEPER-2955) - Enable Clover code coverage report +* [ZOOKEEPER-2968](https://issues.apache.org/jira/browse/ZOOKEEPER-2968) - Add C client code coverage tests +* [ZOOKEEPER-3074](https://issues.apache.org/jira/browse/ZOOKEEPER-3074) - Flaky test:org.apache.zookeeper.server.ServerStatsTest.testLatencyMetrics +* [ZOOKEEPER-3157](https://issues.apache.org/jira/browse/ZOOKEEPER-3157) - Improve FuzzySnapshotRelatedTest to avoid flaky due to issues like connection loss +* [ZOOKEEPER-3205](https://issues.apache.org/jira/browse/ZOOKEEPER-3205) - Jute - o.a.jute.BinaryInputArchive Test cases +* [ZOOKEEPER-3455](https://issues.apache.org/jira/browse/ZOOKEEPER-3455) - Java 13 build failure on trunk: UnifiedServerSocketTest.testConnectWithoutSSLToStrictServer +* [ZOOKEEPER-3480](https://issues.apache.org/jira/browse/ZOOKEEPER-3480) - Flaky test CommitProcessorMetricsTest.testConcurrentRequestProcessingInCommitProcessor +* [ZOOKEEPER-3495](https://issues.apache.org/jira/browse/ZOOKEEPER-3495) - Broken test in JDK12+: SnapshotDigestTest.testDifferentDigestVersion +* [ZOOKEEPER-3542](https://issues.apache.org/jira/browse/ZOOKEEPER-3542) - X509UtilTest#testClientRenegotiationFails is flaky on JDK8 + linux on machines with 2 cores + +## Wish + +* [ZOOKEEPER-2795](https://issues.apache.org/jira/browse/ZOOKEEPER-2795) - Change log level for "ZKShutdownHandler is not registered" error message +* [ZOOKEEPER-3073](https://issues.apache.org/jira/browse/ZOOKEEPER-3073) - fix couple of typos +* [ZOOKEEPER-3302](https://issues.apache.org/jira/browse/ZOOKEEPER-3302) - ZooKeeper C client does not compile on Fedora 29 +* [ZOOKEEPER-3421](https://issues.apache.org/jira/browse/ZOOKEEPER-3421) - Better insight into Observer connections + +## Task + +* [ZOOKEEPER-925](https://issues.apache.org/jira/browse/ZOOKEEPER-925) - Consider maven site generation to replace our forrest site and documentation generation +* [ZOOKEEPER-1604](https://issues.apache.org/jira/browse/ZOOKEEPER-1604) - remove rpm/deb/... packaging +* [ZOOKEEPER-2658](https://issues.apache.org/jira/browse/ZOOKEEPER-2658) - Trunk / branch-3.5 build broken. +* [ZOOKEEPER-2709](https://issues.apache.org/jira/browse/ZOOKEEPER-2709) - Clarify documentation around "auth" ACL scheme +* [ZOOKEEPER-3002](https://issues.apache.org/jira/browse/ZOOKEEPER-3002) - Upgrade branches 3.5 and trunk to Java 1.8 +* [ZOOKEEPER-3017](https://issues.apache.org/jira/browse/ZOOKEEPER-3017) - Link libm in CMake on FreeBSD +* [ZOOKEEPER-3061](https://issues.apache.org/jira/browse/ZOOKEEPER-3061) - add more details to 'Unhandled scenario for peer' log.warn message +* [ZOOKEEPER-3062](https://issues.apache.org/jira/browse/ZOOKEEPER-3062) - introduce fsync.warningthresholdms constant for FileTxnLog LOG.warn message +* [ZOOKEEPER-3067](https://issues.apache.org/jira/browse/ZOOKEEPER-3067) - Optionally suppress client environment logging. +* [ZOOKEEPER-3087](https://issues.apache.org/jira/browse/ZOOKEEPER-3087) - Fix findbug warning introduced by ZOOKEEPER-3084. +* [ZOOKEEPER-3120](https://issues.apache.org/jira/browse/ZOOKEEPER-3120) - add NetBeans nbproject directory to .gitignore +* [ZOOKEEPER-3136](https://issues.apache.org/jira/browse/ZOOKEEPER-3136) - Reduce log in ClientBase in case of ConnectException +* [ZOOKEEPER-3197](https://issues.apache.org/jira/browse/ZOOKEEPER-3197) - Improve documentation in ZooKeeperServer.superSecret +* [ZOOKEEPER-3230](https://issues.apache.org/jira/browse/ZOOKEEPER-3230) - Add Apache NetBeans Maven project files to .gitignore +* [ZOOKEEPER-3362](https://issues.apache.org/jira/browse/ZOOKEEPER-3362) - Create a simple checkstyle file +* [ZOOKEEPER-3406](https://issues.apache.org/jira/browse/ZOOKEEPER-3406) - Update website for new mailing lists +* [ZOOKEEPER-3407](https://issues.apache.org/jira/browse/ZOOKEEPER-3407) - Update POM file with new information +* [ZOOKEEPER-3431](https://issues.apache.org/jira/browse/ZOOKEEPER-3431) - Enable BookKeeper checkstyle configuration +* [ZOOKEEPER-3441](https://issues.apache.org/jira/browse/ZOOKEEPER-3441) - OWASP is flagging jackson-databind-2.9.9.jar for CVE-2019-12814 +* [ZOOKEEPER-3545](https://issues.apache.org/jira/browse/ZOOKEEPER-3545) - Fix LICENSE files for netty dependency + +## Sub-task + +* [ZOOKEEPER-169](https://issues.apache.org/jira/browse/ZOOKEEPER-169) - Content needed: "Connecting to ZooKeeper" +* [ZOOKEEPER-236](https://issues.apache.org/jira/browse/ZOOKEEPER-236) - SSL Support for Atomic Broadcast protocol +* [ZOOKEEPER-1626](https://issues.apache.org/jira/browse/ZOOKEEPER-1626) - Zookeeper C client should be tolerant of clock adjustments +* [ZOOKEEPER-1660](https://issues.apache.org/jira/browse/ZOOKEEPER-1660) - Add documentation for dynamic reconfiguration +* [ZOOKEEPER-1872](https://issues.apache.org/jira/browse/ZOOKEEPER-1872) - QuorumPeer is not shutdown in few cases +* [ZOOKEEPER-2047](https://issues.apache.org/jira/browse/ZOOKEEPER-2047) - testTruncationNullLog fails on windows +* [ZOOKEEPER-2069](https://issues.apache.org/jira/browse/ZOOKEEPER-2069) - Netty Support for ClientCnxnSocket +* [ZOOKEEPER-2080](https://issues.apache.org/jira/browse/ZOOKEEPER-2080) - Fix deadlock in dynamic reconfiguration +* [ZOOKEEPER-2119](https://issues.apache.org/jira/browse/ZOOKEEPER-2119) - Netty client docs +* [ZOOKEEPER-2122](https://issues.apache.org/jira/browse/ZOOKEEPER-2122) - Impplement SSL support in the Zookeeper C client library +* [ZOOKEEPER-2123](https://issues.apache.org/jira/browse/ZOOKEEPER-2123) - Provide implementation of X509 AuthenticationProvider +* [ZOOKEEPER-2125](https://issues.apache.org/jira/browse/ZOOKEEPER-2125) - SSL on Netty client-server communication +* [ZOOKEEPER-2134](https://issues.apache.org/jira/browse/ZOOKEEPER-2134) - AsyncHammerTest.testHammer fails intermittently +* [ZOOKEEPER-2137](https://issues.apache.org/jira/browse/ZOOKEEPER-2137) - Make testPortChange() less flaky +* [ZOOKEEPER-2152](https://issues.apache.org/jira/browse/ZOOKEEPER-2152) - Intermittent failure in TestReconfig.cc +* [ZOOKEEPER-2153](https://issues.apache.org/jira/browse/ZOOKEEPER-2153) - X509 Authentication Documentation +* [ZOOKEEPER-2168](https://issues.apache.org/jira/browse/ZOOKEEPER-2168) - Add C APIs for new createContainer Methods +* [ZOOKEEPER-2396](https://issues.apache.org/jira/browse/ZOOKEEPER-2396) - Login object in ZooKeeperSaslClient is static +* [ZOOKEEPER-2608](https://issues.apache.org/jira/browse/ZOOKEEPER-2608) - Create CLI option for TTL ephemerals +* [ZOOKEEPER-2609](https://issues.apache.org/jira/browse/ZOOKEEPER-2609) - Add TTL Node APIs to C client +* [ZOOKEEPER-2692](https://issues.apache.org/jira/browse/ZOOKEEPER-2692) - Fix race condition in testWatchAutoResetWithPending +* [ZOOKEEPER-2707](https://issues.apache.org/jira/browse/ZOOKEEPER-2707) - Fix "Unexpected bean exists!" issue in WatcherTests +* [ZOOKEEPER-2750](https://issues.apache.org/jira/browse/ZOOKEEPER-2750) - Document SSL Support for Atomic Broadcast protocol +* [ZOOKEEPER-2753](https://issues.apache.org/jira/browse/ZOOKEEPER-2753) - Introduce a python script for generating flaky test report +* [ZOOKEEPER-2754](https://issues.apache.org/jira/browse/ZOOKEEPER-2754) - Set up Apache Jenkins job that runs the flaky test analyzer script. +* [ZOOKEEPER-2935](https://issues.apache.org/jira/browse/ZOOKEEPER-2935) - [QP MutualAuth]: Port ZOOKEEPER-1045 implementation from branch-3.5 to trunk +* [ZOOKEEPER-2939](https://issues.apache.org/jira/browse/ZOOKEEPER-2939) - Deal with maxbuffer as it relates to proposals +* [ZOOKEEPER-2940](https://issues.apache.org/jira/browse/ZOOKEEPER-2940) - Deal with maxbuffer as it relates to large requests from clients +* [ZOOKEEPER-3022](https://issues.apache.org/jira/browse/ZOOKEEPER-3022) - Step 1.1 - Create docs and it maven structure +* [ZOOKEEPER-3028](https://issues.apache.org/jira/browse/ZOOKEEPER-3028) - Create assembly in pom.xml +* [ZOOKEEPER-3029](https://issues.apache.org/jira/browse/ZOOKEEPER-3029) - Create pom files for jute, server and client +* [ZOOKEEPER-3030](https://issues.apache.org/jira/browse/ZOOKEEPER-3030) - Step 1.3 - Create zk-contrib maven structure +* [ZOOKEEPER-3031](https://issues.apache.org/jira/browse/ZOOKEEPER-3031) - Step 1.4 - Create zk-client maven structure +* [ZOOKEEPER-3032](https://issues.apache.org/jira/browse/ZOOKEEPER-3032) - Step 1.6 - Create zk-server maven structure +* [ZOOKEEPER-3033](https://issues.apache.org/jira/browse/ZOOKEEPER-3033) - Step 1.2 - Create zk-recipes maven structure +* [ZOOKEEPER-3046](https://issues.apache.org/jira/browse/ZOOKEEPER-3046) - testManyChildWatchersAutoReset is flaky +* [ZOOKEEPER-3080](https://issues.apache.org/jira/browse/ZOOKEEPER-3080) - Step 1.5 - Separate jute structure +* [ZOOKEEPER-3103](https://issues.apache.org/jira/browse/ZOOKEEPER-3103) - Pluggable metrics system for ZooKeeper - MetricsProvider API definition +* [ZOOKEEPER-3123](https://issues.apache.org/jira/browse/ZOOKEEPER-3123) - MetricsProvider Lifecycle in ZooKeeper Server +* [ZOOKEEPER-3143](https://issues.apache.org/jira/browse/ZOOKEEPER-3143) - Pluggable metrics system for ZooKeeper - Data Collection on Server +* [ZOOKEEPER-3150](https://issues.apache.org/jira/browse/ZOOKEEPER-3150) - Data integrity check when loading snapshot/txns from disk +* [ZOOKEEPER-3153](https://issues.apache.org/jira/browse/ZOOKEEPER-3153) - Create MarkDown files and build process for them +* [ZOOKEEPER-3154](https://issues.apache.org/jira/browse/ZOOKEEPER-3154) - Update release process to use the MarkDown solution +* [ZOOKEEPER-3155](https://issues.apache.org/jira/browse/ZOOKEEPER-3155) - Remove Forrest XMLs and their build process from the project +* [ZOOKEEPER-3171](https://issues.apache.org/jira/browse/ZOOKEEPER-3171) - Create pom.xml for recipes and contrib +* [ZOOKEEPER-3172](https://issues.apache.org/jira/browse/ZOOKEEPER-3172) - Quorum TLS - fix port unification to allow rolling upgrades +* [ZOOKEEPER-3173](https://issues.apache.org/jira/browse/ZOOKEEPER-3173) - Quorum TLS - support PEM trust/key stores +* [ZOOKEEPER-3174](https://issues.apache.org/jira/browse/ZOOKEEPER-3174) - Quorum TLS - support reloading trust/key store +* [ZOOKEEPER-3175](https://issues.apache.org/jira/browse/ZOOKEEPER-3175) - Quorum TLS - test improvements +* [ZOOKEEPER-3176](https://issues.apache.org/jira/browse/ZOOKEEPER-3176) - Quorum TLS - add SSL config options +* [ZOOKEEPER-3193](https://issues.apache.org/jira/browse/ZOOKEEPER-3193) - Flaky: org.apache.zookeeper.test.SaslAuthFailNotifyTest +* [ZOOKEEPER-3194](https://issues.apache.org/jira/browse/ZOOKEEPER-3194) - Quorum TLS - fix copy/paste bug in ZKTrustManager +* [ZOOKEEPER-3202](https://issues.apache.org/jira/browse/ZOOKEEPER-3202) - Flaky test: org.apache.zookeeper.test.ClientSSLTest.testClientServerSSL +* [ZOOKEEPER-3222](https://issues.apache.org/jira/browse/ZOOKEEPER-3222) - Flaky: multiple intermittent segfaults in C++ tests +* [ZOOKEEPER-3223](https://issues.apache.org/jira/browse/ZOOKEEPER-3223) - Configure Spotbugs +* [ZOOKEEPER-3224](https://issues.apache.org/jira/browse/ZOOKEEPER-3224) - CI integration with maven +* [ZOOKEEPER-3225](https://issues.apache.org/jira/browse/ZOOKEEPER-3225) - Create code coverage analysis with maven build +* [ZOOKEEPER-3226](https://issues.apache.org/jira/browse/ZOOKEEPER-3226) - Activate C Client with a profile, disabled by default +* [ZOOKEEPER-3229](https://issues.apache.org/jira/browse/ZOOKEEPER-3229) - [TLS] add AES-256 ciphers to default cipher list +* [ZOOKEEPER-3233](https://issues.apache.org/jira/browse/ZOOKEEPER-3233) - Run github pre-commit hook tests on 4 threads +* [ZOOKEEPER-3251](https://issues.apache.org/jira/browse/ZOOKEEPER-3251) - Add new server metric types: percentile counter and counter set +* [ZOOKEEPER-3256](https://issues.apache.org/jira/browse/ZOOKEEPER-3256) - Enable OWASP checks to Maven build +* [ZOOKEEPER-3267](https://issues.apache.org/jira/browse/ZOOKEEPER-3267) - Add watcher metrics +* [ZOOKEEPER-3268](https://issues.apache.org/jira/browse/ZOOKEEPER-3268) - Add commit processor metrics +* [ZOOKEEPER-3275](https://issues.apache.org/jira/browse/ZOOKEEPER-3275) - Fix release targets: package, tar, mvn-deploy +* [ZOOKEEPER-3285](https://issues.apache.org/jira/browse/ZOOKEEPER-3285) - Move assembly into its own sub-module +* [ZOOKEEPER-3288](https://issues.apache.org/jira/browse/ZOOKEEPER-3288) - add a new doc:ZookeeperCLI.md +* [ZOOKEEPER-3301](https://issues.apache.org/jira/browse/ZOOKEEPER-3301) - Enforce the quota limit +* [ZOOKEEPER-3305](https://issues.apache.org/jira/browse/ZOOKEEPER-3305) - Add Quorum Packet metrics +* [ZOOKEEPER-3309](https://issues.apache.org/jira/browse/ZOOKEEPER-3309) - Add sync processor metrics +* [ZOOKEEPER-3310](https://issues.apache.org/jira/browse/ZOOKEEPER-3310) - Add metrics for prep processor +* [ZOOKEEPER-3319](https://issues.apache.org/jira/browse/ZOOKEEPER-3319) - Add metrics for follower and observer +* [ZOOKEEPER-3321](https://issues.apache.org/jira/browse/ZOOKEEPER-3321) - Add metrics for Leader +* [ZOOKEEPER-3323](https://issues.apache.org/jira/browse/ZOOKEEPER-3323) - Add TxnSnapLog metrics +* [ZOOKEEPER-3324](https://issues.apache.org/jira/browse/ZOOKEEPER-3324) - Add read/write metrics for top level znodes +* [ZOOKEEPER-3326](https://issues.apache.org/jira/browse/ZOOKEEPER-3326) - Add session/connection related metrics +* [ZOOKEEPER-3327](https://issues.apache.org/jira/browse/ZOOKEEPER-3327) - Add unrecoverable error count +* [ZOOKEEPER-3366](https://issues.apache.org/jira/browse/ZOOKEEPER-3366) - Pluggable metrics system for ZooKeeper - move remaining metrics to MetricsProvider +* [ZOOKEEPER-3379](https://issues.apache.org/jira/browse/ZOOKEEPER-3379) - De-flaky test in Quorum Packet Metrics +* [ZOOKEEPER-3383](https://issues.apache.org/jira/browse/ZOOKEEPER-3383) - Improve prep processor metric accuracy and de-flaky unit test +* [ZOOKEEPER-3401](https://issues.apache.org/jira/browse/ZOOKEEPER-3401) - Fix metric PROPOSAL_ACK_CREATION_LATENCY +* [ZOOKEEPER-3409](https://issues.apache.org/jira/browse/ZOOKEEPER-3409) - Compile Java code with -Xdoclint +* [ZOOKEEPER-3443](https://issues.apache.org/jira/browse/ZOOKEEPER-3443) - Add support for PKCS12 trust/key stores +* [ZOOKEEPER-3465](https://issues.apache.org/jira/browse/ZOOKEEPER-3465) - Introduce BookKeeper checkstyle configuration +* [ZOOKEEPER-3470](https://issues.apache.org/jira/browse/ZOOKEEPER-3470) - Flaky test: LearnerMetricsTest.testLearnerMetricsTest() +* [ZOOKEEPER-3474](https://issues.apache.org/jira/browse/ZOOKEEPER-3474) - Enable BookKeeper checkstyle configuration on zookeeper-promethus +* [ZOOKEEPER-3475](https://issues.apache.org/jira/browse/ZOOKEEPER-3475) - Enable BookKeeper checkstyle configuration on zookeeper-server +* [ZOOKEEPER-3512](https://issues.apache.org/jira/browse/ZOOKEEPER-3512) - Real time data integrity check during broadcast time +* [ZOOKEEPER-3517](https://issues.apache.org/jira/browse/ZOOKEEPER-3517) - Turn on BookKeeper checkstyle configuration at project level +* [ZOOKEEPER-3528](https://issues.apache.org/jira/browse/ZOOKEEPER-3528) - Revisit AsyncCallback javadoc +* [ZOOKEEPER-3529](https://issues.apache.org/jira/browse/ZOOKEEPER-3529) - add a new doc: zookeeperUseCases.md + + + + + -These release notes include new developer and user facing incompatibilities, features, and major improvements. -* [Migration Instructions](#migration) -* [Changes](#changes) - - -## Migration Instructions when Upgrading to 3.0.0 -
- -*You should only have to read this section if you are upgrading from a previous version of ZooKeeper to version 3.0.0, otw skip down to [changes](#changes)* -A small number of changes in this release have resulted in non-backward compatible Zookeeper client user code and server instance data. The following instructions provide details on how to migrate code and date from version 2.2.1 to version 3.0.0. -Note: ZooKeeper increments the major version number (major.minor.fix) when backward incompatible changes are made to the source base. As part of the migration from SourceForge we changed the package structure (com.yahoo.zookeeper.* to org.apache.zookeeper.*) and felt it was a good time to incorporate some changes that we had been withholding. As a result the following will be required when migrating from 2.2.1 to 3.0.0 version of ZooKeeper. -* [Migrating Client Code](#migration_code) -* [Migrating Server Data](#migration_data) -* [Migrating Server Configuration](#migration_config) - -### Migrating Client Code -The underlying client-server protocol has changed in version 3.0.0 -of ZooKeeper. As a result clients must be upgraded along with -serving clusters to ensure proper operation of the system (old -pre-3.0.0 clients are not guaranteed to operate against upgraded -3.0.0 servers and vice-versa). - -#### Watch Management -In previous releases of ZooKeeper any watches registered by clients were lost if the client lost a connection to a ZooKeeper server. -This meant that developers had to track watches they were interested in and reregister them if a session disconnect event was received. -In this release the client library tracks watches that a client has registered and reregisters the watches when a connection is made to a new server. -Applications that still manually reregister interest should continue working properly as long as they are able to handle unsolicited watches. -For example, an old application may register a watch for /foo and /goo, lose the connection, and reregister only /goo. -As long as the application is able to receive a notification for /foo, (probably ignoring it) it does not need to be changed. -One caveat to the watch management: it is possible to miss an event for the creation and deletion of a znode if watching for creation and both the create and delete happens while the client is disconnected from ZooKeeper. - -This release also allows clients to specify call specific watch functions. -This gives the developer the ability to modularize logic in different watch functions rather than cramming everything in the watch function attached to the ZooKeeper handle. -Call specific watch functions receive all session events for as long as they are active, but will only receive the watch callbacks for which they are registered. - - -#### Java API - -1. The java package structure has changed from **com.yahoo.zookeeper*** to **org.apache.zookeeper***. This will probably affect all of your java code which makes use of ZooKeeper APIs (typically import statements) -1. A number of constants used in the client ZooKeeper API were re-specified using enums (rather than ints). See [ZOOKEEPER-7](https://issues.apache.org/jira/browse/ZOOKEEPER-7), [ZOOKEEPER-132](https://issues.apache.org/jira/browse/ZOOKEEPER-132) and [ZOOKEEPER-139](https://issues.apache.org/jira/browse/ZOOKEEPER-139) for full details -1. [ZOOKEEPER-18](https://issues.apache.org/jira/browse/ZOOKEEPER-18) removed KeeperStateChanged, use KeeperStateDisconnected instead - -Also see [the current Java API](http://zookeeper.apache.org/docs/current/api/index.html) - - -#### C API - -1. A number of constants used in the client ZooKeeper API were renamed in order to reduce namespace collision, see [ZOOKEEPER-6](https://issues.apache.org/jira/browse/ZOOKEEPER-6) for full details - - -### Migrating Server Data -The following issues resulted in changes to the on-disk data format (the snapshot and transaction log files contained within the ZK data directory) and require a migration utility to be run. - -* [ZOOKEEPER-27 Unique DB identifiers for servers and clients](https://issues.apache.org/jira/browse/ZOOKEEPER-27) -* [ZOOKEEPER-32 CRCs for ZooKeeper data](https://issues.apache.org/jira/browse/ZOOKEEPER-32) -* [ZOOKEEPER-33 Better ACL management](https://issues.apache.org/jira/browse/ZOOKEEPER-33) -* [ZOOKEEPER-38 headers (version+) in log/snap files](https://issues.apache.org/jira/browse/ZOOKEEPER-38) - -**The following must be run once, and only once, when upgrading the ZooKeeper server instances to version 3.0.0.** - -###### Note -> The and directories referenced below are specified by the *dataLogDir* - and *dataDir* specification in your ZooKeeper config file respectively. *dataLogDir* defaults to - the value of *dataDir* if not specified explicitly in the ZooKeeper server config file (in which - case provide the same directory for both parameters to the upgrade utility). - -1. Shutdown the ZooKeeper server cluster. -1. Backup your and directories -1. Run upgrade using - * `bin/zkServer.sh upgrade ` - - or - - * `java -classpath pathtolog4j:pathtozookeeper.jar UpgradeMain ` - - where is the directory where all transaction logs (log.*) are stored. is the directory where all the snapshots (snapshot.*) are stored. -1. Restart the cluster. - -If you have any failure during the upgrade procedure keep reading to sanitize your database. - -This is how upgrade works in ZooKeeper. This will help you troubleshoot in case you have problems while upgrading - -1. Upgrade moves files from `` and `` to `/version-1/` and `/version-1` respectively (version-1 sub-directory is created by the upgrade utility). -1. Upgrade creates a new version sub-directory `/version-2` and `/version-2` -1. Upgrade reads the old database from `/version-1` and `/version-1` into the memory and creates a new upgraded snapshot. -1. Upgrade writes the new database in `/version-2`. - -Troubleshooting. - - -1. In case you start ZooKeeper 3.0 without upgrading from 2.0 on a 2.0 database - the servers will start up with an empty database. - This is because the servers assume that `/version-2` and `/version-2` will have the database to start with. Since this will be empty - in case of no upgrade, the servers will start with an empty database. In such a case, shutdown the ZooKeeper servers, remove the version-2 directory (remember - this will lead to loss of updates after you started 3.0.) - and then start the upgrade procedure. -1. If the upgrade fails while trying to rename files into the version-1 directory, you should try and move all the files under `/version-1` - and `/version-1` to `` and `` respectively. Then try upgrade again. -1. If you do not wish to run with ZooKeeper 3.0 and prefer to run with ZooKeeper 2.0 and have already upgraded - you can run ZooKeeper 2 with - the `` and `` directories changed to `/version-1` and `/version-1`. Remember that you will lose all the updates that you made after the upgrade. - - -### Migrating Server Configuration - -There is a significant change to the ZooKeeper server configuration file. - -The default election algorithm, specified by the *electionAlg* configuration attribute, has -changed from a default of *0* to a default of *3*. See -[Cluster Options](zookeeperAdmin.html#sc_clusterOptions) section of the administrators guide, specifically -the *electionAlg* and *server.X* properties. - -You will either need to explicitly set *electionAlg* to its previous default value -of *0* or change your *server.X* options to include the leader election port. - - - -## Changes Since ZooKeeper 2.2.1 - -Version 2.2.1 code, documentation, binaries, etc... are still accessible on [SourceForge](http://sourceforge.net/projects/zookeeper) - -| Issue | Notes | -|-------|-------| -|[ZOOKEEPER-43](https://issues.apache.org/jira/browse/ZOOKEEPER-43)|Server side of auto reset watches.| -|[ZOOKEEPER-132](https://issues.apache.org/jira/browse/ZOOKEEPER-132)|Create Enum to replace CreateFlag in ZooKepper.create method| -|[ZOOKEEPER-139](https://issues.apache.org/jira/browse/ZOOKEEPER-139)|Create Enums for WatcherEvent's KeeperState and EventType| -|[ZOOKEEPER-18](https://issues.apache.org/jira/browse/ZOOKEEPER-18)|keeper state inconsistency| -|[ZOOKEEPER-38](https://issues.apache.org/jira/browse/ZOOKEEPER-38)|headers in log/snap files| -|[ZOOKEEPER-8](https://issues.apache.org/jira/browse/ZOOKEEPER-8)|Stat enchaned to include num of children and size| -|[ZOOKEEPER-6](https://issues.apache.org/jira/browse/ZOOKEEPER-6)|List of problem identifiers in zookeeper.h| -|[ZOOKEEPER-7](https://issues.apache.org/jira/browse/ZOOKEEPER-7)|Use enums rather than ints for types and state| -|[ZOOKEEPER-27](https://issues.apache.org/jira/browse/ZOOKEEPER-27)|Unique DB identifiers for servers and clients| -|[ZOOKEEPER-32](https://issues.apache.org/jira/browse/ZOOKEEPER-32)|CRCs for ZooKeeper data| -|[ZOOKEEPER-33](https://issues.apache.org/jira/browse/ZOOKEEPER-33)|Better ACL management| -|[ZOOKEEPER-203](https://issues.apache.org/jira/browse/ZOOKEEPER-203)|fix datadir typo in releasenotes| -|[ZOOKEEPER-145](https://issues.apache.org/jira/browse/ZOOKEEPER-145)|write detailed release notes for users migrating from 2.x to 3.0| -|[ZOOKEEPER-23](https://issues.apache.org/jira/browse/ZOOKEEPER-23)|Auto reset of watches on reconnect| -|[ZOOKEEPER-191](https://issues.apache.org/jira/browse/ZOOKEEPER-191)|forrest docs for upgrade.| -|[ZOOKEEPER-201](https://issues.apache.org/jira/browse/ZOOKEEPER-201)|validate magic number when reading snapshot and transaction logs| -|[ZOOKEEPER-200](https://issues.apache.org/jira/browse/ZOOKEEPER-200)|the magic number for snapshot and log must be different| -|[ZOOKEEPER-199](https://issues.apache.org/jira/browse/ZOOKEEPER-199)|fix log messages in persistence code| -|[ZOOKEEPER-197](https://issues.apache.org/jira/browse/ZOOKEEPER-197)|create checksums for snapshots| -|[ZOOKEEPER-198](https://issues.apache.org/jira/browse/ZOOKEEPER-198)|apache license header missing from FollowerSyncRequest.java| -|[ZOOKEEPER-5](https://issues.apache.org/jira/browse/ZOOKEEPER-5)|Upgrade Feature in Zookeeper server.| -|[ZOOKEEPER-194](https://issues.apache.org/jira/browse/ZOOKEEPER-194)|Fix terminology in zookeeperAdmin.xml| -|[ZOOKEEPER-151](https://issues.apache.org/jira/browse/ZOOKEEPER-151)|Document change to server configuration| -|[ZOOKEEPER-193](https://issues.apache.org/jira/browse/ZOOKEEPER-193)|update java example doc to compile with latest zookeeper| -|[ZOOKEEPER-187](https://issues.apache.org/jira/browse/ZOOKEEPER-187)|CreateMode api docs missing| -|[ZOOKEEPER-186](https://issues.apache.org/jira/browse/ZOOKEEPER-186)|add new "releasenotes.xml" to forrest documentation| -|[ZOOKEEPER-190](https://issues.apache.org/jira/browse/ZOOKEEPER-190)|Reorg links to docs and navs to docs into related sections| -|[ZOOKEEPER-189](https://issues.apache.org/jira/browse/ZOOKEEPER-189)|forrest build not validated xml of input documents| -|[ZOOKEEPER-188](https://issues.apache.org/jira/browse/ZOOKEEPER-188)|Check that election port is present for all servers| -|[ZOOKEEPER-185](https://issues.apache.org/jira/browse/ZOOKEEPER-185)|Improved version of FLETest| -|[ZOOKEEPER-184](https://issues.apache.org/jira/browse/ZOOKEEPER-184)|tests: An explicit include derective is needed for the usage of memcpy functions| -|[ZOOKEEPER-183](https://issues.apache.org/jira/browse/ZOOKEEPER-183)|Array subscript is above array bounds in od_completion, src/cli.c.| -|[ZOOKEEPER-182](https://issues.apache.org/jira/browse/ZOOKEEPER-182)|zookeeper_init accepts empty host-port string and returns valid pointer to zhandle_t.| -|[ZOOKEEPER-17](https://issues.apache.org/jira/browse/ZOOKEEPER-17)|zookeeper_init doc needs clarification| -|[ZOOKEEPER-181](https://issues.apache.org/jira/browse/ZOOKEEPER-181)|Some Source Forge Documents did not get moved over: javaExample, zookeeperTutorial, zookeeperInternals| -|[ZOOKEEPER-180](https://issues.apache.org/jira/browse/ZOOKEEPER-180)|Placeholder sections needed in document for new topics that the umbrella jira discusses| -|[ZOOKEEPER-179](https://issues.apache.org/jira/browse/ZOOKEEPER-179)|Programmer's Guide "Basic Operations" section is missing content| -|[ZOOKEEPER-178](https://issues.apache.org/jira/browse/ZOOKEEPER-178)|FLE test.| -|[ZOOKEEPER-159](https://issues.apache.org/jira/browse/ZOOKEEPER-159)|Cover two corner cases of leader election| -|[ZOOKEEPER-156](https://issues.apache.org/jira/browse/ZOOKEEPER-156)|update programmer guide with acl details from old wiki page| -|[ZOOKEEPER-154](https://issues.apache.org/jira/browse/ZOOKEEPER-154)|reliability graph diagram in overview doc needs context| -|[ZOOKEEPER-157](https://issues.apache.org/jira/browse/ZOOKEEPER-157)|Peer can't find existing leader| -|[ZOOKEEPER-155](https://issues.apache.org/jira/browse/ZOOKEEPER-155)|improve "the zookeeper project" section of overview doc| -|[ZOOKEEPER-140](https://issues.apache.org/jira/browse/ZOOKEEPER-140)|Deadlock in QuorumCnxManager| -|[ZOOKEEPER-147](https://issues.apache.org/jira/browse/ZOOKEEPER-147)|This is version of the documents with most of the [tbd...] scrubbed out| -|[ZOOKEEPER-150](https://issues.apache.org/jira/browse/ZOOKEEPER-150)|zookeeper build broken| -|[ZOOKEEPER-136](https://issues.apache.org/jira/browse/ZOOKEEPER-136)|sync causes hang in all followers of quorum.| -|[ZOOKEEPER-134](https://issues.apache.org/jira/browse/ZOOKEEPER-134)|findbugs cleanup| -|[ZOOKEEPER-133](https://issues.apache.org/jira/browse/ZOOKEEPER-133)|hudson tests failing intermittently| -|[ZOOKEEPER-144](https://issues.apache.org/jira/browse/ZOOKEEPER-144)|add tostring support for watcher event, and enums for event type/state| -|[ZOOKEEPER-21](https://issues.apache.org/jira/browse/ZOOKEEPER-21)|Improve zk ctor/watcher| -|[ZOOKEEPER-142](https://issues.apache.org/jira/browse/ZOOKEEPER-142)|Provide Javadoc as to the maximum size of the data byte array that may be stored within a znode| -|[ZOOKEEPER-93](https://issues.apache.org/jira/browse/ZOOKEEPER-93)|Create Documentation for Zookeeper| -|[ZOOKEEPER-117](https://issues.apache.org/jira/browse/ZOOKEEPER-117)|threading issues in Leader election| -|[ZOOKEEPER-137](https://issues.apache.org/jira/browse/ZOOKEEPER-137)|client watcher objects can lose events| -|[ZOOKEEPER-131](https://issues.apache.org/jira/browse/ZOOKEEPER-131)|Old leader election can elect a dead leader over and over again| -|[ZOOKEEPER-130](https://issues.apache.org/jira/browse/ZOOKEEPER-130)|update build.xml to support apache release process| -|[ZOOKEEPER-118](https://issues.apache.org/jira/browse/ZOOKEEPER-118)|findbugs flagged switch statement in followerrequestprocessor.run| -|[ZOOKEEPER-115](https://issues.apache.org/jira/browse/ZOOKEEPER-115)|Potential NPE in QuorumCnxManager| -|[ZOOKEEPER-114](https://issues.apache.org/jira/browse/ZOOKEEPER-114)|cleanup ugly event messages in zookeeper client| -|[ZOOKEEPER-112](https://issues.apache.org/jira/browse/ZOOKEEPER-112)|src/java/main ZooKeeper.java has test code embedded into it.| -|[ZOOKEEPER-39](https://issues.apache.org/jira/browse/ZOOKEEPER-39)|Use Watcher objects rather than boolean on read operations.| -|[ZOOKEEPER-97](https://issues.apache.org/jira/browse/ZOOKEEPER-97)|supports optional output directory in code generator.| -|[ZOOKEEPER-101](https://issues.apache.org/jira/browse/ZOOKEEPER-101)|Integrate ZooKeeper with "violations" feature on hudson| -|[ZOOKEEPER-105](https://issues.apache.org/jira/browse/ZOOKEEPER-105)|Catch Zookeeper exceptions and print on the stderr.| -|[ZOOKEEPER-42](https://issues.apache.org/jira/browse/ZOOKEEPER-42)|Change Leader Election to fast tcp.| -|[ZOOKEEPER-48](https://issues.apache.org/jira/browse/ZOOKEEPER-48)|auth_id now handled correctly when no auth ids present| -|[ZOOKEEPER-44](https://issues.apache.org/jira/browse/ZOOKEEPER-44)|Create sequence flag children with prefixes of 0's so that they can be lexicographically sorted.| -|[ZOOKEEPER-108](https://issues.apache.org/jira/browse/ZOOKEEPER-108)|Fix sync operation reordering on a Quorum.| -|[ZOOKEEPER-25](https://issues.apache.org/jira/browse/ZOOKEEPER-25)|Fuse module for Zookeeper.| -|[ZOOKEEPER-58](https://issues.apache.org/jira/browse/ZOOKEEPER-58)|Race condition on ClientCnxn.java| -|[ZOOKEEPER-56](https://issues.apache.org/jira/browse/ZOOKEEPER-56)|Add clover support to build.xml.| -|[ZOOKEEPER-75](https://issues.apache.org/jira/browse/ZOOKEEPER-75)|register the ZooKeeper mailing lists with nabble.com| -|[ZOOKEEPER-54](https://issues.apache.org/jira/browse/ZOOKEEPER-54)|remove sleeps in the tests.| -|[ZOOKEEPER-55](https://issues.apache.org/jira/browse/ZOOKEEPER-55)|build.xml failes to retrieve a release number from SVN and the ant target "dist" fails| -|[ZOOKEEPER-89](https://issues.apache.org/jira/browse/ZOOKEEPER-89)|invoke WhenOwnerListener.whenNotOwner when the ZK connection fails| -|[ZOOKEEPER-90](https://issues.apache.org/jira/browse/ZOOKEEPER-90)|invoke WhenOwnerListener.whenNotOwner when the ZK session expires and the znode is the leader| -|[ZOOKEEPER-82](https://issues.apache.org/jira/browse/ZOOKEEPER-82)|Make the ZooKeeperServer more DI friendly.| -|[ZOOKEEPER-110](https://issues.apache.org/jira/browse/ZOOKEEPER-110)|Build script relies on svnant, which is not compatible with subversion 1.5 working copies| -|[ZOOKEEPER-111](https://issues.apache.org/jira/browse/ZOOKEEPER-111)|Significant cleanup of existing tests.| -|[ZOOKEEPER-122](https://issues.apache.org/jira/browse/ZOOKEEPER-122)|Fix NPE in jute's Utils.toCSVString.| -|[ZOOKEEPER-123](https://issues.apache.org/jira/browse/ZOOKEEPER-123)|Fix the wrong class is specified for the logger.| -|[ZOOKEEPER-2](https://issues.apache.org/jira/browse/ZOOKEEPER-2)|Fix synchronization issues in QuorumPeer and FastLeader election.| -|[ZOOKEEPER-125](https://issues.apache.org/jira/browse/ZOOKEEPER-125)|Remove unwanted class declaration in FastLeaderElection.| -|[ZOOKEEPER-61](https://issues.apache.org/jira/browse/ZOOKEEPER-61)|Address in client/server test cases.| -|[ZOOKEEPER-75](https://issues.apache.org/jira/browse/ZOOKEEPER-75)|cleanup the library directory| -|[ZOOKEEPER-109](https://issues.apache.org/jira/browse/ZOOKEEPER-109)|cleanup of NPE and Resource issue nits found by static analysis| -|[ZOOKEEPER-76](https://issues.apache.org/jira/browse/ZOOKEEPER-76)|Commit 677109 removed the cobertura library, but not the build targets.| -|[ZOOKEEPER-63](https://issues.apache.org/jira/browse/ZOOKEEPER-63)|Race condition in client close| -|[ZOOKEEPER-70](https://issues.apache.org/jira/browse/ZOOKEEPER-70)|Add skeleton forrest doc structure for ZooKeeper| -|[ZOOKEEPER-79](https://issues.apache.org/jira/browse/ZOOKEEPER-79)|Document jacob's leader election on the wiki recipes page| -|[ZOOKEEPER-73](https://issues.apache.org/jira/browse/ZOOKEEPER-73)|Move ZK wiki from SourceForge to Apache| -|[ZOOKEEPER-72](https://issues.apache.org/jira/browse/ZOOKEEPER-72)|Initial creation/setup of ZooKeeper ASF site.| -|[ZOOKEEPER-71](https://issues.apache.org/jira/browse/ZOOKEEPER-71)|Determine what to do re ZooKeeper Changelog| -|[ZOOKEEPER-68](https://issues.apache.org/jira/browse/ZOOKEEPER-68)|parseACLs in ZooKeeper.java fails to parse elements of ACL, should be lastIndexOf rather than IndexOf| -|[ZOOKEEPER-130](https://issues.apache.org/jira/browse/ZOOKEEPER-130)|update build.xml to support apache release process.| -|[ZOOKEEPER-131](https://issues.apache.org/jira/browse/ZOOKEEPER-131)|Fix Old leader election can elect a dead leader over and over again.| -|[ZOOKEEPER-137](https://issues.apache.org/jira/browse/ZOOKEEPER-137)|client watcher objects can lose events| -|[ZOOKEEPER-117](https://issues.apache.org/jira/browse/ZOOKEEPER-117)|threading issues in Leader election| -|[ZOOKEEPER-128](https://issues.apache.org/jira/browse/ZOOKEEPER-128)|test coverage on async client operations needs to be improved| -|[ZOOKEEPER-127](https://issues.apache.org/jira/browse/ZOOKEEPER-127)|Use of non-standard election ports in config breaks services| -|[ZOOKEEPER-53](https://issues.apache.org/jira/browse/ZOOKEEPER-53)|tests failing on solaris.| -|[ZOOKEEPER-172](https://issues.apache.org/jira/browse/ZOOKEEPER-172)|FLE Test| -|[ZOOKEEPER-41](https://issues.apache.org/jira/browse/ZOOKEEPER-41)|Sample startup script| -|[ZOOKEEPER-33](https://issues.apache.org/jira/browse/ZOOKEEPER-33)|Better ACL management| -|[ZOOKEEPER-49](https://issues.apache.org/jira/browse/ZOOKEEPER-49)|SetACL does not work| -|[ZOOKEEPER-20](https://issues.apache.org/jira/browse/ZOOKEEPER-20)|Child watches are not triggered when the node is deleted| -|[ZOOKEEPER-15](https://issues.apache.org/jira/browse/ZOOKEEPER-15)|handle failure better in build.xml:test| -|[ZOOKEEPER-11](https://issues.apache.org/jira/browse/ZOOKEEPER-11)|ArrayList is used instead of List| -|[ZOOKEEPER-45](https://issues.apache.org/jira/browse/ZOOKEEPER-45)|Restructure the SVN repository after initial import | -|[ZOOKEEPER-1](https://issues.apache.org/jira/browse/ZOOKEEPER-1)|Initial ZooKeeper code contribution from Yahoo!| From 97b2f8cb64061645e1d7b50bf8ad17ecba76a377 Mon Sep 17 00:00:00 2001 From: ravowlga123 Date: Mon, 23 Dec 2019 16:49:27 +0100 Subject: [PATCH 004/118] ZOOKEEPER-3648: remove Hadoop logo in the ZooKeeper documentation I have removed hadoop logo from header.html present in zookeeper-documentation and also the image hadoop-logo.jpg from images. Please do let me know if I have missed anything. Author: ravowlga123 Reviewers: Enrico Olivelli , Justin Ling Mao Closes #1179 from ravowlga123/ZOOKEEPER-3648 (cherry picked from commit 56f508f1828e4392caab1e50ed76b81b84a7357d) Signed-off-by: Enrico Olivelli --- .../src/main/resources/markdown/html/header.html | 3 --- .../resources/markdown/images/hadoop-logo.jpg | Bin 9443 -> 0 bytes 2 files changed, 3 deletions(-) delete mode 100644 zookeeper-docs/src/main/resources/markdown/images/hadoop-logo.jpg diff --git a/zookeeper-docs/src/main/resources/markdown/html/header.html b/zookeeper-docs/src/main/resources/markdown/html/header.html index ce82ecea565..ef07e99b325 100644 --- a/zookeeper-docs/src/main/resources/markdown/html/header.html +++ b/zookeeper-docs/src/main/resources/markdown/html/header.html @@ -19,9 +19,6 @@ Apache > ZooKeeper
- diff --git a/zookeeper-docs/src/main/resources/markdown/images/hadoop-logo.jpg b/zookeeper-docs/src/main/resources/markdown/images/hadoop-logo.jpg deleted file mode 100644 index 809525d9f158f6163c7692b5e317153ae68bf23f..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 9443 zcmbVxWmFv9vUU$H!6AX*7GMY@L4yZLuwVlTGLYad8Qcj3LIS}DcOBeaLvRR?;4rwm zyMB}Rob#S@@1O79yLHfSiDX8@`P$Wf;~i05zl zbD#x)@y}-yP-8pF5*6_;uYZk_#&(7#465dKj&=^_b{`l7xt{_&PoD}1KjjtX;bB1a z&-jI(3IPCm8t5OV2)6lRBay)+ejE|3xo=r&XIgcneFCWhzCm>v0 z+-8J|gFk*OJAGr`Dtd?V6vn5D{E zDImZ1S$GZY{c!Q_KcJ*~$ohzlode9rFCh3_NLX6tM0!~(E;BD=pTf}oH{N{~z6I8UKhXX{ z_CEvW_kTk6H?V)>ngZ^jgHW4?P69{(CwbSh4!nH?MYUDNbyv?KckJAy&#lP~t zIfs~$AFDq7t%p>xLL?;?Di}p-qL)CXc}iH02g@Xw!$Qr1+dU`|lx&$B!+P{H zYwP_BJrwqpw|oa!vAMDad;+miVui8y0EVV2g=+z=B zRp!A0-4ekr#u?cHoXD1L-W{78Z|p5U#%71eElMpDLnZipJhr9`

*R*qiz_!A~?ZS89anRimGxt(}8A$LVLM=d**S}itKT4vq9r3n4X zFxE`i<_oF9o-pls&UTgZag1+{csUH}dIwa@I7aZP$$l^l@8Moy{ql|bg)Yf(O}4{R zS1YLBl*r3p#xf+`RB1u%OVjN}%SZC-IHI@OIXdU+y47SIH&M>s!>sLlOj&Zdha?|! zS>`dPvH2&r%@Hk&-)KFh-xJN>0VN~HxT0=pQlLny4vYY*($q+I(4kYtdpdLCTnpAF zRV{Xj$)giC^m8$}Vk`;#t?Kbu^{&k6Pj2={F}_SPpUyle-iQt)D|>lYC397n&$7mF zBDJ~J8UwiRagb)PJmvoVk+xhhTKsw_1kc{${lWWTuvWrvrNf=&lRJQUgucri!>S${ z92f57_BiJaJwzQXXrZzM*nHq}hU;a;+W6p^G|hk-iBu;71K3uqn2*{uFJz;9x$dYQ zE&pih`uE-BU&Mupj)BS$RDNJ!VxlPF-X99XCc?+Zy+=YsN=iaRLPB<*<^dV`LkbcS zN=C|uw6yf}^kfg1SeWQoXz1wa{vs~aqu5y3gg7{abmS!DbpPx258{#qJ^nAmMT=rr z6qo;<-6-lpaTbcX&_KAT{8uFg{;(E0Iu9VP@+P3%j(2i9uo!xg-k|br$L8*R;0J`ZJN1MsgshLb|;0vhKGHhng3d}Zg%=m02VPW zJBHJYEZlKx9Fg4jrYL?#WrW(uzHCEo*qUE{^-I-LmKj8)N3y06hcUYo#?ce+tmI9S zTqT7ln@?_IEOU0T9*uDBnZU`_ytq=ef$)#3y>dE3*=ZadWNYkz0LJLj7pg)e#JJs? z5%Q|P$==!U^q4|ue6I+6GK|AMCG#R#Zfcl{1{M|dX)<*5k+A(Pn4C|A*yngWF-Frf ze%4gErI$Nu``(q3&yZ{O;#%Cq!!2;DlTQ`g-!-2vz^&c)4lr$M-71^2 zOjK?>)jS{5`b`tTIu^WJ;Qe;OoYoIsq7WvNMc6<$FIXbkK~P<+GZ3ejyGBN(SW%4I z&Ac!=6Tmmzx$up97d>w2qEmGAh&G9(Q;i*lM^junf8JmFX%wH1;l}8LWQli$cada; zerR3FlHj>kv1&Gf|JS65-3FV>2Z@ksE#@@U2}Fx`^hC4OR-LV&nVFJGr0yy!p`NM2 z7er!Z%5EEco6gJXi1edJ4V%G;GpC>e2e;#+inmw?gUD1<{>{!ep5a2FlR8Y%TIt-&4J`8`7E~~xVVt%IooFHiWb1Bf<*5!{Gpb4Sou9!?3u+dNX`gaboNnfsr!@i%s&1~0oX4*!}=BS=N zYK3&m>wJU+Y1RVU-~wBzFY+u?&(cWJ$5R@G^kLdniCQ2f>q{f6Pz0}+Vf{1s+?0h% zeV&nrQx#wfX~XVUfM9p2vh1`d^*@EGUnI{y$~Gpxj+y!)*KXY_3pOnG$a$W!JK26B zcOw(AC{M5Z5|?lTo{=-_Jrxg2YOEeu8AT2qzWx{{zl;rApUnZc=wwycys0`(4HHA} z#_$T#_@#Wk^qqmtsKrH9q#HYN?6}&(=1@aWR;;6lRv#3p0z1i%n*dz#`vusC4>>gB zLm0=2mz28LO2E_@cfc+*lN2ItKHu-gTqg~GOaL#eif9;zWCeyzvN5y2$DnY+NykNhLNn&_#RMi#>6D|1E0#4)~i(h}HvhFIk*E?4&auW6flx8Zd# ztvs#rjivLVDuQ%DZcUC6d_cX^a)Gxc8TylwWI-15ahC;`0Pe2B3-n#`H9b}*eDhYu zw6(80gM*B2rRdc$TuVW-(>7XPg(`hlVwNc`N21y7c2~|c#$P)sVkaX`<VKtLY7;Ya$lZZDO1tK@9tQf({y?Y-)r8kdg*)|RW9ILXW|h)Bf=*Ju)lh)2yg{LJSVJ}a+J-@ISG z4C70+`E`rD6HONhrIn+qvIa_Kt zQ?q)cvwe`9WVTb?e8W#5!~Db+&GP|RC(yd{IyE+EhP=?qqy|Rx2#>CQi|iK*yRDWb zm~Q2S!aAa5wDx4&W!}s2(mwv0pMD^Y%gNZxsFh_lSF-bH-7}h^(^kfwn~8_Gw|BCJ zqru??{ryO^t|^Mq4#QtX5=x;!-6XB^hi`N6lD>B;yh)jH+Yd^&nqC_2AAN9nYQ@xK zN-&+Qdmx~%!4xg0=8)`>;^nJ0S3Jw_v?{3L`;4<`=>r8GSqZ|2f)&Ge53R^%?B(I#d+UvdK_dOFrld24c5 z<;#Y4Au?_HUG^KsNwm2+?OF&l4Ii5E&`ERa#6cn>m673dUnZ_B4E@qtTq;V)haIv= z_5?b4Lr*IIy#$ok!uAKK^dLKzmI!=(s`q|*U*X@;gO;x4+a)8&-V{~5lhr7_DT1dD zs)@xT?|_+6_F-j?ZSJLlki5s>{wPWKvi3Jw#+4CX3Oy_mn=93DtwVx4@f{UoCrX7(2i2-xAIg5<}nXZ;Pf$Z)dB#tXTdNEX| zZz?o74GMc{6IVCGMK1hpKx5K!J-ZLj=oE&c5q4I2xA&~tpZaYX+F3wrIBM!x!dNZlVXHJiG2KYoZo&%w^11B=JB#mtMmBqj2|+z*+9mWrg;hP# z*D8e8{sF~YfC+0h{5ezSL@gwJ6wIoRzFA!DljSyMZXMcJCYS=%`bJ^dBJyE@ zhR|MIer4IlR)2f@*&@-EPKe`y1XGlA-E(AQyCYqzzrch|&coGBReu6sF?p7h%TKT! zZ`-c6<3x4cT)kBX;vx9)B!{MThIlv8RMA-Mxs{a>@^L^{iE%{O=NH&`yajZQ`1Izg zsc~CBIh*?1SogPTaJfkq;VJKPG08#e8#;;V_IBOZbSn{9s)vh=Vu^|(m_-V>=!nA6 z2dz#8DNEtAV`19~mYkNi2Vm7Z08Q+~Vhckud~RQqW*geH{P;q_Tf@0=4Qoyfdr~iF z7L&EI=@U(Zu-|WP=>wOyQm*&V7a><7dWCUWjVWsg0lCxehT864dk$9AT1UBcTuq!rs`(dJyPwO znzF4jP4gCaYSq&*?fK4M?|Popbg=aYQ;~aOqMl2t0=2<@cu^qZ1zG9Q< z7k!AO?~9-VACHM7Iwx~=uZmR9zF6Xz>rf`_mv9d-YgB(qtr^Wa&lY>R`xOSwg&Ek0=R|unFg_ta`?ABR{9LxsI@NZRuglt5^zF~M`(I!? z`*E)}R{*=x&FtJ0wjm|xAo;Kc8TIK4%Bu@7p4-O8#>ePHlP(6chaZl#*Dl*l=Xedg zm*xgi$Jhie67kRIZk^ndrCUJ^tIh;nw+!RgBi9S_{yP^Y`VyC)dZ1u^H- zhgZ~GRi=7rMFT@GxCO1d*u}g_8jD517q2_zEHFHzHmyV0oPRhB*19cd%FJVw^XVI* zH|}4UxF^KN7LZiR=qkDwyFlU9^T_mU-KL5@5kTeRt-kA?8m&wZe;d%5)Lov0*NQhC zsQUYZE@XjIZ32MT%AdJvR&fjUxTX&$IdR~dj>PWqxZ9HT5}B&(R7<$G_+%yAQguAj zH6Wz3aL`Lw1l_!i=uW;VPIXe#E#~D(D;LzqtkB`oID}NoLnAe;q?`;da^3E&lNAvg zbbNhK&HePPytXQ?ih7OOPp@d^@#ws39xmT1^5Pq(V4E4S1x|-}*)!D=FUfQMN}O}A z#pA5)x(d=CDpv$-K1Z;3n~5g=dR3XE6`rO$5iB!vDp|)S726QFWiNL3ua5Z6{ZmbYLUdFGV}RI&zytft6?DE?+q7DlBLZM%V+hDj6%W-MDiZcCJ`uCv`;R zt?45Z^u8_i(G){>ta;9E^H2t*XpQ_iQn^=543^oZu61=vJFB2^u(jSm&01s-tO zdGfXD{_<{`mCdTB{aZW1stoZtM&&~R3741}G<7-*0RC~rwci@sPc;Jd!<|z^?DgXSnn)E&V)5rtI!oYAwE@NXz>~qk#T?3W65j%29PHsj467~YSqAPc*K7kk+JiJ;$O053qgh#1o=UTBi_tRB6 zKXNF}jAaQ=qa)P(Jo%F`-9EjKO;a{u$(cnoKOrJ>R#0kbsl@LMEvIF{naXGpvm~69lnAWLa>87znV2Em|3theOKVMQ@^j@Y zjn6X%L+Pt|3;}4up0<~jK3VjP0!4(j15@4hC-x=*zKckYdY^WGk1g?iJ@ZGV(q1~w zI5xQ1}V67FWOa_ERn;l1Zs=NL=wO7B-*y^!i+J(w==!7|R~h!OE`h;9m=az;wmpc!_a3P?Wv{vV6tFS z;kJbKXqO*+1N^hSUv0Zg)lxp!T@a;P$?r1P+NhSbD@&rY2|#C=Nn)A%o|#mSMCwXK z7QX@9{UotRocD@U(qLM{yA<0MDKD&FmesjL^bh9`fGXwp+C>qti!v5Xk-t&aLpGo-P zw}$Ju?Q_g@5GMGHbFN8ua%WP227Khof=)_S%sf^a)hmWDP%kEa4DLF2?a+pm2Ik0W zoDt4ZSL-CnOO&eZ7T_PJ{u(%W^ZQH7NYuQT7kRKw>C3f=AV$d)uGj|8oZb6w*9~6| z6U8l@iO4k|o30=2CED!Z8I7`O+m7IRSQ<$Zv`p=SJ;z2)zCT58y)b!)JW_}GCH5euj5R7;PKwN@j4BSU(W=%O_nFdX|thakp z`z>F$*KSJZvs34>JgHDn{(c^ETJoN$yGnrI9kd9CfxlT{d3Q z{_vL>*vu2RZy8f@HJtsOdY#6aaysF3?-K%XrpdZVi8330)YV7}6N!mwgw;z4y!eid zL6dSzsSXB9n%@KG{FX~djn3~+G8K9(;TL+C4vqVkVisN<<6mTn{d2wPLQ=ED5Ios! z_td(2ZFL|V$;C4Xvw}fcacMW%zQ57tn>kiq5G1SpUf*D}LZMuYL>dMaK2xR%=YAXdBWIG!(l*-XQFV_IP z44$h~=6GrL3r{xSou@#*%nkYvdb>npZKG1Rz+iKNRMQQ9deP?=EgW{KkGAC|Utf!H zhKEVg+>BZnjd_?#7zG$tNZZmjlW~}e;LI5_GV2SUNN8_bWlcP`s}j~j_|bj?o_z!X zo{g6yLMB8%_QsfRBZDvXy2~Ex)d+V49j`?asSM>Z(qAi{jZGIux4RIyf6ALGf%Wd3 z{qzdBEl*Y+JHV8R!4nh zX@vN22e6-4eX9ytGK|6OraqSoFP6N;w7cS8Te{?G=la}3*l*KY`gT#;f{QN_nROT@ zzg?eDQR^^#4}o28B1C9sUo~wPyJqCKOlT{^H-z`%C^EU>DlqL$hmxa8mNm&5Gu6p~ z26;?RyzXT@>{F5pih6D%lfF<<#;{CXW)tbYR~eJGNv)gO*pIM$>R6>d77`0fSW17# ziaS|$mi_KB7RK8aCCXcXHE6x7Bd9 z=T5=aj0IAOw@Mh^k#>9Rz&6=-@-$|lBoob-^}Fst-31;);nw=$F5dHc~BLH$!?f1AdLu316E zT|({L{%1(xr^$~*W|n(q!e-d4J=#6Y?@OFH3t-pD5Uz3$R<5QV0QC=%9(^pMrU~Th z=~h#s-Kp;2RC!ok7fE^;nS3@ufnJ)ow_5S_N%92Sq{z9(^%#uxr~%`O%?B@*)udkb zflwgZINwCknRy}e~Cs#hf5p}*i-sOvtxRl-YCq*u`3P?+~pCb`~qXFInUA1t<} zeH8o=#;I488`CKp1VM{gyXwCg-Sf739(gh2=M~iwpmxvWbBKNEN)(TksZkKWxI?gR z(X^XV!$7eio+8wy#U=Acxz|(@WgDnX1&NQd?Ia_&K_iePUWakw%#oA1u+RI=z**>! z<(0hC6TTx1Oo7TXx+WsGjji0wdW$&W-gpO{itOr2cpf-u0fWw;UOd|;k&gM+e&Y_9 zY&mzlo;aXCidDJozXRGbEVn#&ry{aa^>Pu_GaSe=vhQ>aP-RkjWq((^|{}#cs#plVP zN{D%P55YTtFyk`pM{$nH0acxD)%{UeL$wBmEHhVJ2pi(mQhCiso}K!5YpB#~Ac0+v zRz!}KM?}o{gdj3+-gk?JeD9%uM-5FiG9z3om?Q_vT${}f(0|B%d8L>o-?vL8Ad+OP z^|K&cHs;av#fuMF*1~>-Ga{ik^2NHEvaZox+dok?cP;gUOhiN~*lnELKXm!X`9-*> z;GFH|DOpaVV#?KON?6Ucu+^@k`YS4C<&?UtF0)$o-uL*OD(}J>SpzL52C41u0@k9KFt*p{ zQDPT_sYH`L`LnS@9Fd&}+=(fr?KWB1-c7VcPV`B} zLx+Or^nqkY?=)sOdQ2x{P2&qL;K7zt_llV#HFDsyNPF_=oxU2ShJw0mccHL(O1(KMomh?vNQBY)y(e5_>UN#qe~AOPC31Ah0{VQw=|QO#2nxp4j7tFIzF8T04`Z fcAfm8ha<7VMx5Bv*{CHRB$`C|Pvy(!Zu0*Dl(S0n From dbe29a9cffb0a6616e74df9c3f81e45cdc90e87a Mon Sep 17 00:00:00 2001 From: Michael Han Date: Mon, 23 Dec 2019 16:50:59 +0100 Subject: [PATCH 005/118] ZOOKEEPER-3056: fix documentation. snapshot.trust.empty can be used in both zoo.cfg file (with name snapshot.trust.empty), and as a system property (with name zookeeper.snapshot.trust.empty). Author: Michael Han Reviewers: Enrico Olivelli Closes #1178 from hanm/ZOOKEEPER-3056 (cherry picked from commit 04e91c3889b2d0a477036ae088794b6f5b3b04ba) Signed-off-by: Enrico Olivelli --- zookeeper-docs/src/main/resources/markdown/zookeeperAdmin.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/zookeeper-docs/src/main/resources/markdown/zookeeperAdmin.md b/zookeeper-docs/src/main/resources/markdown/zookeeperAdmin.md index 08a054e129d..8e0574cb2f5 100644 --- a/zookeeper-docs/src/main/resources/markdown/zookeeperAdmin.md +++ b/zookeeper-docs/src/main/resources/markdown/zookeeperAdmin.md @@ -969,7 +969,7 @@ property, when available, is noted below. By default, this feautre is disabled, set "true" to enable it. * *snapshot.trust.empty* : - (Java system property only: **zookeeper.snapshot.trust.empty**) + (Java system property: **zookeeper.snapshot.trust.empty**) **New in 3.5.6:** This property controls whether or not ZooKeeper should treat missing snapshot files as a fatal state that can't be recovered from. From 034bcda589ae9d64ab3467b254179ed37f9b1635 Mon Sep 17 00:00:00 2001 From: Fangmin Lyu Date: Fri, 27 Dec 2019 22:33:26 +0100 Subject: [PATCH 006/118] ZOOKEEPER-3512: Real time data consistency check during broadcast time This is the 2nd part of data consistency based on digest, it checks the DataTree digest on every txn during broadcast time. Author: Fangmin Lyu Author: Enrico Olivelli Reviewers: Enrico Olivelli , Andor Molnar , Michael Han Closes #1059 from lvfangmin/ZOOKEEPER-3512 (cherry picked from commit 2805e8982c0e8964e5f8d86f4b01a293d75d4aed) Signed-off-by: Enrico Olivelli --- pom.xml | 6 + .../apache/zookeeper/graph/TxnLogSource.java | 12 +- .../main/resources/markdown/zookeeperAdmin.md | 59 +++- .../src/main/resources/zookeeper.jute | 4 + zookeeper-server/pom.xml | 7 +- .../org/apache/zookeeper/server/DataNode.java | 2 +- .../org/apache/zookeeper/server/DataTree.java | 135 ++++++++- .../zookeeper/server/DigestCalculator.java | 6 +- .../apache/zookeeper/server/LogFormatter.java | 13 +- .../apache/zookeeper/server/NodeHashMap.java | 6 - .../zookeeper/server/NodeHashMapImpl.java | 30 +- .../server/PrepRequestProcessor.java | 193 +++++++++++- .../org/apache/zookeeper/server/Request.java | 12 +- .../zookeeper/server/SnapshotFormatter.java | 5 + .../apache/zookeeper/server/TxnLogEntry.java | 50 ++++ .../server/TxnLogProposalIterator.java | 17 +- .../apache/zookeeper/server/ZKDatabase.java | 21 +- .../zookeeper/server/ZooKeeperServer.java | 28 +- .../server/persistence/FileSnap.java | 13 +- .../server/persistence/FileTxnLog.java | 20 +- .../server/persistence/FileTxnSnapLog.java | 10 +- .../zookeeper/server/persistence/TxnLog.java | 28 ++ .../server/persistence/TxnLogToolkit.java | 6 +- .../zookeeper/server/persistence/Util.java | 9 + .../server/quorum/CommitProcessor.java | 1 + .../zookeeper/server/quorum/Follower.java | 10 +- .../quorum/FollowerZooKeeperServer.java | 4 +- .../zookeeper/server/quorum/Leader.java | 2 +- .../server/quorum/LeaderZooKeeperServer.java | 2 +- .../zookeeper/server/quorum/Learner.java | 24 +- .../server/quorum/LearnerHandler.java | 9 - .../zookeeper/server/quorum/Observer.java | 21 +- .../zookeeper/server/quorum/QuorumBean.java | 10 + .../zookeeper/server/quorum/QuorumMXBean.java | 3 + .../apache/zookeeper/server/util/AdHash.java | 14 +- .../zookeeper/server/util/LogChopper.java | 6 +- .../zookeeper/server/util/SerializeUtils.java | 35 ++- .../apache/zookeeper/server/DataTreeTest.java | 35 ++- .../PrepRequestProcessorMetricsTest.java | 7 +- .../server/PrepRequestProcessorTest.java | 64 +++- .../zookeeper/server/TxnLogDigestTest.java | 277 ++++++++++++++++++ .../persistence/FileTxnSnapLogTest.java | 46 ++- .../quorum/FuzzySnapshotRelatedTest.java | 127 ++++++++ .../server/quorum/LeaderBeanTest.java | 18 +- .../server/quorum/QuorumDigestTest.java | 263 +++++++++++++++++ .../server/quorum/QuorumPeerTestBase.java | 39 ++- .../zookeeper/server/quorum/Zab1_0Test.java | 12 +- .../zookeeper/server/util/AdHashTest.java | 2 +- .../test/GetProposalFromTxnTest.java | 7 +- .../org/apache/zookeeper/test/JMXEnv.java | 1 + 50 files changed, 1557 insertions(+), 174 deletions(-) create mode 100644 zookeeper-server/src/main/java/org/apache/zookeeper/server/TxnLogEntry.java create mode 100644 zookeeper-server/src/test/java/org/apache/zookeeper/server/TxnLogDigestTest.java create mode 100644 zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/QuorumDigestTest.java diff --git a/pom.xml b/pom.xml index c501bcdac7f..3f5d9bd6805 100755 --- a/pom.xml +++ b/pom.xml @@ -280,6 +280,7 @@ 1.7.25 0.5.0 + 1.48 4.12 1.2.17 2.27.0 @@ -399,6 +400,11 @@ log4j ${log4j.version} + + org.jmockit + jmockit + ${jmockit.version} + junit junit diff --git a/zookeeper-contrib/zookeeper-contrib-loggraph/src/main/java/org/apache/zookeeper/graph/TxnLogSource.java b/zookeeper-contrib/zookeeper-contrib-loggraph/src/main/java/org/apache/zookeeper/graph/TxnLogSource.java index 809c4551390..ad2e2589376 100644 --- a/zookeeper-contrib/zookeeper-contrib-loggraph/src/main/java/org/apache/zookeeper/graph/TxnLogSource.java +++ b/zookeeper-contrib/zookeeper-contrib-loggraph/src/main/java/org/apache/zookeeper/graph/TxnLogSource.java @@ -31,6 +31,7 @@ import org.apache.jute.InputArchive; import org.apache.jute.Record; import org.apache.zookeeper.server.TraceFormatter; +import org.apache.zookeeper.server.TxnLogEntry; import org.apache.zookeeper.server.persistence.FileHeader; import org.apache.zookeeper.server.persistence.FileTxnLog; import org.apache.zookeeper.server.util.SerializeUtils; @@ -180,8 +181,10 @@ private LogEntry readNextEntry() { throw new IOException("CRC doesn't match " + crcValue + " vs " + crc.getValue()); } - TxnHeader hdr = new TxnHeader(); - Record r = SerializeUtils.deserializeTxn(bytes, hdr); + + TxnLogEntry logEntry = SerializeUtils.deserializeTxn(bytes); + TxnHeader hdr = logEntry.getHeader(); + Record r = logEntry.getTxn(); switch (hdr.getType()) { case OpCode.createSession: { @@ -327,8 +330,9 @@ public TxnLogSource(String file) throws IOException { if (logStream.readByte("EOR") != 'B') { throw new EOFException("Last transaction was partial."); } - TxnHeader hdr = new TxnHeader(); - Record r = SerializeUtils.deserializeTxn(bytes, hdr); + TxnLogEntry logEntry = SerializeUtils.deserializeTxn(bytes); + TxnHeader hdr = logEntry.getHeader(); + Record r = logEntry.getTxn(); if (starttime == 0) { starttime = hdr.getTime(); diff --git a/zookeeper-docs/src/main/resources/markdown/zookeeperAdmin.md b/zookeeper-docs/src/main/resources/markdown/zookeeperAdmin.md index 8e0574cb2f5..fb7d7894783 100644 --- a/zookeeper-docs/src/main/resources/markdown/zookeeperAdmin.md +++ b/zookeeper-docs/src/main/resources/markdown/zookeeperAdmin.md @@ -964,8 +964,63 @@ property, when available, is noted below. * *digest.enabled* : (Java system property only: **zookeeper.digest.enabled**) **New in 3.6.0:** - The digest feature is added to self-verify the correctness inside - ZooKeeper when loading database from disk, and syncing with leader. + 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 + 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 + 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 + 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 + 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 + 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 + 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 + 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 + 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 + 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, + which is expected to happen during rolling upgrade + 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 + 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, + we can include that after that issue is fixed. + By default, this feautre is disabled, set "true" to enable it. * *snapshot.trust.empty* : diff --git a/zookeeper-jute/src/main/resources/zookeeper.jute b/zookeeper-jute/src/main/resources/zookeeper.jute index 6d553657540..898838f62f6 100644 --- a/zookeeper-jute/src/main/resources/zookeeper.jute +++ b/zookeeper-jute/src/main/resources/zookeeper.jute @@ -278,6 +278,10 @@ module org.apache.zookeeper.server.persistence { } module org.apache.zookeeper.txn { + class TxnDigest { + int version; + long treeDigest; + } class TxnHeader { long clientId; int cxid; diff --git a/zookeeper-server/pom.xml b/zookeeper-server/pom.xml index 307e65028e4..7a72b3780a1 100755 --- a/zookeeper-server/pom.xml +++ b/zookeeper-server/pom.xml @@ -148,6 +148,11 @@ mockito-core test + + org.jmockit + jmockit + test + junit junit @@ -281,7 +286,7 @@ ${surefire-forkcount} false - -Xmx512m -Dtest.junit.threads=${surefire-forkcount} -Dzookeeper.junit.threadid=${surefire.forkNumber} + -Xmx512m -Dtest.junit.threads=${surefire-forkcount} -Dzookeeper.junit.threadid=${surefire.forkNumber} -javaagent:${org.jmockit:jmockit:jar} ${project.basedir} true diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/DataNode.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/DataNode.java index e35482cc72b..8ac8f618522 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/DataNode.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/DataNode.java @@ -204,7 +204,7 @@ public void setDigest(long digest) { this.digest = digest; } - public byte[] getData() { + public synchronized byte[] getData() { return data; } diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/DataTree.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/DataTree.java index 0dc54580699..d3529cf94e6 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/DataTree.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/DataTree.java @@ -77,6 +77,7 @@ import org.apache.zookeeper.txn.SetACLTxn; import org.apache.zookeeper.txn.SetDataTxn; import org.apache.zookeeper.txn.Txn; +import org.apache.zookeeper.txn.TxnDigest; import org.apache.zookeeper.txn.TxnHeader; import org.apache.zookeeper.util.ServiceUtils; import org.slf4j.Logger; @@ -178,6 +179,8 @@ public class DataTree { // The digest associated with the highest zxid in the data tree. private volatile ZxidDigest lastProcessedZxidDigest; + private boolean firstMismatchTxn = true; + // Will be notified when digest mismatch event triggered. private final List digestWatchers = new ArrayList<>(); @@ -465,15 +468,7 @@ public void createNode(final String path, byte[] data, List acl, long ephem int lastSlash = path.lastIndexOf('/'); String parentName = path.substring(0, lastSlash); String childName = path.substring(lastSlash + 1); - StatPersisted stat = new StatPersisted(); - stat.setCtime(time); - stat.setMtime(time); - stat.setCzxid(zxid); - stat.setMzxid(zxid); - stat.setPzxid(zxid); - stat.setVersion(0); - stat.setAversion(0); - stat.setEphemeralOwner(ephemeralOwner); + StatPersisted stat = createStat(zxid, time, ephemeralOwner); DataNode parent = nodes.get(parentName); if (parent == null) { throw new KeeperException.NoNodeException(); @@ -777,7 +772,7 @@ public int getAllChildrenNumber(String path) { return nodes.size() - 2; } - return (int) nodes.keySet().parallelStream().filter(key -> key.startsWith(path + "/")).count(); + return (int) nodes.entrySet().parallelStream().filter(entry -> entry.getKey().startsWith(path + "/")).count(); } public Stat setACL(String path, List acl, int version) throws KeeperException.NoNodeException { @@ -868,6 +863,12 @@ public int hashCode() { public volatile long lastProcessedZxid = 0; + public ProcessTxnResult processTxn(TxnHeader header, Record txn, TxnDigest digest) { + ProcessTxnResult result = processTxn(header, txn); + compareDigest(header, txn, digest); + return result; + } + public ProcessTxnResult processTxn(TxnHeader header, Record txn) { return this.processTxn(header, txn, false); } @@ -1713,9 +1714,10 @@ public boolean serializeZxidDigest(OutputArchive oa) throws IOException { * digestFromLoadedSnapshot. * * @param ia the input stream to read from + * @param startZxidOfSnapshot the zxid of snapshot file * @return the true if it deserialized successfully */ - public boolean deserializeZxidDigest(InputArchive ia) throws IOException { + public boolean deserializeZxidDigest(InputArchive ia, long startZxidOfSnapshot) throws IOException { if (!ZooKeeperServer.isDigestEnabled()) { return false; } @@ -1725,7 +1727,41 @@ public boolean deserializeZxidDigest(InputArchive ia) throws IOException { zxidDigest.deserialize(ia); if (zxidDigest.zxid > 0) { digestFromLoadedSnapshot = zxidDigest; + LOG.info("The digest in the snapshot has digest version of {}, " + + ", with zxid as 0x{}, and digest value as {}", + digestFromLoadedSnapshot.digestVersion, + Long.toHexString(digestFromLoadedSnapshot.zxid), + digestFromLoadedSnapshot.digest); + } else { + digestFromLoadedSnapshot = null; + LOG.info("The digest value is empty in snapshot"); + } + + // There is possibility that the start zxid of a snapshot might + // be larger than the digest zxid in snapshot. + // + // Known cases: + // + // The new leader set the last processed zxid to be the new + // epoch + 0, which is not mapping to any txn, and it uses + // this to take snapshot, which is possible if we don't + // clean database before switching to LOOKING. In this case + // the currentZxidDigest will be the zxid of last epoch and + // it's smaller than the zxid of the snapshot file. + // + // It's safe to reset the targetZxidDigest to null and start + // to compare digest when replaying the first txn, since it's + // a non fuzzy snapshot. + if (digestFromLoadedSnapshot != null && digestFromLoadedSnapshot.zxid < startZxidOfSnapshot) { + LOG.info("The zxid of snapshot digest 0x{} is smaller " + + "than the known snapshot highest zxid, the snapshot " + + "started with zxid 0x{}. It will be invalid to use " + + "this snapshot digest associated with this zxid, will " + + "ignore comparing it.", Long.toHexString(digestFromLoadedSnapshot.zxid), + Long.toHexString(startZxidOfSnapshot)); + digestFromLoadedSnapshot = null; } + return true; } catch (EOFException e) { LOG.warn("Got EOF exception while reading the digest, likely due to the reading an older snapshot."); @@ -1754,9 +1790,56 @@ public void compareSnapshotDigests(long zxid) { } digestFromLoadedSnapshot = null; } else if (digestFromLoadedSnapshot.zxid != 0 && zxid > digestFromLoadedSnapshot.zxid) { - LOG.error( - "Watching for zxid 0x{} during snapshot recovery, but it wasn't found.", - Long.toHexString(digestFromLoadedSnapshot.zxid)); + RATE_LOGGER.rateLimitLog("The txn 0x{} of snapshot digest does not " + + "exist.", Long.toHexString(digestFromLoadedSnapshot.zxid)); + } + } + + /** + * Compares the digest of the tree with the digest present in transaction digest. + * If there is any error, logs and alerts the watchers. + * + * @param header transaction header being applied + * @param txn transaction + * @param digest transaction digest + * + * @return false if digest in the txn doesn't match what we have now in + * the data tree + */ + public boolean compareDigest(TxnHeader header, Record txn, TxnDigest digest) { + long zxid = header.getZxid(); + + if (!ZooKeeperServer.isDigestEnabled() || digest == null) { + return true; + } + // do not compare digest if we're still in fuzzy state + if (digestFromLoadedSnapshot != null) { + return true; + } + // do not compare digest if there is digest version change + if (digestCalculator.getDigestVersion() != digest.getVersion()) { + RATE_LOGGER.rateLimitLog("Digest version not the same on zxid.", + String.valueOf(zxid)); + return true; + } + + long logDigest = digest.getTreeDigest(); + long actualDigest = getTreeDigest(); + if (logDigest != actualDigest) { + reportDigestMismatch(zxid); + LOG.debug("Digest in log: {}, actual tree: {}", logDigest, actualDigest); + if (firstMismatchTxn) { + LOG.error("First digest mismatch on txn: {}, {}, " + + "expected digest is {}, actual digest is {}, ", + header, txn, digest, actualDigest); + firstMismatchTxn = false; + } + return false; + } else { + RATE_LOGGER.flush(); + LOG.debug("Digests are matching for Zxid: {}, Digest in log " + + "and actual tree: {}", Long.toHexString(zxid), logDigest); + return true; } } @@ -1838,7 +1921,7 @@ public void deserialize(InputArchive ia) throws IOException { if (digestVersion < 2) { String d = ia.readString("digest"); if (d != null) { - digest = Long.parseLong(d); + digest = Long.parseLong(d, 16); } } else { digest = ia.readLong("digest"); @@ -1853,10 +1936,30 @@ public int getDigestVersion() { return digestVersion; } - public Long getDigest() { + public long getDigest() { return digest; } } + /** + * Create a node stat from the given params. + * + * @param zxid the zxid associated with the txn + * @param time the time when the txn is created + * @param ephemeralOwner the owner if the node is an ephemeral + * @return the stat + */ + public static StatPersisted createStat(long zxid, long time, long ephemeralOwner) { + StatPersisted stat = new StatPersisted(); + stat.setCtime(time); + stat.setMtime(time); + stat.setCzxid(zxid); + stat.setMzxid(zxid); + stat.setPzxid(zxid); + stat.setVersion(0); + stat.setAversion(0); + stat.setEphemeralOwner(ephemeralOwner); + return stat; + } } diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/DigestCalculator.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/DigestCalculator.java index ca5041f2008..1209aba115c 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/DigestCalculator.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/DigestCalculator.java @@ -22,15 +22,11 @@ import java.util.zip.CRC32; import org.apache.zookeeper.ZooDefs; import org.apache.zookeeper.data.StatPersisted; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; /** * Defines how to calculate the digest for a given node. */ -class DigestCalculator { - - private static final Logger LOG = LoggerFactory.getLogger(DigestCalculator.class); +public class DigestCalculator { // The hardcoded digest version, should bump up this version whenever // we changed the digest method or fields. diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/LogFormatter.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/LogFormatter.java index e02a63df296..81392dfcb1d 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/LogFormatter.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/LogFormatter.java @@ -32,6 +32,7 @@ import org.apache.zookeeper.server.persistence.FileHeader; import org.apache.zookeeper.server.persistence.FileTxnLog; import org.apache.zookeeper.server.util.SerializeUtils; +import org.apache.zookeeper.txn.TxnDigest; import org.apache.zookeeper.txn.TxnHeader; import org.apache.zookeeper.util.ServiceUtils; import org.slf4j.Logger; @@ -75,6 +76,9 @@ public static void main(String[] args) throws Exception { + " txnlog format version " + fhdr.getVersion()); + // enable digest + ZooKeeperServer.setDigestEnabled(true); + int count = 0; while (true) { long crcValue; @@ -98,15 +102,18 @@ public static void main(String[] args) throws Exception { if (crcValue != crc.getValue()) { throw new IOException("CRC doesn't match " + crcValue + " vs " + crc.getValue()); } - TxnHeader hdr = new TxnHeader(); - Record txn = SerializeUtils.deserializeTxn(bytes, hdr); + TxnLogEntry entry = SerializeUtils.deserializeTxn(bytes); + TxnHeader hdr = entry.getHeader(); + Record txn = entry.getTxn(); + TxnDigest digest = entry.getDigest(); System.out.println( DateFormat.getDateTimeInstance(DateFormat.SHORT, DateFormat.LONG).format(new Date(hdr.getTime())) + " session 0x" + Long.toHexString(hdr.getClientId()) + " cxid 0x" + Long.toHexString(hdr.getCxid()) + " zxid 0x" + Long.toHexString(hdr.getZxid()) + " " + Request.op2String(hdr.getType()) - + " " + txn); + + " " + txn + + " " + digest); if (logStream.readByte("EOR") != 'B') { LOG.error("Last transaction was partial."); throw new EOFException("Last transaction was partial."); diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/NodeHashMap.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/NodeHashMap.java index f19bbfb3455..136ff7ba044 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/NodeHashMap.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/NodeHashMap.java @@ -20,7 +20,6 @@ import java.util.Map; import java.util.Set; -import java.util.concurrent.ConcurrentHashMap; /** * The interface defined to manage the hash based on the entries in the @@ -59,11 +58,6 @@ public interface NodeHashMap { */ DataNode remove(String path); - /** - * Return all key set view inside this map. - */ - ConcurrentHashMap.KeySetView keySet(); - /** * Return all the entries inside this map. */ diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/NodeHashMapImpl.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/NodeHashMapImpl.java index b28130bdde6..eb56e90a712 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/NodeHashMapImpl.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/NodeHashMapImpl.java @@ -21,6 +21,7 @@ import java.util.Map; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; +import org.apache.zookeeper.ZooDefs; import org.apache.zookeeper.server.util.AdHash; /** @@ -29,13 +30,17 @@ */ public class NodeHashMapImpl implements NodeHashMap { - private final ConcurrentHashMap nodes = new ConcurrentHashMap(); - - private AdHash hash = new AdHash(); + private final ConcurrentHashMap nodes; + private final boolean digestEnabled; private final DigestCalculator digestCalculator; + private final AdHash hash; + public NodeHashMapImpl(DigestCalculator digestCalculator) { this.digestCalculator = digestCalculator; + nodes = new ConcurrentHashMap<>(); + hash = new AdHash(); + digestEnabled = ZooKeeperServer.isDigestEnabled(); } @Override @@ -67,11 +72,6 @@ public DataNode remove(String path) { return oldNode; } - @Override - public ConcurrentHashMap.KeySetView keySet() { - return nodes.keySet(); - } - @Override public Set> entrySet() { return nodes.entrySet(); @@ -80,7 +80,7 @@ public Set> entrySet() { @Override public void clear() { nodes.clear(); - hash = new AdHash(); + hash.clear(); } @Override @@ -102,13 +102,21 @@ public void postChange(String path, DataNode node) { } private void addDigest(String path, DataNode node) { - if (ZooKeeperServer.isDigestEnabled()) { + // Excluding everything under '/zookeeper/' for digest calculation. + if (path.startsWith(ZooDefs.ZOOKEEPER_NODE_SUBTREE)) { + return; + } + if (digestEnabled) { hash.addDigest(digestCalculator.calculateDigest(path, node)); } } private void removeDigest(String path, DataNode node) { - if (ZooKeeperServer.isDigestEnabled()) { + // Excluding everything under '/zookeeper/' for digest calculation. + if (path.startsWith(ZooDefs.ZOOKEEPER_NODE_SUBTREE)) { + return; + } + if (digestEnabled) { hash.removeDigest(digestCalculator.calculateDigest(path, node)); } } 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 70d989a34ac..74720ed4a1b 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 @@ -56,6 +56,7 @@ import org.apache.zookeeper.proto.SetACLRequest; import org.apache.zookeeper.proto.SetDataRequest; import org.apache.zookeeper.server.ZooKeeperServer.ChangeRecord; +import org.apache.zookeeper.server.ZooKeeperServer.PrecalculatedDigest; import org.apache.zookeeper.server.auth.ProviderRegistry; import org.apache.zookeeper.server.auth.ServerAuthenticationProvider; import org.apache.zookeeper.server.quorum.LeaderZooKeeperServer; @@ -76,6 +77,7 @@ import org.apache.zookeeper.txn.SetACLTxn; import org.apache.zookeeper.txn.SetDataTxn; import org.apache.zookeeper.txn.Txn; +import org.apache.zookeeper.txn.TxnDigest; import org.apache.zookeeper.txn.TxnHeader; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -100,9 +102,15 @@ public class PrepRequestProcessor extends ZooKeeperCriticalThread implements Req LinkedBlockingQueue submittedRequests = new LinkedBlockingQueue(); private final RequestProcessor nextProcessor; + private final boolean digestEnabled; + private DigestCalculator digestCalculator; ZooKeeperServer zks; + public enum DigestOpCode { + NOOP, ADD, REMOVE, UPDATE; + } + public PrepRequestProcessor(ZooKeeperServer zks, RequestProcessor nextProcessor) { super( "ProcessThread(sid:" + zks.getServerId() @@ -110,6 +118,10 @@ public PrepRequestProcessor(ZooKeeperServer zks, RequestProcessor nextProcessor) + "):", zks.getZooKeeperServerListener()); this.nextProcessor = nextProcessor; this.zks = zks; + this.digestEnabled = ZooKeeperServer.isDigestEnabled(); + if (this.digestEnabled) { + this.digestCalculator = new DigestCalculator(); + } } /** @@ -159,6 +171,12 @@ private ChangeRecord getRecordForPath(String path) throws KeeperException.NoNode children = n.getChildren(); } lastChange = new ChangeRecord(-1, path, n.stat, children.size(), zks.getZKDatabase().aclForNode(n)); + + if (digestEnabled) { + lastChange.precalculatedDigest = new PrecalculatedDigest( + digestCalculator.calculateDigest(path, n), 0); + } + lastChange.data = n.getData(); } } } @@ -297,8 +315,12 @@ private String validatePathForCreate(String path, long sessionId) throws BadArgu * @param record */ protected void pRequest2Txn(int type, long zxid, Request request, Record record, boolean deserialize) throws KeeperException, IOException, RequestProcessorException { - request.setHdr(new TxnHeader(request.sessionId, request.cxid, zxid, Time.currentWallTime(), type)); + if (request.getHdr() == null) { + request.setHdr(new TxnHeader(request.sessionId, request.cxid, zxid, + Time.currentWallTime(), type)); + } + PrecalculatedDigest precalculatedDigest; switch (type) { case OpCode.create: case OpCode.create2: @@ -321,8 +343,15 @@ protected void pRequest2Txn(int type, long zxid, Request request, Record record, request.setTxn(new DeleteTxn(path)); parentRecord = parentRecord.duplicate(request.getHdr().getZxid()); parentRecord.childCount--; + parentRecord.stat.setPzxid(request.getHdr().getZxid()); + parentRecord.precalculatedDigest = precalculateDigest( + DigestOpCode.UPDATE, parentPath, parentRecord.data, parentRecord.stat); addChangeRecord(parentRecord); - addChangeRecord(new ChangeRecord(request.getHdr().getZxid(), path, null, -1, null)); + + nodeRecord = new ChangeRecord(request.getHdr().getZxid(), path, null, -1, null); + nodeRecord.precalculatedDigest = precalculateDigest(DigestOpCode.REMOVE, path); + setTxnDigest(request, nodeRecord.precalculatedDigest); + addChangeRecord(nodeRecord); break; } case OpCode.delete: @@ -343,8 +372,15 @@ protected void pRequest2Txn(int type, long zxid, Request request, Record record, request.setTxn(new DeleteTxn(path)); parentRecord = parentRecord.duplicate(request.getHdr().getZxid()); parentRecord.childCount--; + parentRecord.stat.setPzxid(request.getHdr().getZxid()); + parentRecord.precalculatedDigest = precalculateDigest( + DigestOpCode.UPDATE, parentPath, parentRecord.data, parentRecord.stat); addChangeRecord(parentRecord); - addChangeRecord(new ChangeRecord(request.getHdr().getZxid(), path, null, -1, null)); + + nodeRecord = new ChangeRecord(request.getHdr().getZxid(), path, null, -1, null); + nodeRecord.precalculatedDigest = precalculateDigest(DigestOpCode.REMOVE, path); + setTxnDigest(request, nodeRecord.precalculatedDigest); + addChangeRecord(nodeRecord); break; case OpCode.setData: zks.sessionTracker.checkSession(request.sessionId, request.getOwner()); @@ -360,6 +396,12 @@ protected void pRequest2Txn(int type, long zxid, Request request, Record record, request.setTxn(new SetDataTxn(path, setDataRequest.getData(), newVersion)); nodeRecord = nodeRecord.duplicate(request.getHdr().getZxid()); nodeRecord.stat.setVersion(newVersion); + nodeRecord.stat.setMtime(request.getHdr().getTime()); + nodeRecord.stat.setMzxid(zxid); + nodeRecord.data = setDataRequest.getData(); + nodeRecord.precalculatedDigest = precalculateDigest( + DigestOpCode.UPDATE, path, nodeRecord.data, nodeRecord.stat); + setTxnDigest(request, nodeRecord.precalculatedDigest); addChangeRecord(nodeRecord); break; case OpCode.reconfig: @@ -490,10 +532,20 @@ protected void pRequest2Txn(int type, long zxid, Request request, Record record, nodeRecord = getRecordForPath(ZooDefs.CONFIG_NODE); zks.checkACL(request.cnxn, nodeRecord.acl, ZooDefs.Perms.WRITE, request.authInfo, null, null); - request.setTxn(new SetDataTxn(ZooDefs.CONFIG_NODE, request.qv.toString().getBytes(), -1)); + SetDataTxn setDataTxn = new SetDataTxn(ZooDefs.CONFIG_NODE, request.qv.toString().getBytes(), -1); + request.setTxn(setDataTxn); nodeRecord = nodeRecord.duplicate(request.getHdr().getZxid()); nodeRecord.stat.setVersion(-1); + nodeRecord.stat.setMtime(request.getHdr().getTime()); + nodeRecord.stat.setMzxid(zxid); + nodeRecord.data = setDataTxn.getData(); + // Reconfig is currently a noop from digest computation + // perspective since config node is not covered by the digests. + nodeRecord.precalculatedDigest = precalculateDigest( + DigestOpCode.NOOP, ZooDefs.CONFIG_NODE, nodeRecord.data, nodeRecord.stat); + setTxnDigest(request, nodeRecord.precalculatedDigest); addChangeRecord(nodeRecord); + break; case OpCode.setACL: zks.sessionTracker.checkSession(request.sessionId, request.getOwner()); @@ -510,6 +562,9 @@ protected void pRequest2Txn(int type, long zxid, Request request, Record record, request.setTxn(new SetACLTxn(path, listACL, newVersion)); nodeRecord = nodeRecord.duplicate(request.getHdr().getZxid()); nodeRecord.stat.setAversion(newVersion); + nodeRecord.precalculatedDigest = precalculateDigest( + DigestOpCode.UPDATE, path, nodeRecord.data, nodeRecord.stat); + setTxnDigest(request, nodeRecord.precalculatedDigest); addChangeRecord(nodeRecord); break; case OpCode.createSession: @@ -542,7 +597,20 @@ protected void pRequest2Txn(int type, long zxid, Request request, Record record, } } for (String path2Delete : es) { - addChangeRecord(new ChangeRecord(request.getHdr().getZxid(), path2Delete, null, 0, null)); + if (digestEnabled) { + parentPath = getParentPathAndValidate(path2Delete); + parentRecord = getRecordForPath(parentPath); + parentRecord = parentRecord.duplicate(request.getHdr().getZxid()); + parentRecord.stat.setPzxid(request.getHdr().getZxid()); + parentRecord.precalculatedDigest = precalculateDigest( + DigestOpCode.UPDATE, parentPath, parentRecord.data, parentRecord.stat); + addChangeRecord(parentRecord); + } + nodeRecord = new ChangeRecord( + request.getHdr().getZxid(), path2Delete, null, 0, null); + nodeRecord.precalculatedDigest = precalculateDigest( + DigestOpCode.REMOVE, path2Delete); + addChangeRecord(nodeRecord); } if (ZooKeeperServer.isCloseSessionTxnEnabled()) { request.setTxn(new CloseSessionTxn(new ArrayList(es))); @@ -569,6 +637,12 @@ protected void pRequest2Txn(int type, long zxid, Request request, Record record, LOG.warn("unknown type {}", type); break; } + + // If the txn is not going to mutate anything, like createSession, + // we just set the current tree digest in it + if (request.getTxnDigest() == null && digestEnabled) { + setTxnDigest(request); + } } private void pRequest2TxnCreate(int type, Request request, Record record, boolean deserialize) throws IOException, KeeperException { @@ -628,15 +702,31 @@ private void pRequest2TxnCreate(int type, Request request, Record record, boolea } else { request.setTxn(new CreateTxn(path, data, listACL, createMode.isEphemeral(), newCversion)); } - StatPersisted s = new StatPersisted(); - if (createMode.isEphemeral()) { - s.setEphemeralOwner(request.sessionId); + + TxnHeader hdr = request.getHdr(); + long ephemeralOwner = 0; + if (createMode.isContainer()) { + ephemeralOwner = EphemeralType.CONTAINER_EPHEMERAL_OWNER; + } else if (createMode.isTTL()) { + ephemeralOwner = EphemeralType.TTL.toEphemeralOwner(ttl); + } else if (createMode.isEphemeral()) { + ephemeralOwner = request.sessionId; } + StatPersisted s = DataTree.createStat(hdr.getZxid(), hdr.getTime(), ephemeralOwner); parentRecord = parentRecord.duplicate(request.getHdr().getZxid()); parentRecord.childCount++; parentRecord.stat.setCversion(newCversion); + parentRecord.stat.setPzxid(request.getHdr().getZxid()); + parentRecord.precalculatedDigest = precalculateDigest( + DigestOpCode.UPDATE, parentPath, parentRecord.data, parentRecord.stat); addChangeRecord(parentRecord); - addChangeRecord(new ChangeRecord(request.getHdr().getZxid(), path, s, 0, listACL)); + ChangeRecord nodeRecord = new ChangeRecord( + request.getHdr().getZxid(), path, s, 0, listACL); + nodeRecord.data = data; + nodeRecord.precalculatedDigest = precalculateDigest( + DigestOpCode.ADD, path, nodeRecord.data, s); + setTxnDigest(request, nodeRecord.precalculatedDigest); + addChangeRecord(nodeRecord); } private void validatePath(String path, long sessionId) throws BadArgumentsException { @@ -724,6 +814,8 @@ protected void pRequest(Request request) throws RequestProcessorException { //Store off current pending change records in case we need to rollback Map pendingChanges = getPendingChanges(multiRequest); + request.setHdr(new TxnHeader(request.sessionId, request.cxid, zxid, + Time.currentWallTime(), request.type)); for (Op op : multiRequest) { Record subrequest = op.toRequestRecord(); @@ -741,7 +833,7 @@ protected void pRequest(Request request) throws RequestProcessorException { /* Prep the request and convert to a Txn */ try { pRequest2Txn(op.getType(), zxid, request, subrequest, false); - type = request.getHdr().getType(); + type = op.getType(); txn = request.getTxn(); } catch (KeeperException e) { ke = e; @@ -774,8 +866,10 @@ protected void pRequest(Request request) throws RequestProcessorException { } } - request.setHdr(new TxnHeader(request.sessionId, request.cxid, zxid, Time.currentWallTime(), request.type)); request.setTxn(new MultiTxn(txns)); + if (digestEnabled) { + setTxnDigest(request); + } break; @@ -956,4 +1050,81 @@ public void shutdown() { nextProcessor.shutdown(); } + /** + * Calculate the node digest and tree digest after the change. + * + * @param type the type of operations about the digest change + * @param path the path of the node + * @param data the data of the node + * @param s the stat of the node + * + * @return PrecalculatedDigest the pair of node and tree digest + */ + private PrecalculatedDigest precalculateDigest(DigestOpCode type, String path, + byte[] data, StatPersisted s) throws KeeperException.NoNodeException { + + if (!digestEnabled) { + return null; + } + + long prevNodeDigest; + long newNodeDigest; + + switch (type) { + case ADD: + prevNodeDigest = 0; + newNodeDigest = digestCalculator.calculateDigest(path, data, s); + break; + case REMOVE: + prevNodeDigest = getRecordForPath(path).precalculatedDigest.nodeDigest; + newNodeDigest = 0; + break; + case UPDATE: + prevNodeDigest = getRecordForPath(path).precalculatedDigest.nodeDigest; + newNodeDigest = digestCalculator.calculateDigest(path, data, s); + break; + case NOOP: + newNodeDigest = prevNodeDigest = 0; + break; + default: + return null; + } + long treeDigest = getCurrentTreeDigest() - prevNodeDigest + newNodeDigest; + return new PrecalculatedDigest(newNodeDigest, treeDigest); + } + + private PrecalculatedDigest precalculateDigest( + DigestOpCode type, String path) throws KeeperException.NoNodeException { + return precalculateDigest(type, path, null, null); + } + + /** + * Query the current tree digest from DataTree or outstandingChanges list. + * + * @return current tree digest + */ + private long getCurrentTreeDigest() { + long digest; + synchronized (zks.outstandingChanges) { + if (zks.outstandingChanges.isEmpty()) { + digest = zks.getZKDatabase().getDataTree().getTreeDigest(); + LOG.debug("Digest got from data tree is: {}", digest); + } else { + digest = zks.outstandingChanges.peekLast().precalculatedDigest.treeDigest; + LOG.debug("Digest got from outstandingChanges is: {}", digest); + } + } + return digest; + } + + private void setTxnDigest(Request request) { + request.setTxnDigest(new TxnDigest(digestCalculator.getDigestVersion(), getCurrentTreeDigest())); + } + + private void setTxnDigest(Request request, PrecalculatedDigest preCalculatedDigest) { + if (preCalculatedDigest == null) { + return; + } + request.setTxnDigest(new TxnDigest(digestCalculator.getDigestVersion(), preCalculatedDigest.treeDigest)); + } } 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 63cc30b7eb6..d0fb7da4d2e 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 @@ -29,6 +29,7 @@ import org.apache.zookeeper.metrics.SummarySet; import org.apache.zookeeper.server.quorum.flexible.QuorumVerifier; import org.apache.zookeeper.server.util.AuthUtil; +import org.apache.zookeeper.txn.TxnDigest; import org.apache.zookeeper.txn.TxnHeader; /** @@ -105,6 +106,8 @@ public Request(long sessionId, int xid, int type, TxnHeader hdr, Record txn, lon public QuorumVerifier qv = null; + private TxnDigest txnDigest; + /** * If this is a create or close request for a local-only session. */ @@ -435,7 +438,6 @@ public void logLatency(SummarySet metric, String key) { logLatency(metric, key, Time.currentWallTime()); } - /** * Returns comma separated list of users authenticated in the current * session @@ -462,4 +464,12 @@ public String getUsers() { } return users.toString(); } + + public TxnDigest getTxnDigest() { + return txnDigest; + } + + public void setTxnDigest(TxnDigest txnDigest) { + this.txnDigest = txnDigest; + } } diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/SnapshotFormatter.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/SnapshotFormatter.java index a276ef3d0b2..5fce206b53f 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/SnapshotFormatter.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/SnapshotFormatter.java @@ -115,6 +115,11 @@ public void run(String snapshotFileName, boolean dumpData, boolean dumpJson) thr private void printDetails(DataTree dataTree, Map sessions, boolean dumpData, long fileNameZxid) { long dtZxid = printZnodeDetails(dataTree, dumpData); printSessionDetails(dataTree, sessions); + DataTree.ZxidDigest targetZxidDigest = dataTree.getDigestFromLoadedSnapshot(); + if (targetZxidDigest != null) { + System.out.println(String.format("Target zxid digest is: %s, %s", + Long.toHexString(targetZxidDigest.zxid), targetZxidDigest.digest)); + } System.out.println(String.format("----%nLast zxid: 0x%s", Long.toHexString(Math.max(fileNameZxid, dtZxid)))); } diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/TxnLogEntry.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/TxnLogEntry.java new file mode 100644 index 00000000000..352eb81da90 --- /dev/null +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/TxnLogEntry.java @@ -0,0 +1,50 @@ +/** + * 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; + +import org.apache.jute.Record; +import org.apache.zookeeper.txn.TxnDigest; +import org.apache.zookeeper.txn.TxnHeader; + +/** + * A helper class to represent the txn entry. + */ +public final class TxnLogEntry { + private final Record txn; + private final TxnHeader header; + private final TxnDigest digest; + + public TxnLogEntry(Record txn, TxnHeader header, TxnDigest digest) { + this.txn = txn; + this.header = header; + this.digest = digest; + } + + public Record getTxn() { + return txn; + } + + public TxnHeader getHeader() { + return header; + } + + public TxnDigest getDigest() { + return digest; + } +} diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/TxnLogProposalIterator.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/TxnLogProposalIterator.java index 5e080e1fbdf..847e3b2fabe 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/TxnLogProposalIterator.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/TxnLogProposalIterator.java @@ -18,16 +18,13 @@ package org.apache.zookeeper.server; -import java.io.ByteArrayOutputStream; import java.io.IOException; import java.util.Iterator; -import org.apache.jute.BinaryOutputArchive; -import org.apache.jute.Record; import org.apache.zookeeper.server.persistence.TxnLog.TxnIterator; +import org.apache.zookeeper.server.persistence.Util; import org.apache.zookeeper.server.quorum.Leader; import org.apache.zookeeper.server.quorum.Leader.Proposal; import org.apache.zookeeper.server.quorum.QuorumPacket; -import org.apache.zookeeper.txn.TxnHeader; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -61,19 +58,11 @@ public boolean hasNext() { @Override public Proposal next() { - ByteArrayOutputStream baos = new ByteArrayOutputStream(); - BinaryOutputArchive boa = BinaryOutputArchive.getArchive(baos); Proposal p = new Proposal(); try { - TxnHeader hdr = itr.getHeader(); - Record txn = itr.getTxn(); - hdr.serialize(boa, "hdr"); - if (txn != null) { - txn.serialize(boa, "txn"); - } - baos.close(); + byte[] serializedData = Util.marshallTxnEntry(itr.getHeader(), itr.getTxn(), itr.getDigest()); - QuorumPacket pp = new QuorumPacket(Leader.PROPOSAL, itr.getHeader().getZxid(), baos.toByteArray(), null); + QuorumPacket pp = new QuorumPacket(Leader.PROPOSAL, itr.getHeader().getZxid(), serializedData, null); p.packet = pp; p.request = null; diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/ZKDatabase.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/ZKDatabase.java index c9c6d54cdb8..f758f5de5ae 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/ZKDatabase.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/ZKDatabase.java @@ -55,6 +55,7 @@ import org.apache.zookeeper.server.quorum.QuorumPacket; import org.apache.zookeeper.server.quorum.flexible.QuorumVerifier; import org.apache.zookeeper.server.util.SerializeUtils; +import org.apache.zookeeper.txn.TxnDigest; import org.apache.zookeeper.txn.TxnHeader; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -268,8 +269,8 @@ public ConcurrentHashMap getSessionWithTimeOuts() { } private final PlayBackListener commitProposalPlaybackListener = new PlayBackListener() { - public void onTxnLoaded(TxnHeader hdr, Record txn) { - addCommittedProposal(hdr, txn); + public void onTxnLoaded(TxnHeader hdr, Record txn, TxnDigest digest) { + addCommittedProposal(hdr, txn, digest); } }; @@ -285,7 +286,8 @@ public long loadDataBase() throws IOException { initialized = true; long loadTime = Time.currentElapsedTime() - startTime; ServerMetrics.getMetrics().DB_INIT_TIME.add(loadTime); - LOG.info("Snapshot loaded in {} ms", loadTime); + LOG.info("Snapshot loaded in {} ms, highest zxid is 0x{}, digest is {}", + loadTime, Long.toHexString(zxid), dataTree.getTreeDigest()); return zxid; } @@ -300,8 +302,9 @@ public long fastForwardDataBase() throws IOException { return zxid; } - private void addCommittedProposal(TxnHeader hdr, Record txn) { + private void addCommittedProposal(TxnHeader hdr, Record txn, TxnDigest digest) { Request r = new Request(0, hdr.getCxid(), hdr.getType(), hdr, txn, hdr.getZxid()); + r.setTxnDigest(digest); addCommittedProposal(r); } @@ -468,14 +471,15 @@ public void setlastProcessedZxid(long zxid) { } /** - * the process txn on the data + * the process txn on the data and perform digest comparision. * @param hdr the txnheader for the txn * @param txn the transaction that needs to be processed + * @param digest the expected digest. A null value would skip the check * @return the result of processing the transaction on this * datatree/zkdatabase */ - public ProcessTxnResult processTxn(TxnHeader hdr, Record txn) { - return dataTree.processTxn(hdr, txn); + public ProcessTxnResult processTxn(TxnHeader hdr, Record txn, TxnDigest digest) { + return dataTree.processTxn(hdr, txn, digest); } /** @@ -745,4 +749,7 @@ public long getTxnSize() { return snapLog.getTotalLogSize(); } + public boolean compareDigest(TxnHeader header, Record txn, TxnDigest digest) { + return dataTree.compareDigest(header, txn, digest); + } } 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 1c9dda77256..1a2d9a7e06a 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 @@ -79,6 +79,7 @@ import org.apache.zookeeper.server.util.OSMXBean; import org.apache.zookeeper.server.util.RequestPathMetricsCollector; import org.apache.zookeeper.txn.CreateSessionTxn; +import org.apache.zookeeper.txn.TxnDigest; import org.apache.zookeeper.txn.TxnHeader; import org.apache.zookeeper.util.ServiceUtils; import org.slf4j.Logger; @@ -875,11 +876,24 @@ private int requestThrottleInflight() { return 0; } + static class PrecalculatedDigest { + final long nodeDigest; + final long treeDigest; + + PrecalculatedDigest(long nodeDigest, long treeDigest) { + this.nodeDigest = nodeDigest; + this.treeDigest = treeDigest; + } + } + + /** * This structure is used to facilitate information sharing between PrepRP * and FinalRP. */ static class ChangeRecord { + PrecalculatedDigest precalculatedDigest; + byte[] data; ChangeRecord(long zxid, String path, StatPersisted stat, int childCount, List acl) { this.zxid = zxid; @@ -904,7 +918,11 @@ ChangeRecord duplicate(long zxid) { if (this.stat != null) { DataTree.copyStatPersisted(this.stat, stat); } - return new ChangeRecord(zxid, path, stat, childCount, acl == null ? new ArrayList<>() : new ArrayList<>(acl)); + ChangeRecord changeRecord = new ChangeRecord(zxid, path, stat, childCount, + acl == null ? new ArrayList<>() : new ArrayList<>(acl)); + changeRecord.precalculatedDigest = precalculatedDigest; + changeRecord.data = data; + return changeRecord; } } @@ -1677,7 +1695,7 @@ private void processSasl(ByteBuffer incomingBuffer, ServerCnxn cnxn, RequestHead // entry point for quorum/Learner.java public ProcessTxnResult processTxn(TxnHeader hdr, Record txn) { processTxnForSessionEvents(null, hdr, txn); - return processTxnInDB(hdr, txn); + return processTxnInDB(hdr, txn, null); } // entry point for FinalRequestProcessor.java @@ -1693,7 +1711,7 @@ public ProcessTxnResult processTxn(Request request) { return new ProcessTxnResult(); } synchronized (outstandingChanges) { - ProcessTxnResult rc = processTxnInDB(hdr, request.getTxn()); + ProcessTxnResult rc = processTxnInDB(hdr, request.getTxn(), request.getTxnDigest()); // request.hdr is set for write requests, which are the only ones // that add to outstandingChanges. @@ -1739,11 +1757,11 @@ private void processTxnForSessionEvents(Request request, TxnHeader hdr, Record t } } - private ProcessTxnResult processTxnInDB(TxnHeader hdr, Record txn) { + private ProcessTxnResult processTxnInDB(TxnHeader hdr, Record txn, TxnDigest digest) { if (hdr == null) { return new ProcessTxnResult(); } else { - return getZKDatabase().processTxn(hdr, txn); + return getZKDatabase().processTxn(hdr, txn, digest); } } diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/persistence/FileSnap.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/persistence/FileSnap.java index 9cc7118ee3d..fde577a05c5 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/persistence/FileSnap.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/persistence/FileSnap.java @@ -78,15 +78,24 @@ public long deserialize(DataTree dt, Map sessions) throws IOExcep return -1L; } File snap = null; + long snapZxid = -1; boolean foundValid = false; for (int i = 0, snapListSize = snapList.size(); i < snapListSize; i++) { snap = snapList.get(i); LOG.info("Reading snapshot {}", snap); + snapZxid = Util.getZxidFromName(snap.getName(), SNAPSHOT_FILE_PREFIX); try (CheckedInputStream snapIS = SnapStream.getInputStream(snap)) { InputArchive ia = BinaryInputArchive.getArchive(snapIS); deserialize(dt, sessions, ia); SnapStream.checkSealIntegrity(snapIS, ia); - if (dt.deserializeZxidDigest(ia)) { + + // Digest feature was added after the CRC to make it backward + // compatible, the older code can still read snapshots which + // includes digest. + // + // To check the intact, after adding digest we added another + // CRC check. + if (dt.deserializeZxidDigest(ia, snapZxid)) { SnapStream.checkSealIntegrity(snapIS, ia); } @@ -99,7 +108,7 @@ public long deserialize(DataTree dt, Map sessions) throws IOExcep if (!foundValid) { throw new IOException("Not able to find valid snapshots in " + snapDir); } - dt.lastProcessedZxid = Util.getZxidFromName(snap.getName(), SNAPSHOT_FILE_PREFIX); + dt.lastProcessedZxid = snapZxid; lastSnapshotInfo = new SnapshotInfo(dt.lastProcessedZxid, snap.lastModified() / 1000); // compare the digest if this is not a fuzzy snapshot, we want to compare diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/persistence/FileTxnLog.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/persistence/FileTxnLog.java index 4d35e833083..973e741969e 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/persistence/FileTxnLog.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/persistence/FileTxnLog.java @@ -44,7 +44,9 @@ import org.apache.jute.Record; import org.apache.zookeeper.server.ServerMetrics; import org.apache.zookeeper.server.ServerStats; +import org.apache.zookeeper.server.TxnLogEntry; import org.apache.zookeeper.server.util.SerializeUtils; +import org.apache.zookeeper.txn.TxnDigest; import org.apache.zookeeper.txn.TxnHeader; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -260,6 +262,11 @@ public synchronized void close() throws IOException { * returns true iff something appended, otw false */ public synchronized boolean append(TxnHeader hdr, Record txn) throws IOException { + return append(hdr, txn, null); + } + + @Override + public synchronized boolean append(TxnHeader hdr, Record txn, TxnDigest digest) throws IOException { if (hdr == null) { return false; } @@ -287,7 +294,7 @@ public synchronized boolean append(TxnHeader hdr, Record txn) throws IOException streamsToFlush.add(fos); } filePadding.padFile(fos.getChannel()); - byte[] buf = Util.marshallTxnEntry(hdr, txn); + byte[] buf = Util.marshallTxnEntry(hdr, txn, digest); if (buf == null || buf.length == 0) { throw new IOException("Faulty serialization for header " + "and txn"); } @@ -612,6 +619,7 @@ public static class FileTxnIterator implements TxnLog.TxnIterator { long zxid; TxnHeader hdr; Record record; + TxnDigest digest; File logFile; InputArchive ia; static final String CRC_ERROR = "CRC check failed"; @@ -768,8 +776,10 @@ public boolean next() throws IOException { if (crcValue != crc.getValue()) { throw new IOException(CRC_ERROR); } - hdr = new TxnHeader(); - record = SerializeUtils.deserializeTxn(bytes, hdr); + TxnLogEntry logEntry = SerializeUtils.deserializeTxn(bytes); + hdr = logEntry.getHeader(); + record = logEntry.getTxn(); + digest = logEntry.getDigest(); } catch (EOFException e) { LOG.debug("EOF exception", e); inputStream.close(); @@ -808,6 +818,10 @@ public Record getTxn() { return record; } + public TxnDigest getDigest() { + return digest; + } + /** * close the iterator * and release the resources. diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/persistence/FileTxnSnapLog.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/persistence/FileTxnSnapLog.java index 51f2dbb4ef5..661beb2bd40 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/persistence/FileTxnSnapLog.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/persistence/FileTxnSnapLog.java @@ -38,6 +38,7 @@ import org.apache.zookeeper.server.ZooTrace; import org.apache.zookeeper.server.persistence.TxnLog.TxnIterator; import org.apache.zookeeper.txn.CreateSessionTxn; +import org.apache.zookeeper.txn.TxnDigest; import org.apache.zookeeper.txn.TxnHeader; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -85,7 +86,9 @@ public class FileTxnSnapLog { * restored. */ public interface PlayBackListener { - void onTxnLoaded(TxnHeader hdr, Record rec); + + void onTxnLoaded(TxnHeader hdr, Record rec, TxnDigest digest); + } /** @@ -336,6 +339,7 @@ public long fastForwardFromEdits( } try { processTransaction(hdr, dt, sessions, itr.getTxn()); + dt.compareDigest(hdr, itr.getTxn(), itr.getDigest()); txnLoaded++; } catch (KeeperException.NoNodeException e) { throw new IOException("Failed to process transaction type: " @@ -344,7 +348,7 @@ public long fastForwardFromEdits( + e.getMessage(), e); } - listener.onTxnLoaded(hdr, itr.getTxn()); + listener.onTxnLoaded(hdr, itr.getTxn(), itr.getDigest()); if (!itr.next()) { break; } @@ -558,7 +562,7 @@ public File[] getSnapshotLogs(long zxid) { * @throws IOException */ public boolean append(Request si) throws IOException { - return txnLog.append(si.getHdr(), si.getTxn()); + return txnLog.append(si.getHdr(), si.getTxn(), si.getTxnDigest()); } /** diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/persistence/TxnLog.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/persistence/TxnLog.java index af2a684ab40..b5572087fb0 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/persistence/TxnLog.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/persistence/TxnLog.java @@ -22,6 +22,7 @@ import java.io.IOException; import org.apache.jute.Record; import org.apache.zookeeper.server.ServerStats; +import org.apache.zookeeper.txn.TxnDigest; import org.apache.zookeeper.txn.TxnHeader; /** @@ -51,6 +52,16 @@ public interface TxnLog extends Closeable { */ boolean append(TxnHeader hdr, Record r) throws IOException; + /** + * Append a request to the transaction log with a digset + * @param hdr the transaction header + * @param r the transaction itself + * @param digest transaction digest + * returns true iff something appended, otw false + * @throws IOException + */ + boolean append(TxnHeader hdr, Record r, TxnDigest digest) throws IOException; + /** * Start reading the transaction logs * from a given zxid @@ -96,6 +107,11 @@ public interface TxnLog extends Closeable { */ long getTxnLogSyncElapsedTime(); + /** + * close the transactions logs + */ + void close() throws IOException; + /** * Sets the total size of all log files */ @@ -124,12 +140,24 @@ interface TxnIterator extends Closeable { */ Record getTxn(); + /** + * @return the digest associated with the transaction. + */ + TxnDigest getDigest(); + /** * go to the next transaction record. * @throws IOException */ boolean next() throws IOException; + /** + * close files and release the + * resources + * @throws IOException + */ + void close() throws IOException; + /** * Get an estimated storage space used to store transaction records * that will return by this iterator diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/persistence/TxnLogToolkit.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/persistence/TxnLogToolkit.java index 984fb03e281..389caea6470 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/persistence/TxnLogToolkit.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/persistence/TxnLogToolkit.java @@ -49,6 +49,7 @@ import org.apache.jute.Record; import org.apache.zookeeper.server.ExitCode; import org.apache.zookeeper.server.Request; +import org.apache.zookeeper.server.TxnLogEntry; import org.apache.zookeeper.server.util.LogChopper; import org.apache.zookeeper.server.util.SerializeUtils; import org.apache.zookeeper.txn.CreateContainerTxn; @@ -283,8 +284,9 @@ private void printTxn(byte[] bytes) throws IOException { } private void printTxn(byte[] bytes, String prefix) throws IOException { - TxnHeader hdr = new TxnHeader(); - Record txn = SerializeUtils.deserializeTxn(bytes, hdr); + TxnLogEntry logEntry = SerializeUtils.deserializeTxn(bytes); + TxnHeader hdr = logEntry.getHeader(); + Record txn = logEntry.getTxn(); String txnStr = getFormattedTxnStr(txn); String txns = String.format( "%s session 0x%s cxid 0x%s zxid 0x%s %s %s", diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/persistence/Util.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/persistence/Util.java index 0a35293996b..a4d834453d8 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/persistence/Util.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/persistence/Util.java @@ -34,6 +34,7 @@ import org.apache.jute.InputArchive; import org.apache.jute.OutputArchive; import org.apache.jute.Record; +import org.apache.zookeeper.txn.TxnDigest; import org.apache.zookeeper.txn.TxnHeader; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -180,6 +181,11 @@ public static byte[] readTxnBytes(InputArchive ia) throws IOException { * @throws IOException */ public static byte[] marshallTxnEntry(TxnHeader hdr, Record txn) throws IOException { + return marshallTxnEntry(hdr, txn, null); + } + + public static byte[] marshallTxnEntry(TxnHeader hdr, Record txn, TxnDigest digest) + throws IOException { ByteArrayOutputStream baos = new ByteArrayOutputStream(); OutputArchive boa = BinaryOutputArchive.getArchive(baos); @@ -187,6 +193,9 @@ public static byte[] marshallTxnEntry(TxnHeader hdr, Record txn) throws IOExcept if (txn != null) { txn.serialize(boa, "txn"); } + if (digest != null) { + digest.serialize(boa, "digest"); + } return baos.toByteArray(); } 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 4e521874b6e..01f9f0d2738 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 @@ -345,6 +345,7 @@ public void run() { */ topPending.setHdr(request.getHdr()); topPending.setTxn(request.getTxn()); + topPending.setTxnDigest(request.getTxnDigest()); topPending.zxid = request.zxid; topPending.commitRecvTime = request.commitRecvTime; request = topPending; diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Follower.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Follower.java index 4d80d93358f..eb6742f5c6d 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Follower.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Follower.java @@ -27,11 +27,13 @@ import org.apache.zookeeper.common.Time; import org.apache.zookeeper.server.Request; import org.apache.zookeeper.server.ServerMetrics; +import org.apache.zookeeper.server.TxnLogEntry; import org.apache.zookeeper.server.quorum.QuorumPeer.QuorumServer; 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.txn.SetDataTxn; +import org.apache.zookeeper.txn.TxnDigest; import org.apache.zookeeper.txn.TxnHeader; /** @@ -160,8 +162,10 @@ protected void processPacket(QuorumPacket qp) throws Exception { break; case Leader.PROPOSAL: ServerMetrics.getMetrics().LEARNER_PROPOSAL_RECEIVED_COUNT.add(1); - TxnHeader hdr = new TxnHeader(); - Record txn = SerializeUtils.deserializeTxn(qp.getData(), hdr); + TxnLogEntry logEntry = SerializeUtils.deserializeTxn(qp.getData()); + TxnHeader hdr = logEntry.getHeader(); + Record txn = logEntry.getTxn(); + TxnDigest digest = logEntry.getDigest(); if (hdr.getZxid() != lastQueued + 1) { LOG.warn( "Got zxid 0x{} expected 0x{}", @@ -176,7 +180,7 @@ protected void processPacket(QuorumPacket qp) throws Exception { self.setLastSeenQuorumVerifier(qv, true); } - fzk.logRequest(hdr, txn); + fzk.logRequest(hdr, txn, digest); if (hdr != null) { /* * Request header is created only by the leader, so this is only set diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/FollowerZooKeeperServer.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/FollowerZooKeeperServer.java index 12e552f9d7d..8d371ae5790 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/FollowerZooKeeperServer.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/FollowerZooKeeperServer.java @@ -33,6 +33,7 @@ import org.apache.zookeeper.server.SyncRequestProcessor; import org.apache.zookeeper.server.ZKDatabase; import org.apache.zookeeper.server.persistence.FileTxnSnapLog; +import org.apache.zookeeper.txn.TxnDigest; import org.apache.zookeeper.txn.TxnHeader; import org.apache.zookeeper.util.ServiceUtils; import org.slf4j.Logger; @@ -78,8 +79,9 @@ protected void setupRequestProcessors() { LinkedBlockingQueue pendingTxns = new LinkedBlockingQueue(); - public void logRequest(TxnHeader hdr, Record txn) { + public void logRequest(TxnHeader hdr, Record txn, TxnDigest digest) { Request request = new Request(hdr.getClientId(), hdr.getCxid(), hdr.getType(), hdr, txn, hdr.getZxid()); + request.setTxnDigest(digest); if ((request.zxid & 0xffffffffL) != 0) { pendingTxns.add(request); } 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 f0eca518b9b..5371bea91d9 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 @@ -279,7 +279,7 @@ public boolean isQuorumSynced(QuorumVerifier qv) { private final List serverSockets = new LinkedList<>(); - Leader(QuorumPeer self, LeaderZooKeeperServer zk) throws IOException { + public Leader(QuorumPeer self, LeaderZooKeeperServer zk) throws IOException { this.self = self; this.proposalStats = new BufferStats(); diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/LeaderZooKeeperServer.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/LeaderZooKeeperServer.java index 27de4519177..8834b5fbdb1 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/LeaderZooKeeperServer.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/LeaderZooKeeperServer.java @@ -53,7 +53,7 @@ public class LeaderZooKeeperServer extends QuorumZooKeeperServer { /** * @throws IOException */ - LeaderZooKeeperServer(FileTxnSnapLog logFactory, QuorumPeer self, ZKDatabase zkDb) throws IOException { + public LeaderZooKeeperServer(FileTxnSnapLog logFactory, QuorumPeer self, ZKDatabase zkDb) throws IOException { super(logFactory, self.tickTime, self.minSessionTimeout, self.maxSessionTimeout, self.clientPortListenBacklog, zkDb, self); } 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 3d6ff1b79c1..ad9b0bba2b0 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 @@ -50,6 +50,7 @@ import org.apache.zookeeper.server.ExitCode; import org.apache.zookeeper.server.Request; import org.apache.zookeeper.server.ServerCnxn; +import org.apache.zookeeper.server.TxnLogEntry; import org.apache.zookeeper.server.ZooTrace; import org.apache.zookeeper.server.quorum.QuorumPeer.QuorumServer; import org.apache.zookeeper.server.quorum.flexible.QuorumVerifier; @@ -57,6 +58,7 @@ import org.apache.zookeeper.server.util.SerializeUtils; import org.apache.zookeeper.server.util.ZxidUtils; import org.apache.zookeeper.txn.SetDataTxn; +import org.apache.zookeeper.txn.TxnDigest; import org.apache.zookeeper.txn.TxnHeader; import org.apache.zookeeper.util.ServiceUtils; import org.slf4j.Logger; @@ -73,6 +75,7 @@ static class PacketInFlight { TxnHeader hdr; Record rec; + TxnDigest digest; } @@ -551,6 +554,7 @@ protected void syncWithLeader(long newLeaderZxid) throws Exception { //If we are not going to take the snapshot be sure the transactions are not applied in memory // but written out to the transaction log boolean writeToTxnLog = !snapshotNeeded; + TxnLogEntry logEntry; // we are now going to start getting transactions to apply followed by an UPTODATE outerLoop: while (self.isRunning()) { @@ -558,8 +562,10 @@ protected void syncWithLeader(long newLeaderZxid) throws Exception { switch (qp.getType()) { case Leader.PROPOSAL: PacketInFlight pif = new PacketInFlight(); - pif.hdr = new TxnHeader(); - pif.rec = SerializeUtils.deserializeTxn(qp.getData(), pif.hdr); + logEntry = SerializeUtils.deserializeTxn(qp.getData()); + pif.hdr = logEntry.getHeader(); + pif.rec = logEntry.getTxn(); + pif.digest = logEntry.getDigest(); if (pif.hdr.getZxid() != lastQueued + 1) { LOG.warn( "Got zxid 0x{} expected 0x{}", @@ -606,21 +612,26 @@ protected void syncWithLeader(long newLeaderZxid) throws Exception { case Leader.INFORM: case Leader.INFORMANDACTIVATE: PacketInFlight packet = new PacketInFlight(); - packet.hdr = new TxnHeader(); if (qp.getType() == Leader.INFORMANDACTIVATE) { ByteBuffer buffer = ByteBuffer.wrap(qp.getData()); long suggestedLeaderId = buffer.getLong(); byte[] remainingdata = new byte[buffer.remaining()]; buffer.get(remainingdata); - packet.rec = SerializeUtils.deserializeTxn(remainingdata, packet.hdr); + logEntry = SerializeUtils.deserializeTxn(remainingdata); + packet.hdr = logEntry.getHeader(); + packet.rec = logEntry.getTxn(); + packet.digest = logEntry.getDigest(); QuorumVerifier qv = self.configFromString(new String(((SetDataTxn) packet.rec).getData())); boolean majorChange = self.processReconfig(qv, suggestedLeaderId, qp.getZxid(), true); if (majorChange) { throw new Exception("changes proposed in reconfig"); } } else { - packet.rec = SerializeUtils.deserializeTxn(qp.getData(), packet.hdr); + logEntry = SerializeUtils.deserializeTxn(qp.getData()); + packet.rec = logEntry.getTxn(); + packet.hdr = logEntry.getHeader(); + packet.digest = logEntry.getDigest(); // Log warning message if txn comes out-of-order if (packet.hdr.getZxid() != lastQueued + 1) { LOG.warn( @@ -697,7 +708,7 @@ protected void syncWithLeader(long newLeaderZxid) throws Exception { if (zk instanceof FollowerZooKeeperServer) { FollowerZooKeeperServer fzk = (FollowerZooKeeperServer) zk; for (PacketInFlight p : packetsNotCommitted) { - fzk.logRequest(p.hdr, p.rec); + fzk.logRequest(p.hdr, p.rec, p.digest); } for (Long zxid : packetsCommitted) { fzk.commit(zxid); @@ -721,6 +732,7 @@ protected void syncWithLeader(long newLeaderZxid) throws Exception { Request request = new Request(null, p.hdr.getClientId(), p.hdr.getCxid(), p.hdr.getType(), null, null); request.setTxn(p.rec); request.setHdr(p.hdr); + request.setTxnDigest(p.digest); ozk.commitRequest(request); } } else { diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/LearnerHandler.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/LearnerHandler.java index e7d1d40144b..3bab398b41e 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/LearnerHandler.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/LearnerHandler.java @@ -50,9 +50,7 @@ import org.apache.zookeeper.server.quorum.QuorumPeer.LearnerType; import org.apache.zookeeper.server.quorum.auth.QuorumAuthServer; import org.apache.zookeeper.server.util.MessageTracker; -import org.apache.zookeeper.server.util.SerializeUtils; import org.apache.zookeeper.server.util.ZxidUtils; -import org.apache.zookeeper.txn.TxnHeader; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -397,13 +395,6 @@ public static String packetToString(QuorumPacket p) { break; case Leader.PROPOSAL: type = "PROPOSAL"; - TxnHeader hdr = new TxnHeader(); - try { - SerializeUtils.deserializeTxn(p.getData(), hdr); - // mess = "transaction: " + txn.toString(); - } catch (IOException e) { - LOG.warn("Unexpected exception", e); - } break; case Leader.REQUEST: type = "REQUEST"; diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Observer.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Observer.java index 02683e4c59b..47c58b938cd 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Observer.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/Observer.java @@ -24,10 +24,12 @@ import org.apache.zookeeper.server.ObserverBean; import org.apache.zookeeper.server.Request; import org.apache.zookeeper.server.ServerMetrics; +import org.apache.zookeeper.server.TxnLogEntry; import org.apache.zookeeper.server.quorum.QuorumPeer.QuorumServer; import org.apache.zookeeper.server.quorum.flexible.QuorumVerifier; import org.apache.zookeeper.server.util.SerializeUtils; import org.apache.zookeeper.txn.SetDataTxn; +import org.apache.zookeeper.txn.TxnDigest; import org.apache.zookeeper.txn.TxnHeader; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -168,6 +170,10 @@ private QuorumServer findLearnerMaster() { * @throws Exception */ protected void processPacket(QuorumPacket qp) throws Exception { + TxnLogEntry logEntry; + TxnHeader hdr; + TxnDigest digest; + Record txn; switch (qp.getType()) { case Leader.PING: ping(qp); @@ -189,26 +195,31 @@ protected void processPacket(QuorumPacket qp) throws Exception { break; case Leader.INFORM: ServerMetrics.getMetrics().LEARNER_COMMIT_RECEIVED_COUNT.add(1); - TxnHeader hdr = new TxnHeader(); - Record txn = SerializeUtils.deserializeTxn(qp.getData(), hdr); + logEntry = SerializeUtils.deserializeTxn(qp.getData()); + hdr = logEntry.getHeader(); + txn = logEntry.getTxn(); + digest = logEntry.getDigest(); Request request = new Request(hdr.getClientId(), hdr.getCxid(), hdr.getType(), hdr, txn, 0); request.logLatency(ServerMetrics.getMetrics().COMMIT_PROPAGATION_LATENCY); + request.setTxnDigest(digest); ObserverZooKeeperServer obs = (ObserverZooKeeperServer) zk; obs.commitRequest(request); break; case Leader.INFORMANDACTIVATE: - hdr = new TxnHeader(); - // get new designated leader from (current) leader's message ByteBuffer buffer = ByteBuffer.wrap(qp.getData()); long suggestedLeaderId = buffer.getLong(); byte[] remainingdata = new byte[buffer.remaining()]; buffer.get(remainingdata); - txn = SerializeUtils.deserializeTxn(remainingdata, hdr); + logEntry = SerializeUtils.deserializeTxn(remainingdata); + hdr = logEntry.getHeader(); + txn = logEntry.getTxn(); + digest = logEntry.getDigest(); QuorumVerifier qv = self.configFromString(new String(((SetDataTxn) txn).getData())); request = new Request(hdr.getClientId(), hdr.getCxid(), hdr.getType(), hdr, txn, 0); + request.setTxnDigest(digest); obs = (ObserverZooKeeperServer) zk; boolean majorChange = self.processReconfig(qv, suggestedLeaderId, qp.getZxid(), true); diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumBean.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumBean.java index 456bb575f35..aae4552f498 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumBean.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumBean.java @@ -19,6 +19,7 @@ package org.apache.zookeeper.server.quorum; import org.apache.zookeeper.jmx.ZKMBeanInfo; +import org.apache.zookeeper.server.ZooKeeperServer; public class QuorumBean implements QuorumMXBean, ZKMBeanInfo { @@ -81,4 +82,13 @@ public void setObserverElectionDelayMS(long delayMS) { Observer.setObserverElectionDelayMs(delayMS); } + @Override + public boolean getDigestEnabled() { + return ZooKeeperServer.isDigestEnabled(); + } + + @Override + public void disableDigest() { + ZooKeeperServer.setDigestEnabled(false); + } } diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumMXBean.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumMXBean.java index 33dac06f079..a7c450672d8 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumMXBean.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumMXBean.java @@ -73,4 +73,7 @@ public interface QuorumMXBean { */ void setObserverElectionDelayMS(long delayMS); + boolean getDigestEnabled(); + + void disableDigest(); } diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/util/AdHash.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/util/AdHash.java index e74cb05bb7e..f3de65ebb81 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/util/AdHash.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/util/AdHash.java @@ -29,7 +29,7 @@ public class AdHash { /* we use 64 bits so that we can be fast an efficient */ - private long hash; + private volatile long hash; /** * Add new digest to the hash value maintained in this class. @@ -53,13 +53,6 @@ public AdHash removeDigest(long digest) { return this; } - /** - * Return hex string of the hash value. - */ - public String toHexString() { - return Long.toHexString(hash); - } - /** * Return the long value of the hash. */ @@ -79,7 +72,10 @@ public int hashCode() { @Override public String toString() { - return toHexString(); + return Long.toHexString(hash); } + public void clear() { + hash = 0; + } } diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/util/LogChopper.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/util/LogChopper.java index e0e52a9d88d..ccc87334aa7 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/util/LogChopper.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/util/LogChopper.java @@ -33,6 +33,7 @@ import org.apache.jute.Record; import org.apache.yetus.audience.InterfaceAudience; import org.apache.zookeeper.server.ExitCode; +import org.apache.zookeeper.server.TxnLogEntry; import org.apache.zookeeper.server.persistence.FileHeader; import org.apache.zookeeper.server.persistence.FileTxnLog; import org.apache.zookeeper.txn.TxnHeader; @@ -111,8 +112,9 @@ public static boolean chop(InputStream is, OutputStream os, long zxid) throws IO if (crcValue != crc.getValue()) { throw new IOException("CRC doesn't match " + crcValue + " vs " + crc.getValue()); } - TxnHeader hdr = new TxnHeader(); - Record txn = SerializeUtils.deserializeTxn(bytes, hdr); + TxnLogEntry entry = SerializeUtils.deserializeTxn(bytes); + TxnHeader hdr = entry.getHeader(); + Record txn = entry.getTxn(); if (logStream.readByte("EOR") != 'B') { System.out.println("Last transaction was partial."); throw new EOFException("Last transaction was partial."); diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/util/SerializeUtils.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/util/SerializeUtils.java index 2454c43f91e..fcc5c8f4b70 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/util/SerializeUtils.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/util/SerializeUtils.java @@ -19,23 +19,22 @@ package org.apache.zookeeper.server.util; import java.io.ByteArrayInputStream; -import java.io.ByteArrayOutputStream; import java.io.EOFException; import java.io.IOException; import java.util.HashMap; import java.util.Map; import java.util.Map.Entry; import org.apache.jute.BinaryInputArchive; -import org.apache.jute.BinaryOutputArchive; import org.apache.jute.InputArchive; import org.apache.jute.OutputArchive; import org.apache.jute.Record; import org.apache.zookeeper.ZooDefs.OpCode; -import org.apache.zookeeper.common.IOUtils; import org.apache.zookeeper.server.DataTree; import org.apache.zookeeper.server.Request; +import org.apache.zookeeper.server.TxnLogEntry; import org.apache.zookeeper.server.ZooKeeperServer; import org.apache.zookeeper.server.ZooTrace; +import org.apache.zookeeper.server.persistence.Util; import org.apache.zookeeper.txn.CloseSessionTxn; import org.apache.zookeeper.txn.CreateContainerTxn; import org.apache.zookeeper.txn.CreateSessionTxn; @@ -47,6 +46,7 @@ import org.apache.zookeeper.txn.MultiTxn; import org.apache.zookeeper.txn.SetACLTxn; import org.apache.zookeeper.txn.SetDataTxn; +import org.apache.zookeeper.txn.TxnDigest; import org.apache.zookeeper.txn.TxnHeader; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -55,7 +55,8 @@ public class SerializeUtils { private static final Logger LOG = LoggerFactory.getLogger(SerializeUtils.class); - public static Record deserializeTxn(byte[] txnBytes, TxnHeader hdr) throws IOException { + public static TxnLogEntry deserializeTxn(byte[] txnBytes) throws IOException { + TxnHeader hdr = new TxnHeader(); final ByteArrayInputStream bais = new ByteArrayInputStream(txnBytes); InputArchive ia = BinaryInputArchive.getArchive(bais); @@ -128,7 +129,19 @@ public static Record deserializeTxn(byte[] txnBytes, TxnHeader hdr) throws IOExc } } } - return txn; + TxnDigest digest = null; + + if (ZooKeeperServer.isDigestEnabled()) { + digest = new TxnDigest(); + try { + digest.deserialize(ia, "digest"); + } catch (EOFException exception) { + // may not have digest in the txn + digest = null; + } + } + + return new TxnLogEntry(txn, hdr, digest); } public static void deserializeSnapshot(DataTree dt, InputArchive ia, Map sessions) throws IOException { @@ -162,19 +175,13 @@ public static byte[] serializeRequest(Request request) { if (request == null || request.getHdr() == null) { return null; } - ByteArrayOutputStream baos = new ByteArrayOutputStream(); - BinaryOutputArchive boa = BinaryOutputArchive.getArchive(baos); + byte[] data = new byte[32]; try { - request.getHdr().serialize(boa, "hdr"); - if (request.getTxn() != null) { - request.getTxn().serialize(boa, "txn"); - } + data = Util.marshallTxnEntry(request.getHdr(), request.getTxn(), request.getTxnDigest()); } catch (IOException e) { LOG.error("This really should be impossible", e); - } finally { - IOUtils.cleanup(LOG, baos); } - return baos.toByteArray(); + return data; } } diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/DataTreeTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/DataTreeTest.java index 11eb8365d9c..da9a12a3c52 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/DataTreeTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/DataTreeTest.java @@ -21,6 +21,8 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; @@ -37,6 +39,7 @@ import java.util.concurrent.atomic.AtomicBoolean; import org.apache.jute.BinaryInputArchive; import org.apache.jute.BinaryOutputArchive; +import org.apache.jute.InputArchive; import org.apache.jute.Record; import org.apache.zookeeper.KeeperException.NoNodeException; import org.apache.zookeeper.KeeperException.NodeExistsException; @@ -217,7 +220,7 @@ public void testDigestUpdatedWhenReplayCreateTxnForExistNode() { dt.processTxn(new TxnHeader(13, 1000, 1, 30, ZooDefs.OpCode.create), new CreateTxn("/foo", "".getBytes(), ZooDefs.Ids.OPEN_ACL_UNSAFE, false, 2)); // check the current digest value - assertEquals(dt.getTreeDigest(), dt.getLastProcessedZxidDigest().digest); + assertEquals(dt.getTreeDigest(), dt.getLastProcessedZxidDigest().getDigest()); } finally { ZooKeeperServer.setDigestEnabled(false); } @@ -480,6 +483,36 @@ public void testGetAllChildrenNumber() throws Exception { assertEquals(8, dt.getAllChildrenNumber("/")); } + @Test + public void testDeserializeZxidDigest() throws Exception { + try { + ZooKeeperServer.setDigestEnabled(true); + DataTree dt = new DataTree(); + dt.processTxn(new TxnHeader(13, 1000, 1, 30, ZooDefs.OpCode.create), + new CreateTxn("/foo", "".getBytes(), ZooDefs.Ids.OPEN_ACL_UNSAFE, false, 1), null); + + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + BinaryOutputArchive oa = BinaryOutputArchive.getArchive(baos); + dt.serializeZxidDigest(oa); + baos.flush(); + + DataTree.ZxidDigest zd = dt.getLastProcessedZxidDigest(); + assertNotNull(zd); + + // deserialize data tree + InputArchive ia = BinaryInputArchive.getArchive( + new ByteArrayInputStream(baos.toByteArray())); + dt.deserializeZxidDigest(ia, zd.getZxid()); + assertNotNull(dt.getDigestFromLoadedSnapshot()); + + ia = BinaryInputArchive.getArchive(new ByteArrayInputStream(baos.toByteArray())); + dt.deserializeZxidDigest(ia, zd.getZxid() + 1); + assertNull(dt.getDigestFromLoadedSnapshot()); + } finally { + ZooKeeperServer.setDigestEnabled(false); + } + } + @Test public void testDataTreeMetrics() throws Exception { ServerMetrics.getMetrics().resetAll(); diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/PrepRequestProcessorMetricsTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/PrepRequestProcessorMetricsTest.java index 7aa74d321de..77dc6c78a54 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/PrepRequestProcessorMetricsTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/PrepRequestProcessorMetricsTest.java @@ -76,6 +76,9 @@ public void setup() { DataNode node = new DataNode(new byte[1], null, mock(StatPersisted.class)); when(db.getNode(anyString())).thenReturn(node); + DataTree dataTree = mock(DataTree.class); + when(db.getDataTree()).thenReturn(dataTree); + Set ephemerals = new HashSet<>(); ephemerals.add("/crystalmountain"); ephemerals.add("/stevenspass"); @@ -158,7 +161,9 @@ public void testPrepRequestProcessorMetrics() throws Exception { assertEquals(1L, values.get("cnt_close_session_prep_time")); assertThat((long) values.get("max_close_session_prep_time"), greaterThanOrEqualTo(0L)); - assertEquals(5L, values.get("outstanding_changes_queued")); + // With digest feature, we have two more OUTSTANDING_CHANGES_QUEUED than w/o digest + // The expected should 5 in open source until we upstream the digest feature + assertEquals(7L, values.get("outstanding_changes_queued")); } private class SimpleWatcher implements Watcher { diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/PrepRequestProcessorTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/PrepRequestProcessorTest.java index 264601de076..48e58905f84 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/PrepRequestProcessorTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/PrepRequestProcessorTest.java @@ -22,6 +22,8 @@ import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; import java.io.ByteArrayOutputStream; import java.io.File; import java.io.IOException; @@ -36,6 +38,7 @@ import java.util.concurrent.TimeUnit; import org.apache.jute.BinaryOutputArchive; import org.apache.jute.Record; +import org.apache.zookeeper.CreateMode; import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.KeeperException.SessionExpiredException; import org.apache.zookeeper.KeeperException.SessionMovedException; @@ -45,9 +48,18 @@ import org.apache.zookeeper.ZooDefs.Ids; import org.apache.zookeeper.ZooDefs.OpCode; import org.apache.zookeeper.data.Id; +import org.apache.zookeeper.proto.CreateRequest; +import org.apache.zookeeper.proto.ReconfigRequest; import org.apache.zookeeper.proto.RequestHeader; import org.apache.zookeeper.proto.SetDataRequest; import org.apache.zookeeper.server.ZooKeeperServer.ChangeRecord; +import org.apache.zookeeper.server.persistence.FileTxnSnapLog; +import org.apache.zookeeper.server.quorum.Leader; +import org.apache.zookeeper.server.quorum.LeaderBeanTest; +import org.apache.zookeeper.server.quorum.LeaderZooKeeperServer; +import org.apache.zookeeper.server.quorum.QuorumPeer; +import org.apache.zookeeper.server.quorum.QuorumPeerConfig; +import org.apache.zookeeper.server.quorum.flexible.QuorumVerifier; import org.apache.zookeeper.test.ClientBase; import org.apache.zookeeper.txn.ErrorTxn; import org.junit.After; @@ -56,6 +68,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; + public class PrepRequestProcessorTest extends ClientBase { private static final Logger LOG = LoggerFactory.getLogger(PrepRequestProcessorTest.class); @@ -108,13 +121,21 @@ private Request createRequest(Record record, int opCode) throws IOException { } private Request createRequest(Record record, int opCode, long sessionId) throws IOException { + return createRequest(record, opCode, sessionId, false); + } + + private Request createRequest(Record record, int opCode, boolean admin) throws IOException { + return createRequest(record, opCode, 1L, admin); + } + + private Request createRequest(Record record, int opCode, long sessionId, boolean admin) throws IOException { // encoding ByteArrayOutputStream baos = new ByteArrayOutputStream(); BinaryOutputArchive boa = BinaryOutputArchive.getArchive(baos); record.serialize(boa, "request"); baos.close(); // Id - List ids = Arrays.asList(Ids.ANYONE_ID_UNSAFE); + List ids = Arrays.asList(admin ? new Id("super", "super user") : Ids.ANYONE_ID_UNSAFE); return new Request(null, sessionId, 0, opCode, ByteBuffer.wrap(baos.toByteArray()), ids); } @@ -123,7 +144,7 @@ private void process(List ops) throws Exception { processor = new PrepRequestProcessor(zks, new MyRequestProcessor()); Record record = new MultiOperationRecord(ops); - Request req = createRequest(record, OpCode.multi); + Request req = createRequest(record, OpCode.multi, false); processor.pRequest(req); assertTrue("request hasn't been processed in chain", pLatch.await(5, TimeUnit.SECONDS)); @@ -156,6 +177,43 @@ public void testMultiOutstandingChange() throws Exception { assertEquals("Record zxid wasn't set correctly", 2, cr.zxid); } + @Test + public void testReconfigWithAnotherOutstandingChange() throws Exception { + QuorumPeer qp = new QuorumPeer(); + QuorumVerifier quorumVerifierMock = mock(QuorumVerifier.class); + when(quorumVerifierMock.getAllMembers()).thenReturn(LeaderBeanTest.getMockedPeerViews(qp.getId())); + + qp.setQuorumVerifier(quorumVerifierMock, false); + FileTxnSnapLog snapLog = new FileTxnSnapLog(tmpDir, tmpDir); + LeaderZooKeeperServer lzks = new LeaderZooKeeperServer(snapLog, qp, new ZKDatabase(snapLog)); + qp.leader = new Leader(qp, lzks); + lzks.sessionTracker = new MySessionTracker(); + ZooKeeperServer.setDigestEnabled(true); + processor = new PrepRequestProcessor(lzks, new MyRequestProcessor()); + + Record record = new CreateRequest("/foo", "data".getBytes(), Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT.toFlag()); + pLatch = new CountDownLatch(1); + processor.pRequest(createRequest(record, OpCode.create, false)); + assertTrue("request hasn't been processed in chain", pLatch.await(5, TimeUnit.SECONDS)); + + boolean isReconfigEnabledPreviously = QuorumPeerConfig.isReconfigEnabled(); + boolean isStandaloneEnabledPreviously = QuorumPeerConfig.isStandaloneEnabled(); + QuorumPeerConfig.setReconfigEnabled(true); + QuorumPeerConfig.setStandaloneEnabled(false); + try { + String newMember = "server.0=localhost:" + PortAssignment.unique() + ":" + PortAssignment.unique() + ":participant"; + record = new ReconfigRequest(null, null, newMember, 0); + pLatch = new CountDownLatch(1); + processor.pRequest(createRequest(record, OpCode.reconfig, true)); + assertTrue("request hasn't been processed in chain", pLatch.await(5, TimeUnit.SECONDS)); + assertEquals(outcome.getHdr().getType(), OpCode.reconfig); // Verifies that there was no error. + } finally { + // reset the reconfig option + QuorumPeerConfig.setReconfigEnabled(isReconfigEnabledPreviously); + QuorumPeerConfig.setStandaloneEnabled(isStandaloneEnabledPreviously); + } + } + /** * ZOOKEEPER-2052: * This test checks that if a multi operation aborted, and during the multi there is side effect @@ -234,7 +292,7 @@ public void testInvalidPath() throws Exception { processor = new PrepRequestProcessor(zks, new MyRequestProcessor()); SetDataRequest record = new SetDataRequest("", new byte[0], -1); - Request req = createRequest(record, OpCode.setData); + Request req = createRequest(record, OpCode.setData, false); processor.pRequest(req); pLatch.await(); assertEquals(outcome.getHdr().getType(), OpCode.error); diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/TxnLogDigestTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/TxnLogDigestTest.java new file mode 100644 index 00000000000..ae914af5b5d --- /dev/null +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/TxnLogDigestTest.java @@ -0,0 +1,277 @@ +/** + * 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; + +import static org.hamcrest.Matchers.greaterThan; +import static org.junit.Assert.assertThat; +import java.io.File; +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import mockit.Invocation; +import mockit.Mock; +import mockit.MockUp; +import org.apache.jute.Record; +import org.apache.zookeeper.CreateMode; +import org.apache.zookeeper.Op; +import org.apache.zookeeper.ZooDefs; +import org.apache.zookeeper.ZooKeeper; +import org.apache.zookeeper.ZooKeeper.States; +import org.apache.zookeeper.server.metric.SimpleCounter; +import org.apache.zookeeper.server.persistence.FileTxnLog; +import org.apache.zookeeper.server.persistence.TxnLog.TxnIterator; +import org.apache.zookeeper.server.quorum.QuorumPeerMainTest; +import org.apache.zookeeper.test.ClientBase; +import org.apache.zookeeper.txn.TxnDigest; +import org.apache.zookeeper.txn.TxnHeader; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class TxnLogDigestTest extends ClientBase { + + private static final Logger LOG = + LoggerFactory.getLogger(TxnLogDigestTest.class); + + private ZooKeeper zk; + private ZooKeeperServer server; + + @Before + public void setUp() throws Exception { + super.setUp(); + server = serverFactory.getZooKeeperServer(); + zk = createClient(); + } + + @After + public void tearDown() throws Exception { + // server will be closed in super.tearDown + super.tearDown(); + + if (zk != null) { + zk.close(); + } + MockedFileTxnLog.reset(); + } + + @Override + public void setupCustomizedEnv() { + ZooKeeperServer.setDigestEnabled(true); + } + + @Override + public void cleanUpCustomizedEnv() { + ZooKeeperServer.setDigestEnabled(false); + } + + @BeforeClass + public static void applyMockUps() { + new MockedFileTxnLog(); + } + + /** + * Check that the digest stored in the txn matches the digest calculated + * from DataTree. + */ + @Test + public void digestFromTxnLogsMatchesTree() throws Exception { + // reset the mismatch metrics + SimpleCounter digestMistachesCount = (SimpleCounter) ServerMetrics.getMetrics().DIGEST_MISMATCHES_COUNT; + digestMistachesCount.reset(); + + // trigger some write ops + performOperations(createClient(), "/digestFromTxnLogsMatchesTree"); + + // make sure there is no digest mismatch + Assert.assertEquals(0, digestMistachesCount.get()); + + // verify that the digest is wrote to disk with txn + TxnDigest lastDigest = getLastTxnLogDigest(); + Assert.assertNotNull(lastDigest); + Assert.assertEquals(server.getZKDatabase().getDataTree().getTreeDigest(), + lastDigest.getTreeDigest()); + } + + /** + * Test the compatible when enable/disable digest: + * + * * check that txns which were written with digest can be read when + * digest is disabled + * * check that txns which were written without digest can be read + * when digest is enabled. + */ + @Test + public void checkTxnCompatibleWithAndWithoutDigest() throws Exception { + // 1. start server with digest disabled + restartServerWithDigestFlag(false); + + // trigger some write ops + Map expectedNodes = performOperations(createClient(), "/p1"); + + // reset the mismatch metrics + SimpleCounter digestMistachesCount = (SimpleCounter) ServerMetrics.getMetrics().DIGEST_MISMATCHES_COUNT; + digestMistachesCount.reset(); + + // 2. restart server with digest enabled + restartServerWithDigestFlag(true); + + // make sure the data wrote when digest was disabled can be + // successfully read + checkNodes(expectedNodes); + + Map expectedNodes1 = performOperations(createClient(), "/p2"); + + // make sure there is no digest mismatch + Assert.assertEquals(0, digestMistachesCount.get()); + + // 3. disable the digest again and make sure everything is fine + restartServerWithDigestFlag(false); + + checkNodes(expectedNodes); + checkNodes(expectedNodes1); + } + + /** + * Simulate the scenario where txn is missing, and make sure the + * digest code can catch this issue. + */ + @Test + public void testTxnMissing() throws Exception { + // updated MockedFileTxnLog to skip append txn on specific txn + MockedFileTxnLog.skipAppendZxid = 3; + + // trigger some write operations + performOperations(createClient(), "/testTxnMissing"); + + // restart server to load the corrupted txn file + SimpleCounter digestMistachesCount = (SimpleCounter) ServerMetrics.getMetrics().DIGEST_MISMATCHES_COUNT; + digestMistachesCount.reset(); + + restartServerWithDigestFlag(true); + + // check that digest mismatch is reported + assertThat("mismtach should be reported", digestMistachesCount.get(), greaterThan(0L)); + + // restart server with digest disabled + digestMistachesCount.reset(); + restartServerWithDigestFlag(false); + + // check that no digest mismatch is reported + Assert.assertEquals(0, digestMistachesCount.get()); + } + + private void restartServerWithDigestFlag(boolean digestEnabled) + throws Exception { + stopServer(); + QuorumPeerMainTest.waitForOne(zk, States.CONNECTING); + + ZooKeeperServer.setDigestEnabled(digestEnabled); + + startServer(); + QuorumPeerMainTest.waitForOne(zk, States.CONNECTED); + } + + private TxnDigest getLastTxnLogDigest() throws IOException { + TxnIterator itr = new FileTxnLog(new File(tmpDir, "version-2")).read(1); + TxnDigest lastDigest = null; + while (itr.next()) { + lastDigest = itr.getDigest(); + } + return lastDigest; + } + + public static void create(ZooKeeper client, String path, CreateMode mode) + throws Exception { + client.create(path, path.getBytes(), ZooDefs.Ids.OPEN_ACL_UNSAFE, mode); + } + + /** + * Helper method to trigger various write ops inside ZK. + */ + public static Map performOperations( + ZooKeeper client, String prefix) throws Exception { + Map nodes = new HashMap<>(); + + String path = prefix; + create(client, path, CreateMode.PERSISTENT); + nodes.put(path, path); + + path = prefix + "/child1"; + create(client, path, CreateMode.PERSISTENT); + nodes.put(path, path); + + path = prefix + "/child2"; + create(client, path, CreateMode.PERSISTENT); + client.delete(prefix + "/child2", -1); + + path = prefix + "/child1/leaf"; + create(client, path, CreateMode.PERSISTENT); + String updatedData = "updated data"; + client.setData(path, updatedData.getBytes(), -1); + nodes.put(path, updatedData); + + List subTxns = new ArrayList(); + for (int i = 0; i < 3; i++) { + path = prefix + "/m" + i; + subTxns.add(Op.create(path, path.getBytes(), + ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT)); + nodes.put(path, path); + } + client.multi(subTxns); + client.close(); + + return nodes; + } + + private void checkNodes(Map expectedNodes) throws Exception { + ZooKeeper client = createClient(); + try { + for (Map.Entry entry: expectedNodes.entrySet()) { + Assert.assertEquals(entry.getValue(), + new String(client.getData(entry.getKey(), false, null))); + } + } finally { + client.close(); + } + } + + public static final class MockedFileTxnLog extends MockUp { + static long skipAppendZxid = -1; + + @Mock + public synchronized boolean append(Invocation invocation, TxnHeader hdr, + Record txn, TxnDigest digest) throws IOException { + if (hdr != null && hdr.getZxid() == skipAppendZxid) { + LOG.info("skipping txn {}", skipAppendZxid); + return true; + } + return invocation.proceed(hdr, txn, digest); + } + + public static void reset() { + skipAppendZxid = -1; + } + }; +} diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/persistence/FileTxnSnapLogTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/persistence/FileTxnSnapLogTest.java index 1412a932b5b..34d57f415dd 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/persistence/FileTxnSnapLogTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/persistence/FileTxnSnapLogTest.java @@ -22,6 +22,7 @@ import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotEquals; import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; import java.io.File; @@ -39,10 +40,12 @@ import org.apache.zookeeper.server.DataNode; import org.apache.zookeeper.server.DataTree; import org.apache.zookeeper.server.Request; +import org.apache.zookeeper.server.ZooKeeperServer; import org.apache.zookeeper.test.ClientBase; import org.apache.zookeeper.test.TestUtils; import org.apache.zookeeper.txn.CreateTxn; import org.apache.zookeeper.txn.SetDataTxn; +import org.apache.zookeeper.txn.TxnDigest; import org.apache.zookeeper.txn.TxnHeader; import org.junit.After; import org.junit.Before; @@ -209,7 +212,7 @@ private void attemptAutoCreateDB( long zxid = fileTxnSnapLog.restore(new DataTree(), sessions, new FileTxnSnapLog.PlayBackListener() { @Override - public void onTxnLoaded(TxnHeader hdr, Record rec) { + public void onTxnLoaded(TxnHeader hdr, Record rec, TxnDigest digest) { // empty by default } }); @@ -353,4 +356,45 @@ public void testACLCreatedDuringFuzzySnapshotSync() throws IOException { assertEquals(ZooDefs.Ids.CREATOR_ALL_ACL, followerDataTree.getACL(a1)); } + @Test + public void testEmptySnapshotSerialization() throws IOException { + File dataDir = ClientBase.createEmptyTestDir(); + FileTxnSnapLog snaplog = new FileTxnSnapLog(dataDir, dataDir); + DataTree dataTree = new DataTree(); + ConcurrentHashMap sessions = new ConcurrentHashMap<>(); + + ZooKeeperServer.setDigestEnabled(true); + snaplog.save(dataTree, sessions, true); + snaplog.restore(dataTree, sessions, (hdr, rec, digest) -> { }); + + assertNull(dataTree.getDigestFromLoadedSnapshot()); + } + + @Test + public void testSnapshotSerializationCompatibility() throws IOException { + testSnapshotSerializationCompatibility(true, false); + testSnapshotSerializationCompatibility(false, false); + testSnapshotSerializationCompatibility(true, true); + testSnapshotSerializationCompatibility(false, true); + } + + void testSnapshotSerializationCompatibility(Boolean digestEnabled, Boolean snappyEnabled) throws IOException { + File dataDir = ClientBase.createEmptyTestDir(); + FileTxnSnapLog snaplog = new FileTxnSnapLog(dataDir, dataDir); + DataTree dataTree = new DataTree(); + ConcurrentHashMap sessions = new ConcurrentHashMap<>(); + SnapStream.setStreamMode(snappyEnabled ? SnapStream.StreamMode.SNAPPY : SnapStream.StreamMode.DEFAULT_MODE); + + ZooKeeperServer.setDigestEnabled(digestEnabled); + TxnHeader txnHeader = new TxnHeader(1, 1, 1, 1 + 1, ZooDefs.OpCode.create); + CreateTxn txn = new CreateTxn("/" + 1, "data".getBytes(), null, false, 1); + Request request = new Request(1, 1, 1, txnHeader, txn, 1); + dataTree.processTxn(request.getHdr(), request.getTxn()); + snaplog.save(dataTree, sessions, true); + + int expectedNodeCount = dataTree.getNodeCount(); + ZooKeeperServer.setDigestEnabled(!digestEnabled); + snaplog.restore(dataTree, sessions, (hdr, rec, digest) -> { }); + assertEquals(expectedNodeCount, dataTree.getNodeCount()); + } } diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/FuzzySnapshotRelatedTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/FuzzySnapshotRelatedTest.java index 21a9d27347a..fd3374ded3b 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/FuzzySnapshotRelatedTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/FuzzySnapshotRelatedTest.java @@ -19,27 +19,34 @@ package org.apache.zookeeper.server.quorum; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; import java.io.IOException; import java.util.Arrays; import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import javax.security.sasl.SaslException; import org.apache.jute.OutputArchive; +import org.apache.zookeeper.AsyncCallback.MultiCallback; import org.apache.zookeeper.CreateMode; import org.apache.zookeeper.KeeperException.NoNodeException; import org.apache.zookeeper.KeeperException.NodeExistsException; import org.apache.zookeeper.Op; +import org.apache.zookeeper.OpResult; 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.data.ACL; import org.apache.zookeeper.data.Stat; +import org.apache.zookeeper.metrics.MetricsUtils; import org.apache.zookeeper.server.DataNode; import org.apache.zookeeper.server.DataTree; +import org.apache.zookeeper.server.ServerMetrics; import org.apache.zookeeper.server.ZKDatabase; import org.apache.zookeeper.server.ZooKeeperServer; import org.apache.zookeeper.server.persistence.FileTxnSnapLog; @@ -65,6 +72,8 @@ public class FuzzySnapshotRelatedTest extends QuorumPeerTestBase { @Before public void setup() throws Exception { + ZooKeeperServer.setDigestEnabled(true); + LOG.info("Start up a 3 server quorum"); final int ENSEMBLE_SERVERS = 3; clientPorts = new int[ENSEMBLE_SERVERS]; @@ -108,6 +117,8 @@ public TestQPMain getTestQPMain() { @After public void tearDown() throws Exception { + ZooKeeperServer.setDigestEnabled(false); + if (mt != null) { for (MainThread t : mt) { t.shutdown(); @@ -233,6 +244,82 @@ public void testPZxidUpdatedWhenLoadingSnapshot() throws Exception { compareStat(parent, leaderId, followerA); } + @Test + public void testMultiOpDigestConsistentDuringSnapshot() throws Exception { + ServerMetrics.getMetrics().resetAll(); + + LOG.info("Create some txns"); + final String path = "/testMultiOpDigestConsistentDuringSnapshot"; + createEmptyNode(zk[followerA], path, CreateMode.PERSISTENT); + + CustomDataTree dt = + (CustomDataTree) mt[followerA].main.quorumPeer.getZkDb().getDataTree(); + final CountDownLatch setDataLatch = new CountDownLatch(1); + final CountDownLatch continueSetDataLatch = new CountDownLatch(1); + final ZooKeeper followerZk = zk[followerA]; + dt.setDigestSerializeListener(new DigestSerializeListener() { + @Override + public void process() { + LOG.info("Trigger a multi op in async"); + followerZk.multi(Arrays.asList( + Op.create("/multi0", "/multi0".getBytes(), + Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT), + Op.setData(path, "new data".getBytes(), -1) + ), new MultiCallback() { + @Override + public void processResult(int rc, String path, Object ctx, + List opResults) {} + }, null); + + LOG.info("Wait for the signal to continue"); + try { + setDataLatch.await(3, TimeUnit.SECONDS); + } catch (Exception e) { + LOG.error("Error while waiting for set data txn, {}", e); + } + } + + @Override + public void finished() { + LOG.info("Finished writing digest out, continue"); + continueSetDataLatch.countDown(); + } + }); + + dt.setDataListener(new SetDataTxnListener() { + @Override + public void process() { + setDataLatch.countDown(); + try { + continueSetDataLatch.await(3, TimeUnit.SECONDS); + } catch (Exception e) { + LOG.error("Error while waiting for continue signal, {}", e); + } + } + }); + + LOG.info("Trigger a snapshot"); + ZooKeeperServer zkServer = mt[followerA].main.quorumPeer.getActiveServer(); + zkServer.takeSnapshot(true); + checkNoMismatchReported(); + + LOG.info("Restart the server to load the snapshot again"); + mt[followerA].shutdown(); + QuorumPeerMainTest.waitForOne(zk[followerA], States.CONNECTING); + mt[followerA].start(); + QuorumPeerMainTest.waitForOne(zk[followerA], States.CONNECTED); + + LOG.info("Make sure there is nothing caught in the digest mismatch"); + checkNoMismatchReported(); + + } + + private void checkNoMismatchReported() { + long mismatch = (long) MetricsUtils.currentServerMetrics().get("digest_mismatches_count"); + + assertFalse("The mismatch count should be zero but is: " + mismatch, mismatch > 0); + } + private void addSerializeListener(int sid, String parent, String child) { final ZooKeeper zkClient = zk[sid]; CustomDataTree dt = (CustomDataTree) mt[sid].main.quorumPeer.getZkDb().getDataTree(); @@ -323,10 +410,22 @@ interface NodeCreateListener { } + interface DigestSerializeListener { + void process(); + + void finished(); + } + + interface SetDataTxnListener { + void process(); + } + static class CustomDataTree extends DataTree { Map nodeCreateListeners = new HashMap(); Map listeners = new HashMap(); + DigestSerializeListener digestListener; + SetDataTxnListener setListener; @Override public void serializeNodeData(OutputArchive oa, String path, DataNode node) throws IOException { @@ -362,6 +461,34 @@ public void addNodeCreateListener(String path, NodeCreateListener listener) { nodeCreateListeners.put(path, listener); } + public void setDigestSerializeListener(DigestSerializeListener listener) { + this.digestListener = listener; + } + + public void setDataListener(SetDataTxnListener listener) { + this.setListener = listener; + } + + @Override + public boolean serializeZxidDigest(OutputArchive oa) throws IOException { + if (digestListener != null) { + digestListener.process(); + } + boolean result = super.serializeZxidDigest(oa); + if (digestListener != null) { + digestListener.finished(); + } + return result; + } + + public Stat setData(String path, byte data[], int version, long zxid, + long time) throws NoNodeException { + if (setListener != null) { + setListener.process(); + } + + return super.setData(path, data, version, zxid, time); + } } interface NodeSerializeListener { diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/LeaderBeanTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/LeaderBeanTest.java index 7ce583d58cf..0c6938be401 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/LeaderBeanTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/LeaderBeanTest.java @@ -60,19 +60,23 @@ public class LeaderBeanTest { private QuorumPeer qp; private QuorumVerifier quorumVerifierMock; - @Before - public void setUp() throws IOException, X509Exception { - qp = new QuorumPeer(); - long myId = qp.getId(); - + public static Map getMockedPeerViews(long myId) { int clientPort = PortAssignment.unique(); Map peersView = new HashMap(); InetAddress clientIP = InetAddress.getLoopbackAddress(); - peersView.put(Long.valueOf(myId), new QuorumServer(myId, new InetSocketAddress(clientIP, PortAssignment.unique()), new InetSocketAddress(clientIP, PortAssignment.unique()), new InetSocketAddress(clientIP, clientPort), LearnerType.PARTICIPANT)); + peersView.put(Long.valueOf(myId), + new QuorumServer(myId, new InetSocketAddress(clientIP, PortAssignment.unique()), + new InetSocketAddress(clientIP, PortAssignment.unique()), + new InetSocketAddress(clientIP, clientPort), LearnerType.PARTICIPANT)); + return peersView; + } + @Before + public void setUp() throws IOException, X509Exception { + qp = new QuorumPeer(); quorumVerifierMock = mock(QuorumVerifier.class); - when(quorumVerifierMock.getAllMembers()).thenReturn(peersView); + when(quorumVerifierMock.getAllMembers()).thenReturn(getMockedPeerViews(qp.getId())); qp.setQuorumVerifier(quorumVerifierMock, false); File tmpDir = ClientBase.createEmptyTestDir(); diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/QuorumDigestTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/QuorumDigestTest.java new file mode 100644 index 00000000000..691b45513a7 --- /dev/null +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/QuorumDigestTest.java @@ -0,0 +1,263 @@ +/** + * 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.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.concurrent.atomic.AtomicBoolean; +import mockit.Invocation; +import mockit.Mock; +import mockit.MockUp; +import org.apache.jute.Record; +import org.apache.zookeeper.AsyncCallback.StringCallback; +import org.apache.zookeeper.CreateMode; +import org.apache.zookeeper.ZooDefs; +import org.apache.zookeeper.ZooKeeper; +import org.apache.zookeeper.ZooKeeper.States; +import org.apache.zookeeper.data.Stat; +import org.apache.zookeeper.server.DataTree; +import org.apache.zookeeper.server.DataTree.ProcessTxnResult; +import org.apache.zookeeper.server.ServerMetrics; +import org.apache.zookeeper.server.TxnLogDigestTest; +import org.apache.zookeeper.server.ZooKeeperServer; +import org.apache.zookeeper.server.metric.SimpleCounter; +import org.apache.zookeeper.txn.TxnDigest; +import org.apache.zookeeper.txn.TxnHeader; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class QuorumDigestTest extends QuorumPeerTestBase { + + private static final Logger LOG = + LoggerFactory.getLogger(QuorumDigestTest.class); + + private Servers servers; + private String forceSnapSyncValue; + + @BeforeClass + public static void applyMockUps() { + new DataTreeMock(); + } + + @Before + public void setup() throws Exception { + forceSnapSyncValue = System.getProperty(LearnerHandler.FORCE_SNAP_SYNC); + ZooKeeperServer.setDigestEnabled(true); + ((SimpleCounter) ServerMetrics.getMetrics().DIGEST_MISMATCHES_COUNT).reset(); + servers = LaunchServers(3, 1, null); + } + + @After + public void tearDown() throws Exception { + if (servers != null) { + servers.shutDownAllServers(); + } + ZooKeeperServer.setDigestEnabled(false); + System.clearProperty(LearnerHandler.FORCE_SNAP_SYNC); + DataTreeMock.reset(); + } + + /** + * Check positive case without digest mismatch during diff sync. + */ + @Test + public void testDigestMatchesDuringDiffSync() throws Exception { + triggerSync(false); + } + + /** + * Check positive case without digest mismatch during snap sync. + */ + @Test + public void testDigestMatchesDuringSnapSync() throws Exception { + triggerSync(true); + + // have some extra txns + int leader = servers.findLeader(); + TxnLogDigestTest.performOperations(servers.zk[leader], + "/testDigestMatchesDuringSnapSync"); + Assert.assertEquals(0L, getMismatchDigestCount()); + } + + @Test + public void testDigestMatchesWithAsyncRequests() throws Exception { + + int leader = servers.findLeader(); + + final ZooKeeper client = servers.zk[leader]; + final AtomicBoolean stopped = new AtomicBoolean(true); + final String prefix = "/testDigestMatchesWithAsyncRequests"; + + // start a thread to send requests asynchronously, + Thread createTrafficThread = new Thread () { + @Override + public void run() { + int i = 0; + while (!stopped.get()) { + String path = prefix + "-" + i; + client.create(path, path.getBytes(), ZooDefs.Ids.OPEN_ACL_UNSAFE, + CreateMode.PERSISTENT, new StringCallback() { + @Override + public void processResult(int rc, String path, + Object ctx, String name) { + // ignore the result + } + }, null); + try { + Thread.sleep(10); + } catch (InterruptedException e) { /* ignore */ } + } + } + }; + createTrafficThread.start(); + + // shutdown a follower and observer + List targets = Arrays.asList( + servers.findAnyFollower(), servers.findAnyObserver()); + stopServers(targets); + + // start the follower and observer to have a diff sync + startServers(targets); + + // make sure there is no digest mismatch + Assert.assertEquals(0L, getMismatchDigestCount()); + + // stop the leader + targets = Arrays.asList(leader); + stopServers(targets); + startServers(targets); + + // make sure there is no digest mismatch + Assert.assertEquals(0L, getMismatchDigestCount()); + + stopped.set(true); + } + + /** + * Check negative case by injecting txn miss during syncing. + */ + @Test + public void testDigestMismatchesWhenTxnLost() throws Exception { + // make sure there is no mismatch after all servers start up + Assert.assertEquals(0L, getMismatchDigestCount()); + + // shutdown a follower and observer + List targets = Arrays.asList( + servers.findAnyFollower(), servers.findAnyObserver()); + stopServers(targets); + + int leader = servers.findLeader(); + triggerOps(leader, "/p1"); + + Assert.assertEquals(0L, getMismatchDigestCount()); + + DataTreeMock.skipTxnZxid = "100000006"; + + // start the follower and observer to have a diff sync + startServers(targets); + + long misMatchCount = getMismatchDigestCount(); + Assert.assertNotEquals(0L, misMatchCount); + + triggerOps(leader, "/p2"); + Assert.assertNotEquals(misMatchCount, getMismatchDigestCount()); + } + + private void stopServers(List sids) throws InterruptedException { + for (int sid : sids) { + if (sid != -1) { + servers.mt[sid].shutdown(); + waitForOne(servers.zk[sid], States.CONNECTING); + } + } + } + + private void startServers(List sids) throws InterruptedException { + for (int sid : sids) { + servers.mt[sid].start(); + waitForOne(servers.zk[sid], States.CONNECTED); + } + } + + private void triggerOps(int sid, String prefix) throws Exception { + TxnLogDigestTest.performOperations(servers.zk[sid], prefix); + servers.restartClient(sid, null); + waitForOne(servers.zk[sid], States.CONNECTED); + } + + private void triggerSync(boolean snapSync) throws Exception { + if (snapSync) { + System.setProperty(LearnerHandler.FORCE_SNAP_SYNC, "true"); + } + + // make sure there is no mismatch after all servers start up + Assert.assertEquals(0L, getMismatchDigestCount()); + + int leader = servers.findLeader(); + triggerOps(leader, "/p1"); + + Assert.assertEquals(0L, getMismatchDigestCount()); + + // shutdown a follower and observer + List targets = Arrays.asList( + servers.findAnyFollower(), servers.findAnyObserver()); + stopServers(targets); + + // do some extra writes + triggerOps(leader, "/p2"); + + // start the follower and observer to have a diff sync + startServers(targets); + + Assert.assertEquals(0L, getMismatchDigestCount()); + } + + public static long getMismatchDigestCount() { + return ((SimpleCounter) ServerMetrics.getMetrics().DIGEST_MISMATCHES_COUNT).get(); + } + + public static final class DataTreeMock extends MockUp { + + static String skipTxnZxid = ""; + + @Mock + public ProcessTxnResult processTxn(Invocation invocation, + TxnHeader header, Record txn, TxnDigest digest) { + if (header != null && Long.toHexString(header.getZxid()).equals(skipTxnZxid)) { + LOG.info("skip process txn {}", header.getZxid()); + ProcessTxnResult rc = new ProcessTxnResult(); + rc.path = ""; + rc.stat = new Stat(); + rc.multiResult = new ArrayList(); + return rc; + } + return invocation.proceed(header, txn, digest); + } + + public static void reset() { + skipTxnZxid = ""; + } + } +} diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/QuorumPeerTestBase.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/QuorumPeerTestBase.java index d2e582211eb..aa75218b03b 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/QuorumPeerTestBase.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/QuorumPeerTestBase.java @@ -438,34 +438,65 @@ public void restartClient(int clientIndex, Watcher watcher) throws IOException, public int findLeader() { for (int i = 0; i < mt.length; i++) { if (mt[i].main.quorumPeer.leader != null) { + LOG.info("Leader is {}", i); return i; } } + LOG.info("Cannot find Leader"); return -1; } + public int findAnyFollower() { + for (int i = 0; i < mt.length; i++) { + if (mt[i].main.quorumPeer.follower != null) { + LOG.info("Follower is {}", i); + return i; + } + } + LOG.info("Cannot find any follower"); + return -1; + } + + public int findAnyObserver() { + for (int i = 0; i < mt.length; i++) { + if (mt[i].main.quorumPeer.observer != null) { + LOG.info("Observer is {}", i); + return i; + } + } + LOG.info("Cannot find any observer"); + return -1; + } } protected Servers LaunchServers(int numServers) throws IOException, InterruptedException { return LaunchServers(numServers, null); } + protected Servers LaunchServers(int numServers, Integer tickTime) throws IOException, InterruptedException { + return LaunchServers(numServers, 0, tickTime); + } + /** * This is a helper function for launching a set of servers * - * @param numServers the number of servers + * @param numServers the number of participant servers + * @param numObserver the number of observer servers * @param tickTime A ticktime to pass to MainThread * @return * @throws IOException * @throws InterruptedException */ - protected Servers LaunchServers(int numServers, Integer tickTime) throws IOException, InterruptedException { - int SERVER_COUNT = numServers; + protected Servers LaunchServers(int numServers, int numObservers, Integer tickTime) throws IOException, InterruptedException { + int SERVER_COUNT = numServers + numObservers; QuorumPeerMainTest.Servers svrs = new QuorumPeerMainTest.Servers(); svrs.clientPorts = new int[SERVER_COUNT]; StringBuilder sb = new StringBuilder(); for (int i = 0; i < SERVER_COUNT; i++) { svrs.clientPorts[i] = PortAssignment.unique(); - sb.append("server." + i + "=127.0.0.1:" + PortAssignment.unique() + ":" + PortAssignment.unique() + ";" + svrs.clientPorts[i] + "\n"); + String role = i < numServers ? "participant" : "observer"; + sb.append(String.format("server.%d=127.0.0.1:%d:%d:%s;127.0.0.1:%d\n", + i, PortAssignment.unique(), PortAssignment.unique(), role, + svrs.clientPorts[i])); } String quorumCfgSection = sb.toString(); diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/Zab1_0Test.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/Zab1_0Test.java index 5302416edb0..c106f4cc7fc 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/Zab1_0Test.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/Zab1_0Test.java @@ -360,8 +360,8 @@ public void testPopulatedLeaderConversation(PopulatedLeaderConversation conversa for (int i = 1; i <= ops; i++) { zxid = ZxidUtils.makeZxid(1, i); String path = "/foo-" + i; - zkDb.processTxn(new TxnHeader(13, 1000 + i, zxid, 30 - + i, ZooDefs.OpCode.create), new CreateTxn(path, "fpjwasalsohere".getBytes(), ZooDefs.Ids.OPEN_ACL_UNSAFE, false, 1)); + zkDb.processTxn(new TxnHeader(13, 1000 + i, zxid, 30 + i, ZooDefs.OpCode.create), + new CreateTxn(path, "fpjwasalsohere".getBytes(), ZooDefs.Ids.OPEN_ACL_UNSAFE, false, 1), null); Stat stat = new Stat(); assertEquals("fpjwasalsohere", new String(zkDb.getData(path, stat, null))); } @@ -585,7 +585,7 @@ public void converseWithFollower(InputArchive ia, OutputArchive oa, Follower f) // Setup a database with a single /foo node ZKDatabase zkDb = new ZKDatabase(new FileTxnSnapLog(tmpDir, tmpDir)); final long firstZxid = ZxidUtils.makeZxid(1, 1); - zkDb.processTxn(new TxnHeader(13, 1313, firstZxid, 33, ZooDefs.OpCode.create), new CreateTxn("/foo", "data1".getBytes(), ZooDefs.Ids.OPEN_ACL_UNSAFE, false, 1)); + zkDb.processTxn(new TxnHeader(13, 1313, firstZxid, 33, ZooDefs.OpCode.create), new CreateTxn("/foo", "data1".getBytes(), ZooDefs.Ids.OPEN_ACL_UNSAFE, false, 1), null); Stat stat = new Stat(); assertEquals("data1", new String(zkDb.getData("/foo", stat, null))); @@ -719,7 +719,7 @@ public void converseWithFollower(InputArchive ia, OutputArchive oa, Follower f) // Setup a database with a single /foo node ZKDatabase zkDb = new ZKDatabase(new FileTxnSnapLog(tmpDir, tmpDir)); final long firstZxid = ZxidUtils.makeZxid(1, 1); - zkDb.processTxn(new TxnHeader(13, 1313, firstZxid, 33, ZooDefs.OpCode.create), new CreateTxn("/foo", "data1".getBytes(), ZooDefs.Ids.OPEN_ACL_UNSAFE, false, 1)); + zkDb.processTxn(new TxnHeader(13, 1313, firstZxid, 33, ZooDefs.OpCode.create), new CreateTxn("/foo", "data1".getBytes(), ZooDefs.Ids.OPEN_ACL_UNSAFE, false, 1), null); Stat stat = new Stat(); assertEquals("data1", new String(zkDb.getData("/foo", stat, null))); @@ -949,8 +949,8 @@ public void converseWithObserver(InputArchive ia, OutputArchive oa, Observer o) ZKDatabase zkDb = new ZKDatabase(new FileTxnSnapLog(tmpDir, tmpDir)); final long foo1Zxid = ZxidUtils.makeZxid(1, 1); final long foo2Zxid = ZxidUtils.makeZxid(1, 2); - zkDb.processTxn(new TxnHeader(13, 1313, foo1Zxid, 33, ZooDefs.OpCode.create), new CreateTxn("/foo1", "data1".getBytes(), ZooDefs.Ids.OPEN_ACL_UNSAFE, false, 1)); - zkDb.processTxn(new TxnHeader(13, 1313, foo2Zxid, 33, ZooDefs.OpCode.create), new CreateTxn("/foo2", "data1".getBytes(), ZooDefs.Ids.OPEN_ACL_UNSAFE, false, 1)); + zkDb.processTxn(new TxnHeader(13, 1313, foo1Zxid, 33, ZooDefs.OpCode.create), new CreateTxn("/foo1", "data1".getBytes(), ZooDefs.Ids.OPEN_ACL_UNSAFE, false, 1), null); + zkDb.processTxn(new TxnHeader(13, 1313, foo2Zxid, 33, ZooDefs.OpCode.create), new CreateTxn("/foo2", "data1".getBytes(), ZooDefs.Ids.OPEN_ACL_UNSAFE, false, 1), null); Stat stat = new Stat(); assertEquals("data1", new String(zkDb.getData("/foo1", stat, null))); assertEquals("data1", new String(zkDb.getData("/foo2", stat, null))); diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/util/AdHashTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/util/AdHashTest.java index e88301199e8..671d484647b 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/util/AdHashTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/util/AdHashTest.java @@ -87,7 +87,7 @@ public void testAdHash() throws Exception { removeListOfDigests(hashall, bucket3); removeListOfDigests(hashall, bucket2); removeListOfDigests(hashall, bucket1); - assertEquals("empty hashall's digest should be 0", hashall.toHexString(), "0"); + assertEquals("empty hashall's digest should be 0", hashall.toString(), "0"); AdHash hash45 = new AdHash(); addListOfDigests(hash45, bucket4); diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/test/GetProposalFromTxnTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/test/GetProposalFromTxnTest.java index 41109916f08..207caf5b53c 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/test/GetProposalFromTxnTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/test/GetProposalFromTxnTest.java @@ -34,6 +34,7 @@ import org.apache.zookeeper.data.Stat; import org.apache.zookeeper.server.ServerCnxnFactory; import org.apache.zookeeper.server.SyncRequestProcessor; +import org.apache.zookeeper.server.TxnLogEntry; import org.apache.zookeeper.server.ZKDatabase; import org.apache.zookeeper.server.ZooKeeperServer; import org.apache.zookeeper.server.quorum.Leader.Proposal; @@ -105,8 +106,10 @@ public void testGetProposalFromTxn() throws Exception { // Get zxid of create requests while (itr.hasNext()) { Proposal proposal = itr.next(); - TxnHeader hdr = new TxnHeader(); - Record rec = SerializeUtils.deserializeTxn(proposal.packet.getData(), hdr); + TxnLogEntry logEntry = SerializeUtils.deserializeTxn( + proposal.packet.getData()); + TxnHeader hdr = logEntry.getHeader(); + Record rec = logEntry.getTxn(); if (hdr.getType() == OpCode.create) { retrievedZxids.add(hdr.getZxid()); createCount++; diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/test/JMXEnv.java b/zookeeper-server/src/test/java/org/apache/zookeeper/test/JMXEnv.java index 6fbb6200641..76845edd163 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/test/JMXEnv.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/test/JMXEnv.java @@ -55,6 +55,7 @@ public static void setUp() throws IOException { cs.start(); JMXServiceURL addr = cs.getAddress(); + LOG.info("connecting to addr {}", addr); cc = JMXConnectorFactory.connect(addr); } From 59f7bbf478151ad5bbce6610bff78dc3d7671eb6 Mon Sep 17 00:00:00 2001 From: Enrico Olivelli Date: Mon, 6 Jan 2020 11:51:49 +0100 Subject: [PATCH 007/118] ZOOKEEPER-3668: Clean up release package for 3.6.0 - add LICENSE files for Snappy and Metrics core - update Copyright year - copy airlift reference to the NOTICE file of the binary packages - copy the Java 8 disclaimer from branch-3.5 Author: Enrico Olivelli Reviewers: Andor Molnar Closes #1196 from eolivelli/fix/license-stuff --- NOTICE.txt | 2 +- README.md | 5 + .../src/main/resources/NOTICE.txt | 40 +++- .../lib/metrics-core-5.2.5.jar_LICENSE.txt | 202 ++++++++++++++++++ .../lib/snappy-java-1.1.7.jar_LICENSE.txt | 202 ++++++++++++++++++ 5 files changed, 449 insertions(+), 2 deletions(-) create mode 100644 zookeeper-server/src/main/resources/lib/metrics-core-5.2.5.jar_LICENSE.txt create mode 100644 zookeeper-server/src/main/resources/lib/snappy-java-1.1.7.jar_LICENSE.txt diff --git a/NOTICE.txt b/NOTICE.txt index 9ce75ba6097..ff3661df0a0 100644 --- a/NOTICE.txt +++ b/NOTICE.txt @@ -1,5 +1,5 @@ Apache ZooKeeper -Copyright 2009-2014 The Apache Software Foundation +Copyright 2009-2019 The Apache Software Foundation This product includes software developed at The Apache Software Foundation (http://www.apache.org/). diff --git a/README.md b/README.md index 9a248e12ece..0f9c4b2af13 100644 --- a/README.md +++ b/README.md @@ -35,6 +35,11 @@ is voted on and approved by the Apache ZooKeeper PMC: https://repo1.maven.org/maven2/org/apache/zookeeper/zookeeper/ +## Java 8 + +If you are going to compile with Java 1.8, you should use a +recent release at u211 or above. + # Contributing We always welcome new contributors to the project! See [How to Contribute](https://cwiki.apache.org/confluence/display/ZOOKEEPER/HowToContribute) for details on how to submit patch through pull request and our contribution workflow. diff --git a/zookeeper-server/src/main/resources/NOTICE.txt b/zookeeper-server/src/main/resources/NOTICE.txt index 065140a212c..9077cc0d9c6 100644 --- a/zookeeper-server/src/main/resources/NOTICE.txt +++ b/zookeeper-server/src/main/resources/NOTICE.txt @@ -1,9 +1,15 @@ Apache ZooKeeper -Copyright 2009-2016 The Apache Software Foundation +Copyright 2009-2019 The Apache Software Foundation This product includes software developed at The Apache Software Foundation (http://www.apache.org/). +This product includes software components originally +developed for Airlift (https://github.com/airlift/airlift), +licensed under the Apache 2.0 license. The licensing terms +for Airlift code can be found at: +https://github.com/airlift/airlift/blob/master/LICENSE + This product includes software developed by The Netty Project (http://netty.io/) Copyright 2011 The Netty Project @@ -99,3 +105,35 @@ framework implementation, which can be obtained at: * HOMEPAGE: * http://felix.apache.org/ +The bundled library Metrics Core NOTICE file reports the following items + +Metrics +Copyright 2010-2013 Coda Hale and Yammer, Inc. + +This product includes software developed by Coda Hale and Yammer, Inc. + +This product includes code derived from the JSR-166 project (ThreadLocalRandom, Striped64, +LongAdder), which was released with the following comments: + + Written by Doug Lea with assistance from members of JCP JSR-166 + Expert Group and released to the public domain, as explained at + http://creativecommons.org/publicdomain/zero/1.0/ + +The Nappy Java NOTICE file reports the following items: + +This product includes software developed by Google + Snappy: http://code.google.com/p/snappy/ (New BSD License) + +This product includes software developed by Apache + PureJavaCrc32C from apache-hadoop-common http://hadoop.apache.org/ + (Apache 2.0 license) + +This library containd statically linked libstdc++. This inclusion is allowed by +"GCC RUntime Library Exception" +http://gcc.gnu.org/onlinedocs/libstdc++/manual/license.html + +== Contributors == + * Tatu Saloranta + * Providing benchmark suite + * Alec Wysoker + * Performance and memory usage improvement diff --git a/zookeeper-server/src/main/resources/lib/metrics-core-5.2.5.jar_LICENSE.txt b/zookeeper-server/src/main/resources/lib/metrics-core-5.2.5.jar_LICENSE.txt new file mode 100644 index 00000000000..e4ba40426da --- /dev/null +++ b/zookeeper-server/src/main/resources/lib/metrics-core-5.2.5.jar_LICENSE.txt @@ -0,0 +1,202 @@ + + Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ + + TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + + 1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + + 2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + + 3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + + 4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + + 5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + + 6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + + 7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + + 8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + + 9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + + END OF TERMS AND CONDITIONS + + APPENDIX: How to apply the Apache License to your work. + + To apply the Apache License to your work, attach the following + boilerplate notice, with the fields enclosed by brackets "[]" + replaced with your own identifying information. (Don't include + the brackets!) The text should be enclosed in the appropriate + comment syntax for the file format. We also recommend that a + file or class name and description of purpose be included on the + same "printed page" as the copyright notice for easier + identification within third-party archives. + + Copyright 2010-2012 Coda Hale and Yammer, Inc. + + Licensed 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. diff --git a/zookeeper-server/src/main/resources/lib/snappy-java-1.1.7.jar_LICENSE.txt b/zookeeper-server/src/main/resources/lib/snappy-java-1.1.7.jar_LICENSE.txt new file mode 100644 index 00000000000..d6456956733 --- /dev/null +++ b/zookeeper-server/src/main/resources/lib/snappy-java-1.1.7.jar_LICENSE.txt @@ -0,0 +1,202 @@ + + Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ + + TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + + 1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + + 2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + + 3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + + 4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + + 5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + + 6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + + 7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + + 8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + + 9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + + END OF TERMS AND CONDITIONS + + APPENDIX: How to apply the Apache License to your work. + + To apply the Apache License to your work, attach the following + boilerplate notice, with the fields enclosed by brackets "[]" + replaced with your own identifying information. (Don't include + the brackets!) The text should be enclosed in the appropriate + comment syntax for the file format. We also recommend that a + file or class name and description of purpose be included on the + same "printed page" as the copyright notice for easier + identification within third-party archives. + + Copyright [yyyy] [name of copyright owner] + + Licensed 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. From d01dac1fc4983fdde60dee64f0dd2b083a4e6bf2 Mon Sep 17 00:00:00 2001 From: Damien Diederen Date: Mon, 6 Jan 2020 13:04:35 +0100 Subject: [PATCH 008/118] ZOOKEEPER-3640: Implement "batch mode" in cli_mt Batch mode never was implemented in `cli_mt`. This patch seems to work, but: 1. There may be a cleaner way of waiting for the completion; 2. ~~`nanosleep` is POSIX; the Windows path should probably use `Sleep`~~ (DONE). symat: Comments welcome. Author: Damien Diederen Reviewers: andor@apache.org Closes #1173 from ztzg/ZOOKEEPER-3640-implement-batch-mode-in-cli-mt (cherry picked from commit d7bc7b135f486f0b91bd7e40b150f39813ef9a9a) Signed-off-by: Andor Molnar --- zookeeper-client/zookeeper-client-c/src/cli.c | 25 +++++++++++++++++-- 1 file changed, 23 insertions(+), 2 deletions(-) diff --git a/zookeeper-client/zookeeper-client-c/src/cli.c b/zookeeper-client/zookeeper-client-c/src/cli.c index b2314345aee..34f2b99ef7e 100644 --- a/zookeeper-client/zookeeper-client-c/src/cli.c +++ b/zookeeper-client/zookeeper-client-c/src/cli.c @@ -736,6 +736,19 @@ int handleBatchMode(const char* arg, const char** buf) { return 1; } +#ifdef THREADED +static void millisleep(int ms) { +#ifdef WIN32 + Sleep(ms); +#else /* !WIN32 */ + struct timespec ts; + ts.tv_sec = ms / 1000; + ts.tv_nsec = (ms % 1000) * 1000000; // to nanoseconds + nanosleep(&ts, NULL); +#endif /* WIN32 */ +} +#endif /* THREADED */ + int main(int argc, char **argv) { static struct option long_options[] = { {"host", required_argument, NULL, 'h'}, //hostPort @@ -896,9 +909,17 @@ int main(int argc, char **argv) { #endif #ifdef THREADED + if (batchMode) { + processline(cmd); + } while(!shutdownThisThing) { - int rc; - int len = sizeof(buffer) - bufoff -1; + int rc, len; + if (batchMode) { + // We are just waiting for the asynchronous command to complete. + millisleep(10); + continue; + } + len = sizeof(buffer) - bufoff -1; if (len <= 0) { fprintf(stderr, "Can't handle lines that long!\n"); exit(2); From 260c8896593055f705857d8e41f1094bd2fadb33 Mon Sep 17 00:00:00 2001 From: Colm O hEigeartaigh Date: Mon, 6 Jan 2020 13:07:32 +0100 Subject: [PATCH 009/118] ZOOKEEPER-3638: Update Jetty to 9.4.24.v20191120 Author: Colm O hEigeartaigh Reviewers: eolivelli@apache.org, andor@apache.org Closes #1165 from coheigea/ZOOKEEPER-3638 (cherry picked from commit 178e8de664bed018c339108f168ae4ae4989b3b6) Signed-off-by: Andor Molnar --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 3f5d9bd6805..c523e9d2295 100755 --- a/pom.xml +++ b/pom.xml @@ -287,7 +287,7 @@ 1.3 1.2 4.1.42.Final - 9.4.18.v20190429 + 9.4.24.v20191120 2.9.10.1 1.1.1 2.11 From 3aa3029868b03e92ecf20376eb7d947629afca7a Mon Sep 17 00:00:00 2001 From: ravowlga123 Date: Thu, 9 Jan 2020 11:27:14 +0100 Subject: [PATCH 010/118] ZOOKEEPER-2083: Remove deprecated class AuthFastLeaderElection As per [ZOOKEEPER-2083](https://jira.apache.org/jira/browse/ZOOKEEPER-2083) we need to remove class AuthFastLeaderElection.java so I made changes in Quorumpeer.java by removing two cases 1 and 2 present in createElectionAlgorithm as QuorumPeerconfig.electioalg is always 3. Please do let me know if anything additional needs to be done. Author: ravowlga123 Reviewers: Enrico Olivelli , Andor Molnar , Patrick Hunt Closes #1171 from ravowlga123/ZOOKEEPER-2083 (cherry picked from commit 590e3cbcf1967d4de6ebc08a938584e3b4caf7c4) Signed-off-by: Enrico Olivelli --- .../main/resources/markdown/zookeeperAdmin.md | 11 +- .../server/quorum/AuthFastLeaderElection.java | 956 ------------------ .../zookeeper/server/quorum/QuorumPeer.java | 6 +- .../server/quorum/QuorumPeerConfig.java | 4 +- .../test/resources/findbugsExcludeFile.xml | 5 - 5 files changed, 10 insertions(+), 972 deletions(-) delete mode 100644 zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/AuthFastLeaderElection.java diff --git a/zookeeper-docs/src/main/resources/markdown/zookeeperAdmin.md b/zookeeper-docs/src/main/resources/markdown/zookeeperAdmin.md index fb7d7894783..7776f6431ca 100644 --- a/zookeeper-docs/src/main/resources/markdown/zookeeperAdmin.md +++ b/zookeeper-docs/src/main/resources/markdown/zookeeperAdmin.md @@ -1080,12 +1080,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. Currently, algorithm 3 is the default. + 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 are now - **deprecated**. We have the intention - of removing them in the next release, at which point only the - FastLeaderElection will be available. + >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). > * *initLimit* : (No Java system property) diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/AuthFastLeaderElection.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/AuthFastLeaderElection.java deleted file mode 100644 index cb0fec8d928..00000000000 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/AuthFastLeaderElection.java +++ /dev/null @@ -1,956 +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.server.quorum; - -import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; -import java.io.IOException; -import java.net.DatagramPacket; -import java.net.DatagramSocket; -import java.net.InetAddress; -import java.net.InetSocketAddress; -import java.net.SocketException; -import java.nio.ByteBuffer; -import java.util.Collection; -import java.util.Collections; -import java.util.HashMap; -import java.util.Map; -import java.util.Random; -import java.util.Set; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.LinkedBlockingQueue; -import java.util.concurrent.Semaphore; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicLong; -import org.apache.zookeeper.common.Time; -import org.apache.zookeeper.jmx.MBeanRegistry; -import org.apache.zookeeper.server.ZooKeeperThread; -import org.apache.zookeeper.server.quorum.QuorumPeer.QuorumServer; -import org.apache.zookeeper.server.quorum.QuorumPeer.ServerState; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * @deprecated This class has been deprecated as of release 3.4.0. - */ -@Deprecated -public class AuthFastLeaderElection implements Election { - - private static final Logger LOG = LoggerFactory.getLogger(AuthFastLeaderElection.class); - - /* Sequence numbers for messages */ - static int sequencer = 0; - static int maxTag = 0; - - /* - * Determine how much time a process has to wait once it believes that it - * has reached the end of leader election. - */ - static int finalizeWait = 100; - - /* - * Challenge counter to avoid replay attacks - */ - - static int challengeCounter = 0; - - /* - * Flag to determine whether to authenticate or not - */ - - private boolean authEnabled = false; - - public static class Notification { - - /* - * Proposed leader - */ long leader; - - /* - * zxid of the proposed leader - */ long zxid; - - /* - * Epoch - */ long epoch; - - /* - * current state of sender - */ QuorumPeer.ServerState state; - - /* - * Address of the sender - */ InetSocketAddress addr; - - } - - /* - * Messages to send, both Notifications and Acks - */ - public static class ToSend { - - enum mType { - crequest, - challenge, - notification, - ack - } - - ToSend(mType type, long tag, long leader, long zxid, long epoch, ServerState state, InetSocketAddress addr) { - - switch (type) { - case crequest: - this.type = 0; - this.tag = tag; - this.leader = leader; - this.zxid = zxid; - this.epoch = epoch; - this.state = state; - this.addr = addr; - - break; - case challenge: - this.type = 1; - this.tag = tag; - this.leader = leader; - this.zxid = zxid; - this.epoch = epoch; - this.state = state; - this.addr = addr; - - break; - case notification: - this.type = 2; - this.leader = leader; - this.zxid = zxid; - this.epoch = epoch; - this.state = QuorumPeer.ServerState.LOOKING; - this.tag = tag; - this.addr = addr; - - break; - case ack: - this.type = 3; - this.tag = tag; - this.leader = leader; - this.zxid = zxid; - this.epoch = epoch; - this.state = state; - this.addr = addr; - - break; - default: - break; - } - } - - /* - * Message type: 0 notification, 1 acknowledgement - */ int type; - - /* - * Proposed leader in the case of notification - */ long leader; - - /* - * id contains the tag for acks, and zxid for notifications - */ long zxid; - - /* - * Epoch - */ long epoch; - - /* - * Current state; - */ QuorumPeer.ServerState state; - - /* - * Message tag - */ long tag; - - InetSocketAddress addr; - - } - - LinkedBlockingQueue sendqueue; - - LinkedBlockingQueue recvqueue; - - private class Messenger { - - final DatagramSocket mySocket; - long lastProposedLeader; - long lastProposedZxid; - long lastEpoch; - final Set ackset; - final ConcurrentHashMap challengeMap; - final ConcurrentHashMap challengeMutex; - final ConcurrentHashMap ackMutex; - final ConcurrentHashMap> addrChallengeMap; - - class WorkerReceiver extends ZooKeeperThread { - - DatagramSocket mySocket; - Messenger myMsg; - - WorkerReceiver(DatagramSocket s, Messenger msg) { - super("WorkerReceiver-" + s.getRemoteSocketAddress()); - mySocket = s; - myMsg = msg; - } - - boolean saveChallenge(long tag, long challenge) { - Semaphore s = challengeMutex.get(tag); - if (s != null) { - synchronized (Messenger.this) { - challengeMap.put(tag, challenge); - challengeMutex.remove(tag); - } - - s.release(); - } else { - LOG.error("No challenge mutex object"); - } - - return true; - } - - public void run() { - byte[] responseBytes = new byte[48]; - ByteBuffer responseBuffer = ByteBuffer.wrap(responseBytes); - DatagramPacket responsePacket = new DatagramPacket(responseBytes, responseBytes.length); - while (true) { - // Sleeps on receive - try { - responseBuffer.clear(); - mySocket.receive(responsePacket); - } catch (IOException e) { - LOG.warn("Ignoring exception receiving", e); - } - // Receive new message - if (responsePacket.getLength() != responseBytes.length) { - LOG.warn("Got a short response: {} {}", responsePacket.getLength(), responsePacket.toString()); - continue; - } - responseBuffer.clear(); - int type = responseBuffer.getInt(); - if ((type > 3) || (type < 0)) { - LOG.warn("Got bad Msg type: {}", type); - continue; - } - long tag = responseBuffer.getLong(); - - QuorumPeer.ServerState ackstate = QuorumPeer.ServerState.LOOKING; - switch (responseBuffer.getInt()) { - case 0: - ackstate = QuorumPeer.ServerState.LOOKING; - break; - case 1: - ackstate = QuorumPeer.ServerState.LEADING; - break; - case 2: - ackstate = QuorumPeer.ServerState.FOLLOWING; - break; - default: - LOG.warn("unknown type {}", responseBuffer.getInt()); - break; - } - - Vote current = self.getCurrentVote(); - - switch (type) { - case 0: - // Receive challenge request - ToSend c = new ToSend( - ToSend.mType.challenge, - tag, - current.getId(), - current.getZxid(), - logicalclock.get(), - self.getPeerState(), - (InetSocketAddress) responsePacket.getSocketAddress()); - sendqueue.offer(c); - break; - case 1: - // Receive challenge and store somewhere else - long challenge = responseBuffer.getLong(); - saveChallenge(tag, challenge); - - break; - case 2: - Notification n = new Notification(); - n.leader = responseBuffer.getLong(); - n.zxid = responseBuffer.getLong(); - n.epoch = responseBuffer.getLong(); - n.state = ackstate; - n.addr = (InetSocketAddress) responsePacket.getSocketAddress(); - - if ((myMsg.lastEpoch <= n.epoch) - && ((n.zxid > myMsg.lastProposedZxid) - || ((n.zxid == myMsg.lastProposedZxid) - && (n.leader > myMsg.lastProposedLeader)))) { - myMsg.lastProposedZxid = n.zxid; - myMsg.lastProposedLeader = n.leader; - myMsg.lastEpoch = n.epoch; - } - - long recChallenge; - InetSocketAddress addr = (InetSocketAddress) responsePacket.getSocketAddress(); - if (authEnabled) { - ConcurrentHashMap tmpMap = addrChallengeMap.get(addr); - if (tmpMap != null) { - if (tmpMap.get(tag) != null) { - recChallenge = responseBuffer.getLong(); - - if (tmpMap.get(tag) == recChallenge) { - recvqueue.offer(n); - - ToSend a = new ToSend( - ToSend.mType.ack, - tag, - current.getId(), - current.getZxid(), - logicalclock.get(), - self.getPeerState(), - addr); - - sendqueue.offer(a); - } else { - LOG.warn("Incorrect challenge: {}, {}", recChallenge, addrChallengeMap.toString()); - } - } else { - LOG.warn("No challenge for host: {} {}", addr, tag); - } - } - } else { - recvqueue.offer(n); - - ToSend a = new ToSend( - ToSend.mType.ack, - tag, - current.getId(), - current.getZxid(), - logicalclock.get(), - self.getPeerState(), - (InetSocketAddress) responsePacket.getSocketAddress()); - - sendqueue.offer(a); - } - break; - - // Upon reception of an ack message, remove it from the - // queue - case 3: - Semaphore s = ackMutex.get(tag); - - if (s != null) { - s.release(); - } else { - LOG.error("Empty ack semaphore"); - } - - ackset.add(tag); - - if (authEnabled) { - ConcurrentHashMap tmpMap = addrChallengeMap.get(responsePacket.getSocketAddress()); - if (tmpMap != null) { - tmpMap.remove(tag); - } else { - LOG.warn("No such address in the ensemble configuration {}", responsePacket.getSocketAddress()); - } - } - - if (ackstate != QuorumPeer.ServerState.LOOKING) { - Notification outofsync = new Notification(); - outofsync.leader = responseBuffer.getLong(); - outofsync.zxid = responseBuffer.getLong(); - outofsync.epoch = responseBuffer.getLong(); - outofsync.state = ackstate; - outofsync.addr = (InetSocketAddress) responsePacket.getSocketAddress(); - - recvqueue.offer(outofsync); - } - - break; - // Default case - default: - LOG.warn("Received message of incorrect type {}", type); - break; - } - } - } - - } - - class WorkerSender extends ZooKeeperThread { - - Random rand; - int maxAttempts; - int ackWait = finalizeWait; - - /* - * Receives a socket and max number of attempts as input - */ - - WorkerSender(int attempts) { - super("WorkerSender"); - maxAttempts = attempts; - rand = new Random(java.lang.Thread.currentThread().getId() + Time.currentElapsedTime()); - } - - long genChallenge() { - byte[] buf = new byte[8]; - - buf[0] = (byte) ((challengeCounter & 0xff000000) >>> 24); - buf[1] = (byte) ((challengeCounter & 0x00ff0000) >>> 16); - buf[2] = (byte) ((challengeCounter & 0x0000ff00) >>> 8); - buf[3] = (byte) ((challengeCounter & 0x000000ff)); - - challengeCounter++; - int secret = rand.nextInt(java.lang.Integer.MAX_VALUE); - - buf[4] = (byte) ((secret & 0xff000000) >>> 24); - buf[5] = (byte) ((secret & 0x00ff0000) >>> 16); - buf[6] = (byte) ((secret & 0x0000ff00) >>> 8); - buf[7] = (byte) ((secret & 0x000000ff)); - - return (((long) (buf[0] & 0xFF)) << 56) - + (((long) (buf[1] & 0xFF)) << 48) - + (((long) (buf[2] & 0xFF)) << 40) - + (((long) (buf[3] & 0xFF)) << 32) - + (((long) (buf[4] & 0xFF)) << 24) - + (((long) (buf[5] & 0xFF)) << 16) - + (((long) (buf[6] & 0xFF)) << 8) - + ((long) (buf[7] & 0xFF)); - } - - public void run() { - while (true) { - try { - ToSend m = sendqueue.take(); - process(m); - } catch (InterruptedException e) { - break; - } - - } - } - - @SuppressFBWarnings( - value = "RV_RETURN_VALUE_IGNORED", - justification = "tryAcquire result not chacked, but it is not an issue") - private void process(ToSend m) { - int attempts = 0; - byte[] zeroes; - byte[] requestBytes = new byte[48]; - DatagramPacket requestPacket = new DatagramPacket(requestBytes, requestBytes.length); - ByteBuffer requestBuffer = ByteBuffer.wrap(requestBytes); - - switch (m.type) { - case 0: - /* - * Building challenge request packet to send - */ - requestBuffer.clear(); - requestBuffer.putInt(ToSend.mType.crequest.ordinal()); - requestBuffer.putLong(m.tag); - requestBuffer.putInt(m.state.ordinal()); - zeroes = new byte[32]; - requestBuffer.put(zeroes); - - requestPacket.setLength(48); - try { - requestPacket.setSocketAddress(m.addr); - } catch (IllegalArgumentException e) { - // Sun doesn't include the address that causes this - // exception to be thrown, so we wrap the exception - // in order to capture this critical detail. - throw new IllegalArgumentException("Unable to set socket address on packet, msg:" + e.getMessage() - + " with addr:" + m.addr, e); - } - - try { - if (challengeMap.get(m.tag) == null) { - mySocket.send(requestPacket); - } - } catch (IOException e) { - LOG.warn("Exception while sending challenge: ", e); - } - - break; - case 1: - /* - * Building challenge packet to send - */ - - long newChallenge; - ConcurrentHashMap tmpMap = addrChallengeMap.get(m.addr); - if (tmpMap != null) { - Long tmpLong = tmpMap.get(m.tag); - if (tmpLong != null) { - newChallenge = tmpLong; - } else { - newChallenge = genChallenge(); - } - - tmpMap.put(m.tag, newChallenge); - - requestBuffer.clear(); - requestBuffer.putInt(ToSend.mType.challenge.ordinal()); - requestBuffer.putLong(m.tag); - requestBuffer.putInt(m.state.ordinal()); - requestBuffer.putLong(newChallenge); - zeroes = new byte[24]; - requestBuffer.put(zeroes); - - requestPacket.setLength(48); - try { - requestPacket.setSocketAddress(m.addr); - } catch (IllegalArgumentException e) { - // Sun doesn't include the address that causes this - // exception to be thrown, so we wrap the exception - // in order to capture this critical detail. - throw new IllegalArgumentException("Unable to set socket address on packet, msg:" + e.getMessage() - + " with addr:" + m.addr, e); - } - - try { - mySocket.send(requestPacket); - } catch (IOException e) { - LOG.warn("Exception while sending challenge: ", e); - } - } else { - LOG.error("Address is not in the configuration: {}", m.addr); - } - - break; - case 2: - - /* - * Building notification packet to send - */ - - requestBuffer.clear(); - requestBuffer.putInt(m.type); - requestBuffer.putLong(m.tag); - requestBuffer.putInt(m.state.ordinal()); - requestBuffer.putLong(m.leader); - requestBuffer.putLong(m.zxid); - requestBuffer.putLong(m.epoch); - zeroes = new byte[8]; - requestBuffer.put(zeroes); - - requestPacket.setLength(48); - try { - requestPacket.setSocketAddress(m.addr); - } catch (IllegalArgumentException e) { - // Sun doesn't include the address that causes this - // exception to be thrown, so we wrap the exception - // in order to capture this critical detail. - throw new IllegalArgumentException("Unable to set socket address on packet, msg:" + e.getMessage() - + " with addr:" + m.addr, e); - } - - boolean myChallenge = false; - boolean myAck = false; - - while (attempts < maxAttempts) { - try { - /* - * Try to obtain a challenge only if does not have - * one yet - */ - - if (!myChallenge && authEnabled) { - ToSend crequest = new ToSend( - ToSend.mType.crequest, - m.tag, - m.leader, - m.zxid, - m.epoch, - QuorumPeer.ServerState.LOOKING, - m.addr); - sendqueue.offer(crequest); - - try { - double timeout = ackWait * java.lang.Math.pow(2, attempts); - - Semaphore s = new Semaphore(0); - synchronized (Messenger.this) { - challengeMutex.put(m.tag, s); - s.tryAcquire((long) timeout, TimeUnit.MILLISECONDS); - myChallenge = challengeMap.containsKey(m.tag); - } - } catch (InterruptedException e) { - LOG.warn("Challenge request exception: ", e); - } - } - - /* - * If don't have challenge yet, skip sending - * notification - */ - - if (authEnabled && !myChallenge) { - attempts++; - continue; - } - - if (authEnabled) { - requestBuffer.position(40); - Long tmpLong = challengeMap.get(m.tag); - if (tmpLong != null) { - requestBuffer.putLong(tmpLong); - } else { - LOG.warn("No challenge with tag: {}", m.tag); - } - } - mySocket.send(requestPacket); - try { - Semaphore s = new Semaphore(0); - double timeout = ackWait * java.lang.Math.pow(10, attempts); - ackMutex.put(m.tag, s); - s.tryAcquire((int) timeout, TimeUnit.MILLISECONDS); - } catch (InterruptedException e) { - LOG.warn("Ack exception: ", e); - } - - if (ackset.remove(m.tag)) { - myAck = true; - } - - } catch (IOException e) { - LOG.warn("Sending exception: ", e); - /* - * Do nothing, just try again - */ - } - if (myAck) { - /* - * Received ack successfully, so return - */ - challengeMap.remove(m.tag); - - return; - } else { - attempts++; - } - } - /* - * Return message to queue for another attempt later if - * epoch hasn't changed. - */ - if (m.epoch == logicalclock.get()) { - challengeMap.remove(m.tag); - sendqueue.offer(m); - } - break; - case 3: - - requestBuffer.clear(); - requestBuffer.putInt(m.type); - requestBuffer.putLong(m.tag); - requestBuffer.putInt(m.state.ordinal()); - requestBuffer.putLong(m.leader); - requestBuffer.putLong(m.zxid); - requestBuffer.putLong(m.epoch); - - requestPacket.setLength(48); - try { - requestPacket.setSocketAddress(m.addr); - } catch (IllegalArgumentException e) { - // Sun doesn't include the address that causes this - // exception to be thrown, so we wrap the exception - // in order to capture this critical detail. - throw new IllegalArgumentException("Unable to set socket address on packet, msg:" + e.getMessage() - + " with addr:" + m.addr, e); - } - - try { - mySocket.send(requestPacket); - } catch (IOException e) { - LOG.warn("Exception while sending ack: ", e); - } - break; - default: - LOG.warn("unknown type {}", m.type); - break; - } - } - - } - - Messenger(int threads, DatagramSocket s) { - mySocket = s; - ackset = Collections.newSetFromMap(new ConcurrentHashMap()); - challengeMap = new ConcurrentHashMap(); - challengeMutex = new ConcurrentHashMap(); - ackMutex = new ConcurrentHashMap(); - addrChallengeMap = new ConcurrentHashMap>(); - lastProposedLeader = 0; - lastProposedZxid = 0; - lastEpoch = 0; - - for (int i = 0; i < threads; ++i) { - Thread t = new Thread(new WorkerSender(3), "WorkerSender Thread: " + (i + 1)); - t.setDaemon(true); - t.start(); - } - - for (QuorumServer server : self.getVotingView().values()) { - InetAddress address = server.addr.getReachableOrOne().getAddress(); - InetSocketAddress saddr = new InetSocketAddress(address, port); - addrChallengeMap.put(saddr, new ConcurrentHashMap()); - } - - Thread t = new Thread(new WorkerReceiver(s, this), "WorkerReceiver Thread"); - t.start(); - } - - } - - QuorumPeer self; - int port; - AtomicLong logicalclock = new AtomicLong(); /* Election instance */ - DatagramSocket mySocket; - long proposedLeader; - long proposedZxid; - - public AuthFastLeaderElection(QuorumPeer self, boolean auth) { - this.authEnabled = auth; - starter(self); - } - - public AuthFastLeaderElection(QuorumPeer self) { - starter(self); - } - - private void starter(QuorumPeer self) { - this.self = self; - port = self.getVotingView().get(self.getId()).electionAddr.getAllPorts().get(0); - proposedLeader = -1; - proposedZxid = -1; - - try { - mySocket = new DatagramSocket(port); - // mySocket.setSoTimeout(20000); - } catch (SocketException e1) { - e1.printStackTrace(); - throw new RuntimeException(); - } - sendqueue = new LinkedBlockingQueue(2 * self.getVotingView().size()); - recvqueue = new LinkedBlockingQueue(2 * self.getVotingView().size()); - new Messenger(self.getVotingView().size() * 2, mySocket); - } - - private void leaveInstance() { - logicalclock.incrementAndGet(); - } - - private void sendNotifications() { - for (QuorumServer server : self.getView().values()) { - - InetSocketAddress address = self.getView().get(server.id).electionAddr.getReachableOrOne(); - ToSend notmsg = new ToSend( - ToSend.mType.notification, - AuthFastLeaderElection.sequencer++, - proposedLeader, - proposedZxid, - logicalclock.get(), - QuorumPeer.ServerState.LOOKING, - address); - - sendqueue.offer(notmsg); - } - } - - private boolean totalOrderPredicate(long id, long zxid) { - return (zxid > proposedZxid) || ((zxid == proposedZxid) && (id > proposedLeader)); - - } - - private boolean termPredicate(Map votes, long l, long zxid) { - - Collection votesCast = votes.values(); - int count = 0; - /* - * First make the views consistent. Sometimes peers will have different - * zxids for a server depending on timing. - */ - for (Vote v : votesCast) { - if ((v.getId() == l) && (v.getZxid() == zxid)) { - count++; - } - } - - return count > (self.getVotingView().size() / 2); - - } - - /** - * There is nothing to shutdown in this implementation of - * leader election, so we simply have an empty method. - */ - public void shutdown() { - } - - /** - * Invoked in QuorumPeer to find or elect a new leader. - * - * @throws InterruptedException - */ - public Vote lookForLeader() throws InterruptedException { - try { - self.jmxLeaderElectionBean = new LeaderElectionBean(); - MBeanRegistry.getInstance().register(self.jmxLeaderElectionBean, self.jmxLocalPeerBean); - } catch (Exception e) { - LOG.warn("Failed to register with JMX", e); - self.jmxLeaderElectionBean = null; - } - - try { - HashMap recvset = new HashMap(); - - HashMap outofelection = new HashMap(); - - logicalclock.incrementAndGet(); - - proposedLeader = self.getId(); - proposedZxid = self.getLastLoggedZxid(); - - LOG.info("Election tally"); - sendNotifications(); - - /* - * Loop in which we exchange notifications until we find a leader - */ - - while (self.getPeerState() == ServerState.LOOKING) { - /* - * Remove next notification from queue, times out after 2 times - * the termination time - */ - Notification n = recvqueue.poll(2 * finalizeWait, TimeUnit.MILLISECONDS); - - /* - * Sends more notifications if haven't received enough. - * Otherwise processes new notification. - */ - if (n == null) { - if (((!outofelection.isEmpty()) || (recvset.size() > 1))) { - sendNotifications(); - } - } else { - switch (n.state) { - case LOOKING: - if (n.epoch > logicalclock.get()) { - logicalclock.set(n.epoch); - recvset.clear(); - if (totalOrderPredicate(n.leader, n.zxid)) { - proposedLeader = n.leader; - proposedZxid = n.zxid; - } - sendNotifications(); - } else if (n.epoch < logicalclock.get()) { - break; - } else if (totalOrderPredicate(n.leader, n.zxid)) { - proposedLeader = n.leader; - proposedZxid = n.zxid; - - sendNotifications(); - } - - recvset.put(n.addr, new Vote(n.leader, n.zxid)); - - // If have received from all nodes, then terminate - if (self.getVotingView().size() == recvset.size()) { - self.setPeerState((proposedLeader == self.getId()) - ? ServerState.LEADING - : ServerState.FOLLOWING); - // if (self.state == ServerState.FOLLOWING) { - // Thread.sleep(100); - // } - leaveInstance(); - return new Vote(proposedLeader, proposedZxid); - - } else if (termPredicate(recvset, proposedLeader, proposedZxid)) { - // Otherwise, wait for a fixed amount of time - LOG.info("Passed predicate"); - Thread.sleep(finalizeWait); - - // Notification probe = recvqueue.peek(); - - // Verify if there is any change in the proposed leader - while ((!recvqueue.isEmpty()) - && !totalOrderPredicate(recvqueue.peek().leader, recvqueue.peek().zxid)) { - recvqueue.poll(); - } - if (recvqueue.isEmpty()) { - // LOG.warn("Proposed leader: " + - // proposedLeader); - self.setPeerState((proposedLeader == self.getId()) - ? ServerState.LEADING - : ServerState.FOLLOWING); - - leaveInstance(); - return new Vote(proposedLeader, proposedZxid); - } - } - break; - case LEADING: - outofelection.put(n.addr, new Vote(n.leader, n.zxid)); - - if (termPredicate(outofelection, n.leader, n.zxid)) { - - self.setPeerState((n.leader == self.getId()) ? ServerState.LEADING : ServerState.FOLLOWING); - - leaveInstance(); - return new Vote(n.leader, n.zxid); - } - break; - case FOLLOWING: - outofelection.put(n.addr, new Vote(n.leader, n.zxid)); - - if (termPredicate(outofelection, n.leader, n.zxid)) { - - self.setPeerState((n.leader == self.getId()) ? ServerState.LEADING : ServerState.FOLLOWING); - - leaveInstance(); - return new Vote(n.leader, n.zxid); - } - break; - default: - break; - } - } - } - - return null; - } finally { - try { - if (self.jmxLeaderElectionBean != null) { - MBeanRegistry.getInstance().unregister(self.jmxLeaderElectionBean); - } - } catch (Exception e) { - LOG.warn("Failed to unregister with JMX", e); - } - self.jmxLeaderElectionBean = null; - } - } - -} diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumPeer.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumPeer.java index 72d06ecbd05..415a738080e 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumPeer.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumPeer.java @@ -1227,11 +1227,9 @@ protected Election createElectionAlgorithm(int electionAlgorithm) { //TODO: use a factory rather than a switch switch (electionAlgorithm) { case 1: - le = new AuthFastLeaderElection(this); - break; + throw new UnsupportedOperationException("Election Algorithm 1 is not supported."); case 2: - le = new AuthFastLeaderElection(this, true); - break; + throw new UnsupportedOperationException("Election Algorithm 2 is not supported."); case 3: QuorumCnxManager qcm = createCnxnManager(); QuorumCnxManager oldQcm = qcmRef.getAndSet(qcm); 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 459a057d605..99fc2ed8244 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 @@ -312,8 +312,8 @@ public void parseProperties(Properties zkProp) throws IOException, ConfigExcepti connectToLearnerMasterLimit = Integer.parseInt(value); } else if (key.equals("electionAlg")) { electionAlg = Integer.parseInt(value); - if (electionAlg != 1 && electionAlg != 2 && electionAlg != 3) { - throw new ConfigException("Invalid electionAlg value. Only 1, 2, 3 are supported."); + if (electionAlg != 3) { + throw new ConfigException("Invalid electionAlg value. Only 3 is supported."); } } else if (key.equals("quorumListenOnAllIPs")) { quorumListenOnAllIPs = Boolean.parseBoolean(value); diff --git a/zookeeper-server/src/test/resources/findbugsExcludeFile.xml b/zookeeper-server/src/test/resources/findbugsExcludeFile.xml index 28ac46858b8..2a352ccfb03 100644 --- a/zookeeper-server/src/test/resources/findbugsExcludeFile.xml +++ b/zookeeper-server/src/test/resources/findbugsExcludeFile.xml @@ -117,11 +117,6 @@ - - - - - From 901d82c7fa2fe2ff9cc6a408ab0b8a490667def7 Mon Sep 17 00:00:00 2001 From: ravowlga123 Date: Thu, 9 Jan 2020 22:51:51 +0100 Subject: [PATCH 011/118] ZOOKEEPER-3649: Add a line break in ls -s CLI As per the ticket [ZOOKEEEPER-3649](https://issues.apache.org/jira/browse/ZOOKEEPER-3649) we need to add line break ls -s command. To achieve the desired behavior I have added new line character in option.add of LsCommand.printChildren(). Please do let me know if made changes gives us the desired behavior or if anything else needs to be changed. Author: ravowlga123 Reviewers: eolivelli@apache.org, andor@apache.org Closes #1183 from ravowlga123/ZOOKEEPER-3649 and squashes the following commits: f2e1997d6 [ravowlga123] ZOOKEEPER-3649 Removed additional new line from printChildren b36d63664 [ravowlga123] ZOOKEEPER-3649 Add a line break in ls -s CLI (cherry picked from commit 570285af427fd69471a682a071a47483f8f7ddf8) Signed-off-by: Andor Molnar --- .../src/main/java/org/apache/zookeeper/cli/LsCommand.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/cli/LsCommand.java b/zookeeper-server/src/main/java/org/apache/zookeeper/cli/LsCommand.java index a66821c2c38..770a4a965a7 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/cli/LsCommand.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/cli/LsCommand.java @@ -133,11 +133,10 @@ private void printChildren(List children, Stat stat) { } out.append(child); } - out.append("]"); + out.append("]\n"); if (stat != null) { new StatPrinter(out).print(stat); } - out.append("\n"); } } From dc673cb39d89162289f2ebfc7997d4e20efb7f17 Mon Sep 17 00:00:00 2001 From: ravowlga123 Date: Thu, 9 Jan 2020 22:56:39 +0100 Subject: [PATCH 012/118] ZOOKEEPER-3411: Remove deprecated ls2 and rmr command Removed Ls2Command.java, DeleteAllCommand.printdeprecatedwarning(), statements creating new objects for ls2 and rmr in ZookeeperMain.java. Updated zookeeperCLI.md and ZookeeperTest.java. Please do let me know if any additional changes are needed Author: ravowlga123 Reviewers: andor@apache.org Closes #1175 from ravowlga123/ZOOKEEPER-3411 (cherry picked from commit 27b92caefd57a60309af06ebce29e56954ca9aac) Signed-off-by: Andor Molnar --- .../main/resources/markdown/zookeeperCLI.md | 19 ----- .../org/apache/zookeeper/ZooKeeperMain.java | 4 - .../zookeeper/cli/DeleteAllCommand.java | 7 -- .../org/apache/zookeeper/cli/Ls2Command.java | 77 ------------------- .../org/apache/zookeeper/ZooKeeperTest.java | 6 -- 5 files changed, 113 deletions(-) delete mode 100644 zookeeper-server/src/main/java/org/apache/zookeeper/cli/Ls2Command.java diff --git a/zookeeper-docs/src/main/resources/markdown/zookeeperCLI.md b/zookeeper-docs/src/main/resources/markdown/zookeeperCLI.md index 717c4f13190..3ca676e504d 100644 --- a/zookeeper-docs/src/main/resources/markdown/zookeeperCLI.md +++ b/zookeeper-docs/src/main/resources/markdown/zookeeperCLI.md @@ -48,13 +48,11 @@ ZooKeeper -server host:port cmd args history listquota path ls [-s] [-w] [-R] path - ls2 path [watch] printwatches on|off quit reconfig [-s] [-v version] [[-file path] | [-members serverID=host:port1:port2;port3[,...]*]] | [-add serverId=host:port1:port2;port3[,...]]* [-remove serverId[,...]*] redo cmdno removewatches path [-c|-d|-a] [-l] - rmr path set [-s] [-v version] path data setAcl [-s] [-v version] [-R] path acl setquota -n|-b val path @@ -324,15 +322,6 @@ Listing the child nodes of one path WatchedEvent state:SyncConnected type:NodeChildrenChanged path:/brokers ``` -## ls2 - -'ls2' has been deprecated. Please use 'ls [-s] path' instead. - -```bash -[zkshell: 7] ls2 / - 'ls2' has been deprecated. Please use 'ls [-s] path' instead. -``` - ## printwatches A switch to turn on/off whether printing watches or not. @@ -422,14 +411,6 @@ Remove the watches under a node. ``` -## rmr -The command 'rmr' has been deprecated. Please use 'deleteall' instead. - -```bash -[zkshell: 4] rmr /zk-latencies4 - The command 'rmr' has been deprecated. Please use 'deleteall' instead -``` - ## set Set/update the data on a path. 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 734cf7ec0ee..dad878cf87c 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/ZooKeeperMain.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/ZooKeeperMain.java @@ -53,7 +53,6 @@ import org.apache.zookeeper.cli.GetConfigCommand; import org.apache.zookeeper.cli.GetEphemeralsCommand; import org.apache.zookeeper.cli.ListQuotaCommand; -import org.apache.zookeeper.cli.Ls2Command; import org.apache.zookeeper.cli.LsCommand; import org.apache.zookeeper.cli.MalformedCommandException; import org.apache.zookeeper.cli.ReconfigCommand; @@ -105,12 +104,9 @@ public boolean getPrintWatches() { new CreateCommand().addToMap(commandMapCli); new DeleteCommand().addToMap(commandMapCli); new DeleteAllCommand().addToMap(commandMapCli); - // Depricated: rmr - new DeleteAllCommand("rmr").addToMap(commandMapCli); new SetCommand().addToMap(commandMapCli); new GetCommand().addToMap(commandMapCli); new LsCommand().addToMap(commandMapCli); - new Ls2Command().addToMap(commandMapCli); new GetAclCommand().addToMap(commandMapCli); new SetAclCommand().addToMap(commandMapCli); new StatCommand().addToMap(commandMapCli); diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/cli/DeleteAllCommand.java b/zookeeper-server/src/main/java/org/apache/zookeeper/cli/DeleteAllCommand.java index 507693f298a..cbeceb0e08e 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/cli/DeleteAllCommand.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/cli/DeleteAllCommand.java @@ -66,7 +66,6 @@ public CliCommand parse(String[] cmdArgs) throws CliParseException { @Override public boolean exec() throws CliException { - printDeprecatedWarning(); int batchSize; try { batchSize = cl.hasOption("b") ? Integer.parseInt(cl.getOptionValue("b")) : 1000; @@ -88,10 +87,4 @@ public boolean exec() throws CliException { return false; } - private void printDeprecatedWarning() { - if ("rmr".equals(args[0])) { - err.println("The command 'rmr' has been deprecated. " + "Please use 'deleteall' instead."); - } - } - } diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/cli/Ls2Command.java b/zookeeper-server/src/main/java/org/apache/zookeeper/cli/Ls2Command.java deleted file mode 100644 index 0bdaff6dcda..00000000000 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/cli/Ls2Command.java +++ /dev/null @@ -1,77 +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.cli; - -import java.util.List; -import org.apache.commons.cli.CommandLine; -import org.apache.commons.cli.Options; -import org.apache.commons.cli.ParseException; -import org.apache.commons.cli.Parser; -import org.apache.commons.cli.PosixParser; -import org.apache.zookeeper.KeeperException; -import org.apache.zookeeper.data.Stat; - -/** - * ls2 command for cli - */ -public class Ls2Command extends CliCommand { - - private static Options options = new Options(); - private String[] args; - - public Ls2Command() { - super("ls2", "path [watch]"); - } - - @Override - public CliCommand parse(String[] cmdArgs) throws CliParseException { - Parser parser = new PosixParser(); - CommandLine cl; - try { - cl = parser.parse(options, cmdArgs); - } catch (ParseException ex) { - throw new CliParseException(ex); - } - args = cl.getArgs(); - if (args.length < 2) { - throw new CliParseException(getUsageStr()); - } - - return this; - } - - @Override - public boolean exec() throws CliException { - err.println("'ls2' has been deprecated. " + "Please use 'ls [-s] path' instead."); - String path = args[1]; - boolean watch = args.length > 2; - Stat stat = new Stat(); - List children; - try { - children = zk.getChildren(path, watch, stat); - } catch (IllegalArgumentException ex) { - throw new MalformedPathException(ex.getMessage()); - } catch (KeeperException | InterruptedException ex) { - throw new CliWrapperException(ex); - } - out.println(children); - new StatPrinter(out).print(stat); - return watch; - } - -} diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/ZooKeeperTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/ZooKeeperTest.java index d19f627e069..4467a1ac471 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/ZooKeeperTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/ZooKeeperTest.java @@ -154,13 +154,7 @@ public void testDeleteRecursiveCli() throws IOException, InterruptedException, C assertTrue(children.contains("c")); ZooKeeperMain zkMain = new ZooKeeperMain(zk); - // 'rmr' is deprecated, so the test here is just for backwards - // compatibility. - String cmdstring0 = "rmr /a/b/v"; String cmdstring1 = "deleteall /a"; - zkMain.cl.parseCommand(cmdstring0); - assertFalse(zkMain.processZKCmd(zkMain.cl)); - assertEquals(null, zk.exists("/a/b/v", null)); zkMain.cl.parseCommand(cmdstring1); assertFalse(zkMain.processZKCmd(zkMain.cl)); assertNull(zk.exists("/a", null)); From dc57f63926f724a2fd8fa38d967932d2af3e74be Mon Sep 17 00:00:00 2001 From: Enrico Olivelli Date: Sat, 11 Jan 2020 08:56:49 +0100 Subject: [PATCH 013/118] Update release notes --- .../main/resources/markdown/releasenotes.md | 28 +++++++++++-------- 1 file changed, 17 insertions(+), 11 deletions(-) diff --git a/zookeeper-docs/src/main/resources/markdown/releasenotes.md b/zookeeper-docs/src/main/resources/markdown/releasenotes.md index 522e8bab780..f5b25aee9eb 100644 --- a/zookeeper-docs/src/main/resources/markdown/releasenotes.md +++ b/zookeeper-docs/src/main/resources/markdown/releasenotes.md @@ -1,5 +1,5 @@ + # Release Notes - ZooKeeper - Version 3.6.0 ## New Feature * [ZOOKEEPER-27](https://issues.apache.org/jira/browse/ZOOKEEPER-27) - Unique DB identifiers for servers and clients -* [ZOOKEEPER-1112](https://issues.apache.org/jira/browse/ZOOKEEPER-1112) - Add support for C client for SASL authentication * [ZOOKEEPER-1260](https://issues.apache.org/jira/browse/ZOOKEEPER-1260) - Audit logging in ZooKeeper servers. * [ZOOKEEPER-1634](https://issues.apache.org/jira/browse/ZOOKEEPER-1634) - A new feature proposal to ZooKeeper: authentication enforcement * [ZOOKEEPER-1703](https://issues.apache.org/jira/browse/ZOOKEEPER-1703) - Please add instructions for running the tutorial * [ZOOKEEPER-1962](https://issues.apache.org/jira/browse/ZOOKEEPER-1962) - Add a CLI command to recursively list a znode and children -* [ZOOKEEPER-2163](https://issues.apache.org/jira/browse/ZOOKEEPER-2163) - Introduce new ZNode type: container -* [ZOOKEEPER-2169](https://issues.apache.org/jira/browse/ZOOKEEPER-2169) - Enable creation of nodes with TTLs * [ZOOKEEPER-2875](https://issues.apache.org/jira/browse/ZOOKEEPER-2875) - Add ant task for running OWASP dependency report * [ZOOKEEPER-2933](https://issues.apache.org/jira/browse/ZOOKEEPER-2933) - Ability to monitor the jute.maxBuffer usage in real-time * [ZOOKEEPER-2994](https://issues.apache.org/jira/browse/ZOOKEEPER-2994) - Tool required to recover log and snapshot entries with CRC errors @@ -47,8 +45,7 @@ limitations under the License. * [ZOOKEEPER-3344](https://issues.apache.org/jira/browse/ZOOKEEPER-3344) - write a new script:zkSnapShotToolkit.sh to encapsulate SnapshotFormatter and doc the usage * [ZOOKEEPER-3371](https://issues.apache.org/jira/browse/ZOOKEEPER-3371) - Port unification for admin server * [ZOOKEEPER-3447](https://issues.apache.org/jira/browse/ZOOKEEPER-3447) - add a doc: zookeeperMonitor.md -* [ZOOKEEPER-3657](https://issues.apache.org/jira/browse/ZOOKEEPER-3657) - Implementing snapshot schedule to avoid high latency issue due to disk contention - + ## Improvement * [ZOOKEEPER-261](https://issues.apache.org/jira/browse/ZOOKEEPER-261) - Reinitialized servers should not participate in leader election @@ -73,6 +70,7 @@ limitations under the License. * [ZOOKEEPER-2024](https://issues.apache.org/jira/browse/ZOOKEEPER-2024) - Major throughput improvement with mixed workloads * [ZOOKEEPER-2040](https://issues.apache.org/jira/browse/ZOOKEEPER-2040) - Server to log underlying cause of SASL connection problems * [ZOOKEEPER-2079](https://issues.apache.org/jira/browse/ZOOKEEPER-2079) - Stop daemon with "kill" rather than "kill -9" +* [ZOOKEEPER-2083](https://issues.apache.org/jira/browse/ZOOKEEPER-2083) - Remove deprecated LE implementations * [ZOOKEEPER-2084](https://issues.apache.org/jira/browse/ZOOKEEPER-2084) - Document local session parameters * [ZOOKEEPER-2087](https://issues.apache.org/jira/browse/ZOOKEEPER-2087) - Few UX improvements in ZooInspector * [ZOOKEEPER-2098](https://issues.apache.org/jira/browse/ZOOKEEPER-2098) - QuorumCnxManager: use BufferedOutputStream for initial msg @@ -244,6 +242,7 @@ limitations under the License. * [ZOOKEEPER-3398](https://issues.apache.org/jira/browse/ZOOKEEPER-3398) - Learner.connectToLeader() may take too long to time-out * [ZOOKEEPER-3400](https://issues.apache.org/jira/browse/ZOOKEEPER-3400) - Add documentation on local sessions * [ZOOKEEPER-3402](https://issues.apache.org/jira/browse/ZOOKEEPER-3402) - Add a multiRead operation +* [ZOOKEEPER-3411](https://issues.apache.org/jira/browse/ZOOKEEPER-3411) - remove the deprecated CLI: ls2 and rmr * [ZOOKEEPER-3416](https://issues.apache.org/jira/browse/ZOOKEEPER-3416) - Remove redundant ServerCnxnFactoryAccessor * [ZOOKEEPER-3418](https://issues.apache.org/jira/browse/ZOOKEEPER-3418) - Improve quorum throughput through eager ACL checks of requests on local servers * [ZOOKEEPER-3423](https://issues.apache.org/jira/browse/ZOOKEEPER-3423) - use the maven-like way to ignore the generated version java files and doc the cmd:'./zkServer.sh version' @@ -284,7 +283,12 @@ limitations under the License. * [ZOOKEEPER-3620](https://issues.apache.org/jira/browse/ZOOKEEPER-3620) - Allow to override calls to System.exit in server side code * [ZOOKEEPER-3630](https://issues.apache.org/jira/browse/ZOOKEEPER-3630) - Autodetection of SSL library during Zookeeper C client build * [ZOOKEEPER-3636](https://issues.apache.org/jira/browse/ZOOKEEPER-3636) - find back the missing configuration property in the zookeeperAdmin page when moving from xml to markdown - +* [ZOOKEEPER-3638](https://issues.apache.org/jira/browse/ZOOKEEPER-3638) - Update Jetty to 9.4.24.v20191120 +* [ZOOKEEPER-3640](https://issues.apache.org/jira/browse/ZOOKEEPER-3640) - Implement "batch mode" in cli_mt +* [ZOOKEEPER-3648](https://issues.apache.org/jira/browse/ZOOKEEPER-3648) - remove Hadoop logo in the ZooKeeper documentation +* [ZOOKEEPER-3649](https://issues.apache.org/jira/browse/ZOOKEEPER-3649) - ls -s CLI need a line break + + ## Bug * [ZOOKEEPER-706](https://issues.apache.org/jira/browse/ZOOKEEPER-706) - large numbers of watches can cause session re-establishment to fail @@ -580,9 +584,10 @@ limitations under the License. * [ZOOKEEPER-3605](https://issues.apache.org/jira/browse/ZOOKEEPER-3605) - ZOOKEEPER-3242 add a connection throttle. Default constructor needs to set it * [ZOOKEEPER-3633](https://issues.apache.org/jira/browse/ZOOKEEPER-3633) - AdminServer commands throw NPE when only secure client port is used * [ZOOKEEPER-3641](https://issues.apache.org/jira/browse/ZOOKEEPER-3641) - New ZOO_VERSION define breaks Perl & Python contribs +* [ZOOKEEPER-3644](https://issues.apache.org/jira/browse/ZOOKEEPER-3644) - Data loss after upgrading standalone ZK server 3.4.14 to 3.5.6 with snapshot.trust.empty=true * [ZOOKEEPER-3651](https://issues.apache.org/jira/browse/ZOOKEEPER-3651) - NettyServerCnxnFactoryTest is flaky * [ZOOKEEPER-3653](https://issues.apache.org/jira/browse/ZOOKEEPER-3653) - Audit Log feature fails in a stand alone zookeeper setup - + ## Test * [ZOOKEEPER-1441](https://issues.apache.org/jira/browse/ZOOKEEPER-1441) - Some test cases are failing because Port bind issue. @@ -609,14 +614,15 @@ limitations under the License. * [ZOOKEEPER-3480](https://issues.apache.org/jira/browse/ZOOKEEPER-3480) - Flaky test CommitProcessorMetricsTest.testConcurrentRequestProcessingInCommitProcessor * [ZOOKEEPER-3495](https://issues.apache.org/jira/browse/ZOOKEEPER-3495) - Broken test in JDK12+: SnapshotDigestTest.testDifferentDigestVersion * [ZOOKEEPER-3542](https://issues.apache.org/jira/browse/ZOOKEEPER-3542) - X509UtilTest#testClientRenegotiationFails is flaky on JDK8 + linux on machines with 2 cores - + + ## Wish * [ZOOKEEPER-2795](https://issues.apache.org/jira/browse/ZOOKEEPER-2795) - Change log level for "ZKShutdownHandler is not registered" error message * [ZOOKEEPER-3073](https://issues.apache.org/jira/browse/ZOOKEEPER-3073) - fix couple of typos * [ZOOKEEPER-3302](https://issues.apache.org/jira/browse/ZOOKEEPER-3302) - ZooKeeper C client does not compile on Fedora 29 * [ZOOKEEPER-3421](https://issues.apache.org/jira/browse/ZOOKEEPER-3421) - Better insight into Observer connections - + ## Task * [ZOOKEEPER-925](https://issues.apache.org/jira/browse/ZOOKEEPER-925) - Consider maven site generation to replace our forrest site and documentation generation @@ -639,6 +645,7 @@ limitations under the License. * [ZOOKEEPER-3431](https://issues.apache.org/jira/browse/ZOOKEEPER-3431) - Enable BookKeeper checkstyle configuration * [ZOOKEEPER-3441](https://issues.apache.org/jira/browse/ZOOKEEPER-3441) - OWASP is flagging jackson-databind-2.9.9.jar for CVE-2019-12814 * [ZOOKEEPER-3545](https://issues.apache.org/jira/browse/ZOOKEEPER-3545) - Fix LICENSE files for netty dependency +* [ZOOKEEPER-3668](https://issues.apache.org/jira/browse/ZOOKEEPER-3668) - Clean up release package for 3.6.0 ## Sub-task @@ -709,7 +716,6 @@ limitations under the License. * [ZOOKEEPER-3275](https://issues.apache.org/jira/browse/ZOOKEEPER-3275) - Fix release targets: package, tar, mvn-deploy * [ZOOKEEPER-3285](https://issues.apache.org/jira/browse/ZOOKEEPER-3285) - Move assembly into its own sub-module * [ZOOKEEPER-3288](https://issues.apache.org/jira/browse/ZOOKEEPER-3288) - add a new doc:ZookeeperCLI.md -* [ZOOKEEPER-3301](https://issues.apache.org/jira/browse/ZOOKEEPER-3301) - Enforce the quota limit * [ZOOKEEPER-3305](https://issues.apache.org/jira/browse/ZOOKEEPER-3305) - Add Quorum Packet metrics * [ZOOKEEPER-3309](https://issues.apache.org/jira/browse/ZOOKEEPER-3309) - Add sync processor metrics * [ZOOKEEPER-3310](https://issues.apache.org/jira/browse/ZOOKEEPER-3310) - Add metrics for prep processor From fea678d7837ac0673a39dc10499fc8a526f351db Mon Sep 17 00:00:00 2001 From: Enrico Olivelli Date: Fri, 17 Jan 2020 17:08:56 +0100 Subject: [PATCH 014/118] ZOOKEEPER-3695: Source release tarball does not match repository in 3.6.0 - use the same configuration as in Apache Parent Pom to generate the source tarball - drop the custom made configuration for the source tarball Author: Enrico Olivelli Reviewers: andor@apache.org Closes #1226 from eolivelli/fix/ZOOKEEPER-3695-source-tarball and squashes the following commits: 781d95e5c [Enrico Olivelli] Fix typo 21eac30dd [Enrico Olivelli] Fix formatting and add comment 7eca978f6 [Enrico Olivelli] ZOOKEEPER-3695 Source release tarball does not match repository in 3.6.0 - use the same configuration as in Apache Parent Pom to generate the source tarball - drop the custom made configuration for the source tarball (cherry picked from commit 9053f7c431bb17ed79c2be129b6ba4ba18d15ab1) Signed-off-by: Andor Molnar --- pom.xml | 40 +++++- zookeeper-assembly/pom.xml | 17 +-- .../src/main/assembly/source-package.xml | 118 ------------------ 3 files changed, 40 insertions(+), 135 deletions(-) delete mode 100644 zookeeper-assembly/src/main/assembly/source-package.xml diff --git a/pom.xml b/pom.xml index c523e9d2295..d68b7c0d5a8 100755 --- a/pom.xml +++ b/pom.xml @@ -267,6 +267,45 @@ true + + apache-release + + + + org.apache.maven.plugins + maven-assembly-plugin + + + org.apache.apache.resources + apache-source-release-assembly-descriptor + 1.0.6 + + + + + source-release-assembly-tar-gz + package + + single + + + true + + + ${sourceReleaseAssemblyDescriptor} + + apache-zookeeper-${project.version} + + tar.gz + + posix + + + + + + + @@ -681,7 +720,6 @@ - org.apache.maven.plugins maven-antrun-plugin diff --git a/zookeeper-assembly/pom.xml b/zookeeper-assembly/pom.xml index 36964956c98..15210a36673 100755 --- a/zookeeper-assembly/pom.xml +++ b/zookeeper-assembly/pom.xml @@ -124,21 +124,6 @@ org.apache.maven.plugins maven-assembly-plugin - - source-assembly - package - - single - - - - ${project.basedir}/src/main/assembly/source-package.xml - - apache-zookeeper-${project.version} - false - posix - - bin-assembly package @@ -191,4 +176,4 @@ - \ No newline at end of file + diff --git a/zookeeper-assembly/src/main/assembly/source-package.xml b/zookeeper-assembly/src/main/assembly/source-package.xml deleted file mode 100644 index 15a5c68df79..00000000000 --- a/zookeeper-assembly/src/main/assembly/source-package.xml +++ /dev/null @@ -1,118 +0,0 @@ - - - source-package - - tar.gz - - true - - src/main/assembly/components.xml - - - - - ${project.basedir}/../zookeeper-assembly - - **/target/** - - zookeeper-assembly - ${rw.file.permission} - ${rwx.file.permission} - - - ${project.basedir}/../zookeeper-client - - **/target/** - - zookeeper-client - ${rw.file.permission} - ${rwx.file.permission} - - - ${project.basedir}/../zookeeper-contrib - - **/target/** - - zookeeper-contrib - ${rw.file.permission} - ${rwx.file.permission} - - - ${project.basedir}/../zookeeper-docs - - **/target/** - - zookeeper-docs - ${rw.file.permission} - ${rwx.file.permission} - - - ${project.basedir}/../zookeeper-it - - **/target/** - - zookeeper-it - ${rw.file.permission} - ${rwx.file.permission} - - - ${project.basedir}/../zookeeper-jute - - **/target/** - - zookeeper-jute - ${rw.file.permission} - ${rwx.file.permission} - - - ${project.basedir}/../zookeeper-recipes - - **/target/** - - zookeeper-recipes - ${rw.file.permission} - ${rwx.file.permission} - - - ${project.basedir}/../zookeeper-server - - **/target/** - - zookeeper-server - ${rw.file.permission} - ${rwx.file.permission} - - - ${project.basedir}/.. - . - - pom.xml - excludeFindBugsFilter.xml - owaspSuppressions.xml - checktyle.xml - checktyleSuppressions.xml - - ${rw.file.permission} - - - From b4d9e5730b44cad90c70c1271b895e7fdda5af06 Mon Sep 17 00:00:00 2001 From: Enrico Olivelli Date: Sat, 18 Jan 2020 11:06:01 -0800 Subject: [PATCH 015/118] ZOOKEEPER-3677: owasp checker failing for - CVE-2019-17571 Apache Log4j 1.2 deserialization of untrusted data in SocketServer Suppress error for CVE-2019-17571 as it does not affect us. We are not running the log4j server. Author: Enrico Olivelli Reviewers: phunt@apache.org Closes #1209 from eolivelli/fix/ZOOKEEPER-3677-owasp-log4j Change-Id: I0ef24a7b142cd32ccf4f5c18f9e0c0132a413d6c (cherry picked from commit 3bd6b1950eea1fabeac4bc477c8828939d008a4a) Signed-off-by: Patrick Hunt --- owaspSuppressions.xml | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/owaspSuppressions.xml b/owaspSuppressions.xml index 5c4bc33bcf5..ae94db494b9 100644 --- a/owaspSuppressions.xml +++ b/owaspSuppressions.xml @@ -41,4 +41,9 @@ this writing --> CVE-2019-3826 + + + CVE-2019-17571 + From 261f8b8b913fdf3fb42d0a0fc5763b51abd3f830 Mon Sep 17 00:00:00 2001 From: Sujith Simon Date: Mon, 20 Jan 2020 13:09:29 +0530 Subject: [PATCH 016/118] ZOOKEEPER-3667: Setting jute.maxbuffer value in hexadecimal throws Exception Author: sujithsimon22 Reviewers: Mohammad Arshad Closes #1222 from sujithsimon22/3667 (cherry picked from commit 49ad75b18bfe26e853050f5add6f10f567399058) Signed-off-by: Mohammad Arshad --- .../org/apache/zookeeper/common/ZKConfig.java | 2 +- .../zookeeper/client/ZKClientConfigTest.java | 25 +++++++++++++++++++ 2 files changed, 26 insertions(+), 1 deletion(-) diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/common/ZKConfig.java b/zookeeper-server/src/main/java/org/apache/zookeeper/common/ZKConfig.java index d7bed7024f7..11cdc06bfd0 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/common/ZKConfig.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/common/ZKConfig.java @@ -273,7 +273,7 @@ public boolean getBoolean(String key, boolean defaultValue) { public int getInt(String key, int defaultValue) { String value = getProperty(key); if (value != null) { - return Integer.parseInt(value.trim()); + return Integer.decode(value.trim()); } return defaultValue; } diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/client/ZKClientConfigTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/client/ZKClientConfigTest.java index e0c25a104b9..9a9b6223e00 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/client/ZKClientConfigTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/client/ZKClientConfigTest.java @@ -185,4 +185,29 @@ public void testIntegerRetrievalFromProperty() { assertEquals(value, result); } + @Test + public void testIntegerRetrievalFromHexadecimalProperty() { + int hexaValue = 0x3000000; + String wrongValue = "0xwel"; + int defaultValue = 100; + // property is set in hexadecimal value + ZKClientConfig zkClientConfig = new ZKClientConfig(); + zkClientConfig.setProperty(ZKConfig.JUTE_MAXBUFFER, + Integer.toString(hexaValue)); + int result = zkClientConfig.getInt(ZKConfig.JUTE_MAXBUFFER, defaultValue); + assertEquals(result, hexaValue); + zkClientConfig.setProperty(ZKConfig.JUTE_MAXBUFFER, + wrongValue); + try { + result = zkClientConfig.getInt(ZKConfig.JUTE_MAXBUFFER, defaultValue); + fail("NumberFormatException is expected"); + } catch (NumberFormatException exception) { + // do nothing + } + zkClientConfig.setProperty(ZKConfig.JUTE_MAXBUFFER, + " " + hexaValue + " "); + result = zkClientConfig.getInt(ZKConfig.JUTE_MAXBUFFER, defaultValue); + assertEquals(result, hexaValue); + } + } From 1066de70f21e31f394be57b970d6d4b079a04e4d Mon Sep 17 00:00:00 2001 From: Sujith Simon Date: Mon, 20 Jan 2020 13:36:59 +0530 Subject: [PATCH 017/118] ZOOKEEPER-3613: ZKConfig fails to return proper value on getBoolean()when user accidentally includes spaces at the end of the value Author: sujithsimon22 Reviewers: Allan Lyu ,Justin Mao Ling ,Mohammad Arshad Closes #1190 from sujithsimon22/3613 (cherry picked from commit dcef1a6f33d8120a93f011da1870a503aafb0312) Signed-off-by: Mohammad Arshad --- .../org/apache/zookeeper/common/ZKConfig.java | 2 +- .../apache/zookeeper/common/ZKConfigTest.java | 96 +++++++++++++++++++ 2 files changed, 97 insertions(+), 1 deletion(-) create mode 100644 zookeeper-server/src/test/java/org/apache/zookeeper/common/ZKConfigTest.java diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/common/ZKConfig.java b/zookeeper-server/src/main/java/org/apache/zookeeper/common/ZKConfig.java index 11cdc06bfd0..6bbe698e3d2 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/common/ZKConfig.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/common/ZKConfig.java @@ -253,7 +253,7 @@ public boolean getBoolean(String key, boolean defaultValue) { if (propertyValue == null) { return defaultValue; } else { - return Boolean.parseBoolean(propertyValue); + return Boolean.parseBoolean(propertyValue.trim()); } } diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/common/ZKConfigTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/common/ZKConfigTest.java new file mode 100644 index 00000000000..eff1bfb3d29 --- /dev/null +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/common/ZKConfigTest.java @@ -0,0 +1,96 @@ +/* + * 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.common; + + +import static org.junit.Assert.assertEquals; +import java.util.concurrent.TimeUnit; +import org.junit.After; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.Timeout; + + + +public class ZKConfigTest { + + X509Util x509Util = new ClientX509Util(); + + @Rule + public Timeout timeout = new Timeout(10, TimeUnit.SECONDS); + + @After + public void tearDown() throws Exception { + System.clearProperty(x509Util.getSslProtocolProperty()); + } + + // property is not set we should get the default value + @Test + public void testBooleanRetrievalFromPropertyDefault() { + ZKConfig conf = new ZKConfig(); + String prop = "UnSetProperty" + System.currentTimeMillis(); + boolean defaultValue = false; + boolean result = conf.getBoolean(prop, defaultValue); + assertEquals(defaultValue, result); + } + + // property is set to an valid boolean, we should get the set value + @Test + public void testBooleanRetrievalFromProperty() { + boolean value = true; + boolean defaultValue = false; + System.setProperty(x509Util.getSslProtocolProperty(), Boolean.toString(value)); + ZKConfig conf = new ZKConfig(); + boolean result = conf.getBoolean(x509Util.getSslProtocolProperty(), defaultValue); + assertEquals(value, result); + } + + // property is set but with white spaces in the beginning + @Test + public void testBooleanRetrievalFromPropertyWithWhitespacesInBeginning() { + boolean value = true; + boolean defaultValue = false; + System.setProperty(x509Util.getSslProtocolProperty(), " " + value); + ZKConfig conf = new ZKConfig(); + boolean result = conf.getBoolean(x509Util.getSslProtocolProperty(), defaultValue); + assertEquals(value, result); + } + + // property is set but with white spaces at the end + @Test + public void testBooleanRetrievalFromPropertyWithWhitespacesAtEnd() { + boolean value = true; + boolean defaultValue = false; + System.setProperty(x509Util.getSslProtocolProperty(), value + " "); + ZKConfig conf = new ZKConfig(); + boolean result = conf.getBoolean(x509Util.getSslProtocolProperty(), defaultValue); + assertEquals(value, result); + } + + // property is set but with white spaces at the beginning and the end + @Test + public void testBooleanRetrievalFromPropertyWithWhitespacesAtBeginningAndEnd() { + boolean value = true; + boolean defaultValue = false; + System.setProperty(x509Util.getSslProtocolProperty(), " " + value + " "); + ZKConfig conf = new ZKConfig(); + boolean result = conf.getBoolean(x509Util.getSslProtocolProperty(), defaultValue); + assertEquals(value, result); + } +} From ad425cea0fae18619653154a48a7699b8f40ed0a Mon Sep 17 00:00:00 2001 From: David Mollitor Date: Tue, 21 Jan 2020 17:53:18 +0100 Subject: [PATCH 018/118] ZOOKEEPER-3669: Use switch Statement in ClientCnxn SendThread Author: David Mollitor Reviewers: fangmin@apache.org, andor@apache.org Closes #1197 from belugabehr/ZOOKEEPER-3669 (cherry picked from commit 517ecde9dd3cfe574118231ec242ea73b80a4e79) Signed-off-by: Andor Molnar --- .../java/org/apache/zookeeper/ClientCnxn.java | 34 +++++++++---------- 1 file changed, 17 insertions(+), 17 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 3859382fadd..d3d8e853a52 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/ClientCnxn.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/ClientCnxn.java @@ -867,24 +867,24 @@ void readResponse(ByteBuffer incomingBuffer) throws IOException { ReplyHeader replyHdr = new ReplyHeader(); replyHdr.deserialize(bbia, "header"); - if (replyHdr.getXid() == PING_XID) { - LOG.debug( - "Got ping response for session id: 0x{} after {}ms.", + switch (replyHdr.getXid()) { + case PING_XID: + LOG.debug("Got ping response for session id: 0x{} after {}ms.", Long.toHexString(sessionId), ((System.nanoTime() - lastPingSentNs) / 1000000)); return; - } - if (replyHdr.getXid() == AUTHPACKET_XID) { + case AUTHPACKET_XID: + LOG.debug("Got auth session id: 0x{}", Long.toHexString(sessionId)); if (replyHdr.getErr() == KeeperException.Code.AUTHFAILED.intValue()) { state = States.AUTH_FAILED; - eventThread.queueEvent(new WatchedEvent(Watcher.Event.EventType.None, Watcher.Event.KeeperState.AuthFailed, null)); + eventThread.queueEvent(new WatchedEvent(Watcher.Event.EventType.None, + Watcher.Event.KeeperState.AuthFailed, null)); eventThread.queueEventOfDeath(); } - LOG.debug("Got auth session id: 0x{}", Long.toHexString(sessionId)); - return; - } - if (replyHdr.getXid() == NOTIFICATION_XID) { - LOG.debug("Got notification session id: 0x{}", Long.toHexString(sessionId)); + return; + case NOTIFICATION_XID: + LOG.debug("Got notification session id: 0x{}", + Long.toHexString(sessionId)); WatcherEvent event = new WatcherEvent(); event.deserialize(bbia, "response"); @@ -895,18 +895,18 @@ void readResponse(ByteBuffer incomingBuffer) throws IOException { event.setPath("/"); } else if (serverPath.length() > chrootPath.length()) { event.setPath(serverPath.substring(chrootPath.length())); - } else { - LOG.warn( - "Got server path {} which is too short for chroot path {}.", - event.getPath(), - chrootPath); - } + } else { + LOG.warn("Got server path {} which is too short for chroot path {}.", + event.getPath(), chrootPath); + } } WatchedEvent we = new WatchedEvent(event); LOG.debug("Got {} for session id 0x{}", we, Long.toHexString(sessionId)); eventThread.queueEvent(we); return; + default: + break; } // If SASL authentication is currently in progress, construct and From cf2e5d1cbdf836cf5cf7e491057d78c85cd6d3ac Mon Sep 17 00:00:00 2001 From: Mate Szalay-Beko Date: Wed, 22 Jan 2020 11:08:42 +0100 Subject: [PATCH 019/118] ZOOKEEPER-3482: add unit tests for client SASL authentication over SSL We had some issues about configuring Kerberos authentication with SSL, so I created some unit tests to verify that these features work together. The problem was originally reported on 3.5.5. There was some conflicts to backport the tests to 3.5, I will prepare a separate PR on that branch. Author: Mate Szalay-Beko Author: Mate Szalay-Beko Reviewers: eolivelli@apache.org, andor@apache.org Closes #1204 from symat/ZOOKEEPER-3482-master and squashes the following commits: 53194f564 [Mate Szalay-Beko] ZOOKEEPER-3482: update SASL related documentation 576ac0b21 [Mate Szalay-Beko] Merge remote-tracking branch 'apache/master' into ZOOKEEPER-3482-master 28da105c2 [Mate Szalay-Beko] ZOOKEEPER-3482: reload SASL and Kerberos configs before executing the tests 396c73e8a [Mate Szalay-Beko] ZOOKEEPER-3482: add unit tests for client SASL authentication over SSL (cherry picked from commit b7dd0e49c5b25fffc1be8e619a3164bc680ef8bd) Signed-off-by: Andor Molnar --- .../main/resources/markdown/zookeeperAdmin.md | 27 +- .../markdown/zookeeperProgrammers.md | 27 +- .../server/quorum/auth/KerberosTestUtils.java | 8 + .../quorum/auth/QuorumAuthTestBase.java | 5 + .../quorum/auth/QuorumKerberosAuthTest.java | 6 + .../auth/QuorumKerberosHostBasedAuthTest.java | 7 + .../test/SaslDigestAuthOverSSLTest.java | 152 +++++++++++ .../test/SaslKerberosAuthOverSSLTest.java | 246 ++++++++++++++++++ 8 files changed, 468 insertions(+), 10 deletions(-) create mode 100644 zookeeper-server/src/test/java/org/apache/zookeeper/test/SaslDigestAuthOverSSLTest.java create mode 100644 zookeeper-server/src/test/java/org/apache/zookeeper/test/SaslKerberosAuthOverSSLTest.java diff --git a/zookeeper-docs/src/main/resources/markdown/zookeeperAdmin.md b/zookeeper-docs/src/main/resources/markdown/zookeeperAdmin.md index 7776f6431ca..1f41230a8e5 100644 --- a/zookeeper-docs/src/main/resources/markdown/zookeeperAdmin.md +++ b/zookeeper-docs/src/main/resources/markdown/zookeeperAdmin.md @@ -1298,6 +1298,11 @@ 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), +and [SASL authentication for ZooKeeper](https://cwiki.apache.org/confluence/display/ZOOKEEPER/ZooKeeper+and+SASL). + * *DigestAuthenticationProvider.superDigest* : (Java system property: **zookeeper.DigestAuthenticationProvider.superDigest**) By default this feature is **disabled** @@ -1461,11 +1466,29 @@ encryption/authentication/authorization performed by the service. TBD * *client.portUnification*: - (Java system properties: **zookeeper.client.portUnification**) + (Java system property: **zookeeper.client.portUnification**) 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. + (e.g. the zk/myhost@EXAMPLE.COM client principal will be authenticated in ZooKeeper as zk/myhost) + Default: false + diff --git a/zookeeper-docs/src/main/resources/markdown/zookeeperProgrammers.md b/zookeeper-docs/src/main/resources/markdown/zookeeperProgrammers.md index 08d30c8158a..5d071590d8f 100644 --- a/zookeeper-docs/src/main/resources/markdown/zookeeperProgrammers.md +++ b/zookeeper-docs/src/main/resources/markdown/zookeeperProgrammers.md @@ -1276,8 +1276,11 @@ handle is undefined behavior and should be avoided. The following list contains configuration properties for the Java client. You can set any of these properties using Java system properties. For server properties, please check the -following reference -[Server configuration section.](zookeeperAdmin.html#sc_configuration) +[Server configuration section of the Admin Guide](zookeeperAdmin.html#sc_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). + * *zookeeper.sasl.client* : Set the value to **false** to disable @@ -1286,6 +1289,13 @@ following reference * *zookeeper.sasl.clientconfig* : Specifies the context key in the JAAS login file. Default is "Client". +* *zookeeper.server.principal* : + Specifies the server principal to be used by the client for authentication, while connecting to the zookeeper + server, when Kerberos authentication is enabled. If this configuration is provided, then + the ZooKeeper client will NOT USE any of the following parameters to determine the server principal: + zookeeper.sasl.client.username, zookeeper.sasl.client.canonicalize.hostname, zookeeper.server.realm + Note: this config parameter is working only for ZooKeeper 3.5.7+, 3.6.0+ + * *zookeeper.sasl.client.username* : Traditionally, a principal is divided into three parts: the primary, the instance, and the realm. The format of a typical Kerberos V5 principal is primary/instance@REALM. @@ -1294,6 +1304,13 @@ following reference username/IP@realm, where username is the value of zookeeper.sasl.client.username, IP is the server IP, and realm is the value of zookeeper.server.realm. +* *zookeeper.sasl.client.canonicalize.hostname* : + Expecting the zookeeper.server.principal parameter is not provided, the ZooKeeper client will try to + determine the 'instance' (host) part of the ZooKeeper server principal. First it takes the hostname provided + as the ZooKeeper server connection string. Then it tries to 'canonicalize' the address by getting + the fully qualified domain name belonging to the address. You can disable this 'canonicalization' + by setting: zookeeper.sasl.client.canonicalize.hostname=false + * *zookeeper.server.realm* : Realm part of the server principal. By default it is the client principal realm. @@ -1337,12 +1354,6 @@ following reference * *zookeeper.kinit* : Specifies path to kinit binary. Default is "/usr/bin/kinit". -* *zookeeper.server.principal* : - Specifies the server principal to be used by the client for authentication, while connecting to the zookeeper - server, when Kerberos authentication is enabled. A couple of ways to specify the server principal can be as - "zookeeper.server.principal = **zookeeper/zookeeper.apache.org@APACHE.ORG**" or - "zookeeper.server.principal = **zookeeper/zookeeper.apache.org**" - ### C Binding diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/auth/KerberosTestUtils.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/auth/KerberosTestUtils.java index 30dd3e4db82..755712b79f9 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/auth/KerberosTestUtils.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/auth/KerberosTestUtils.java @@ -38,6 +38,14 @@ public static String getServerPrincipal() { return "zkquorum/localhost@EXAMPLE.COM"; } + public static String getClientPrincipal() { + return getClientUsername() + "/localhost@EXAMPLE.COM"; + } + + public static String getClientUsername() { + return "zkclient"; + } + public static String getHostLearnerPrincipal() { return "learner/_HOST@EXAMPLE.COM"; } diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/auth/QuorumAuthTestBase.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/auth/QuorumAuthTestBase.java index 3b52cc009d7..97650d0c694 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/auth/QuorumAuthTestBase.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/auth/QuorumAuthTestBase.java @@ -25,6 +25,7 @@ import java.util.ArrayList; import java.util.List; import java.util.Map; +import javax.security.auth.login.Configuration; import org.apache.commons.io.FileUtils; import org.apache.zookeeper.PortAssignment; import org.apache.zookeeper.ZKTestCase; @@ -56,6 +57,10 @@ public static void setupJaasConfig(String jaasEntries) { // could not create tmp directory to hold JAAS conf file : test will // fail now. } + + // refresh the SASL configuration in this JVM (making sure that we use the latest config + // even if other tests already have been executed and initialized the SASL configs before) + Configuration.getConfiguration().refresh(); } public static void cleanupJaasConfig() { diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/auth/QuorumKerberosAuthTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/auth/QuorumKerberosAuthTest.java index b9d662a16e2..e4534dfd21f 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/auth/QuorumKerberosAuthTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/auth/QuorumKerberosAuthTest.java @@ -40,6 +40,10 @@ public class QuorumKerberosAuthTest extends KerberosSecurityTestcase { static { String keytabFilePath = FilenameUtils.normalize(KerberosTestUtils.getKeytabFile(), true); + + // note: we use "refreshKrb5Config=true" to refresh the kerberos config in the JVM, + // making sure that we use the latest config even if other tests already have been executed + // and initialized the kerberos client configs before) String jaasEntries = "" + "QuorumServer {\n" + " com.sun.security.auth.module.Krb5LoginModule required\n" @@ -50,6 +54,7 @@ public class QuorumKerberosAuthTest extends KerberosSecurityTestcase { + " storeKey=true\n" + " useTicketCache=false\n" + " debug=false\n" + + " refreshKrb5Config=true\n" + " principal=\"" + KerberosTestUtils.getServerPrincipal() + "\";\n" @@ -63,6 +68,7 @@ public class QuorumKerberosAuthTest extends KerberosSecurityTestcase { + " storeKey=true\n" + " useTicketCache=false\n" + " debug=false\n" + + " refreshKrb5Config=true\n" + " principal=\"" + KerberosTestUtils.getLearnerPrincipal() + "\";\n" diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/auth/QuorumKerberosHostBasedAuthTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/auth/QuorumKerberosHostBasedAuthTest.java index 93867550dcf..fc150490e61 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/auth/QuorumKerberosHostBasedAuthTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/auth/QuorumKerberosHostBasedAuthTest.java @@ -53,6 +53,10 @@ private static void setupJaasConfigEntries( String hostLearnerPrincipal, String hostNamedLearnerPrincipal) { String keytabFilePath = FilenameUtils.normalize(KerberosTestUtils.getKeytabFile(), true); + + // note: we use "refreshKrb5Config=true" to refresh the kerberos config in the JVM, + // making sure that we use the latest config even if other tests already have been executed + // and initialized the kerberos client configs before) String jaasEntries = "QuorumServer {\n" + " com.sun.security.auth.module.Krb5LoginModule required\n" + " useKeyTab=true\n" @@ -61,6 +65,7 @@ private static void setupJaasConfigEntries( + " storeKey=true\n" + " useTicketCache=false\n" + " debug=false\n" + + " refreshKrb5Config=true\n" + " principal=\"" + KerberosTestUtils.replaceHostPattern(hostServerPrincipal) + "\";\n" + "};\n" @@ -72,6 +77,7 @@ private static void setupJaasConfigEntries( + " storeKey=true\n" + " useTicketCache=false\n" + " debug=false\n" + + " refreshKrb5Config=true\n" + " principal=\"" + KerberosTestUtils.replaceHostPattern(hostLearnerPrincipal) + "\";\n" + "};\n" @@ -83,6 +89,7 @@ private static void setupJaasConfigEntries( + " storeKey=true\n" + " useTicketCache=false\n" + " debug=false\n" + + " refreshKrb5Config=true\n" + " principal=\"" + hostNamedLearnerPrincipal + "\";\n" + "};\n"; diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/test/SaslDigestAuthOverSSLTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/test/SaslDigestAuthOverSSLTest.java new file mode 100644 index 00000000000..09d92978ca1 --- /dev/null +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/test/SaslDigestAuthOverSSLTest.java @@ -0,0 +1,152 @@ +/* + * 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.client.ZKClientConfig.LOGIN_CONTEXT_NAME_KEY; +import static org.junit.Assert.fail; +import java.io.File; +import java.io.FileWriter; +import java.io.IOException; +import java.io.PrintWriter; +import java.net.InetSocketAddress; +import javax.security.auth.login.Configuration; +import org.apache.commons.io.FileUtils; +import org.apache.zookeeper.CreateMode; +import org.apache.zookeeper.Environment; +import org.apache.zookeeper.KeeperException; +import org.apache.zookeeper.PortAssignment; +import org.apache.zookeeper.ZooDefs.Ids; +import org.apache.zookeeper.ZooKeeper; +import org.apache.zookeeper.client.ZKClientConfig; +import org.apache.zookeeper.common.ClientX509Util; +import org.apache.zookeeper.server.ServerCnxnFactory; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + + +public class SaslDigestAuthOverSSLTest extends ClientBase { + + private ClientX509Util clientX509Util; + private File saslConfFile; + + @Before + @Override + public void setUp() throws Exception { + initSaslConfig(); + clientX509Util = setUpSSLWithNoAuth(); + + String host = "localhost"; + int port = PortAssignment.unique(); + hostPort = host + ":" + port; + + serverFactory = ServerCnxnFactory.createFactory(); + serverFactory.configure(new InetSocketAddress(host, port), maxCnxns, -1, true); + + super.setUp(); + } + + + @After + @Override + public void tearDown() throws Exception { + super.tearDown(); + clearSslSetting(clientX509Util); + clearSaslConfig(); + } + + + @Test + public void testAuth() throws Exception { + ZooKeeper zk = createClient(); + try { + zk.create("/path1", null, Ids.CREATOR_ALL_ACL, CreateMode.PERSISTENT); + Thread.sleep(1000); + } catch (KeeperException e) { + fail("test failed :" + e); + } finally { + zk.close(); + } + } + + + public void initSaslConfig() { + System.setProperty("zookeeper.authProvider.1", "org.apache.zookeeper.server.auth.SASLAuthenticationProvider"); + System.setProperty(LOGIN_CONTEXT_NAME_KEY, "ClientUsingDigest"); + try { + File tmpDir = createTmpDir(); + saslConfFile = new File(tmpDir, "jaas.conf"); + PrintWriter saslConf = new PrintWriter(new FileWriter(saslConfFile)); + saslConf.println("Server {"); + saslConf.println("org.apache.zookeeper.server.auth.DigestLoginModule required"); + saslConf.println("user_super=\"test\";"); + saslConf.println("};"); + saslConf.println("ClientUsingDigest {"); + saslConf.println("org.apache.zookeeper.server.auth.DigestLoginModule required"); + saslConf.println("username=\"super\""); + saslConf.println("password=\"test\";"); + saslConf.println("};"); + saslConf.close(); + System.setProperty(Environment.JAAS_CONF_KEY, saslConfFile.getAbsolutePath()); + } catch (IOException e) { + LOG.error("could not create tmp directory to hold JAAS conf file, test will fail...", e); + } + + // refresh the SASL configuration in this JVM (making sure that we use the latest config + // even if other tests already have been executed and initialized the SASL configs before) + Configuration.getConfiguration().refresh(); + } + + public void clearSaslConfig() { + FileUtils.deleteQuietly(saslConfFile); + System.clearProperty(Environment.JAAS_CONF_KEY); + System.clearProperty("zookeeper.authProvider.1"); + } + + public ClientX509Util setUpSSLWithNoAuth() { + String testDataPath = System.getProperty("test.data.dir", "src/test/resources/data"); + System.setProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN_FACTORY, "org.apache.zookeeper.server.NettyServerCnxnFactory"); + System.setProperty(ZKClientConfig.ZOOKEEPER_CLIENT_CNXN_SOCKET, "org.apache.zookeeper.ClientCnxnSocketNetty"); + System.setProperty(ZKClientConfig.SECURE_CLIENT, "true"); + System.setProperty("zookeeper.ssl.clientAuth", "none"); + System.setProperty("zookeeper.ssl.quorum.clientAuth", "none"); + + ClientX509Util x509Util = new ClientX509Util(); + System.setProperty(x509Util.getSslTruststoreLocationProperty(), testDataPath + "/ssl/testTrustStore.jks"); + System.setProperty(x509Util.getSslTruststorePasswdProperty(), "testpass"); + System.setProperty(x509Util.getSslKeystoreLocationProperty(), testDataPath + "/ssl/testKeyStore.jks"); + System.setProperty(x509Util.getSslKeystorePasswdProperty(), "testpass"); + + return x509Util; + } + + public void clearSslSetting(ClientX509Util clientX509Util) { + System.clearProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN_FACTORY); + System.clearProperty(ZKClientConfig.ZOOKEEPER_CLIENT_CNXN_SOCKET); + System.clearProperty(ZKClientConfig.SECURE_CLIENT); + System.clearProperty(clientX509Util.getSslTruststoreLocationProperty()); + System.clearProperty(clientX509Util.getSslTruststorePasswdProperty()); + System.clearProperty(clientX509Util.getSslKeystoreLocationProperty()); + System.clearProperty(clientX509Util.getSslKeystorePasswdProperty()); + System.clearProperty("zookeeper.ssl.clientAuth"); + System.clearProperty("zookeeper.ssl.quorum.clientAuth"); + clientX509Util.close(); + } + +} diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/test/SaslKerberosAuthOverSSLTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/test/SaslKerberosAuthOverSSLTest.java new file mode 100644 index 00000000000..d466c8b8d39 --- /dev/null +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/test/SaslKerberosAuthOverSSLTest.java @@ -0,0 +1,246 @@ +/* + * 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.client.ZKClientConfig.ENABLE_CLIENT_SASL_KEY; +import static org.apache.zookeeper.client.ZKClientConfig.LOGIN_CONTEXT_NAME_KEY; +import static org.apache.zookeeper.client.ZKClientConfig.ZK_SASL_CLIENT_USERNAME; +import static org.apache.zookeeper.client.ZKClientConfig.ZOOKEEPER_SERVER_PRINCIPAL; +import static org.apache.zookeeper.client.ZKClientConfig.ZOOKEEPER_SERVER_REALM; +import static org.junit.Assert.fail; +import java.io.File; +import java.io.FileWriter; +import java.io.IOException; +import java.io.PrintWriter; +import java.net.InetSocketAddress; +import java.util.Properties; +import javax.security.auth.login.Configuration; +import org.apache.commons.io.FileUtils; +import org.apache.zookeeper.CreateMode; +import org.apache.zookeeper.Environment; +import org.apache.zookeeper.KeeperException; +import org.apache.zookeeper.PortAssignment; +import org.apache.zookeeper.ZooDefs.Ids; +import org.apache.zookeeper.ZooKeeper; +import org.apache.zookeeper.client.ZKClientConfig; +import org.apache.zookeeper.common.ClientX509Util; +import org.apache.zookeeper.server.ServerCnxnFactory; +import org.apache.zookeeper.server.quorum.auth.KerberosTestUtils; +import org.apache.zookeeper.server.quorum.auth.MiniKdc; +import org.junit.After; +import org.junit.AfterClass; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Test; + +public class SaslKerberosAuthOverSSLTest extends ClientBase { + + private ClientX509Util clientX509Util; + private File keytabFileForKerberosPrincipals; + private File saslConfFile; + + private static MiniKdc kdc; + private static File kdcWorkDir; + private static Properties conf; + + + + @BeforeClass + public static void setupKdc() { + startMiniKdc(); + } + + @AfterClass + public static void tearDownKdc() { + stopMiniKdc(); + FileUtils.deleteQuietly(kdcWorkDir); + } + + + + @Before + @Override + public void setUp() throws Exception { + initSaslConfig(); + clientX509Util = setUpSSLWithNoAuth(); + + String host = "localhost"; + int port = PortAssignment.unique(); + hostPort = host + ":" + port; + + serverFactory = ServerCnxnFactory.createFactory(); + serverFactory.configure(new InetSocketAddress(host, port), maxCnxns, -1, true); + + super.setUp(); + } + + + @After + @Override + public void tearDown() throws Exception { + super.tearDown(); + clearSslSetting(clientX509Util); + clearSaslConfig(); + } + + + @Test + public void testAuth() throws Exception { + ZooKeeper zk = createClient(); + try { + zk.create("/path1", null, Ids.CREATOR_ALL_ACL, CreateMode.PERSISTENT); + Thread.sleep(1000); + } catch (KeeperException e) { + fail("test failed :" + e); + } finally { + zk.close(); + } + } + + + public void initSaslConfig() throws Exception { + + // registering the server and client users in the KDC mini server + keytabFileForKerberosPrincipals = new File(KerberosTestUtils.getKeytabFile()); + String clientPrincipal = KerberosTestUtils.getClientPrincipal(); + String serverPrincipal = KerberosTestUtils.getServerPrincipal(); + clientPrincipal = clientPrincipal.substring(0, clientPrincipal.lastIndexOf("@")); + serverPrincipal = serverPrincipal.substring(0, serverPrincipal.lastIndexOf("@")); + kdc.createPrincipal(keytabFileForKerberosPrincipals, clientPrincipal, serverPrincipal); + + // client-side SASL config + System.setProperty(ZOOKEEPER_SERVER_PRINCIPAL, KerberosTestUtils.getServerPrincipal()); + System.setProperty(ENABLE_CLIENT_SASL_KEY, "true"); + System.setProperty(ZOOKEEPER_SERVER_REALM, KerberosTestUtils.getRealm()); + System.setProperty(LOGIN_CONTEXT_NAME_KEY, "ClientUsingKerberos"); + + // server side SASL config + System.setProperty("zookeeper.authProvider.1", "org.apache.zookeeper.server.auth.SASLAuthenticationProvider"); + System.setProperty(SaslTestUtil.requireSASLAuthProperty, "true"); + + // generating the SASL config to use (contains sections both for the client and the server) + // note: we use "refreshKrb5Config=true" to refresh the kerberos config in the JVM, + // making sure that we use the latest config even if other tests already have been executed + // and initialized the kerberos client configs before) + try { + File tmpDir = createTmpDir(); + saslConfFile = new File(tmpDir, "jaas.conf"); + PrintWriter saslConf = new PrintWriter(new FileWriter(saslConfFile)); + saslConf.println("Server {"); + saslConf.println(" com.sun.security.auth.module.Krb5LoginModule required"); + saslConf.println(" storeKey=\"true\""); + saslConf.println(" useTicketCache=\"false\""); + saslConf.println(" useKeyTab=\"true\""); + saslConf.println(" doNotPrompt=\"true\""); + saslConf.println(" debug=\"true\""); + saslConf.println(" refreshKrb5Config=\"true\""); + saslConf.println(" keyTab=\"" + keytabFileForKerberosPrincipals.getAbsolutePath() + "\""); + saslConf.println(" principal=\"" + KerberosTestUtils.getServerPrincipal() + "\";"); + saslConf.println("};"); + saslConf.println("ClientUsingKerberos {"); + saslConf.println(" com.sun.security.auth.module.Krb5LoginModule required"); + saslConf.println(" storeKey=\"false\""); + saslConf.println(" useTicketCache=\"false\""); + saslConf.println(" useKeyTab=\"true\""); + saslConf.println(" doNotPrompt=\"true\""); + saslConf.println(" debug=\"true\""); + saslConf.println(" refreshKrb5Config=\"true\""); + saslConf.println(" keyTab=\"" + keytabFileForKerberosPrincipals.getAbsolutePath() + "\""); + saslConf.println(" principal=\"" + KerberosTestUtils.getClientPrincipal() + "\";"); + saslConf.println("};"); + saslConf.close(); + System.setProperty(Environment.JAAS_CONF_KEY, saslConfFile.getAbsolutePath()); + + } catch (IOException e) { + LOG.error("could not create tmp directory to hold JAAS conf file, test will fail...", e); + } + + // refresh the SASL configuration in this JVM (making sure that we use the latest config + // even if other tests already have been executed and initialized the SASL configs before) + Configuration.getConfiguration().refresh(); + } + + public void clearSaslConfig() { + FileUtils.deleteQuietly(keytabFileForKerberosPrincipals); + FileUtils.deleteQuietly(saslConfFile); + + System.clearProperty(Environment.JAAS_CONF_KEY); + System.clearProperty(ZK_SASL_CLIENT_USERNAME); + System.clearProperty(ENABLE_CLIENT_SASL_KEY); + System.clearProperty(LOGIN_CONTEXT_NAME_KEY); + System.clearProperty("zookeeper.authProvider.1"); + + System.clearProperty(SaslTestUtil.requireSASLAuthProperty); + System.clearProperty(ZOOKEEPER_SERVER_PRINCIPAL); + } + + public ClientX509Util setUpSSLWithNoAuth() { + String testDataPath = System.getProperty("test.data.dir", "src/test/resources/data"); + System.setProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN_FACTORY, "org.apache.zookeeper.server.NettyServerCnxnFactory"); + System.setProperty(ZKClientConfig.ZOOKEEPER_CLIENT_CNXN_SOCKET, "org.apache.zookeeper.ClientCnxnSocketNetty"); + System.setProperty(ZKClientConfig.SECURE_CLIENT, "true"); + System.setProperty("zookeeper.ssl.clientAuth", "none"); + System.setProperty("zookeeper.ssl.quorum.clientAuth", "none"); + + ClientX509Util x509Util = new ClientX509Util(); + System.setProperty(x509Util.getSslTruststoreLocationProperty(), testDataPath + "/ssl/testTrustStore.jks"); + System.setProperty(x509Util.getSslTruststorePasswdProperty(), "testpass"); + System.setProperty(x509Util.getSslKeystoreLocationProperty(), testDataPath + "/ssl/testKeyStore.jks"); + System.setProperty(x509Util.getSslKeystorePasswdProperty(), "testpass"); + + return x509Util; + } + + public void clearSslSetting(ClientX509Util clientX509Util) { + System.clearProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN_FACTORY); + System.clearProperty(ZKClientConfig.ZOOKEEPER_CLIENT_CNXN_SOCKET); + System.clearProperty(ZKClientConfig.SECURE_CLIENT); + System.clearProperty(clientX509Util.getSslTruststoreLocationProperty()); + System.clearProperty(clientX509Util.getSslTruststorePasswdProperty()); + System.clearProperty(clientX509Util.getSslKeystoreLocationProperty()); + System.clearProperty(clientX509Util.getSslKeystorePasswdProperty()); + System.clearProperty("javax.net.debug"); + System.clearProperty("zookeeper.ssl.clientAuth"); + System.clearProperty("zookeeper.ssl.quorum.clientAuth"); + clientX509Util.close(); + } + + + + public static void startMiniKdc() { + try { + kdcWorkDir = createEmptyTestDir(); + conf = MiniKdc.createConf(); + conf.setProperty("debug", "true"); + + kdc = new MiniKdc(conf, kdcWorkDir); + kdc.start(); + } catch (Exception e) { + throw new RuntimeException("failed to start MiniKdc", e); + } + + } + + public static void stopMiniKdc() { + if (kdc != null) { + kdc.stop(); + } + } + +} From b0debb06dd88e6097021160c9effe3cbfbd3d5f3 Mon Sep 17 00:00:00 2001 From: Patrick Hunt Date: Thu, 23 Jan 2020 11:14:39 +0100 Subject: [PATCH 020/118] ZOOKEEPER-3699: upgrade jackson-databind to address CVE-2019-20330 Latest version of jackson-databind is 2.9.10.2. Change-Id: Id2b0f17c2dfa9a9765fd4893643007b49f06816d Author: Patrick Hunt Reviewers: Norbert Kalmar Closes #1232 from phunt/ZOOKEEPER-3699 (cherry picked from commit 2d29c569f2c3ca879e1d026a7086f6e376187198) Signed-off-by: Norbert Kalmar --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index d68b7c0d5a8..d7f7a7c4f4e 100755 --- a/pom.xml +++ b/pom.xml @@ -327,7 +327,7 @@ 1.2 4.1.42.Final 9.4.24.v20191120 - 2.9.10.1 + 2.9.10.2 1.1.1 2.11 1.1.7 From 8352f78e752a019387c1eb6095085152e7d1d4ad Mon Sep 17 00:00:00 2001 From: Mate Szalay-Beko Date: Thu, 23 Jan 2020 13:43:39 +0100 Subject: [PATCH 021/118] ZOOKEEPER-3698: fixing NoRouteToHostException when starting large cluster locally When we tested RC 3.6.0, we had a problem of starting ZooKeeper cluster with large number (11+) of ensemble members locally on mac. We found exceptions in the logs when the new MultiAddress feature tries to filter the unreachable hosts from the address list. This involves the calling of the InetAddress.isReachable method with a default timeout of 500ms, which goes down to a native call in java and basically try to do a ping (an ICMP echo request) to the host. Naturally, the localhost should be always reachable. The problem was that on mac we have the ICMP rate limit set to 250 by default. In this patch we: - changed the reachability check behavior by disabling the check if there is only a single address provided (so we wouldn't be able to filter the unreachable addresses anyway). - added and documented a configuration parameter to disable the reachability check for testing. (default: enabled) - added and documented a configuration parameter to set the timeout for the reachability checks. (default: 1000ms) Author: Mate Szalay-Beko Reviewers: eolivelli@apache.org, andor@apache.org Closes #1228 from symat/ZOOKEEPER-3698-branch-3.6 --- .../main/resources/markdown/zookeeperAdmin.md | 29 +++++++++++++++++++ .../zookeeper/server/quorum/Learner.java | 7 ++++- .../server/quorum/MultipleAddresses.java | 18 +++++++++++- .../server/quorum/QuorumCnxManager.java | 26 ++++++++++------- .../zookeeper/server/quorum/QuorumPeer.java | 22 ++++++++++++++ .../server/quorum/QuorumPeerConfig.java | 21 ++++++++++++++ .../server/quorum/QuorumPeerMain.java | 2 ++ .../server/quorum/MultipleAddressesTest.java | 29 +++++++++++++++++++ .../server/quorum/QuorumPeerMainTest.java | 2 +- 9 files changed, 143 insertions(+), 13 deletions(-) diff --git a/zookeeper-docs/src/main/resources/markdown/zookeeperAdmin.md b/zookeeper-docs/src/main/resources/markdown/zookeeperAdmin.md index 1f41230a8e5..cfcc3e58180 100644 --- a/zookeeper-docs/src/main/resources/markdown/zookeeperAdmin.md +++ b/zookeeper-docs/src/main/resources/markdown/zookeeperAdmin.md @@ -1488,6 +1488,19 @@ and [SASL authentication for ZooKeeper](https://cwiki.apache.org/confluence/disp You can instruct ZooKeeper to remove the realm from the client principal name during authentication. (e.g. the zk/myhost@EXAMPLE.COM client principal will be authenticated in ZooKeeper as zk/myhost) Default: false + +* *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 + 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 + 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 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**. @@ -1565,6 +1578,22 @@ the variable does. ZAB protocol and the Fast Leader Election protocol. Default value is **false**. +* *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 + 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 + 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. + + #### Disabling data directory autocreation 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 ad9b0bba2b0..2fd2db4d5cd 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 @@ -266,7 +266,12 @@ protected void sockConnect(Socket sock, InetSocketAddress addr, int timeout) thr protected void connectToLeader(MultipleAddresses multiAddr, String hostname) throws IOException { this.leaderAddr = multiAddr; - Set addresses = multiAddr.getAllReachableAddresses(); + Set addresses; + if (self.isMultiAddressReachabilityCheckEnabled()) { + addresses = multiAddr.getAllReachableAddresses(); + } else { + addresses = multiAddr.getAllAddresses(); + } ExecutorService executor = Executors.newFixedThreadPool(addresses.size()); CountDownLatch latch = new CountDownLatch(addresses.size()); AtomicReference socket = new AtomicReference<>(null); diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/MultipleAddresses.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/MultipleAddresses.java index 730b182ae65..3ee63fa9d6c 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/MultipleAddresses.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/MultipleAddresses.java @@ -40,7 +40,7 @@ * See ZOOKEEPER-3188 for a discussion of this feature. */ public final class MultipleAddresses { - private static final Duration DEFAULT_TIMEOUT = Duration.ofMillis(500); + public static final Duration DEFAULT_TIMEOUT = Duration.ofMillis(1000); private static Set newConcurrentHashSet() { return Collections.newSetFromMap(new ConcurrentHashMap<>()); @@ -150,6 +150,12 @@ public Set getAllReachableAddresses() { */ public InetSocketAddress getReachableOrOne() { InetSocketAddress address; + + // if there is only a single address provided then we don't do any reachability check + if (addresses.size() == 1) { + return getOne(); + } + try { address = getReachableAddress(); } catch (NoRouteToHostException e) { @@ -179,6 +185,16 @@ public InetSocketAddress getOne() { return addresses.iterator().next(); } + + /** + * Returns the number of addresses in the set. + * + * @return the number of addresses. + */ + public int size() { + return addresses.size(); + } + private boolean checkIfAddressIsReachable(InetSocketAddress address) { if (address.isUnresolved()) { return false; 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 55c1dc27139..a92fc526ded 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 @@ -35,6 +35,7 @@ import java.nio.BufferUnderflowException; import java.nio.ByteBuffer; import java.nio.channels.UnresolvedAddressException; +import java.time.Duration; import java.util.ArrayList; import java.util.Collections; import java.util.Enumeration; @@ -534,7 +535,8 @@ private void handleConnection(Socket sock, DataInputStream din) throws IOExcepti try { InitialMessage init = InitialMessage.parse(protocolVersion, din); sid = init.sid; - electionAddr = new MultipleAddresses(init.electionAddr); + electionAddr = new MultipleAddresses(init.electionAddr, + Duration.ofMillis(self.getMultiAddressReachabilityCheckTimeoutMs())); } catch (InitialMessage.InitialMessageException ex) { LOG.error(ex.toString()); closeSocket(sock); @@ -637,10 +639,12 @@ public void toSend(Long sid, ByteBuffer b) { synchronized boolean connectOne(long sid, MultipleAddresses electionAddr) { if (senderWorkerMap.get(sid) != null) { LOG.debug("There is a connection already for server {}", sid); - // since ZOOKEEPER-3188 we can use multiple election addresses to reach a server. It is possible, that the - // one we are using is already dead and we need to clean-up, so when we will create a new connection - // then we will choose an other one, which is actually reachable - senderWorkerMap.get(sid).asyncValidateIfSocketIsStillReachable(); + if (electionAddr.size() > 1 && self.isMultiAddressReachabilityCheckEnabled()) { + // since ZOOKEEPER-3188 we can use multiple election addresses to reach a server. It is possible, that the + // one we are using is already dead and we need to clean-up, so when we will create a new connection + // then we will choose an other one, which is actually reachable + senderWorkerMap.get(sid).asyncValidateIfSocketIsStillReachable(); + } return true; } @@ -653,7 +657,7 @@ synchronized boolean connectOne(long sid, MultipleAddresses electionAddr) { sock = new Socket(); } setSockOpts(sock); - sock.connect(electionAddr.getReachableAddress(), cnxTO); + sock.connect(electionAddr.getReachableOrOne(), cnxTO); if (sock instanceof SSLSocket) { SSLSocket sslSock = (SSLSocket) sock; sslSock.startHandshake(); @@ -706,10 +710,12 @@ synchronized boolean connectOne(long sid, MultipleAddresses electionAddr) { synchronized void connectOne(long sid) { if (senderWorkerMap.get(sid) != null) { LOG.debug("There is a connection already for server {}", sid); - // since ZOOKEEPER-3188 we can use multiple election addresses to reach a server. It is possible, that the - // one we are using is already dead and we need to clean-up, so when we will create a new connection - // then we will choose an other one, which is actually reachable - senderWorkerMap.get(sid).asyncValidateIfSocketIsStillReachable(); + if (self.isMultiAddressReachabilityCheckEnabled()) { + // since ZOOKEEPER-3188 we can use multiple election addresses to reach a server. It is possible, that the + // one we are using is already dead and we need to clean-up, so when we will create a new connection + // then we will choose an other one, which is actually reachable + senderWorkerMap.get(sid).asyncValidateIfSocketIsStillReachable(); + } return; } synchronized (self.QV_LOCK) { diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumPeer.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumPeer.java index 415a738080e..011cc0a015d 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumPeer.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumPeer.java @@ -161,6 +161,28 @@ public void setObserverMasterPort(int observerMasterPort) { this.observerMasterPort = observerMasterPort; } + private int multiAddressReachabilityCheckTimeoutMs = (int) MultipleAddresses.DEFAULT_TIMEOUT.toMillis(); + + public int getMultiAddressReachabilityCheckTimeoutMs() { + return multiAddressReachabilityCheckTimeoutMs; + } + + public void setMultiAddressReachabilityCheckTimeoutMs(int multiAddressReachabilityCheckTimeoutMs) { + this.multiAddressReachabilityCheckTimeoutMs = multiAddressReachabilityCheckTimeoutMs; + LOG.info("multiAddress.reachabilityCheckTimeoutMs set to {}", multiAddressReachabilityCheckTimeoutMs); + } + + private boolean multiAddressReachabilityCheckEnabled = true; + + public boolean isMultiAddressReachabilityCheckEnabled() { + return multiAddressReachabilityCheckEnabled; + } + + public void setMultiAddressReachabilityCheckEnabled(boolean multiAddressReachabilityCheckEnabled) { + this.multiAddressReachabilityCheckEnabled = multiAddressReachabilityCheckEnabled; + LOG.info("multiAddress.reachabilityCheckEnabled set to {}", multiAddressReachabilityCheckEnabled); + } + public static class QuorumServer { public MultipleAddresses addr = new MultipleAddresses(); 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 99fc2ed8244..102d6995fdf 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 @@ -120,6 +120,15 @@ public class QuorumPeerConfig { protected String quorumServerLoginContext = QuorumAuth.QUORUM_SERVER_SASL_LOGIN_CONTEXT_DFAULT_VALUE; protected int quorumCnxnThreadsSize; + // multi address related configs + private boolean multiAddressReachabilityCheckEnabled = + Boolean.parseBoolean(System.getProperty("zookeeper.multiAddress.reachabilityCheckEnabled", + "true")); + private int multiAddressReachabilityCheckTimeoutMs = + Integer.parseInt(System.getProperty("zookeeper.multiAddress.reachabilityCheckTimeoutMs", + String.valueOf(MultipleAddresses.DEFAULT_TIMEOUT.toMillis()))); + + /** * Minimum snapshot retain count. * @see org.apache.zookeeper.server.PurgeTxnLog#purge(File, File, int) @@ -389,6 +398,10 @@ public void parseProperties(Properties zkProp) throws IOException, ConfigExcepti } else if (key.startsWith("metricsProvider.")) { String keyForMetricsProvider = key.substring(16); metricsProviderConfiguration.put(keyForMetricsProvider, value); + } else if (key.equals("multiAddress.reachabilityCheckTimeoutMs")) { + multiAddressReachabilityCheckTimeoutMs = Integer.parseInt(value); + } else if (key.equals("multiAddress.reachabilityCheckEnabled")) { + multiAddressReachabilityCheckEnabled = Boolean.parseBoolean(value); } else { System.setProperty("zookeeper." + key, value); } @@ -926,6 +939,14 @@ public Boolean getQuorumListenOnAllIPs() { return quorumListenOnAllIPs; } + public boolean isMultiAddressReachabilityCheckEnabled() { + return multiAddressReachabilityCheckEnabled; + } + + public int getMultiAddressReachabilityCheckTimeoutMs() { + return multiAddressReachabilityCheckTimeoutMs; + } + public static boolean isStandaloneEnabled() { return standaloneEnabled; } diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumPeerMain.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumPeerMain.java index 054a47dcd69..04aec05acee 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumPeerMain.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumPeerMain.java @@ -204,6 +204,8 @@ public void runFromConfig(QuorumPeerConfig config) throws IOException, AdminServ if (config.sslQuorumReloadCertFiles) { quorumPeer.getX509Util().enableCertFileReloading(); } + quorumPeer.setMultiAddressReachabilityCheckEnabled(config.isMultiAddressReachabilityCheckEnabled()); + quorumPeer.setMultiAddressReachabilityCheckTimeoutMs(config.getMultiAddressReachabilityCheckTimeoutMs()); // sets quorum sasl authentication configurations quorumPeer.setQuorumSaslEnabled(config.quorumEnableSasl); diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/MultipleAddressesTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/MultipleAddressesTest.java index 99203204279..e8f08c6d90c 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/MultipleAddressesTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/MultipleAddressesTest.java @@ -24,6 +24,7 @@ import java.net.UnknownHostException; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collections; import java.util.HashSet; import java.util.List; import java.util.Set; @@ -108,6 +109,26 @@ public void testGetValidAddressWithNotValid() throws NoRouteToHostException { multipleAddresses.getReachableAddress(); } + @Test + public void testGetReachableOrOneWithSingleReachableAddress() { + InetSocketAddress reachableAddress = new InetSocketAddress("127.0.0.1", PortAssignment.unique()); + + MultipleAddresses multipleAddresses = new MultipleAddresses(Collections.singletonList(reachableAddress)); + InetSocketAddress actualReturnedAddress = multipleAddresses.getReachableOrOne(); + + Assert.assertEquals(reachableAddress, actualReturnedAddress); + } + + @Test + public void testGetReachableOrOneWithSingleUnreachableAddress() { + InetSocketAddress unreachableAddress = new InetSocketAddress("unreachable.address.zookeeper.apache.com", 1234); + + MultipleAddresses multipleAddresses = new MultipleAddresses(Collections.singletonList(unreachableAddress)); + InetSocketAddress actualReturnedAddress = multipleAddresses.getReachableOrOne(); + + Assert.assertEquals(unreachableAddress, actualReturnedAddress); + } + @Test public void testRecreateSocketAddresses() throws UnknownHostException { List searchedAddresses = Arrays.stream(InetAddress.getAllByName("google.com")) @@ -180,6 +201,14 @@ public void testEquals() { Assert.assertNotEquals(multipleAddresses, multipleAddressesNotEquals); } + @Test + public void testSize() { + List addresses = getAddressList(); + MultipleAddresses multipleAddresses = new MultipleAddresses(addresses); + + Assert.assertEquals(PORTS_AMOUNT, multipleAddresses.size()); + } + public List getPortList() { return IntStream.range(0, PORTS_AMOUNT).mapToObj(i -> PortAssignment.unique()).collect(Collectors.toList()); } 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 c9beb09476e..fed5feaa7e1 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 @@ -472,7 +472,7 @@ public void testBadPeerAddressInQuorum() throws Exception { LineNumberReader r = new LineNumberReader(new StringReader(os.toString())); String line; boolean found = false; - Pattern p = Pattern.compile(".*None of the addresses .* are reachable for sid 2"); + Pattern p = Pattern.compile(".*Cannot open channel to .* at election address .*"); while ((line = r.readLine()) != null) { found = p.matcher(line).matches(); if (found) { From 6495a7afd2e69056331c2327585be3baff711013 Mon Sep 17 00:00:00 2001 From: Andor Molnar Date: Thu, 23 Jan 2020 16:46:28 +0100 Subject: [PATCH 022/118] ZOOKEEPER-1936: Server exits when unable to create data directory due to race Resurrecting an ancient ticket which could be fixed with a simple patch. Jira mentions a scenario when auto purging tool is in use and Zookeeper server could have a race condition when creating snapshot and data directories. (directory auto creating is enabled by default) Double checking the directory existence might help with it. Author: Andor Molnar Reviewers: Enrico Olivelli Closes #1225 from anmolnar/ZOOKEEPER-1936 (cherry picked from commit 689c8b2c4ed264f1dd75db7cbde1f86fe4408f50) Signed-off-by: Enrico Olivelli --- .../apache/zookeeper/server/persistence/FileTxnSnapLog.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/persistence/FileTxnSnapLog.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/persistence/FileTxnSnapLog.java index 661beb2bd40..b6014ef8091 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/persistence/FileTxnSnapLog.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/persistence/FileTxnSnapLog.java @@ -132,7 +132,7 @@ public FileTxnSnapLog(File dataDir, File snapDir) throws IOException { ZOOKEEPER_DATADIR_AUTOCREATE)); } - if (!this.dataDir.mkdirs()) { + if (!this.dataDir.mkdirs() && !this.dataDir.exists()) { throw new DatadirException("Unable to create data directory " + this.dataDir); } } @@ -151,7 +151,7 @@ public FileTxnSnapLog(File dataDir, File snapDir) throws IOException { ZOOKEEPER_DATADIR_AUTOCREATE)); } - if (!this.snapDir.mkdirs()) { + if (!this.snapDir.mkdirs() && !this.snapDir.exists()) { throw new DatadirException("Unable to create snap directory " + this.snapDir); } } From 56a6a034c71bdee3d822cf35ebbfb0ec6f8b44c7 Mon Sep 17 00:00:00 2001 From: Patrick Hunt Date: Thu, 23 Jan 2020 16:50:58 +0100 Subject: [PATCH 023/118] ZOOKEEPER-3704: upgrade maven dependency-check to 5.3.0 Upgrade to the latest version of maven dependency checker. Change-Id: I5ac9c77bb02f54784ff3ed4bb668fe38fd88ee11 Author: Patrick Hunt Reviewers: Enrico Olivelli Closes #1231 from phunt/ZOOKEEPER-3704 (cherry picked from commit 1fbaa26d6827263b2e9e8c18d8699f4bf799eda3) Signed-off-by: Enrico Olivelli --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index d7f7a7c4f4e..b2eea70fb81 100755 --- a/pom.xml +++ b/pom.xml @@ -631,7 +631,7 @@ org.owasp dependency-check-maven - 5.2.4 + 5.3.0 org.apache.maven.plugins From 2a6f1e34d93668bd24a21c37f1f0c5367c7c798a Mon Sep 17 00:00:00 2001 From: Mate Szalay-Beko Date: Mon, 27 Jan 2020 11:26:15 +0100 Subject: [PATCH 024/118] ZOOKEEPER-3567: add SSL support for zkpython This PR is about adding SSL support for zkPython, based on the C-binding. I also fixed the zkpython ant build to work with the current maven top-level build. I also added a new python test case to try to connect to ZooKeeper with SSL. You can test this patch in the following way: ``` # cleanup everything, just to be on the safe side: git clean -xdf # on ubuntu 16.4 make sure you have the following packages installed apt-get install -y libcppunit-dev openssl libssl-dev python-setuptools python2.7 python2.7-dev # make a full build (incl. C-client) mvn clean install -DskipTests -Pfull-build # we only support python2, so e.g. on ubuntu 18.4 you need to switch to python2 update-alternatives --install /usr/bin/python python /usr/bin/python2.7 1 # compile and test zkpython cd zookeeper-contrib/zookeeper-contrib-zkpython/ ant compile ant test ``` Author: Mate Szalay-Beko Reviewers: andor@apache.org Closes #1121 from symat/ZOOKEEPER-3567 and squashes the following commits: a5839cb56 [Mate Szalay-Beko] Merge remote-tracking branch 'apache/master' into ZOOKEEPER-3567 d25d61024 [Mate Szalay-Beko] ZOOKEEPER-3567: fix build issues after top-level ant removal a8869c969 [Mate Szalay-Beko] Merge remote-tracking branch 'apache/master' into HEAD b92f686e8 [Mate Szalay-Beko] ZOOKEEPER-3567: fix license check issue 0150986da [Mate Szalay-Beko] ZOOKEEPER-3567: removing code duplication: re-use test SSL certificate generator from C-client tests 7d91359d3 [Mate Szalay-Beko] ZOOKEEPER-3567: add SSL support for zkpython (cherry picked from commit e4758ba70b35dd38ff0bba9e534790391f53b050) Signed-off-by: Andor Molnar --- README_packaging.md | 6 +- pom.xml | 1 + zookeeper-contrib/build-contrib.xml | 8 +-- zookeeper-contrib/ivysettings.xml | 41 +++++++++++ .../zookeeper-contrib-zkpython/README | 8 ++- .../zookeeper-contrib-zkpython/build.xml | 2 +- .../src/c/pyzk_docstrings.h | 27 ++++++++ .../src/c/zookeeper.c | 44 +++++++++--- .../src/python/setup.py | 4 +- .../src/test/connection_test.py | 31 +++++++++ .../src/test/run_tests.sh | 5 +- .../src/test/zkServer.sh | 69 +++++++++++++------ .../src/test/zktestbase.py | 5 +- .../src/test/zoo.cfg | 14 ++++ 14 files changed, 217 insertions(+), 48 deletions(-) create mode 100644 zookeeper-contrib/ivysettings.xml create mode 100644 zookeeper-contrib/zookeeper-contrib-zkpython/src/test/zoo.cfg diff --git a/README_packaging.md b/README_packaging.md index b290dd9f93d..9c87e4cd483 100644 --- a/README_packaging.md +++ b/README_packaging.md @@ -19,11 +19,11 @@ yum install python-setuptools yum install openssl openssl-devel ``` -On Ubuntu: +On Ubuntu (in case of 16.4+): ``` -apt-get install cppunit -apt-get install python-setuptools +apt-get install libcppunit-dev +apt-get install python-setuptools python2.7-dev apt-get install openssl libssl-dev ``` diff --git a/pom.xml b/pom.xml index b2eea70fb81..070faf70298 100755 --- a/pom.xml +++ b/pom.xml @@ -854,6 +854,7 @@ 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 diff --git a/zookeeper-contrib/build-contrib.xml b/zookeeper-contrib/build-contrib.xml index 70bb8644f52..34f3c9b03b3 100644 --- a/zookeeper-contrib/build-contrib.xml +++ b/zookeeper-contrib/build-contrib.xml @@ -32,7 +32,7 @@ - + @@ -47,7 +47,7 @@ - + @@ -179,7 +179,7 @@ - + @@ -190,7 +190,7 @@ - diff --git a/zookeeper-contrib/ivysettings.xml b/zookeeper-contrib/ivysettings.xml new file mode 100644 index 00000000000..15c98b6ec4e --- /dev/null +++ b/zookeeper-contrib/ivysettings.xml @@ -0,0 +1,41 @@ + + + + + + + + + + + + + + + + + + + + + diff --git a/zookeeper-contrib/zookeeper-contrib-zkpython/README b/zookeeper-contrib/zookeeper-contrib-zkpython/README index 89d99989756..56154618356 100644 --- a/zookeeper-contrib/zookeeper-contrib-zkpython/README +++ b/zookeeper-contrib/zookeeper-contrib-zkpython/README @@ -7,14 +7,18 @@ DEPENDENCIES: This has only been tested against SVN (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. +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. +E.g. setting up tpyhon 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 + BUILD AND INSTALL: ------------------- -To install, make sure that the C client has been built and that the libraries are installed in /usr/local/lib (or change this directory in setup.py). Then run: +To install, make sure that the C client has been built (use `mvn clean install -DskipTests -Pfull-build` in the root folder of zookeeper) or that the zookeeper C libraries are installed in /usr/local/lib (or change this directory in setup.py). Then run: ant install diff --git a/zookeeper-contrib/zookeeper-contrib-zkpython/build.xml b/zookeeper-contrib/zookeeper-contrib-zkpython/build.xml index c229deec654..99f634abe81 100644 --- a/zookeeper-contrib/zookeeper-contrib-zkpython/build.xml +++ b/zookeeper-contrib/zookeeper-contrib-zkpython/build.xml @@ -20,10 +20,10 @@ - + /dev/null +if [ $? -eq 0 ] +then + pid=`lsof -i :22182 | grep LISTEN | awk '{print $2}'` + if [ -n "$pid" ] then - rm -rf /tmp/zkdata - else - rm -rf ${base_dir}/build/tmp + kill -9 $pid fi fi -if [ "x${base_dir}" == "x" ] + + + +if [ "x$1" == "xstartClean" ] then -zk_base="../../" -else -zk_base="${base_dir}" + rm -rf ${WORK_DIR} fi -CLASSPATH="$CLASSPATH:${zk_base}/build/classes" + + +CLASSPATH="$CLASSPATH:${PROJECT_ROOT}/zookeeper-server/target/classes" CLASSPATH="$CLASSPATH:${zk_base}/conf" -for i in "${zk_base}"/build/lib/*.jar +for i in "${PROJECT_ROOT}"/zookeeper-server/target/lib/*.jar do CLASSPATH="$CLASSPATH:$i" done -for i in "${zk_base}"/zookeeper-server/src/main/resource/lib/*.jar +for i in "${PROJECT_ROOT}"/zookeeper-server/src/main/resource/lib/*.jar do CLASSPATH="$CLASSPATH:$i" done @@ -57,15 +77,20 @@ done case $1 in start|startClean) - if [ "x${base_dir}" == "x" ] - then - mkdir -p /tmp/zkdata - java -cp $CLASSPATH org.apache.zookeeper.server.ZooKeeperServerMain 22182 /tmp/zkdata &> /tmp/zk.log & - else - mkdir -p ${base_dir}/build/tmp/zkdata - java -cp $CLASSPATH org.apache.zookeeper.server.ZooKeeperServerMain 22182 ${base_dir}/build/tmp/zkdata &> ${base_dir}/build/tmp/zk.log & - fi - sleep 5 + mkdir -p ${WORK_DIR}/zkdata + + rm -rf ${WORK_DIR}/ssl + mkdir -p ${WORK_DIR}/ssl + cp ${PROJECT_ROOT}/zookeeper-client/zookeeper-client-c/ssl/gencerts.sh ${WORK_DIR}/ssl/ + cd ${WORK_DIR}/ssl/ + ./gencerts.sh + cd - + + sed "s#WORKDIR#${WORK_DIR}#g" ${TEST_DIR}/zoo.cfg > "${WORK_DIR}/zoo.cfg" + java -Dzookeeper.extendedTypesEnabled=true -Dznode.container.checkIntervalMs=100 -cp $CLASSPATH org.apache.zookeeper.server.ZooKeeperServerMain "${WORK_DIR}/zoo.cfg" &> "${WORK_DIR}/zoo.log" & + pid=$! + echo -n $! > ${WORK_DIR}/../zk.pid + sleep 5 ;; stop) # Already killed above diff --git a/zookeeper-contrib/zookeeper-contrib-zkpython/src/test/zktestbase.py b/zookeeper-contrib/zookeeper-contrib-zkpython/src/test/zktestbase.py index 8229418e3e7..cafbec56669 100755 --- a/zookeeper-contrib/zookeeper-contrib-zkpython/src/test/zktestbase.py +++ b/zookeeper-contrib/zookeeper-contrib-zkpython/src/test/zktestbase.py @@ -22,10 +22,13 @@ class TestBase(unittest.TestCase): SERVER_PORT = 22182 - + SERVER_SSL_PORT = 22183 + def __init__(self,methodName='runTest'): unittest.TestCase.__init__(self,methodName) self.host = "localhost:%d" % self.SERVER_PORT + self.sslhost = "localhost:%d" % self.SERVER_SSL_PORT + self.sslcert = "./target/zkpython_tests/ssl/server.crt,./target/zkpython_tests/ssl/client.crt,./target/zkpython_tests/ssl/clientkey.pem,password" self.connected = False self.handle = -1 logdir = os.environ.get("ZKPY_LOG_DIR") diff --git a/zookeeper-contrib/zookeeper-contrib-zkpython/src/test/zoo.cfg b/zookeeper-contrib/zookeeper-contrib-zkpython/src/test/zoo.cfg new file mode 100644 index 00000000000..2d4fc31d9fd --- /dev/null +++ b/zookeeper-contrib/zookeeper-contrib-zkpython/src/test/zoo.cfg @@ -0,0 +1,14 @@ +tickTime=500 +initLimit=10 +syncLimit=5 +dataDir=WORKDIR/zkdata +maxClientCnxns=200 + +clientPort=22182 +secureClientPort=22183 +serverCnxnFactory=org.apache.zookeeper.server.NettyServerCnxnFactory +ssl.keyStore.location=WORKDIR/ssl/server.jks +ssl.keyStore.password=password +ssl.trustStore.location=WORKDIR/ssl/servertrust.jks +ssl.trustStore.password=password + From eedf26bb12bc33169ba2756fccd9527c75afe6ca Mon Sep 17 00:00:00 2001 From: Mate Szalay-Beko Date: Tue, 28 Jan 2020 15:22:30 +0100 Subject: [PATCH 025/118] ZOOKEEPER-1105: wait for server response in C client zookeeper_close **Thanks for Lincoln Lee for the original fix!** In the current implementation, we always get a WARN in server side ("EndOfStreamException: Unable to read additional data from client") whenever we close a zookeeper handler from the C client. This also happens in the end of every execution of the command line C client. The reason is that currently we don't wait for the response from the server when we initiate the closing of the client connection, and we terminate the socket on the client side too early. I tested the patch both on linux and windows. I also tested it both with NIO and Netty server side socket implementations. Author: Mate Szalay-Beko Reviewers: Andor Molnar , Norbert Kalmar Closes #1176 from symat/ZOOKEEPER-1105 (cherry picked from commit 57be7aedd698cb824ade7373aaa4d8264e1b4eb7) Signed-off-by: Norbert Kalmar --- .../zookeeper-client-c/src/zookeeper.c | 70 +++++++++++++++---- 1 file changed, 57 insertions(+), 13 deletions(-) diff --git a/zookeeper-client/zookeeper-client-c/src/zookeeper.c b/zookeeper-client/zookeeper-client-c/src/zookeeper.c index 7ab5eed33cb..d2b73561773 100644 --- a/zookeeper-client/zookeeper-client-c/src/zookeeper.c +++ b/zookeeper-client/zookeeper-client-c/src/zookeeper.c @@ -3495,6 +3495,49 @@ static int add_multi_completion(zhandle_t *zh, int xid, void_completion_t dc, return add_completion(zh, xid, COMPLETION_MULTI, dc, data, 0,0, clist); } +/** + * After sending the close request, we are waiting for a given millisecs for + * getting the answer and/or for the socket to be closed by the server. + * + * This function should not be called while we still want to process + * any response from the server. It must be called after adaptor_finish called, + * in order not to mess with the I/O receiver thread in multi-threaded mode. + */ +int wait_for_session_to_be_closed(zhandle_t *zh, int timeout_ms) +{ + int ret = 0; +#ifndef WIN32 + struct pollfd fd_s[1]; +#else + fd_set rfds; + struct timeval waittime = {timeout_ms / 1000, (timeout_ms % 1000) * 1000}; +#endif + + if (zh == NULL) { + return ZBADARGUMENTS; + } + +#ifndef WIN32 + fd_s[0].fd = zh->fd->sock; + fd_s[0].events = POLLIN; + ret = poll(fd_s, 1, timeout_ms); +#else + FD_ZERO(&rfds); + FD_SET(zh->fd->sock , &rfds); + ret = select(zh->fd->sock + 1, &rfds, NULL, NULL, &waittime); +#endif + + if (ret == 0){ + LOG_WARN(LOGCALLBACK(zh), "Timed out (%dms) during waiting for server's reply after sending a close request, sessionId=%#llx\n", + timeout_ms, zh->client_id.client_id); + } else if (ret < 0) { + LOG_WARN(LOGCALLBACK(zh), "System error (%d) happened while waiting for server's reply, sessionId=%#llx\n", + ret, zh->client_id.client_id); + } + + return ZOK; +} + int zookeeper_close(zhandle_t *zh) { int rc=ZOK; @@ -3520,32 +3563,33 @@ int zookeeper_close(zhandle_t *zh) } /* No need to decrement the counter since we're just going to * destroy the handle later. */ - if (is_connected(zh)){ + if (is_connected(zh)) { struct oarchive *oa; struct RequestHeader h = {get_xid(), ZOO_CLOSE_OP}; LOG_INFO(LOGCALLBACK(zh), "Closing zookeeper sessionId=%#llx to %s\n", - zh->client_id.client_id,zoo_get_current_server(zh)); + zh->client_id.client_id, zoo_get_current_server(zh)); oa = create_buffer_oarchive(); rc = serialize_RequestHeader(oa, "header", &h); - rc = rc < 0 ? rc : queue_buffer_bytes(&zh->to_send, get_buffer(oa), - get_buffer_len(oa)); + rc = rc < 0 ? rc : queue_buffer_bytes(&zh->to_send, get_buffer(oa), get_buffer_len(oa)); /* We queued the buffer, so don't free it */ close_buffer_oarchive(&oa, 0); if (rc < 0) { + LOG_DEBUG(LOGCALLBACK(zh), "Error during closing zookeeper session, sessionId=%#llx to %s (error: %d)\n", + zh->client_id.client_id, zoo_get_current_server(zh), rc); rc = ZMARSHALLINGERROR; - goto finish; - } + } else { + /* make sure the close request is sent; we set timeout to an arbitrary + * (but reasonable) number of milliseconds since we want the call to block*/ + rc = adaptor_send_queue(zh, 3000); - /* make sure the close request is sent; we set timeout to an arbitrary - * (but reasonable) number of milliseconds since we want the call to block*/ - rc=adaptor_send_queue(zh, 3000); - }else{ - LOG_INFO(LOGCALLBACK(zh), "Freeing zookeeper resources for sessionId=%#llx\n", - zh->client_id.client_id); + /* give some time to the server to process the session close request properly */ + rc = rc < 0 ? rc : wait_for_session_to_be_closed(zh, 1500); + } + } else { rc = ZOK; } -finish: + LOG_INFO(LOGCALLBACK(zh), "Freeing zookeeper resources for sessionId=%#llx\n", zh->client_id.client_id); destroy(zh); adaptor_destroy(zh); free(zh->fd); From 9b6badb5b43d6b6d2ef5c84ad5659d48cf968558 Mon Sep 17 00:00:00 2001 From: Andor Molnar Date: Tue, 28 Jan 2020 17:26:14 +0100 Subject: [PATCH 026/118] ZOOKEEPER-3701: Split brain on log disk full Issue described here: https://issues.apache.org/jira/browse/ZOOKEEPER-3701 Proposing a fix with catching `IOException` within the truncate method to properly return with `false` if truncate fails. Author: Andor Molnar Author: Andor Molnar Reviewers: Ivan Kelly , Enrico Olivelli , Norbert Kalmar Closes #1233 from anmolnar/ZOOKEEPER-3701 (cherry picked from commit a4bc9857e19b6fa3832813d7e47f683b582565b8) Signed-off-by: Norbert Kalmar --- .../server/persistence/FileSnap.java | 2 + .../server/persistence/FileTxnLog.java | 3 +- .../server/persistence/FileTxnSnapLog.java | 49 ++++++++++------- .../zookeeper/server/SnapshotDigestTest.java | 2 + .../zookeeper/server/quorum/LearnerTest.java | 53 +++++++++++++++++++ .../apache/zookeeper/test/TruncateTest.java | 7 ++- 6 files changed, 92 insertions(+), 24 deletions(-) diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/persistence/FileSnap.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/persistence/FileSnap.java index fde577a05c5..b608c21ea0b 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/persistence/FileSnap.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/persistence/FileSnap.java @@ -260,6 +260,8 @@ public synchronized void serialize( Util.getZxidFromName(snapShot.getName(), SNAPSHOT_FILE_PREFIX), snapShot.lastModified() / 1000); } + } else { + throw new IOException("FileSnap has already been closed"); } } diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/persistence/FileTxnLog.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/persistence/FileTxnLog.java index 973e741969e..62969bac8a6 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/persistence/FileTxnLog.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/persistence/FileTxnLog.java @@ -20,6 +20,7 @@ import java.io.BufferedInputStream; import java.io.BufferedOutputStream; +import java.io.Closeable; import java.io.EOFException; import java.io.File; import java.io.FileInputStream; @@ -93,7 +94,7 @@ * 0 padded to EOF (filled during preallocation stage) * */ -public class FileTxnLog implements TxnLog { +public class FileTxnLog implements TxnLog, Closeable { private static final Logger LOG; diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/persistence/FileTxnSnapLog.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/persistence/FileTxnSnapLog.java index b6014ef8091..0ec12d84ab9 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/persistence/FileTxnSnapLog.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/persistence/FileTxnSnapLog.java @@ -499,23 +499,28 @@ public void save( * @return true if able to truncate the log, false if not * @throws IOException */ - public boolean truncateLog(long zxid) throws IOException { - // close the existing txnLog and snapLog - close(); - - // truncate it - FileTxnLog truncLog = new FileTxnLog(dataDir); - boolean truncated = truncLog.truncate(zxid); - truncLog.close(); - - // re-open the txnLog and snapLog - // I'd rather just close/reopen this object itself, however that - // would have a big impact outside ZKDatabase as there are other - // objects holding a reference to this object. - txnLog = new FileTxnLog(dataDir); - snapLog = new FileSnap(snapDir); - - return truncated; + public boolean truncateLog(long zxid) { + try { + // close the existing txnLog and snapLog + close(); + + // truncate it + try (FileTxnLog truncLog = new FileTxnLog(dataDir)) { + boolean truncated = truncLog.truncate(zxid); + + // re-open the txnLog and snapLog + // I'd rather just close/reopen this object itself, however that + // would have a big impact outside ZKDatabase as there are other + // objects holding a reference to this object. + txnLog = new FileTxnLog(dataDir); + snapLog = new FileSnap(snapDir); + + return truncated; + } + } catch (IOException e) { + LOG.error("Unable to truncate Txn log", e); + return false; + } } /** @@ -594,8 +599,14 @@ public void rollLog() throws IOException { * @throws IOException */ public void close() throws IOException { - txnLog.close(); - snapLog.close(); + if (txnLog != null) { + txnLog.close(); + txnLog = null; + } + if (snapLog != null) { + snapLog.close(); + snapLog = null; + } } @SuppressWarnings("serial") diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/SnapshotDigestTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/SnapshotDigestTest.java index 40f07a8ac51..d40debfa7ad 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/SnapshotDigestTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/SnapshotDigestTest.java @@ -194,6 +194,8 @@ private void reloadSnapshotAndCheckDigest() throws Exception { startServer(); QuorumPeerMainTest.waitForOne(zk, States.CONNECTED); + server = serverFactory.getZooKeeperServer(); + // Snapshot digests always match assertEquals(0L, ServerMetrics.getMetrics().DIGEST_MISMATCHES_COUNT.get()); diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/LearnerTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/LearnerTest.java index 1d121ac8bde..63381fc7640 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/LearnerTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/LearnerTest.java @@ -20,7 +20,10 @@ import static java.util.Arrays.asList; import static java.util.Collections.emptySet; +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.CoreMatchers.is; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; import static org.mockito.Mockito.mock; @@ -36,17 +39,20 @@ import java.util.ArrayList; import java.util.HashSet; import java.util.Set; +import java.util.function.Consumer; import org.apache.jute.BinaryInputArchive; import org.apache.jute.BinaryOutputArchive; import org.apache.zookeeper.ZKTestCase; import org.apache.zookeeper.ZooDefs; import org.apache.zookeeper.common.X509Exception; import org.apache.zookeeper.data.ACL; +import org.apache.zookeeper.server.ExitCode; import org.apache.zookeeper.server.ZKDatabase; import org.apache.zookeeper.server.persistence.FileTxnSnapLog; import org.apache.zookeeper.test.TestUtils; import org.apache.zookeeper.txn.CreateTxn; import org.apache.zookeeper.txn.TxnHeader; +import org.apache.zookeeper.util.ServiceUtils; import org.junit.Test; public class LearnerTest extends ZKTestCase { @@ -300,4 +306,51 @@ public void syncTest() throws Exception { } } + @Test + public void truncFailTest() throws Exception { + final boolean[] exitProcCalled = {false}; + + ServiceUtils.setSystemExitProcedure(new Consumer() { + @Override + public void accept(Integer exitCode) { + exitProcCalled[0] = true; + assertThat("System.exit() was called with invalid exit code", exitCode, equalTo(ExitCode.QUORUM_PACKET_ERROR.getValue())); + } + }); + + File tmpFile = File.createTempFile("test", ".dir", testData); + tmpFile.delete(); + try { + FileTxnSnapLog txnSnapLog = new FileTxnSnapLog(tmpFile, tmpFile); + SimpleLearner sl = new SimpleLearner(txnSnapLog); + long startZxid = sl.zk.getLastProcessedZxid(); + + // Set up bogus streams + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + BinaryOutputArchive oa = BinaryOutputArchive.getArchive(baos); + sl.leaderOs = BinaryOutputArchive.getArchive(new ByteArrayOutputStream()); + + // make streams and socket do something innocuous + sl.bufferedOutput = new BufferedOutputStream(System.out); + sl.sock = new Socket(); + + // fake messages from the server + QuorumPacket qp = new QuorumPacket(Leader.TRUNC, 0, null, null); + oa.writeRecord(qp, null); + + // setup the messages to be streamed to follower + sl.leaderIs = BinaryInputArchive.getArchive(new ByteArrayInputStream(baos.toByteArray())); + + try { + sl.syncWithLeader(3); + } catch (EOFException e) { + } + + sl.zk.shutdown(); + + assertThat("System.exit() should have been called", exitProcCalled[0], is(true)); + } finally { + TestUtils.deleteFileRecursively(tmpFile); + } + } } diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/test/TruncateTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/test/TruncateTest.java index f7a448e3802..089a7644aab 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/test/TruncateTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/test/TruncateTest.java @@ -18,7 +18,9 @@ package org.apache.zookeeper.test; +import static org.hamcrest.CoreMatchers.is; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; import java.io.File; @@ -125,10 +127,7 @@ public void testTruncationNullLog() throws Exception { assertTrue("Failed to delete log file: " + logs[i].getName(), logs[i].delete()); } try { - zkdb.truncateLog(1); - assertTrue("Should not get here", false); - } catch (IOException e) { - assertTrue("Should have received an IOException", true); + assertThat("truncateLog() should return false if truncation fails instead of throwing exception", zkdb.truncateLog(1), is(false)); } catch (NullPointerException npe) { fail("This should not throw NPE!"); } From 2c232e3a8a330d5804ec6f121536629c37b530ed Mon Sep 17 00:00:00 2001 From: Enrico Olivelli Date: Wed, 29 Jan 2020 14:15:28 +0100 Subject: [PATCH 027/118] Update release notes before 3.6.0rc1 --- .../src/main/resources/markdown/releasenotes.md | 13 +++++++++++++ 1 file changed, 13 insertions(+) diff --git a/zookeeper-docs/src/main/resources/markdown/releasenotes.md b/zookeeper-docs/src/main/resources/markdown/releasenotes.md index f5b25aee9eb..dfab2d35771 100644 --- a/zookeeper-docs/src/main/resources/markdown/releasenotes.md +++ b/zookeeper-docs/src/main/resources/markdown/releasenotes.md @@ -48,6 +48,8 @@ limitations under the License. ## Improvement +* [ZOOKEEPER-3482](https://issues.apache.org/jira/browse/ZOOKEEPER-3482) - SASL (Kerberos) Authentication with SSL for clients and Quorum +* [ZOOKEEPER-3567](https://issues.apache.org/jira/browse/ZOOKEEPER-3567) - add SSL support for zkpython * [ZOOKEEPER-261](https://issues.apache.org/jira/browse/ZOOKEEPER-261) - Reinitialized servers should not participate in leader election * [ZOOKEEPER-761](https://issues.apache.org/jira/browse/ZOOKEEPER-761) - Remove *synchronous* calls from the *single-threaded* C clieant API, since they are documented not to work * [ZOOKEEPER-974](https://issues.apache.org/jira/browse/ZOOKEEPER-974) - Configurable listen socket backlog for the client port @@ -291,6 +293,15 @@ limitations under the License. ## Bug +* [ZOOKEEPER-3677](https://issues.apache.org/jira/browse/ZOOKEEPER-3677) - Setting jute.maxbuffer value in hexadecimal throws Exception +* [ZOOKEEPER-3695](https://issues.apache.org/jira/browse/ZOOKEEPER-3695) - Source release tarball does not match repository in 3.6.0 +* [ZOOKEEPER-3667](https://issues.apache.org/jira/browse/ZOOKEEPER-3667) - owasp checker failing for - CVE-2019-17571 Apache Log4j 1.2 deserialization of untrusted data in SocketServer +* [ZOOKEEPER-3613](https://issues.apache.org/jira/browse/ZOOKEEPER-3613) - ZKConfig fails to return proper value on getBoolean()when user accidentally includes spaces at the end of the value +* [ZOOKEEPER-3699](https://issues.apache.org/jira/browse/ZOOKEEPER-3699) - upgrade jackson-databind to address CVE-2019-20330 +* [ZOOKEEPER-3698](https://issues.apache.org/jira/browse/ZOOKEEPER-3698) - fixing NoRouteToHostException when starting large cluster locally +* [ZOOKEEPER-1936](https://issues.apache.org/jira/browse/ZOOKEEPER-1936) - Server exits when unable to create data directory due to race +* [ZOOKEEPER-3701](https://issues.apache.org/jira/browse/ZOOKEEPER-3701) - Split brain on log disk full +* [ZOOKEEPER-1105](https://issues.apache.org/jira/browse/ZOOKEEPER-1105) - wait for server response in C client zookeeper_close * [ZOOKEEPER-706](https://issues.apache.org/jira/browse/ZOOKEEPER-706) - large numbers of watches can cause session re-establishment to fail * [ZOOKEEPER-1029](https://issues.apache.org/jira/browse/ZOOKEEPER-1029) - C client bug in zookeeper_init (if bad hostname is given) * [ZOOKEEPER-1077](https://issues.apache.org/jira/browse/ZOOKEEPER-1077) - C client lib doesn't build on Solaris @@ -625,6 +636,8 @@ limitations under the License. ## Task +* [ZOOKEEPER-3669](https://issues.apache.org/jira/browse/ZOOKEEPER-3669) - Use switch Statement in ClientCnxn SendThread +* [ZOOKEEPER-3704](https://issues.apache.org/jira/browse/ZOOKEEPER-3704) - upgrade maven dependency-check to 5.3.0 * [ZOOKEEPER-925](https://issues.apache.org/jira/browse/ZOOKEEPER-925) - Consider maven site generation to replace our forrest site and documentation generation * [ZOOKEEPER-1604](https://issues.apache.org/jira/browse/ZOOKEEPER-1604) - remove rpm/deb/... packaging * [ZOOKEEPER-2658](https://issues.apache.org/jira/browse/ZOOKEEPER-2658) - Trunk / branch-3.5 build broken. From 1ec248185669016ec0705eb9c10fddf0328bacc5 Mon Sep 17 00:00:00 2001 From: maoling Date: Thu, 30 Jan 2020 09:43:09 +0100 Subject: [PATCH 028/118] ZOOKEEPER-3231: Purge task may lost data when the recent snapshots are all invalid - Purge task uses `FileTxnSnapLog#findNRecentSnapshot`, which's likely to lost data when the recent 3 snapshots are all invalid(a new valid snapshot has not generated yet) and at the same time, Purge task(`e.g ./zkCleanup.sh -n 3`) has started a new round work to clean up the preceding snapshots. we will lose all the data.that's a small probability events, but it's reproducible. - Overall, using `snaplog.findNValidSnapshots` to make sure the purge task tries to retain N valid snapshots(rather than N snapshots) to avoid a risk of data loss. - For the unit test, it's not easy to use the `mock` way for the following reasons: - when we want to test the `dataDir` which some Snapshots are valid, others not.Just writing a little data contents to the snapshot to make it valid/invalid has a better flexibility. - too much code changes in the `PurgeTxnTest.java`(pass the original UT) and `FileTxnSnapLog.java`(have some handles) - more details in the [ZOOKEEPER-3231](https://issues.apache.org/jira/browse/ZOOKEEPER-3231) Author: maoling Reviewers: enixon@apache.org, andor@apache.org Closes #1079 from maoling/ZOOKEEPER-3231 and squashes the following commits: 674175bef [maoling] setUp() & tearDown(). 472dfd33c [maoling] ZOOKEEPER-3231:Purge task may lost data when the recent snapshots are all invalid (cherry picked from commit 2abdfbc261d4ba2f9d17b51f2b46ab748a7d451f) Signed-off-by: Andor Molnar --- .../apache/zookeeper/server/PurgeTxnLog.java | 2 +- .../server/persistence/FileSnap.java | 2 +- .../server/persistence/FileTxnSnapLog.java | 12 ++ .../apache/zookeeper/server/PurgeTxnTest.java | 114 ++++++++++++++---- 4 files changed, 103 insertions(+), 27 deletions(-) diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/PurgeTxnLog.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/PurgeTxnLog.java index 0c9206647e1..d152fabd8dd 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/PurgeTxnLog.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/PurgeTxnLog.java @@ -79,7 +79,7 @@ public static void purge(File dataDir, File snapDir, int num) throws IOException FileTxnSnapLog txnLog = new FileTxnSnapLog(dataDir, snapDir); - List snaps = txnLog.findNRecentSnapshots(num); + List snaps = txnLog.findNValidSnapshots(num); int numSnaps = snaps.size(); if (numSnaps > 0) { purgeOlderSnapshots(txnLog, snaps.get(numSnaps - 1)); diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/persistence/FileSnap.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/persistence/FileSnap.java index b608c21ea0b..4c092323857 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/persistence/FileSnap.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/persistence/FileSnap.java @@ -159,7 +159,7 @@ public File findMostRecentSnapshot() throws IOException { * less than n in case enough snapshots are not available). * @throws IOException */ - private List findNValidSnapshots(int n) throws IOException { + protected List findNValidSnapshots(int n) throws IOException { List files = Util.sortDataDir(snapDir.listFiles(), SNAPSHOT_FILE_PREFIX, false); int count = 0; List list = new ArrayList(); diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/persistence/FileTxnSnapLog.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/persistence/FileTxnSnapLog.java index 0ec12d84ab9..eddeae8d93e 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/persistence/FileTxnSnapLog.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/persistence/FileTxnSnapLog.java @@ -547,6 +547,18 @@ public List findNRecentSnapshots(int n) throws IOException { return snaplog.findNRecentSnapshots(n); } + /** + * the n recent valid snapshots + * @param n the number of recent valid snapshots + * @return the list of n recent valid snapshots, with + * the most recent in front + * @throws IOException + */ + public List findNValidSnapshots(int n) throws IOException { + FileSnap snaplog = new FileSnap(snapDir); + return snaplog.findNValidSnapshots(n); + } + /** * get the snapshot logs which may contain transactions newer than the given zxid. * This includes logs with starting zxid greater than given zxid, as well as the diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/PurgeTxnTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/PurgeTxnTest.java index 52d336034f9..29c1027949e 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/PurgeTxnTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/PurgeTxnTest.java @@ -25,6 +25,7 @@ import static org.junit.Assert.fail; import java.io.File; import java.io.IOException; +import java.io.OutputStream; import java.util.ArrayList; import java.util.Collections; import java.util.List; @@ -32,16 +33,23 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; +import java.util.zip.CheckedOutputStream; +import org.apache.jute.BinaryOutputArchive; +import org.apache.jute.OutputArchive; import org.apache.zookeeper.CreateMode; import org.apache.zookeeper.PortAssignment; import org.apache.zookeeper.ZKTestCase; import org.apache.zookeeper.ZooDefs.Ids; import org.apache.zookeeper.ZooKeeper; import org.apache.zookeeper.data.Stat; +import org.apache.zookeeper.server.persistence.FileHeader; +import org.apache.zookeeper.server.persistence.FileSnap; import org.apache.zookeeper.server.persistence.FileTxnSnapLog; +import org.apache.zookeeper.server.persistence.SnapStream; import org.apache.zookeeper.server.persistence.Util; import org.apache.zookeeper.test.ClientBase; import org.junit.After; +import org.junit.Before; import org.junit.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -54,6 +62,11 @@ public class PurgeTxnTest extends ZKTestCase { private static final long OP_TIMEOUT_IN_MILLIS = 90000; private File tmpDir; + @Before + public void setUp() throws Exception { + tmpDir = ClientBase.createTmpDir(); + } + @After public void teardown() { if (null != tmpDir) { @@ -67,7 +80,6 @@ public void teardown() { */ @Test public void testPurge() throws Exception { - tmpDir = ClientBase.createTmpDir(); ClientBase.setupTestEnv(); ZooKeeperServer zks = new ZooKeeperServer(tmpDir, tmpDir, 3000); SyncRequestProcessor.setSnapCount(100); @@ -89,7 +101,7 @@ public void testPurge() throws Exception { // now corrupt the snapshot PurgeTxnLog.purge(tmpDir, tmpDir, 3); FileTxnSnapLog snaplog = new FileTxnSnapLog(tmpDir, tmpDir); - List listLogs = snaplog.findNRecentSnapshots(4); + List listLogs = snaplog.findNValidSnapshots(4); int numSnaps = 0; for (File ff : listLogs) { if (ff.getName().startsWith("snapshot")) { @@ -111,7 +123,6 @@ public void testPurge() throws Exception { */ @Test public void testPurgeWhenLogRollingInProgress() throws Exception { - tmpDir = ClientBase.createTmpDir(); ClientBase.setupTestEnv(); ZooKeeperServer zks = new ZooKeeperServer(tmpDir, tmpDir, 3000); SyncRequestProcessor.setSnapCount(30); @@ -158,21 +169,20 @@ public void run() { } /** - * Tests finding n recent snapshots from set of snapshots and data logs + * Tests finding n recent valid snapshots from set of snapshots and data logs */ @Test - public void testFindNRecentSnapshots() throws Exception { + public void testFindNValidSnapshots() throws Exception { int nRecentSnap = 4; // n recent snap shots int nRecentCount = 30; int offset = 0; - tmpDir = ClientBase.createTmpDir(); File version2 = new File(tmpDir.toString(), "version-2"); assertTrue("Failed to create version_2 dir:" + version2.toString(), version2.mkdir()); - // Test that with no snaps, findNRecentSnapshots returns empty list + // Test that with no snaps, findNValidSnapshots returns empty list FileTxnSnapLog txnLog = new FileTxnSnapLog(tmpDir, tmpDir); - List foundSnaps = txnLog.findNRecentSnapshots(1); + List foundSnaps = txnLog.findNValidSnapshots(1); assertEquals(0, foundSnaps.size()); List expectedNRecentSnapFiles = new ArrayList(); @@ -184,6 +194,7 @@ public void testFindNRecentSnapshots() throws Exception { // simulate snapshot file File snapFile = new File(version2 + "/snapshot." + Long.toHexString(--counter)); assertTrue("Failed to create snap File:" + snapFile.toString(), snapFile.createNewFile()); + makeValidSnapshot(snapFile); // add the n recent snap files for assertion if (i < nRecentSnap) { expectedNRecentSnapFiles.add(snapFile); @@ -192,17 +203,17 @@ public void testFindNRecentSnapshots() throws Exception { // Test that when we ask for recent snaps we get the number we asked for and // the files we expected - List nRecentSnapFiles = txnLog.findNRecentSnapshots(nRecentSnap); - assertEquals("exactly 4 snapshots ", 4, nRecentSnapFiles.size()); - expectedNRecentSnapFiles.removeAll(nRecentSnapFiles); + List nRecentValidSnapFiles = txnLog.findNValidSnapshots(nRecentSnap); + assertEquals("exactly 4 snapshots ", 4, nRecentValidSnapFiles.size()); + expectedNRecentSnapFiles.removeAll(nRecentValidSnapFiles); assertEquals("Didn't get the recent snap files", 0, expectedNRecentSnapFiles.size()); // Test that when asking for more snaps than we created, we still only get snaps // not logs or anything else (per ZOOKEEPER-2420) - nRecentSnapFiles = txnLog.findNRecentSnapshots(nRecentCount + 5); - assertEquals(nRecentCount, nRecentSnapFiles.size()); - for (File f : nRecentSnapFiles) { - assertTrue("findNRecentSnapshots() returned a non-snapshot: " + nRecentValidSnapFiles = txnLog.findNValidSnapshots(nRecentCount + 5); + assertEquals(nRecentCount, nRecentValidSnapFiles.size()); + for (File f : nRecentValidSnapFiles) { + assertTrue("findNValidSnapshots() returned a non-snapshot: " + f.getPath(), (Util.getZxidFromName(f.getName(), "snapshot") != -1)); } @@ -220,7 +231,6 @@ public void testSnapFilesGreaterThanToRetain() throws Exception { int fileAboveRecentCount = 4; int fileToPurgeCount = 2; AtomicInteger offset = new AtomicInteger(0); - tmpDir = ClientBase.createTmpDir(); File version2 = new File(tmpDir.toString(), "version-2"); assertTrue("Failed to create version_2 dir:" + version2.toString(), version2.mkdir()); List snapsToPurge = new ArrayList(); @@ -272,7 +282,6 @@ public void testSnapFilesEqualsToRetainWithPrecedingLog() throws Exception { public void internalTestSnapFilesEqualsToRetain(boolean testWithPrecedingLogFile) throws Exception { int nRecentCount = 3; AtomicInteger offset = new AtomicInteger(0); - tmpDir = ClientBase.createTmpDir(); File version2 = new File(tmpDir.toString(), "version-2"); assertTrue("Failed to create version_2 dir:" + version2.toString(), version2.mkdir()); List snaps = new ArrayList(); @@ -295,7 +304,6 @@ public void testSnapFilesLessThanToRetain() throws Exception { int nRecentCount = 4; int fileToPurgeCount = 2; AtomicInteger offset = new AtomicInteger(0); - tmpDir = ClientBase.createTmpDir(); File version2 = new File(tmpDir.toString(), "version-2"); assertTrue("Failed to create version_2 dir:" + version2.toString(), version2.mkdir()); List snapsToPurge = new ArrayList(); @@ -327,7 +335,6 @@ public void testSnapFilesLessThanToRetain() throws Exception { */ @Test public void testPurgeTxnLogWithDataDir() throws Exception { - tmpDir = ClientBase.createTmpDir(); File dataDir = new File(tmpDir, "dataDir"); File dataLogDir = new File(tmpDir, "dataLogDir"); @@ -348,6 +355,7 @@ public void testPurgeTxnLogWithDataDir() throws Exception { // simulate snapshot file File snapFile = new File(dataDirVersion2, "snapshot." + Long.toHexString(i)); snapFile.createNewFile(); + makeValidSnapshot(snapFile); } int numberOfSnapFilesToKeep = 10; @@ -358,8 +366,6 @@ public void testPurgeTxnLogWithDataDir() throws Exception { assertEquals(numberOfSnapFilesToKeep, dataDirVersion2.listFiles().length); // Since for each snapshot we have a log file with same zxid, expect same # logs as snaps to be kept assertEquals(numberOfSnapFilesToKeep, dataLogDirVersion2.listFiles().length); - ClientBase.recursiveDelete(tmpDir); - } /** @@ -368,7 +374,6 @@ public void testPurgeTxnLogWithDataDir() throws Exception { */ @Test public void testPurgeTxnLogWithoutDataDir() throws Exception { - tmpDir = ClientBase.createTmpDir(); File dataDir = new File(tmpDir, "dataDir"); File dataLogDir = new File(tmpDir, "dataLogDir"); @@ -388,6 +393,7 @@ public void testPurgeTxnLogWithoutDataDir() throws Exception { // simulate snapshot file File snapFile = new File(dataLogDirVersion2, "snapshot." + Long.toHexString(i)); snapFile.createNewFile(); + makeValidSnapshot(snapFile); } int numberOfSnapFilesToKeep = 10; @@ -398,8 +404,6 @@ public void testPurgeTxnLogWithoutDataDir() throws Exception { numberOfSnapFilesToKeep * 2, // Since for each snapshot we have a log file with same zxid, expect same # logs as snaps to be kept dataLogDirVersion2.listFiles().length); - ClientBase.recursiveDelete(tmpDir); - } /** @@ -423,7 +427,6 @@ public void testPurgeDoesNotDeleteOverlappingLogFile() throws Exception { SyncRequestProcessor.setSnapCount(SNAP_RETAIN_COUNT * NUM_ZNODES_PER_SNAPSHOT * 10); // Create Zookeeper and connect to it. - tmpDir = ClientBase.createTmpDir(); ClientBase.setupTestEnv(); ZooKeeperServer zks = new ZooKeeperServer(tmpDir, tmpDir, 3000); final int PORT = Integer.parseInt(HOSTPORT.split(":")[1]); @@ -484,6 +487,45 @@ public void testPurgeDoesNotDeleteOverlappingLogFile() throws Exception { zks.shutdown(); } + @Test + public void testPurgeTxnLogWhenRecentSnapshotsAreAllInvalid() throws Exception { + File dataDir = new File(tmpDir, "dataDir"); + File dataLogDir = new File(tmpDir, "dataLogDir"); + + File dataDirVersion2 = new File(dataDir, "version-2"); + dataDirVersion2.mkdirs(); + File dataLogDirVersion2 = new File(dataLogDir, "version-2"); + dataLogDirVersion2.mkdirs(); + + // create dummy log and transaction file + int totalFiles = 10; + int numberOfSnapFilesToKeep = 3; + + // create transaction and snapshot files in different-different + // directories + for (int i = 0; i < totalFiles; i++) { + // simulate log file + File logFile = new File(dataLogDirVersion2, "log." + Long.toHexString(i)); + logFile.createNewFile(); + // simulate snapshot file + File snapFile = new File(dataDirVersion2, "snapshot." + Long.toHexString(i)); + snapFile.createNewFile(); + if (i < (totalFiles - numberOfSnapFilesToKeep)) { + makeValidSnapshot(snapFile); + } else { + makeInvalidSnapshot(snapFile); + } + } + + // scenario where four parameter are passed + String[] args = new String[]{dataLogDir.getAbsolutePath(), dataDir.getAbsolutePath(), "-n", Integer.toString(numberOfSnapFilesToKeep)}; + PurgeTxnLog.main(args); + //Since the recent 3 snapshots are all invalid,when purging, we can assert that 6 snapshot files are retained(3 invalid snapshots and 3 retained valid snapshots) + assertEquals(numberOfSnapFilesToKeep + numberOfSnapFilesToKeep, dataDirVersion2.listFiles().length); + // Since for each snapshot we have a log file with same zxid, expect same # logs as snaps to be kept + assertEquals(numberOfSnapFilesToKeep + numberOfSnapFilesToKeep, dataLogDirVersion2.listFiles().length); + } + private File createDataDirLogFile(File version_2, int Zxid) throws IOException { File logFile = new File(version_2 + "/log." + Long.toHexString(Zxid)); assertTrue("Failed to create log File:" + logFile.toString(), logFile.createNewFile()); @@ -553,4 +595,26 @@ public void run() { return znodes; } + private void makeValidSnapshot(File snapFile) throws IOException { + SnapStream.setStreamMode(SnapStream.StreamMode.CHECKED); + CheckedOutputStream os = SnapStream.getOutputStream(snapFile); + OutputArchive oa = BinaryOutputArchive.getArchive(os); + FileHeader header = new FileHeader(FileSnap.SNAP_MAGIC, 2, 1); + header.serialize(oa, "fileheader"); + SnapStream.sealStream(os, oa); + os.flush(); + os.close(); + + assertTrue(SnapStream.isValidSnapshot(snapFile)); + } + + private void makeInvalidSnapshot(File snapFile) throws IOException { + SnapStream.setStreamMode(SnapStream.StreamMode.CHECKED); + OutputStream os = SnapStream.getOutputStream(snapFile); + os.write(1); + os.flush(); + os.close(); + + assertFalse(SnapStream.isValidSnapshot(snapFile)); + } } From 955b4bf1ac037a7d0f53956ba637df0c7be69c1a Mon Sep 17 00:00:00 2001 From: Enrico Olivelli Date: Sat, 1 Feb 2020 12:08:39 +0100 Subject: [PATCH 029/118] ZOOKEEPER-3695: Source release tarball does not match repository in 3.6.0 - part2 - upgrade to Apache Parent 23 - disable maven remote plugin - move source assembly execution before sources generation - move git properties resolution to the correct phase Author: Enrico Olivelli Reviewers: Norbert Kalmar Closes #1238 from eolivelli/fix/ZOOKEEPER-3695-part2 (cherry picked from commit d3ce1fa7e0b9804fb7752cbb3818c03341d8c87e) Signed-off-by: Enrico Olivelli --- pom.xml | 16 +++++++++++++--- 1 file changed, 13 insertions(+), 3 deletions(-) diff --git a/pom.xml b/pom.xml index 070faf70298..28a120a9628 100755 --- a/pom.xml +++ b/pom.xml @@ -23,7 +23,7 @@ org.apache apache - 21 + 23 @@ -284,7 +284,7 @@ source-release-assembly-tar-gz - package + initialize single @@ -664,6 +664,16 @@ + + + maven-remote-resources-plugin + + + process-resource-bundles + none + + + @@ -677,7 +687,7 @@ revision - validate + generate-resources From fc43a29e8737c50201aa34157c4f93c31b224c39 Mon Sep 17 00:00:00 2001 From: David Mollitor Date: Mon, 3 Feb 2020 04:30:41 +0100 Subject: [PATCH 030/118] ZOOKEEPER-3708: Move Logging Code into Logging Guard in Learner Author: David Mollitor Reviewers: eolivelli@apache.org, andor@apache.org Closes #1240 from belugabehr/ZOOKEEPER-3708 (cherry picked from commit 68e1f7d4c54429caa74441c7444ef1fe0f75867c) Signed-off-by: Andor Molnar --- .../java/org/apache/zookeeper/server/quorum/Learner.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) 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 2fd2db4d5cd..1ee9e07f743 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 @@ -182,11 +182,11 @@ void readPacket(QuorumPacket pp) throws IOException { leaderIs.readRecord(pp, "packet"); messageTracker.trackReceived(pp.getType()); } - long traceMask = ZooTrace.SERVER_PACKET_TRACE_MASK; - if (pp.getType() == Leader.PING) { - traceMask = ZooTrace.SERVER_PING_TRACE_MASK; - } if (LOG.isTraceEnabled()) { + final long traceMask = + (pp.getType() == Leader.PING) ? ZooTrace.SERVER_PING_TRACE_MASK + : ZooTrace.SERVER_PACKET_TRACE_MASK; + ZooTrace.logQuorumPacket(LOG, traceMask, 'i', pp); } } From 4d36edeb13701f91c5154d086cb47b1a9153270d Mon Sep 17 00:00:00 2001 From: Patrick Hunt Date: Tue, 4 Feb 2020 10:35:10 +0100 Subject: [PATCH 031/118] =?UTF-8?q?ZOOKEEPER-3716:=20upgrade=20netty=204.1?= =?UTF-8?q?.42=20to=20address=20CVE-2019-20444=20CVE-20=E2=80=A6?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit …19-20445 Updated netty to version 4.1.45 to address CVE-2019-20444 CVE-2019-20445 Change-Id: Iabcbd1f0b75a03d2a555084de841d7f8caedf6bd Author: Patrick Hunt Reviewers: Enrico Olivelli Closes #1245 from phunt/ZOOKEEPER-3716 (cherry picked from commit 13c8a35051acbc1f8c3369ed260df2ab22275f58) Signed-off-by: Enrico Olivelli --- pom.xml | 2 +- ....Final.LICENSE.txt => netty-buffer-4.1.45.Final.LICENSE.txt} | 0 ...2.Final.LICENSE.txt => netty-codec-4.1.45.Final.LICENSE.txt} | 0 ....Final.LICENSE.txt => netty-common-4.1.45.Final.LICENSE.txt} | 0 ...Final.LICENSE.txt => netty-handler-4.1.45.Final.LICENSE.txt} | 0 ...inal.LICENSE.txt => netty-resolver-4.1.45.Final.LICENSE.txt} | 0 ...nal.LICENSE.txt => netty-transport-4.1.45.Final.LICENSE.txt} | 0 ...xt => netty-transport-native-epoll-4.1.45.Final.LICENSE.txt} | 0 ...netty-transport-native-unix-common-4.1.45.Final.LICENSE.txt} | 0 9 files changed, 1 insertion(+), 1 deletion(-) rename zookeeper-server/src/main/resources/lib/{netty-buffer-4.1.42.Final.LICENSE.txt => netty-buffer-4.1.45.Final.LICENSE.txt} (100%) rename zookeeper-server/src/main/resources/lib/{netty-codec-4.1.42.Final.LICENSE.txt => netty-codec-4.1.45.Final.LICENSE.txt} (100%) rename zookeeper-server/src/main/resources/lib/{netty-common-4.1.42.Final.LICENSE.txt => netty-common-4.1.45.Final.LICENSE.txt} (100%) rename zookeeper-server/src/main/resources/lib/{netty-handler-4.1.42.Final.LICENSE.txt => netty-handler-4.1.45.Final.LICENSE.txt} (100%) rename zookeeper-server/src/main/resources/lib/{netty-resolver-4.1.42.Final.LICENSE.txt => netty-resolver-4.1.45.Final.LICENSE.txt} (100%) rename zookeeper-server/src/main/resources/lib/{netty-transport-4.1.42.Final.LICENSE.txt => netty-transport-4.1.45.Final.LICENSE.txt} (100%) rename zookeeper-server/src/main/resources/lib/{netty-transport-native-epoll-4.1.42.Final.LICENSE.txt => netty-transport-native-epoll-4.1.45.Final.LICENSE.txt} (100%) rename zookeeper-server/src/main/resources/lib/{netty-transport-native-unix-common-4.1.42.Final.LICENSE.txt => netty-transport-native-unix-common-4.1.45.Final.LICENSE.txt} (100%) diff --git a/pom.xml b/pom.xml index 28a120a9628..a5130f41205 100755 --- a/pom.xml +++ b/pom.xml @@ -325,7 +325,7 @@ 2.27.0 1.3 1.2 - 4.1.42.Final + 4.1.45.Final 9.4.24.v20191120 2.9.10.2 1.1.1 diff --git a/zookeeper-server/src/main/resources/lib/netty-buffer-4.1.42.Final.LICENSE.txt b/zookeeper-server/src/main/resources/lib/netty-buffer-4.1.45.Final.LICENSE.txt similarity index 100% rename from zookeeper-server/src/main/resources/lib/netty-buffer-4.1.42.Final.LICENSE.txt rename to zookeeper-server/src/main/resources/lib/netty-buffer-4.1.45.Final.LICENSE.txt diff --git a/zookeeper-server/src/main/resources/lib/netty-codec-4.1.42.Final.LICENSE.txt b/zookeeper-server/src/main/resources/lib/netty-codec-4.1.45.Final.LICENSE.txt similarity index 100% rename from zookeeper-server/src/main/resources/lib/netty-codec-4.1.42.Final.LICENSE.txt rename to zookeeper-server/src/main/resources/lib/netty-codec-4.1.45.Final.LICENSE.txt diff --git a/zookeeper-server/src/main/resources/lib/netty-common-4.1.42.Final.LICENSE.txt b/zookeeper-server/src/main/resources/lib/netty-common-4.1.45.Final.LICENSE.txt similarity index 100% rename from zookeeper-server/src/main/resources/lib/netty-common-4.1.42.Final.LICENSE.txt rename to zookeeper-server/src/main/resources/lib/netty-common-4.1.45.Final.LICENSE.txt diff --git a/zookeeper-server/src/main/resources/lib/netty-handler-4.1.42.Final.LICENSE.txt b/zookeeper-server/src/main/resources/lib/netty-handler-4.1.45.Final.LICENSE.txt similarity index 100% rename from zookeeper-server/src/main/resources/lib/netty-handler-4.1.42.Final.LICENSE.txt rename to zookeeper-server/src/main/resources/lib/netty-handler-4.1.45.Final.LICENSE.txt diff --git a/zookeeper-server/src/main/resources/lib/netty-resolver-4.1.42.Final.LICENSE.txt b/zookeeper-server/src/main/resources/lib/netty-resolver-4.1.45.Final.LICENSE.txt similarity index 100% rename from zookeeper-server/src/main/resources/lib/netty-resolver-4.1.42.Final.LICENSE.txt rename to zookeeper-server/src/main/resources/lib/netty-resolver-4.1.45.Final.LICENSE.txt diff --git a/zookeeper-server/src/main/resources/lib/netty-transport-4.1.42.Final.LICENSE.txt b/zookeeper-server/src/main/resources/lib/netty-transport-4.1.45.Final.LICENSE.txt similarity index 100% rename from zookeeper-server/src/main/resources/lib/netty-transport-4.1.42.Final.LICENSE.txt rename to zookeeper-server/src/main/resources/lib/netty-transport-4.1.45.Final.LICENSE.txt diff --git a/zookeeper-server/src/main/resources/lib/netty-transport-native-epoll-4.1.42.Final.LICENSE.txt b/zookeeper-server/src/main/resources/lib/netty-transport-native-epoll-4.1.45.Final.LICENSE.txt similarity index 100% rename from zookeeper-server/src/main/resources/lib/netty-transport-native-epoll-4.1.42.Final.LICENSE.txt rename to zookeeper-server/src/main/resources/lib/netty-transport-native-epoll-4.1.45.Final.LICENSE.txt diff --git a/zookeeper-server/src/main/resources/lib/netty-transport-native-unix-common-4.1.42.Final.LICENSE.txt b/zookeeper-server/src/main/resources/lib/netty-transport-native-unix-common-4.1.45.Final.LICENSE.txt similarity index 100% rename from zookeeper-server/src/main/resources/lib/netty-transport-native-unix-common-4.1.42.Final.LICENSE.txt rename to zookeeper-server/src/main/resources/lib/netty-transport-native-unix-common-4.1.45.Final.LICENSE.txt From d95c5a3d6f1e0be77c4838d85c526d226b7798a3 Mon Sep 17 00:00:00 2001 From: Mate Szalay-Beko Date: Tue, 4 Feb 2020 10:39:02 +0100 Subject: [PATCH 032/118] ZOOKEEPER-3715: fix Kerberos test failures for new JDK versions MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Using OpenJDK 8u.242 or OpenJDK 11.0.6, we have some kerberos exceptions when running the following, Kerberos Authentication related tests: - QuorumKerberosAuthTest - QuorumKerberosHostBasedAuthTest - SaslKerberosAuthOverSSLTest After trying this with different JDK versions, we see that the problem seems to appear: - between OpenJDK 8u.232 and 8u.242 for java 8 - and between 11.0.3 and 11.0.6 for java 11 There are a lot of kerberos related changes after 8u.232: see https://hg.openjdk.java.net/jdk8u/jdk8u/jdk I didn't really found the root cause of the issue, but the problem disappeared after upgrading the Apache Kerby. Kerby is used only by the tests to start a local embedded KDC server. I also checked the dependencies of the new Kerby version and there is nothing to exclude there as far as I saw. I also improved the logging of errors during Kerberos authentication problems by printing out some more exceptions. Author: Mate Szalay-Beko Reviewers: Enrico Olivelli , Norbert Kalmar Closes #1244 from symat/ZOOKEEPER-3715 (cherry picked from commit 6763f73cb64d87145e8e642d9e4fcc1e0d509216) Signed-off-by: Enrico Olivelli --- pom.xml | 2 +- .../java/org/apache/zookeeper/client/ZooKeeperSaslClient.java | 2 +- .../zookeeper/server/quorum/auth/SaslQuorumAuthLearner.java | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/pom.xml b/pom.xml index a5130f41205..71e16d84a96 100755 --- a/pom.xml +++ b/pom.xml @@ -331,7 +331,7 @@ 1.1.1 2.11 1.1.7 - 1.1.0 + 2.0.0 1.60 3.2.2 2.6 diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/client/ZooKeeperSaslClient.java b/zookeeper-server/src/main/java/org/apache/zookeeper/client/ZooKeeperSaslClient.java index 9021f46f413..b0598bca87f 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/client/ZooKeeperSaslClient.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/client/ZooKeeperSaslClient.java @@ -335,7 +335,7 @@ public byte[] run() throws SaslException { error += " Zookeeper Client will go to AUTH_FAILED state."; LOG.error(error); saslState = SaslState.FAILED; - throw new SaslException(error); + throw new SaslException(error, e); } } } else { diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/auth/SaslQuorumAuthLearner.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/auth/SaslQuorumAuthLearner.java index d9e44ff6cc1..12cec788a65 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/auth/SaslQuorumAuthLearner.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/auth/SaslQuorumAuthLearner.java @@ -195,7 +195,7 @@ public byte[] run() throws SaslException { + " '-Dsun.net.spi.nameservice.provider.1=dns,sun' to your server's JVMFLAGS environment."; } LOG.error(error); - throw new SaslException(error); + throw new SaslException(error, e); } } } else { From 3008ab70ea418ef97a4ccff968b96a4b5198a96c Mon Sep 17 00:00:00 2001 From: Jordan Zimmerman Date: Tue, 4 Feb 2020 10:43:30 +0100 Subject: [PATCH 033/118] ZOOKEEPER-3703: publish a test JAR It would be very helpful to Apache Curator and others if ZooKeeper published its testing code as a Maven Test JAR. Curator, for example, could use it to improve its testing server to make it easier to inject error conditions without having to have forced time delays and other hacks. NOTE: if we move forward with gRPC (ZOOKEEPER-102) that would be in a new module and this would be required. So, might as well do it now. Author: Jordan Zimmerman Reviewers: Enrico Olivelli , Norbert Kalmar Closes #1229 from Randgalt/ZOOKEEPER-3703-publish-test-jar (cherry picked from commit 6db92d7dfa426488b191afba319a16b96f4a36c6) Signed-off-by: Enrico Olivelli --- zookeeper-server/pom.xml | 15 ++++++++++++++- 1 file changed, 14 insertions(+), 1 deletion(-) diff --git a/zookeeper-server/pom.xml b/zookeeper-server/pom.xml index 7a72b3780a1..5fb044a9422 100755 --- a/zookeeper-server/pom.xml +++ b/zookeeper-server/pom.xml @@ -295,7 +295,20 @@ - + + + org.apache.maven.plugins + maven-jar-plugin + + + publish-test-jar + + test-jar + + + + + From 5bf63d7773b3f20a5da56ba94b839ee977dfcd98 Mon Sep 17 00:00:00 2001 From: Enrico Olivelli Date: Tue, 4 Feb 2020 10:51:22 +0100 Subject: [PATCH 034/118] Update release notes for 3.6.0rc2 --- zookeeper-docs/src/main/resources/markdown/releasenotes.md | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/zookeeper-docs/src/main/resources/markdown/releasenotes.md b/zookeeper-docs/src/main/resources/markdown/releasenotes.md index dfab2d35771..5436884cb8e 100644 --- a/zookeeper-docs/src/main/resources/markdown/releasenotes.md +++ b/zookeeper-docs/src/main/resources/markdown/releasenotes.md @@ -48,6 +48,7 @@ limitations under the License. ## Improvement +* [ZOOKEEPER-3703](https://issues.apache.org/jira/browse/ZOOKEEPER-3703) - publish a test JAR * [ZOOKEEPER-3482](https://issues.apache.org/jira/browse/ZOOKEEPER-3482) - SASL (Kerberos) Authentication with SSL for clients and Quorum * [ZOOKEEPER-3567](https://issues.apache.org/jira/browse/ZOOKEEPER-3567) - add SSL support for zkpython * [ZOOKEEPER-261](https://issues.apache.org/jira/browse/ZOOKEEPER-261) - Reinitialized servers should not participate in leader election @@ -636,6 +637,9 @@ limitations under the License. ## Task +* [ZOOKEEPER-3715](https://issues.apache.org/jira/browse/ZOOKEEPER-3715) - fix Kerberos test failures for new JDK versions +* [ZOOKEEPER-3716](https://issues.apache.org/jira/browse/ZOOKEEPER-3716) - upgrade netty 4.1.42 to address CVE-2019-20444 CVE-2019-20445 + * [ZOOKEEPER-3669](https://issues.apache.org/jira/browse/ZOOKEEPER-3669) - Use switch Statement in ClientCnxn SendThread * [ZOOKEEPER-3704](https://issues.apache.org/jira/browse/ZOOKEEPER-3704) - upgrade maven dependency-check to 5.3.0 * [ZOOKEEPER-925](https://issues.apache.org/jira/browse/ZOOKEEPER-925) - Consider maven site generation to replace our forrest site and documentation generation From 824d26f92b4b5577e4386d7d8798c47830287bd1 Mon Sep 17 00:00:00 2001 From: maoling Date: Fri, 31 Jan 2020 12:01:23 +0100 Subject: [PATCH 035/118] ZOOKEEPER-3710: [trivial bug] fix compile error in PurgeTxnTest introduced by ZOOKEEPER-3231 - Link to [PR-1079](https://github.com/apache/zookeeper/pull/1079#issuecomment-580275886) - more details in the [ZOOKEEPER-3710](https://issues.apache.org/jira/browse/ZOOKEEPER-3710) Author: maoling Reviewers: Enrico Olivelli , Norbert Kalmar Closes #1239 from maoling/ZOOKEEPER-3710 --- .../test/java/org/apache/zookeeper/server/PurgeTxnTest.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/PurgeTxnTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/PurgeTxnTest.java index 29c1027949e..84d907649cf 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/PurgeTxnTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/PurgeTxnTest.java @@ -597,7 +597,7 @@ public void run() { private void makeValidSnapshot(File snapFile) throws IOException { SnapStream.setStreamMode(SnapStream.StreamMode.CHECKED); - CheckedOutputStream os = SnapStream.getOutputStream(snapFile); + CheckedOutputStream os = SnapStream.getOutputStream(snapFile, true); OutputArchive oa = BinaryOutputArchive.getArchive(os); FileHeader header = new FileHeader(FileSnap.SNAP_MAGIC, 2, 1); header.serialize(oa, "fileheader"); @@ -610,7 +610,7 @@ private void makeValidSnapshot(File snapFile) throws IOException { private void makeInvalidSnapshot(File snapFile) throws IOException { SnapStream.setStreamMode(SnapStream.StreamMode.CHECKED); - OutputStream os = SnapStream.getOutputStream(snapFile); + OutputStream os = SnapStream.getOutputStream(snapFile, true); os.write(1); os.flush(); os.close(); From d4681c7bda159fac7aa864b0c71074ce95feaf38 Mon Sep 17 00:00:00 2001 From: Mate Szalay-Beko Date: Sun, 16 Feb 2020 10:25:14 +0100 Subject: [PATCH 036/118] ZOOKEEPER-3720: Fix rolling upgrade failure (invalid protocol version) The multi-address feature introduced in ZOOKEEPER-3188 required changes in the Quorum protocol as we had to send all addresses in the connection initiation message to enable the receiving side to choose a reachable address in case of network failure. The new code can handle both the old and the new protocol versions to avoid 'invalid protocol' error e.g. during rolling restarts. However, the new protocol version still can not be used during rolling upgrade if the old servers are not supporting this protocol. In this case the old and the new servers would form two distinct partitions until all the servers get upgraded. To support rolling upgrades too, we want to disable the MultiAddress feature by default and use the old protocol. If the user would like enable the MultiAddress feature on a 3.6.0 cluster, she/he can do it either by 1) starting the cluster from scratch (without rolling upgrade), or 2) performing a rolling upgrade without the MultiAddress feature enabled then doing a rolling restart with a new configuration where the MultiAddress feature is enabled. During the rolling restart there will be no partitions, as all the servers in the cluster now will run ZooKeeper version 3.6.0 which understands now both protocols. The changes in this patch: - introducing new config property: multiAddress.enabled, disabled by default - updating QuorumCnxManager to be able to use both protocol versions and to use the old one if MultiAddress is disabled - failing with ConfigException if the user provides multiple addresses in the config while having MultiAddress disabled - updating the existing MultiAddress related tests to enable the feature first - add some new tests - update the documentation Testing: - I ran all the unit tests - Using https://github.com/symat/zk-rolling-upgrade-test - I tested rolling upgrade from 3.5.6 - I tested rolling restart to enable the MultiAddress feature - Using https://github.com/symat/zookeeper-docker-test - I tested the MultiAddress feature by disabling some virtual interfaces and waiting for the cluster to recover Author: Mate Szalay-Beko Reviewers: Enrico Olivelli , Norbert Kalmar , Andor Molnar Closes #1251 from symat/ZOOKEEPER-3720 (cherry picked from commit 3aa922c5737c9ef0879f290181cb281261c965e0) Signed-off-by: Enrico Olivelli --- .../main/resources/markdown/zookeeperAdmin.md | 30 +++++++++++++++-- .../server/quorum/QuorumCnxManager.java | 33 ++++++++++++++----- .../zookeeper/server/quorum/QuorumPeer.java | 24 +++++++++++++- .../server/quorum/QuorumPeerConfig.java | 16 ++++++--- .../server/quorum/QuorumPeerMain.java | 1 + .../server/quorum/CnxManagerTest.java | 10 +++--- .../zookeeper/server/quorum/LearnerTest.java | 8 +++++ .../QuorumPeerMainMultiAddressTest.java | 31 +++++++++++++++++ .../server/quorum/QuorumSSLTest.java | 3 ++ .../server/quorum/QuorumServerTest.java | 19 +++++++++++ .../quorum/auth/QuorumAuthTestBase.java | 8 +++++ .../quorum/auth/QuorumAuthUpgradeTest.java | 2 ++ .../quorum/auth/QuorumDigestAuthTest.java | 2 ++ .../quorum/auth/QuorumKerberosAuthTest.java | 2 ++ .../auth/QuorumKerberosHostBasedAuthTest.java | 2 ++ 15 files changed, 169 insertions(+), 22 deletions(-) diff --git a/zookeeper-docs/src/main/resources/markdown/zookeeperAdmin.md b/zookeeper-docs/src/main/resources/markdown/zookeeperAdmin.md index cfcc3e58180..1c6e130e6a0 100644 --- a/zookeeper-docs/src/main/resources/markdown/zookeeperAdmin.md +++ b/zookeeper-docs/src/main/resources/markdown/zookeeperAdmin.md @@ -1132,7 +1132,8 @@ of servers -- that is, when deploying clusters of servers. 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)). - This helps to increase availability and adds network level + 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 a working interface in case a network error. The different addresses can be specified @@ -1141,7 +1142,18 @@ of servers -- that is, when deploying clusters of servers. server.1=zoo1-net1:2888:3888|zoo1-net2:2889:3889 server.2=zoo2-net1:2888:3888|zoo2-net2:2889:3889 server.3=zoo3-net1:2888:3888|zoo3-net2:2889:3889 - + + + ###### Note + >By enabling this feature, the Quorum protocol (ZooKeeper Server-Server protocol) will change. + The users will not notice this and when anyone starts a ZooKeeper cluster with the new config, + everything will work normally. However, it's not possible to enable this feature and specify + multiple addresses during a rolling upgrade if the old ZooKeeper cluster didn't support the + *multiAddress* feature (and the new Quorum protocol). In case if you need this feature but you + also need to perform a rolling upgrade from a ZooKeeper cluster older than *3.6.0*, then you + first need to do the rolling upgrade without enabling the MultiAddress feature and later make + a separate rolling restart with the new configuration where **multiAddress.enabled** is set + to **true** and multiple addresses are provided. * *syncLimit* : (No Java system property) @@ -1489,6 +1501,16 @@ and [SASL authentication for ZooKeeper](https://cwiki.apache.org/confluence/disp (e.g. the zk/myhost@EXAMPLE.COM client principal will be authenticated in ZooKeeper as zk/myhost) Default: false +* *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 + **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**. + * *multiAddress.reachabilityCheckTimeoutMs* : (Java system property: **zookeeper.multiAddress.reachabilityCheckTimeoutMs**) **New in 3.6.0:** @@ -1501,7 +1523,8 @@ and [SASL authentication for ZooKeeper](https://cwiki.apache.org/confluence/disp 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*. @@ -1593,6 +1616,7 @@ the variable does. 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/quorum/QuorumCnxManager.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumCnxManager.java index a92fc526ded..45ffeb84903 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 @@ -37,6 +37,8 @@ import java.nio.channels.UnresolvedAddressException; import java.time.Duration; import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; import java.util.Collections; import java.util.Enumeration; import java.util.HashSet; @@ -113,9 +115,12 @@ public class QuorumCnxManager { private AtomicLong observerCounter = new AtomicLong(-1); /* - * Protocol identifier used among peers + * Protocol identifier used among peers (must be a negative number for backward compatibility reasons) */ - public static final long PROTOCOL_VERSION = -65535L; + // the following protocol version was sent in every connection initiation message since ZOOKEEPER-107 released in 3.5.0 + public static final long PROTOCOL_VERSION_V1 = -65536L; + // ZOOKEEPER-3188 introduced multiple addresses in the connection initiation message, released in 3.6.0 + public static final long PROTOCOL_VERSION_V2 = -65535L; /* * Max buffer size to be read from the network. @@ -218,7 +223,7 @@ public static class InitialMessageException extends Exception { public static InitialMessage parse(Long protocolVersion, DataInputStream din) throws InitialMessageException, IOException { Long sid; - if (protocolVersion != PROTOCOL_VERSION) { + if (protocolVersion != PROTOCOL_VERSION_V1 && protocolVersion != PROTOCOL_VERSION_V2) { throw new InitialMessageException("Got unrecognized protocol version %s", protocolVersion); } @@ -236,6 +241,8 @@ public static InitialMessage parse(Long protocolVersion, DataInputStream din) th throw new InitialMessageException("Read only %s bytes out of %s sent by server %s", num_read, remaining, sid); } + // in PROTOCOL_VERSION_V1 we expect to get a single address here represented as a 'host:port' string + // in PROTOCOL_VERSION_V2 we expect to get multiple addresses like: 'host1:port1|host2:port2|...' String[] addressStrings = new String(b).split("\\|"); List addresses = new ArrayList<>(addressStrings.length); for (String addr : addressStrings) { @@ -416,10 +423,20 @@ private boolean startConnection(Socket sock, Long sid) throws IOException { // Sending id and challenge - // represents protocol version (in other words - message type) - dout.writeLong(PROTOCOL_VERSION); + // First sending the protocol version (in other words - message type). + // For backward compatibility reasons we stick to the old protocol version, unless the MultiAddress + // feature is enabled. During rolling upgrade, we must make sure that all the servers can + // understand the protocol version we use to avoid multiple partitions. see ZOOKEEPER-3720 + long protocolVersion = self.isMultiAddressEnabled() ? PROTOCOL_VERSION_V2 : PROTOCOL_VERSION_V1; + dout.writeLong(protocolVersion); dout.writeLong(self.getId()); - String addr = self.getElectionAddress().getAllAddresses().stream() + + // now we send our election address. For the new protocol version, we can send multiple addresses. + Collection addressesToSend = protocolVersion == PROTOCOL_VERSION_V2 + ? self.getElectionAddress().getAllAddresses() + : Arrays.asList(self.getElectionAddress().getOne()); + + String addr = addressesToSend.stream() .map(NetUtils::formatInetAddr).collect(Collectors.joining("|")); byte[] addr_bytes = addr.getBytes(); dout.writeInt(addr_bytes.length); @@ -639,7 +656,7 @@ public void toSend(Long sid, ByteBuffer b) { synchronized boolean connectOne(long sid, MultipleAddresses electionAddr) { if (senderWorkerMap.get(sid) != null) { LOG.debug("There is a connection already for server {}", sid); - if (electionAddr.size() > 1 && self.isMultiAddressReachabilityCheckEnabled()) { + if (self.isMultiAddressEnabled() && electionAddr.size() > 1 && self.isMultiAddressReachabilityCheckEnabled()) { // since ZOOKEEPER-3188 we can use multiple election addresses to reach a server. It is possible, that the // one we are using is already dead and we need to clean-up, so when we will create a new connection // then we will choose an other one, which is actually reachable @@ -710,7 +727,7 @@ synchronized boolean connectOne(long sid, MultipleAddresses electionAddr) { synchronized void connectOne(long sid) { if (senderWorkerMap.get(sid) != null) { LOG.debug("There is a connection already for server {}", sid); - if (self.isMultiAddressReachabilityCheckEnabled()) { + if (self.isMultiAddressEnabled() && self.isMultiAddressReachabilityCheckEnabled()) { // since ZOOKEEPER-3188 we can use multiple election addresses to reach a server. It is possible, that the // one we are using is already dead and we need to clean-up, so when we will create a new connection // then we will choose an other one, which is actually reachable diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumPeer.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumPeer.java index 011cc0a015d..dbd23e92279 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumPeer.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumPeer.java @@ -161,8 +161,22 @@ public void setObserverMasterPort(int observerMasterPort) { this.observerMasterPort = observerMasterPort; } - private int multiAddressReachabilityCheckTimeoutMs = (int) MultipleAddresses.DEFAULT_TIMEOUT.toMillis(); + public static final String CONFIG_KEY_MULTI_ADDRESS_ENABLED = "zookeeper.multiAddress.enabled"; + public static final String CONFIG_DEFAULT_MULTI_ADDRESS_ENABLED = "false"; + + private boolean multiAddressEnabled = true; + public boolean isMultiAddressEnabled() { + return multiAddressEnabled; + } + + public void setMultiAddressEnabled(boolean multiAddressEnabled) { + this.multiAddressEnabled = multiAddressEnabled; + LOG.info("multiAddress.enabled set to {}", multiAddressEnabled); + } + public static final String CONFIG_KEY_MULTI_ADDRESS_REACHABILITY_CHECK_TIMEOUT_MS = "zookeeper.multiAddress.reachabilityCheckTimeoutMs"; + + private int multiAddressReachabilityCheckTimeoutMs = (int) MultipleAddresses.DEFAULT_TIMEOUT.toMillis(); public int getMultiAddressReachabilityCheckTimeoutMs() { return multiAddressReachabilityCheckTimeoutMs; } @@ -172,6 +186,8 @@ public void setMultiAddressReachabilityCheckTimeoutMs(int multiAddressReachabili LOG.info("multiAddress.reachabilityCheckTimeoutMs set to {}", multiAddressReachabilityCheckTimeoutMs); } + public static final String CONFIG_KEY_MULTI_ADDRESS_REACHABILITY_CHECK_ENABLED = "zookeeper.multiAddress.reachabilityCheckEnabled"; + private boolean multiAddressReachabilityCheckEnabled = true; public boolean isMultiAddressReachabilityCheckEnabled() { @@ -274,6 +290,12 @@ public QuorumServer(long sid, String addressStr) throws ConfigException { } } + boolean multiAddressEnabled = Boolean.parseBoolean( + System.getProperty(QuorumPeer.CONFIG_KEY_MULTI_ADDRESS_ENABLED, QuorumPeer.CONFIG_DEFAULT_MULTI_ADDRESS_ENABLED)); + if (!multiAddressEnabled && serverAddresses.length > 1) { + throw new ConfigException("Multiple address feature is disabled, but multiple addresses were specified for sid " + sid); + } + for (String serverAddress : serverAddresses) { String serverParts[] = ConfigUtils.getHostAndPort(serverAddress); if ((serverClientParts.length > 2) || (serverParts.length < 3) 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 102d6995fdf..ae73ef35e0c 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 @@ -121,13 +121,13 @@ public class QuorumPeerConfig { protected int quorumCnxnThreadsSize; // multi address related configs + private boolean multiAddressEnabled = Boolean.parseBoolean( + System.getProperty(QuorumPeer.CONFIG_KEY_MULTI_ADDRESS_ENABLED, QuorumPeer.CONFIG_DEFAULT_MULTI_ADDRESS_ENABLED)); private boolean multiAddressReachabilityCheckEnabled = - Boolean.parseBoolean(System.getProperty("zookeeper.multiAddress.reachabilityCheckEnabled", - "true")); + Boolean.parseBoolean(System.getProperty(QuorumPeer.CONFIG_KEY_MULTI_ADDRESS_REACHABILITY_CHECK_ENABLED, "true")); private int multiAddressReachabilityCheckTimeoutMs = - Integer.parseInt(System.getProperty("zookeeper.multiAddress.reachabilityCheckTimeoutMs", - String.valueOf(MultipleAddresses.DEFAULT_TIMEOUT.toMillis()))); - + Integer.parseInt(System.getProperty(QuorumPeer.CONFIG_KEY_MULTI_ADDRESS_REACHABILITY_CHECK_TIMEOUT_MS, + String.valueOf(MultipleAddresses.DEFAULT_TIMEOUT.toMillis()))); /** * Minimum snapshot retain count. @@ -398,6 +398,8 @@ public void parseProperties(Properties zkProp) throws IOException, ConfigExcepti } else if (key.startsWith("metricsProvider.")) { String keyForMetricsProvider = key.substring(16); metricsProviderConfiguration.put(keyForMetricsProvider, value); + } else if (key.equals("multiAddress.enabled")) { + multiAddressEnabled = Boolean.parseBoolean(value); } else if (key.equals("multiAddress.reachabilityCheckTimeoutMs")) { multiAddressReachabilityCheckTimeoutMs = Integer.parseInt(value); } else if (key.equals("multiAddress.reachabilityCheckEnabled")) { @@ -939,6 +941,10 @@ public Boolean getQuorumListenOnAllIPs() { return quorumListenOnAllIPs; } + public boolean isMultiAddressEnabled() { + return multiAddressEnabled; + } + public boolean isMultiAddressReachabilityCheckEnabled() { return multiAddressReachabilityCheckEnabled; } diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumPeerMain.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumPeerMain.java index 04aec05acee..a6f94ec3bd1 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumPeerMain.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumPeerMain.java @@ -204,6 +204,7 @@ public void runFromConfig(QuorumPeerConfig config) throws IOException, AdminServ if (config.sslQuorumReloadCertFiles) { quorumPeer.getX509Util().enableCertFileReloading(); } + quorumPeer.setMultiAddressEnabled(config.isMultiAddressEnabled()); quorumPeer.setMultiAddressReachabilityCheckEnabled(config.isMultiAddressReachabilityCheckEnabled()); quorumPeer.setMultiAddressReachabilityCheckTimeoutMs(config.getMultiAddressReachabilityCheckTimeoutMs()); diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/CnxManagerTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/CnxManagerTest.java index 371fc390e3d..269b6a88102 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/CnxManagerTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/CnxManagerTest.java @@ -261,7 +261,7 @@ public void testCnxManagerSpinLock() throws Exception { InetSocketAddress otherAddr = peers.get(2L).electionAddr.getReachableOrOne(); DataOutputStream dout = new DataOutputStream(sc.socket().getOutputStream()); - dout.writeLong(QuorumCnxManager.PROTOCOL_VERSION); + dout.writeLong(QuorumCnxManager.PROTOCOL_VERSION_V1); dout.writeLong(2); String addr = otherAddr.getHostString() + ":" + otherAddr.getPort(); byte[] addr_bytes = addr.getBytes(); @@ -604,7 +604,7 @@ public void testInitialMessage() throws Exception { dout.writeBytes(hostport); din = new DataInputStream(new ByteArrayInputStream(bos.toByteArray())); - msg = InitialMessage.parse(QuorumCnxManager.PROTOCOL_VERSION, din); + msg = InitialMessage.parse(QuorumCnxManager.PROTOCOL_VERSION_V1, din); fail("long message accepted"); } catch (InitialMessage.InitialMessageException ex) { } @@ -620,7 +620,7 @@ public void testInitialMessage() throws Exception { dout.writeBytes(hostport); din = new DataInputStream(new ByteArrayInputStream(bos.toByteArray())); - msg = InitialMessage.parse(QuorumCnxManager.PROTOCOL_VERSION, din); + msg = InitialMessage.parse(QuorumCnxManager.PROTOCOL_VERSION_V1, din); fail("bad hostport accepted"); } catch (InitialMessage.InitialMessageException ex) { } @@ -637,7 +637,7 @@ public void testInitialMessage() throws Exception { // now parse it din = new DataInputStream(new ByteArrayInputStream(bos.toByteArray())); - msg = InitialMessage.parse(QuorumCnxManager.PROTOCOL_VERSION, din); + msg = InitialMessage.parse(QuorumCnxManager.PROTOCOL_VERSION_V1, din); assertEquals(Long.valueOf(5), msg.sid); assertEquals(Arrays.asList(new InetSocketAddress("10.0.0.2", 3888)), msg.electionAddr); } catch (InitialMessage.InitialMessageException ex) { @@ -656,7 +656,7 @@ public void testInitialMessage() throws Exception { // now parse it din = new DataInputStream(new ByteArrayInputStream(bos.toByteArray())); - msg = InitialMessage.parse(QuorumCnxManager.PROTOCOL_VERSION, din); + msg = InitialMessage.parse(QuorumCnxManager.PROTOCOL_VERSION_V2, din); assertEquals(Long.valueOf(5), msg.sid); assertEquals(Arrays.asList(new InetSocketAddress("1.1.1.1", 9999), new InetSocketAddress("2.2.2.2", 8888), diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/LearnerTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/LearnerTest.java index 63381fc7640..97ab5a26f97 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/LearnerTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/LearnerTest.java @@ -53,6 +53,7 @@ import org.apache.zookeeper.txn.CreateTxn; import org.apache.zookeeper.txn.TxnHeader; import org.apache.zookeeper.util.ServiceUtils; +import org.junit.After; import org.junit.Test; public class LearnerTest extends ZKTestCase { @@ -134,6 +135,11 @@ protected Socket createSocket() throws X509Exception, IOException { } } + @After + public void cleanup() { + System.clearProperty(QuorumPeer.CONFIG_KEY_MULTI_ADDRESS_ENABLED); + } + @Test(expected = IOException.class) public void connectionRetryTimeoutTest() throws Exception { Learner learner = new TestLearner(); @@ -178,6 +184,7 @@ public void connectionInitLimitTimeoutTest() throws Exception { @Test public void shouldTryMultipleAddresses() throws Exception { + System.setProperty(QuorumPeer.CONFIG_KEY_MULTI_ADDRESS_ENABLED, "true"); TestLearner learner = new TestLearner(); learner.self = new QuorumPeer(); learner.self.setTickTime(2000); @@ -205,6 +212,7 @@ public void shouldTryMultipleAddresses() throws Exception { @Test public void multipleAddressesSomeAreFailing() throws Exception { + System.setProperty(QuorumPeer.CONFIG_KEY_MULTI_ADDRESS_ENABLED, "true"); TestLearner learner = new TestLearner(); learner.self = new QuorumPeer(); learner.self.setTickTime(2000); diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/QuorumPeerMainMultiAddressTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/QuorumPeerMainMultiAddressTest.java index ae3557315d0..9c2906c122f 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/QuorumPeerMainMultiAddressTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/QuorumPeerMainMultiAddressTest.java @@ -19,6 +19,7 @@ package org.apache.zookeeper.server.quorum; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.fail; import java.io.IOException; import java.util.Arrays; import java.util.List; @@ -52,6 +53,7 @@ public class QuorumPeerMainMultiAddressTest extends QuorumPeerTestBase { @Before public void setUp() throws Exception { + System.setProperty(QuorumPeer.CONFIG_KEY_MULTI_ADDRESS_ENABLED, "true"); ClientBase.setupTestEnv(); System.setProperty("zookeeper.DigestAuthenticationProvider.superDigest", "super:D/InIHSb7yEEbrWz8b9l71RjZJU="/* password is 'test'*/); QuorumPeerConfig.setReconfigEnabled(true); @@ -63,6 +65,7 @@ public void setUp() throws Exception { @After public void tearDown() throws Exception { super.tearDown(); + System.clearProperty(QuorumPeer.CONFIG_KEY_MULTI_ADDRESS_ENABLED); System.clearProperty("zookeeper.jmx.log4j.disable"); } @@ -230,6 +233,34 @@ public void shouldReconfigIncrementally_IPv6() throws Exception { checkIfZooKeeperQuorumWorks(newQuorumConfig); } + @Test + public void shouldFailToReconfigWithMultipleAddressesWhenFeatureIsDisabled() throws Exception { + System.setProperty(QuorumPeer.CONFIG_KEY_MULTI_ADDRESS_ENABLED, "false"); + + // we have three ZK servers, each server has a single quorumPort and single electionPort registered + QuorumServerConfigBuilder initialQuorumConfig = new QuorumServerConfigBuilder(hostName, 3, 1); + + // we launch the three servers, each server should use the same initial config + launchServers(Arrays.asList(initialQuorumConfig, initialQuorumConfig, initialQuorumConfig)); + + checkIfZooKeeperQuorumWorks(initialQuorumConfig); + + // we create a new config where we add a new address to one of the servers with random available ports + QuorumServerConfigBuilder newQuorumConfig = new QuorumServerConfigBuilder(initialQuorumConfig) + .addNewServerAddress(FIRST_SERVER); + + ZooKeeperAdmin zkAdmin = newZooKeeperAdmin(initialQuorumConfig); + + // initiating a new incremental reconfig by using the updated ports, expecting exceptions here + try { + ReconfigTest.reconfig(zkAdmin, newQuorumConfig.buildAsStringList(), null, null, -1); + fail("Reconfig succeeded with multiple addresses without exception when the MultiAddress feature is disabled"); + } catch (KeeperException.BadArgumentsException e) { + // do nothing, this is what we expected + } catch (Exception e) { + fail("Reconfig failed in a wrong way. We expected KeeperException.BadArgumentsException."); + } + } private void launchServers(List builders) throws IOException, InterruptedException { diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/QuorumSSLTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/QuorumSSLTest.java index 09245c53d72..593edbd9245 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/QuorumSSLTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/QuorumSSLTest.java @@ -430,6 +430,7 @@ public void setSSLSystemProperties() { @After public void cleanUp() throws Exception { + System.clearProperty(QuorumPeer.CONFIG_KEY_MULTI_ADDRESS_ENABLED); clearSSLSystemProperties(); if (q1 != null) { q1.shutdown(); @@ -480,6 +481,7 @@ public void testQuorumSSL() throws Exception { @Test public void testQuorumSSLWithMultipleAddresses() throws Exception { + System.setProperty(QuorumPeer.CONFIG_KEY_MULTI_ADDRESS_ENABLED, "true"); quorumConfiguration = generateMultiAddressQuorumConfiguration(); q1 = new MainThread(1, clientPortQp1, quorumConfiguration, SSL_QUORUM_ENABLED); @@ -598,6 +600,7 @@ public void testHostnameVerificationWithInvalidIpAddressAndInvalidHostname() thr @Test public void testHostnameVerificationForInvalidMultiAddressServerConfig() throws Exception { + System.setProperty(QuorumPeer.CONFIG_KEY_MULTI_ADDRESS_ENABLED, "true"); quorumConfiguration = generateMultiAddressQuorumConfiguration(); String badhostnameKeystorePath = tmpDir + "/badhost.jks"; diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/QuorumServerTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/QuorumServerTest.java index c8cbadac547..6f5155bf022 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/QuorumServerTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/QuorumServerTest.java @@ -23,6 +23,7 @@ import org.apache.zookeeper.ZKTestCase; import org.apache.zookeeper.server.quorum.QuorumPeer.QuorumServer; import org.apache.zookeeper.server.quorum.QuorumPeerConfig.ConfigException; +import org.junit.After; import org.junit.Test; public class QuorumServerTest extends ZKTestCase { @@ -31,6 +32,11 @@ public class QuorumServerTest extends ZKTestCase { private String ipv6n2 = "[2600:0:0:0:0:0:1:0]"; private String ipv4config = "127.0.0.1:1234:1236"; + @After + public void tearDown() { + System.clearProperty(QuorumPeer.CONFIG_KEY_MULTI_ADDRESS_ENABLED); + } + @Test public void testToString() throws ConfigException { String provided = ipv4config + ":participant;0.0.0.0:1237"; @@ -88,6 +94,19 @@ public void unbalancedIpv6LiteralsInClientConfigFailToBeParsed() throws ConfigEx new QuorumServer(0, ipv4config + ":participant;[::1:1237"); } + @Test(expected = ConfigException.class) + public void shouldNotAllowMultipleAddressesWhenMultiAddressFeatureIsDisabled() throws ConfigException { + System.setProperty(QuorumPeer.CONFIG_KEY_MULTI_ADDRESS_ENABLED, "false"); + new QuorumServer(0, "127.0.0.1:1234:1236|127.0.0.1:2234:2236"); + } + + @Test + public void shouldAllowMultipleAddressesWhenMultiAddressFeatureIsEnabled() throws ConfigException { + System.setProperty(QuorumPeer.CONFIG_KEY_MULTI_ADDRESS_ENABLED, "true"); + QuorumServer qs = new QuorumServer(0, "127.0.0.1:1234:1236|127.0.0.1:2234:2236"); + assertEquals("MultiAddress parse error", "127.0.0.1:1234:1236|127.0.0.1:2234:2236:participant", qs.toString()); + } + @Test public void testWildcard() throws KeeperException.BadArgumentsException { String[] addrs = new String[]{"127.0.0.1", "[0:0:0:0:0:0:0:1]", "0.0.0.0", "[::]"}; diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/auth/QuorumAuthTestBase.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/auth/QuorumAuthTestBase.java index 97650d0c694..d365c9a97d5 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/auth/QuorumAuthTestBase.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/auth/QuorumAuthTestBase.java @@ -29,8 +29,10 @@ import org.apache.commons.io.FileUtils; import org.apache.zookeeper.PortAssignment; import org.apache.zookeeper.ZKTestCase; +import org.apache.zookeeper.server.quorum.QuorumPeer; import org.apache.zookeeper.server.quorum.QuorumPeerTestBase.MainThread; import org.apache.zookeeper.test.ClientBase; +import org.junit.After; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -69,6 +71,11 @@ public static void cleanupJaasConfig() { } } + @After + public void tearDown() throws Exception { + System.clearProperty(QuorumPeer.CONFIG_KEY_MULTI_ADDRESS_ENABLED); + } + protected String startQuorum(final int serverCount, Map authConfigs, int authServerCount) throws IOException { return this.startQuorum(serverCount, authConfigs, authServerCount, false); @@ -76,6 +83,7 @@ protected String startQuorum(final int serverCount, Map authConf protected String startMultiAddressQuorum(final int serverCount, Map authConfigs, int authServerCount) throws IOException { + System.setProperty(QuorumPeer.CONFIG_KEY_MULTI_ADDRESS_ENABLED, "true"); return this.startQuorum(serverCount, authConfigs, authServerCount, true); } diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/auth/QuorumAuthUpgradeTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/auth/QuorumAuthUpgradeTest.java index b944221928d..ecdfaeb469a 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/auth/QuorumAuthUpgradeTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/auth/QuorumAuthUpgradeTest.java @@ -67,8 +67,10 @@ public class QuorumAuthUpgradeTest extends QuorumAuthTestBase { } @After + @Override public void tearDown() throws Exception { shutdownAll(); + super.tearDown(); } @AfterClass diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/auth/QuorumDigestAuthTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/auth/QuorumDigestAuthTest.java index 7462979d7cd..cb28389efab 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/auth/QuorumDigestAuthTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/auth/QuorumDigestAuthTest.java @@ -60,11 +60,13 @@ public class QuorumDigestAuthTest extends QuorumAuthTestBase { } @After + @Override public void tearDown() throws Exception { for (MainThread mainThread : mt) { mainThread.shutdown(); mainThread.deleteBaseDir(); } + super.tearDown(); } @AfterClass diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/auth/QuorumKerberosAuthTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/auth/QuorumKerberosAuthTest.java index e4534dfd21f..28b93c7b487 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/auth/QuorumKerberosAuthTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/auth/QuorumKerberosAuthTest.java @@ -88,11 +88,13 @@ public static void setUp() throws Exception { } @After + @Override public void tearDown() throws Exception { for (MainThread mainThread : mt) { mainThread.shutdown(); mainThread.deleteBaseDir(); } + super.tearDown(); } @AfterClass diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/auth/QuorumKerberosHostBasedAuthTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/auth/QuorumKerberosHostBasedAuthTest.java index fc150490e61..c9b93c57bde 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/auth/QuorumKerberosHostBasedAuthTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/auth/QuorumKerberosHostBasedAuthTest.java @@ -113,11 +113,13 @@ public static void setUp() throws Exception { } @After + @Override public void tearDown() throws Exception { for (MainThread mainThread : mt) { mainThread.shutdown(); mainThread.deleteBaseDir(); } + super.tearDown(); } @AfterClass From 3da01e0b522204c1086e83e808825bed7c1997fb Mon Sep 17 00:00:00 2001 From: Enrico Olivelli Date: Mon, 17 Feb 2020 19:57:54 +0100 Subject: [PATCH 037/118] Fix release notes for 3.6.0rc3 --- zookeeper-docs/src/main/resources/markdown/releasenotes.md | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/zookeeper-docs/src/main/resources/markdown/releasenotes.md b/zookeeper-docs/src/main/resources/markdown/releasenotes.md index 5436884cb8e..e51e0809162 100644 --- a/zookeeper-docs/src/main/resources/markdown/releasenotes.md +++ b/zookeeper-docs/src/main/resources/markdown/releasenotes.md @@ -294,6 +294,8 @@ limitations under the License. ## Bug +* [ZOOKEEPER-3231](https://issues.apache.org/jira/browse/ZOOKEEPER-3231) - Purge task may lost data when the recent snapshots are all invalid +* [ZOOKEEPER-3720](https://issues.apache.org/jira/browse/ZOOKEEPER-3720) - Fix rolling upgrade failure (invalid protocol version) * [ZOOKEEPER-3677](https://issues.apache.org/jira/browse/ZOOKEEPER-3677) - Setting jute.maxbuffer value in hexadecimal throws Exception * [ZOOKEEPER-3695](https://issues.apache.org/jira/browse/ZOOKEEPER-3695) - Source release tarball does not match repository in 3.6.0 * [ZOOKEEPER-3667](https://issues.apache.org/jira/browse/ZOOKEEPER-3667) - owasp checker failing for - CVE-2019-17571 Apache Log4j 1.2 deserialization of untrusted data in SocketServer @@ -639,7 +641,7 @@ limitations under the License. * [ZOOKEEPER-3715](https://issues.apache.org/jira/browse/ZOOKEEPER-3715) - fix Kerberos test failures for new JDK versions * [ZOOKEEPER-3716](https://issues.apache.org/jira/browse/ZOOKEEPER-3716) - upgrade netty 4.1.42 to address CVE-2019-20444 CVE-2019-20445 - +* [ZOOKEEPER-3708](https://issues.apache.org/jira/browse/ZOOKEEPER-3708) - Move Logging Code into Logging Guard in Learner * [ZOOKEEPER-3669](https://issues.apache.org/jira/browse/ZOOKEEPER-3669) - Use switch Statement in ClientCnxn SendThread * [ZOOKEEPER-3704](https://issues.apache.org/jira/browse/ZOOKEEPER-3704) - upgrade maven dependency-check to 5.3.0 * [ZOOKEEPER-925](https://issues.apache.org/jira/browse/ZOOKEEPER-925) - Consider maven site generation to replace our forrest site and documentation generation From 9807f0a89504c983b6a44f75bcce41c776d700e3 Mon Sep 17 00:00:00 2001 From: Enrico Olivelli Date: Sun, 23 Feb 2020 08:41:20 -0800 Subject: [PATCH 038/118] ZOOKEEPER-3734: upgrade jackson-databind to address CVE-2020-8840 Author: Enrico Olivelli Reviewers: phunt@apache.org Closes #1262 from eolivelli/fix/ZOOKEEPER-3734-jackson-again Change-Id: I21df2c160f8dd9c2542c153e4aa53500b58e8144 (cherry picked from commit f0c6ae5eb205d7c546483a6f1769cb58b68169a6) Signed-off-by: Patrick Hunt --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 71e16d84a96..3d28fa972ac 100755 --- a/pom.xml +++ b/pom.xml @@ -327,7 +327,7 @@ 1.2 4.1.45.Final 9.4.24.v20191120 - 2.9.10.2 + 2.9.10.3 1.1.1 2.11 1.1.7 From 3c4f24bc2a3da998a25ad1e17fcfcd7606519e43 Mon Sep 17 00:00:00 2001 From: Enrico Olivelli Date: Sun, 23 Feb 2020 21:14:46 -0800 Subject: [PATCH 039/118] ZOOKEEPER-3733: Fix issues reported in 3.6.0rc3 - metrics library LICENSE file has wrong file name - spotbugs is not passing when '-SNAPSHOT' suffix is removed because the Info.java interface sets a null value for "qualifier" - the name of the directory inside the source tarbal is not consistent with the file name and with 3.5 tradition Author: Enrico Olivelli Reviewers: phunt@apache.org Closes #1258 from eolivelli/fix/ZOOKEEPER-3733 Change-Id: I70f4bf96fcd3d8371e1251a60199fe5ef1fdd4d4 (cherry picked from commit b2db3918cb2b681e0446cfe3247730deacc0567b) Signed-off-by: Patrick Hunt --- pom.xml | 1 + .../src/main/java/org/apache/zookeeper/version/util/VerGen.java | 2 +- ...5.2.5.jar_LICENSE.txt => metrics-core-3.2.5.jar_LICENSE.txt} | 0 3 files changed, 2 insertions(+), 1 deletion(-) rename zookeeper-server/src/main/resources/lib/{metrics-core-5.2.5.jar_LICENSE.txt => metrics-core-3.2.5.jar_LICENSE.txt} (100%) diff --git a/pom.xml b/pom.xml index 3d28fa972ac..7612a0aaf34 100755 --- a/pom.xml +++ b/pom.xml @@ -295,6 +295,7 @@ ${sourceReleaseAssemblyDescriptor} apache-zookeeper-${project.version} + false tar.gz 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 3e9e62aee1c..25bffd72866 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 @@ -80,7 +80,7 @@ public static void generateFile(File outputDir, Version version, String rev, Str 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 ? null : "\"" + version.qualifier + "\"") + ";\n"); + w.write(" String QUALIFIER=" + (version.qualifier == null ? "\"\"" : "\"" + version.qualifier + "\"") + ";\n"); if (rev.equals("-1")) { System.out.println("Unknown REVISION number, using " + rev); } diff --git a/zookeeper-server/src/main/resources/lib/metrics-core-5.2.5.jar_LICENSE.txt b/zookeeper-server/src/main/resources/lib/metrics-core-3.2.5.jar_LICENSE.txt similarity index 100% rename from zookeeper-server/src/main/resources/lib/metrics-core-5.2.5.jar_LICENSE.txt rename to zookeeper-server/src/main/resources/lib/metrics-core-3.2.5.jar_LICENSE.txt From 59193605109621c6cec779ee13a4afa1aee34274 Mon Sep 17 00:00:00 2001 From: Enrico Olivelli Date: Tue, 25 Feb 2020 14:56:22 +0100 Subject: [PATCH 040/118] Update release notes --- zookeeper-docs/src/main/resources/markdown/releasenotes.md | 2 ++ 1 file changed, 2 insertions(+) diff --git a/zookeeper-docs/src/main/resources/markdown/releasenotes.md b/zookeeper-docs/src/main/resources/markdown/releasenotes.md index e51e0809162..4cf8d1628ad 100644 --- a/zookeeper-docs/src/main/resources/markdown/releasenotes.md +++ b/zookeeper-docs/src/main/resources/markdown/releasenotes.md @@ -639,6 +639,8 @@ limitations under the License. ## Task +* [ZOOKEEPER-3733](https://issues.apache.org/jira/browse/ZOOKEEPER-3733) - Fix issues reported in 3.6.0rc3 +* [ZOOKEEPER-3734](https://issues.apache.org/jira/browse/ZOOKEEPER-3734) - upgrade jackson-databind to address CVE-2020-8840 * [ZOOKEEPER-3715](https://issues.apache.org/jira/browse/ZOOKEEPER-3715) - fix Kerberos test failures for new JDK versions * [ZOOKEEPER-3716](https://issues.apache.org/jira/browse/ZOOKEEPER-3716) - upgrade netty 4.1.42 to address CVE-2019-20444 CVE-2019-20445 * [ZOOKEEPER-3708](https://issues.apache.org/jira/browse/ZOOKEEPER-3708) - Move Logging Code into Logging Guard in Learner From 06d4a8fe53a2b99189d3a7aefcf06f91799fdefc Mon Sep 17 00:00:00 2001 From: Mate Szalay-Beko Date: Thu, 27 Feb 2020 14:22:06 +0100 Subject: [PATCH 041/118] ZOOKEEPER-3741: fix buffer length in C client causing warning with new gcc We get a warning that we are trying to call like `sprintf(buf,"%s:%d",addrstr,port);`, and both `buf` and `addrstr` are 128 long char arrays. So in theory, we can overflow. The fix is to increase the length of the destination string array (`buf`). Actually this problem only causing compile time warning / failure only on the 3.5.5 and 3.5.6. On 3.5.7 this was fixed and on 3.6+ branches the compiler can not detect the problem due to some code refactoring made by ZOOKEEPER-3068, but the issue is still present on the master branch. Author: Mate Szalay-Beko Reviewers: Enrico Olivelli , Norbert Kalmar Closes #1273 from symat/ZOOKEEPER-3741 (cherry picked from commit 4b45ff1bda65b27d2771322d4613fccfb8a726b0) Signed-off-by: Norbert Kalmar --- zookeeper-client/zookeeper-client-c/src/zookeeper.c | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/zookeeper-client/zookeeper-client-c/src/zookeeper.c b/zookeeper-client/zookeeper-client-c/src/zookeeper.c index d2b73561773..3ba3c526697 100644 --- a/zookeeper-client/zookeeper-client-c/src/zookeeper.c +++ b/zookeeper-client/zookeeper-client-c/src/zookeeper.c @@ -4807,7 +4807,7 @@ int zoo_add_auth(zhandle_t *zh,const char* scheme,const char* cert, static const char* format_endpoint_info(const struct sockaddr_storage* ep) { - static char buf[128] = { 0 }; + static char buf[134] = { 0 }; char addrstr[INET6_ADDRSTRLEN] = { 0 }; const char *fmtstring; void *inaddr; From 3dcc996ee6089d8001d699d2461decddf4d76c4f Mon Sep 17 00:00:00 2001 From: Christopher Tubbs Date: Sat, 29 Feb 2020 21:42:47 -0800 Subject: [PATCH 042/118] ZOOKEEPER-3737: Detect log4j 1.2 jmx support better * Look for jmx class that exists only in the log4j 1.2 jar, but not in the log4j2 1.2 compatibility jar. * Check if disabled before attempting to detect log4j 1.2 jmx classes. * Update log messages to highlight that they are referring to log4j 1.2 and not log4j2 or other versions. * Minor javadoc fixup Author: Christopher Tubbs Reviewers: phunt@apache.org Closes #1270 from ctubbsii/ZK-3737 Change-Id: I7ebd7a28386d3fee33fcc9078d7f573dc766e8ee (cherry picked from commit 5a2332058deba04313ad37fa05cbb1515c83b8e6) Signed-off-by: Patrick Hunt --- .../org/apache/zookeeper/jmx/ManagedUtil.java | 24 +++++++++---------- 1 file changed, 11 insertions(+), 13 deletions(-) diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/jmx/ManagedUtil.java b/zookeeper-server/src/main/java/org/apache/zookeeper/jmx/ManagedUtil.java index b476daad095..633cb17c2e1 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/jmx/ManagedUtil.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/jmx/ManagedUtil.java @@ -35,27 +35,25 @@ public class ManagedUtil { private static boolean isLog4jJmxEnabled() { boolean enabled = false; - try { - Class.forName("org.apache.log4j.spi.LoggerRepository"); - - if (Boolean.getBoolean("zookeeper.jmx.log4j.disable")) { - LOG.info("Log4j found but jmx support is disabled."); - } else { + if (Boolean.getBoolean("zookeeper.jmx.log4j.disable")) { + LOG.info("Log4j 1.2 jmx support is disabled by property."); + } else { + try { + Class.forName("org.apache.log4j.jmx.HierarchyDynamicMBean"); enabled = true; - LOG.info("Log4j found with jmx enabled."); + LOG.info("Log4j 1.2 jmx support found and enabled."); + } catch (ClassNotFoundException e) { + LOG.info("Log4j 1.2 jmx support not found; jmx disabled."); } - - } catch (ClassNotFoundException e) { - LOG.info("Log4j not found."); } return enabled; } /** - * Register the log4j JMX mbeans. Set environment variable + * Register the log4j JMX mbeans. Set system property * "zookeeper.jmx.log4j.disable" to true to disable registration. - * See http://logging.apache.org/log4j/1.2/apidocs/index.html?org/apache/log4j/jmx/package-summary.html + * @see http://logging.apache.org/log4j/1.2/apidocs/index.html?org/apache/log4j/jmx/package-summary.html * @throws JMException if registration fails */ @SuppressWarnings("rawtypes") @@ -108,7 +106,7 @@ public static void registerLog4jMBeans() throws JMException { hdm.getClass().getMethod("addLoggerMBean", String.class).invoke(hdm, loggerName); } } catch (Exception e) { - LOG.error("Problems while registering log4j jmx beans!", e); + LOG.error("Problems while registering log4j 1.2 jmx beans!", e); throw new JMException(e.toString()); } } From a93ff0fe631d1c96ee056a79e3c16535ab33c794 Mon Sep 17 00:00:00 2001 From: Christopher Tubbs Date: Mon, 2 Mar 2020 17:08:47 +0100 Subject: [PATCH 043/118] ZOOKEEPER-3738: Use mavanagaiata for git commit id Use mavanagaiata plugin to get git commit id for VerGen instead of properties-maven-plugin (which is broken in some environments). Also add the commit to the jar manifests for easy reference when given a jar of unknown origin (especially useful for SNAPSHOT builds). Author: Christopher Tubbs Reviewers: Enrico Olivelli , Norbert Kalmar Closes #1268 from ctubbsii/ZK-3738 (cherry picked from commit 050e56ae48b978a9f8c04dfe00e1647e4301f5a0) Signed-off-by: Norbert Kalmar --- .gitignore | 1 - pom.xml | 47 +++++++++++++++------------------------- zookeeper-server/pom.xml | 19 +--------------- 3 files changed, 18 insertions(+), 49 deletions(-) diff --git a/.gitignore b/.gitignore index cf5ff24f045..55937cec57e 100644 --- a/.gitignore +++ b/.gitignore @@ -9,7 +9,6 @@ zookeeper-server/version-2/* # SVN .svn .revision -git.properties # Eclipse .metadata diff --git a/pom.xml b/pom.xml index 7612a0aaf34..cebbec8d212 100755 --- a/pom.xml +++ b/pom.xml @@ -525,11 +525,6 @@ - - org.codehaus.mojo - properties-maven-plugin - 1.0.0 - org.apache.maven.plugins maven-compiler-plugin @@ -551,6 +546,13 @@ org.apache.maven.plugins maven-jar-plugin 3.1.0 + + + + ${mvngit.commit.id} + + + org.apache.maven.plugins @@ -601,10 +603,12 @@ 1.6.0 - pl.project13.maven - git-commit-id-plugin - 2.2.5 - false + com.github.koraktor + mavanagaiata + 0.9.4 + + true + org.codehaus.mojo @@ -680,33 +684,17 @@ - pl.project13.maven - git-commit-id-plugin + com.github.koraktor + mavanagaiata find-current-git-revision - revision + commit - generate-resources + validate - - false - true - false - ${project.basedir}/.git - git - false - true - ${project.basedir}/zookeeper-server/src/main/resources/git.properties - properties - - false - false - -dirty - - org.openclover @@ -847,7 +835,6 @@ **/log4j.properties **/README.md - **/git.properties **/findbugsExcludeFile.xml **/checkstyle-noframes-sorted.xsl **/configure.ac diff --git a/zookeeper-server/pom.xml b/zookeeper-server/pom.xml index 5fb044a9422..07822f01bb4 100755 --- a/zookeeper-server/pom.xml +++ b/zookeeper-server/pom.xml @@ -167,23 +167,6 @@ - - org.codehaus.mojo - properties-maven-plugin - - - initialize - - read-project-properties - - - - ${basedir}/src/main/resources/git.properties - - - - - org.codehaus.mojo build-helper-maven-plugin @@ -249,7 +232,7 @@ org.apache.zookeeper.version.util.VerGen ${project.version} - ${git.commit.id} + ${mvngit.commit.id} ${build.time} ${project.basedir}/target/generated-sources/java From 562863979515140cdf00b848aea2810706226745 Mon Sep 17 00:00:00 2001 From: Mate Szalay-Beko Date: Tue, 3 Mar 2020 10:53:43 +0100 Subject: [PATCH 044/118] ZOOKEEPER-3740: fix flaky PurgeTxnTest.testPurgeWhenLogRollingInProgress This test is always passing for me when I execute only the PurgeTxnTest test class locally, but when I execute all the tests (with multiple parallel threads, using `mvn clean install`), then it always fails. It is failing frequently on the `zookeeper-master-maven` Jenkins job as well. The test starts three threads, performing 1000 ZNode creation in each thread and timeouts if the threads are not finished in 90 seconds. Currently it is not easy to tell based on the logs if the timeout happens because the operations are still in progress or if one of the threads terminated due to an unexpected exception. In this patch I: - increased the timeout from 90 to 120 seconds - added an extra logic to actually fail because of the Exception on the threads, if any happen during the execution - I decreased the number of ZNode creations to 750 (from the original 1000) Applying this patch locally fixed my issues, I hope it will be enough to fix the test on Jenkins as well. Author: Mate Szalay-Beko Reviewers: Enrico Olivelli , Norbert Kalmar Closes #1274 from symat/ZOOKEEPER-3740 (cherry picked from commit 118dee4b8a4a69699eb00e12bef2907cbbece160) Signed-off-by: Norbert Kalmar --- .../apache/zookeeper/server/PurgeTxnTest.java | 40 +++++++++++-------- 1 file changed, 23 insertions(+), 17 deletions(-) diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/PurgeTxnTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/PurgeTxnTest.java index 84d907649cf..804e23726ee 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/PurgeTxnTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/PurgeTxnTest.java @@ -33,6 +33,7 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; import java.util.zip.CheckedOutputStream; import org.apache.jute.BinaryOutputArchive; import org.apache.jute.OutputArchive; @@ -59,7 +60,7 @@ public class PurgeTxnTest extends ZKTestCase { private static final Logger LOG = LoggerFactory.getLogger(PurgeTxnTest.class); private static String HOSTPORT = "127.0.0.1:" + PortAssignment.unique(); private static final int CONNECTION_TIMEOUT = 3000; - private static final long OP_TIMEOUT_IN_MILLIS = 90000; + private static final long OP_TIMEOUT_IN_MILLIS = 120000; private File tmpDir; @Before @@ -561,25 +562,25 @@ private List manyClientOps(final ZooKeeper zk, final CountDownLatch doPu Thread[] ths = new Thread[thCount]; final List znodes = Collections.synchronizedList(new ArrayList()); final CountDownLatch finished = new CountDownLatch(thCount); + final AtomicReference exception = new AtomicReference<>(); for (int indx = 0; indx < thCount; indx++) { final String myprefix = prefix + "-" + indx; - Thread th = new Thread() { - public void run() { - for (int i = 0; i < 1000; i++) { - try { - String mynode = myprefix + "-" + i; - znodes.add(mynode); - zk.create(mynode, new byte[0], Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); - } catch (Exception e) { - LOG.error("Unexpected exception", e); - } - if (i == 200) { - doPurge.countDown(); - } + Thread th = new Thread(() -> { + for (int i = 0; i < 750; i++) { + try { + String mynode = myprefix + "-" + i; + znodes.add(mynode); + zk.create(mynode, new byte[0], Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); + } catch (Exception e) { + LOG.error("Unexpected exception during ZkClient ops", e); + exception.set(e); + } + if (i == 200) { + doPurge.countDown(); } - finished.countDown(); } - }; + finished.countDown(); + }); ths[indx] = th; } @@ -587,7 +588,12 @@ public void run() { thread.start(); } try { - assertTrue("ZkClient ops is not finished!", finished.await(OP_TIMEOUT_IN_MILLIS, TimeUnit.MILLISECONDS)); + boolean operationsFinishedSuccessfully = finished.await(OP_TIMEOUT_IN_MILLIS, TimeUnit.MILLISECONDS); + if (exception.get() != null) { + LOG.error("unexpected exception during running ZkClient ops:", exception.get()); + fail("unexpected exception during running ZkClient ops, see in the logs above"); + } + assertTrue("ZkClient ops not finished in time!", operationsFinishedSuccessfully); } catch (InterruptedException ie) { LOG.error("Unexpected exception", ie); fail("Unexpected exception occurred!"); From e99e78d25da5e995d399903db0e44657a1e0de93 Mon Sep 17 00:00:00 2001 From: Enrico Olivelli Date: Tue, 3 Mar 2020 22:17:43 +0100 Subject: [PATCH 045/118] Start ZooKeeper 3.6.1-SNAPSHOT --- pom.xml | 2 +- zookeeper-assembly/pom.xml | 2 +- zookeeper-client/pom.xml | 2 +- zookeeper-client/zookeeper-client-c/pom.xml | 2 +- zookeeper-contrib/pom.xml | 2 +- zookeeper-contrib/zookeeper-contrib-loggraph/pom.xml | 2 +- zookeeper-contrib/zookeeper-contrib-rest/pom.xml | 2 +- zookeeper-contrib/zookeeper-contrib-zooinspector/pom.xml | 2 +- zookeeper-docs/pom.xml | 2 +- zookeeper-jute/pom.xml | 2 +- zookeeper-metrics-providers/pom.xml | 2 +- .../zookeeper-prometheus-metrics/pom.xml | 2 +- zookeeper-recipes/pom.xml | 2 +- zookeeper-recipes/zookeeper-recipes-election/pom.xml | 2 +- zookeeper-recipes/zookeeper-recipes-lock/pom.xml | 2 +- zookeeper-recipes/zookeeper-recipes-queue/pom.xml | 2 +- zookeeper-server/pom.xml | 2 +- 17 files changed, 17 insertions(+), 17 deletions(-) diff --git a/pom.xml b/pom.xml index cebbec8d212..bcb7e02e244 100755 --- a/pom.xml +++ b/pom.xml @@ -31,7 +31,7 @@ parent pom - 3.6.0-SNAPSHOT + 3.6.1-SNAPSHOT Apache ZooKeeper ZooKeeper is a centralized service for maintaining configuration information, naming, diff --git a/zookeeper-assembly/pom.xml b/zookeeper-assembly/pom.xml index 15210a36673..d9acf7a8684 100755 --- a/zookeeper-assembly/pom.xml +++ b/zookeeper-assembly/pom.xml @@ -23,7 +23,7 @@ org.apache.zookeeper parent - 3.6.0-SNAPSHOT + 3.6.1-SNAPSHOT .. diff --git a/zookeeper-client/pom.xml b/zookeeper-client/pom.xml index e05ef4c37e3..1ff2af84469 100755 --- a/zookeeper-client/pom.xml +++ b/zookeeper-client/pom.xml @@ -23,7 +23,7 @@ org.apache.zookeeper parent - 3.6.0-SNAPSHOT + 3.6.1-SNAPSHOT .. diff --git a/zookeeper-client/zookeeper-client-c/pom.xml b/zookeeper-client/zookeeper-client-c/pom.xml index 3536d1a3c27..3fc093052a0 100755 --- a/zookeeper-client/zookeeper-client-c/pom.xml +++ b/zookeeper-client/zookeeper-client-c/pom.xml @@ -23,7 +23,7 @@ org.apache.zookeeper zookeeper-client - 3.6.0-SNAPSHOT + 3.6.1-SNAPSHOT .. diff --git a/zookeeper-contrib/pom.xml b/zookeeper-contrib/pom.xml index 075363f3cfb..7cb947bc792 100755 --- a/zookeeper-contrib/pom.xml +++ b/zookeeper-contrib/pom.xml @@ -24,7 +24,7 @@ org.apache.zookeeper parent - 3.6.0-SNAPSHOT + 3.6.1-SNAPSHOT org.apache.zookeeper diff --git a/zookeeper-contrib/zookeeper-contrib-loggraph/pom.xml b/zookeeper-contrib/zookeeper-contrib-loggraph/pom.xml index 5e9449538e4..5eaca1c7bb4 100755 --- a/zookeeper-contrib/zookeeper-contrib-loggraph/pom.xml +++ b/zookeeper-contrib/zookeeper-contrib-loggraph/pom.xml @@ -23,7 +23,7 @@ org.apache.zookeeper zookeeper-contrib - 3.6.0-SNAPSHOT + 3.6.1-SNAPSHOT org.apache.zookeeper diff --git a/zookeeper-contrib/zookeeper-contrib-rest/pom.xml b/zookeeper-contrib/zookeeper-contrib-rest/pom.xml index aa9ea4dabd2..3339a0008ea 100755 --- a/zookeeper-contrib/zookeeper-contrib-rest/pom.xml +++ b/zookeeper-contrib/zookeeper-contrib-rest/pom.xml @@ -23,7 +23,7 @@ org.apache.zookeeper zookeeper-contrib - 3.6.0-SNAPSHOT + 3.6.1-SNAPSHOT org.apache.zookeeper diff --git a/zookeeper-contrib/zookeeper-contrib-zooinspector/pom.xml b/zookeeper-contrib/zookeeper-contrib-zooinspector/pom.xml index 87c52d91da9..da9e14854cc 100755 --- a/zookeeper-contrib/zookeeper-contrib-zooinspector/pom.xml +++ b/zookeeper-contrib/zookeeper-contrib-zooinspector/pom.xml @@ -23,7 +23,7 @@ org.apache.zookeeper zookeeper-contrib - 3.6.0-SNAPSHOT + 3.6.1-SNAPSHOT org.apache.zookeeper diff --git a/zookeeper-docs/pom.xml b/zookeeper-docs/pom.xml index eb56544de5d..35cb14650db 100644 --- a/zookeeper-docs/pom.xml +++ b/zookeeper-docs/pom.xml @@ -23,7 +23,7 @@ org.apache.zookeeper parent - 3.6.0-SNAPSHOT + 3.6.1-SNAPSHOT .. diff --git a/zookeeper-jute/pom.xml b/zookeeper-jute/pom.xml index c84c8d3b87c..bf570f48cef 100755 --- a/zookeeper-jute/pom.xml +++ b/zookeeper-jute/pom.xml @@ -23,7 +23,7 @@ org.apache.zookeeper parent - 3.6.0-SNAPSHOT + 3.6.1-SNAPSHOT .. diff --git a/zookeeper-metrics-providers/pom.xml b/zookeeper-metrics-providers/pom.xml index 34fd91965f6..f458614a42c 100755 --- a/zookeeper-metrics-providers/pom.xml +++ b/zookeeper-metrics-providers/pom.xml @@ -23,7 +23,7 @@ org.apache.zookeeper parent - 3.6.0-SNAPSHOT + 3.6.1-SNAPSHOT .. diff --git a/zookeeper-metrics-providers/zookeeper-prometheus-metrics/pom.xml b/zookeeper-metrics-providers/zookeeper-prometheus-metrics/pom.xml index d58e7e92776..65950ffa429 100755 --- a/zookeeper-metrics-providers/zookeeper-prometheus-metrics/pom.xml +++ b/zookeeper-metrics-providers/zookeeper-prometheus-metrics/pom.xml @@ -23,7 +23,7 @@ org.apache.zookeeper zookeeper-metrics-providers - 3.6.0-SNAPSHOT + 3.6.1-SNAPSHOT .. diff --git a/zookeeper-recipes/pom.xml b/zookeeper-recipes/pom.xml index 95fd060dec9..23fe3c405b4 100755 --- a/zookeeper-recipes/pom.xml +++ b/zookeeper-recipes/pom.xml @@ -24,7 +24,7 @@ org.apache.zookeeper parent - 3.6.0-SNAPSHOT + 3.6.1-SNAPSHOT org.apache.zookeeper diff --git a/zookeeper-recipes/zookeeper-recipes-election/pom.xml b/zookeeper-recipes/zookeeper-recipes-election/pom.xml index 636e46a6760..eb7a00bcc95 100755 --- a/zookeeper-recipes/zookeeper-recipes-election/pom.xml +++ b/zookeeper-recipes/zookeeper-recipes-election/pom.xml @@ -23,7 +23,7 @@ org.apache.zookeeper zookeeper-recipes - 3.6.0-SNAPSHOT + 3.6.1-SNAPSHOT org.apache.zookeeper diff --git a/zookeeper-recipes/zookeeper-recipes-lock/pom.xml b/zookeeper-recipes/zookeeper-recipes-lock/pom.xml index 64a3753f661..f4bdc2486da 100755 --- a/zookeeper-recipes/zookeeper-recipes-lock/pom.xml +++ b/zookeeper-recipes/zookeeper-recipes-lock/pom.xml @@ -23,7 +23,7 @@ org.apache.zookeeper zookeeper-recipes - 3.6.0-SNAPSHOT + 3.6.1-SNAPSHOT org.apache.zookeeper diff --git a/zookeeper-recipes/zookeeper-recipes-queue/pom.xml b/zookeeper-recipes/zookeeper-recipes-queue/pom.xml index bf234b047d5..b4705d77e1b 100755 --- a/zookeeper-recipes/zookeeper-recipes-queue/pom.xml +++ b/zookeeper-recipes/zookeeper-recipes-queue/pom.xml @@ -23,7 +23,7 @@ org.apache.zookeeper zookeeper-recipes - 3.6.0-SNAPSHOT + 3.6.1-SNAPSHOT org.apache.zookeeper diff --git a/zookeeper-server/pom.xml b/zookeeper-server/pom.xml index 07822f01bb4..b2c4dc1a29d 100755 --- a/zookeeper-server/pom.xml +++ b/zookeeper-server/pom.xml @@ -23,7 +23,7 @@ org.apache.zookeeper parent - 3.6.0-SNAPSHOT + 3.6.1-SNAPSHOT .. From 29b0122c621b6c383b381c4a30162577c92039ca Mon Sep 17 00:00:00 2001 From: Mate Szalay-Beko Date: Thu, 12 Mar 2020 14:47:27 +0100 Subject: [PATCH 046/118] ZOOKEEPER-2164: Quorum members can not rejoin after restart Ever since ZOOKEEPER-107 (released in 3.5.0) the servers are sending their addresses during initial connection requests. The receiving server can potentially use these addresses to send back a new connection request if the challenge is won by the receiver. If the server config contains wildcard address (e.g. 0.0.0.0 in case of IPv4) then the first connection request sent by A to B will contain this address. If the ID of A is smaller than the ID of B, then A will lose the challenge and the second connection request sent back by B will never reach A, as B will send the initial message to 0.0.0.0. So in any 3.5+ ZooKeeper, if wildcard addresses are used in the configs, then there might be some servers never able to rejoin to the quorum after they got restarted. In 3.5+ for backward compatibility reasons (needed during rolling upgrade) there is a version of the QuorumCnxManager.connectOne() method that needs no election address but use the last known address to initiate the connection. In this commit, we simply call this method if the address is a wildcard address. I also added a few restart realted tests, to make sure that restart still works when we don't use wildcard addresses. We can not test the original error with unit tests, as it would require to start the quorum on multiple hosts. I also tested the patch for rolling restart manually both with and without wildcard addresses in the config. Author: Mate Szalay-Beko Reviewers: Enrico Olivelli , Norbert Kalmar Closes #1254 from symat/ZOOKEEPER-2164 --- .../server/quorum/QuorumCnxManager.java | 67 +++++++-- .../server/quorum/CnxManagerTest.java | 11 ++ .../zookeeper/test/QuorumRestartTest.java | 133 ++++++++++++++++++ .../org/apache/zookeeper/test/QuorumUtil.java | 19 +++ 4 files changed, 216 insertions(+), 14 deletions(-) create mode 100644 zookeeper-server/src/test/java/org/apache/zookeeper/test/QuorumRestartTest.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 45ffeb84903..66f6883470a 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 @@ -32,6 +32,7 @@ import java.net.Socket; import java.net.SocketException; import java.net.SocketTimeoutException; +import java.net.UnknownHostException; import java.nio.BufferUnderflowException; import java.nio.ByteBuffer; import java.nio.channels.UnresolvedAddressException; @@ -266,12 +267,33 @@ public static InitialMessage parse(Long protocolVersion, DataInputStream din) th } catch (ArrayIndexOutOfBoundsException e) { throw new InitialMessageException("No port number in: %s", addr); } - addresses.add(new InetSocketAddress(host_port[0], port)); + if (!isWildcardAddress(host_port[0])) { + addresses.add(new InetSocketAddress(host_port[0], port)); + } } return new InitialMessage(sid, addresses); } + /** + * Returns true if the specified hostname is a wildcard address, + * like 0.0.0.0 for IPv4 or :: for IPv6 + * + * (the function is package-private to be visible for testing) + */ + static boolean isWildcardAddress(final String hostname) { + try { + return InetAddress.getByName(hostname).isAnyLocalAddress(); + } catch (UnknownHostException e) { + // if we can not resolve, it can not be a wildcard address + return false; + } + } + + @Override + public String toString() { + return "InitialMessage{sid=" + sid + ", electionAddr=" + electionAddr + '}'; + } } public QuorumCnxManager(QuorumPeer self, final long mySid, Map view, @@ -415,6 +437,7 @@ public void run() { private boolean startConnection(Socket sock, Long sid) throws IOException { DataOutputStream dout = null; DataInputStream din = null; + LOG.debug("startConnection (myId:{} --> sid:{})", self.getId(), sid); try { // Use BufferedOutputStream to reduce the number of IP packets. This is // important for x-DC scenarios. @@ -459,13 +482,11 @@ private boolean startConnection(Socket sock, Long sid) throws IOException { // If lost the challenge, then drop the new connection if (sid > self.getId()) { - LOG.info( - "Have smaller server identifier, so dropping the connection: ({}, {})", - sid, - self.getId()); + LOG.info("Have smaller server identifier, so dropping the connection: (myId:{} --> sid:{})", self.getId(), sid); closeSocket(sock); // Otherwise proceed with the connection } else { + LOG.debug("Have larger server identifier, so keeping the connection: (myId:{} --> sid:{})", self.getId(), sid); SendWorker sw = new SendWorker(sock, sid); RecvWorker rw = new RecvWorker(sock, din, sid, sw); sw.setRecv(rw); @@ -501,9 +522,11 @@ public void receiveConnection(final Socket sock) { try { din = new DataInputStream(new BufferedInputStream(sock.getInputStream())); + LOG.debug("Sync handling of connection request received from: {}", sock.getRemoteSocketAddress()); handleConnection(sock, din); } catch (IOException e) { LOG.error("Exception handling connection, addr: {}, closing server connection", sock.getRemoteSocketAddress()); + LOG.debug("Exception details: ", e); closeSocket(sock); } } @@ -514,10 +537,12 @@ public void receiveConnection(final Socket sock) { */ public void receiveConnectionAsync(final Socket sock) { try { + LOG.debug("Async handling of connection request received from: {}", sock.getRemoteSocketAddress()); connectionExecutor.execute(new QuorumConnectionReceiverThread(sock)); connectionThreadCnt.incrementAndGet(); } catch (Throwable e) { LOG.error("Exception handling connection, addr: {}, closing server connection", sock.getRemoteSocketAddress()); + LOG.debug("Exception details: ", e); closeSocket(sock); } } @@ -552,10 +577,13 @@ private void handleConnection(Socket sock, DataInputStream din) throws IOExcepti try { InitialMessage init = InitialMessage.parse(protocolVersion, din); sid = init.sid; - electionAddr = new MultipleAddresses(init.electionAddr, - Duration.ofMillis(self.getMultiAddressReachabilityCheckTimeoutMs())); + if (!init.electionAddr.isEmpty()) { + electionAddr = new MultipleAddresses(init.electionAddr, + Duration.ofMillis(self.getMultiAddressReachabilityCheckTimeoutMs())); + } + LOG.debug("Initial message parsed by {}: {}", self.getId(), init.toString()); } catch (InitialMessage.InitialMessageException ex) { - LOG.error(ex.toString()); + LOG.error("Initial message parsing error!", ex); closeSocket(sock); return; } @@ -601,6 +629,10 @@ private void handleConnection(Socket sock, DataInputStream din) throws IOExcepti connectOne(sid); } + } else if (sid == self.getId()) { + // we saw this case in ZOOKEEPER-2164 + LOG.warn("We got a connection request from a server with our own ID. " + + "This should be either a configuration error, or a bug."); } else { // Otherwise start worker threads to receive data. SendWorker sw = new SendWorker(sock, sid); RecvWorker rw = new RecvWorker(sock, din, sid, sw); @@ -745,6 +777,7 @@ synchronized void connectOne(long sid) { Map lastProposedView = lastSeenQV.getAllMembers(); if (lastCommittedView.containsKey(sid)) { knownId = true; + LOG.debug("Server {} knows {} already, it is in the lastCommittedView", self.getId(), sid); if (connectOne(sid, lastCommittedView.get(sid).electionAddr)) { return; } @@ -754,6 +787,8 @@ synchronized void connectOne(long sid) { && (!knownId || (lastProposedView.get(sid).electionAddr != lastCommittedView.get(sid).electionAddr))) { knownId = true; + LOG.debug("Server {} knows {} already, it is in the lastProposedView", self.getId(), sid); + if (connectOne(sid, lastProposedView.get(sid).electionAddr)) { return; } @@ -821,7 +856,7 @@ public void halt() { */ public void softHalt() { for (SendWorker sw : senderWorkerMap.values()) { - LOG.debug("Halting sender: {}", sw); + LOG.debug("Server {} is soft-halting sender towards: {}", self.getId(), sw); sw.finish(); } } @@ -925,6 +960,7 @@ void setSocketBindErrorHandler(Runnable errorHandler) { @Override public void run() { if (!shutdown) { + LOG.debug("Listener thread started, myId: {}", self.getId()); Set addresses; if (self.getQuorumListenOnAllIPs()) { @@ -977,7 +1013,7 @@ public void run() { * Halts this listener thread. */ void halt() { - LOG.debug("Trying to close listeners"); + LOG.debug("Halt called: Trying to close listeners"); if (listenerHandlers != null) { LOG.debug("Closing listener: {}", QuorumCnxManager.this.mySid); for (ListenerHandler handler : listenerHandlers) { @@ -1044,12 +1080,12 @@ private void acceptConnections() { while ((!shutdown) && (portBindMaxRetry == 0 || numRetries < portBindMaxRetry)) { try { serverSocket = createNewServerSocket(); - LOG.info("My election bind port: {}", address.toString()); + LOG.info("{} is accepting connections now, my election bind port: {}", QuorumCnxManager.this.mySid, address.toString()); while (!shutdown) { try { client = serverSocket.accept(); setSockOpts(client); - LOG.info("Received connection request {}", client.getRemoteSocketAddress()); + LOG.info("Received connection request from {}", client.getRemoteSocketAddress()); // Receive and handle the connection request // asynchronously if the quorum sasl authentication is // enabled. This is required because sasl server @@ -1173,7 +1209,7 @@ synchronized RecvWorker getRecvWorker() { } synchronized boolean finish() { - LOG.debug("Calling finish for {}", sid); + LOG.debug("Calling SendWorker.finish for {}", sid); if (!running) { /* @@ -1240,6 +1276,7 @@ public void run() { LOG.error("Failed to send last message. Shutting down thread.", e); this.finish(); } + LOG.debug("SendWorker thread started towards {}. myId: {}", sid, QuorumCnxManager.this.mySid); try { while (running && !shutdown && sock != null) { @@ -1337,6 +1374,7 @@ class RecvWorker extends ZooKeeperThread { * @return boolean Value of variable running */ synchronized boolean finish() { + LOG.debug("RecvWorker.finish called. sid: {}. myId: {}", sid, QuorumCnxManager.this.mySid); if (!running) { /* * Avoids running finish() twice. @@ -1354,6 +1392,7 @@ synchronized boolean finish() { public void run() { threadCnt.incrementAndGet(); try { + LOG.debug("RecvWorker thread towards {} started. myId: {}", sid, QuorumCnxManager.this.mySid); while (running && !shutdown && sock != null) { /** * Reads the first int to determine the length of the @@ -1377,7 +1416,7 @@ public void run() { QuorumCnxManager.this.mySid, e); } finally { - LOG.warn("Interrupting SendWorker"); + LOG.warn("Interrupting SendWorker thread from RecvWorker. sid: {}. myId: {}", sid, QuorumCnxManager.this.mySid); sw.finish(); closeSocket(sock); } diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/CnxManagerTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/CnxManagerTest.java index 269b6a88102..a99e33678f0 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/CnxManagerTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/CnxManagerTest.java @@ -667,6 +667,17 @@ public void testInitialMessage() throws Exception { } } + @Test + public void testWildcardAddressRecognition() { + assertTrue(QuorumCnxManager.InitialMessage.isWildcardAddress("0.0.0.0")); + assertTrue(QuorumCnxManager.InitialMessage.isWildcardAddress("::")); + assertFalse(QuorumCnxManager.InitialMessage.isWildcardAddress("some.unresolvable.host.com")); + assertFalse(QuorumCnxManager.InitialMessage.isWildcardAddress("127.0.0.1")); + assertFalse(QuorumCnxManager.InitialMessage.isWildcardAddress("255.255.255.255")); + assertFalse(QuorumCnxManager.InitialMessage.isWildcardAddress("1.2.3.4")); + assertFalse(QuorumCnxManager.InitialMessage.isWildcardAddress("www.google.com")); + } + private String createLongString(int size) { StringBuilder sb = new StringBuilder(size); for (int i = 0; i < size; i++) { diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/test/QuorumRestartTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/test/QuorumRestartTest.java new file mode 100644 index 00000000000..afdafb790c5 --- /dev/null +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/test/QuorumRestartTest.java @@ -0,0 +1,133 @@ +/* + * 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.client.ZKClientConfig.ZOOKEEPER_CLIENT_CNXN_SOCKET; +import static org.junit.Assert.assertTrue; +import org.apache.zookeeper.ZKTestCase; +import org.apache.zookeeper.server.ServerCnxnFactory; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class QuorumRestartTest extends ZKTestCase { + + private static final Logger LOG = LoggerFactory.getLogger(QuorumRestartTest.class); + private QuorumUtil qu; + + @Before + public void setUp() throws Exception { + System.setProperty(ZOOKEEPER_CLIENT_CNXN_SOCKET, "org.apache.zookeeper.ClientCnxnSocketNetty"); + System.setProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN_FACTORY, "org.apache.zookeeper.server.NettyServerCnxnFactory"); + + // starting a 3 node ensemble without observers + qu = new QuorumUtil(1, 2); + qu.startAll(); + } + + + /** + * A basic test for rolling restart. We are restarting the ZooKeeper servers one by one, + * starting from the first server. We always make sure that all the nodes joined to the + * Quorum before moving forward. + * @throws Exception + */ + @Test + public void testRollingRestart() throws Exception { + for (int serverToRestart = 1; serverToRestart <= 3; serverToRestart++) { + LOG.info("***** restarting: " + serverToRestart); + qu.shutdown(serverToRestart); + + assertTrue(String.format("Timeout during waiting for server %d to go down", serverToRestart), + ClientBase.waitForServerDown("127.0.0.1:" + qu.getPeer(serverToRestart).clientPort, ClientBase.CONNECTION_TIMEOUT)); + + qu.restart(serverToRestart); + + final String errorMessage = "Not all the quorum members are connected after restarting server " + serverToRestart; + waitFor(errorMessage, () -> qu.allPeersAreConnected(), 30); + + LOG.info("***** Restart {} succeeded", serverToRestart); + } + } + + /** + * Testing one of the errors reported in ZOOKEEPER-2164, when some servers can not + * rejoin to the Quorum after restarting the servers backwards + * @throws Exception + */ + @Test + public void testRollingRestartBackwards() throws Exception { + for (int serverToRestart = 3; serverToRestart >= 1; serverToRestart--) { + LOG.info("***** restarting: " + serverToRestart); + qu.shutdown(serverToRestart); + + assertTrue(String.format("Timeout during waiting for server %d to go down", serverToRestart), + ClientBase.waitForServerDown("127.0.0.1:" + qu.getPeer(serverToRestart).clientPort, ClientBase.CONNECTION_TIMEOUT)); + + qu.restart(serverToRestart); + + final String errorMessage = "Not all the quorum members are connected after restarting server " + serverToRestart; + waitFor(errorMessage, () -> qu.allPeersAreConnected(), 30); + + LOG.info("***** Restart {} succeeded", serverToRestart); + } + } + + + /** + * Testing one of the errors reported in ZOOKEEPER-2164, when some servers can not + * rejoin to the Quorum after restarting the current leader multiple times + * @throws Exception + */ + @Test + public void testRestartingLeaderMultipleTimes() throws Exception { + for (int restartCount = 1; restartCount <= 3; restartCount++) { + int leaderId = qu.getLeaderServer(); + LOG.info("***** new leader: " + leaderId); + 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, 30); + + qu.restart(leaderId); + + errorMessage = "Not all the quorum members are connected after restarting the old leader"; + waitFor(errorMessage, () -> qu.allPeersAreConnected(), 30); + + LOG.info("***** Leader Restart {} succeeded", restartCount); + } + } + + + @After + public void tearDown() throws Exception { + qu.shutdownAll(); + System.clearProperty(ZOOKEEPER_CLIENT_CNXN_SOCKET); + System.clearProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN_FACTORY); + } + + + + +} diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/test/QuorumUtil.java b/zookeeper-server/src/test/java/org/apache/zookeeper/test/QuorumUtil.java index 14e3beebe76..f7935851900 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/test/QuorumUtil.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/test/QuorumUtil.java @@ -25,12 +25,14 @@ import java.io.IOException; import java.net.InetSocketAddress; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collections; import java.util.HashMap; import java.util.LinkedHashSet; import java.util.List; import java.util.Map; import java.util.Set; +import java.util.TreeSet; import org.apache.zookeeper.PortAssignment; import org.apache.zookeeper.server.quorum.Election; import org.apache.zookeeper.server.quorum.QuorumPeer; @@ -51,6 +53,8 @@ public class QuorumUtil { // TODO refactor QuorumBase to be special case of this private static final Logger LOG = LoggerFactory.getLogger(QuorumUtil.class); + private static final Set CONNECTED_STATES = new TreeSet<>( + Arrays.asList(QuorumPeer.ServerState.LEADING, QuorumPeer.ServerState.FOLLOWING, QuorumPeer.ServerState.OBSERVING)); public static class PeerStruct { @@ -274,6 +278,12 @@ public String getConnectString(QuorumPeer peer) { return "127.0.0.1:" + peer.getClientPort(); } + public boolean allPeersAreConnected() { + return peers.values().stream() + .map(ps -> ps.peer) + .allMatch(peer -> CONNECTED_STATES.contains(peer.getPeerState())); + } + public QuorumPeer getLeaderQuorumPeer() { for (PeerStruct ps : peers.values()) { if (ps.peer.leader != null) { @@ -320,6 +330,15 @@ public int getLeaderServer() { return index; } + public boolean leaderExists() { + for (int i = 1; i <= ALL; i++) { + if (getPeer(i).peer.leader != null) { + return true; + } + } + return false; + } + public String getConnectionStringForServer(final int index) { return "127.0.0.1:" + getPeer(index).clientPort; } From f7eca7041b76b8f6abe9f3ec0aa649ee0772d319 Mon Sep 17 00:00:00 2001 From: Patrick Hunt Date: Sat, 14 Mar 2020 17:45:57 +0100 Subject: [PATCH 047/118] ZOOKEEPER-3751: upgrade jackson-databind to 2.10 from 2.9 Change-Id: I3c548bcb8e67f83cf6d9fb553d54a6cf9bacf5f3 Author: Patrick Hunt Reviewers: Enrico Olivelli , Christopher Tubbs, Jan Hentschel Closes #1283 from phunt/zk3751 (cherry picked from commit 222685123018f9d0d35bfd35cc4cc5924f4348bc) Signed-off-by: Enrico Olivelli --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index bcb7e02e244..33d5b5bce20 100755 --- a/pom.xml +++ b/pom.xml @@ -328,7 +328,7 @@ 1.2 4.1.45.Final 9.4.24.v20191120 - 2.9.10.3 + 2.10.3 1.1.1 2.11 1.1.7 From cb8456c1229bb199c29e90a9d312699c3706fc3d Mon Sep 17 00:00:00 2001 From: tison Date: Sat, 14 Mar 2020 17:50:15 +0100 Subject: [PATCH 048/118] ZOOKEEPER-3745: Update copyright notices from 2019 to 2020 Author: tison Reviewers: Enrico Olivelli Closes #1278 from TisonKun/ZOOKEEPER-3745 (cherry picked from commit a5a4743733b8939464af82c1ee68a593fadbe362) Signed-off-by: Enrico Olivelli --- NOTICE.txt | 2 +- zookeeper-docs/src/main/resources/markdown/zookeeperCLI.md | 2 +- .../src/main/resources/markdown/zookeeperMonitor.md | 4 ++-- zookeeper-docs/src/main/resources/markdown/zookeeperTools.md | 2 +- .../src/main/resources/markdown/zookeeperUseCases.md | 4 ++-- zookeeper-server/src/main/resources/NOTICE.txt | 2 +- 6 files changed, 8 insertions(+), 8 deletions(-) diff --git a/NOTICE.txt b/NOTICE.txt index ff3661df0a0..4c4f8b220fe 100644 --- a/NOTICE.txt +++ b/NOTICE.txt @@ -1,5 +1,5 @@ Apache ZooKeeper -Copyright 2009-2019 The Apache Software Foundation +Copyright 2009-2020 The Apache Software Foundation This product includes software developed at The Apache Software Foundation (http://www.apache.org/). diff --git a/zookeeper-docs/src/main/resources/markdown/zookeeperCLI.md b/zookeeper-docs/src/main/resources/markdown/zookeeperCLI.md index 3ca676e504d..637e6b4596a 100644 --- a/zookeeper-docs/src/main/resources/markdown/zookeeperCLI.md +++ b/zookeeper-docs/src/main/resources/markdown/zookeeperCLI.md @@ -1,5 +1,5 @@ - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - 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 1a28de74a496bcca0407cfc9ba3c73014cbe9e11 Mon Sep 17 00:00:00 2001 From: tison Date: Tue, 31 Mar 2020 13:39:49 +0200 Subject: [PATCH 056/118] 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 (cherry picked from commit a4c97d255ad7bb86c61a35d01415de1caebd3dac) Signed-off-by: Norbert Kalmar --- 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 68466c8767fa1b3ab0e355278921752392e2ea8b Mon Sep 17 00:00:00 2001 From: Mate Szalay-Beko Date: Tue, 7 Apr 2020 09:07:50 +0200 Subject: [PATCH 057/118] 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 (cherry picked from commit a548253408384d9e943b151c942d795017dca70c) Signed-off-by: Enrico Olivelli --- .../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 5062c39cb028115a9b79f9b28d6837065e70bf40 Mon Sep 17 00:00:00 2001 From: Norbert Kalmar Date: Tue, 7 Apr 2020 15:40:39 +0200 Subject: [PATCH 058/118] 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 --- .../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 17374c0d4e0af2b59d2b4a56d1bc2c4041e6315d Mon Sep 17 00:00:00 2001 From: Enrico Olivelli Date: Tue, 7 Apr 2020 17:00:35 +0200 Subject: [PATCH 059/118] ZOOKEEPER-3785: Make sources buildable with JDK14 Author: Enrico Olivelli Reviewers: Mate Szalay-Beko , TisunKun Closes #1308 from eolivelli/fix/jdk14 (cherry picked from commit a8fb880218410ec3b31374d957e5a0bd6c8cdf28) Signed-off-by: Enrico Olivelli --- .../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 1d25e0f6d2688d13f5853a05c87efbc522c82f35 Mon Sep 17 00:00:00 2001 From: Enrico Olivelli Date: Fri, 10 Apr 2020 16:34:28 +0200 Subject: [PATCH 060/118] ZooKeeper 3.6.1 release notes --- .../main/resources/markdown/releasenotes.md | 770 +----------------- 1 file changed, 29 insertions(+), 741 deletions(-) diff --git a/zookeeper-docs/src/main/resources/markdown/releasenotes.md b/zookeeper-docs/src/main/resources/markdown/releasenotes.md index 4cf8d1628ad..66750347807 100644 --- a/zookeeper-docs/src/main/resources/markdown/releasenotes.md +++ b/zookeeper-docs/src/main/resources/markdown/releasenotes.md @@ -15,759 +15,47 @@ limitations under the License. //--> +# Release Notes - ZooKeeper - Version 3.6.1 -# Release Notes - ZooKeeper - Version 3.6.0 +## Bug + +* [ZOOKEEPER-2164](https://issues.apache.org/jira/browse/ZOOKEEPER-2164) - fast leader election keeps failing +* [ZOOKEEPER-3706](https://issues.apache.org/jira/browse/ZOOKEEPER-3706) - ZooKeeper.close() would leak SendThread when the network is broken +* [ZOOKEEPER-3737](https://issues.apache.org/jira/browse/ZOOKEEPER-3737) - Unable to eliminate log4j1 transitive dependency +* [ZOOKEEPER-3738](https://issues.apache.org/jira/browse/ZOOKEEPER-3738) - Avoid use of broken codehaus properties-maven-plugin +* [ZOOKEEPER-3739](https://issues.apache.org/jira/browse/ZOOKEEPER-3739) - Remove use of com.sun.nio.file.SensitivityWatchEventModifier +* [ZOOKEEPER-3745](https://issues.apache.org/jira/browse/ZOOKEEPER-3745) - Update copyright notices from 2019 to 2020 +* [ZOOKEEPER-3758](https://issues.apache.org/jira/browse/ZOOKEEPER-3758) - Update from 3.5.7 to 3.6.0 does not work +* [ZOOKEEPER-3760](https://issues.apache.org/jira/browse/ZOOKEEPER-3760) - remove a useless throwing CliException +* [ZOOKEEPER-3769](https://issues.apache.org/jira/browse/ZOOKEEPER-3769) - fast leader election does not end if leader is taken down +* [ZOOKEEPER-3776](https://issues.apache.org/jira/browse/ZOOKEEPER-3776) - Cluster stuck not forming up quorum +* [ZOOKEEPER-3778](https://issues.apache.org/jira/browse/ZOOKEEPER-3778) - Cannot upgrade from 3.5.7 to 3.6.0 due to multiAddress.reachabilityCheckEnabled +* [ZOOKEEPER-3780](https://issues.apache.org/jira/browse/ZOOKEEPER-3780) - restore Version.getRevision() to be backward compatible ## New Feature -* [ZOOKEEPER-27](https://issues.apache.org/jira/browse/ZOOKEEPER-27) - Unique DB identifiers for servers and clients -* [ZOOKEEPER-1260](https://issues.apache.org/jira/browse/ZOOKEEPER-1260) - Audit logging in ZooKeeper servers. -* [ZOOKEEPER-1634](https://issues.apache.org/jira/browse/ZOOKEEPER-1634) - A new feature proposal to ZooKeeper: authentication enforcement -* [ZOOKEEPER-1703](https://issues.apache.org/jira/browse/ZOOKEEPER-1703) - Please add instructions for running the tutorial -* [ZOOKEEPER-1962](https://issues.apache.org/jira/browse/ZOOKEEPER-1962) - Add a CLI command to recursively list a znode and children -* [ZOOKEEPER-2875](https://issues.apache.org/jira/browse/ZOOKEEPER-2875) - Add ant task for running OWASP dependency report -* [ZOOKEEPER-2933](https://issues.apache.org/jira/browse/ZOOKEEPER-2933) - Ability to monitor the jute.maxBuffer usage in real-time -* [ZOOKEEPER-2994](https://issues.apache.org/jira/browse/ZOOKEEPER-2994) - Tool required to recover log and snapshot entries with CRC errors -* [ZOOKEEPER-3066](https://issues.apache.org/jira/browse/ZOOKEEPER-3066) - Expose on JMX of Followers the id of the current leader -* [ZOOKEEPER-3091](https://issues.apache.org/jira/browse/ZOOKEEPER-3091) - Prometheus.io integration -* [ZOOKEEPER-3092](https://issues.apache.org/jira/browse/ZOOKEEPER-3092) - Pluggable metrics system for ZooKeeper -* [ZOOKEEPER-3114](https://issues.apache.org/jira/browse/ZOOKEEPER-3114) - Built-in data consistency check inside ZooKeeper -* [ZOOKEEPER-3137](https://issues.apache.org/jira/browse/ZOOKEEPER-3137) - add a utility to truncate logs to a zxid -* [ZOOKEEPER-3140](https://issues.apache.org/jira/browse/ZOOKEEPER-3140) - Allow Followers to host Observers -* [ZOOKEEPER-3160](https://issues.apache.org/jira/browse/ZOOKEEPER-3160) - Custom User SSLContext -* [ZOOKEEPER-3167](https://issues.apache.org/jira/browse/ZOOKEEPER-3167) - add an API and the corresponding CLI to get total count of recursive sub nodes under a specific path -* [ZOOKEEPER-3209](https://issues.apache.org/jira/browse/ZOOKEEPER-3209) - New `getEphemerals` api to get all the ephemeral nodes created by the session -* [ZOOKEEPER-3244](https://issues.apache.org/jira/browse/ZOOKEEPER-3244) - Add option to snapshot based on log size -* [ZOOKEEPER-3269](https://issues.apache.org/jira/browse/ZOOKEEPER-3269) - Testable facade would benefit from a queueEvent() method -* [ZOOKEEPER-3311](https://issues.apache.org/jira/browse/ZOOKEEPER-3311) - Allow a delay to the transaction log flush -* [ZOOKEEPER-3331](https://issues.apache.org/jira/browse/ZOOKEEPER-3331) - Automatically add IP authorization for Netty connections -* [ZOOKEEPER-3343](https://issues.apache.org/jira/browse/ZOOKEEPER-3343) - Add a new doc: zookeeperTools.md -* [ZOOKEEPER-3344](https://issues.apache.org/jira/browse/ZOOKEEPER-3344) - write a new script:zkSnapShotToolkit.sh to encapsulate SnapshotFormatter and doc the usage -* [ZOOKEEPER-3371](https://issues.apache.org/jira/browse/ZOOKEEPER-3371) - Port unification for admin server -* [ZOOKEEPER-3447](https://issues.apache.org/jira/browse/ZOOKEEPER-3447) - add a doc: zookeeperMonitor.md - -## Improvement +* [ZOOKEEPER-3689](https://issues.apache.org/jira/browse/ZOOKEEPER-3689) - zkCli/ZooKeeperMain relies on system properties for TLS config +* [ZOOKEEPER-3712](https://issues.apache.org/jira/browse/ZOOKEEPER-3712) - Add setKeepAlive support for NIOServerCnxn -* [ZOOKEEPER-3703](https://issues.apache.org/jira/browse/ZOOKEEPER-3703) - publish a test JAR -* [ZOOKEEPER-3482](https://issues.apache.org/jira/browse/ZOOKEEPER-3482) - SASL (Kerberos) Authentication with SSL for clients and Quorum -* [ZOOKEEPER-3567](https://issues.apache.org/jira/browse/ZOOKEEPER-3567) - add SSL support for zkpython -* [ZOOKEEPER-261](https://issues.apache.org/jira/browse/ZOOKEEPER-261) - Reinitialized servers should not participate in leader election -* [ZOOKEEPER-761](https://issues.apache.org/jira/browse/ZOOKEEPER-761) - Remove *synchronous* calls from the *single-threaded* C clieant API, since they are documented not to work -* [ZOOKEEPER-974](https://issues.apache.org/jira/browse/ZOOKEEPER-974) - Configurable listen socket backlog for the client port -* [ZOOKEEPER-1177](https://issues.apache.org/jira/browse/ZOOKEEPER-1177) - Enabling a large number of watches for a large number of clients -* [ZOOKEEPER-1416](https://issues.apache.org/jira/browse/ZOOKEEPER-1416) - Persistent Recursive Watch -* [ZOOKEEPER-1423](https://issues.apache.org/jira/browse/ZOOKEEPER-1423) - 4lw and jmx should expose the size of the datadir/datalogdir -* [ZOOKEEPER-1425](https://issues.apache.org/jira/browse/ZOOKEEPER-1425) - add version command to the zookeeper client shell -* [ZOOKEEPER-1426](https://issues.apache.org/jira/browse/ZOOKEEPER-1426) - add version command to the zookeeper server -* [ZOOKEEPER-1467](https://issues.apache.org/jira/browse/ZOOKEEPER-1467) - Make server principal configurable at client side. -* [ZOOKEEPER-1504](https://issues.apache.org/jira/browse/ZOOKEEPER-1504) - Multi-thread NIOServerCnxn -* [ZOOKEEPER-1506](https://issues.apache.org/jira/browse/ZOOKEEPER-1506) - Re-try DNS hostname -> IP resolution if node connection fails -* [ZOOKEEPER-1525](https://issues.apache.org/jira/browse/ZOOKEEPER-1525) - Plumb ZooKeeperServer object into auth plugins -* [ZOOKEEPER-1651](https://issues.apache.org/jira/browse/ZOOKEEPER-1651) - Add support for compressed snapshot -* [ZOOKEEPER-1748](https://issues.apache.org/jira/browse/ZOOKEEPER-1748) - TCP keepalive for leader election connections -* [ZOOKEEPER-1907](https://issues.apache.org/jira/browse/ZOOKEEPER-1907) - Improve Thread handling -* [ZOOKEEPER-1908](https://issues.apache.org/jira/browse/ZOOKEEPER-1908) - setAcl should be have a recursive function -* [ZOOKEEPER-1948](https://issues.apache.org/jira/browse/ZOOKEEPER-1948) - Enable JMX remote monitoring -* [ZOOKEEPER-1963](https://issues.apache.org/jira/browse/ZOOKEEPER-1963) - Make JDK 7 the minimum requirement for Zookeeper -* [ZOOKEEPER-1994](https://issues.apache.org/jira/browse/ZOOKEEPER-1994) - Backup config files. -* [ZOOKEEPER-2024](https://issues.apache.org/jira/browse/ZOOKEEPER-2024) - Major throughput improvement with mixed workloads -* [ZOOKEEPER-2040](https://issues.apache.org/jira/browse/ZOOKEEPER-2040) - Server to log underlying cause of SASL connection problems -* [ZOOKEEPER-2079](https://issues.apache.org/jira/browse/ZOOKEEPER-2079) - Stop daemon with "kill" rather than "kill -9" -* [ZOOKEEPER-2083](https://issues.apache.org/jira/browse/ZOOKEEPER-2083) - Remove deprecated LE implementations -* [ZOOKEEPER-2084](https://issues.apache.org/jira/browse/ZOOKEEPER-2084) - Document local session parameters -* [ZOOKEEPER-2087](https://issues.apache.org/jira/browse/ZOOKEEPER-2087) - Few UX improvements in ZooInspector -* [ZOOKEEPER-2098](https://issues.apache.org/jira/browse/ZOOKEEPER-2098) - QuorumCnxManager: use BufferedOutputStream for initial msg -* [ZOOKEEPER-2107](https://issues.apache.org/jira/browse/ZOOKEEPER-2107) - zookeeper client should support custom HostProviders -* [ZOOKEEPER-2110](https://issues.apache.org/jira/browse/ZOOKEEPER-2110) - Typo fixes in the ZK documentation -* [ZOOKEEPER-2126](https://issues.apache.org/jira/browse/ZOOKEEPER-2126) - Improve exit log messsage of EventThread and SendThread by adding SessionId -* [ZOOKEEPER-2139](https://issues.apache.org/jira/browse/ZOOKEEPER-2139) - Support multiple ZooKeeper client, with different configurations, in a single JVM -* [ZOOKEEPER-2140](https://issues.apache.org/jira/browse/ZOOKEEPER-2140) - NettyServerCnxn and NIOServerCnxn code should be improved -* [ZOOKEEPER-2149](https://issues.apache.org/jira/browse/ZOOKEEPER-2149) - Logging of client address when socket connection established -* [ZOOKEEPER-2176](https://issues.apache.org/jira/browse/ZOOKEEPER-2176) - Unclear error message should be info not error -* [ZOOKEEPER-2179](https://issues.apache.org/jira/browse/ZOOKEEPER-2179) - Typo in Watcher.java -* [ZOOKEEPER-2183](https://issues.apache.org/jira/browse/ZOOKEEPER-2183) - Concurrent Testing Processes and Port Assignments -* [ZOOKEEPER-2185](https://issues.apache.org/jira/browse/ZOOKEEPER-2185) - Run server with -XX:+HeapDumpOnOutOfMemoryError and -XX:OnOutOfMemoryError='kill %p'. -* [ZOOKEEPER-2191](https://issues.apache.org/jira/browse/ZOOKEEPER-2191) - Continue supporting prior Ant versions that don't implement the threads attribute for the JUnit task. -* [ZOOKEEPER-2194](https://issues.apache.org/jira/browse/ZOOKEEPER-2194) - Let DataNode.getChildren() return an unmodifiable view of its children set -* [ZOOKEEPER-2205](https://issues.apache.org/jira/browse/ZOOKEEPER-2205) - Log type of unexpected quorum packet in learner handler loop -* [ZOOKEEPER-2206](https://issues.apache.org/jira/browse/ZOOKEEPER-2206) - Add missing packet types to LearnerHandler.packetToString() -* [ZOOKEEPER-2207](https://issues.apache.org/jira/browse/ZOOKEEPER-2207) - Enhance error logs with LearnerHandler.packetToString() -* [ZOOKEEPER-2208](https://issues.apache.org/jira/browse/ZOOKEEPER-2208) - Log type of unexpected quorum packet in observer loop -* [ZOOKEEPER-2214](https://issues.apache.org/jira/browse/ZOOKEEPER-2214) - Findbugs warning: LearnerHandler.packetToString Dead store to local variable -* [ZOOKEEPER-2223](https://issues.apache.org/jira/browse/ZOOKEEPER-2223) - support method-level JUnit testcase -* [ZOOKEEPER-2238](https://issues.apache.org/jira/browse/ZOOKEEPER-2238) - Support limiting the maximum number of connections/clients to a zookeeper server. -* [ZOOKEEPER-2240](https://issues.apache.org/jira/browse/ZOOKEEPER-2240) - Make the three-node minimum more explicit in documentation and on website -* [ZOOKEEPER-2270](https://issues.apache.org/jira/browse/ZOOKEEPER-2270) - Allow MBeanRegistry to be overridden for better unit tests -* [ZOOKEEPER-2306](https://issues.apache.org/jira/browse/ZOOKEEPER-2306) - Remove file delete duplicate code from test code -* [ZOOKEEPER-2315](https://issues.apache.org/jira/browse/ZOOKEEPER-2315) - Change client connect zk service timeout log level from Info to Warn level -* [ZOOKEEPER-2326](https://issues.apache.org/jira/browse/ZOOKEEPER-2326) - Include connected server address:port in log -* [ZOOKEEPER-2359](https://issues.apache.org/jira/browse/ZOOKEEPER-2359) - ZooKeeper client has unnecessary logs for watcher removal errors -* [ZOOKEEPER-2368](https://issues.apache.org/jira/browse/ZOOKEEPER-2368) - Client watches are not disconnected on close -* [ZOOKEEPER-2373](https://issues.apache.org/jira/browse/ZOOKEEPER-2373) - Licenses section missing from pom file -* [ZOOKEEPER-2378](https://issues.apache.org/jira/browse/ZOOKEEPER-2378) - upgrade ivy to recent version -* [ZOOKEEPER-2392](https://issues.apache.org/jira/browse/ZOOKEEPER-2392) - Update netty to 3.7.1.Final -* [ZOOKEEPER-2402](https://issues.apache.org/jira/browse/ZOOKEEPER-2402) - Document client side properties -* [ZOOKEEPER-2410](https://issues.apache.org/jira/browse/ZOOKEEPER-2410) - add time unit to 'ELECTION TOOK' log.info message -* [ZOOKEEPER-2433](https://issues.apache.org/jira/browse/ZOOKEEPER-2433) - ZooKeeperSaslServer: allow user principals in subject -* [ZOOKEEPER-2479](https://issues.apache.org/jira/browse/ZOOKEEPER-2479) - Add 'electionTimeTaken' value in LeaderMXBean and FollowerMXBean -* [ZOOKEEPER-2489](https://issues.apache.org/jira/browse/ZOOKEEPER-2489) - Upgrade Jetty dependency to a recent stable release version. -* [ZOOKEEPER-2505](https://issues.apache.org/jira/browse/ZOOKEEPER-2505) - Use shared library instead of static library in C client unit test -* [ZOOKEEPER-2507](https://issues.apache.org/jira/browse/ZOOKEEPER-2507) - C unit test improvement: line break between 'ZooKeeper server started' and 'Running' -* [ZOOKEEPER-2511](https://issues.apache.org/jira/browse/ZOOKEEPER-2511) - Implement AutoCloseable in ZooKeeper.java -* [ZOOKEEPER-2557](https://issues.apache.org/jira/browse/ZOOKEEPER-2557) - Update gitignore to account for other file extensions -* [ZOOKEEPER-2594](https://issues.apache.org/jira/browse/ZOOKEEPER-2594) - Use TLS for downloading artifacts during build -* [ZOOKEEPER-2620](https://issues.apache.org/jira/browse/ZOOKEEPER-2620) - Add comments to testReadOnlySnapshotDir and testReadOnlyTxnLogDir indicating that the tests will fail when run as root -* [ZOOKEEPER-2630](https://issues.apache.org/jira/browse/ZOOKEEPER-2630) - Use interface type instead of implementation type when appropriate. -* [ZOOKEEPER-2632](https://issues.apache.org/jira/browse/ZOOKEEPER-2632) - Add option to inform JIRA_PASSWORD at CLI prompt -* [ZOOKEEPER-2638](https://issues.apache.org/jira/browse/ZOOKEEPER-2638) - ZooKeeper should log which serverCnxnFactory is used during startup -* [ZOOKEEPER-2641](https://issues.apache.org/jira/browse/ZOOKEEPER-2641) - AvgRequestLatency metric improves to be more accurate -* [ZOOKEEPER-2655](https://issues.apache.org/jira/browse/ZOOKEEPER-2655) - Improve NIOServerCnxn#isZKServerRunning to reflect the semantics correctly -* [ZOOKEEPER-2662](https://issues.apache.org/jira/browse/ZOOKEEPER-2662) - Export a metric for txn log sync times -* [ZOOKEEPER-2672](https://issues.apache.org/jira/browse/ZOOKEEPER-2672) - Remove CHANGE.txt -* [ZOOKEEPER-2682](https://issues.apache.org/jira/browse/ZOOKEEPER-2682) - Make it optional to fail build on test failure -* [ZOOKEEPER-2697](https://issues.apache.org/jira/browse/ZOOKEEPER-2697) - Handle graceful stop of ZookKeeper client -* [ZOOKEEPER-2744](https://issues.apache.org/jira/browse/ZOOKEEPER-2744) - Typos in the comments of ZooKeeper class -* [ZOOKEEPER-2767](https://issues.apache.org/jira/browse/ZOOKEEPER-2767) - Correct the exception messages in X509Util if truststore location or password is not configured -* [ZOOKEEPER-2788](https://issues.apache.org/jira/browse/ZOOKEEPER-2788) - The define of MAX_CONNECTION_ATTEMPTS in QuorumCnxManager.java seems useless, should it be removed? -* [ZOOKEEPER-2815](https://issues.apache.org/jira/browse/ZOOKEEPER-2815) - 1. Using try clause to close resource; 2. Others code refactoring for PERSISTENCE module -* [ZOOKEEPER-2816](https://issues.apache.org/jira/browse/ZOOKEEPER-2816) - Code refactoring for `ZK_SERVER` module -* [ZOOKEEPER-2824](https://issues.apache.org/jira/browse/ZOOKEEPER-2824) - `FileChannel#size` info should be added to `FileTxnLog#commit` to solve the confuse that reason is too large log or too busy disk I/O -* [ZOOKEEPER-2825](https://issues.apache.org/jira/browse/ZOOKEEPER-2825) - 1. Remove unnecessary import; 2. `contains` instead of `indexOf > -1` for more readable; 3. Standardize `StringBuilder#append` usage for CLIENT module -* [ZOOKEEPER-2826](https://issues.apache.org/jira/browse/ZOOKEEPER-2826) - Code refactoring for `CLI` module -* [ZOOKEEPER-2829](https://issues.apache.org/jira/browse/ZOOKEEPER-2829) - Interface usability / compatibility improvements through Java annotation. -* [ZOOKEEPER-2856](https://issues.apache.org/jira/browse/ZOOKEEPER-2856) - ZooKeeperSaslClient#respondToServer should log exception message of SaslException -* [ZOOKEEPER-2864](https://issues.apache.org/jira/browse/ZOOKEEPER-2864) - Add script to run a java api compatibility tool -* [ZOOKEEPER-2865](https://issues.apache.org/jira/browse/ZOOKEEPER-2865) - Reconfig Causes Inconsistent Configuration file among the nodes -* [ZOOKEEPER-2870](https://issues.apache.org/jira/browse/ZOOKEEPER-2870) - Improve the efficiency of AtomicFileOutputStream -* [ZOOKEEPER-2880](https://issues.apache.org/jira/browse/ZOOKEEPER-2880) - Rename README.txt to README.md -* [ZOOKEEPER-2887](https://issues.apache.org/jira/browse/ZOOKEEPER-2887) - define dependency versions in build.xml to be easily overridden in build.properties -* [ZOOKEEPER-2892](https://issues.apache.org/jira/browse/ZOOKEEPER-2892) - Improve lazy initialize and close stream for `PrepRequestProcessor` -* [ZOOKEEPER-2896](https://issues.apache.org/jira/browse/ZOOKEEPER-2896) - Remove unused imports from org.apache.zookeeper.test.CreateTest.java -* [ZOOKEEPER-2904](https://issues.apache.org/jira/browse/ZOOKEEPER-2904) - Remove unused imports from org.apache.zookeeper.server.quorum.WatchLeakTest -* [ZOOKEEPER-2915](https://issues.apache.org/jira/browse/ZOOKEEPER-2915) - Use "strict" conflict management in ivy -* [ZOOKEEPER-2950](https://issues.apache.org/jira/browse/ZOOKEEPER-2950) - Add keys for the Zxid from the stat command to check_zookeeper.py -* [ZOOKEEPER-2952](https://issues.apache.org/jira/browse/ZOOKEEPER-2952) - Upgrade third party libraries to address vulnerabilities -* [ZOOKEEPER-2967](https://issues.apache.org/jira/browse/ZOOKEEPER-2967) - Add check to validate dataDir and dataLogDir parameters at startup -* [ZOOKEEPER-2999](https://issues.apache.org/jira/browse/ZOOKEEPER-2999) - CMake build should use target-level commands -* [ZOOKEEPER-3012](https://issues.apache.org/jira/browse/ZOOKEEPER-3012) - Fix unit test: testDataDirAndDataLogDir should not use hardcode test folders -* [ZOOKEEPER-3019](https://issues.apache.org/jira/browse/ZOOKEEPER-3019) - Add a metric to track number of slow fsyncs -* [ZOOKEEPER-3020](https://issues.apache.org/jira/browse/ZOOKEEPER-3020) - Review of SyncRequestProcessor -* [ZOOKEEPER-3037](https://issues.apache.org/jira/browse/ZOOKEEPER-3037) - Add JvmPauseMonitor to ZooKeeper -* [ZOOKEEPER-3043](https://issues.apache.org/jira/browse/ZOOKEEPER-3043) - QuorumKerberosHostBasedAuthTest fails on Linux box: Unable to parse:includedir /etc/krb5.conf.d/ -* [ZOOKEEPER-3044](https://issues.apache.org/jira/browse/ZOOKEEPER-3044) - OutOfMemoryError exceptions in Jenkins when running tests -* [ZOOKEEPER-3063](https://issues.apache.org/jira/browse/ZOOKEEPER-3063) - Track outstanding changes with ArrayDeque -* [ZOOKEEPER-3068](https://issues.apache.org/jira/browse/ZOOKEEPER-3068) - Improve C client logging of IPv6 hosts -* [ZOOKEEPER-3071](https://issues.apache.org/jira/browse/ZOOKEEPER-3071) - Add a config parameter to control transaction log size -* [ZOOKEEPER-3077](https://issues.apache.org/jira/browse/ZOOKEEPER-3077) - Build native C library outside of source directory -* [ZOOKEEPER-3078](https://issues.apache.org/jira/browse/ZOOKEEPER-3078) - Remove unused print_completion_queue function -* [ZOOKEEPER-3083](https://issues.apache.org/jira/browse/ZOOKEEPER-3083) - Remove some redundant and noisy log lines -* [ZOOKEEPER-3084](https://issues.apache.org/jira/browse/ZOOKEEPER-3084) - Exit when ZooKeeper cannot bind to the leader election port -* [ZOOKEEPER-3085](https://issues.apache.org/jira/browse/ZOOKEEPER-3085) - Define constant exit code and add documents -* [ZOOKEEPER-3094](https://issues.apache.org/jira/browse/ZOOKEEPER-3094) - Make BufferSizeTest reliable -* [ZOOKEEPER-3095](https://issues.apache.org/jira/browse/ZOOKEEPER-3095) - Connect string fix for non-existent hosts -* [ZOOKEEPER-3097](https://issues.apache.org/jira/browse/ZOOKEEPER-3097) - Use Runnable instead of Thread for working items in WorkerService to improve the throughput of CommitProcessor -* [ZOOKEEPER-3098](https://issues.apache.org/jira/browse/ZOOKEEPER-3098) - Add additional server metrics -* [ZOOKEEPER-3109](https://issues.apache.org/jira/browse/ZOOKEEPER-3109) - Avoid long unavailable time due to voter changed mind when activating the leader during election -* [ZOOKEEPER-3110](https://issues.apache.org/jira/browse/ZOOKEEPER-3110) - Improve the closeSession throughput in PrepRequestProcessor -* [ZOOKEEPER-3116](https://issues.apache.org/jira/browse/ZOOKEEPER-3116) - Make the DataTree.approximateDataSize more efficient -* [ZOOKEEPER-3124](https://issues.apache.org/jira/browse/ZOOKEEPER-3124) - Add the correct comment to show why we need the special logic to handle cversion and pzxid -* [ZOOKEEPER-3142](https://issues.apache.org/jira/browse/ZOOKEEPER-3142) - Extend SnapshotFormatter to dump data in json format -* [ZOOKEEPER-3146](https://issues.apache.org/jira/browse/ZOOKEEPER-3146) - Limit the maximum client connections per IP in NettyServerCnxnFactory -* [ZOOKEEPER-3152](https://issues.apache.org/jira/browse/ZOOKEEPER-3152) - Port ZK netty stack to netty 4 -* [ZOOKEEPER-3159](https://issues.apache.org/jira/browse/ZOOKEEPER-3159) - Flaky: ClientRequestTimeoutTest.testClientRequestTimeout -* [ZOOKEEPER-3161](https://issues.apache.org/jira/browse/ZOOKEEPER-3161) - Refactor QuorumPeerMainTest.java: move commonly used functions to base class -* [ZOOKEEPER-3163](https://issues.apache.org/jira/browse/ZOOKEEPER-3163) - Use session map to improve the performance when closing session in Netty -* [ZOOKEEPER-3177](https://issues.apache.org/jira/browse/ZOOKEEPER-3177) - Refactor request throttle logic in NIO and Netty to keep the same behavior and make the code easier to maintain -* [ZOOKEEPER-3179](https://issues.apache.org/jira/browse/ZOOKEEPER-3179) - Add snapshot compression to reduce the disk IO -* [ZOOKEEPER-3180](https://issues.apache.org/jira/browse/ZOOKEEPER-3180) - Add response cache to improve the throughput of read heavy traffic -* [ZOOKEEPER-3183](https://issues.apache.org/jira/browse/ZOOKEEPER-3183) - Interrupting or notifying the WatcherCleaner thread during shutdown if it is waiting for dead watchers get certain number(watcherCleanThreshold) and also stop adding incoming deadWatcher to deadWatchersList when shutdown is initiated. -* [ZOOKEEPER-3188](https://issues.apache.org/jira/browse/ZOOKEEPER-3188) - Improve resilience to network -* [ZOOKEEPER-3190](https://issues.apache.org/jira/browse/ZOOKEEPER-3190) - Spell check on the Zookeeper server files -* [ZOOKEEPER-3195](https://issues.apache.org/jira/browse/ZOOKEEPER-3195) - TLS - disable client-initiated renegotiation -* [ZOOKEEPER-3203](https://issues.apache.org/jira/browse/ZOOKEEPER-3203) - Tracking and exposing the non voting followers in ZK -* [ZOOKEEPER-3208](https://issues.apache.org/jira/browse/ZOOKEEPER-3208) - Remove the SSLTest.java.orig introduced in ZOOKEEPER-3032 -* [ZOOKEEPER-3216](https://issues.apache.org/jira/browse/ZOOKEEPER-3216) - Make init/sync limit tunable via JMX -* [ZOOKEEPER-3219](https://issues.apache.org/jira/browse/ZOOKEEPER-3219) - Fix flaky FileChangeWatcherTest -* [ZOOKEEPER-3228](https://issues.apache.org/jira/browse/ZOOKEEPER-3228) - [TLS] Fix key usage extension in test certs -* [ZOOKEEPER-3232](https://issues.apache.org/jira/browse/ZOOKEEPER-3232) - make the log of notification about LE more readable -* [ZOOKEEPER-3234](https://issues.apache.org/jira/browse/ZOOKEEPER-3234) - Add Travis-CI configuration file -* [ZOOKEEPER-3235](https://issues.apache.org/jira/browse/ZOOKEEPER-3235) - Enable secure processing and disallow DTDs in the SAXParserFactory -* [ZOOKEEPER-3236](https://issues.apache.org/jira/browse/ZOOKEEPER-3236) - Upgrade BouncyCastle -* [ZOOKEEPER-3237](https://issues.apache.org/jira/browse/ZOOKEEPER-3237) - Allow IPv6 wildcard address in peer config -* [ZOOKEEPER-3238](https://issues.apache.org/jira/browse/ZOOKEEPER-3238) - Add rel="noopener noreferrer" to target blank link in zookeeper-contrib-huebrowser -* [ZOOKEEPER-3239](https://issues.apache.org/jira/browse/ZOOKEEPER-3239) - Adding EnsembleAuthProvider to verify the ensemble name -* [ZOOKEEPER-3240](https://issues.apache.org/jira/browse/ZOOKEEPER-3240) - Close socket on Learner shutdown to avoid dangling socket -* [ZOOKEEPER-3242](https://issues.apache.org/jira/browse/ZOOKEEPER-3242) - Add server side connecting throttling -* [ZOOKEEPER-3243](https://issues.apache.org/jira/browse/ZOOKEEPER-3243) - Add server side request throttling -* [ZOOKEEPER-3245](https://issues.apache.org/jira/browse/ZOOKEEPER-3245) - Add useful metrics for ZK pipeline and request/server states -* [ZOOKEEPER-3249](https://issues.apache.org/jira/browse/ZOOKEEPER-3249) - Avoid reverting the cversion and pzxid during replaying txns with fuzzy snapshot -* [ZOOKEEPER-3250](https://issues.apache.org/jira/browse/ZOOKEEPER-3250) - typo in doc - zookeeperInternals -* [ZOOKEEPER-3255](https://issues.apache.org/jira/browse/ZOOKEEPER-3255) - add a banner to make the startup of zk server more cool -* [ZOOKEEPER-3257](https://issues.apache.org/jira/browse/ZOOKEEPER-3257) - Merge count and byte update of Stat -* [ZOOKEEPER-3262](https://issues.apache.org/jira/browse/ZOOKEEPER-3262) - Update dependencies flagged by OWASP report -* [ZOOKEEPER-3263](https://issues.apache.org/jira/browse/ZOOKEEPER-3263) - Illegal reflective access in zookeer's kerberosUtil -* [ZOOKEEPER-3272](https://issues.apache.org/jira/browse/ZOOKEEPER-3272) - Clean up netty4 code per Norman Maurer's review comments -* [ZOOKEEPER-3273](https://issues.apache.org/jira/browse/ZOOKEEPER-3273) - Sync BouncyCastle version in Maven build and Ant build -* [ZOOKEEPER-3274](https://issues.apache.org/jira/browse/ZOOKEEPER-3274) - Use CompositeByteBuf to queue data in NettyServerCnxn -* [ZOOKEEPER-3276](https://issues.apache.org/jira/browse/ZOOKEEPER-3276) - Make X509UtilTest.testCreateSSLServerSocketWithPort less flaky -* [ZOOKEEPER-3277](https://issues.apache.org/jira/browse/ZOOKEEPER-3277) - Add trace listener in NettyServerCnxnFactory only if trace logging is enabled -* [ZOOKEEPER-3291](https://issues.apache.org/jira/browse/ZOOKEEPER-3291) - improve error message when JAVA_HOME is set to the wrong value -* [ZOOKEEPER-3312](https://issues.apache.org/jira/browse/ZOOKEEPER-3312) - Upgrade Jetty to 9.4.15.v20190215 -* [ZOOKEEPER-3314](https://issues.apache.org/jira/browse/ZOOKEEPER-3314) - Document the possibility of MultiCallback receiving a null pointer -* [ZOOKEEPER-3332](https://issues.apache.org/jira/browse/ZOOKEEPER-3332) - TxnLogToolkit should print multi transactions readably -* [ZOOKEEPER-3335](https://issues.apache.org/jira/browse/ZOOKEEPER-3335) - Improve the usage of Collections -* [ZOOKEEPER-3339](https://issues.apache.org/jira/browse/ZOOKEEPER-3339) - Improve Debug and Trace Log Statements -* [ZOOKEEPER-3340](https://issues.apache.org/jira/browse/ZOOKEEPER-3340) - Introduce CircularBlockingQueue in QuorumCnxManager.java -* [ZOOKEEPER-3341](https://issues.apache.org/jira/browse/ZOOKEEPER-3341) - Remove Superfluous ByteBuffer Duplicate -* [ZOOKEEPER-3347](https://issues.apache.org/jira/browse/ZOOKEEPER-3347) - Improve PathTrie Consistency -* [ZOOKEEPER-3348](https://issues.apache.org/jira/browse/ZOOKEEPER-3348) - Make TxnLog and TxnLog Iterator Closable -* [ZOOKEEPER-3350](https://issues.apache.org/jira/browse/ZOOKEEPER-3350) - Get rid of CommonNames -* [ZOOKEEPER-3351](https://issues.apache.org/jira/browse/ZOOKEEPER-3351) - Migrate qa-test-pullrequest ant task to maven -* [ZOOKEEPER-3353](https://issues.apache.org/jira/browse/ZOOKEEPER-3353) - Admin commands for showing initial settings -* [ZOOKEEPER-3354](https://issues.apache.org/jira/browse/ZOOKEEPER-3354) - Improve efficiency of DeleteAllCommand -* [ZOOKEEPER-3359](https://issues.apache.org/jira/browse/ZOOKEEPER-3359) - Batch commits in the CommitProcessor -* [ZOOKEEPER-3360](https://issues.apache.org/jira/browse/ZOOKEEPER-3360) - Misprint in WriteLock javadoc -* [ZOOKEEPER-3364](https://issues.apache.org/jira/browse/ZOOKEEPER-3364) - Compile with strict options in order to check code quality -* [ZOOKEEPER-3365](https://issues.apache.org/jira/browse/ZOOKEEPER-3365) - Use Concurrent HashMap in NettyServerCnxnFactory -* [ZOOKEEPER-3369](https://issues.apache.org/jira/browse/ZOOKEEPER-3369) - Maven release artifacts cleanup -* [ZOOKEEPER-3370](https://issues.apache.org/jira/browse/ZOOKEEPER-3370) - Remove SVN specific revision generation -* [ZOOKEEPER-3372](https://issues.apache.org/jira/browse/ZOOKEEPER-3372) - Cleanup pom.xml in order to let Maven clients import as few dependencies as possible -* [ZOOKEEPER-3378](https://issues.apache.org/jira/browse/ZOOKEEPER-3378) - Set the quorum cnxn timeout independently from syncLimit -* [ZOOKEEPER-3382](https://issues.apache.org/jira/browse/ZOOKEEPER-3382) - Update Documentation: If you only have one storage device -* [ZOOKEEPER-3385](https://issues.apache.org/jira/browse/ZOOKEEPER-3385) - Add admin command to display leader -* [ZOOKEEPER-3386](https://issues.apache.org/jira/browse/ZOOKEEPER-3386) - Add admin command to display voting view -* [ZOOKEEPER-3388](https://issues.apache.org/jira/browse/ZOOKEEPER-3388) - Allow client port to support plaintext and encrypted connections simultaneously -* [ZOOKEEPER-3391](https://issues.apache.org/jira/browse/ZOOKEEPER-3391) - Drop unused CSVInputArchive and XMLInputArchive -* [ZOOKEEPER-3392](https://issues.apache.org/jira/browse/ZOOKEEPER-3392) - Add admin command to display last snapshot information -* [ZOOKEEPER-3394](https://issues.apache.org/jira/browse/ZOOKEEPER-3394) - Delay observer reconnect when all learner masters have been tried -* [ZOOKEEPER-3395](https://issues.apache.org/jira/browse/ZOOKEEPER-3395) - Document individual admin commands in markdown -* [ZOOKEEPER-3396](https://issues.apache.org/jira/browse/ZOOKEEPER-3396) - Flaky test in RestoreCommittedLogTest -* [ZOOKEEPER-3398](https://issues.apache.org/jira/browse/ZOOKEEPER-3398) - Learner.connectToLeader() may take too long to time-out -* [ZOOKEEPER-3400](https://issues.apache.org/jira/browse/ZOOKEEPER-3400) - Add documentation on local sessions -* [ZOOKEEPER-3402](https://issues.apache.org/jira/browse/ZOOKEEPER-3402) - Add a multiRead operation -* [ZOOKEEPER-3411](https://issues.apache.org/jira/browse/ZOOKEEPER-3411) - remove the deprecated CLI: ls2 and rmr -* [ZOOKEEPER-3416](https://issues.apache.org/jira/browse/ZOOKEEPER-3416) - Remove redundant ServerCnxnFactoryAccessor -* [ZOOKEEPER-3418](https://issues.apache.org/jira/browse/ZOOKEEPER-3418) - Improve quorum throughput through eager ACL checks of requests on local servers -* [ZOOKEEPER-3423](https://issues.apache.org/jira/browse/ZOOKEEPER-3423) - use the maven-like way to ignore the generated version java files and doc the cmd:'./zkServer.sh version' -* [ZOOKEEPER-3430](https://issues.apache.org/jira/browse/ZOOKEEPER-3430) - Observability improvement: provide top N read / write path queries -* [ZOOKEEPER-3436](https://issues.apache.org/jira/browse/ZOOKEEPER-3436) - Enhance Mavenized Make C client -* [ZOOKEEPER-3437](https://issues.apache.org/jira/browse/ZOOKEEPER-3437) - Improve sync throttling on a learner master -* [ZOOKEEPER-3439](https://issues.apache.org/jira/browse/ZOOKEEPER-3439) - Observability improvements on client / server connection close -* [ZOOKEEPER-3448](https://issues.apache.org/jira/browse/ZOOKEEPER-3448) - Introduce MessageTracker to assist debug leader and leaner connectivity issues -* [ZOOKEEPER-3453](https://issues.apache.org/jira/browse/ZOOKEEPER-3453) - missing 'SET' in zkCli on windows -* [ZOOKEEPER-3457](https://issues.apache.org/jira/browse/ZOOKEEPER-3457) - Code optimization in QuorumCnxManager -* [ZOOKEEPER-3459](https://issues.apache.org/jira/browse/ZOOKEEPER-3459) - Add admin command to display synced state of peer -* [ZOOKEEPER-3472](https://issues.apache.org/jira/browse/ZOOKEEPER-3472) - Treat check request as a write request which needs to wait for the check txn commit from leader -* [ZOOKEEPER-3473](https://issues.apache.org/jira/browse/ZOOKEEPER-3473) - Improving successful TLS handshake throughput with concurrent control -* [ZOOKEEPER-3484](https://issues.apache.org/jira/browse/ZOOKEEPER-3484) - Improve the throughput by optimizing the synchronization around outstandingChanges -* [ZOOKEEPER-3491](https://issues.apache.org/jira/browse/ZOOKEEPER-3491) - Specify commitLogCount value using a system property -* [ZOOKEEPER-3492](https://issues.apache.org/jira/browse/ZOOKEEPER-3492) - Add weights to server side connection throttling -* [ZOOKEEPER-3494](https://issues.apache.org/jira/browse/ZOOKEEPER-3494) - No need to depend on netty-all (SSL) -* [ZOOKEEPER-3501](https://issues.apache.org/jira/browse/ZOOKEEPER-3501) - unify the method:op2String() -* [ZOOKEEPER-3502](https://issues.apache.org/jira/browse/ZOOKEEPER-3502) - improve the server command: zabstate to have a better observation on the process of leader election -* [ZOOKEEPER-3503](https://issues.apache.org/jira/browse/ZOOKEEPER-3503) - Add server side large request throttling -* [ZOOKEEPER-3506](https://issues.apache.org/jira/browse/ZOOKEEPER-3506) - correct the SessionTrackerImpl#initializeNextSession's javaDoc about how to generate the sessionId -* [ZOOKEEPER-3509](https://issues.apache.org/jira/browse/ZOOKEEPER-3509) - Revisit log format -* [ZOOKEEPER-3519](https://issues.apache.org/jira/browse/ZOOKEEPER-3519) - upgrade dependency-check to 5.2.1 -* [ZOOKEEPER-3522](https://issues.apache.org/jira/browse/ZOOKEEPER-3522) - Consistency guarantees discussion. -* [ZOOKEEPER-3523](https://issues.apache.org/jira/browse/ZOOKEEPER-3523) - Replace dummy watcher with a unified singleton -* [ZOOKEEPER-3525](https://issues.apache.org/jira/browse/ZOOKEEPER-3525) - Add project status badges to README -* [ZOOKEEPER-3530](https://issues.apache.org/jira/browse/ZOOKEEPER-3530) - Include compiled C-client in the binary tarball -* [ZOOKEEPER-3532](https://issues.apache.org/jira/browse/ZOOKEEPER-3532) - Provide a docker-based environment to work on a known OS -* [ZOOKEEPER-3537](https://issues.apache.org/jira/browse/ZOOKEEPER-3537) - Leader election - Use of out of election messages -* [ZOOKEEPER-3548](https://issues.apache.org/jira/browse/ZOOKEEPER-3548) - Redundant zxid check in SnapStream.isValidSnapshot -* [ZOOKEEPER-3560](https://issues.apache.org/jira/browse/ZOOKEEPER-3560) - Add response cache to serve get children (2) requests. -* [ZOOKEEPER-3570](https://issues.apache.org/jira/browse/ZOOKEEPER-3570) - make the special client xid constant -* [ZOOKEEPER-3571](https://issues.apache.org/jira/browse/ZOOKEEPER-3571) - Create test base directory on test started -* [ZOOKEEPER-3593](https://issues.apache.org/jira/browse/ZOOKEEPER-3593) - fix the default value of jute.maxbuffer in client side and an optimization for the documentation -* [ZOOKEEPER-3595](https://issues.apache.org/jira/browse/ZOOKEEPER-3595) - Fsync parameter for serialize method is ingnored -* [ZOOKEEPER-3599](https://issues.apache.org/jira/browse/ZOOKEEPER-3599) - cli.c: Resuscitate "old-style" argument parsing -* [ZOOKEEPER-3606](https://issues.apache.org/jira/browse/ZOOKEEPER-3606) - add JMXHOSTNAME to zkServer.sh to enable user to change the exposed hostname of jmx service -* [ZOOKEEPER-3620](https://issues.apache.org/jira/browse/ZOOKEEPER-3620) - Allow to override calls to System.exit in server side code -* [ZOOKEEPER-3630](https://issues.apache.org/jira/browse/ZOOKEEPER-3630) - Autodetection of SSL library during Zookeeper C client build -* [ZOOKEEPER-3636](https://issues.apache.org/jira/browse/ZOOKEEPER-3636) - find back the missing configuration property in the zookeeperAdmin page when moving from xml to markdown -* [ZOOKEEPER-3638](https://issues.apache.org/jira/browse/ZOOKEEPER-3638) - Update Jetty to 9.4.24.v20191120 -* [ZOOKEEPER-3640](https://issues.apache.org/jira/browse/ZOOKEEPER-3640) - Implement "batch mode" in cli_mt -* [ZOOKEEPER-3648](https://issues.apache.org/jira/browse/ZOOKEEPER-3648) - remove Hadoop logo in the ZooKeeper documentation -* [ZOOKEEPER-3649](https://issues.apache.org/jira/browse/ZOOKEEPER-3649) - ls -s CLI need a line break - - -## Bug +## Improvement -* [ZOOKEEPER-3231](https://issues.apache.org/jira/browse/ZOOKEEPER-3231) - Purge task may lost data when the recent snapshots are all invalid -* [ZOOKEEPER-3720](https://issues.apache.org/jira/browse/ZOOKEEPER-3720) - Fix rolling upgrade failure (invalid protocol version) -* [ZOOKEEPER-3677](https://issues.apache.org/jira/browse/ZOOKEEPER-3677) - Setting jute.maxbuffer value in hexadecimal throws Exception -* [ZOOKEEPER-3695](https://issues.apache.org/jira/browse/ZOOKEEPER-3695) - Source release tarball does not match repository in 3.6.0 -* [ZOOKEEPER-3667](https://issues.apache.org/jira/browse/ZOOKEEPER-3667) - owasp checker failing for - CVE-2019-17571 Apache Log4j 1.2 deserialization of untrusted data in SocketServer -* [ZOOKEEPER-3613](https://issues.apache.org/jira/browse/ZOOKEEPER-3613) - ZKConfig fails to return proper value on getBoolean()when user accidentally includes spaces at the end of the value -* [ZOOKEEPER-3699](https://issues.apache.org/jira/browse/ZOOKEEPER-3699) - upgrade jackson-databind to address CVE-2019-20330 -* [ZOOKEEPER-3698](https://issues.apache.org/jira/browse/ZOOKEEPER-3698) - fixing NoRouteToHostException when starting large cluster locally -* [ZOOKEEPER-1936](https://issues.apache.org/jira/browse/ZOOKEEPER-1936) - Server exits when unable to create data directory due to race -* [ZOOKEEPER-3701](https://issues.apache.org/jira/browse/ZOOKEEPER-3701) - Split brain on log disk full -* [ZOOKEEPER-1105](https://issues.apache.org/jira/browse/ZOOKEEPER-1105) - wait for server response in C client zookeeper_close -* [ZOOKEEPER-706](https://issues.apache.org/jira/browse/ZOOKEEPER-706) - large numbers of watches can cause session re-establishment to fail -* [ZOOKEEPER-1029](https://issues.apache.org/jira/browse/ZOOKEEPER-1029) - C client bug in zookeeper_init (if bad hostname is given) -* [ZOOKEEPER-1077](https://issues.apache.org/jira/browse/ZOOKEEPER-1077) - C client lib doesn't build on Solaris -* [ZOOKEEPER-1256](https://issues.apache.org/jira/browse/ZOOKEEPER-1256) - ClientPortBindTest is failing on Mac OS X -* [ZOOKEEPER-1366](https://issues.apache.org/jira/browse/ZOOKEEPER-1366) - Zookeeper should be tolerant of clock adjustments -* [ZOOKEEPER-1371](https://issues.apache.org/jira/browse/ZOOKEEPER-1371) - Remove dependency on log4j in the source code. -* [ZOOKEEPER-1392](https://issues.apache.org/jira/browse/ZOOKEEPER-1392) - Should not allow to read ACL when not authorized to read node -* [ZOOKEEPER-1460](https://issues.apache.org/jira/browse/ZOOKEEPER-1460) - IPv6 literal address not supported for quorum members -* [ZOOKEEPER-1580](https://issues.apache.org/jira/browse/ZOOKEEPER-1580) - QuorumPeer.setRunning is not used -* [ZOOKEEPER-1636](https://issues.apache.org/jira/browse/ZOOKEEPER-1636) - c-client crash when zoo_amulti failed -* [ZOOKEEPER-1782](https://issues.apache.org/jira/browse/ZOOKEEPER-1782) - zookeeper.superUser is not as super as superDigest -* [ZOOKEEPER-1803](https://issues.apache.org/jira/browse/ZOOKEEPER-1803) - Add description for pzxid in programmer's guide. -* [ZOOKEEPER-1807](https://issues.apache.org/jira/browse/ZOOKEEPER-1807) - Observers spam each other creating connections to the election addr -* [ZOOKEEPER-1818](https://issues.apache.org/jira/browse/ZOOKEEPER-1818) - Fix don't care for trunk -* [ZOOKEEPER-1823](https://issues.apache.org/jira/browse/ZOOKEEPER-1823) - zkTxnLogToolkit -dump should support printing transaction data as a string -* [ZOOKEEPER-1853](https://issues.apache.org/jira/browse/ZOOKEEPER-1853) - zkCli.sh can't issue a CREATE command containing spaces in the data -* [ZOOKEEPER-1893](https://issues.apache.org/jira/browse/ZOOKEEPER-1893) - automake: use serial-tests option -* [ZOOKEEPER-1898](https://issues.apache.org/jira/browse/ZOOKEEPER-1898) - ZooKeeper Java cli shell always returns "0" as exit code -* [ZOOKEEPER-1917](https://issues.apache.org/jira/browse/ZOOKEEPER-1917) - Apache Zookeeper logs cleartext admin passwords -* [ZOOKEEPER-1919](https://issues.apache.org/jira/browse/ZOOKEEPER-1919) - Update the C implementation of removeWatches to have it match ZOOKEEPER-1910 -* [ZOOKEEPER-1927](https://issues.apache.org/jira/browse/ZOOKEEPER-1927) - zkServer.sh fails to read dataDir (and others) from zoo.cfg on Solaris 10 (grep issue, manifests as FAILED TO WRITE PID). -* [ZOOKEEPER-1932](https://issues.apache.org/jira/browse/ZOOKEEPER-1932) - Remove deprecated LeaderElection class -* [ZOOKEEPER-1949](https://issues.apache.org/jira/browse/ZOOKEEPER-1949) - recipes jar not included in the distribution package -* [ZOOKEEPER-1952](https://issues.apache.org/jira/browse/ZOOKEEPER-1952) - Default log directory and file name can be changed -* [ZOOKEEPER-1990](https://issues.apache.org/jira/browse/ZOOKEEPER-1990) - suspicious instantiation of java Random instances -* [ZOOKEEPER-1991](https://issues.apache.org/jira/browse/ZOOKEEPER-1991) - zkServer.sh returns with a zero exit status when a ZooKeeper process is already running -* [ZOOKEEPER-2006](https://issues.apache.org/jira/browse/ZOOKEEPER-2006) - Standalone mode won't take client port from dynamic config -* [ZOOKEEPER-2008](https://issues.apache.org/jira/browse/ZOOKEEPER-2008) - System test fails due to missing leader election port -* [ZOOKEEPER-2013](https://issues.apache.org/jira/browse/ZOOKEEPER-2013) - typos in zookeeperProgrammers -* [ZOOKEEPER-2014](https://issues.apache.org/jira/browse/ZOOKEEPER-2014) - Only admin should be allowed to reconfig a cluster -* [ZOOKEEPER-2026](https://issues.apache.org/jira/browse/ZOOKEEPER-2026) - Startup order in ServerCnxnFactory-ies is wrong -* [ZOOKEEPER-2029](https://issues.apache.org/jira/browse/ZOOKEEPER-2029) - Leader.LearnerCnxAcceptor should handle exceptions in run() -* [ZOOKEEPER-2030](https://issues.apache.org/jira/browse/ZOOKEEPER-2030) - dynamicConfigFile should have an absolute path, not a relative path, to the dynamic configuration file -* [ZOOKEEPER-2049](https://issues.apache.org/jira/browse/ZOOKEEPER-2049) - Yosemite build failure: htonll conflict -* [ZOOKEEPER-2052](https://issues.apache.org/jira/browse/ZOOKEEPER-2052) - Unable to delete a node when the node has no children -* [ZOOKEEPER-2054](https://issues.apache.org/jira/browse/ZOOKEEPER-2054) - test-patch.sh: don't set ulimit -n -* [ZOOKEEPER-2056](https://issues.apache.org/jira/browse/ZOOKEEPER-2056) - Zookeeper 3.4.x and 3.5.0-alpha is not OSGi compliant -* [ZOOKEEPER-2058](https://issues.apache.org/jira/browse/ZOOKEEPER-2058) - rat: exclude *.cer files -* [ZOOKEEPER-2060](https://issues.apache.org/jira/browse/ZOOKEEPER-2060) - Trace bug in NettyServerCnxnFactory -* [ZOOKEEPER-2062](https://issues.apache.org/jira/browse/ZOOKEEPER-2062) - RemoveWatchesTest takes forever to run -* [ZOOKEEPER-2064](https://issues.apache.org/jira/browse/ZOOKEEPER-2064) - Prevent resource leak in various classes -* [ZOOKEEPER-2072](https://issues.apache.org/jira/browse/ZOOKEEPER-2072) - Netty Server Should Configure Child Channel Pipeline By Specifying ChannelPipelineFactory -* [ZOOKEEPER-2073](https://issues.apache.org/jira/browse/ZOOKEEPER-2073) - Memory leak on zookeeper_close -* [ZOOKEEPER-2074](https://issues.apache.org/jira/browse/ZOOKEEPER-2074) - Incorrect exit codes for "./zkCli.sh cmd arg" -* [ZOOKEEPER-2096](https://issues.apache.org/jira/browse/ZOOKEEPER-2096) - C client builds with incorrect error codes in VisualStudio 2010+ -* [ZOOKEEPER-2109](https://issues.apache.org/jira/browse/ZOOKEEPER-2109) - Typo in src/c/src/load_gen.c -* [ZOOKEEPER-2111](https://issues.apache.org/jira/browse/ZOOKEEPER-2111) - Not isAlive states should be synchronized in ClientCnxn -* [ZOOKEEPER-2114](https://issues.apache.org/jira/browse/ZOOKEEPER-2114) - jute generated allocate_* functions are not externally visible -* [ZOOKEEPER-2116](https://issues.apache.org/jira/browse/ZOOKEEPER-2116) - zkCli.sh doesn't honor host:port parameter -* [ZOOKEEPER-2124](https://issues.apache.org/jira/browse/ZOOKEEPER-2124) - Allow Zookeeper version string to have underscore '_' -* [ZOOKEEPER-2133](https://issues.apache.org/jira/browse/ZOOKEEPER-2133) - zkperl: Segmentation fault if getting a node with null value -* [ZOOKEEPER-2142](https://issues.apache.org/jira/browse/ZOOKEEPER-2142) - JMX ObjectName is incorrect for observers -* [ZOOKEEPER-2146](https://issues.apache.org/jira/browse/ZOOKEEPER-2146) - BinaryInputArchive readString should check length before allocating memory -* [ZOOKEEPER-2156](https://issues.apache.org/jira/browse/ZOOKEEPER-2156) - If JAVA_HOME is not set zk startup and fetching status command execution result misleads user. -* [ZOOKEEPER-2157](https://issues.apache.org/jira/browse/ZOOKEEPER-2157) - Upgrade option should be removed from zkServer.sh usage -* [ZOOKEEPER-2171](https://issues.apache.org/jira/browse/ZOOKEEPER-2171) - avoid reverse lookups in QuorumCnxManager -* [ZOOKEEPER-2172](https://issues.apache.org/jira/browse/ZOOKEEPER-2172) - Cluster crashes when reconfig a new node as a participant -* [ZOOKEEPER-2173](https://issues.apache.org/jira/browse/ZOOKEEPER-2173) - ZK startup failure should be handled with proper error message -* [ZOOKEEPER-2174](https://issues.apache.org/jira/browse/ZOOKEEPER-2174) - JUnit4ZKTestRunner logs test failure for all exceptions even if the test method is annotated with an expected exception. -* [ZOOKEEPER-2178](https://issues.apache.org/jira/browse/ZOOKEEPER-2178) - Native client fails compilation on Windows. -* [ZOOKEEPER-2182](https://issues.apache.org/jira/browse/ZOOKEEPER-2182) - Several test suites are not running during pre-commit, because their names do not end with "Test". -* [ZOOKEEPER-2184](https://issues.apache.org/jira/browse/ZOOKEEPER-2184) - Zookeeper Client should re-resolve hosts when connection attempts fail -* [ZOOKEEPER-2186](https://issues.apache.org/jira/browse/ZOOKEEPER-2186) - QuorumCnxManager#receiveConnection may crash with random input -* [ZOOKEEPER-2187](https://issues.apache.org/jira/browse/ZOOKEEPER-2187) - remove duplicated code between CreateRequest{,2} -* [ZOOKEEPER-2190](https://issues.apache.org/jira/browse/ZOOKEEPER-2190) - In StandaloneDisabledTest, testReconfig() shouldn't take leaving servers as joining servers -* [ZOOKEEPER-2193](https://issues.apache.org/jira/browse/ZOOKEEPER-2193) - reconfig command completes even if parameter is wrong obviously -* [ZOOKEEPER-2195](https://issues.apache.org/jira/browse/ZOOKEEPER-2195) - fsync.warningthresholdms in zoo.cfg not working -* [ZOOKEEPER-2197](https://issues.apache.org/jira/browse/ZOOKEEPER-2197) - non-ascii character in FinalRequestProcessor.java -* [ZOOKEEPER-2198](https://issues.apache.org/jira/browse/ZOOKEEPER-2198) - Set default test.junit.threads to 1. -* [ZOOKEEPER-2201](https://issues.apache.org/jira/browse/ZOOKEEPER-2201) - Network issues can cause cluster to hang due to near-deadlock -* [ZOOKEEPER-2210](https://issues.apache.org/jira/browse/ZOOKEEPER-2210) - clock_gettime is not available in os x -* [ZOOKEEPER-2211](https://issues.apache.org/jira/browse/ZOOKEEPER-2211) - PurgeTxnLog does not correctly purge when snapshots and logs are at different locations -* [ZOOKEEPER-2212](https://issues.apache.org/jira/browse/ZOOKEEPER-2212) - distributed race condition related to QV version -* [ZOOKEEPER-2213](https://issues.apache.org/jira/browse/ZOOKEEPER-2213) - Empty path in Set crashes server and prevents restart -* [ZOOKEEPER-2221](https://issues.apache.org/jira/browse/ZOOKEEPER-2221) - Zookeeper JettyAdminServer server should start on configured IP. -* [ZOOKEEPER-2224](https://issues.apache.org/jira/browse/ZOOKEEPER-2224) - Four letter command hangs when network is slow -* [ZOOKEEPER-2227](https://issues.apache.org/jira/browse/ZOOKEEPER-2227) - stmk four-letter word fails execution at server while reading trace mask argument. -* [ZOOKEEPER-2229](https://issues.apache.org/jira/browse/ZOOKEEPER-2229) - Several four-letter words are undocumented. -* [ZOOKEEPER-2235](https://issues.apache.org/jira/browse/ZOOKEEPER-2235) - License update -* [ZOOKEEPER-2239](https://issues.apache.org/jira/browse/ZOOKEEPER-2239) - JMX State from LocalPeerBean incorrect -* [ZOOKEEPER-2243](https://issues.apache.org/jira/browse/ZOOKEEPER-2243) - Supported platforms is completely out of date -* [ZOOKEEPER-2244](https://issues.apache.org/jira/browse/ZOOKEEPER-2244) - On Windows zookeeper fails to restart -* [ZOOKEEPER-2245](https://issues.apache.org/jira/browse/ZOOKEEPER-2245) - SimpleSysTest test cases fails -* [ZOOKEEPER-2247](https://issues.apache.org/jira/browse/ZOOKEEPER-2247) - Zookeeper service becomes unavailable when leader fails to write transaction log -* [ZOOKEEPER-2249](https://issues.apache.org/jira/browse/ZOOKEEPER-2249) - CRC check failed when preAllocSize smaller than node data -* [ZOOKEEPER-2251](https://issues.apache.org/jira/browse/ZOOKEEPER-2251) - Add Client side packet response timeout to avoid infinite wait. -* [ZOOKEEPER-2252](https://issues.apache.org/jira/browse/ZOOKEEPER-2252) - Random test case failure in org.apache.zookeeper.test.StaticHostProviderTest -* [ZOOKEEPER-2256](https://issues.apache.org/jira/browse/ZOOKEEPER-2256) - Zookeeper is not using specified JMX port in zkEnv.sh -* [ZOOKEEPER-2261](https://issues.apache.org/jira/browse/ZOOKEEPER-2261) - When only secureClientPort is configured connections, configuration, connection_stat_reset, and stats admin commands throw NullPointerException -* [ZOOKEEPER-2264](https://issues.apache.org/jira/browse/ZOOKEEPER-2264) - Wrong error message when secureClientPortAddress is configured but secureClientPort is not configured -* [ZOOKEEPER-2269](https://issues.apache.org/jira/browse/ZOOKEEPER-2269) - NullPointerException in RemotePeerBean -* [ZOOKEEPER-2279](https://issues.apache.org/jira/browse/ZOOKEEPER-2279) - QuorumPeer loadDataBase() error message is incorrect -* [ZOOKEEPER-2281](https://issues.apache.org/jira/browse/ZOOKEEPER-2281) - ZK Server startup fails if there are spaces in the JAVA_HOME path -* [ZOOKEEPER-2282](https://issues.apache.org/jira/browse/ZOOKEEPER-2282) - chroot not stripped from path in asynchronous callbacks -* [ZOOKEEPER-2283](https://issues.apache.org/jira/browse/ZOOKEEPER-2283) - traceFile property is not used in the ZooKeeper, it should be removed from documentation -* [ZOOKEEPER-2284](https://issues.apache.org/jira/browse/ZOOKEEPER-2284) - LogFormatter and SnapshotFormatter does not handle FileNotFoundException gracefully -* [ZOOKEEPER-2294](https://issues.apache.org/jira/browse/ZOOKEEPER-2294) - Ant target generate-clover-reports is broken -* [ZOOKEEPER-2295](https://issues.apache.org/jira/browse/ZOOKEEPER-2295) - TGT refresh time logic is wrong -* [ZOOKEEPER-2297](https://issues.apache.org/jira/browse/ZOOKEEPER-2297) - NPE is thrown while creating "key manager" and "trust manager" -* [ZOOKEEPER-2299](https://issues.apache.org/jira/browse/ZOOKEEPER-2299) - NullPointerException in LocalPeerBean for ClientAddress -* [ZOOKEEPER-2302](https://issues.apache.org/jira/browse/ZOOKEEPER-2302) - Some test cases are not running because wrongly named -* [ZOOKEEPER-2307](https://issues.apache.org/jira/browse/ZOOKEEPER-2307) - ZooKeeper not starting because acceptedEpoch is less than the currentEpoch -* [ZOOKEEPER-2311](https://issues.apache.org/jira/browse/ZOOKEEPER-2311) - assert in setup_random -* [ZOOKEEPER-2316](https://issues.apache.org/jira/browse/ZOOKEEPER-2316) - comment does not match code logic -* [ZOOKEEPER-2317](https://issues.apache.org/jira/browse/ZOOKEEPER-2317) - Non-OSGi compatible version -* [ZOOKEEPER-2319](https://issues.apache.org/jira/browse/ZOOKEEPER-2319) - UnresolvedAddressException cause the QuorumCnxManager.Listener exit -* [ZOOKEEPER-2325](https://issues.apache.org/jira/browse/ZOOKEEPER-2325) - Data inconsistency if all snapshots empty or missing -* [ZOOKEEPER-2330](https://issues.apache.org/jira/browse/ZOOKEEPER-2330) - ZooKeeper close API does not close Login thread. -* [ZOOKEEPER-2335](https://issues.apache.org/jira/browse/ZOOKEEPER-2335) - Java Compilation Error in ClientCnxn.java -* [ZOOKEEPER-2338](https://issues.apache.org/jira/browse/ZOOKEEPER-2338) - c bindings should create socket's with SOCK_CLOEXEC to avoid fd leaks on fork/exec -* [ZOOKEEPER-2340](https://issues.apache.org/jira/browse/ZOOKEEPER-2340) - JMX is disabled even if JMXDISABLE is false -* [ZOOKEEPER-2349](https://issues.apache.org/jira/browse/ZOOKEEPER-2349) - Update documentation for snapCount -* [ZOOKEEPER-2355](https://issues.apache.org/jira/browse/ZOOKEEPER-2355) - Ephemeral node is never deleted if follower fails while reading the proposal packet -* [ZOOKEEPER-2364](https://issues.apache.org/jira/browse/ZOOKEEPER-2364) - "ant docs" fails on branch-3.5 due to missing releasenotes.xml. -* [ZOOKEEPER-2366](https://issues.apache.org/jira/browse/ZOOKEEPER-2366) - Reconfiguration of client port causes a socket leak -* [ZOOKEEPER-2375](https://issues.apache.org/jira/browse/ZOOKEEPER-2375) - Prevent multiple initialization of login object in each ZooKeeperSaslClient instance -* [ZOOKEEPER-2379](https://issues.apache.org/jira/browse/ZOOKEEPER-2379) - recent commit broke findbugs qabot check -* [ZOOKEEPER-2380](https://issues.apache.org/jira/browse/ZOOKEEPER-2380) - Deadlock between leader shutdown and forwarding ACK to the leader -* [ZOOKEEPER-2383](https://issues.apache.org/jira/browse/ZOOKEEPER-2383) - Startup race in ZooKeeperServer -* [ZOOKEEPER-2385](https://issues.apache.org/jira/browse/ZOOKEEPER-2385) - Zookeeper trunk build is failing on windows -* [ZOOKEEPER-2388](https://issues.apache.org/jira/browse/ZOOKEEPER-2388) - Unit tests failing on Solaris -* [ZOOKEEPER-2393](https://issues.apache.org/jira/browse/ZOOKEEPER-2393) - Revert run-time dependency on log4j and slf4j-log4j12 -* [ZOOKEEPER-2405](https://issues.apache.org/jira/browse/ZOOKEEPER-2405) - getTGT() in Login.java mishandles confidential information -* [ZOOKEEPER-2413](https://issues.apache.org/jira/browse/ZOOKEEPER-2413) - ContainerManager doesn't close the Timer it creates when stop() is called -* [ZOOKEEPER-2418](https://issues.apache.org/jira/browse/ZOOKEEPER-2418) - txnlog diff sync can skip sending some transactions to followers -* [ZOOKEEPER-2442](https://issues.apache.org/jira/browse/ZOOKEEPER-2442) - Socket leak in QuorumCnxManager connectOne -* [ZOOKEEPER-2450](https://issues.apache.org/jira/browse/ZOOKEEPER-2450) - Upgrade Netty version due to security vulnerability (CVE-2014-3488) -* [ZOOKEEPER-2458](https://issues.apache.org/jira/browse/ZOOKEEPER-2458) - Remove license file for servlet-api dependency -* [ZOOKEEPER-2459](https://issues.apache.org/jira/browse/ZOOKEEPER-2459) - Update NOTICE file with Netty notice -* [ZOOKEEPER-2460](https://issues.apache.org/jira/browse/ZOOKEEPER-2460) - Remove javacc dependency from public Maven pom -* [ZOOKEEPER-2464](https://issues.apache.org/jira/browse/ZOOKEEPER-2464) - NullPointerException on ContainerManager -* [ZOOKEEPER-2465](https://issues.apache.org/jira/browse/ZOOKEEPER-2465) - Documentation copyright notice is out of date. -* [ZOOKEEPER-2467](https://issues.apache.org/jira/browse/ZOOKEEPER-2467) - NullPointerException when redo Command is passed negative value -* [ZOOKEEPER-2470](https://issues.apache.org/jira/browse/ZOOKEEPER-2470) - ServerConfig#parse(String[]) ignores tickTime -* [ZOOKEEPER-2474](https://issues.apache.org/jira/browse/ZOOKEEPER-2474) - add a way for client to reattach to a session when using ZKClientConfig -* [ZOOKEEPER-2477](https://issues.apache.org/jira/browse/ZOOKEEPER-2477) - documentation should refer to Java cli shell and not C cli shell -* [ZOOKEEPER-2500](https://issues.apache.org/jira/browse/ZOOKEEPER-2500) - Fix compilation warnings for CliException classes -* [ZOOKEEPER-2517](https://issues.apache.org/jira/browse/ZOOKEEPER-2517) - jute.maxbuffer is ignored -* [ZOOKEEPER-2536](https://issues.apache.org/jira/browse/ZOOKEEPER-2536) - When provide path for "dataDir" with trailing space, it is taking correct path (by trucating space) for snapshot but creating temporary file with some junk folder name for zookeeper_server.pid -* [ZOOKEEPER-2539](https://issues.apache.org/jira/browse/ZOOKEEPER-2539) - Throwing nullpointerException when run the command "config -c" when client port is mentioned as separate and not like new style -* [ZOOKEEPER-2548](https://issues.apache.org/jira/browse/ZOOKEEPER-2548) - zooInspector does not start on Windows -* [ZOOKEEPER-2558](https://issues.apache.org/jira/browse/ZOOKEEPER-2558) - Potential memory leak in recordio.c -* [ZOOKEEPER-2563](https://issues.apache.org/jira/browse/ZOOKEEPER-2563) - A revisit to setquota -* [ZOOKEEPER-2573](https://issues.apache.org/jira/browse/ZOOKEEPER-2573) - Modify Info.REVISION to adapt git repo -* [ZOOKEEPER-2574](https://issues.apache.org/jira/browse/ZOOKEEPER-2574) - PurgeTxnLog can inadvertently delete required txn log files -* [ZOOKEEPER-2576](https://issues.apache.org/jira/browse/ZOOKEEPER-2576) - After svn to git migration ZooKeeper Precommit jenkins job is failing. -* [ZOOKEEPER-2579](https://issues.apache.org/jira/browse/ZOOKEEPER-2579) - ZooKeeper server should verify that dataDir and snapDir are writeable before starting -* [ZOOKEEPER-2581](https://issues.apache.org/jira/browse/ZOOKEEPER-2581) - Not handled NullPointerException while creating key manager and trustManager -* [ZOOKEEPER-2606](https://issues.apache.org/jira/browse/ZOOKEEPER-2606) - SaslServerCallbackHandler#handleAuthorizeCallback() should log the exception -* [ZOOKEEPER-2611](https://issues.apache.org/jira/browse/ZOOKEEPER-2611) - zoo_remove_watchers - can remove the wrong watch -* [ZOOKEEPER-2617](https://issues.apache.org/jira/browse/ZOOKEEPER-2617) - correct a few spelling typos -* [ZOOKEEPER-2621](https://issues.apache.org/jira/browse/ZOOKEEPER-2621) - ZooKeeper doesn't start on MINGW32 (Windows) -* [ZOOKEEPER-2622](https://issues.apache.org/jira/browse/ZOOKEEPER-2622) - ZooTrace.logQuorumPacket does nothing -* [ZOOKEEPER-2628](https://issues.apache.org/jira/browse/ZOOKEEPER-2628) - Investigate and fix findbug warnings -* [ZOOKEEPER-2633](https://issues.apache.org/jira/browse/ZOOKEEPER-2633) - Build failure in contrib/zkfuse with gcc 6.x -* [ZOOKEEPER-2635](https://issues.apache.org/jira/browse/ZOOKEEPER-2635) - Regenerate documentation -* [ZOOKEEPER-2636](https://issues.apache.org/jira/browse/ZOOKEEPER-2636) - Fix C build break. -* [ZOOKEEPER-2642](https://issues.apache.org/jira/browse/ZOOKEEPER-2642) - ZooKeeper reconfig API backward compatibility fix -* [ZOOKEEPER-2647](https://issues.apache.org/jira/browse/ZOOKEEPER-2647) - Fix TestReconfigServer.cc -* [ZOOKEEPER-2651](https://issues.apache.org/jira/browse/ZOOKEEPER-2651) - Missing src/pom.template in release -* [ZOOKEEPER-2678](https://issues.apache.org/jira/browse/ZOOKEEPER-2678) - Large databases take a long time to regain a quorum -* [ZOOKEEPER-2680](https://issues.apache.org/jira/browse/ZOOKEEPER-2680) - Correct DataNode.getChildren() inconsistent behaviour. -* [ZOOKEEPER-2683](https://issues.apache.org/jira/browse/ZOOKEEPER-2683) - RaceConditionTest is flaky -* [ZOOKEEPER-2684](https://issues.apache.org/jira/browse/ZOOKEEPER-2684) - Fix a crashing bug in the mixed workloads commit processor -* [ZOOKEEPER-2687](https://issues.apache.org/jira/browse/ZOOKEEPER-2687) - Deadlock while shutting down the Leader server. -* [ZOOKEEPER-2690](https://issues.apache.org/jira/browse/ZOOKEEPER-2690) - Update documentation source for ZOOKEEPER-2574 -* [ZOOKEEPER-2693](https://issues.apache.org/jira/browse/ZOOKEEPER-2693) - DOS attack on wchp/wchc four letter words (4lw) -* [ZOOKEEPER-2694](https://issues.apache.org/jira/browse/ZOOKEEPER-2694) - sync CLI command does not wait for result from server -* [ZOOKEEPER-2722](https://issues.apache.org/jira/browse/ZOOKEEPER-2722) - Flaky Test: org.apache.zookeeper.test.ReadOnlyModeTest.testSessionEstablishment -* [ZOOKEEPER-2725](https://issues.apache.org/jira/browse/ZOOKEEPER-2725) - Upgrading to a global session fails with a multiop -* [ZOOKEEPER-2726](https://issues.apache.org/jira/browse/ZOOKEEPER-2726) - Patch for ZOOKEEPER-2693 introduces potential race condition -* [ZOOKEEPER-2735](https://issues.apache.org/jira/browse/ZOOKEEPER-2735) - Typo fixes in some scripts -* [ZOOKEEPER-2737](https://issues.apache.org/jira/browse/ZOOKEEPER-2737) - NettyServerCnxFactory leaks connection if exception happens while writing to a channel. -* [ZOOKEEPER-2743](https://issues.apache.org/jira/browse/ZOOKEEPER-2743) - Netty connection leaks JMX connection bean upon connection close in certain race conditions. -* [ZOOKEEPER-2747](https://issues.apache.org/jira/browse/ZOOKEEPER-2747) - Fix ZooKeeperAdmin Compilation Warning -* [ZOOKEEPER-2757](https://issues.apache.org/jira/browse/ZOOKEEPER-2757) - Incorrect path crashes zkCli -* [ZOOKEEPER-2758](https://issues.apache.org/jira/browse/ZOOKEEPER-2758) - Typo: transasction --> transaction -* [ZOOKEEPER-2775](https://issues.apache.org/jira/browse/ZOOKEEPER-2775) - ZK Client not able to connect with Xid out of order error -* [ZOOKEEPER-2777](https://issues.apache.org/jira/browse/ZOOKEEPER-2777) - There is a typo in zk.py which prevents from using/compiling it. -* [ZOOKEEPER-2778](https://issues.apache.org/jira/browse/ZOOKEEPER-2778) - Potential server deadlock between follower sync with leader and follower receiving external connection requests. -* [ZOOKEEPER-2785](https://issues.apache.org/jira/browse/ZOOKEEPER-2785) - Server inappropriately throttles connections under load before SASL completes -* [ZOOKEEPER-2786](https://issues.apache.org/jira/browse/ZOOKEEPER-2786) - Flaky test: org.apache.zookeeper.test.ClientTest.testNonExistingOpCode -* [ZOOKEEPER-2797](https://issues.apache.org/jira/browse/ZOOKEEPER-2797) - Invalid TTL from misbehaving client nukes zookeeper -* [ZOOKEEPER-2798](https://issues.apache.org/jira/browse/ZOOKEEPER-2798) - Fix flaky test: org.apache.zookeeper.test.ReadOnlyModeTest.testConnectionEvents -* [ZOOKEEPER-2804](https://issues.apache.org/jira/browse/ZOOKEEPER-2804) - Node creation fails with NPE if ACLs are null -* [ZOOKEEPER-2806](https://issues.apache.org/jira/browse/ZOOKEEPER-2806) - Flaky test: org.apache.zookeeper.server.quorum.FLEBackwardElectionRoundTest.testBackwardElectionRound -* [ZOOKEEPER-2808](https://issues.apache.org/jira/browse/ZOOKEEPER-2808) - ACL with index 1 might be removed if it's only being used once -* [ZOOKEEPER-2818](https://issues.apache.org/jira/browse/ZOOKEEPER-2818) - Improve the ZooKeeper#setACL java doc -* [ZOOKEEPER-2819](https://issues.apache.org/jira/browse/ZOOKEEPER-2819) - Changing membership configuration via rolling restart does not work on 3.5.x. -* [ZOOKEEPER-2822](https://issues.apache.org/jira/browse/ZOOKEEPER-2822) - Wrong `ObjectName` about `MBeanServer` in JMX module -* [ZOOKEEPER-2841](https://issues.apache.org/jira/browse/ZOOKEEPER-2841) - ZooKeeper public include files leak porting changes -* [ZOOKEEPER-2845](https://issues.apache.org/jira/browse/ZOOKEEPER-2845) - Data inconsistency issue due to retain database in leader election -* [ZOOKEEPER-2847](https://issues.apache.org/jira/browse/ZOOKEEPER-2847) - Cannot bind to client port when reconfig based on old static config -* [ZOOKEEPER-2852](https://issues.apache.org/jira/browse/ZOOKEEPER-2852) - Snapshot size factor is not read from system property -* [ZOOKEEPER-2853](https://issues.apache.org/jira/browse/ZOOKEEPER-2853) - The lastZxidSeen in FileTxnLog.java is never being assigned -* [ZOOKEEPER-2859](https://issues.apache.org/jira/browse/ZOOKEEPER-2859) - CMake build doesn't support OS X -* [ZOOKEEPER-2861](https://issues.apache.org/jira/browse/ZOOKEEPER-2861) - Main-Class JAR manifest attribute is incorrect -* [ZOOKEEPER-2862](https://issues.apache.org/jira/browse/ZOOKEEPER-2862) - Incorrect javadoc syntax for web links in StaticHostProvider.java -* [ZOOKEEPER-2874](https://issues.apache.org/jira/browse/ZOOKEEPER-2874) - Windows Debug builds don't link with `/MTd` -* [ZOOKEEPER-2886](https://issues.apache.org/jira/browse/ZOOKEEPER-2886) - Permanent session moved error in multi-op only connections -* [ZOOKEEPER-2890](https://issues.apache.org/jira/browse/ZOOKEEPER-2890) - Local automatic variable is left uninitialized and then freed. -* [ZOOKEEPER-2891](https://issues.apache.org/jira/browse/ZOOKEEPER-2891) - Invalid processing of zookeeper_close for mutli-request -* [ZOOKEEPER-2893](https://issues.apache.org/jira/browse/ZOOKEEPER-2893) - very poor choice of logging if client fails to connect to server -* [ZOOKEEPER-2894](https://issues.apache.org/jira/browse/ZOOKEEPER-2894) - Memory and completions leak on zookeeper_close -* [ZOOKEEPER-2901](https://issues.apache.org/jira/browse/ZOOKEEPER-2901) - Session ID that is negative causes mis-calculation of Ephemeral Type -* [ZOOKEEPER-2905](https://issues.apache.org/jira/browse/ZOOKEEPER-2905) - Don't include `config.h` in `zookeeper.h` -* [ZOOKEEPER-2906](https://issues.apache.org/jira/browse/ZOOKEEPER-2906) - The OWASP dependency check jar should not be included in the default classpath -* [ZOOKEEPER-2909](https://issues.apache.org/jira/browse/ZOOKEEPER-2909) - Create ant task to generate ivy dependency reports -* [ZOOKEEPER-2913](https://issues.apache.org/jira/browse/ZOOKEEPER-2913) - testEphemeralNodeDeletion is flaky -* [ZOOKEEPER-2914](https://issues.apache.org/jira/browse/ZOOKEEPER-2914) - compiler warning using java 9 -* [ZOOKEEPER-2920](https://issues.apache.org/jira/browse/ZOOKEEPER-2920) - Upgrade OWASP Dependency Check to 3.2.1 -* [ZOOKEEPER-2923](https://issues.apache.org/jira/browse/ZOOKEEPER-2923) - The comment of the variable matchSyncs in class CommitProcessor has a mistake. -* [ZOOKEEPER-2924](https://issues.apache.org/jira/browse/ZOOKEEPER-2924) - Flaky Test: org.apache.zookeeper.test.LoadFromLogTest.testRestoreWithTransactionErrors -* [ZOOKEEPER-2926](https://issues.apache.org/jira/browse/ZOOKEEPER-2926) - Data inconsistency issue due to the flaw in the session management -* [ZOOKEEPER-2931](https://issues.apache.org/jira/browse/ZOOKEEPER-2931) - WriteLock recipe: incorrect znode ordering when the sessionId is part of the znode name -* [ZOOKEEPER-2934](https://issues.apache.org/jira/browse/ZOOKEEPER-2934) - c versions of election and queue recipes do not compile -* [ZOOKEEPER-2936](https://issues.apache.org/jira/browse/ZOOKEEPER-2936) - Duplicate Keys in log4j.properties config files -* [ZOOKEEPER-2944](https://issues.apache.org/jira/browse/ZOOKEEPER-2944) - Specify correct overflow value -* [ZOOKEEPER-2948](https://issues.apache.org/jira/browse/ZOOKEEPER-2948) - Failing c unit tests on apache jenkins -* [ZOOKEEPER-2949](https://issues.apache.org/jira/browse/ZOOKEEPER-2949) - SSL ServerName not set when using hostname, some proxies may failed to proxy the request. -* [ZOOKEEPER-2951](https://issues.apache.org/jira/browse/ZOOKEEPER-2951) - zkServer.cmd does not start when JAVA_HOME ends with a \ -* [ZOOKEEPER-2953](https://issues.apache.org/jira/browse/ZOOKEEPER-2953) - Flaky Test: testNoLogBeforeLeaderEstablishment -* [ZOOKEEPER-2959](https://issues.apache.org/jira/browse/ZOOKEEPER-2959) - ignore accepted epoch and LEADERINFO ack from observers when a newly elected leader computes new epoch -* [ZOOKEEPER-2961](https://issues.apache.org/jira/browse/ZOOKEEPER-2961) - Fix testElectionFraud Flakyness -* [ZOOKEEPER-2964](https://issues.apache.org/jira/browse/ZOOKEEPER-2964) - "Conf" command returns dataDir and dataLogDir opposingly -* [ZOOKEEPER-2978](https://issues.apache.org/jira/browse/ZOOKEEPER-2978) - fix potential null pointer exception when deleting node -* [ZOOKEEPER-2982](https://issues.apache.org/jira/browse/ZOOKEEPER-2982) - Re-try DNS hostname -> IP resolution -* [ZOOKEEPER-2988](https://issues.apache.org/jira/browse/ZOOKEEPER-2988) - NPE triggered if server receives a vote for a server id not in their voting view -* [ZOOKEEPER-2992](https://issues.apache.org/jira/browse/ZOOKEEPER-2992) - The eclipse build target fails due to protocol redirection: http->https -* [ZOOKEEPER-2993](https://issues.apache.org/jira/browse/ZOOKEEPER-2993) - .ignore file prevents adding src/java/main/org/apache/jute/compiler/generated dir to git repo -* [ZOOKEEPER-2997](https://issues.apache.org/jira/browse/ZOOKEEPER-2997) - CMake should not force static CRT linking -* [ZOOKEEPER-2998](https://issues.apache.org/jira/browse/ZOOKEEPER-2998) - CMake declares incorrect ZooKeeper version -* [ZOOKEEPER-3001](https://issues.apache.org/jira/browse/ZOOKEEPER-3001) - Incorrect log message when try to delete container node -* [ZOOKEEPER-3006](https://issues.apache.org/jira/browse/ZOOKEEPER-3006) - Potential NPE in ZKDatabase#calculateTxnLogSizeLimit -* [ZOOKEEPER-3007](https://issues.apache.org/jira/browse/ZOOKEEPER-3007) - Potential NPE in ReferenceCountedACLCache#deserialize -* [ZOOKEEPER-3009](https://issues.apache.org/jira/browse/ZOOKEEPER-3009) - Potential NPE in NIOServerCnxnFactory -* [ZOOKEEPER-3025](https://issues.apache.org/jira/browse/ZOOKEEPER-3025) - cmake windows build is broken on jenkins -* [ZOOKEEPER-3027](https://issues.apache.org/jira/browse/ZOOKEEPER-3027) - Accidently removed public API of FileTxnLog.setPreallocSize() -* [ZOOKEEPER-3034](https://issues.apache.org/jira/browse/ZOOKEEPER-3034) - Facing issues while building from source -* [ZOOKEEPER-3038](https://issues.apache.org/jira/browse/ZOOKEEPER-3038) - Cleanup some nitpicks in TTL implementation -* [ZOOKEEPER-3039](https://issues.apache.org/jira/browse/ZOOKEEPER-3039) - TxnLogToolkit uses Scanner badly -* [ZOOKEEPER-3041](https://issues.apache.org/jira/browse/ZOOKEEPER-3041) - Typo in error message, affects log analysis -* [ZOOKEEPER-3050](https://issues.apache.org/jira/browse/ZOOKEEPER-3050) - owasp ant target is highlighting jetty version needs to be updated -* [ZOOKEEPER-3051](https://issues.apache.org/jira/browse/ZOOKEEPER-3051) - owasp complaining about jackson version used -* [ZOOKEEPER-3056](https://issues.apache.org/jira/browse/ZOOKEEPER-3056) - Fails to load database with missing snapshot file but valid transaction log file -* [ZOOKEEPER-3057](https://issues.apache.org/jira/browse/ZOOKEEPER-3057) - Fix IPv6 literal usage -* [ZOOKEEPER-3059](https://issues.apache.org/jira/browse/ZOOKEEPER-3059) - EventThread leak in case of Sasl AuthFailed -* [ZOOKEEPER-3072](https://issues.apache.org/jira/browse/ZOOKEEPER-3072) - Race condition in throttling -* [ZOOKEEPER-3079](https://issues.apache.org/jira/browse/ZOOKEEPER-3079) - Fix unsafe use of sprintf(3) for creating IP address strings -* [ZOOKEEPER-3082](https://issues.apache.org/jira/browse/ZOOKEEPER-3082) - Fix server snapshot behavior when out of disk space -* [ZOOKEEPER-3093](https://issues.apache.org/jira/browse/ZOOKEEPER-3093) - sync zerror(int rc) with newest error definitions -* [ZOOKEEPER-3104](https://issues.apache.org/jira/browse/ZOOKEEPER-3104) - Potential data inconsistency due to NEWLEADER packet being sent too early during SNAP sync -* [ZOOKEEPER-3105](https://issues.apache.org/jira/browse/ZOOKEEPER-3105) - Character coding problem occur when create a node using python3 -* [ZOOKEEPER-3113](https://issues.apache.org/jira/browse/ZOOKEEPER-3113) - EphemeralType.get() fails to verify ephemeralOwner when currentElapsedTime() is small enough -* [ZOOKEEPER-3117](https://issues.apache.org/jira/browse/ZOOKEEPER-3117) - Correct the LeaderBean.followerInfo to only return the followers list -* [ZOOKEEPER-3125](https://issues.apache.org/jira/browse/ZOOKEEPER-3125) - Pzxid inconsistent issue when replaying a txn for a deleted node -* [ZOOKEEPER-3127](https://issues.apache.org/jira/browse/ZOOKEEPER-3127) - Fixing potential data inconsistency due to update last processed zxid with partial multi-op txn -* [ZOOKEEPER-3131](https://issues.apache.org/jira/browse/ZOOKEEPER-3131) - org.apache.zookeeper.server.WatchManager resource leak -* [ZOOKEEPER-3144](https://issues.apache.org/jira/browse/ZOOKEEPER-3144) - Potential ephemeral nodes inconsistent due to global session inconsistent with fuzzy snapshot -* [ZOOKEEPER-3145](https://issues.apache.org/jira/browse/ZOOKEEPER-3145) - Potential watch missing issue due to stale pzxid when replaying CloseSession txn with fuzzy snapshot -* [ZOOKEEPER-3156](https://issues.apache.org/jira/browse/ZOOKEEPER-3156) - ZOOKEEPER-2184 causes kerberos principal to not have resolved host name -* [ZOOKEEPER-3162](https://issues.apache.org/jira/browse/ZOOKEEPER-3162) - Broken lock semantics in C client lock-recipe -* [ZOOKEEPER-3210](https://issues.apache.org/jira/browse/ZOOKEEPER-3210) - Typo in zookeeperInternals doc -* [ZOOKEEPER-3212](https://issues.apache.org/jira/browse/ZOOKEEPER-3212) - Fix website with adding doap.rdf back -* [ZOOKEEPER-3217](https://issues.apache.org/jira/browse/ZOOKEEPER-3217) - owasp job flagging slf4j on trunk -* [ZOOKEEPER-3218](https://issues.apache.org/jira/browse/ZOOKEEPER-3218) - zk server reopened,the interval for observer connect to the new leader is too long,then session expired -* [ZOOKEEPER-3253](https://issues.apache.org/jira/browse/ZOOKEEPER-3253) - client should not send requests with cxid=-4, -2, or -1 -* [ZOOKEEPER-3265](https://issues.apache.org/jira/browse/ZOOKEEPER-3265) - Build failure on branch-3.4 -* [ZOOKEEPER-3296](https://issues.apache.org/jira/browse/ZOOKEEPER-3296) - Cannot join quorum due to Quorum SSLSocket connection not closed explicitly when there is handshake issue -* [ZOOKEEPER-3306](https://issues.apache.org/jira/browse/ZOOKEEPER-3306) - Node may not accessible due the the inconsistent ACL reference map after SNAP sync -* [ZOOKEEPER-3320](https://issues.apache.org/jira/browse/ZOOKEEPER-3320) - Leader election port stop listen when hostname unresolvable for some time -* [ZOOKEEPER-3356](https://issues.apache.org/jira/browse/ZOOKEEPER-3356) - Request throttling in Netty is not working as expected and could cause direct buffer OOM issue -* [ZOOKEEPER-3373](https://issues.apache.org/jira/browse/ZOOKEEPER-3373) - need change description for "Single System Image" guarantee in document -* [ZOOKEEPER-3399](https://issues.apache.org/jira/browse/ZOOKEEPER-3399) - Remove logging in getGlobalOutstandingLimit for optimal performance. -* [ZOOKEEPER-3404](https://issues.apache.org/jira/browse/ZOOKEEPER-3404) - BouncyCastle upgrade to 1.61 might cause flaky test issues -* [ZOOKEEPER-3405](https://issues.apache.org/jira/browse/ZOOKEEPER-3405) - owasp flagging jackson-databind -* [ZOOKEEPER-3433](https://issues.apache.org/jira/browse/ZOOKEEPER-3433) - zkpython build broken after maven migration -* [ZOOKEEPER-3440](https://issues.apache.org/jira/browse/ZOOKEEPER-3440) - Fix Apache RAT check by excluding binary files (images) -* [ZOOKEEPER-3471](https://issues.apache.org/jira/browse/ZOOKEEPER-3471) - Potential lock unavailable due to dangling ephemeral nodes left during local session upgrading -* [ZOOKEEPER-3479](https://issues.apache.org/jira/browse/ZOOKEEPER-3479) - Logging false leader election times -* [ZOOKEEPER-3496](https://issues.apache.org/jira/browse/ZOOKEEPER-3496) - Transaction larger than jute.maxbuffer makes ZooKeeper unavailable -* [ZOOKEEPER-3498](https://issues.apache.org/jira/browse/ZOOKEEPER-3498) - In zookeeper-jute project generated source should not be in target\classes folder -* [ZOOKEEPER-3510](https://issues.apache.org/jira/browse/ZOOKEEPER-3510) - Frequent 'zkServer.sh stop' failures when running C test suite -* [ZOOKEEPER-3518](https://issues.apache.org/jira/browse/ZOOKEEPER-3518) - owasp check flagging jackson-databind 2.9.9.1 -* [ZOOKEEPER-3531](https://issues.apache.org/jira/browse/ZOOKEEPER-3531) - Synchronization on ACLCache cause cluster to hang when network/disk issues happen during datatree serialization -* [ZOOKEEPER-3540](https://issues.apache.org/jira/browse/ZOOKEEPER-3540) - Client port unavailable after binding the same client port during reconfig -* [ZOOKEEPER-3546](https://issues.apache.org/jira/browse/ZOOKEEPER-3546) - Containers that never have children stay forever -* [ZOOKEEPER-3559](https://issues.apache.org/jira/browse/ZOOKEEPER-3559) - Update Jackson to 2.9.10 -* [ZOOKEEPER-3563](https://issues.apache.org/jira/browse/ZOOKEEPER-3563) - dependency check failing on 3.4 and 3.5 branches - CVE-2019-16869 on Netty -* [ZOOKEEPER-3590](https://issues.apache.org/jira/browse/ZOOKEEPER-3590) - Zookeeper is unable to set the zookeeper.sasl.client.canonicalize.hostname using system variable -* [ZOOKEEPER-3605](https://issues.apache.org/jira/browse/ZOOKEEPER-3605) - ZOOKEEPER-3242 add a connection throttle. Default constructor needs to set it -* [ZOOKEEPER-3633](https://issues.apache.org/jira/browse/ZOOKEEPER-3633) - AdminServer commands throw NPE when only secure client port is used -* [ZOOKEEPER-3641](https://issues.apache.org/jira/browse/ZOOKEEPER-3641) - New ZOO_VERSION define breaks Perl & Python contribs -* [ZOOKEEPER-3644](https://issues.apache.org/jira/browse/ZOOKEEPER-3644) - Data loss after upgrading standalone ZK server 3.4.14 to 3.5.6 with snapshot.trust.empty=true -* [ZOOKEEPER-3651](https://issues.apache.org/jira/browse/ZOOKEEPER-3651) - NettyServerCnxnFactoryTest is flaky -* [ZOOKEEPER-3653](https://issues.apache.org/jira/browse/ZOOKEEPER-3653) - Audit Log feature fails in a stand alone zookeeper setup - -## Test +* [ZOOKEEPER-3685](https://issues.apache.org/jira/browse/ZOOKEEPER-3685) - Use JDK Arrays Equals for Jute +* [ZOOKEEPER-3686](https://issues.apache.org/jira/browse/ZOOKEEPER-3686) - Use JDK Arrays hashCode for Jute +* [ZOOKEEPER-3708](https://issues.apache.org/jira/browse/ZOOKEEPER-3708) - Move Logging Code into Logging Guard in Learner +* [ZOOKEEPER-3741](https://issues.apache.org/jira/browse/ZOOKEEPER-3741) - Fix ZooKeeper 3.5 C client build on Fedora8 +* [ZOOKEEPER-3755](https://issues.apache.org/jira/browse/ZOOKEEPER-3755) - Use maven to create fatjar +* [ZOOKEEPER-3756](https://issues.apache.org/jira/browse/ZOOKEEPER-3756) - Members failing to rejoin quorum +* [ZOOKEEPER-3785](https://issues.apache.org/jira/browse/ZOOKEEPER-3785) - Make sources buildable with JDK14 -* [ZOOKEEPER-1441](https://issues.apache.org/jira/browse/ZOOKEEPER-1441) - Some test cases are failing because Port bind issue. -* [ZOOKEEPER-2017](https://issues.apache.org/jira/browse/ZOOKEEPER-2017) - New tests for reconfig failure cases -* [ZOOKEEPER-2204](https://issues.apache.org/jira/browse/ZOOKEEPER-2204) - LearnerSnapshotThrottlerTest.testHighContentionWithTimeout fails occasionally -* [ZOOKEEPER-2415](https://issues.apache.org/jira/browse/ZOOKEEPER-2415) - SessionTest is using Thread deprecated API. -* [ZOOKEEPER-2484](https://issues.apache.org/jira/browse/ZOOKEEPER-2484) - Flaky Test: org.apache.zookeeper.test.LoadFromLogTest.testLoadFailure -* [ZOOKEEPER-2508](https://issues.apache.org/jira/browse/ZOOKEEPER-2508) - Many ZooKeeper tests are flaky because they proceed with zk operation without connecting to ZooKeeper server. -* [ZOOKEEPER-2577](https://issues.apache.org/jira/browse/ZOOKEEPER-2577) - Flaky Test: org.apache.zookeeper.server.quorum.ReconfigDuringLeaderSyncTest.testDuringLeaderSync -* [ZOOKEEPER-2656](https://issues.apache.org/jira/browse/ZOOKEEPER-2656) - Fix ServerConfigTest#testValidArguments test case failures -* [ZOOKEEPER-2664](https://issues.apache.org/jira/browse/ZOOKEEPER-2664) - ClientPortBindTest#testBindByAddress may fail due to "No such device" exception -* [ZOOKEEPER-2686](https://issues.apache.org/jira/browse/ZOOKEEPER-2686) - Flaky Test: org.apache.zookeeper.test.WatcherTest. -* [ZOOKEEPER-2716](https://issues.apache.org/jira/browse/ZOOKEEPER-2716) - Flaky Test: org.apache.zookeeper.server.SessionTrackerTest.testAddSessionAfterSessionExpiry -* [ZOOKEEPER-2718](https://issues.apache.org/jira/browse/ZOOKEEPER-2718) - org.apache.zookeeper.server.quorum.StandaloneDisabledTest fails intermittently -* [ZOOKEEPER-2742](https://issues.apache.org/jira/browse/ZOOKEEPER-2742) - Few test cases of org.apache.zookeeper.ZooKeeperTest fails in Windows -* [ZOOKEEPER-2746](https://issues.apache.org/jira/browse/ZOOKEEPER-2746) - Leader hand-off during dynamic reconfig is best effort, while test always expects it -* [ZOOKEEPER-2796](https://issues.apache.org/jira/browse/ZOOKEEPER-2796) - Test org.apache.zookeeper.ZooKeeperTest.testCreateNodeWithoutData is broken by ZOOKEEPER-2757 -* [ZOOKEEPER-2955](https://issues.apache.org/jira/browse/ZOOKEEPER-2955) - Enable Clover code coverage report -* [ZOOKEEPER-2968](https://issues.apache.org/jira/browse/ZOOKEEPER-2968) - Add C client code coverage tests -* [ZOOKEEPER-3074](https://issues.apache.org/jira/browse/ZOOKEEPER-3074) - Flaky test:org.apache.zookeeper.server.ServerStatsTest.testLatencyMetrics -* [ZOOKEEPER-3157](https://issues.apache.org/jira/browse/ZOOKEEPER-3157) - Improve FuzzySnapshotRelatedTest to avoid flaky due to issues like connection loss -* [ZOOKEEPER-3205](https://issues.apache.org/jira/browse/ZOOKEEPER-3205) - Jute - o.a.jute.BinaryInputArchive Test cases -* [ZOOKEEPER-3455](https://issues.apache.org/jira/browse/ZOOKEEPER-3455) - Java 13 build failure on trunk: UnifiedServerSocketTest.testConnectWithoutSSLToStrictServer -* [ZOOKEEPER-3480](https://issues.apache.org/jira/browse/ZOOKEEPER-3480) - Flaky test CommitProcessorMetricsTest.testConcurrentRequestProcessingInCommitProcessor -* [ZOOKEEPER-3495](https://issues.apache.org/jira/browse/ZOOKEEPER-3495) - Broken test in JDK12+: SnapshotDigestTest.testDifferentDigestVersion -* [ZOOKEEPER-3542](https://issues.apache.org/jira/browse/ZOOKEEPER-3542) - X509UtilTest#testClientRenegotiationFails is flaky on JDK8 + linux on machines with 2 cores - - ## Wish -* [ZOOKEEPER-2795](https://issues.apache.org/jira/browse/ZOOKEEPER-2795) - Change log level for "ZKShutdownHandler is not registered" error message -* [ZOOKEEPER-3073](https://issues.apache.org/jira/browse/ZOOKEEPER-3073) - fix couple of typos -* [ZOOKEEPER-3302](https://issues.apache.org/jira/browse/ZOOKEEPER-3302) - ZooKeeper C client does not compile on Fedora 29 -* [ZOOKEEPER-3421](https://issues.apache.org/jira/browse/ZOOKEEPER-3421) - Better insight into Observer connections - +* [ZOOKEEPER-3763](https://issues.apache.org/jira/browse/ZOOKEEPER-3763) - Restore ZKUtil.deleteRecursive in order to help compatibility of applications with 3.5 and 3.6 + ## Task -* [ZOOKEEPER-3733](https://issues.apache.org/jira/browse/ZOOKEEPER-3733) - Fix issues reported in 3.6.0rc3 -* [ZOOKEEPER-3734](https://issues.apache.org/jira/browse/ZOOKEEPER-3734) - upgrade jackson-databind to address CVE-2020-8840 -* [ZOOKEEPER-3715](https://issues.apache.org/jira/browse/ZOOKEEPER-3715) - fix Kerberos test failures for new JDK versions -* [ZOOKEEPER-3716](https://issues.apache.org/jira/browse/ZOOKEEPER-3716) - upgrade netty 4.1.42 to address CVE-2019-20444 CVE-2019-20445 -* [ZOOKEEPER-3708](https://issues.apache.org/jira/browse/ZOOKEEPER-3708) - Move Logging Code into Logging Guard in Learner * [ZOOKEEPER-3669](https://issues.apache.org/jira/browse/ZOOKEEPER-3669) - Use switch Statement in ClientCnxn SendThread -* [ZOOKEEPER-3704](https://issues.apache.org/jira/browse/ZOOKEEPER-3704) - upgrade maven dependency-check to 5.3.0 -* [ZOOKEEPER-925](https://issues.apache.org/jira/browse/ZOOKEEPER-925) - Consider maven site generation to replace our forrest site and documentation generation -* [ZOOKEEPER-1604](https://issues.apache.org/jira/browse/ZOOKEEPER-1604) - remove rpm/deb/... packaging -* [ZOOKEEPER-2658](https://issues.apache.org/jira/browse/ZOOKEEPER-2658) - Trunk / branch-3.5 build broken. -* [ZOOKEEPER-2709](https://issues.apache.org/jira/browse/ZOOKEEPER-2709) - Clarify documentation around "auth" ACL scheme -* [ZOOKEEPER-3002](https://issues.apache.org/jira/browse/ZOOKEEPER-3002) - Upgrade branches 3.5 and trunk to Java 1.8 -* [ZOOKEEPER-3017](https://issues.apache.org/jira/browse/ZOOKEEPER-3017) - Link libm in CMake on FreeBSD -* [ZOOKEEPER-3061](https://issues.apache.org/jira/browse/ZOOKEEPER-3061) - add more details to 'Unhandled scenario for peer' log.warn message -* [ZOOKEEPER-3062](https://issues.apache.org/jira/browse/ZOOKEEPER-3062) - introduce fsync.warningthresholdms constant for FileTxnLog LOG.warn message -* [ZOOKEEPER-3067](https://issues.apache.org/jira/browse/ZOOKEEPER-3067) - Optionally suppress client environment logging. -* [ZOOKEEPER-3087](https://issues.apache.org/jira/browse/ZOOKEEPER-3087) - Fix findbug warning introduced by ZOOKEEPER-3084. -* [ZOOKEEPER-3120](https://issues.apache.org/jira/browse/ZOOKEEPER-3120) - add NetBeans nbproject directory to .gitignore -* [ZOOKEEPER-3136](https://issues.apache.org/jira/browse/ZOOKEEPER-3136) - Reduce log in ClientBase in case of ConnectException -* [ZOOKEEPER-3197](https://issues.apache.org/jira/browse/ZOOKEEPER-3197) - Improve documentation in ZooKeeperServer.superSecret -* [ZOOKEEPER-3230](https://issues.apache.org/jira/browse/ZOOKEEPER-3230) - Add Apache NetBeans Maven project files to .gitignore -* [ZOOKEEPER-3362](https://issues.apache.org/jira/browse/ZOOKEEPER-3362) - Create a simple checkstyle file -* [ZOOKEEPER-3406](https://issues.apache.org/jira/browse/ZOOKEEPER-3406) - Update website for new mailing lists -* [ZOOKEEPER-3407](https://issues.apache.org/jira/browse/ZOOKEEPER-3407) - Update POM file with new information -* [ZOOKEEPER-3431](https://issues.apache.org/jira/browse/ZOOKEEPER-3431) - Enable BookKeeper checkstyle configuration -* [ZOOKEEPER-3441](https://issues.apache.org/jira/browse/ZOOKEEPER-3441) - OWASP is flagging jackson-databind-2.9.9.jar for CVE-2019-12814 -* [ZOOKEEPER-3545](https://issues.apache.org/jira/browse/ZOOKEEPER-3545) - Fix LICENSE files for netty dependency -* [ZOOKEEPER-3668](https://issues.apache.org/jira/browse/ZOOKEEPER-3668) - Clean up release package for 3.6.0 - -## Sub-task - -* [ZOOKEEPER-169](https://issues.apache.org/jira/browse/ZOOKEEPER-169) - Content needed: "Connecting to ZooKeeper" -* [ZOOKEEPER-236](https://issues.apache.org/jira/browse/ZOOKEEPER-236) - SSL Support for Atomic Broadcast protocol -* [ZOOKEEPER-1626](https://issues.apache.org/jira/browse/ZOOKEEPER-1626) - Zookeeper C client should be tolerant of clock adjustments -* [ZOOKEEPER-1660](https://issues.apache.org/jira/browse/ZOOKEEPER-1660) - Add documentation for dynamic reconfiguration -* [ZOOKEEPER-1872](https://issues.apache.org/jira/browse/ZOOKEEPER-1872) - QuorumPeer is not shutdown in few cases -* [ZOOKEEPER-2047](https://issues.apache.org/jira/browse/ZOOKEEPER-2047) - testTruncationNullLog fails on windows -* [ZOOKEEPER-2069](https://issues.apache.org/jira/browse/ZOOKEEPER-2069) - Netty Support for ClientCnxnSocket -* [ZOOKEEPER-2080](https://issues.apache.org/jira/browse/ZOOKEEPER-2080) - Fix deadlock in dynamic reconfiguration -* [ZOOKEEPER-2119](https://issues.apache.org/jira/browse/ZOOKEEPER-2119) - Netty client docs -* [ZOOKEEPER-2122](https://issues.apache.org/jira/browse/ZOOKEEPER-2122) - Impplement SSL support in the Zookeeper C client library -* [ZOOKEEPER-2123](https://issues.apache.org/jira/browse/ZOOKEEPER-2123) - Provide implementation of X509 AuthenticationProvider -* [ZOOKEEPER-2125](https://issues.apache.org/jira/browse/ZOOKEEPER-2125) - SSL on Netty client-server communication -* [ZOOKEEPER-2134](https://issues.apache.org/jira/browse/ZOOKEEPER-2134) - AsyncHammerTest.testHammer fails intermittently -* [ZOOKEEPER-2137](https://issues.apache.org/jira/browse/ZOOKEEPER-2137) - Make testPortChange() less flaky -* [ZOOKEEPER-2152](https://issues.apache.org/jira/browse/ZOOKEEPER-2152) - Intermittent failure in TestReconfig.cc -* [ZOOKEEPER-2153](https://issues.apache.org/jira/browse/ZOOKEEPER-2153) - X509 Authentication Documentation -* [ZOOKEEPER-2168](https://issues.apache.org/jira/browse/ZOOKEEPER-2168) - Add C APIs for new createContainer Methods -* [ZOOKEEPER-2396](https://issues.apache.org/jira/browse/ZOOKEEPER-2396) - Login object in ZooKeeperSaslClient is static -* [ZOOKEEPER-2608](https://issues.apache.org/jira/browse/ZOOKEEPER-2608) - Create CLI option for TTL ephemerals -* [ZOOKEEPER-2609](https://issues.apache.org/jira/browse/ZOOKEEPER-2609) - Add TTL Node APIs to C client -* [ZOOKEEPER-2692](https://issues.apache.org/jira/browse/ZOOKEEPER-2692) - Fix race condition in testWatchAutoResetWithPending -* [ZOOKEEPER-2707](https://issues.apache.org/jira/browse/ZOOKEEPER-2707) - Fix "Unexpected bean exists!" issue in WatcherTests -* [ZOOKEEPER-2750](https://issues.apache.org/jira/browse/ZOOKEEPER-2750) - Document SSL Support for Atomic Broadcast protocol -* [ZOOKEEPER-2753](https://issues.apache.org/jira/browse/ZOOKEEPER-2753) - Introduce a python script for generating flaky test report -* [ZOOKEEPER-2754](https://issues.apache.org/jira/browse/ZOOKEEPER-2754) - Set up Apache Jenkins job that runs the flaky test analyzer script. -* [ZOOKEEPER-2935](https://issues.apache.org/jira/browse/ZOOKEEPER-2935) - [QP MutualAuth]: Port ZOOKEEPER-1045 implementation from branch-3.5 to trunk -* [ZOOKEEPER-2939](https://issues.apache.org/jira/browse/ZOOKEEPER-2939) - Deal with maxbuffer as it relates to proposals -* [ZOOKEEPER-2940](https://issues.apache.org/jira/browse/ZOOKEEPER-2940) - Deal with maxbuffer as it relates to large requests from clients -* [ZOOKEEPER-3022](https://issues.apache.org/jira/browse/ZOOKEEPER-3022) - Step 1.1 - Create docs and it maven structure -* [ZOOKEEPER-3028](https://issues.apache.org/jira/browse/ZOOKEEPER-3028) - Create assembly in pom.xml -* [ZOOKEEPER-3029](https://issues.apache.org/jira/browse/ZOOKEEPER-3029) - Create pom files for jute, server and client -* [ZOOKEEPER-3030](https://issues.apache.org/jira/browse/ZOOKEEPER-3030) - Step 1.3 - Create zk-contrib maven structure -* [ZOOKEEPER-3031](https://issues.apache.org/jira/browse/ZOOKEEPER-3031) - Step 1.4 - Create zk-client maven structure -* [ZOOKEEPER-3032](https://issues.apache.org/jira/browse/ZOOKEEPER-3032) - Step 1.6 - Create zk-server maven structure -* [ZOOKEEPER-3033](https://issues.apache.org/jira/browse/ZOOKEEPER-3033) - Step 1.2 - Create zk-recipes maven structure -* [ZOOKEEPER-3046](https://issues.apache.org/jira/browse/ZOOKEEPER-3046) - testManyChildWatchersAutoReset is flaky -* [ZOOKEEPER-3080](https://issues.apache.org/jira/browse/ZOOKEEPER-3080) - Step 1.5 - Separate jute structure -* [ZOOKEEPER-3103](https://issues.apache.org/jira/browse/ZOOKEEPER-3103) - Pluggable metrics system for ZooKeeper - MetricsProvider API definition -* [ZOOKEEPER-3123](https://issues.apache.org/jira/browse/ZOOKEEPER-3123) - MetricsProvider Lifecycle in ZooKeeper Server -* [ZOOKEEPER-3143](https://issues.apache.org/jira/browse/ZOOKEEPER-3143) - Pluggable metrics system for ZooKeeper - Data Collection on Server -* [ZOOKEEPER-3150](https://issues.apache.org/jira/browse/ZOOKEEPER-3150) - Data integrity check when loading snapshot/txns from disk -* [ZOOKEEPER-3153](https://issues.apache.org/jira/browse/ZOOKEEPER-3153) - Create MarkDown files and build process for them -* [ZOOKEEPER-3154](https://issues.apache.org/jira/browse/ZOOKEEPER-3154) - Update release process to use the MarkDown solution -* [ZOOKEEPER-3155](https://issues.apache.org/jira/browse/ZOOKEEPER-3155) - Remove Forrest XMLs and their build process from the project -* [ZOOKEEPER-3171](https://issues.apache.org/jira/browse/ZOOKEEPER-3171) - Create pom.xml for recipes and contrib -* [ZOOKEEPER-3172](https://issues.apache.org/jira/browse/ZOOKEEPER-3172) - Quorum TLS - fix port unification to allow rolling upgrades -* [ZOOKEEPER-3173](https://issues.apache.org/jira/browse/ZOOKEEPER-3173) - Quorum TLS - support PEM trust/key stores -* [ZOOKEEPER-3174](https://issues.apache.org/jira/browse/ZOOKEEPER-3174) - Quorum TLS - support reloading trust/key store -* [ZOOKEEPER-3175](https://issues.apache.org/jira/browse/ZOOKEEPER-3175) - Quorum TLS - test improvements -* [ZOOKEEPER-3176](https://issues.apache.org/jira/browse/ZOOKEEPER-3176) - Quorum TLS - add SSL config options -* [ZOOKEEPER-3193](https://issues.apache.org/jira/browse/ZOOKEEPER-3193) - Flaky: org.apache.zookeeper.test.SaslAuthFailNotifyTest -* [ZOOKEEPER-3194](https://issues.apache.org/jira/browse/ZOOKEEPER-3194) - Quorum TLS - fix copy/paste bug in ZKTrustManager -* [ZOOKEEPER-3202](https://issues.apache.org/jira/browse/ZOOKEEPER-3202) - Flaky test: org.apache.zookeeper.test.ClientSSLTest.testClientServerSSL -* [ZOOKEEPER-3222](https://issues.apache.org/jira/browse/ZOOKEEPER-3222) - Flaky: multiple intermittent segfaults in C++ tests -* [ZOOKEEPER-3223](https://issues.apache.org/jira/browse/ZOOKEEPER-3223) - Configure Spotbugs -* [ZOOKEEPER-3224](https://issues.apache.org/jira/browse/ZOOKEEPER-3224) - CI integration with maven -* [ZOOKEEPER-3225](https://issues.apache.org/jira/browse/ZOOKEEPER-3225) - Create code coverage analysis with maven build -* [ZOOKEEPER-3226](https://issues.apache.org/jira/browse/ZOOKEEPER-3226) - Activate C Client with a profile, disabled by default -* [ZOOKEEPER-3229](https://issues.apache.org/jira/browse/ZOOKEEPER-3229) - [TLS] add AES-256 ciphers to default cipher list -* [ZOOKEEPER-3233](https://issues.apache.org/jira/browse/ZOOKEEPER-3233) - Run github pre-commit hook tests on 4 threads -* [ZOOKEEPER-3251](https://issues.apache.org/jira/browse/ZOOKEEPER-3251) - Add new server metric types: percentile counter and counter set -* [ZOOKEEPER-3256](https://issues.apache.org/jira/browse/ZOOKEEPER-3256) - Enable OWASP checks to Maven build -* [ZOOKEEPER-3267](https://issues.apache.org/jira/browse/ZOOKEEPER-3267) - Add watcher metrics -* [ZOOKEEPER-3268](https://issues.apache.org/jira/browse/ZOOKEEPER-3268) - Add commit processor metrics -* [ZOOKEEPER-3275](https://issues.apache.org/jira/browse/ZOOKEEPER-3275) - Fix release targets: package, tar, mvn-deploy -* [ZOOKEEPER-3285](https://issues.apache.org/jira/browse/ZOOKEEPER-3285) - Move assembly into its own sub-module -* [ZOOKEEPER-3288](https://issues.apache.org/jira/browse/ZOOKEEPER-3288) - add a new doc:ZookeeperCLI.md -* [ZOOKEEPER-3305](https://issues.apache.org/jira/browse/ZOOKEEPER-3305) - Add Quorum Packet metrics -* [ZOOKEEPER-3309](https://issues.apache.org/jira/browse/ZOOKEEPER-3309) - Add sync processor metrics -* [ZOOKEEPER-3310](https://issues.apache.org/jira/browse/ZOOKEEPER-3310) - Add metrics for prep processor -* [ZOOKEEPER-3319](https://issues.apache.org/jira/browse/ZOOKEEPER-3319) - Add metrics for follower and observer -* [ZOOKEEPER-3321](https://issues.apache.org/jira/browse/ZOOKEEPER-3321) - Add metrics for Leader -* [ZOOKEEPER-3323](https://issues.apache.org/jira/browse/ZOOKEEPER-3323) - Add TxnSnapLog metrics -* [ZOOKEEPER-3324](https://issues.apache.org/jira/browse/ZOOKEEPER-3324) - Add read/write metrics for top level znodes -* [ZOOKEEPER-3326](https://issues.apache.org/jira/browse/ZOOKEEPER-3326) - Add session/connection related metrics -* [ZOOKEEPER-3327](https://issues.apache.org/jira/browse/ZOOKEEPER-3327) - Add unrecoverable error count -* [ZOOKEEPER-3366](https://issues.apache.org/jira/browse/ZOOKEEPER-3366) - Pluggable metrics system for ZooKeeper - move remaining metrics to MetricsProvider -* [ZOOKEEPER-3379](https://issues.apache.org/jira/browse/ZOOKEEPER-3379) - De-flaky test in Quorum Packet Metrics -* [ZOOKEEPER-3383](https://issues.apache.org/jira/browse/ZOOKEEPER-3383) - Improve prep processor metric accuracy and de-flaky unit test -* [ZOOKEEPER-3401](https://issues.apache.org/jira/browse/ZOOKEEPER-3401) - Fix metric PROPOSAL_ACK_CREATION_LATENCY -* [ZOOKEEPER-3409](https://issues.apache.org/jira/browse/ZOOKEEPER-3409) - Compile Java code with -Xdoclint -* [ZOOKEEPER-3443](https://issues.apache.org/jira/browse/ZOOKEEPER-3443) - Add support for PKCS12 trust/key stores -* [ZOOKEEPER-3465](https://issues.apache.org/jira/browse/ZOOKEEPER-3465) - Introduce BookKeeper checkstyle configuration -* [ZOOKEEPER-3470](https://issues.apache.org/jira/browse/ZOOKEEPER-3470) - Flaky test: LearnerMetricsTest.testLearnerMetricsTest() -* [ZOOKEEPER-3474](https://issues.apache.org/jira/browse/ZOOKEEPER-3474) - Enable BookKeeper checkstyle configuration on zookeeper-promethus -* [ZOOKEEPER-3475](https://issues.apache.org/jira/browse/ZOOKEEPER-3475) - Enable BookKeeper checkstyle configuration on zookeeper-server -* [ZOOKEEPER-3512](https://issues.apache.org/jira/browse/ZOOKEEPER-3512) - Real time data integrity check during broadcast time -* [ZOOKEEPER-3517](https://issues.apache.org/jira/browse/ZOOKEEPER-3517) - Turn on BookKeeper checkstyle configuration at project level -* [ZOOKEEPER-3528](https://issues.apache.org/jira/browse/ZOOKEEPER-3528) - Revisit AsyncCallback javadoc -* [ZOOKEEPER-3529](https://issues.apache.org/jira/browse/ZOOKEEPER-3529) - add a new doc: zookeeperUseCases.md - - - - - - - - +* [ZOOKEEPER-3677](https://issues.apache.org/jira/browse/ZOOKEEPER-3677) - owasp checker failing for - CVE-2019-17571 Apache Log4j 1.2 deserialization of untrusted data in SocketServer +* [ZOOKEEPER-3751](https://issues.apache.org/jira/browse/ZOOKEEPER-3751) - upgrade jackson-databind to 2.10 from 2.9 From 754cf015f3bc45c39e1ad66e9a535ec280d972b8 Mon Sep 17 00:00:00 2001 From: Vladislav Tyulbashev Date: Sat, 11 Apr 2020 06:19:04 +0000 Subject: [PATCH 061/118] 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 (cherry picked from commit 726f6843eba75bb3673284b52d8ca3801c5431d3) Signed-off-by: Mate Szalay-Beko --- .../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 e2aef19f4be4cd8015b66dbf63bb860efe4745f4 Mon Sep 17 00:00:00 2001 From: Michael Han Date: Sun, 12 Apr 2020 11:14:45 +0200 Subject: [PATCH 062/118] 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 (cherry picked from commit 4d32f6cf39f76d606b436bd4c04a8d1bc9c60148) Signed-off-by: Enrico Olivelli --- .../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 1a2d9a7e06a..2594b156b43 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 @@ -1419,7 +1419,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 37e3032ceab81e53a783e80ee995dba1fe815899 Mon Sep 17 00:00:00 2001 From: Enrico Olivelli Date: Sun, 12 Apr 2020 14:20:37 +0200 Subject: [PATCH 063/118] Add ZOOKEEPER-3793 to release notes --- zookeeper-docs/src/main/resources/markdown/releasenotes.md | 1 + 1 file changed, 1 insertion(+) diff --git a/zookeeper-docs/src/main/resources/markdown/releasenotes.md b/zookeeper-docs/src/main/resources/markdown/releasenotes.md index 66750347807..149bebcc673 100644 --- a/zookeeper-docs/src/main/resources/markdown/releasenotes.md +++ b/zookeeper-docs/src/main/resources/markdown/releasenotes.md @@ -31,6 +31,7 @@ limitations under the License. * [ZOOKEEPER-3776](https://issues.apache.org/jira/browse/ZOOKEEPER-3776) - Cluster stuck not forming up quorum * [ZOOKEEPER-3778](https://issues.apache.org/jira/browse/ZOOKEEPER-3778) - Cannot upgrade from 3.5.7 to 3.6.0 due to multiAddress.reachabilityCheckEnabled * [ZOOKEEPER-3780](https://issues.apache.org/jira/browse/ZOOKEEPER-3780) - restore Version.getRevision() to be backward compatible +* [ZOOKEEPER-3793](https://issues.apache.org/jira/browse/ZOOKEEPER-3793) - Request throttling is broken when RequestThrottler is disabled or configured incorrectly. ## New Feature From 8a8ec3d55c9aac4266a2e60000a91df8606525cb Mon Sep 17 00:00:00 2001 From: Patrick Hunt Date: Wed, 15 Apr 2020 11:44:17 +0200 Subject: [PATCH 064/118] 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 (cherry picked from commit 8a1e0b3643d85ffa362207880095e08f1dbc47da) Signed-off-by: Enrico Olivelli --- 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 2e66d88652b..39e01b0450e 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 4d1ddefe044e5d69783c4cbbc3d25772a152ec72 Mon Sep 17 00:00:00 2001 From: Christopher Tubbs Date: Wed, 15 Apr 2020 16:11:41 +0200 Subject: [PATCH 065/118] ZOOKEEPER-3738: Fix VerGen when mvngit.commit.id is null Workaround for exec-maven-plugin treating an empty `${mvngit.commit.id}` as null and passing an incorrect number of arguments to VerGen. This change allows the revision to be omitted in the command-line args to VerGen. This allows the mavanagaiata-maven-plugin to provide the git commit id, when available, to the VerGen command. This change is superseded in ZooKeeper 3.7.0 and later by ZOOKEEPER-3786 (https://github.com/apache/zookeeper/pull/1310), which simplifies generating the version information and removes VerGen. Author: Christopher Tubbs Reviewers: Enrico Olivelli , Mate Szalay-Beko Closes #1321 from ctubbsii/fix-vergen-for-36-mavanagaiata --- .../java/org/apache/zookeeper/version/util/VerGen.java | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) 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 9bb699f090d..6ae97e1566c 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 @@ -35,7 +35,7 @@ public class VerGen { static void printUsage() { System.out.print("Usage:\tjava -cp org.apache.zookeeper." - + "version.util.VerGen maj.min.micro[-qualifier] rev buildDate outputDirectory"); + + "version.util.VerGen maj.min.micro[-qualifier] [rev] buildDate outputDirectory"); System.exit(ExitCode.UNEXPECTED_ERROR.getValue()); } @@ -177,9 +177,12 @@ public static Version parseVersionString(String input) { * */ public static void main(String[] args) { - if (args.length != 4) { + if (args.length != 3 && args.length != 4) { printUsage(); } + if (args.length == 3) { + args = new String[]{args[0], null, args[1], args[2]}; + } try { Version version = parseVersionString(args[0]); if (version == null) { From fbe16c929347bcefe2f393098d9f8ca972a48ae6 Mon Sep 17 00:00:00 2001 From: Enrico Olivelli Date: Wed, 15 Apr 2020 16:15:47 +0200 Subject: [PATCH 066/118] add ZOOKEEPER-3794 to release notes --- zookeeper-docs/src/main/resources/markdown/releasenotes.md | 1 + 1 file changed, 1 insertion(+) diff --git a/zookeeper-docs/src/main/resources/markdown/releasenotes.md b/zookeeper-docs/src/main/resources/markdown/releasenotes.md index 149bebcc673..89a5d8ccc84 100644 --- a/zookeeper-docs/src/main/resources/markdown/releasenotes.md +++ b/zookeeper-docs/src/main/resources/markdown/releasenotes.md @@ -57,6 +57,7 @@ limitations under the License. * [ZOOKEEPER-3669](https://issues.apache.org/jira/browse/ZOOKEEPER-3669) - Use switch Statement in ClientCnxn SendThread * [ZOOKEEPER-3677](https://issues.apache.org/jira/browse/ZOOKEEPER-3677) - owasp checker failing for - CVE-2019-17571 Apache Log4j 1.2 deserialization of untrusted data in SocketServer * [ZOOKEEPER-3751](https://issues.apache.org/jira/browse/ZOOKEEPER-3751) - upgrade jackson-databind to 2.10 from 2.9 +* [ZOOKEEPER-3794](https://issues.apache.org/jira/browse/ZOOKEEPER-3794) - upgrade netty to address CVE-2020-11612 From d7b78e023af7d8dcf2530022da8565360fe64221 Mon Sep 17 00:00:00 2001 From: Enrico Olivelli Date: Wed, 15 Apr 2020 16:24:11 +0200 Subject: [PATCH 067/118] Prepared 3.6.1 --- pom.xml | 4 ++-- zookeeper-assembly/pom.xml | 2 +- zookeeper-client/pom.xml | 2 +- zookeeper-client/zookeeper-client-c/CMakeLists.txt | 2 +- zookeeper-client/zookeeper-client-c/configure.ac | 2 +- zookeeper-client/zookeeper-client-c/pom.xml | 2 +- zookeeper-contrib/pom.xml | 2 +- zookeeper-contrib/zookeeper-contrib-loggraph/pom.xml | 2 +- zookeeper-contrib/zookeeper-contrib-rest/pom.xml | 2 +- zookeeper-contrib/zookeeper-contrib-zooinspector/pom.xml | 2 +- zookeeper-docs/pom.xml | 2 +- zookeeper-jute/pom.xml | 2 +- zookeeper-metrics-providers/pom.xml | 2 +- .../zookeeper-prometheus-metrics/pom.xml | 2 +- zookeeper-recipes/pom.xml | 2 +- zookeeper-recipes/zookeeper-recipes-election/pom.xml | 2 +- zookeeper-recipes/zookeeper-recipes-lock/pom.xml | 2 +- zookeeper-recipes/zookeeper-recipes-queue/pom.xml | 2 +- zookeeper-server/pom.xml | 2 +- 19 files changed, 20 insertions(+), 20 deletions(-) diff --git a/pom.xml b/pom.xml index 39e01b0450e..8dcf57709a8 100755 --- a/pom.xml +++ b/pom.xml @@ -31,7 +31,7 @@ parent pom - 3.6.1-SNAPSHOT + 3.6.1 Apache ZooKeeper ZooKeeper is a centralized service for maintaining configuration information, naming, @@ -70,7 +70,7 @@ scm:git:https://gitbox.apache.org/repos/asf/zookeeper.git scm:git:https://gitbox.apache.org/repos/asf/zookeeper.git https://gitbox.apache.org/repos/asf/zookeeper.git - branch-3.6 + release-3.6.1-0 JIRA diff --git a/zookeeper-assembly/pom.xml b/zookeeper-assembly/pom.xml index d9acf7a8684..d49d6c85266 100755 --- a/zookeeper-assembly/pom.xml +++ b/zookeeper-assembly/pom.xml @@ -23,7 +23,7 @@ org.apache.zookeeper parent - 3.6.1-SNAPSHOT + 3.6.1 .. diff --git a/zookeeper-client/pom.xml b/zookeeper-client/pom.xml index 1ff2af84469..09aa7f86287 100755 --- a/zookeeper-client/pom.xml +++ b/zookeeper-client/pom.xml @@ -23,7 +23,7 @@ org.apache.zookeeper parent - 3.6.1-SNAPSHOT + 3.6.1 .. diff --git a/zookeeper-client/zookeeper-client-c/CMakeLists.txt b/zookeeper-client/zookeeper-client-c/CMakeLists.txt index 06bbf983912..12773fc9271 100644 --- a/zookeeper-client/zookeeper-client-c/CMakeLists.txt +++ b/zookeeper-client/zookeeper-client-c/CMakeLists.txt @@ -16,7 +16,7 @@ cmake_minimum_required(VERSION 3.5) -project(zookeeper VERSION 3.6.0) +project(zookeeper VERSION 3.6.1) set(email user@zookeeper.apache.org) set(description "zookeeper C client") diff --git a/zookeeper-client/zookeeper-client-c/configure.ac b/zookeeper-client/zookeeper-client-c/configure.ac index 96ddaeca03e..718e512c7a1 100644 --- a/zookeeper-client/zookeeper-client-c/configure.ac +++ b/zookeeper-client/zookeeper-client-c/configure.ac @@ -3,7 +3,7 @@ AC_PREREQ(2.59) -AC_INIT([zookeeper C client],3.6.0,[user@zookeeper.apache.org],[zookeeper]) +AC_INIT([zookeeper C client],3.6.1,[user@zookeeper.apache.org],[zookeeper]) AC_CONFIG_SRCDIR([src/zookeeper.c]) # Save initial CFLAGS and CXXFLAGS values before AC_PROG_CC and AC_PROG_CXX diff --git a/zookeeper-client/zookeeper-client-c/pom.xml b/zookeeper-client/zookeeper-client-c/pom.xml index 3fc093052a0..642b640b566 100755 --- a/zookeeper-client/zookeeper-client-c/pom.xml +++ b/zookeeper-client/zookeeper-client-c/pom.xml @@ -23,7 +23,7 @@ org.apache.zookeeper zookeeper-client - 3.6.1-SNAPSHOT + 3.6.1 .. diff --git a/zookeeper-contrib/pom.xml b/zookeeper-contrib/pom.xml index c26849fea3d..fa246c52e0a 100755 --- a/zookeeper-contrib/pom.xml +++ b/zookeeper-contrib/pom.xml @@ -24,7 +24,7 @@ org.apache.zookeeper parent - 3.6.1-SNAPSHOT + 3.6.1 org.apache.zookeeper diff --git a/zookeeper-contrib/zookeeper-contrib-loggraph/pom.xml b/zookeeper-contrib/zookeeper-contrib-loggraph/pom.xml index 5eaca1c7bb4..16b402361d3 100755 --- a/zookeeper-contrib/zookeeper-contrib-loggraph/pom.xml +++ b/zookeeper-contrib/zookeeper-contrib-loggraph/pom.xml @@ -23,7 +23,7 @@ org.apache.zookeeper zookeeper-contrib - 3.6.1-SNAPSHOT + 3.6.1 org.apache.zookeeper diff --git a/zookeeper-contrib/zookeeper-contrib-rest/pom.xml b/zookeeper-contrib/zookeeper-contrib-rest/pom.xml index 3339a0008ea..90dbfd7fcb9 100755 --- a/zookeeper-contrib/zookeeper-contrib-rest/pom.xml +++ b/zookeeper-contrib/zookeeper-contrib-rest/pom.xml @@ -23,7 +23,7 @@ org.apache.zookeeper zookeeper-contrib - 3.6.1-SNAPSHOT + 3.6.1 org.apache.zookeeper diff --git a/zookeeper-contrib/zookeeper-contrib-zooinspector/pom.xml b/zookeeper-contrib/zookeeper-contrib-zooinspector/pom.xml index da9e14854cc..850ad13a2ad 100755 --- a/zookeeper-contrib/zookeeper-contrib-zooinspector/pom.xml +++ b/zookeeper-contrib/zookeeper-contrib-zooinspector/pom.xml @@ -23,7 +23,7 @@ org.apache.zookeeper zookeeper-contrib - 3.6.1-SNAPSHOT + 3.6.1 org.apache.zookeeper diff --git a/zookeeper-docs/pom.xml b/zookeeper-docs/pom.xml index 35cb14650db..c7c4f4d56bb 100644 --- a/zookeeper-docs/pom.xml +++ b/zookeeper-docs/pom.xml @@ -23,7 +23,7 @@ org.apache.zookeeper parent - 3.6.1-SNAPSHOT + 3.6.1 .. diff --git a/zookeeper-jute/pom.xml b/zookeeper-jute/pom.xml index bf570f48cef..9cfd21cc657 100755 --- a/zookeeper-jute/pom.xml +++ b/zookeeper-jute/pom.xml @@ -23,7 +23,7 @@ org.apache.zookeeper parent - 3.6.1-SNAPSHOT + 3.6.1 .. diff --git a/zookeeper-metrics-providers/pom.xml b/zookeeper-metrics-providers/pom.xml index f458614a42c..9f6871b792d 100755 --- a/zookeeper-metrics-providers/pom.xml +++ b/zookeeper-metrics-providers/pom.xml @@ -23,7 +23,7 @@ org.apache.zookeeper parent - 3.6.1-SNAPSHOT + 3.6.1 .. diff --git a/zookeeper-metrics-providers/zookeeper-prometheus-metrics/pom.xml b/zookeeper-metrics-providers/zookeeper-prometheus-metrics/pom.xml index 65950ffa429..4f2edc90ca4 100755 --- a/zookeeper-metrics-providers/zookeeper-prometheus-metrics/pom.xml +++ b/zookeeper-metrics-providers/zookeeper-prometheus-metrics/pom.xml @@ -23,7 +23,7 @@ org.apache.zookeeper zookeeper-metrics-providers - 3.6.1-SNAPSHOT + 3.6.1 .. diff --git a/zookeeper-recipes/pom.xml b/zookeeper-recipes/pom.xml index 23fe3c405b4..c3ab410f727 100755 --- a/zookeeper-recipes/pom.xml +++ b/zookeeper-recipes/pom.xml @@ -24,7 +24,7 @@ org.apache.zookeeper parent - 3.6.1-SNAPSHOT + 3.6.1 org.apache.zookeeper diff --git a/zookeeper-recipes/zookeeper-recipes-election/pom.xml b/zookeeper-recipes/zookeeper-recipes-election/pom.xml index eb7a00bcc95..b238cccf435 100755 --- a/zookeeper-recipes/zookeeper-recipes-election/pom.xml +++ b/zookeeper-recipes/zookeeper-recipes-election/pom.xml @@ -23,7 +23,7 @@ org.apache.zookeeper zookeeper-recipes - 3.6.1-SNAPSHOT + 3.6.1 org.apache.zookeeper diff --git a/zookeeper-recipes/zookeeper-recipes-lock/pom.xml b/zookeeper-recipes/zookeeper-recipes-lock/pom.xml index f4bdc2486da..97d88db65a7 100755 --- a/zookeeper-recipes/zookeeper-recipes-lock/pom.xml +++ b/zookeeper-recipes/zookeeper-recipes-lock/pom.xml @@ -23,7 +23,7 @@ org.apache.zookeeper zookeeper-recipes - 3.6.1-SNAPSHOT + 3.6.1 org.apache.zookeeper diff --git a/zookeeper-recipes/zookeeper-recipes-queue/pom.xml b/zookeeper-recipes/zookeeper-recipes-queue/pom.xml index b4705d77e1b..c2f938d959b 100755 --- a/zookeeper-recipes/zookeeper-recipes-queue/pom.xml +++ b/zookeeper-recipes/zookeeper-recipes-queue/pom.xml @@ -23,7 +23,7 @@ org.apache.zookeeper zookeeper-recipes - 3.6.1-SNAPSHOT + 3.6.1 org.apache.zookeeper diff --git a/zookeeper-server/pom.xml b/zookeeper-server/pom.xml index b2c4dc1a29d..390d40e0a88 100755 --- a/zookeeper-server/pom.xml +++ b/zookeeper-server/pom.xml @@ -23,7 +23,7 @@ org.apache.zookeeper parent - 3.6.1-SNAPSHOT + 3.6.1 .. From 1964b1b015cb8224ce663f07cbf6fad5c566a607 Mon Sep 17 00:00:00 2001 From: Enrico Olivelli Date: Wed, 15 Apr 2020 16:25:55 +0200 Subject: [PATCH 068/118] Prepared 3.6.2-SNAPSHOT --- pom.xml | 4 ++-- zookeeper-assembly/pom.xml | 2 +- zookeeper-client/pom.xml | 2 +- zookeeper-client/zookeeper-client-c/CMakeLists.txt | 2 +- zookeeper-client/zookeeper-client-c/configure.ac | 2 +- zookeeper-client/zookeeper-client-c/pom.xml | 2 +- zookeeper-contrib/pom.xml | 2 +- zookeeper-contrib/zookeeper-contrib-loggraph/pom.xml | 2 +- zookeeper-contrib/zookeeper-contrib-rest/pom.xml | 2 +- zookeeper-contrib/zookeeper-contrib-zooinspector/pom.xml | 2 +- zookeeper-docs/pom.xml | 2 +- zookeeper-jute/pom.xml | 2 +- zookeeper-metrics-providers/pom.xml | 2 +- .../zookeeper-prometheus-metrics/pom.xml | 2 +- zookeeper-recipes/pom.xml | 2 +- zookeeper-recipes/zookeeper-recipes-election/pom.xml | 2 +- zookeeper-recipes/zookeeper-recipes-lock/pom.xml | 2 +- zookeeper-recipes/zookeeper-recipes-queue/pom.xml | 2 +- zookeeper-server/pom.xml | 2 +- 19 files changed, 20 insertions(+), 20 deletions(-) diff --git a/pom.xml b/pom.xml index 8dcf57709a8..a1ceeb08ba1 100755 --- a/pom.xml +++ b/pom.xml @@ -31,7 +31,7 @@ parent pom - 3.6.1 + 3.6.2-SNAPSHOT Apache ZooKeeper ZooKeeper is a centralized service for maintaining configuration information, naming, @@ -70,7 +70,7 @@ scm:git:https://gitbox.apache.org/repos/asf/zookeeper.git scm:git:https://gitbox.apache.org/repos/asf/zookeeper.git https://gitbox.apache.org/repos/asf/zookeeper.git - release-3.6.1-0 + branch-3.6 JIRA diff --git a/zookeeper-assembly/pom.xml b/zookeeper-assembly/pom.xml index d49d6c85266..4427aa8440a 100755 --- a/zookeeper-assembly/pom.xml +++ b/zookeeper-assembly/pom.xml @@ -23,7 +23,7 @@ org.apache.zookeeper parent - 3.6.1 + 3.6.2-SNAPSHOT .. diff --git a/zookeeper-client/pom.xml b/zookeeper-client/pom.xml index 09aa7f86287..3e1edd35c5a 100755 --- a/zookeeper-client/pom.xml +++ b/zookeeper-client/pom.xml @@ -23,7 +23,7 @@ org.apache.zookeeper parent - 3.6.1 + 3.6.2-SNAPSHOT .. diff --git a/zookeeper-client/zookeeper-client-c/CMakeLists.txt b/zookeeper-client/zookeeper-client-c/CMakeLists.txt index 12773fc9271..1350daa36a2 100644 --- a/zookeeper-client/zookeeper-client-c/CMakeLists.txt +++ b/zookeeper-client/zookeeper-client-c/CMakeLists.txt @@ -16,7 +16,7 @@ cmake_minimum_required(VERSION 3.5) -project(zookeeper VERSION 3.6.1) +project(zookeeper VERSION 3.6.2) set(email user@zookeeper.apache.org) set(description "zookeeper C client") diff --git a/zookeeper-client/zookeeper-client-c/configure.ac b/zookeeper-client/zookeeper-client-c/configure.ac index 718e512c7a1..0aec9569ff4 100644 --- a/zookeeper-client/zookeeper-client-c/configure.ac +++ b/zookeeper-client/zookeeper-client-c/configure.ac @@ -3,7 +3,7 @@ AC_PREREQ(2.59) -AC_INIT([zookeeper C client],3.6.1,[user@zookeeper.apache.org],[zookeeper]) +AC_INIT([zookeeper C client],3.6.2,[user@zookeeper.apache.org],[zookeeper]) AC_CONFIG_SRCDIR([src/zookeeper.c]) # Save initial CFLAGS and CXXFLAGS values before AC_PROG_CC and AC_PROG_CXX diff --git a/zookeeper-client/zookeeper-client-c/pom.xml b/zookeeper-client/zookeeper-client-c/pom.xml index 642b640b566..4a56dbdca75 100755 --- a/zookeeper-client/zookeeper-client-c/pom.xml +++ b/zookeeper-client/zookeeper-client-c/pom.xml @@ -23,7 +23,7 @@ org.apache.zookeeper zookeeper-client - 3.6.1 + 3.6.2-SNAPSHOT .. diff --git a/zookeeper-contrib/pom.xml b/zookeeper-contrib/pom.xml index fa246c52e0a..73879217990 100755 --- a/zookeeper-contrib/pom.xml +++ b/zookeeper-contrib/pom.xml @@ -24,7 +24,7 @@ org.apache.zookeeper parent - 3.6.1 + 3.6.2-SNAPSHOT org.apache.zookeeper diff --git a/zookeeper-contrib/zookeeper-contrib-loggraph/pom.xml b/zookeeper-contrib/zookeeper-contrib-loggraph/pom.xml index 16b402361d3..4fc8e2ef1ce 100755 --- a/zookeeper-contrib/zookeeper-contrib-loggraph/pom.xml +++ b/zookeeper-contrib/zookeeper-contrib-loggraph/pom.xml @@ -23,7 +23,7 @@ org.apache.zookeeper zookeeper-contrib - 3.6.1 + 3.6.2-SNAPSHOT org.apache.zookeeper diff --git a/zookeeper-contrib/zookeeper-contrib-rest/pom.xml b/zookeeper-contrib/zookeeper-contrib-rest/pom.xml index 90dbfd7fcb9..daf1965744a 100755 --- a/zookeeper-contrib/zookeeper-contrib-rest/pom.xml +++ b/zookeeper-contrib/zookeeper-contrib-rest/pom.xml @@ -23,7 +23,7 @@ org.apache.zookeeper zookeeper-contrib - 3.6.1 + 3.6.2-SNAPSHOT org.apache.zookeeper diff --git a/zookeeper-contrib/zookeeper-contrib-zooinspector/pom.xml b/zookeeper-contrib/zookeeper-contrib-zooinspector/pom.xml index 850ad13a2ad..dc4f2fee79c 100755 --- a/zookeeper-contrib/zookeeper-contrib-zooinspector/pom.xml +++ b/zookeeper-contrib/zookeeper-contrib-zooinspector/pom.xml @@ -23,7 +23,7 @@ org.apache.zookeeper zookeeper-contrib - 3.6.1 + 3.6.2-SNAPSHOT org.apache.zookeeper diff --git a/zookeeper-docs/pom.xml b/zookeeper-docs/pom.xml index c7c4f4d56bb..1dc9713a7b5 100644 --- a/zookeeper-docs/pom.xml +++ b/zookeeper-docs/pom.xml @@ -23,7 +23,7 @@ org.apache.zookeeper parent - 3.6.1 + 3.6.2-SNAPSHOT .. diff --git a/zookeeper-jute/pom.xml b/zookeeper-jute/pom.xml index 9cfd21cc657..357c15bf653 100755 --- a/zookeeper-jute/pom.xml +++ b/zookeeper-jute/pom.xml @@ -23,7 +23,7 @@ org.apache.zookeeper parent - 3.6.1 + 3.6.2-SNAPSHOT .. diff --git a/zookeeper-metrics-providers/pom.xml b/zookeeper-metrics-providers/pom.xml index 9f6871b792d..5a2172023d1 100755 --- a/zookeeper-metrics-providers/pom.xml +++ b/zookeeper-metrics-providers/pom.xml @@ -23,7 +23,7 @@ org.apache.zookeeper parent - 3.6.1 + 3.6.2-SNAPSHOT .. diff --git a/zookeeper-metrics-providers/zookeeper-prometheus-metrics/pom.xml b/zookeeper-metrics-providers/zookeeper-prometheus-metrics/pom.xml index 4f2edc90ca4..699e62608f6 100755 --- a/zookeeper-metrics-providers/zookeeper-prometheus-metrics/pom.xml +++ b/zookeeper-metrics-providers/zookeeper-prometheus-metrics/pom.xml @@ -23,7 +23,7 @@ org.apache.zookeeper zookeeper-metrics-providers - 3.6.1 + 3.6.2-SNAPSHOT .. diff --git a/zookeeper-recipes/pom.xml b/zookeeper-recipes/pom.xml index c3ab410f727..79308bc0c71 100755 --- a/zookeeper-recipes/pom.xml +++ b/zookeeper-recipes/pom.xml @@ -24,7 +24,7 @@ org.apache.zookeeper parent - 3.6.1 + 3.6.2-SNAPSHOT org.apache.zookeeper diff --git a/zookeeper-recipes/zookeeper-recipes-election/pom.xml b/zookeeper-recipes/zookeeper-recipes-election/pom.xml index b238cccf435..40df3e86a1a 100755 --- a/zookeeper-recipes/zookeeper-recipes-election/pom.xml +++ b/zookeeper-recipes/zookeeper-recipes-election/pom.xml @@ -23,7 +23,7 @@ org.apache.zookeeper zookeeper-recipes - 3.6.1 + 3.6.2-SNAPSHOT org.apache.zookeeper diff --git a/zookeeper-recipes/zookeeper-recipes-lock/pom.xml b/zookeeper-recipes/zookeeper-recipes-lock/pom.xml index 97d88db65a7..20afb7cc6af 100755 --- a/zookeeper-recipes/zookeeper-recipes-lock/pom.xml +++ b/zookeeper-recipes/zookeeper-recipes-lock/pom.xml @@ -23,7 +23,7 @@ org.apache.zookeeper zookeeper-recipes - 3.6.1 + 3.6.2-SNAPSHOT org.apache.zookeeper diff --git a/zookeeper-recipes/zookeeper-recipes-queue/pom.xml b/zookeeper-recipes/zookeeper-recipes-queue/pom.xml index c2f938d959b..04323e96827 100755 --- a/zookeeper-recipes/zookeeper-recipes-queue/pom.xml +++ b/zookeeper-recipes/zookeeper-recipes-queue/pom.xml @@ -23,7 +23,7 @@ org.apache.zookeeper zookeeper-recipes - 3.6.1 + 3.6.2-SNAPSHOT org.apache.zookeeper diff --git a/zookeeper-server/pom.xml b/zookeeper-server/pom.xml index 390d40e0a88..f0699af1236 100755 --- a/zookeeper-server/pom.xml +++ b/zookeeper-server/pom.xml @@ -23,7 +23,7 @@ org.apache.zookeeper parent - 3.6.1 + 3.6.2-SNAPSHOT .. From 7cc5e6aed26af86d30a03715bfd0cad51453c0fc Mon Sep 17 00:00:00 2001 From: Enrico Olivelli Date: Fri, 17 Apr 2020 08:46:41 +0200 Subject: [PATCH 069/118] Fix pom.xml files that contained bad version 3.7.0-SNAPSHOT --- zookeeper-contrib/zookeeper-contrib-fatjar/pom.xml | 2 +- zookeeper-it/pom.xml | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/zookeeper-contrib/zookeeper-contrib-fatjar/pom.xml b/zookeeper-contrib/zookeeper-contrib-fatjar/pom.xml index 9a90dd7fc3c..b6dc5963c8c 100755 --- a/zookeeper-contrib/zookeeper-contrib-fatjar/pom.xml +++ b/zookeeper-contrib/zookeeper-contrib-fatjar/pom.xml @@ -23,7 +23,7 @@ org.apache.zookeeper zookeeper-contrib - 3.7.0-SNAPSHOT + 3.6.2-SNAPSHOT org.apache.zookeeper diff --git a/zookeeper-it/pom.xml b/zookeeper-it/pom.xml index 163930fa3c5..bc907f7ab50 100755 --- a/zookeeper-it/pom.xml +++ b/zookeeper-it/pom.xml @@ -23,7 +23,7 @@ org.apache.zookeeper parent - 3.7.0-SNAPSHOT + 3.6.2-SNAPSHOT zookeeper-it From c5b4c235fdd4abc6d4ebda2d40eb76d9452a417e Mon Sep 17 00:00:00 2001 From: Enrico Olivelli Date: Sat, 18 Apr 2020 10:03:14 -0700 Subject: [PATCH 070/118] ZOOKEEPER-3797: Conflict between fatjar and full-build Maven profiles Enable fatjar module in full-build profile. This is a fix only for branch-3.6, on master branch maven structure is changing, there is no need for this fix Author: Enrico Olivelli Reviewers: phunt@apache.org, breed@apache.org Closes #1323 from eolivelli/fix/ZOOKEEPER-3797 and squashes the following commits: 6dc3270fb [Enrico Olivelli] skip checkstyle 1cff2ff85 [Enrico Olivelli] skip spotbugs 19a869959 [Enrico Olivelli] simplify skipTests e48811917 [Enrico Olivelli] skip tests in zookeeper-it 3d6dc500b [Enrico Olivelli] enable zookeeper-it 940aec8da [Enrico Olivelli] ZOOKEEPER-3797 Conflict between fatjar and full-build Maven profiles in branch-3.6 Change-Id: Ifbfc6d1ac63a9eeab399a1fcd575b81185b36fc3 --- pom.xml | 1 + zookeeper-contrib/pom.xml | 6 ++++++ zookeeper-it/pom.xml | 11 +++++++++++ 3 files changed, 18 insertions(+) diff --git a/pom.xml b/pom.xml index a1ceeb08ba1..b97d385cbe4 100755 --- a/pom.xml +++ b/pom.xml @@ -258,6 +258,7 @@ full-build + zookeeper-it zookeeper-contrib diff --git a/zookeeper-contrib/pom.xml b/zookeeper-contrib/pom.xml index 73879217990..f4f83a002b1 100755 --- a/zookeeper-contrib/pom.xml +++ b/zookeeper-contrib/pom.xml @@ -48,6 +48,12 @@ zookeeper-contrib-fatjar + + full-build + + zookeeper-contrib-fatjar + + diff --git a/zookeeper-it/pom.xml b/zookeeper-it/pom.xml index bc907f7ab50..75f32b8ad13 100755 --- a/zookeeper-it/pom.xml +++ b/zookeeper-it/pom.xml @@ -33,6 +33,17 @@ ZooKeeper system tests + + + full-build + + true + true + true + + + + org.apache.zookeeper From b57d0e9efa04f9dcd6e519cef9df563acb7c5830 Mon Sep 17 00:00:00 2001 From: Enrico Olivelli Date: Sun, 19 Apr 2020 08:36:32 +0200 Subject: [PATCH 071/118] add ZOOKEEPER-3726 and ZOOKEEPER-3797 to release notes --- zookeeper-docs/src/main/resources/markdown/releasenotes.md | 2 ++ 1 file changed, 2 insertions(+) diff --git a/zookeeper-docs/src/main/resources/markdown/releasenotes.md b/zookeeper-docs/src/main/resources/markdown/releasenotes.md index 89a5d8ccc84..15aaaa08926 100644 --- a/zookeeper-docs/src/main/resources/markdown/releasenotes.md +++ b/zookeeper-docs/src/main/resources/markdown/releasenotes.md @@ -32,6 +32,8 @@ limitations under the License. * [ZOOKEEPER-3778](https://issues.apache.org/jira/browse/ZOOKEEPER-3778) - Cannot upgrade from 3.5.7 to 3.6.0 due to multiAddress.reachabilityCheckEnabled * [ZOOKEEPER-3780](https://issues.apache.org/jira/browse/ZOOKEEPER-3780) - restore Version.getRevision() to be backward compatible * [ZOOKEEPER-3793](https://issues.apache.org/jira/browse/ZOOKEEPER-3793) - Request throttling is broken when RequestThrottler is disabled or configured incorrectly. +* [ZOOKEEPER-3726](https://issues.apache.org/jira/browse/ZOOKEEPER-3726) - invalid ipv6 address comparison in C client +* [ZOOKEEPER-3797](https://issues.apache.org/jira/browse/ZOOKEEPER-3797) - Conflict between fatjar and full-build Maven profiles in branch-3.6 ## New Feature From 24001493d5d24a8c36e5fa38031243391191d62b Mon Sep 17 00:00:00 2001 From: Christopher Tubbs Date: Mon, 20 Apr 2020 11:35:50 +0200 Subject: [PATCH 072/118] ZOOKEEPER-3802: Fix rat checks in full-build and fatjar Ensure apache-rat:check passes with and without -Pfull-build and/or -Pfatjar Specifically, this change does: 1. Use wildcards for rat check excludes, so the build passes whether rat detects the fils at the build execution root or within the activated module, so the rat check works whether or not the full-build or fatjar profiles are activated. 2. Ensures properties set at zookeeper-contrib that control rat, spotbugs, and checkstyle, are properly passed down to the contrib submodules when fatjar is activated, by ensuring the intermediate parent pom for zookeeper-contrib is activated when any of its submodules are activated. 3. Also use version of spotbugs that works with newer JDKs so the full build, including rat checks, can be tested with newer JDKs. Author: Christopher Tubbs Reviewers: Enrico Olivelli , Patrick Hunt Closes #1326 from ctubbsii/fix-rat-build-36 --- pom.xml | 33 ++++++++++++++++----------------- zookeeper-contrib/pom.xml | 20 ++++++-------------- zookeeper-it/pom.xml | 15 +++++---------- 3 files changed, 27 insertions(+), 41 deletions(-) diff --git a/pom.xml b/pom.xml index b97d385cbe4..c03f2a8a824 100755 --- a/pom.xml +++ b/pom.xml @@ -266,7 +266,7 @@ fatjar zookeeper-it - zookeeper-contrib/zookeeper-contrib-fatjar + zookeeper-contrib @@ -365,7 +365,7 @@ 3.2.2 2.6 3.2.5 - 3.1.9 + 4.0.2 8.17 @@ -656,7 +656,7 @@ com.github.spotbugs spotbugs-maven-plugin - 3.1.9 + 4.0.0 excludeFindBugsFilter.xml @@ -876,23 +876,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 diff --git a/zookeeper-contrib/pom.xml b/zookeeper-contrib/pom.xml index f4f83a002b1..c741afafe2e 100755 --- a/zookeeper-contrib/pom.xml +++ b/zookeeper-contrib/pom.xml @@ -35,11 +35,9 @@ Contrib projects to Apache ZooKeeper - - zookeeper-contrib-loggraph - zookeeper-contrib-rest - zookeeper-contrib-zooinspector - + + true + @@ -52,6 +50,9 @@ full-build zookeeper-contrib-fatjar + zookeeper-contrib-loggraph + zookeeper-contrib-rest + zookeeper-contrib-zooinspector @@ -59,15 +60,6 @@ - - com.github.spotbugs - spotbugs-maven-plugin - 3.1.9 - - - true - - org.apache.maven.plugins maven-compiler-plugin diff --git a/zookeeper-it/pom.xml b/zookeeper-it/pom.xml index 75f32b8ad13..4719680671f 100755 --- a/zookeeper-it/pom.xml +++ b/zookeeper-it/pom.xml @@ -33,16 +33,11 @@ ZooKeeper system tests - - - full-build - - true - true - true - - - + + true + true + true + From e64a74fabafeb3b20109014149beb3ebd6a48be7 Mon Sep 17 00:00:00 2001 From: Enrico Olivelli Date: Tue, 21 Apr 2020 14:23:03 +0200 Subject: [PATCH 073/118] Add ZOOKEEPER-3802 to release notes --- zookeeper-docs/src/main/resources/markdown/releasenotes.md | 1 + 1 file changed, 1 insertion(+) diff --git a/zookeeper-docs/src/main/resources/markdown/releasenotes.md b/zookeeper-docs/src/main/resources/markdown/releasenotes.md index 15aaaa08926..b0d59895110 100644 --- a/zookeeper-docs/src/main/resources/markdown/releasenotes.md +++ b/zookeeper-docs/src/main/resources/markdown/releasenotes.md @@ -34,6 +34,7 @@ limitations under the License. * [ZOOKEEPER-3793](https://issues.apache.org/jira/browse/ZOOKEEPER-3793) - Request throttling is broken when RequestThrottler is disabled or configured incorrectly. * [ZOOKEEPER-3726](https://issues.apache.org/jira/browse/ZOOKEEPER-3726) - invalid ipv6 address comparison in C client * [ZOOKEEPER-3797](https://issues.apache.org/jira/browse/ZOOKEEPER-3797) - Conflict between fatjar and full-build Maven profiles in branch-3.6 +* [ZOOKEEPER-3802](https://issues.apache.org/jira/browse/ZOOKEEPER-3802) - Fix rat checks in full-build and fatjar ## New Feature From 21221ac692405962bdda3e9ceb63186ef0f180ef Mon Sep 17 00:00:00 2001 From: Damien Diederen Date: Mon, 27 Apr 2020 10:58:25 +0200 Subject: [PATCH 074/118] 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 --- .../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 adb0d297265d876a059ef64b10c53298e4522519 Mon Sep 17 00:00:00 2001 From: David Mollitor Date: Mon, 13 Jan 2020 14:09:29 +0100 Subject: [PATCH 075/118] ZOOKEEPER-3678: Remove Redundant GroupID from Maven POMs Author: David Mollitor Reviewers: Norbert Kalmar Closes #1206 from belugabehr/ZOOKEEPER-3678 (cherry picked from commit 1c5d135d74f16275876c024401dc2de92909b20a) Signed-off-by: Andor Molnar --- zookeeper-assembly/pom.xml | 1 - zookeeper-client/pom.xml | 3 --- zookeeper-client/zookeeper-client-c/pom.xml | 1 - zookeeper-contrib/pom.xml | 1 - zookeeper-contrib/zookeeper-contrib-loggraph/pom.xml | 1 - zookeeper-contrib/zookeeper-contrib-rest/pom.xml | 1 - zookeeper-contrib/zookeeper-contrib-zooinspector/pom.xml | 1 - zookeeper-docs/pom.xml | 1 - zookeeper-jute/pom.xml | 1 - zookeeper-metrics-providers/pom.xml | 1 - .../zookeeper-prometheus-metrics/pom.xml | 1 - zookeeper-recipes/pom.xml | 1 - zookeeper-recipes/zookeeper-recipes-election/pom.xml | 1 - zookeeper-recipes/zookeeper-recipes-lock/pom.xml | 1 - zookeeper-recipes/zookeeper-recipes-queue/pom.xml | 1 - zookeeper-server/pom.xml | 1 - 16 files changed, 18 deletions(-) diff --git a/zookeeper-assembly/pom.xml b/zookeeper-assembly/pom.xml index 4427aa8440a..442074a6f41 100755 --- a/zookeeper-assembly/pom.xml +++ b/zookeeper-assembly/pom.xml @@ -27,7 +27,6 @@ .. - org.apache.zookeeper zookeeper-assembly pom Apache ZooKeeper - Assembly diff --git a/zookeeper-client/pom.xml b/zookeeper-client/pom.xml index 3e1edd35c5a..708a552af88 100755 --- a/zookeeper-client/pom.xml +++ b/zookeeper-client/pom.xml @@ -27,14 +27,11 @@ .. - org.apache.zookeeper zookeeper-client pom Apache ZooKeeper - Client ZooKeeper client - - full-build diff --git a/zookeeper-client/zookeeper-client-c/pom.xml b/zookeeper-client/zookeeper-client-c/pom.xml index 4a56dbdca75..ae0e453a75a 100755 --- a/zookeeper-client/zookeeper-client-c/pom.xml +++ b/zookeeper-client/zookeeper-client-c/pom.xml @@ -27,7 +27,6 @@ .. - org.apache.zookeeper zookeeper-client-c jar Apache ZooKeeper - Client - C diff --git a/zookeeper-contrib/pom.xml b/zookeeper-contrib/pom.xml index c741afafe2e..8a06e41e5fd 100755 --- a/zookeeper-contrib/pom.xml +++ b/zookeeper-contrib/pom.xml @@ -27,7 +27,6 @@ 3.6.2-SNAPSHOT - org.apache.zookeeper zookeeper-contrib pom Apache ZooKeeper - Contrib diff --git a/zookeeper-contrib/zookeeper-contrib-loggraph/pom.xml b/zookeeper-contrib/zookeeper-contrib-loggraph/pom.xml index 4fc8e2ef1ce..31afb9b44da 100755 --- a/zookeeper-contrib/zookeeper-contrib-loggraph/pom.xml +++ b/zookeeper-contrib/zookeeper-contrib-loggraph/pom.xml @@ -26,7 +26,6 @@ 3.6.2-SNAPSHOT - org.apache.zookeeper zookeeper-contrib-loggraph jar Apache ZooKeeper - Contrib - Loggraph diff --git a/zookeeper-contrib/zookeeper-contrib-rest/pom.xml b/zookeeper-contrib/zookeeper-contrib-rest/pom.xml index daf1965744a..29c6fade9ca 100755 --- a/zookeeper-contrib/zookeeper-contrib-rest/pom.xml +++ b/zookeeper-contrib/zookeeper-contrib-rest/pom.xml @@ -26,7 +26,6 @@ 3.6.2-SNAPSHOT - org.apache.zookeeper zookeeper-contrib-rest jar Apache ZooKeeper - Contrib - Rest diff --git a/zookeeper-contrib/zookeeper-contrib-zooinspector/pom.xml b/zookeeper-contrib/zookeeper-contrib-zooinspector/pom.xml index dc4f2fee79c..df572846a00 100755 --- a/zookeeper-contrib/zookeeper-contrib-zooinspector/pom.xml +++ b/zookeeper-contrib/zookeeper-contrib-zooinspector/pom.xml @@ -26,7 +26,6 @@ 3.6.2-SNAPSHOT - org.apache.zookeeper zookeeper-contrib-zooinspector jar Apache ZooKeeper - Contrib - ZooInspector diff --git a/zookeeper-docs/pom.xml b/zookeeper-docs/pom.xml index 1dc9713a7b5..3512158f4db 100644 --- a/zookeeper-docs/pom.xml +++ b/zookeeper-docs/pom.xml @@ -27,7 +27,6 @@ .. - org.apache.zookeeper zookeeper-docs Apache ZooKeeper - Documentation Documentation diff --git a/zookeeper-jute/pom.xml b/zookeeper-jute/pom.xml index 357c15bf653..abd9807be9b 100755 --- a/zookeeper-jute/pom.xml +++ b/zookeeper-jute/pom.xml @@ -27,7 +27,6 @@ .. - org.apache.zookeeper zookeeper-jute jar Apache ZooKeeper - Jute diff --git a/zookeeper-metrics-providers/pom.xml b/zookeeper-metrics-providers/pom.xml index 5a2172023d1..586e93ed85f 100755 --- a/zookeeper-metrics-providers/pom.xml +++ b/zookeeper-metrics-providers/pom.xml @@ -27,7 +27,6 @@ .. - org.apache.zookeeper zookeeper-metrics-providers pom Apache ZooKeeper - Metrics Providers diff --git a/zookeeper-metrics-providers/zookeeper-prometheus-metrics/pom.xml b/zookeeper-metrics-providers/zookeeper-prometheus-metrics/pom.xml index 699e62608f6..821a3e00682 100755 --- a/zookeeper-metrics-providers/zookeeper-prometheus-metrics/pom.xml +++ b/zookeeper-metrics-providers/zookeeper-prometheus-metrics/pom.xml @@ -27,7 +27,6 @@ .. - org.apache.zookeeper zookeeper-prometheus-metrics jar Apache ZooKeeper - Prometheus.io Metrics Provider diff --git a/zookeeper-recipes/pom.xml b/zookeeper-recipes/pom.xml index 79308bc0c71..c09352437b8 100755 --- a/zookeeper-recipes/pom.xml +++ b/zookeeper-recipes/pom.xml @@ -27,7 +27,6 @@ 3.6.2-SNAPSHOT - org.apache.zookeeper zookeeper-recipes pom Apache ZooKeeper - Recipes diff --git a/zookeeper-recipes/zookeeper-recipes-election/pom.xml b/zookeeper-recipes/zookeeper-recipes-election/pom.xml index 40df3e86a1a..09e4de24079 100755 --- a/zookeeper-recipes/zookeeper-recipes-election/pom.xml +++ b/zookeeper-recipes/zookeeper-recipes-election/pom.xml @@ -26,7 +26,6 @@ 3.6.2-SNAPSHOT - org.apache.zookeeper zookeeper-recipes-election jar Apache ZooKeeper - Recipes - Election diff --git a/zookeeper-recipes/zookeeper-recipes-lock/pom.xml b/zookeeper-recipes/zookeeper-recipes-lock/pom.xml index 20afb7cc6af..b8ba8c3e464 100755 --- a/zookeeper-recipes/zookeeper-recipes-lock/pom.xml +++ b/zookeeper-recipes/zookeeper-recipes-lock/pom.xml @@ -26,7 +26,6 @@ 3.6.2-SNAPSHOT - org.apache.zookeeper zookeeper-recipes-lock jar Apache ZooKeeper - Recipes - Lock diff --git a/zookeeper-recipes/zookeeper-recipes-queue/pom.xml b/zookeeper-recipes/zookeeper-recipes-queue/pom.xml index 04323e96827..4373c678c84 100755 --- a/zookeeper-recipes/zookeeper-recipes-queue/pom.xml +++ b/zookeeper-recipes/zookeeper-recipes-queue/pom.xml @@ -26,7 +26,6 @@ 3.6.2-SNAPSHOT - org.apache.zookeeper zookeeper-recipes-queue jar Apache ZooKeeper - Recipes - Queue diff --git a/zookeeper-server/pom.xml b/zookeeper-server/pom.xml index f0699af1236..8a4c4f06311 100755 --- a/zookeeper-server/pom.xml +++ b/zookeeper-server/pom.xml @@ -27,7 +27,6 @@ .. - org.apache.zookeeper zookeeper jar Apache ZooKeeper - Server From 8bd0d5e06be3d508c64c34bf2aa4423bcd8d666f Mon Sep 17 00:00:00 2001 From: David Mollitor Date: Wed, 8 Jan 2020 10:43:49 +0100 Subject: [PATCH 076/118] ZOOKEEPER-3679: Upgrade maven-compiler-plugin For ZooKeeper-jute Author: David Mollitor Reviewers: Enrico Olivelli Closes #1207 from belugabehr/ZOOKEEPER-3679 --- zookeeper-jute/pom.xml | 1 - 1 file changed, 1 deletion(-) diff --git a/zookeeper-jute/pom.xml b/zookeeper-jute/pom.xml index abd9807be9b..f3930d7ac40 100755 --- a/zookeeper-jute/pom.xml +++ b/zookeeper-jute/pom.xml @@ -71,7 +71,6 @@ maven-compiler-plugin - 2.3.2 pre-compile-jute From 5b548ff6a4a10d4ecb66ead55336ee3c76bc76fd Mon Sep 17 00:00:00 2001 From: Patrick Hunt Date: Mon, 4 May 2020 06:49:35 +0000 Subject: [PATCH 077/118] 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 (cherry picked from commit ac4da3ebed34c536c4ea873503b0a6c036fa3910) Signed-off-by: Mate Szalay-Beko --- ....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 6f23dfbbe3684092cb645d52b718c8f99902cff2 Mon Sep 17 00:00:00 2001 From: Mate Szalay-Beko Date: Mon, 4 May 2020 14:13:41 +0200 Subject: [PATCH 078/118] ZOOKEEPER-3817: suppress log4j SmtpAppender related CVE-2020-9488 Author: Mate Szalay-Beko Reviewers: Enrico Olivelli Closes #1346 from symat/ZOOKEEPER-3817 (cherry picked from commit f6b54a6cd227ac37f28803f45d7287c7fd3a8142) Signed-off-by: Enrico Olivelli --- 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 aa930d8757d2e300fe86a6e17e3eddbe9b276e91 Mon Sep 17 00:00:00 2001 From: Andor Molnar Date: Mon, 4 May 2020 14:17:33 +0200 Subject: [PATCH 079/118] 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 (cherry picked from commit 391cb4aa6b54e19a028215e1340232a114c23ed3) Signed-off-by: Enrico Olivelli --- .../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 02816bbd6b4229915954a2cfdaccc4295adca2bc Mon Sep 17 00:00:00 2001 From: lingjinjiang Date: Tue, 5 May 2020 09:18:18 +0200 Subject: [PATCH 080/118] 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 (cherry picked from commit 44de549657a539dcedba58d56e67b8c906c89580) Signed-off-by: Norbert Kalmar --- .../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 cbfb83913bd6911f80821f680accbcfb47cdb90c Mon Sep 17 00:00:00 2001 From: asutosh936 Date: Tue, 12 May 2020 09:39:58 +0000 Subject: [PATCH 081/118] ZOOKEEPER-3761: Upgrade JLine version to 2.14.6 Author: asutosh936 Reviewers: Enrico Olivelli , Mate Szalay-Beko Closes #1292 from asutosh936/ZOOKEEPER-3761 (cherry picked from commit c56506162bfde339e801743396361862cc8c5b41) Signed-off-by: Mate Szalay-Beko --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index c03f2a8a824..023ee8d5579 100755 --- a/pom.xml +++ b/pom.xml @@ -358,7 +358,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 6cf8b02120731922b64a35ca997786314a3584ec Mon Sep 17 00:00:00 2001 From: Mate Szalay-Beko Date: Tue, 12 May 2020 09:53:40 +0000 Subject: [PATCH 082/118] 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 (cherry picked from commit 236e3d9183606512f0e03a1f828ad0d392eb6091) Signed-off-by: Mate Szalay-Beko --- 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 ec3db140fc0..f3a8ba0d09e 100755 --- a/bin/zkServer.sh +++ b/bin/zkServer.sh @@ -229,51 +229,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 1c6e130e6a0..bc16647040f 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 8d7400653d7..2b2a76cf1fd 100644 --- a/zookeeper-docs/src/main/resources/markdown/zookeeperTools.md +++ b/zookeeper-docs/src/main/resources/markdown/zookeeperTools.md @@ -68,6 +68,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 16ddfea7134dffab0dd4220c9ee89c7a0407cf6a Mon Sep 17 00:00:00 2001 From: David Mollitor Date: Wed, 20 May 2020 19:56:26 +0200 Subject: [PATCH 083/118] ZOOKEEPER-3834: Do Not Set Explicit Test Includes in POM Author: David Mollitor Reviewers: Enrico Olivelli , Norbert Kalmar Closes #1355 from belugabehr/ZOOKEEPER-3834 (cherry picked from commit 5e88b541dcc2fd30e60bb437efa7704bbec8f6e6) Signed-off-by: Norbert Kalmar --- 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 09e4de24079..fa4ae9313cf 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 b8ba8c3e464..7766de7fa00 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 4373c678c84..0c68f799a9c 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 977881f61f3fb308eb2a2cc28e699f50fa7a8baa Mon Sep 17 00:00:00 2001 From: Norbert Kalmar Date: Thu, 21 May 2020 11:19:39 +0200 Subject: [PATCH 084/118] ZOOKEEPER-3792: fix apidocs path Author: Norbert Kalmar Reviewers: Mate Szalay-Beko Closes #1362 from nkalmar/master (cherry picked from commit 9e50853da2da5aa982a4b0da790f2da51575b6f1) Signed-off-by: Norbert Kalmar --- 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/skin/init.js | 2 +- .../src/main/resources/markdown/zookeeperProgrammers.md | 2 +- 4 files changed, 4 insertions(+), 4 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 @@